Browse Source

HADOOP-2336. Shell commands to modify file permissions.
(Raghu Angadi via rangadi)


git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk@608651 13f79535-47bb-0310-9956-ffa450edef68

Raghu Angadi 17 years ago
parent
commit
cd1a6b93f6

+ 3 - 0
CHANGES.txt

@@ -147,6 +147,9 @@ Trunk (unreleased changes)
     HADOOP-1719. Improves the utilization of shuffle copier threads.
     HADOOP-1719. Improves the utilization of shuffle copier threads.
     (Amar Kamat via ddas)
     (Amar Kamat via ddas)
 
 
+    HADOOP-2336. Shell commands to modify file permissions.
+    (Raghu Angadi via rangadi)
+ 
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     HADOOP-1898.  Release the lock protecting the last time of the last stack
     HADOOP-1898.  Release the lock protecting the last time of the last stack

+ 1 - 0
src/java/org/apache/hadoop/dfs/DistributedFileSystem.java

@@ -169,6 +169,7 @@ public class DistributedFileSystem extends FileSystem {
       DFSFileInfo f = (DFSFileInfo)infos[i];
       DFSFileInfo f = (DFSFileInfo)infos[i];
       stats[i] = new FileStatus(f.getLen(), f.isDir(), f.getReplication(),
       stats[i] = new FileStatus(f.getLen(), f.isDir(), f.getReplication(),
                                 f.getBlockSize(), f.getModificationTime(),
                                 f.getBlockSize(), f.getModificationTime(),
+                                f.getPermission(), f.getOwner(), f.getGroup(),
                                 new DfsPath(f, this)); // fully-qualify path
                                 new DfsPath(f, this)); // fully-qualify path
     }
     }
     return stats;
     return stats;

+ 8 - 4
src/java/org/apache/hadoop/dfs/HftpFileSystem.java

@@ -143,11 +143,15 @@ public class HftpFileSystem extends FileSystem {
               Long.valueOf(attrs.getValue("size")).longValue(), false,
               Long.valueOf(attrs.getValue("size")).longValue(), false,
               Short.valueOf(attrs.getValue("replication")).shortValue(),
               Short.valueOf(attrs.getValue("replication")).shortValue(),
               Long.valueOf(attrs.getValue("blocksize")).longValue(),
               Long.valueOf(attrs.getValue("blocksize")).longValue(),
-              modif, new Path("hftp", fshostname + ":" + fsport,
-              attrs.getValue("path")))
+              modif, FsPermission.valueOf(attrs.getValue("permission")),
+              attrs.getValue("owner"), attrs.getValue("group"),
+              new Path("hftp", fshostname + ":" + fsport, 
+                       attrs.getValue("path")))
         : new FileStatus(0L, true, 0, 0L,
         : new FileStatus(0L, true, 0, 0L,
-              modif, new Path("hftp", fshostname + ":" + fsport,
-              attrs.getValue("path")));
+              modif, FsPermission.valueOf(attrs.getValue("permission")),
+              attrs.getValue("owner"), attrs.getValue("group"),
+              new Path("hftp", fshostname + ":" + fsport, 
+                       attrs.getValue("path")));
       fslist.add(fs);
       fslist.add(fs);
     }
     }
 
 

+ 3 - 2
src/java/org/apache/hadoop/dfs/ListPathsServlet.java

@@ -23,7 +23,6 @@ import org.znerd.xmlenc.*;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.io.PrintWriter;
-import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URISyntaxException;
 import java.text.SimpleDateFormat;
 import java.text.SimpleDateFormat;
 import java.util.Date;
 import java.util.Date;
@@ -31,7 +30,6 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.Map;
 import java.util.Stack;
 import java.util.Stack;
 import java.util.TimeZone;
 import java.util.TimeZone;
-import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 import java.util.regex.Pattern;
 import java.util.regex.PatternSyntaxException;
 import java.util.regex.PatternSyntaxException;
 import javax.servlet.ServletException;
 import javax.servlet.ServletException;
