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 dh...@apache.org on 2007/12/15 01:12:34 UTC

svn commit: r604340 - in /lucene/hadoop/trunk: ./ src/java/org/apache/hadoop/dfs/ src/java/org/apache/hadoop/fs/ src/java/org/apache/hadoop/fs/permission/ src/test/org/apache/hadoop/fs/

Author: dhruba
Date: Fri Dec 14 16:12:32 2007
New Revision: 604340

URL: http://svn.apache.org/viewvc?rev=604340&view=rev
Log:
HADOOP-2381.  Support permission information in FileStatus. Client
Protocol version changed from 21 to 22.  (Raghu Angadi via dhruba)


Modified:
    lucene/hadoop/trunk/CHANGES.txt
    lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/ClientProtocol.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileStatus.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileSystem.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FsShell.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/RawLocalFileSystem.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/permission/FsPermission.java
    lucene/hadoop/trunk/src/test/org/apache/hadoop/fs/TestLocalFileSystemPermission.java

Modified: lucene/hadoop/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/CHANGES.txt?rev=604340&r1=604339&r2=604340&view=diff
==============================================================================
--- lucene/hadoop/trunk/CHANGES.txt (original)
+++ lucene/hadoop/trunk/CHANGES.txt Fri Dec 14 16:12:32 2007
@@ -33,6 +33,9 @@
     HADOOP-2401.  Only the current leaseholder can abandon a block for
     a HDFS file.  ClientProtocol version changed from 20 to 21.
     (Tsz Wo (Nicholas), SZE via dhruba)
+
+    HADOOP-2381.  Support permission information in FileStatus. Client
+    Protocol version changed from 21 to 22.  (Raghu Angadi via dhruba)
     
   NEW FEATURES
 

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/ClientProtocol.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/ClientProtocol.java?rev=604340&r1=604339&r2=604340&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/ClientProtocol.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/ClientProtocol.java Fri Dec 14 16:12:32 2007
@@ -37,8 +37,9 @@
    * 19 : rollEditLog() returns a token to uniquely identify the editfile.
    * 20 : getContentLength returns the total size in bytes of a directory subtree
    * 21 : add lease holder as a parameter in abandonBlock(...)
+   * 22 : Serialization of FileStatus has changed.
    */
-  public static final long versionID = 21L;
+  public static final long versionID = 22L;
   
   ///////////////////////////////////////
   // File contents

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileStatus.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileStatus.java?rev=604340&r1=604339&r2=604340&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileStatus.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileStatus.java Fri Dec 14 16:12:32 2007
@@ -21,6 +21,7 @@
 import java.io.DataOutput;
 import java.io.IOException;
 
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 
@@ -34,22 +35,33 @@
   private short block_replication;
   private long blocksize;
   private long modification_time;
-
-  public FileStatus() { this(0, false, 0, 0, 0, null); }
-
+  private FsPermission permission;
+  private String owner;
+  private String group;
+  
+  public FileStatus() { this(0, false, 0, 0, 0, null, null, null, null); }
+  
+  //We should deprecate this soon?
   public FileStatus(long length, boolean isdir, int block_replication,
-             long blocksize, long modification_time, Path path) {
-    this(length, isdir, (short)block_replication, blocksize,
-         modification_time, path);
-  }
+                    long blocksize, long modification_time, Path path) {
 
-  public FileStatus(long length, boolean isdir, short block_replication,
-             long blocksize, long modification_time, Path path) {
+    this(length, isdir, block_replication, blocksize, modification_time,
+         null, null, null, path);
+  }
+  
+  public FileStatus(long length, boolean isdir, int block_replication,
+                    long blocksize, long modification_time,
+                    FsPermission permission, String owner, String group, 
+                    Path path) {
     this.length = length;
     this.isdir = isdir;
-    this.block_replication = block_replication;
+    this.block_replication = (short)block_replication;
     this.blocksize = blocksize;
     this.modification_time = modification_time;
+    this.permission = (permission == null) ? 
+                      FsPermission.getDefault() : permission;
+    this.owner = (owner == null) ? "" : owner;
+    this.group = (group == null) ? "" : group;
     this.path = path;
   }
 
@@ -92,10 +104,69 @@
     return modification_time;
   }
 
+  /**
+   * Get FsPermission associated with the file.
+   * @return permssion. If a filesystem does not have a notion of permissions
+   *         or if permissions could not be determined, then default 
+   *         permissions equivalent of "rwxrwxrwx" is returned.
+   */
+  public FsPermission getPermission() {
+    return permission;
+  }
+  
+  /**
+   * Get the owner of the file.
+   * @return owner of the file. The string could be empty if there is no
+   *         notion of owner of a file in a filesystem or if it could not 
+   *         be determined (rare).
+   */
+  public String getOwner() {
+    return owner;
+  }
+  
+  /**
+   * Get the group associated with the file.
+   * @return group for the file. The string could be empty if there is no
+   *         notion of group of a file in a filesystem or if it could not 
+   *         be determined (rare).
+   */
+  public String getGroup() {
+    return group;
+  }
+  
   public Path getPath() {
     return path;
   }
 