@@ -66,6 +64,9 @@ public class ListPathsServlet extends HttpServlet {
       doc.attribute("replication", String.valueOf(i.getReplication()));
       doc.attribute("replication", String.valueOf(i.getReplication()));
       doc.attribute("blocksize", String.valueOf(i.getBlockSize()));
       doc.attribute("blocksize", String.valueOf(i.getBlockSize()));
     }
     }
+    doc.attribute("permission", i.getPermission().toString());
+    doc.attribute("owner", i.getOwner());
+    doc.attribute("group", i.getGroup());
     doc.endTag();
     doc.endTag();
   }
   }
 
 

+ 130 - 0
src/java/org/apache/hadoop/fs/FsShell.java

@@ -1009,6 +1009,98 @@ public class FsShell extends Configured implements Tool {
     }
     }
   }
   }
 
 
+  /**
+   * This class runs a command on a given FileStatus. This can be used for
+   * running various commands like chmod, chown etc.
+   */
+  static abstract class CmdHandler {
+    
+    protected int errorCode = 0;
+    protected boolean okToContinue = true;
+    protected FileSystem fs;
+    protected String cmdName;
+    
+    int getErrorCode() { return errorCode; }
+    boolean okToContinue() { return okToContinue; }
+    FileSystem getFS() { return fs; }
+    String getName() { return cmdName; }
+    
+    protected CmdHandler(String cmdName, FileSystem fs) {
+      this.fs = fs;
+      this.cmdName = cmdName;
+    }
+    
+    public abstract void run(FileStatus file) throws IOException;
+  }
+  
+  ///helper for runCmdHandler*() returns listPaths()
+  private static FileStatus[] cmdHandlerListStatus(CmdHandler handler, 
+                                                   Path path) {
+    try {
+      FileStatus[] files = handler.getFS().listStatus(path);
+      if ( files == null ) {
+        System.err.println(handler.getName() + 
+                           ": could not get listing for '" + path + "'");
+      }
+      return files;
+    } catch (IOException e) {
+      System.err.println(handler.getName() + 
+                         ": could not get get listing for '" + path + "' : " +
+                         e.getMessage().split("\n")[0]);
+    }
+    return null;
+  }
+  
+  
+  /**
+   * Runs the command on a given file with the command handler. 
+   * If recursive is set, command is run recursively.
+   */                                       
+  private static int runCmdHandler(CmdHandler handler, FileStatus stat, 
+                                   boolean recursive) throws IOException {
+    int errors = 0;
+    handler.run(stat);
+    if (recursive && stat.isDir() && handler.okToContinue()) {
+      FileStatus[] files = cmdHandlerListStatus(handler, stat.getPath());
+      if (files == null) {
+        return 1;
+      }
+      for(FileStatus file : files ) {
+        errors += runCmdHandler(handler, file, recursive);
+      }
+    }
+    return errors;
+  }
+
+  ///top level runCmdHandler
+  static int runCmdHandler(CmdHandler handler, String[] args,
+                                   int startIndex, boolean recursive) 
+                                   throws IOException {
+    int errors = 0;
+    
+    for (int i=startIndex; i<args.length; i++) {
+      Path[] paths = handler.getFS().globPaths(new Path(args[i]));
+      for(Path path : paths) {
+        try {
+          FileStatus file = handler.getFS().getFileStatus(path);
+          if (file == null) {
+            System.err.println(handler.getName() + 
+                               ": could not get status for '" + path + "'");
+            errors++;
+          } else {
+            errors += runCmdHandler(handler, file, recursive);
+          }
+        } catch (IOException e) {
+          System.err.println(handler.getName() + 
+                             ": could not get status for '" + path + "': " +
+                             e.getMessage().split("\n")[0]);        
+        }
+      }
+    }
+    
+    return (errors > 0 || handler.getErrorCode() != 0) ? 1 : 0;
+  }
+  
   /**
   /**
    * Return an abbreviated English-language desc of the byte length
    * Return an abbreviated English-language desc of the byte length
    */
    */