+  /* These are provided so that these values could be loaded lazily 
+   * by a filesystem (e.g. local file system).
+   */
+  
+  /**
+   * Sets permission.
+   * @param permission if permission is null, default value is set
+   */
+  protected void setPermission(FsPermission permission) {
+    this.permission = (permission == null) ? 
+                      FsPermission.getDefault() : permission;
+  }
+  
+  /**
+   * Sets owner.
+   * @param owner if it is null, default value is set
+   */  
+  protected void setOwner(String owner) {
+    this.owner = (owner == null) ? "" : owner;
+  }
+  
+  /**
+   * Sets group.
+   * @param group if it is null, default value is set
+   */  
+  protected void setGroup(String group) {
+    this.group = (group == null) ? "" :  group;
+  }
+  
   //////////////////////////////////////////////////
   // Writable
   //////////////////////////////////////////////////
@@ -106,6 +177,9 @@
     out.writeShort(block_replication);
     out.writeLong(blocksize);
     out.writeLong(modification_time);
+    permission.write(out);
+    Text.writeString(out, owner);
+    Text.writeString(out, group);
   }
 
   public void readFields(DataInput in) throws IOException {
@@ -116,6 +190,9 @@
     this.block_replication = in.readShort();
     blocksize = in.readLong();
     modification_time = in.readLong();
+    permission.readFields(in);
+    owner = Text.readString(in);
+    group = Text.readString(in);
   }
 
 }

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileSystem.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileSystem.java?rev=604340&r1=604339&r2=604340&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileSystem.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileSystem.java Fri Dec 14 16:12:32 2007
@@ -387,7 +387,7 @@
                                             long blockSize,
                                             Progressable progress
                                             ) throws IOException {
-    return this.create(f, FsPermission.getDefault(getConf()),
+    return this.create(f, FsPermission.getDefault(),
         overwrite, bufferSize, replication, blockSize, progress);
   }
 
@@ -836,7 +836,7 @@
    * Call {@link #mkdirs(Path, FsPermission)} with default permission.
    */
   public boolean mkdirs(Path f) throws IOException {
-    return mkdirs(f, FsPermission.getDefault(getConf()));
+    return mkdirs(f, FsPermission.getDefault());
   }
 
   /**

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FsShell.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FsShell.java?rev=604340&r1=604339&r2=604340&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FsShell.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FsShell.java Fri Dec 14 16:12:32 2007
@@ -551,9 +551,11 @@
     }
   }
 
-  /* list all files under the directory <i>src</i>*/
+  /* list all files under the directory <i>src</i>
+   * ideally we should provide "-l" option, that lists like "ls -l".
+   */
   private void ls(Path src, boolean recursive, boolean printHeader) throws IOException {
-    Path items[] = fs.listPaths(src);
+    FileStatus items[] = fs.listStatus(src);
     if (items == null) {
       throw new IOException("Could not get listing for " + src);
     } else {
@@ -561,15 +563,18 @@
         System.out.println("Found " + items.length + " items");
       }
       for (int i = 0; i < items.length; i++) {
-        Path cur = items[i];
-        FileStatus stat = fs.getFileStatus(cur);
+        FileStatus stat = items[i];
+        Path cur = stat.getPath();
         String mdate = dateForm.format(new Date(stat.getModificationTime()));
         System.out.println(cur.toUri().getPath() + "\t" 
                            + (stat.isDir() ? 
                               "<dir>\t" : 
                               ("<r " + stat.getReplication() 
                                + ">\t" + stat.getLen()))
-                           + "\t" + mdate);
+                           + "\t" + mdate 
+                           + "\t" + stat.getPermission()
+                           + "\t" + stat.getOwner() 
+                           + "\t" + stat.getGroup());
         if (recursive && stat.isDir()) {
           ls(cur, recursive, printHeader);
         }

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/RawLocalFileSystem.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/RawLocalFileSystem.java?rev=604340&r1=604339&r2=604340&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/RawLocalFileSystem.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/RawLocalFileSystem.java Fri Dec 14 16:12:32 2007
@@ -357,36 +357,78 @@
   }
 
   static class RawLocalFileStatus extends FileStatus {
-    private File file;
-    private PermissionStatus permissions;
-
+    /* We can add extra fields here. It breaks at least CopyFiles.FilePair().
+     * We recognize if the information is already loaded by check if
+     * onwer.equals("").
+     */
+    private boolean isPermissionLoaded() {
+      return !super.getOwner().equals(""); 
+    }
+    
     RawLocalFileStatus(File f, long defaultBlockSize) {
       super(f.length(), f.isDirectory(), 1, defaultBlockSize,
             f.lastModified(), new Path(f.toURI().toString()));
-      file = f;
+    }
+    
+    @Override
+    public FsPermission getPermission() {
+      if (!isPermissionLoaded()) {
+        loadPermissionInfo();
+      }
+      return super.getPermission();
     }
 
-    PermissionStatus getPermissionStatus() {
-      if (permissions == null) {
-        try {
-          permissions = getPermissionStatus(file);
-        }
-        catch(IOException e) {
-          LOG.debug(StringUtils.stringifyException(e));
+    @Override
+    public String getOwner() {
+      if (!isPermissionLoaded()) {
+        loadPermissionInfo();
+      }
+      return super.getOwner();
+    }
+
+    @Override
+    public String getGroup() {
+      if (isPermissionLoaded()) {
+        loadPermissionInfo();
+      }
+      return super.getGroup();
+    }
+
+    /// loads permissions, owner, and group from `ls -ld`
+    private void loadPermissionInfo() {
+      try {
+        StringTokenizer t = new StringTokenizer(
+            execCommand(new File(getPath().toUri()), 
+                        ShellCommand.getGET_PERMISSION_COMMAND()));
+        //expected format
+        //-rw-------    1 username groupname ...
+        setPermission(FsPermission.valueOf(t.nextToken()));
+        t.nextToken();
+        setOwner(t.nextToken());
+        setGroup(t.nextToken());
+      } catch (IOException e) {
+        if (e.getMessage().contains("No such file or directory")) {                                    
+          /* XXX This is a temporary hack till HADOOP-2344 goes in.
+           * will fix it soon.
+           */
+          setPermission(null);
+          setOwner(null);
+          setGroup(null);
+        } else {
+          //this is not expected
+          throw new RuntimeException("Error while running command to get " +
+                                     "file permissions : " + 
+                                     StringUtils.stringifyException(e));
         }
       }
-      return permissions;
     }
 
-    private static PermissionStatus getPermissionStatus(File f
-        ) throws IOException {
-      StringTokenizer t = new StringTokenizer(
-          execCommand(f, ShellCommand.getGET_PERMISSION_COMMAND()));
-      //expected format
-      //-rw-------    1 username groupname ...
-      FsPermission p = FsPermission.valueOf(t.nextToken());
-      t.nextToken();
-      return new PermissionStatus(t.nextToken(), t.nextToken(), p);
+    @Override
+    public void write(DataOutput out) throws IOException {
+      if (!isPermissionLoaded()) {
+        loadPermissionInfo();
+      }
+      super.write(out);
     }
   }
 
@@ -420,9 +462,7 @@
     String[] args = new String[cmd.length + 1];
     System.arraycopy(cmd, 0, args, 0, cmd.length);
     args[cmd.length] = f.getCanonicalPath();
-    LOG.debug("args=" + Arrays.asList(args));
     String output = ShellCommand.execCommand(args);
-    LOG.debug("output=" + output);
     return output;
   }
 }

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/permission/FsPermission.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/permission/FsPermission.java?rev=604340&r1=604339&r2=604340&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/permission/FsPermission.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/permission/FsPermission.java Fri Dec 14 16:12:32 2007
@@ -146,14 +146,9 @@
     conf.setInt(UMASK_LABEL, umask.toShort());
   }
 
-  private static final FsPermission NONE = new FsPermission((short)0);
-
-  /**
-   * Get the default permission from conf.
-   * @param conf
-   */
-  public static FsPermission getDefault(Configuration conf) {
-    return NONE.applyUMask(getUMask(conf));
+  /** Get the default permission. */
+  public static FsPermission getDefault() {
+    return new FsPermission((short)0777);
   }
 
   /**
@@ -169,7 +164,7 @@
     for(int i = 1; i < unixSymbolicPermission.length(); i++) {
       n = n << 1;
       char c = unixSymbolicPermission.charAt(i);
-      n += c == '-' || c == 'T'? 0: 1;
+      n += (c == '-' || c == 'T' || c == 'S') ? 0: 1;
     }
     return new FsPermission((short)n);
   }

Modified: lucene/hadoop/trunk/src/test/org/apache/hadoop/fs/TestLocalFileSystemPermission.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/test/org/apache/hadoop/fs/TestLocalFileSystemPermission.java?rev=604340&r1=604339&r2=604340&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/test/org/apache/hadoop/fs/TestLocalFileSystemPermission.java (original)
+++ lucene/hadoop/trunk/src/test/org/apache/hadoop/fs/TestLocalFileSystemPermission.java Fri Dec 14 16:12:32 2007
@@ -93,9 +93,7 @@
   }
 
   FsPermission getPermission(LocalFileSystem fs, Path p) throws IOException {
-    RawLocalFileSystem.RawLocalFileStatus s
-      = (RawLocalFileSystem.RawLocalFileStatus)fs.getFileStatus(p);
-    return s.getPermissionStatus().getPermission();
+    return fs.getFileStatus(p).getPermission();
   }
 
   /** Test LocalFileSystem.setOwner */
@@ -148,8 +146,6 @@
   }
 
   String getGroup(LocalFileSystem fs, Path p) throws IOException {
-    RawLocalFileSystem.RawLocalFileStatus s
-      = (RawLocalFileSystem.RawLocalFileStatus)fs.getFileStatus(p);
-    return s.getPermissionStatus().getGroupName();
+    return fs.getFileStatus(p).getGroup();
   }
 }