@@ -1158,6 +1250,31 @@ public class FsShell extends Configured implements Tool {
       + ":  Show the last 1KB of the file. \n"
       + ":  Show the last 1KB of the file. \n"
       + "\t\tThe -f option shows apended data as the file grows. \n";
       + "\t\tThe -f option shows apended data as the file grows. \n";
 
 
+    String chmod = FsShellPermissions.CHMOD_USAGE + "\n" +
+      "\t\tChanges permissions of a file.\n" +
+      "\t\tThis works similar to shell's chmod with a few exceptions.\n\n" +
+      "\t-R\tmodifies the files recursively. This is the only option\n" +
+      "\t\tcurrently supported.\n\n" +
+      "\tMODE\tMode is same as mode used for chmod shell command.\n" +
+      "\t\tOnly letters recognized are 'rwxX'. E.g.: a+r,g-w,+rwx,o=r\n\n" +
+      "\tOCTALMODE Mode specifed in 3 digits. Unlike shell command,\n" +
+      "\t\tthis requires all three digits.\n" +
+      "\t\tE.g.: 754 is same as u=rwx,g=rw,o=r\n\n" +
+      "\t\tIf none of 'augo' is specified, 'a' is assumed and unlike\n" +
+      "\t\tshell command, no umask is applied\n";
+    
+    String chown = FsShellPermissions.CHOWN_USAGE + "\n" +
+      "\t\tChanges owner and group of a file.\n" +
+      "\t\tThis is similar to shell's chown with a few exceptions.\n\n" +
+      "\t-R\tmodifies the files recursively. This is the only option\n" +
+      "\t\tcurrently supported.\n\n" +
+      "\t\tIf only owner or group is specified then only owner or\n" +
+      "\t\tgroup is modified. The owner and group names can only\n" + 
+      "\t\tcontain digits and alphabet. These names are case sensitive.\n";
+    
+    String chgrp = FsShellPermissions.CHGRP_USAGE + "\n" +
+      "\t\tThis is equivalent to -chown ... :GROUP ...\n";
+    
     String help = "-help [cmd]: \tDisplays help for given command or all commands if none\n" +
     String help = "-help [cmd]: \tDisplays help for given command or all commands if none\n" +
       "\t\tis specified.\n";
       "\t\tis specified.\n";
 
 
@@ -1213,6 +1330,12 @@ public class FsShell extends Configured implements Tool {
       System.out.println(stat);
       System.out.println(stat);
     } else if ("tail".equals(cmd)) {
     } else if ("tail".equals(cmd)) {
       System.out.println(tail);
       System.out.println(tail);
+    } else if ("chmod".equals(cmd)) {
+      System.out.println(chmod);
+    } else if ("chown".equals(cmd)) {
+      System.out.println(chown);
+    } else if ("chgrp".equals(cmd)) {
+      System.out.println(chgrp);
     } else if ("help".equals(cmd)) {
     } else if ("help".equals(cmd)) {
       System.out.println(help);
       System.out.println(help);
     } else {
     } else {
@@ -1380,6 +1503,9 @@ public class FsShell extends Configured implements Tool {
       System.err.println("           [-test -[ezd] <path>]");
       System.err.println("           [-test -[ezd] <path>]");
       System.err.println("           [-stat [format] <path>]");
       System.err.println("           [-stat [format] <path>]");
       System.err.println("           [" + TAIL_USAGE + "]");
       System.err.println("           [" + TAIL_USAGE + "]");
+      System.err.println("           [" + FsShellPermissions.CHMOD_USAGE + "]");      
+      System.err.println("           [" + FsShellPermissions.CHOWN_USAGE + "]");
+      System.err.println("           [" + FsShellPermissions.CHGRP_USAGE + "]");
       System.err.println("           [-help [cmd]]");
       System.err.println("           [-help [cmd]]");
       System.err.println();
       System.err.println();
       ToolRunner.printGenericCommandUsage(System.err);
       ToolRunner.printGenericCommandUsage(System.err);
@@ -1463,6 +1589,10 @@ public class FsShell extends Configured implements Tool {
         moveToLocal(argv[i++], new Path(argv[i++]));
         moveToLocal(argv[i++], new Path(argv[i++]));
       } else if ("-setrep".equals(cmd)) {
       } else if ("-setrep".equals(cmd)) {
         setReplication(argv, i);           
         setReplication(argv, i);           
+      } else if ("-chmod".equals(cmd) || 
+                 "-chown".equals(cmd) ||
+                 "-chgrp".equals(cmd)) {
+        FsShellPermissions.changePermissions(fs, cmd, argv, i);
       } else if ("-ls".equals(cmd)) {
       } else if ("-ls".equals(cmd)) {
         if (i < argv.length) {
         if (i < argv.length) {
           exitCode = doall(cmd, argv, getConf(), i);
           exitCode = doall(cmd, argv, getConf(), i);

+ 289 - 0
src/java/org/apache/hadoop/fs/FsShellPermissions.java

@@ -0,0 +1,289 @@
+/**
+ * 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;
+
+import java.io.IOException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.hadoop.fs.FsShell.CmdHandler;
+import org.apache.hadoop.fs.permission.FsPermission;
+
+
+/**
+ * This class is the home for file permissions related commands.
+ * Moved to this seperate class since FsShell is getting too large.
+ */
+class FsShellPermissions {
+  
+  /*========== chmod ==========*/
+   
+  /* The pattern is alsmost as flexible as mode allowed by 
+   * chmod shell command. The main restriction is that we recognize only rwxX.
+   * To reduce errors we also enforce 3 digits for octal mode.
+   */  
+  private static Pattern chmodNormalPattern = 
+             Pattern.compile("\\G\\s*([ugoa]*)([+=-]+)([rwxX]+)([,\\s]*)\\s*");
+  private static Pattern chmodOctalPattern =
+            Pattern.compile("^\\s*[+]?([0-7]{3})\\s*$");
+  
+  static String CHMOD_USAGE = 
+                            "-chmod [-R] <MODE[,MODE]... | OCTALMODE> PATH...";
+
+  private static class ChmodHandler extends CmdHandler {
+
+    private short userMode, groupMode, othersMode;
+    private char userType = '+', groupType = '+', othersType='+';
+
+    private void applyNormalPattern(String modeStr, Matcher matcher)
+                                    throws IOException {
+      boolean commaSeperated = false;
+
+      for(int i=0; i < 1 || matcher.end() < modeStr.length(); i++) {
+        if (i>0 && (!commaSeperated || !matcher.find())) {
+          patternError(modeStr);
+        }
+
+        /* groups : 1 : [ugoa]*
+         *          2 : [+-=]
+         *          3 : [rwxX]+
+         *          4 : [,\s]*
+         */
+
+        String str = matcher.group(2);
+        char type = str.charAt(str.length() - 1);
+
+        boolean user, group, others;
+        user = group = others = false;
+
+        for(char c : matcher.group(1).toCharArray()) {
+          switch (c) {
+          case 'u' : user = true; break;
+          case 'g' : group = true; break;
+          case 'o' : others = true; break;
+          case 'a' : break;
+          default  : throw new RuntimeException("Unexpected");          
+          }
+        }
+
+        if (!(user || group || others)) { // same as specifying 'a'
+          user = group = others = true;
+        }
+
+        short  mode = 0;
+        for(char c : matcher.group(3).toCharArray()) {
+          switch (c) {
+          case 'r' : mode |= 4; break;
+          case 'w' : mode |= 2; break;
+          case 'x' : mode |= 1; break;
+          case 'X' : mode |= 8; break;
+          default  : throw new RuntimeException("Unexpected");
+          }
+        }
+
+        if ( user ) {
+          userMode = mode;
+          userType = type;
+        }
+
+        if ( group ) {
+          groupMode = mode;
+          groupType = type;
+        }
+
+        if ( others ) {
+          othersMode = mode;
+          othersType = type;
+        }
+
+        commaSeperated = matcher.group(4).contains(",");
+      }
+    }
+
+    private void applyOctalPattern(String modeStr, Matcher matcher) {
+      userType = groupType = othersType = '=';
+      String str = matcher.group(1);
+      userMode = Short.valueOf(str.substring(0, 1));
+      groupMode = Short.valueOf(str.substring(1, 2));
+      othersMode = Short.valueOf(str.substring(2, 3));      
+    }
+
+    private void patternError(String mode) throws IOException {
+      throw new IOException("chmod : mode '" + mode + 
+                            "' does not match the expected pattern.");      
+    }
+
+    ChmodHandler(FileSystem fs, String modeStr) throws IOException {
+      super("chmod", fs);
+      Matcher matcher = null;
+
+      if ((matcher = chmodNormalPattern.matcher(modeStr)).find()) {
+        applyNormalPattern(modeStr, matcher);
+      } else if ((matcher = chmodOctalPattern.matcher(modeStr)).matches()) {
+        applyOctalPattern(modeStr, matcher);
+      } else {
+        patternError(modeStr);
+      }
+    }
+
+    private int applyChmod(char type, int mode, int existing, boolean exeOk) {
+      boolean capX = false;
+
+      if ((mode&8) != 0) { // convert X to x;
+        capX = true;
+        mode &= ~8;
+        mode |= 1;
+      }
+
+      switch (type) {
+      case '+' : mode = mode | existing; break;
+      case '-' : mode = (~mode) & existing; break;
+      case '=' : break;
+      default  : throw new RuntimeException("Unexpected");      
+      }
+
+      // if X is specified add 'x' only if exeOk or x was already set.
+      if (capX && !exeOk && (mode&1) != 0 && (existing&1) == 0) {
+        mode &= ~1; // remove x
+      }
+
+      return mode;
+    }
+
+    @Override
+    public void run(FileStatus file) throws IOException {
+      FsPermission perms = file.getPermission();
+      int existing = perms.toShort();
+      boolean exeOk = file.isDir() || (existing & 0111) != 0;
+      int newperms = ( applyChmod(userType, userMode, 
+                                  (existing>>>6)&7, exeOk) << 6 |
+                       applyChmod(groupType, groupMode, 
+                                  (existing>>>3)&7, exeOk) << 3 |
+                       applyChmod(othersType, othersMode, existing&7, exeOk) );
+
+      if (existing != newperms) {
+        try {
+          getFS().setPermission(file.getPath(), 
+                                new FsPermission((short)newperms));
+        } catch (IOException e) {
+          System.err.println(getName() + ": changing permissions of '" + 
+                             file.getPath() + "':" + e.getMessage());
+        }
+      }
+    }
+  }
+
+  /*========== chown ==========*/
+  
+  ///allows only alpha-numberic names for owner group
+  static private Pattern chownPattern = 
+         Pattern.compile("^\\s*(\\p{Alnum}+)?([:.](\\p{Alnum}*))?\\s*$");
+  static private Pattern chgrpPattern = 
+         Pattern.compile("^\\s*(\\p{Alnum}+)\\s*$");
+  
+  static String CHOWN_USAGE = "-chown [-R] [OWNER][:[GROUP]] PATH...";
+  static String CHGRP_USAGE = "-chown [-R] GROUP PATH...";  
+
+  private static class ChownHandler extends CmdHandler {
+    protected String owner = null;
+    protected String group = null;
+
+    protected ChownHandler(String cmd, FileSystem fs) { //for chgrp
+      super(cmd, fs);
+    }
+
+    ChownHandler(FileSystem fs, String ownerStr) throws IOException {
+      super("chown", fs);
+      Matcher matcher = chownPattern.matcher(ownerStr);
+      if (!matcher.matches()) {
+        throw new IOException("'" + ownerStr + "' does not match " +
+                              "expected pattern for [owner][:group].");
+      }
+      owner = matcher.group(1);
+      group = matcher.group(3);
+      if (group != null && group.length() == 0) {
+        group = null;
+      }
+      if (owner == null && group == null) {
+        throw new IOException("'" + ownerStr + "' does not specify " +
+                              " onwer or group.");
+      }
+    }
+
+    @Override
+    public void run(FileStatus file) throws IOException {
+      //Should we do case insensitive match?  
+      String newOwner = (owner == null || owner.equals(file.getOwner())) ?
+                        null : owner;
+      String newGroup = (group == null || group.equals(file.getGroup())) ?
+                        null : group;
+
+      if (newOwner != null || newGroup != null) {
+        try {
+          fs.setOwner(file.getPath(), newOwner, newGroup);
+        } catch (IOException e) {
+          System.err.println(getName() + ": changing ownership of '" + 
+                             file.getPath() + "':" + e.getMessage());
+
+        }
+      }
+    }
+  }
+
+  /*========== chgrp ==========*/    
+  
+  private static class ChgrpHandler extends ChownHandler {
+    ChgrpHandler(FileSystem fs, String groupStr) throws IOException {
+      super("chgrp", fs);
+
+      Matcher matcher = chgrpPattern.matcher(groupStr);
+      if (!matcher.matches()) {
+        throw new IOException("'" + groupStr + "' does not match " +
+        "expected pattern for group");
+      }
+      group = matcher.group(1);
+    }
+  }
+
+  static void changePermissions(FileSystem fs, String cmd, 
+                                String argv[], int startIndex)
+                                throws IOException {
+    CmdHandler handler = null;
+    boolean recursive = false;
+
+    // handle common arguments, currently only "-R" 
+    for (; startIndex < argv.length && argv[startIndex].equals("-R"); 
+    startIndex++) {
+      recursive = true;
+    }
+
+    if ( startIndex >= argv.length ) {
+      throw new IOException("Not enough arguments for the command");
+    }
+
+    if (cmd.equals("-chmod")) {
+      handler = new ChmodHandler(fs, argv[startIndex++]);
+    } else if (cmd.equals("-chown")) {
+      handler = new ChownHandler(fs, argv[startIndex++]);
+    } else if (cmd.equals("-chgrp")) {
+      handler = new ChgrpHandler(fs, argv[startIndex++]);
+    }
+
+    FsShell.runCmdHandler(handler, argv, startIndex, recursive);
+  } 
+}

+ 58 - 0
src/test/org/apache/hadoop/dfs/TestDFSShell.java

@@ -301,6 +301,64 @@ public class TestDFSShell extends TestCase {
     }
     }
   }
   }
 
 
+  /**
+   * Test chmod, getOwner(). 
+   * How do we test chown and chgrp?
+   */
+  public void testFilePermissions() throws IOException {
+    Configuration conf = new Configuration();
+    //Temperorily use LocalFileSystem until HADOOP-1298 is committed
+    conf.set("fs.default.name", "local");
+    FileSystem fs = FileSystem.getLocal(conf);
+    
+    FsShell shell = new FsShell();
+    shell.setConf(conf);
+    
+    try {
+     String chmodDir = (new File(TEST_ROOT_DIR, "chmodTest")).getAbsolutePath(); 
+     
+     //first make dir
+     Path dir = new Path(chmodDir);
+     fs.delete(dir);
+     fs.mkdirs(dir);
+
+     shell.run(new String[]{ "-chmod", "u+rwx,g=rw,o-rwx", chmodDir });
+     assertEquals("rwxrw----",
+                  fs.getFileStatus(dir).getPermission().toString());
+
+     //create an empty file
+     Path file = new Path(chmodDir, "file");
+     TestDFSShell.createLocalFile(new File(file.toString()));
+
+     //test octal mode
+     shell.run(new String[]{ "-chmod", "644", file.toString()});
+     assertEquals("rw-r--r--",
+                  fs.getFileStatus(file).getPermission().toString());
+
+     //test recursive
+     shell.run(new String[]{ "-chmod", "-R", "a+rwX", chmodDir });
+     assertEquals("rwxrwxrwx",
+                  fs.getFileStatus(dir).getPermission().toString()); 
+     assertEquals("rw-rw-rw-",
+                  fs.getFileStatus(file).getPermission().toString());
+     
+     //test username
+     assertEquals(System.getProperty("user.name"), 
+                  fs.getFileStatus(file).getOwner());
+
+     fs.delete(dir);     
+    } catch (IOException e) {
+      throw e;
+    } catch (RuntimeException e) {
+      throw e;
+    } catch (Exception e) {
+      throw new IOException(StringUtils.stringifyException(e));
+    } finally {
+      shell.close();
+      fs.close();
+    }
+    
+  }
   /**
   /**
    * Tests various options of DFSShell.
    * Tests various options of DFSShell.
    */
    */

+ 8 - 7
src/webapps/datanode/browseDirectory.jsp

@@ -73,10 +73,9 @@
       // directory
       // directory
       DFSFileInfo[] files = dfs.listPaths(target);
       DFSFileInfo[] files = dfs.listPaths(target);
       //generate a table and dump the info
       //generate a table and dump the info
-      String [] headings = new String[6];
-      headings[0] = "Name"; headings[1] = "Type"; headings[2] = "Size";
-      headings[3] = "Replication"; headings[4] = "Block Size";
-      headings[5] = "Modification Time";
+      String [] headings = { "Name", "Type", "Size", "Replication", 
+                              "Block Size", "Modification Time",
+                              "Permission", "Owner", "Group" };
       out.print("<h3>Contents of directory ");
       out.print("<h3>Contents of directory ");
       JspHelper.printPathWithLinks(dir, out, namenodeInfoPort);
       JspHelper.printPathWithLinks(dir, out, namenodeInfoPort);
       out.print("</h3><hr>");
       out.print("</h3><hr>");
@@ -97,7 +96,7 @@
         jspHelper.addTableHeader(out);
         jspHelper.addTableHeader(out);
         int row=0;
         int row=0;
         jspHelper.addTableRow(out, headings, row++);
         jspHelper.addTableRow(out, headings, row++);
-        String cols [] = new String[6];
+        String cols [] = new String[headings.length];
         for (int i = 0; i < files.length; i++) {
         for (int i = 0; i < files.length; i++) {
           //Get the location of the first block of the file
           //Get the location of the first block of the file
           if (files[i].getPath().toString().endsWith(".crc")) continue;
           if (files[i].getPath().toString().endsWith(".crc")) continue;
@@ -121,7 +120,6 @@
             cols[2] = FsShell.byteDesc(files[i].getLen());
             cols[2] = FsShell.byteDesc(files[i].getLen());
             cols[3] = Short.toString(files[i].getReplication());
             cols[3] = Short.toString(files[i].getReplication());
             cols[4] = FsShell.byteDesc(files[i].getBlockSize());
             cols[4] = FsShell.byteDesc(files[i].getBlockSize());
-            cols[5] = FsShell.dateForm.format(new Date((files[i].getModificationTime())));
           }
           }
           else {
           else {
             String datanodeUrl = req.getRequestURL()+"?dir="+
             String datanodeUrl = req.getRequestURL()+"?dir="+
@@ -132,8 +130,11 @@
             cols[2] = "";
             cols[2] = "";
             cols[3] = "";
             cols[3] = "";
             cols[4] = "";
             cols[4] = "";
-            cols[5] = FsShell.dateForm.format(new Date((files[i].getModificationTime())));
           }
           }
+          cols[5] = FsShell.dateForm.format(new Date((files[i].getModificationTime())));
+          cols[6] = files[i].getPermission().toString();
+          cols[7] = files[i].getOwner();
+          cols[8] = files[i].getGroup();
           jspHelper.addTableRow(out, cols, row++);
           jspHelper.addTableRow(out, cols, row++);
         }
         }
         jspHelper.addTableFooter(out);
         jspHelper.addTableFooter(out);