فهرست منبع

HDFS-245. Adds a symlink implementation to HDFS. This complements the new symlink feature added in HADOOP-6421
(Eli Collins via Sanjay Radia)


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

Sanjay Radia 15 سال پیش
والد
کامیت
8eb688e26d
27فایلهای تغییر یافته به همراه1453 افزوده شده و 437 حذف شده
  1. 3 0
      CHANGES.txt
  2. 51 18
      src/java/org/apache/hadoop/fs/Hdfs.java
  3. 151 60
      src/java/org/apache/hadoop/hdfs/DFSClient.java
  4. 3 2
      src/java/org/apache/hadoop/hdfs/DFSInputStream.java
  5. 12 6
      src/java/org/apache/hadoop/hdfs/DFSOutputStream.java
  6. 123 54
      src/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
  7. 2 2
      src/java/org/apache/hadoop/hdfs/protocol/FSConstants.java
  8. 32 2
      src/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java
  9. 74 0
      src/java/org/apache/hadoop/hdfs/protocol/UnresolvedPathException.java
  10. 209 101
      src/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  11. 36 4
      src/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
  12. 41 22
      src/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
  13. 160 84
      src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  14. 6 2
      src/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
  15. 25 2
      src/java/org/apache/hadoop/hdfs/server/namenode/INode.java
  16. 57 20
      src/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
  17. 78 0
      src/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java
  18. 10 5
      src/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
  19. 108 44
      src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
  20. 5 1
      src/java/org/apache/hadoop/hdfs/server/namenode/metrics/NameNodeMetrics.java
  21. 11 5
      src/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java
  22. 1 0
      src/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageVisitor.java
  23. 8 1
      src/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/LsImageVisitor.java
  24. 1 1
      src/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewer.java
  25. 235 0
      src/test/hdfs/org/apache/hadoop/fs/TestFcHdfsSymlink.java
  26. 8 0
      src/test/hdfs/org/apache/hadoop/hdfs/TestDFSClientRetries.java
  27. 3 1
      src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestFsck.java

+ 3 - 0
CHANGES.txt

@@ -21,6 +21,9 @@ Trunk (unreleased changes)
     HDFS-935. Adds a real user component in Delegation token.
     (Jitendra Nath Pandey via ddas)
 
+    HDFS-245. Adds a symlink implementation to HDFS. This complements the new symlink feature added in HADOOP-6421
+    (Eli Collins via Sanjay Radia)
+
   IMPROVEMENTS
     
     HDFS-703. Replace current fault injection implementation with one

+ 51 - 18
src/java/org/apache/hadoop/fs/Hdfs.java

@@ -83,23 +83,26 @@ public class Hdfs extends AbstractFileSystem {
   }
 
   @Override
-  protected boolean delete(Path f, boolean recursive) throws IOException {
+  protected boolean delete(Path f, boolean recursive) 
+      throws IOException, UnresolvedLinkException {
     return dfs.delete(getUriPath(f), recursive);
   }
 
   @Override
   protected BlockLocation[] getFileBlockLocations(Path p, long start, long len)
-      throws IOException {
+      throws IOException, UnresolvedLinkException {
     return dfs.getBlockLocations(p.toString(), start, len);
   }
 
   @Override
-  protected FileChecksum getFileChecksum(Path f) throws IOException {
+  protected FileChecksum getFileChecksum(Path f) 
+      throws IOException, UnresolvedLinkException {
     return dfs.getFileChecksum(getUriPath(f));
   }
 
   @Override
-  protected FileStatus getFileStatus(Path f) throws IOException {
+  protected FileStatus getFileStatus(Path f) 
+      throws IOException, UnresolvedLinkException {
     HdfsFileStatus fi = dfs.getFileInfo(getUriPath(f));
     if (fi != null) {
       return makeQualified(fi, f);
@@ -107,12 +110,25 @@ public class Hdfs extends AbstractFileSystem {
       throw new FileNotFoundException("File does not exist: " + f.toString());
     }
   }
+  
+  @Override
+  public FileStatus getFileLinkStatus(Path f) 
+      throws IOException, UnresolvedLinkException {
+    HdfsFileStatus fi = dfs.getFileLinkInfo(getUriPath(f));
+    if (fi != null) {
+      return makeQualified(fi, f);
+    } else {
+      throw new FileNotFoundException("File does not exist: " + f);
+    }
+  }  
 
   private FileStatus makeQualified(HdfsFileStatus f, Path parent) {
+    // NB: symlink is made fully-qualified in FileContext. 
     return new FileStatus(f.getLen(), f.isDir(), f.getReplication(),
         f.getBlockSize(), f.getModificationTime(),
         f.getAccessTime(),
         f.getPermission(), f.getOwner(), f.getGroup(),
+        f.isSymlink() ? new Path(f.getSymlink()) : null,
         (f.getFullPath(parent)).makeQualified(
             getUri(), null)); // fully-qualify path
   }
@@ -129,7 +145,8 @@ public class Hdfs extends AbstractFileSystem {
   }
 
   @Override
-  protected FileStatus[] listStatus(Path f) throws IOException {
+  protected FileStatus[] listStatus(Path f) 
+      throws IOException, UnresolvedLinkException {
     HdfsFileStatus[] infos = dfs.listPaths(getUriPath(f));
     if (infos == null)
       throw new FileNotFoundException("File " + f + " does not exist.");
@@ -143,57 +160,73 @@ public class Hdfs extends AbstractFileSystem {
 
   @Override
   protected void mkdir(Path dir, FsPermission permission, boolean createParent)
-    throws IOException {
+    throws IOException, UnresolvedLinkException {
     dfs.mkdirs(getUriPath(dir), permission, createParent);
-
   }
 
   @Override
-  protected FSDataInputStream open(Path f, int bufferSize) throws IOException {
+  protected FSDataInputStream open(Path f, int bufferSize) 
+      throws IOException, UnresolvedLinkException {
     return new DFSClient.DFSDataInputStream(dfs.open(getUriPath(f),
         bufferSize, verifyChecksum));
   }
 
   @Override
-  protected void renameInternal(Path src, Path dst) throws IOException {
+  protected void renameInternal(Path src, Path dst) 
+    throws IOException, UnresolvedLinkException {
     dfs.rename(getUriPath(src), getUriPath(dst));
   }
 
   @Override
   protected void renameInternal(Path src, Path dst, boolean overwrite)
-      throws IOException {
+      throws IOException, UnresolvedLinkException {
     dfs.rename(getUriPath(src), getUriPath(dst),
         overwrite ? Options.Rename.OVERWRITE : Options.Rename.NONE);
   }
 
   @Override
   protected void setOwner(Path f, String username, String groupname)
-    throws IOException {
+    throws IOException, UnresolvedLinkException {
     dfs.setOwner(getUriPath(f), username, groupname);
-
   }
 
   @Override
   protected void setPermission(Path f, FsPermission permission)
-    throws IOException {
+    throws IOException, UnresolvedLinkException {
     dfs.setPermission(getUriPath(f), permission);
-
   }
 
   @Override
   protected boolean setReplication(Path f, short replication)
-    throws IOException {
+    throws IOException, UnresolvedLinkException {
     return dfs.setReplication(getUriPath(f), replication);
   }
 
   @Override
-  protected void setTimes(Path f, long mtime, long atime) throws IOException {
+  protected void setTimes(Path f, long mtime, long atime) 
+    throws IOException, UnresolvedLinkException {
     dfs.setTimes(getUriPath(f), mtime, atime);
-
   }
 
   @Override
-  protected void setVerifyChecksum(boolean verifyChecksum) throws IOException {
+  protected void setVerifyChecksum(boolean verifyChecksum) 
+    throws IOException {
     this.verifyChecksum = verifyChecksum;
   }
+  
+  @Override
+  protected boolean supportsSymlinks() {
+    return true;
+  }  
+  
+  @Override
+  protected void createSymlink(Path target, Path link, boolean createParent)
+    throws IOException, UnresolvedLinkException {
+    dfs.createSymlink(target.toString(), getUriPath(link), createParent);
+  }
+
+  @Override
+  protected Path getLinkTarget(Path p) throws IOException { 
+    return new Path(dfs.getLinkTarget(getUriPath(p)));
+  }
 }

+ 151 - 60
src/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -56,6 +56,7 @@ import org.apache.hadoop.fs.FsStatus;
 import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Options;
+import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -70,6 +71,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
@@ -379,12 +381,14 @@ public class DFSClient implements FSConstants, java.io.Closeable {
   }
 
   static LocatedBlocks callGetBlockLocations(ClientProtocol namenode,
-      String src, long start, long length) throws IOException {
+      String src, long start, long length) 
+      throws IOException, UnresolvedLinkException {
     try {
       return namenode.getBlockLocations(src, start, length);
     } catch(RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
-                                    FileNotFoundException.class);
+                                     FileNotFoundException.class,
+                                     UnresolvedPathException.class);
     }
   }
 
@@ -401,7 +405,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    * as the data-block the task processes. 
    */
   public BlockLocation[] getBlockLocations(String src, long start, 
-    long length) throws IOException {
+    long length) throws IOException, UnresolvedLinkException {
     LocatedBlocks blocks = callGetBlockLocations(namenode, src, start, length);
     if (blocks == null) {
       return new BlockLocation[0];
@@ -430,7 +434,8 @@ public class DFSClient implements FSConstants, java.io.Closeable {
     return blkLocations;
   }
 
-  public DFSInputStream open(String src) throws IOException {
+  public DFSInputStream open(String src) 
+      throws IOException, UnresolvedLinkException {
     return open(src, conf.getInt("io.file.buffer.size", 4096), true, null);
   }
 
@@ -443,8 +448,8 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    */
   @Deprecated
   public DFSInputStream open(String src, int buffersize, boolean verifyChecksum,
-                      FileSystem.Statistics stats
-      ) throws IOException {
+                             FileSystem.Statistics stats)
+      throws IOException, UnresolvedLinkException {
     return open(src, buffersize, verifyChecksum);
   }
   
@@ -455,8 +460,8 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    * inner subclass of InputStream that does the right out-of-band
    * work.
    */
-  public DFSInputStream open(String src, int buffersize, boolean verifyChecksum
-      ) throws IOException {
+  public DFSInputStream open(String src, int buffersize, boolean verifyChecksum)
+      throws IOException, UnresolvedLinkException {
     checkOpen();
     //    Get block info from namenode
     return new DFSInputStream(this, src, buffersize, verifyChecksum);
@@ -468,11 +473,11 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    * @param src stream name
    * @param overwrite do not check for file existence if true
    * @return output stream
+   * @throws UnresolvedLinkException if a symlink is encountered in src.
    * @throws IOException
    */
-  public OutputStream create(String src, 
-                             boolean overwrite
-                             ) throws IOException {
+  public OutputStream create(String src, boolean overwrite) 
+      throws IOException, UnresolvedLinkException {
     return create(src, overwrite, defaultReplication, defaultBlockSize, null);
   }
     
@@ -483,12 +488,13 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    * @param src stream name
    * @param overwrite do not check for file existence if true
    * @return output stream
+   * @throws UnresolvedLinkException if a symlink is encountered in src.
    * @throws IOException
    */
   public OutputStream create(String src, 
                              boolean overwrite,
-                             Progressable progress
-                             ) throws IOException {
+                             Progressable progress)
+      throws IOException, UnresolvedLinkException {
     return create(src, overwrite, defaultReplication, defaultBlockSize, null);
   }
     
@@ -500,13 +506,14 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    * @param overwrite do not check for file existence if true
    * @param replication block replication
    * @return output stream
+   * @throws UnresolvedLinkException if a symlink is encountered in src.
    * @throws IOException
    */
   public OutputStream create(String src, 
                              boolean overwrite, 
                              short replication,
-                             long blockSize
-                             ) throws IOException {
+                             long blockSize)
+      throws IOException, UnresolvedLinkException {
     return create(src, overwrite, replication, blockSize, null);
   }
 
@@ -528,14 +535,15 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    * @param overwrite do not check for file existence if true
    * @param replication block replication
    * @return output stream
+   * @throws UnresolvedLinkException if a symlink is encountered in src.
    * @throws IOException
    */
   public OutputStream create(String src, 
                              boolean overwrite, 
                              short replication,
                              long blockSize,
-                             Progressable progress
-                             ) throws IOException {
+                             Progressable progress)
+      throws IOException, UnresolvedLinkException {
     return create(src, overwrite, replication, blockSize, progress,
         conf.getInt("io.file.buffer.size", 4096));
   }
@@ -550,8 +558,8 @@ public class DFSClient implements FSConstants, java.io.Closeable {
       short replication,
       long blockSize,
       Progressable progress,
-      int buffersize
-      ) throws IOException {
+      int buffersize)
+      throws IOException, UnresolvedLinkException {
     return create(src, FsPermission.getDefault(),
         overwrite ? EnumSet.of(CreateFlag.OVERWRITE) : EnumSet.of(CreateFlag.CREATE), 
         replication, blockSize, progress, buffersize);
@@ -568,8 +576,8 @@ public class DFSClient implements FSConstants, java.io.Closeable {
       short replication,
       long blockSize,
       Progressable progress,
-      int buffersize
-      ) throws IOException {
+      int buffersize)
+      throws IOException, UnresolvedLinkException {
     return create(src, permission, flag, true,
         replication, blockSize, progress, buffersize);
   }
@@ -587,6 +595,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    * @param replication block replication
    * @return output stream
    * @throws IOException
+   * @throws UnresolvedLinkException if src contains a symlink. 
    * @see ClientProtocol#create(String, FsPermission, String, EnumSetWritable, boolean, short, long)
    */
   public OutputStream create(String src, 
@@ -596,8 +605,8 @@ public class DFSClient implements FSConstants, java.io.Closeable {
                              short replication,
                              long blockSize,
                              Progressable progress,
-                             int buffersize
-                             ) throws IOException {
+                             int buffersize)
+    throws IOException, UnresolvedLinkException {
     checkOpen();
     if (permission == null) {
       permission = FsPermission.getDefault();
@@ -616,7 +625,6 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    * Same as {{@link #create(String, FsPermission, EnumSet, short, long,
    *  Progressable, int)}   except that the permission
    *   is absolute (ie has already been masked with umask.
-   * 
    */
   public OutputStream primitiveCreate(String src, 
                              FsPermission absPermission,
@@ -627,14 +635,49 @@ public class DFSClient implements FSConstants, java.io.Closeable {
                              Progressable progress,
                              int buffersize,
                              int bytesPerChecksum)
-    throws IOException {
+      throws IOException, UnresolvedLinkException {
     checkOpen();
     OutputStream result = new DFSOutputStream(this, src, absPermission,
         flag, createParent, replication, blockSize, progress, buffersize,
         bytesPerChecksum);
     leasechecker.put(src, result);
     return result;
-  } 
+  }
+  
+  /**
+   * Creates a symbolic link.
+   * 
+   * @see ClientProtocol#createSymlink(String, String,FsPermission, boolean) 
+   */
+  public void createSymlink(String target, String link, boolean createParent)
+      throws IOException, UnresolvedLinkException {
+    try {
+      FsPermission dirPerm = 
+          FsPermission.getDefault().applyUMask(FsPermission.getUMask(conf)); 
+      namenode.createSymlink(target, link, dirPerm, createParent);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+                                     NSQuotaExceededException.class, 
+                                     DSQuotaExceededException.class,
+                                     FileAlreadyExistsException.class, 
+                                     UnresolvedPathException.class);
+    }
+  }
+
+  /**
+   * Resolve the *first* symlink, if any, in the path.
+   * 
+   * @see ClientProtocol#getLinkTarget(String)
+   */
+  public String getLinkTarget(String path) throws IOException { 
+    checkOpen();
+    try {
+      return namenode.getLinkTarget(path);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+                                     FileNotFoundException.class);
+    }
+  }
 
   /**
    * Append to an existing HDFS file.  
@@ -644,10 +687,11 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    * @param progress for reporting write-progress
    * @return an output stream for writing into the file
    * @throws IOException
+   * @throws UnresolvedLinkException if the path contains a symlink.
    * @see ClientProtocol#append(String, String)
    */
-  OutputStream append(String src, int buffersize, Progressable progress
-      ) throws IOException {
+  OutputStream append(String src, int buffersize, Progressable progress)
+      throws IOException, UnresolvedLinkException {
     checkOpen();
     HdfsFileStatus stat = null;
     LocatedBlock lastBlock = null;
@@ -658,7 +702,8 @@ public class DFSClient implements FSConstants, java.io.Closeable {
       throw re.unwrapRemoteException(FileNotFoundException.class,
                                      AccessControlException.class,
                                      NSQuotaExceededException.class,
-                                     DSQuotaExceededException.class);
+                                     DSQuotaExceededException.class,
+                                     UnresolvedPathException.class);
     }
     OutputStream result = new DFSOutputStream(this, src, buffersize, progress,
         lastBlock, stat, conf.getInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, 
@@ -676,14 +721,15 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    * @return true is successful or false if file does not exist 
    */
   public boolean setReplication(String src, 
-                                short replication
-                                ) throws IOException {
+                                short replication)
+      throws IOException, UnresolvedLinkException {
     try {
       return namenode.setReplication(src, replication);
     } catch(RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
                                      NSQuotaExceededException.class,
-                                     DSQuotaExceededException.class);
+                                     DSQuotaExceededException.class,
+                                     UnresolvedPathException.class);
     }
   }
 
@@ -693,14 +739,16 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    * @deprecated Use {@link #rename(String, String, Options.Rename...)} instead.
    */
   @Deprecated
-  public boolean rename(String src, String dst) throws IOException {
+  public boolean rename(String src, String dst) 
+      throws IOException, UnresolvedLinkException {
     checkOpen();
     try {
       return namenode.rename(src, dst);
     } catch(RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
                                      NSQuotaExceededException.class,
-                                     DSQuotaExceededException.class);
+                                     DSQuotaExceededException.class,
+                                     UnresolvedPathException.class);
     }
   }
 
@@ -708,28 +756,32 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    * Move blocks from src to trg and delete src
    * See {@link ClientProtocol#concat(String, String [])}. 
    */
-  public void concat(String trg, String [] srcs) throws IOException {
+  public void concat(String trg, String [] srcs) 
+      throws IOException, UnresolvedLinkException {
     checkOpen();
     try {
       namenode.concat(trg, srcs);
     } catch(RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
                                      NSQuotaExceededException.class,
-                                     DSQuotaExceededException.class);
+                                     DSQuotaExceededException.class,
+                                     UnresolvedPathException.class);
     }
   }
   /**
    * Rename file or directory.
    * See {@link ClientProtocol#rename(String, String, Options.Rename...)}
    */
-  public void rename(String src, String dst, Options.Rename... options) throws IOException {
+  public void rename(String src, String dst, Options.Rename... options) 
+      throws IOException, UnresolvedLinkException {
     checkOpen();
     try {
       namenode.rename(src, dst, options);
     } catch(RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
                                      NSQuotaExceededException.class,
-                                     DSQuotaExceededException.class);
+                                     DSQuotaExceededException.class,
+                                     UnresolvedPathException.class);
     }
   }
   /**
@@ -737,7 +789,8 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    * See {@link ClientProtocol#delete(String)}. 
    */
   @Deprecated
-  public boolean delete(String src) throws IOException {
+  public boolean delete(String src) 
+      throws IOException, UnresolvedLinkException {
     checkOpen();
     return namenode.delete(src, true);
   }
@@ -747,12 +800,14 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    * delete contents of the directory if non empty and recursive 
    * set to true
    */
-  public boolean delete(String src, boolean recursive) throws IOException {
+  public boolean delete(String src, boolean recursive) 
+      throws IOException, UnresolvedLinkException {
     checkOpen();
     try {
       return namenode.delete(src, recursive);
     } catch(RemoteException re) {
-      throw re.unwrapRemoteException(AccessControlException.class);
+      throw re.unwrapRemoteException(AccessControlException.class,
+                                     UnresolvedPathException.class);
     }
   }
   
@@ -763,24 +818,47 @@ public class DFSClient implements FSConstants, java.io.Closeable {
     return getFileInfo(src) != null;
   }
 
-  public HdfsFileStatus[] listPaths(String src) throws IOException {
+  public HdfsFileStatus[] listPaths(String src) 
+    throws IOException, UnresolvedLinkException {
     checkOpen();
     try {
       return namenode.getListing(src);
     } catch(RemoteException re) {
-      throw re.unwrapRemoteException(AccessControlException.class);
+      throw re.unwrapRemoteException(AccessControlException.class,
+                                     UnresolvedPathException.class);
     }
   }
 
-  public HdfsFileStatus getFileInfo(String src) throws IOException {
+  public HdfsFileStatus getFileInfo(String src) 
+      throws IOException, UnresolvedLinkException {
     checkOpen();
     try {
       return namenode.getFileInfo(src);
     } catch(RemoteException re) {
-      throw re.unwrapRemoteException(AccessControlException.class);
+      throw re.unwrapRemoteException(AccessControlException.class,
+                                     UnresolvedPathException.class);
     }
   }
 
+  /**
+   * Get the file info for a specific file or directory. If src
+   * refers to a symlink then the FileStatus of the link is returned.
+   * @param src path to a file or directory.
+   * @throws IOException
+   * @throws UnresolvedLinkException if the path contains symlinks
+   * @return FileStatus describing src.
+   */
+  public HdfsFileStatus getFileLinkInfo(String src) 
+      throws IOException, UnresolvedLinkException {
+    checkOpen();
+    try {
+      return namenode.getFileLinkInfo(src);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+                                     UnresolvedPathException.class);
+     }
+   }
+
   /**
    * Get the checksum of a file.
    * @param src The file path
@@ -921,15 +999,17 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    * @param src path name.
    * @param permission
    * @throws <code>FileNotFoundException</code> is file does not exist.
+   * @throws UnresolvedLinkException if the path contains a symlink.
    */
-  public void setPermission(String src, FsPermission permission
-                            ) throws IOException {
+  public void setPermission(String src, FsPermission permission)
+      throws IOException, UnresolvedLinkException {
     checkOpen();
     try {
       namenode.setPermission(src, permission);
     } catch(RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
-                                     FileNotFoundException.class);
+                                     FileNotFoundException.class,
+                                     UnresolvedPathException.class);
     }
   }
 
@@ -939,15 +1019,17 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    * @param username user id.
    * @param groupname user group.
    * @throws <code>FileNotFoundException</code> is file does not exist.
+   * @throws UnresolvedLinkException if the path contains a symlink.
    */
-  public void setOwner(String src, String username, String groupname
-                      ) throws IOException {
+  public void setOwner(String src, String username, String groupname)
+      throws IOException, UnresolvedLinkException {
     checkOpen();
     try {
       namenode.setOwner(src, username, groupname);
     } catch(RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
-                                     FileNotFoundException.class);
+                                     FileNotFoundException.class,
+                                     UnresolvedPathException.class);
     }
   }
 
@@ -1075,9 +1157,11 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    * If permission == null, use {@link FsPermission#getDefault()}.
    * @param createParent create missing parent directory if true
    * @return True if the operation success.
+   * @throws UnresolvedLinkException if the path contains a symlink.
    * @see ClientProtocol#mkdirs(String, FsPermission, boolean)
    */
-  public boolean mkdirs(String src, FsPermission permission, boolean createParent)throws IOException{
+  public boolean mkdirs(String src, FsPermission permission, boolean createParent)
+      throws IOException, UnresolvedLinkException {
     checkOpen();
     if (permission == null) {
       permission = FsPermission.getDefault();
@@ -1091,16 +1175,18 @@ public class DFSClient implements FSConstants, java.io.Closeable {
                                      NSQuotaExceededException.class,
                                      DSQuotaExceededException.class,
                                      FileNotFoundException.class,
-                                     FileAlreadyExistsException.class);
+                                     FileAlreadyExistsException.class,
+                                     UnresolvedPathException.class);
     }
   }
   
   /**
    * Same {{@link #mkdirs(String, FsPermission, boolean)} except
    * that the permissions has already been masked against umask.
+   * @throws UnresolvedLinkException if the path contains a symlink.
    */
   public boolean primitiveMkdir(String src, FsPermission absPermission)
-    throws IOException{
+    throws IOException, UnresolvedLinkException {
     checkOpen();
     if (absPermission == null) {
       absPermission = 
@@ -1113,7 +1199,8 @@ public class DFSClient implements FSConstants, java.io.Closeable {
     } catch(RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
                                      NSQuotaExceededException.class,
-                                     DSQuotaExceededException.class);
+                                     DSQuotaExceededException.class,
+                                     UnresolvedPathException.class);
     }
   }
 
@@ -1122,7 +1209,8 @@ public class DFSClient implements FSConstants, java.io.Closeable {
       return namenode.getContentSummary(src);
     } catch(RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
-                                     FileNotFoundException.class);
+                                     FileNotFoundException.class,
+                                     UnresolvedPathException.class);
     }
   }
 
@@ -1131,7 +1219,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#setQuota(String, long, long)
    */
   void setQuota(String src, long namespaceQuota, long diskspaceQuota) 
-                                                 throws IOException {
+      throws IOException, UnresolvedLinkException {
     // sanity check
     if ((namespaceQuota <= 0 && namespaceQuota != FSConstants.QUOTA_DONT_SET &&
          namespaceQuota != FSConstants.QUOTA_RESET) ||
@@ -1148,7 +1236,8 @@ public class DFSClient implements FSConstants, java.io.Closeable {
       throw re.unwrapRemoteException(AccessControlException.class,
                                      FileNotFoundException.class,
                                      NSQuotaExceededException.class,
-                                     DSQuotaExceededException.class);
+                                     DSQuotaExceededException.class,
+                                     UnresolvedPathException.class);
     }
   }
 
@@ -1156,13 +1245,15 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    * set the modification and access time of a file
    * @throws FileNotFoundException if the path is not a file
    */
-  public void setTimes(String src, long mtime, long atime) throws IOException {
+  public void setTimes(String src, long mtime, long atime) 
+      throws IOException, UnresolvedLinkException {
     checkOpen();
     try {
       namenode.setTimes(src, mtime, atime);
     } catch(RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
-                                     FileNotFoundException.class);
+                                     FileNotFoundException.class,
+                                     UnresolvedPathException.class);
     }
   }
 

+ 3 - 2
src/java/org/apache/hadoop/hdfs/DFSInputStream.java

@@ -28,6 +28,7 @@ import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.hadoop.fs.ChecksumException;
 import org.apache.hadoop.fs.FSInputStream;
+import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -86,7 +87,7 @@ class DFSInputStream extends FSInputStream {
   }
   
   DFSInputStream(DFSClient dfsClient, String src, int buffersize, boolean verifyChecksum
-                 ) throws IOException {
+                 ) throws IOException, UnresolvedLinkException {
     this.dfsClient = dfsClient;
     this.verifyChecksum = verifyChecksum;
     this.buffersize = buffersize;
@@ -100,7 +101,7 @@ class DFSInputStream extends FSInputStream {
   /**
    * Grab the open-file info from namenode
    */
-  synchronized void openInfo() throws IOException {
+  synchronized void openInfo() throws IOException, UnresolvedLinkException {
     LocatedBlocks newInfo = DFSClient.callGetBlockLocations(dfsClient.namenode, src, 0, prefetchSize);
     if (DFSClient.LOG.isDebugEnabled()) {
       DFSClient.LOG.debug("newInfo = " + newInfo);

+ 12 - 6
src/java/org/apache/hadoop/hdfs/DFSOutputStream.java

@@ -40,6 +40,7 @@ import org.apache.hadoop.fs.FSOutputSummer;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Syncable;
+import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
@@ -49,6 +50,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
 import org.apache.hadoop.hdfs.protocol.DataTransferProtocol.BlockConstructionStage;
 import org.apache.hadoop.hdfs.protocol.DataTransferProtocol.PipelineAck;
 import org.apache.hadoop.hdfs.security.BlockAccessToken;
@@ -928,7 +930,8 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
     }
 
     private LocatedBlock locateFollowingBlock(long start,
-        DatanodeInfo[] excludedNodes) throws IOException {
+        DatanodeInfo[] excludedNodes) 
+        throws IOException, UnresolvedLinkException {
       int retries = conf.getInt("dfs.client.block.write.locateFollowingBlock.retries", 5);
       long sleeptime = 400;
       while (true) {
@@ -939,9 +942,10 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
           } catch (RemoteException e) {
             IOException ue = 
               e.unwrapRemoteException(FileNotFoundException.class,
-                  AccessControlException.class,
-                  NSQuotaExceededException.class,
-                  DSQuotaExceededException.class);
+                                      AccessControlException.class,
+                                      NSQuotaExceededException.class,
+                                      DSQuotaExceededException.class,
+                                      UnresolvedPathException.class);
             if (ue != e) { 
               throw ue; // no need to retry these exceptions
             }
@@ -1049,7 +1053,8 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
    */
   DFSOutputStream(DFSClient dfsClient, String src, FsPermission masked, EnumSet<CreateFlag> flag,
       boolean createParent, short replication, long blockSize, Progressable progress,
-      int buffersize, int bytesPerChecksum) throws IOException {
+      int buffersize, int bytesPerChecksum) 
+      throws IOException, UnresolvedLinkException {
     this(dfsClient, src, blockSize, progress, bytesPerChecksum);
 
     computePacketChunkSize(dfsClient.writePacketSize, bytesPerChecksum);
@@ -1062,7 +1067,8 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
                                      FileAlreadyExistsException.class,
                                      FileNotFoundException.class,
                                      NSQuotaExceededException.class,
-                                     DSQuotaExceededException.class);
+                                     DSQuotaExceededException.class,
+                                     UnresolvedPathException.class);
     }
     streamer = new DataStreamer();
     streamer.start();

+ 123 - 54
src/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java

@@ -24,6 +24,7 @@ import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.Options;
+import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.FSConstants.UpgradeAction;
@@ -53,11 +54,9 @@ public interface ClientProtocol extends VersionedProtocol {
    * Compared to the previous version the following changes have been introduced:
    * (Only the latest change is reflected.
    * The log of historical changes can be retrieved from the svn).
-   * 57: getFileInfo returns HDFSFileStatus;
-   *     getListing returns HDFSFileStatus[].
-   * 
+   * 58: Add symlink APIs.
    */
-  public static final long versionID = 57L;
+  public static final long versionID = 58L;
   
   ///////////////////////////////////////
   // File contents
@@ -80,10 +79,12 @@ public interface ClientProtocol extends VersionedProtocol {
    * @param length range length
    * @return file length and array of blocks with their locations
    * @throws IOException
+   * @throws UnresolvedLinkException if the path contains a symlink.
    */
-  public LocatedBlocks  getBlockLocations(String src,
-                                          long offset,
-                                          long length) throws IOException;
+  public LocatedBlocks getBlockLocations(String src,
+                                         long offset,
+                                         long length) 
+    throws IOException, UnresolvedLinkException;
 
   /**
    * Get server default values for a number of configuration params.
@@ -122,15 +123,16 @@ public interface ClientProtocol extends VersionedProtocol {
    * @throws QuotaExceededException if the file creation violates 
    *                                any quota restriction
    * @throws IOException if other errors occur.
+   * @throws UnresolvedLinkException if the path contains a symlink. 
    */
   public void create(String src, 
                      FsPermission masked,
-                             String clientName, 
-                             EnumSetWritable<CreateFlag> flag, 
-                             boolean createParent,
-                             short replication,
-                             long blockSize
-                             ) throws IOException;
+                     String clientName, 
+                     EnumSetWritable<CreateFlag> flag, 
+                     boolean createParent,
+                     short replication,
+                     long blockSize)
+      throws IOException, UnresolvedLinkException;
 
   /**
    * Append to the end of the file. 
@@ -144,8 +146,10 @@ public interface ClientProtocol extends VersionedProtocol {
    * configured with the parameter dfs.support.append set to true, otherwise
    * throws an IOException.
    * @throws IOException if other errors occur.
+   * @throws UnresolvedLinkException if the path contains a symlink. 
    */
-  public LocatedBlock append(String src, String clientName) throws IOException;
+  public LocatedBlock append(String src, String clientName) 
+      throws IOException, UnresolvedLinkException;
 
   /**
    * Set replication for an existing file.
@@ -160,16 +164,17 @@ public interface ClientProtocol extends VersionedProtocol {
    * @throws IOException
    * @return true if successful;
    *         false if file does not exist or is a directory
+   * @throws UnresolvedLinkException if the path contains a symlink. 
    */
-  public boolean setReplication(String src, 
-                                short replication
-                                ) throws IOException;
+  public boolean setReplication(String src, short replication) 
+      throws IOException, UnresolvedLinkException;
 
   /**
    * Set permissions for an existing file/directory.
+   * @throws UnresolvedLinkException if the path contains a symlink. 
    */
-  public void setPermission(String src, FsPermission permission
-      ) throws IOException;
+  public void setPermission(String src, FsPermission permission)
+      throws IOException, UnresolvedLinkException;
 
   /**
    * Set owner of a path (i.e. a file or a directory).
@@ -177,18 +182,19 @@ public interface ClientProtocol extends VersionedProtocol {
    * @param src
    * @param username If it is null, the original username remains unchanged.
    * @param groupname If it is null, the original groupname remains unchanged.
+   * @throws UnresolvedLinkException if the path contains a symlink. 
    */
-  public void setOwner(String src, String username, String groupname
-      ) throws IOException;
+  public void setOwner(String src, String username, String groupname)
+      throws IOException, UnresolvedLinkException;
 
   /**
-   * The client can give up on a blcok by calling abandonBlock().
-   * The client can then
-   * either obtain a new block, or complete or abandon the file.
-   * Any partial writes to the block will be discarded.
+   * The client can give up on a block by calling abandonBlock().
+   * The client can then either obtain a new block, or complete or 
+   * abandon the file. Any partial writes to the block will be discarded.
+   * @throws UnresolvedLinkException if the path contains a symlink. 
    */
-  public void abandonBlock(Block b, String src, String holder
-      ) throws IOException;
+  public void abandonBlock(Block b, String src, String holder)
+      throws IOException, UnresolvedLinkException;
 
   /**
    * A client that wants to write an additional block to the 
@@ -208,9 +214,11 @@ public interface ClientProtocol extends VersionedProtocol {
    * @param excludedNodes a list of nodes that should not be
    * allocated for the current block
    * @return LocatedBlock allocated block information.
+   * @throws UnresolvedLinkException if the path contains a symlink. 
    */
   public LocatedBlock addBlock(String src, String clientName,
-      Block previous, DatanodeInfo[] excludedNodes) throws IOException;
+      Block previous, DatanodeInfo[] excludedNodes) 
+      throws IOException, UnresolvedLinkException;
 
   /**
    * The client is done writing data to the given filename, and would 
@@ -227,9 +235,11 @@ public interface ClientProtocol extends VersionedProtocol {
    * blocks have been replicated the minimum number of times.  Thus,
    * DataNode failures may cause a client to call complete() several
    * times before succeeding.
+   * @throws UnresolvedLinkException if the path contains a symlink. 
    */
-  public boolean complete(String src, String clientName,
-                          Block last) throws IOException;
+  public boolean complete(String src, String clientName, Block last) 
+    throws IOException, UnresolvedLinkException;
+                          
 
   /**
    * The client wants to report corrupted blocks (blocks with specified
@@ -248,23 +258,27 @@ public interface ClientProtocol extends VersionedProtocol {
    * @return true if successful, or false if the old name does not exist
    * or if the new name already belongs to the namespace.
    * @throws IOException if the new name is invalid.
+   * @throws UnresolvedLinkException if the path contains a symlink. 
    * @throws QuotaExceededException if the rename would violate 
    *                                any quota restriction
    * @deprecated Use {@link #rename(String, String, Options.Rename...)} instead.
    */
   @Deprecated
-  public boolean rename(String src, String dst) throws IOException;
+  public boolean rename(String src, String dst) 
+      throws IOException, UnresolvedLinkException;
 
   /**
-   * moves blocks from srcs to trg and delete srcs
+   * Moves blocks from srcs to trg and delete srcs
    * 
    * @param trg existing file
    * @param srcs - list of existing files (same block size, same replication)
    * @throws IOException if some arguments are invalid
+   * @throws UnresolvedLinkException if the path contains a symlink. 
    * @throws QuotaExceededException if the rename would violate 
    *                                any quota restriction
    */
-  public void concat(String trg, String [] srcs) throws IOException;
+  public void concat(String trg, String[] srcs) 
+      throws IOException, UnresolvedLinkException;
 
   /**
    * Rename src to dst.
@@ -284,9 +298,10 @@ public interface ClientProtocol extends VersionedProtocol {
    * @param dst new name.
    * @param options Rename options
    * @throws IOException if rename failed
+   * @throws UnresolvedLinkException if the path contains a symlink. 
    */
   public void rename(String src, String dst, Options.Rename... options)
-      throws IOException;
+      throws IOException, UnresolvedLinkException;
   
   /**
    * Delete the given file or directory from the file system.
@@ -296,10 +311,12 @@ public interface ClientProtocol extends VersionedProtocol {
    * @param src existing name.
    * @return true only if the existing file or directory was actually removed 
    * from the file system. 
+   * @throws UnresolvedLinkException if the path contains a symlink. 
    * @deprecated use {@link #delete(String, boolean)} istead.
    */
   @Deprecated
-  public boolean delete(String src) throws IOException;
+  public boolean delete(String src) 
+      throws IOException, UnresolvedLinkException;
 
   /**
    * Delete the given file or directory from the file system.
@@ -311,8 +328,10 @@ public interface ClientProtocol extends VersionedProtocol {
    * else throws an exception.
    * @return true only if the existing file or directory was actually removed 
    * from the file system. 
+   * @throws UnresolvedLinkException if the path contains a symlink. 
    */
-  public boolean delete(String src, boolean recursive) throws IOException;
+  public boolean delete(String src, boolean recursive) 
+     throws IOException, UnresolvedLinkException;
   
   /**
    * Create a directory (or hierarchy of directories) with the given
@@ -322,6 +341,7 @@ public interface ClientProtocol extends VersionedProtocol {
    * @param masked The masked permission of the directory being created
    * @param createParent create missing parent directory if true
    * @return True if the operation success.
+   * @throws UnresolvedLinkException if the path contains a symlink. 
    * @throws {@link AccessControlException} if permission to create file is 
    * denied by the system. As usually on the client side the exception will 
    * be wraped into {@link org.apache.hadoop.ipc.RemoteException}.
@@ -329,12 +349,14 @@ public interface ClientProtocol extends VersionedProtocol {
    *                                any quota restriction.
    */
   public boolean mkdirs(String src, FsPermission masked, boolean createParent)
-      throws IOException;
+      throws IOException, UnresolvedLinkException;
 
   /**
-   * Get a listing of the indicated directory
+   * Get a listing of the indicated directory.
+   * @throws UnresolvedLinkException if the path contains a symlink. 
    */
-  public HdfsFileStatus[] getListing(String src) throws IOException;
+  public HdfsFileStatus[] getListing(String src) 
+      throws IOException, UnresolvedLinkException; 
 
   ///////////////////////////////////////
   // System issues and management
@@ -355,8 +377,10 @@ public interface ClientProtocol extends VersionedProtocol {
    * renewLease().  If a certain amount of time passes since
    * the last call to renewLease(), the NameNode assumes the
    * client has died.
+   * @throws UnresolvedLinkException if the path contains a symlink. 
    */
-  public void renewLease(String clientName) throws IOException;
+  public void renewLease(String clientName) 
+      throws IOException, UnresolvedLinkException;
 
   public int GET_STATS_CAPACITY_IDX = 0;
   public int GET_STATS_USED_IDX = 1;
@@ -388,15 +412,17 @@ public interface ClientProtocol extends VersionedProtocol {
    * otherwise all datanodes if type is ALL.
    */
   public DatanodeInfo[] getDatanodeReport(FSConstants.DatanodeReportType type)
-  throws IOException;
+      throws IOException;
 
   /**
    * Get the block size for the given file.
    * @param filename The name of the file
    * @return The number of bytes in each block
    * @throws IOException
+   * @throws UnresolvedLinkException if the path contains a symlink. 
    */
-  public long getPreferredBlockSize(String filename) throws IOException;
+  public long getPreferredBlockSize(String filename) 
+      throws IOException, UnresolvedLinkException;
 
   /**
    * Enter, leave or get safe mode.
@@ -495,10 +521,10 @@ public interface ClientProtocol extends VersionedProtocol {
    * @throws IOException
    */
   public UpgradeStatusReport distributedUpgradeProgress(UpgradeAction action) 
-  throws IOException;
+      throws IOException;
 
   /**
-   * Dumps namenode data structures into specified file. If file
+   * Dumps namenode data structures into specified file. If the file
    * already exists, then append.
    * @throws IOException
    */
@@ -507,17 +533,33 @@ public interface ClientProtocol extends VersionedProtocol {
   /**
    * Get the file info for a specific file or directory.
    * @param src The string representation of the path to the file
-   * @throws IOException if permission to access file is denied by the system 
+   * @throws UnresolvedLinkException if the path contains symlinks;
+   *         IOException if permission to access file is denied by the system
    * @return object containing information regarding the file
    *         or null if file not found
    */
-  public HdfsFileStatus getFileInfo(String src) throws IOException;
+  public HdfsFileStatus getFileInfo(String src) 
+      throws IOException, UnresolvedLinkException;
 
+  /**
+   * Get the file info for a specific file or directory. If the path 
+   * refers to a symlink then the FileStatus of the symlink is returned.
+   * @param src The string representation of the path to the file
+   * @throws UnresolvedLinkException if the path contains symlinks;
+   *         IOException if permission to access file is denied by the system
+   * @return object containing information regarding the file
+   *         or null if file not found
+   */
+  public HdfsFileStatus getFileLinkInfo(String src) 
+      throws IOException, UnresolvedLinkException;
+  
   /**
    * Get {@link ContentSummary} rooted at the specified directory.
    * @param path The string representation of the path
+   * @throws UnresolvedLinkException if the path contains a symlink. 
    */
-  public ContentSummary getContentSummary(String path) throws IOException;
+  public ContentSummary getContentSummary(String path) 
+      throws IOException, UnresolvedLinkException;
 
   /**
    * Set the quota for a directory.
@@ -532,22 +574,24 @@ public interface ClientProtocol extends VersionedProtocol {
    * the quota to that value, (2) {@link FSConstants#QUOTA_DONT_SET}  implies 
    * the quota will not be changed, and (3) {@link FSConstants#QUOTA_RESET} 
    * implies the quota will be reset. Any other value is a runtime error.
-   *                        
+   * @throws UnresolvedLinkException if the path contains a symlink. 
    * @throws FileNotFoundException if the path is a file or 
    *                               does not exist 
    * @throws QuotaExceededException if the directory size 
    *                                is greater than the given quota
    */
   public void setQuota(String path, long namespaceQuota, long diskspaceQuota)
-                      throws IOException;
+      throws IOException, UnresolvedLinkException;
   
   /**
    * Write all metadata for this file into persistent storage.
    * The file must be currently open for writing.
    * @param src The string representation of the path
    * @param client The string representation of the client
+   * @throws UnresolvedLinkException if the path contains a symlink. 
    */
-  public void fsync(String src, String client) throws IOException;
+  public void fsync(String src, String client) 
+      throws IOException, UnresolvedLinkException;
 
   /**
    * Sets the modification and access time of the file to the specified time.
@@ -558,8 +602,32 @@ public interface ClientProtocol extends VersionedProtocol {
    * @param atime The number of milliseconds since Jan 1, 1970.
    *              Setting atime to -1 means that access time should not be set
    *              by this call.
+   * @throws UnresolvedLinkException if the path contains a symlink. 
    */
-  public void setTimes(String src, long mtime, long atime) throws IOException;
+  public void setTimes(String src, long mtime, long atime) 
+      throws IOException, UnresolvedLinkException;
+
+  /**
+   * Create a symbolic link to a file or directory.
+   * @param target The pathname of the destination that the
+   *               link points to.
+   * @param link The pathname of the link being created.
+   * @param dirPerm permissions to use when creating parent directories
+   * @param createParent - if true then missing parent dirs are created
+   *                       if false then parent must exist
+   * @throws IOException
+   * @throws UnresolvedLinkException if the path contains a symlink. 
+   */
+  public void createSymlink(String target, String link, FsPermission dirPerm, 
+      boolean createParent) throws IOException, UnresolvedLinkException;
+
+  /**
+   * Resolve the first symbolic link on the specified path.
+   * @param path The pathname that needs to be resolved
+   * @return The pathname after resolving the first symbolic link if any.
+   * @throws IOException
+   */
+  public String getLinkTarget(String path) throws IOException; 
   
   /**
    * Get a new generation stamp together with an access token for 
@@ -574,7 +642,7 @@ public interface ClientProtocol extends VersionedProtocol {
    * @throws IOException if any error occurs
    */
   public LocatedBlock updateBlockForPipeline(Block block, String clientName) 
-  throws IOException;
+      throws IOException;
 
   /**
    * Update a pipeline for a block under construction
@@ -587,7 +655,7 @@ public interface ClientProtocol extends VersionedProtocol {
    */
   public void updatePipeline(String clientName, Block oldBlock, 
       Block newBlock, DatanodeID[] newNodes)
-  throws IOException;
+      throws IOException;
 
   /**
    * Get a valid Delegation Token.
@@ -596,7 +664,8 @@ public interface ClientProtocol extends VersionedProtocol {
    * @return Token<DelegationTokenIdentifier>
    * @throws IOException
    */
-  public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer) throws IOException;
+  public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer) 
+      throws IOException;
 
   /**
    * Renew an existing delegation token.

+ 2 - 2
src/java/org/apache/hadoop/hdfs/protocol/FSConstants.java

@@ -91,7 +91,7 @@ public interface FSConstants {
   // Version is reflected in the data storage file.
   // Versions are negative.
   // Decrement LAYOUT_VERSION to define a new version.
-  public static final int LAYOUT_VERSION = -22;
+  public static final int LAYOUT_VERSION = -23;
   // Current version: 
-  // -22: added new OP_CONCAT_DELETE 
+  // -23: Symbolic links
 }

+ 32 - 2
src/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java

@@ -32,6 +32,7 @@ import org.apache.hadoop.io.Writable;
 public class HdfsFileStatus implements Writable {
 
   private byte[] path;  // local name of the inode that's encoded in java UTF8
+  private byte[] symlink; // symlink target encoded in java UTF8
   private long length;
   private boolean isdir;
   private short block_replication;
@@ -47,7 +48,9 @@ public class HdfsFileStatus implements Writable {
   /**
    * default constructor
    */
-  public HdfsFileStatus() { this(0, false, 0, 0, 0, 0, null, null, null, null); }
+  public HdfsFileStatus() { 
+    this(0, false, 0, 0, 0, 0, null, null, null, null, null); 
+  }
   
   /**
    * Constructor
@@ -65,7 +68,7 @@ public class HdfsFileStatus implements Writable {
   public HdfsFileStatus(long length, boolean isdir, int block_replication,
                     long blocksize, long modification_time, long access_time,
                     FsPermission permission, String owner, String group, 
-                    byte[] path) {
+                    byte[] symlink, byte[] path) {
     this.length = length;
     this.isdir = isdir;
     this.block_replication = (short)block_replication;
@@ -76,6 +79,7 @@ public class HdfsFileStatus implements Writable {
                       FsPermission.getDefault() : permission;
     this.owner = (owner == null) ? "" : owner;
     this.group = (group == null) ? "" : group;
+    this.symlink = symlink;
     this.path = path;
   }
 
@@ -95,6 +99,14 @@ public class HdfsFileStatus implements Writable {
     return isdir;
   }
 
+  /**
+   * Is this a symbolic link?
+   * @return true if this is a symbolic link
+   */
+  public boolean isSymlink() {
+    return symlink != null;
+  }
+  
   /**
    * Get the block size of the file.
    * @return the number of bytes
@@ -198,6 +210,14 @@ public class HdfsFileStatus implements Writable {
     return new Path(parent, getLocalName());
   }
 
+  /**
+   * Get the string representation of the symlink.
+   * @return the symlink as a string.
+   */
+  final public String getSymlink() {
+    return DFSUtil.bytes2String(symlink);
+  }
+
   //////////////////////////////////////////////////
   // Writable
   //////////////////////////////////////////////////
@@ -213,6 +233,11 @@ public class HdfsFileStatus implements Writable {
     permission.write(out);
     Text.writeString(out, owner);
     Text.writeString(out, group);
+    out.writeBoolean(isSymlink());
+    if (isSymlink()) {
+      out.writeInt(symlink.length);
+      out.write(symlink);
+    }
   }
 
   public void readFields(DataInput in) throws IOException {
@@ -232,5 +257,10 @@ public class HdfsFileStatus implements Writable {
     permission.readFields(in);
     owner = Text.readString(in);
     group = Text.readString(in);
+    if (in.readBoolean()) {
+      numOfBytes = in.readInt();
+      this.symlink = new byte[numOfBytes];
+      in.readFully(symlink);
+    }
   }
 }

+ 74 - 0
src/java/org/apache/hadoop/hdfs/protocol/UnresolvedPathException.java

@@ -0,0 +1,74 @@
+/**
+ * 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.hdfs.protocol;
+
+import java.io.IOException;
+import org.apache.hadoop.fs.UnresolvedLinkException;
+import org.apache.hadoop.fs.Path;
+
+/** 
+ * Thrown when a symbolic link is encountered in a path.
+ */
+public final class UnresolvedPathException extends UnresolvedLinkException {
+  private static final long serialVersionUID = 1L;
+  private String originalPath;  // The original path containing the link
+  private String linkTarget;    // The target of the link 
+  private String remainingPath; // The path part following the link
+  
+
+  /**
+   * Used by RemoteException to instantiate an UnresolvedPathException.
+   */
+  public UnresolvedPathException(String msg) {
+    super(msg);
+  }
+  
+  public UnresolvedPathException(String originalPath, String remainingPath, 
+      String linkTarget) {
+    this.originalPath  = originalPath;
+    this.remainingPath = remainingPath;
+    this.linkTarget    = linkTarget;
+  }
+
+  public Path getUnresolvedPath() throws IOException {
+    return new Path(originalPath);
+  }
+  
+  public Path getResolvedPath() throws IOException {
+    if (remainingPath == null || "".equals(remainingPath)) {
+      return new Path(linkTarget);
+    }
+    return new Path(linkTarget, remainingPath);
+  }
+
+  @Override
+  public String getMessage() {
+    String msg = super.getMessage();
+    if (msg != null) {
+      return msg;
+    }
+    String myMsg = "Unresolved path " + originalPath;
+    try {
+      return getResolvedPath().toString();
+    } catch (IOException e) {
+      // Ignore
+    }
+    return myMsg;
+  }
+}

+ 209 - 101
src/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -29,6 +29,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
+import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Options.Rename;
@@ -46,6 +47,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.metrics.MetricsContext;
 import org.apache.hadoop.metrics.MetricsRecord;
 import org.apache.hadoop.metrics.MetricsUtil;
+import org.apache.hadoop.security.UserGroupInformation;
 
 /*************************************************
  * FSDirectory stores the filesystem directory state.
@@ -61,9 +63,9 @@ class FSDirectory implements Closeable {
   INodeDirectoryWithQuota rootDir;
   FSImage fsImage;  
   private volatile boolean ready = false;
-  // Metrics record
   private MetricsRecord directoryMetrics = null;
-
+  private static final long UNKNOWN_DISK_SPACE = -1;
+  
   /** Access an existing dfs name directory. */
   FSDirectory(FSNamesystem ns, Configuration conf) {
     this(new FSImage(), ns, conf);
@@ -80,7 +82,7 @@ class FSDirectory implements Closeable {
     fsImage.setFSNamesystem(ns);
     rootDir = new INodeDirectoryWithQuota(INodeDirectory.ROOT_NAME,
         ns.createFsOwnerPermissions(new FsPermission((short)0755)),
-        Integer.MAX_VALUE, -1);
+        Integer.MAX_VALUE, UNKNOWN_DISK_SPACE);
     this.fsImage = fsImage;
     initialize(conf);
   }
@@ -101,7 +103,8 @@ class FSDirectory implements Closeable {
 
   void loadFSImage(Collection<URI> dataDirs,
                    Collection<URI> editsDirs,
-                   StartupOption startOpt) throws IOException {
+                   StartupOption startOpt) 
+      throws IOException {
     // format before starting up if requested
     if (startOpt == StartupOption.FORMAT) {
       fsImage.setStorageDirectories(dataDirs, editsDirs);
@@ -164,7 +167,7 @@ class FSDirectory implements Closeable {
                 String clientMachine,
                 DatanodeDescriptor clientNode,
                 long generationStamp) 
-                throws IOException {
+                throws IOException, UnresolvedLinkException {
     waitForReady();
 
     // Always do an implicit mkdirs for parent directory tree.
@@ -178,7 +181,7 @@ class FSDirectory implements Closeable {
                                  preferredBlockSize, modTime, clientName, 
                                  clientMachine, clientNode);
     synchronized (rootDir) {
-      newNode = addNode(path, newNode, -1, false);
+      newNode = addNode(path, newNode, UNKNOWN_DISK_SPACE, false);
     }
     if (newNode == null) {
       NameNode.stateChangeLog.info("DIR* FSDirectory.addFile: "
@@ -202,9 +205,10 @@ class FSDirectory implements Closeable {
                             short replication,
                             long modificationTime,
                             long atime,
-                            long preferredBlockSize) {
+                            long preferredBlockSize) 
+      throws UnresolvedLinkException {
     INode newNode;
-    long diskspace = -1; // unknown
+    long diskspace = UNKNOWN_DISK_SPACE;
     if (blocks == null)
       newNode = new INodeDirectory(permissions, modificationTime);
     else {
@@ -234,12 +238,14 @@ class FSDirectory implements Closeable {
                               INodeDirectory parentINode,
                               PermissionStatus permissions,
                               Block[] blocks, 
+                              String symlink,
                               short replication,
                               long modificationTime,
                               long atime,
                               long nsQuota,
                               long dsQuota,
-                              long preferredBlockSize) {
+                              long preferredBlockSize) 
+                              throws UnresolvedLinkException {
     // NOTE: This does not update space counts for parents
     // create new inode
     INode newNode;
@@ -250,9 +256,15 @@ class FSDirectory implements Closeable {
       } else {
         newNode = new INodeDirectory(permissions, modificationTime);
       }
-    } else 
-      newNode = new INodeFile(permissions, blocks.length, replication,
-                              modificationTime, atime, preferredBlockSize);
+    } else  {
+      if (symlink.length() != 0) {
+        newNode = new INodeSymlink(symlink, modificationTime, atime, permissions);
+        ((INodeSymlink)newNode).setLinkValue(symlink);
+      } else {
+        newNode = new INodeFile(permissions, blocks.length, replication,
+                                modificationTime, atime, preferredBlockSize);
+      }
+    }
     // add new node to the parent
     INodeDirectory newParent = null;
     synchronized (rootDir) {
@@ -266,6 +278,7 @@ class FSDirectory implements Closeable {
       if(blocks != null) {
         int nrBlocks = blocks.length;
         // Add file->block mapping
+        assert !newNode.isLink();
         INodeFile newF = (INodeFile)newNode;
         for (int i = 0; i < nrBlocks; i++) {
           BlockInfo blockInfo = new BlockInfo(blocks[i], newF.getReplication());
@@ -374,7 +387,8 @@ class FSDirectory implements Closeable {
    * @deprecated Use {@link #renameTo(String, String, Rename...)} instead.
    */
   @Deprecated
-  boolean renameTo(String src, String dst) throws QuotaExceededException {
+  boolean renameTo(String src, String dst) 
+      throws QuotaExceededException, UnresolvedLinkException {
     if (NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("DIR* FSDirectory.renameTo: "
                                   +src+" to "+dst);
@@ -391,7 +405,7 @@ class FSDirectory implements Closeable {
    * @see #unprotectedRenameTo(String, String, long, Options.Rename...)
    */
   void renameTo(String src, String dst, Options.Rename... options)
-      throws IOException {
+      throws IOException, UnresolvedLinkException {
     if (NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("DIR* FSDirectory.renameTo: " + src
           + " to " + dst);
@@ -413,9 +427,9 @@ class FSDirectory implements Closeable {
    */
   @Deprecated
   boolean unprotectedRenameTo(String src, String dst, long timestamp)
-      throws QuotaExceededException {
+    throws QuotaExceededException, UnresolvedLinkException {
     synchronized (rootDir) {
-      INode[] srcInodes = rootDir.getExistingPathINodes(src);
+      INode[] srcInodes = rootDir.getExistingPathINodes(src, false);
 
       // check the validation of the source
       if (srcInodes[srcInodes.length-1] == null) {
@@ -448,7 +462,7 @@ class FSDirectory implements Closeable {
       
       byte[][] dstComponents = INode.getPathComponents(dst);
       INode[] dstInodes = new INode[dstComponents.length];
-      rootDir.getExistingPathINodes(dstComponents, dstInodes);
+      rootDir.getExistingPathINodes(dstComponents, dstInodes, false);
       if (dstInodes[dstInodes.length-1] != null) {
         NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: "
                                      +"failed to rename "+src+" to "+dst+ 
@@ -482,7 +496,7 @@ class FSDirectory implements Closeable {
         
         // add src to the destination
         dstChild = addChildNoQuotaCheck(dstInodes, dstInodes.length - 1,
-            srcChild, -1, false);
+            srcChild, UNKNOWN_DISK_SPACE, false);
         if (dstChild != null) {
           srcChild = null;
           if (NameNode.stateChangeLog.isDebugEnabled()) {
@@ -498,8 +512,8 @@ class FSDirectory implements Closeable {
         if (dstChild == null && srcChild != null) {
           // put it back
           srcChild.setLocalName(srcChildName);
-          addChildNoQuotaCheck(srcInodes, srcInodes.length - 1, srcChild, -1,
-              false);
+          addChildNoQuotaCheck(srcInodes, srcInodes.length - 1, srcChild, 
+              UNKNOWN_DISK_SPACE, false);
         }
       }
       NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: "
@@ -520,7 +534,7 @@ class FSDirectory implements Closeable {
    * @throws IOException if the operation violates any quota limit
    */
   void unprotectedRenameTo(String src, String dst, long timestamp,
-      Options.Rename... options) throws IOException {
+      Options.Rename... options) throws IOException, UnresolvedLinkException {
     boolean overwrite = false;
     if (null != options) {
       for (Rename option : options) {
@@ -531,7 +545,7 @@ class FSDirectory implements Closeable {
     }
     String error = null;
     synchronized (rootDir) {
-      final INode[] srcInodes = rootDir.getExistingPathINodes(src);
+      final INode[] srcInodes = rootDir.getExistingPathINodes(src, false);
       final INode srcInode = srcInodes[srcInodes.length - 1];
       // validate source
       if (srcInode == null) {
@@ -562,7 +576,7 @@ class FSDirectory implements Closeable {
       }
       final byte[][] dstComponents = INode.getPathComponents(dst);
       final INode[] dstInodes = new INode[dstComponents.length];
-      rootDir.getExistingPathINodes(dstComponents, dstInodes);
+      rootDir.getExistingPathINodes(dstComponents, dstInodes, false);
       INode dstInode = dstInodes[dstInodes.length - 1];
       if (dstInodes.length == 1) {
         error = "rename destination cannot be the root";
@@ -630,7 +644,7 @@ class FSDirectory implements Closeable {
         removedSrc.setLocalName(dstComponents[dstInodes.length - 1]);
         // add src as dst to complete rename
         dstChild = addChildNoQuotaCheck(dstInodes, dstInodes.length - 1,
-            removedSrc, -1, false);
+            removedSrc, UNKNOWN_DISK_SPACE, false);
 
         if (dstChild != null) {
           removedSrc = null;
@@ -657,14 +671,14 @@ class FSDirectory implements Closeable {
         if (removedSrc != null) {
           // Rename failed - restore src
           removedSrc.setLocalName(srcChildName);
-          addChildNoQuotaCheck(srcInodes, srcInodes.length - 1, removedSrc, -1,
-              false);
+          addChildNoQuotaCheck(srcInodes, srcInodes.length - 1, removedSrc, 
+              UNKNOWN_DISK_SPACE, false);
         }
         if (removedDst != null) {
           // Rename failed - restore dst
           removedDst.setLocalName(dstChildName);
-          addChildNoQuotaCheck(dstInodes, dstInodes.length - 1, removedDst, -1,
-              false);
+          addChildNoQuotaCheck(dstInodes, dstInodes.length - 1, removedDst, 
+              UNKNOWN_DISK_SPACE, false);
         }
       }
       NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: "
@@ -685,7 +699,8 @@ class FSDirectory implements Closeable {
   Block[] setReplication(String src, 
                          short replication,
                          int[] oldReplication
-                         ) throws QuotaExceededException {
+                         ) throws QuotaExceededException, 
+                         UnresolvedLinkException {
     waitForReady();
     Block[] fileBlocks = unprotectedSetReplication(src, replication, oldReplication);
     if (fileBlocks != null)  // log replication change
@@ -693,21 +708,26 @@ class FSDirectory implements Closeable {
     return fileBlocks;
   }
 
-  Block[] unprotectedSetReplication( String src, 
-                                     short replication,
-                                     int[] oldReplication
-                                     ) throws QuotaExceededException {
-    if (oldReplication == null)
+  Block[] unprotectedSetReplication(String src, 
+                                    short replication,
+                                    int[] oldReplication
+                                    ) throws QuotaExceededException, 
+                                    UnresolvedLinkException {
+    if (oldReplication == null) {
       oldReplication = new int[1];
+    }
     oldReplication[0] = -1;
     Block[] fileBlocks = null;
     synchronized(rootDir) {
-      INode[] inodes = rootDir.getExistingPathINodes(src);
+      INode[] inodes = rootDir.getExistingPathINodes(src, true);
       INode inode = inodes[inodes.length - 1];
-      if (inode == null)
+      if (inode == null) {
         return null;
-      if (inode.isDirectory())
+      }
+      assert !inode.isLink();
+      if (inode.isDirectory()) {
         return null;
+      }
       INodeFile fileNode = (INodeFile)inode;
       oldReplication[0] = fileNode.getReplication();
 
@@ -728,59 +748,63 @@ class FSDirectory implements Closeable {
    * @return the number of bytes 
    * @throws IOException if it is a directory or does not exist.
    */
-  long getPreferredBlockSize(String filename) throws IOException {
+  long getPreferredBlockSize(String filename) 
+      throws IOException, UnresolvedLinkException {
     synchronized (rootDir) {
-      INode fileNode = rootDir.getNode(filename);
-      if (fileNode == null) {
+      INode inode = rootDir.getNode(filename, false);
+      if (inode == null) {
         throw new FileNotFoundException("File does not exist: " + filename);
       }
-      if (fileNode.isDirectory()) {
-        throw new IOException("Getting block size of a directory: " + 
-                              filename);
+      if (inode.isDirectory() || inode.isLink()) {
+        throw new IOException("Getting block size of non-file: "+ filename); 
       }
-      return ((INodeFile)fileNode).getPreferredBlockSize();
+      return ((INodeFile)inode).getPreferredBlockSize();
     }
   }
 
-  boolean exists(String src) {
+  boolean exists(String src) throws UnresolvedLinkException {
     src = normalizePath(src);
     synchronized(rootDir) {
-      INode inode = rootDir.getNode(src);
+      INode inode = rootDir.getNode(src, false);
       if (inode == null) {
          return false;
       }
-      return inode.isDirectory()? true: ((INodeFile)inode).getBlocks() != null;
+      return inode.isDirectory() || inode.isLink() 
+        ? true 
+        : ((INodeFile)inode).getBlocks() != null;
     }
   }
 
   void setPermission(String src, FsPermission permission
-      ) throws FileNotFoundException {
+      ) throws FileNotFoundException, UnresolvedLinkException {
     unprotectedSetPermission(src, permission);
     fsImage.getEditLog().logSetPermissions(src, permission);
   }
 
-  void unprotectedSetPermission(String src, FsPermission permissions
-      ) throws FileNotFoundException {
+  void unprotectedSetPermission(String src, FsPermission permissions) 
+    throws FileNotFoundException, UnresolvedLinkException {
     synchronized(rootDir) {
-        INode inode = rootDir.getNode(src);
-        if(inode == null)
+        INode inode = rootDir.getNode(src, true);
+        if (inode == null) {
             throw new FileNotFoundException("File does not exist: " + src);
+        }
         inode.setPermission(permissions);
     }
   }
 
   void setOwner(String src, String username, String groupname
-      ) throws FileNotFoundException {
+      ) throws FileNotFoundException, UnresolvedLinkException {
     unprotectedSetOwner(src, username, groupname);
     fsImage.getEditLog().logSetOwner(src, username, groupname);
   }
 
-  void unprotectedSetOwner(String src, String username, String groupname
-      ) throws FileNotFoundException {
+  void unprotectedSetOwner(String src, String username, String groupname) 
+    throws FileNotFoundException, UnresolvedLinkException {
     synchronized(rootDir) {
-      INode inode = rootDir.getNode(src);
-      if(inode == null)
+      INode inode = rootDir.getNode(src, true);
+      if (inode == null) {
           throw new FileNotFoundException("File does not exist: " + src);
+      }
       if (username != null) {
         inode.setUser(username);
       }
@@ -796,7 +820,8 @@ class FSDirectory implements Closeable {
    * @param srcs
    * @throws IOException
    */
-  public void concatInternal(String target, String [] srcs) throws IOException{
+  public void concatInternal(String target, String [] srcs) 
+      throws IOException, UnresolvedLinkException {
     synchronized(rootDir) {
       // actual move
       waitForReady();
@@ -817,7 +842,8 @@ class FSDirectory implements Closeable {
    * Must be public because also called from EditLogs
    * NOTE: - it does not update quota (not needed for concat)
    */
-  public void unprotectedConcat(String target, String [] srcs) throws IOException {
+  public void unprotectedConcat(String target, String [] srcs) 
+      throws IOException, UnresolvedLinkException {
     if (NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("DIR* FSNamesystem.concat to "+target);
     }
@@ -861,7 +887,8 @@ class FSDirectory implements Closeable {
    * @param collectedBlocks Blocks under the deleted directory
    * @return true on successful deletion; else false
    */
-  boolean delete(String src, List<Block>collectedBlocks) {
+  boolean delete(String src, List<Block>collectedBlocks) 
+    throws UnresolvedLinkException {
     if (NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("DIR* FSDirectory.delete: " + src);
     }
@@ -878,13 +905,13 @@ class FSDirectory implements Closeable {
   }
   
   /** Return if a directory is empty or not **/
-  boolean isDirEmpty(String src) {
+  boolean isDirEmpty(String src) throws UnresolvedLinkException {
 	   boolean dirNotEmpty = true;
     if (!isDir(src)) {
       return true;
     }
     synchronized(rootDir) {
-      INode targetNode = rootDir.getNode(src);
+      INode targetNode = rootDir.getNode(src, false);
       assert targetNode != null : "should be taken care in isDir() above";
       if (((INodeDirectory)targetNode).getChildren().size() != 0) {
         dirNotEmpty = false;
@@ -894,7 +921,13 @@ class FSDirectory implements Closeable {
   }
 
   boolean isEmpty() {
-    return isDirEmpty("/");
+    try {
+      return isDirEmpty("/");
+    } catch (UnresolvedLinkException e) {
+      NameNode.stateChangeLog.debug("/ cannot be a symlink");
+      assert false : "/ cannot be a symlink";
+      return true;
+    }
   }
 
   /**
@@ -907,7 +940,8 @@ class FSDirectory implements Closeable {
    * @param mtime the time the inode is removed
    * @return deleted inode if deletion succeeds; else null
    */ 
-  INode unprotectedDelete(String src, long mtime) {
+  INode unprotectedDelete(String src, long mtime) 
+    throws UnresolvedLinkException {
     List<Block> collectedBlocks = new ArrayList<Block>();
     INode removedNode = unprotectedDelete(src, collectedBlocks, mtime);
     getFSNamesystem().removePathAndBlocks(src, collectedBlocks);
@@ -923,11 +957,11 @@ class FSDirectory implements Closeable {
    * @return deleted inode if deletion succeeds; else null
    */ 
   INode unprotectedDelete(String src, List<Block> collectedBlocks, 
-      long mtime) {
+      long mtime) throws UnresolvedLinkException {
     src = normalizePath(src);
 
     synchronized (rootDir) {
-      INode[] inodes =  rootDir.getExistingPathINodes(src);
+      INode[] inodes =  rootDir.getExistingPathINodes(src, false);
       INode targetNode = inodes[inodes.length-1];
 
       if (targetNode == null) { // non-existent src
@@ -963,7 +997,7 @@ class FSDirectory implements Closeable {
    * Replaces the specified inode with the specified one.
    */
   void replaceNode(String path, INodeFile oldnode, INodeFile newnode) 
-                                                   throws IOException {
+      throws IOException, UnresolvedLinkException {
     replaceNode(path, oldnode, newnode, true);
   }
   
@@ -971,7 +1005,8 @@ class FSDirectory implements Closeable {
    * @see #replaceNode(String, INodeFile, INodeFile)
    */
   private void replaceNode(String path, INodeFile oldnode, INodeFile newnode,
-                           boolean updateDiskspace) throws IOException {    
+                           boolean updateDiskspace) 
+      throws IOException, UnresolvedLinkException {    
     synchronized (rootDir) {
       long dsOld = oldnode.diskspaceConsumed();
       
@@ -1018,11 +1053,11 @@ class FSDirectory implements Closeable {
    * This function is admittedly very inefficient right now.  We'll
    * make it better later.
    */
-  HdfsFileStatus[] getListing(String src) {
+  HdfsFileStatus[] getListing(String src) throws UnresolvedLinkException {
     String srcs = normalizePath(src);
 
     synchronized (rootDir) {
-      INode targetNode = rootDir.getNode(srcs);
+      INode targetNode = rootDir.getNode(srcs, true);
       if (targetNode == null)
         return null;
       if (!targetNode.isDirectory()) {
@@ -1042,13 +1077,15 @@ class FSDirectory implements Closeable {
 
   /** Get the file info for a specific file.
    * @param src The string representation of the path to the file
+   * @param resolveLink whether to throw UnresolvedLinkException 
    * @return object containing information regarding the file
    *         or null if file not found
    */
-  HdfsFileStatus getFileInfo(String src) {
+  HdfsFileStatus getFileInfo(String src, boolean resolveLink) 
+      throws UnresolvedLinkException {
     String srcs = normalizePath(src);
     synchronized (rootDir) {
-      INode targetNode = rootDir.getNode(srcs);
+      INode targetNode = rootDir.getNode(srcs, resolveLink);
       if (targetNode == null) {
         return null;
       }
@@ -1061,13 +1098,15 @@ class FSDirectory implements Closeable {
   /**
    * Get the blocks associated with the file.
    */
-  Block[] getFileBlocks(String src) {
+  Block[] getFileBlocks(String src) throws UnresolvedLinkException {
     waitForReady();
     synchronized (rootDir) {
-      INode targetNode = rootDir.getNode(src);
+      INode targetNode = rootDir.getNode(src, false);
       if (targetNode == null)
         return null;
-      if(targetNode.isDirectory())
+      if (targetNode.isDirectory())
+        return null;
+      if (targetNode.isLink()) 
         return null;
       return ((INodeFile)targetNode).getBlocks();
     }
@@ -1076,11 +1115,12 @@ class FSDirectory implements Closeable {
   /**
    * Get {@link INode} associated with the file.
    */
-  INodeFile getFileINode(String src) {
+  INodeFile getFileINode(String src) throws UnresolvedLinkException {
     synchronized (rootDir) {
-      INode inode = rootDir.getNode(src);
+      INode inode = rootDir.getNode(src, true);
       if (inode == null || inode.isDirectory())
         return null;
+      assert !inode.isLink();      
       return (INodeFile)inode;
     }
   }
@@ -1097,21 +1137,22 @@ class FSDirectory implements Closeable {
    *         
    * @see INodeDirectory#getExistingPathINodes(byte[][], INode[])
    */
-  INode[] getExistingPathINodes(String path) {
+  INode[] getExistingPathINodes(String path) 
+    throws UnresolvedLinkException {
     synchronized (rootDir){
-      return rootDir.getExistingPathINodes(path);
+      return rootDir.getExistingPathINodes(path, true);
     }
   }
   
   /** 
    * Check whether the filepath could be created
    */
-  boolean isValidToCreate(String src) {
+  boolean isValidToCreate(String src) throws UnresolvedLinkException {
     String srcs = normalizePath(src);
     synchronized (rootDir) {
       if (srcs.startsWith("/") && 
           !srcs.endsWith("/") && 
-          rootDir.getNode(srcs) == null) {
+          rootDir.getNode(srcs, false) == null) {
         return true;
       } else {
         return false;
@@ -1122,9 +1163,9 @@ class FSDirectory implements Closeable {
   /**
    * Check whether the path specifies a directory
    */
-  boolean isDir(String src) {
+  boolean isDir(String src) throws UnresolvedLinkException {
     synchronized (rootDir) {
-      INode node = rootDir.getNode(normalizePath(src));
+      INode node = rootDir.getNode(normalizePath(src), false);
       return node != null && node.isDirectory();
     }
   }
@@ -1140,9 +1181,10 @@ class FSDirectory implements Closeable {
    */
   void updateSpaceConsumed(String path, long nsDelta, long dsDelta)
                                          throws QuotaExceededException,
-                                                FileNotFoundException {
+                                                FileNotFoundException,
+                                                UnresolvedLinkException {
     synchronized (rootDir) {
-      INode[] inodes = rootDir.getExistingPathINodes(path);
+      INode[] inodes = rootDir.getExistingPathINodes(path, false);
       int len = inodes.length;
       if (inodes[len - 1] == null) {
         throw new FileNotFoundException(path + 
@@ -1253,17 +1295,19 @@ class FSDirectory implements Closeable {
    * @throws FileNotFoundException if an ancestor or itself is a file
    * @throws QuotaExceededException if directory creation violates 
    *                                any quota limit
+   * @throws UnresolvedLinkException if a symlink is encountered in src.                      
    */
   boolean mkdirs(String src, PermissionStatus permissions,
       boolean inheritPermission, long now)
-      throws FileAlreadyExistsException, QuotaExceededException {
+      throws FileAlreadyExistsException, QuotaExceededException, 
+             UnresolvedLinkException {
     src = normalizePath(src);
     String[] names = INode.getPathNames(src);
     byte[][] components = INode.getPathComponents(names);
     INode[] inodes = new INode[components.length];
 
     synchronized(rootDir) {
-      rootDir.getExistingPathINodes(components, inodes);
+      rootDir.getExistingPathINodes(components, inodes, false);
 
       // find the index of the first null in inodes[]
       StringBuilder pathbuilder = new StringBuilder();
@@ -1272,7 +1316,7 @@ class FSDirectory implements Closeable {
         pathbuilder.append(Path.SEPARATOR + names[i]);
         if (!inodes[i].isDirectory()) {
           throw new FileAlreadyExistsException("Parent path is not a directory: "
-              + pathbuilder);
+              + pathbuilder+ " "+inodes[i].getLocalName());
         }
       }
 
@@ -1300,11 +1344,12 @@ class FSDirectory implements Closeable {
   /**
    */
   INode unprotectedMkdir(String src, PermissionStatus permissions,
-                          long timestamp) throws QuotaExceededException {
+                          long timestamp) throws QuotaExceededException,
+                          UnresolvedLinkException {
     byte[][] components = INode.getPathComponents(src);
     INode[] inodes = new INode[components.length];
     synchronized (rootDir) {
-      rootDir.getExistingPathINodes(components, inodes);
+      rootDir.getExistingPathINodes(components, inodes, false);
       unprotectedMkdir(inodes, inodes.length-1, components[inodes.length-1],
           permissions, false, timestamp);
       return inodes[inodes.length-1];
@@ -1328,12 +1373,12 @@ class FSDirectory implements Closeable {
    * QuotaExceededException is thrown if it violates quota limit */
   private <T extends INode> T addNode(String src, T child, 
         long childDiskspace, boolean inheritPermission) 
-  throws QuotaExceededException {
+  throws QuotaExceededException, UnresolvedLinkException {
     byte[][] components = INode.getPathComponents(src);
     child.setLocalName(components[components.length-1]);
     INode[] inodes = new INode[components.length];
     synchronized (rootDir) {
-      rootDir.getExistingPathINodes(components, inodes);
+      rootDir.getExistingPathINodes(components, inodes, false);
       return addChild(inodes, inodes.length-1, child, childDiskspace,
                       inheritPermission);
     }
@@ -1488,10 +1533,11 @@ class FSDirectory implements Closeable {
     return src;
   }
 
-  ContentSummary getContentSummary(String src) throws FileNotFoundException {
+  ContentSummary getContentSummary(String src) 
+    throws FileNotFoundException, UnresolvedLinkException {
     String srcs = normalizePath(src);
     synchronized (rootDir) {
-      INode targetNode = rootDir.getNode(srcs);
+      INode targetNode = rootDir.getNode(srcs, false);
       if (targetNode == null) {
         throw new FileNotFoundException("File does not exist: " + srcs);
       }
@@ -1542,6 +1588,8 @@ class FSDirectory implements Closeable {
       if (child.isDirectory()) {
         updateCountForINodeWithQuota((INodeDirectory)child, 
                                      counts, nodesInPath);
+      } else if (child.isLink()) {
+        counts.nsCount += 1;
       } else { // reduce recursive calls
         counts.nsCount += 1;
         counts.dsCount += ((INodeFile)child).diskspaceConsumed();
@@ -1585,9 +1633,11 @@ class FSDirectory implements Closeable {
    * @throws FileNotFoundException if the path does not exist or is a file
    * @throws QuotaExceededException if the directory tree size is 
    *                                greater than the given quota
+   * @throws UnresolvedLinkException if a symlink is encountered in src.
    */
-  INodeDirectory unprotectedSetQuota(String src, long nsQuota, long dsQuota) 
-                       throws FileNotFoundException, QuotaExceededException {
+  INodeDirectory unprotectedSetQuota(String src, long nsQuota, long dsQuota)
+    throws FileNotFoundException, QuotaExceededException, 
+    UnresolvedLinkException {
     // sanity check
     if ((nsQuota < 0 && nsQuota != FSConstants.QUOTA_DONT_SET && 
          nsQuota < FSConstants.QUOTA_RESET) || 
@@ -1601,7 +1651,7 @@ class FSDirectory implements Closeable {
     String srcs = normalizePath(src);
 
     synchronized(rootDir) {
-      INode[] inodes = rootDir.getExistingPathINodes(src);
+      INode[] inodes = rootDir.getExistingPathINodes(src, true);
       INode targetNode = inodes[inodes.length-1];
       if (targetNode == null) {
         throw new FileNotFoundException("Directory does not exist: " + srcs);
@@ -1641,7 +1691,8 @@ class FSDirectory implements Closeable {
    * @see #unprotectedSetQuota(String, long, long)
    */
   void setQuota(String src, long nsQuota, long dsQuota) 
-                throws FileNotFoundException, QuotaExceededException {
+    throws FileNotFoundException, QuotaExceededException,
+    UnresolvedLinkException { 
     synchronized (rootDir) {    
       INodeDirectory dir = unprotectedSetQuota(src, nsQuota, dsQuota);
       if (dir != null) {
@@ -1666,7 +1717,8 @@ class FSDirectory implements Closeable {
     }
   }
 
-  boolean unprotectedSetTimes(String src, long mtime, long atime, boolean force) {
+  boolean unprotectedSetTimes(String src, long mtime, long atime, boolean force) 
+    throws UnresolvedLinkException {
     synchronized(rootDir) {
       INodeFile inode = getFileINode(src);
       return unprotectedSetTimes(src, inode, mtime, atime, force);
@@ -1712,13 +1764,69 @@ class FSDirectory implements Closeable {
     return new HdfsFileStatus(
         node instanceof INodeFile ? ((INodeFile)node).computeFileSize(true) : 0, 
         node.isDirectory(), 
-        node.isDirectory() ? 0 : ((INodeFile)node).getReplication(), 
-        node.isDirectory() ? 0 : ((INodeFile)node).getPreferredBlockSize(),
+        (node.isDirectory() || node.isLink()) ? 0 : ((INodeFile)node).getReplication(), 
+        (node.isDirectory() || node.isLink()) ? 0 : ((INodeFile)node).getPreferredBlockSize(),
         node.getModificationTime(),
         node.getAccessTime(),
         node.getFsPermission(),
         node.getUserName(),
         node.getGroupName(),
+        node.isLink() ? ((INodeSymlink)node).getSymlink() : null,
         path);
   }
+
+  /**
+   * Add the given symbolic link to the fs. Record it in the edits log.
+   */
+  INodeSymlink addSymlink(String path, String target, 
+                          PermissionStatus dirPerms,
+                          boolean createParent) 
+      throws IOException, UnresolvedLinkException {
+    waitForReady();
+
+    final long modTime = FSNamesystem.now();
+    if (createParent) {
+      final String parent = new Path(path).getParent().toString();
+      if (!mkdirs(parent, dirPerms, true, modTime)) {
+        return null;
+      }
+    }
+    final String userName = UserGroupInformation.getCurrentUser().getUserName();
+    INodeSymlink newNode = unprotectedSymlink(path, target, modTime, modTime,
+      new PermissionStatus(userName, null, FsPermission.getDefault()));         
+    if (newNode == null) {
+      NameNode.stateChangeLog.info("DIR* FSDirectory.addSymlink: "
+                                   +"failed to add "+path
+                                   +" to the file system");
+      return null;
+    }
+    fsImage.getEditLog().logSymlink(path, target, modTime, modTime, newNode);
+    
+    NameNode.stateChangeLog.debug("DIR* FSDirectory.addSymlink: "
+                                  +path+" is added to the file system");
+    return newNode;
+  }
+
+  /**
+   * Add the specified path into the namespace. Invoked from edit log processing.
+   */
+  INodeSymlink unprotectedSymlink(String path, String target, long modTime, 
+                                  long atime, PermissionStatus perm) 
+      throws UnresolvedLinkException {
+    INodeSymlink newNode = new INodeSymlink(target, modTime, atime, perm);
+    try {
+      synchronized (rootDir) {
+        newNode = addNode(path, newNode, UNKNOWN_DISK_SPACE, false);
+      }
+    } catch (UnresolvedLinkException e) {
+      /* All UnresolvedLinkExceptions should have been resolved by now, but we
+       * should re-throw them in case that changes so they are not swallowed 
+       * by catching IOException below.
+       */
+      throw e;
+    } catch (IOException e) {
+      return null;
+    }
+    return newNode;
+  }
 }

+ 36 - 4
src/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java

@@ -78,6 +78,7 @@ public class FSEditLog {
   private static final byte OP_SET_QUOTA = 14; // sets name and disk quotas.
   private static final byte OP_RENAME = 15;  // new rename
   private static final byte OP_CONCAT_DELETE = 16; // concat files.
+  private static final byte OP_SYMLINK = 17; // a symbolic link
 
   /* 
    * The following operations are used to control remote edit log streams,
@@ -415,7 +416,7 @@ public class FSEditLog {
 
   @SuppressWarnings("deprecation")
   int loadEditRecords(int logVersion, DataInputStream in,
-                             boolean closeOnExit) throws IOException {
+      boolean closeOnExit) throws IOException {
     FSNamesystem fsNamesys = fsimage.getFSNamesystem();
     FSDirectory fsDir = fsNamesys.dir;
     int numEdits = 0;
@@ -425,7 +426,8 @@ public class FSEditLog {
     int numOpAdd = 0, numOpClose = 0, numOpDelete = 0,
         numOpRenameOld = 0, numOpSetRepl = 0, numOpMkDir = 0,
         numOpSetPerm = 0, numOpSetOwner = 0, numOpSetGenStamp = 0,
-        numOpTimes = 0, numOpRename = 0, numOpConcatDelete = 0, numOpOther = 0;
+        numOpTimes = 0, numOpRename = 0, numOpConcatDelete = 0, 
+        numOpSymlink = 0, numOpOther = 0;
     try {
       while (true) {
         long timestamp = 0;
@@ -577,7 +579,7 @@ public class FSEditLog {
           String s = FSImage.readString(in);
           String d = FSImage.readString(in);
           timestamp = readLong(in);
-          HdfsFileStatus dinfo = fsDir.getFileInfo(d);
+          HdfsFileStatus dinfo = fsDir.getFileInfo(d, false);
           fsDir.unprotectedRenameTo(s, d, timestamp);
           fsNamesys.changeLease(s, d, dinfo);
           break;
@@ -699,6 +701,21 @@ public class FSEditLog {
           fsDir.unprotectedSetTimes(path, mtime, atime, true);
           break;
         }
+        case OP_SYMLINK: {
+          numOpSymlink++;
+          int length = in.readInt();
+          if (length != 4) {
+            throw new IOException("Incorrect data format. " 
+                                  + "symlink operation.");
+          }
+          path = FSImage.readString(in);
+          String value = FSImage.readString(in);
+          mtime = readLong(in);
+          atime = readLong(in);
+          PermissionStatus perm = PermissionStatus.read(in);
+          fsDir.unprotectedSymlink(path, value, mtime, atime, perm);
+          break;
+        }
         case OP_RENAME: {
           if (logVersion > -21) {
             throw new IOException("Unexpected opcode " + opcode
@@ -714,7 +731,7 @@ public class FSEditLog {
           String d = FSImage.readString(in);
           timestamp = readLong(in);
           Rename[] options = readRenameOptions(in);
-          HdfsFileStatus dinfo = fsDir.getFileInfo(d);
+          HdfsFileStatus dinfo = fsDir.getFileInfo(d, false);
           fsDir.unprotectedRenameTo(s, d, timestamp, options);
           fsNamesys.changeLease(s, d, dinfo);
           break;
@@ -1070,6 +1087,21 @@ public class FSEditLog {
       FSEditLog.toLogLong(atime)};
     logEdit(OP_TIMES, new ArrayWritable(DeprecatedUTF8.class, info));
   }
+
+  /** 
+   * Add a create symlink record.
+   */
+  void logSymlink(String path, String value, long mtime, 
+                  long atime, INodeSymlink node) {
+    DeprecatedUTF8 info[] = new DeprecatedUTF8[] { 
+      new DeprecatedUTF8(path),
+      new DeprecatedUTF8(value),
+      FSEditLog.toLogLong(mtime),
+      FSEditLog.toLogLong(atime)};
+    logEdit(OP_SYMLINK, 
+            new ArrayWritable(DeprecatedUTF8.class, info),
+            node.getPermissionStatus());
+  }
   
   static private DeprecatedUTF8 toLogReplication(short replication) {
     return new DeprecatedUTF8(Short.toString(replication));

+ 41 - 22
src/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java

@@ -67,8 +67,9 @@ import org.apache.hadoop.hdfs.server.protocol.CheckpointCommand;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 
 /**
  * FSImage handles checkpointing and logging of the namespace edits.
@@ -361,9 +362,9 @@ public class FSImage extends Storage {
    * @return true if the image needs to be saved or false otherwise
    */
   boolean recoverTransitionRead(Collection<URI> dataDirs,
-                                 Collection<URI> editsDirs,
-                                StartupOption startOpt
-                                ) throws IOException {
+                                Collection<URI> editsDirs,
+                                StartupOption startOpt)
+      throws IOException {
     assert startOpt != StartupOption.FORMAT : 
       "NameNode formatting should be performed before reading the image";
     
@@ -1100,18 +1101,25 @@ public class FSImage extends Storage {
         
         // get quota only when the node is a directory
         long nsQuota = -1L;
-        if (imgVersion <= -16 && blocks == null) {
+        if (imgVersion <= -16 && blocks == null  && numBlocks == -1) {
           nsQuota = in.readLong();
         }
         long dsQuota = -1L;
-        if (imgVersion <= -18 && blocks == null) {
+        if (imgVersion <= -18 && blocks == null && numBlocks == -1) {
           dsQuota = in.readLong();
         }
+
+        // Read the symlink only when the node is a symlink
+        String symlink = "";
+        if (imgVersion <= -23 && numBlocks == -2) {
+          symlink = Text.readString(in);
+        }
         
         PermissionStatus permissions = fsNamesys.getUpgradePermission();
         if (imgVersion <= -11) {
           permissions = PermissionStatus.read(in);
         }
+        
         if (path.length() == 0) { // it is the root
           // update the root's attributes
           if (nsQuota != -1 || dsQuota != -1) {
@@ -1128,7 +1136,7 @@ public class FSImage extends Storage {
         }
         // add new inode
         parentINode = fsDir.addToParent(path, parentINode, permissions,
-                                        blocks, replication, modificationTime, 
+                                        blocks, symlink, replication, modificationTime, 
                                         atime, nsQuota, dsQuota, blockSize);
       }
       
@@ -1312,7 +1320,30 @@ public class FSImage extends Storage {
     int nameLen = name.position();
     out.writeShort(nameLen);
     out.write(name.array(), name.arrayOffset(), nameLen);
-    if (!node.isDirectory()) {  // write file inode
+    if (node.isDirectory()) {
+      out.writeShort(0);  // replication
+      out.writeLong(node.getModificationTime());
+      out.writeLong(0);   // access time
+      out.writeLong(0);   // preferred block size
+      out.writeInt(-1);   // # of blocks
+      out.writeLong(node.getNsQuota());
+      out.writeLong(node.getDsQuota());
+      FILE_PERM.fromShort(node.getFsPermissionShort());
+      PermissionStatus.write(out, node.getUserName(),
+                             node.getGroupName(),
+                             FILE_PERM);
+    } else if (node.isLink()) {
+      out.writeShort(0);  // replication
+      out.writeLong(0);   // modification time
+      out.writeLong(0);   // access time
+      out.writeLong(0);   // preferred block size
+      out.writeInt(-2);   // # of blocks
+      Text.writeString(out, ((INodeSymlink)node).getLinkValue());
+      FILE_PERM.fromShort(node.getFsPermissionShort());
+      PermissionStatus.write(out, node.getUserName(),
+                             node.getGroupName(),
+                             FILE_PERM);      
+    } else {
       INodeFile fileINode = (INodeFile)node;
       out.writeShort(fileINode.getReplication());
       out.writeLong(fileINode.getModificationTime());
@@ -1326,20 +1357,9 @@ public class FSImage extends Storage {
       PermissionStatus.write(out, fileINode.getUserName(),
                              fileINode.getGroupName(),
                              FILE_PERM);
-    } else {   // write directory inode
-      out.writeShort(0);  // replication
-      out.writeLong(node.getModificationTime());
-      out.writeLong(0);   // access time
-      out.writeLong(0);   // preferred block size
-      out.writeInt(-1);    // # of blocks
-      out.writeLong(node.getNsQuota());
-      out.writeLong(node.getDsQuota());
-      FILE_PERM.fromShort(node.getFsPermissionShort());
-      PermissionStatus.write(out, node.getUserName(),
-                             node.getGroupName(),
-                             FILE_PERM);
     }
   }
+  
   /**
    * Save file tree image starting from the given root.
    * This is a recursive procedure, which first saves all children of
@@ -1384,8 +1404,7 @@ public class FSImage extends Storage {
   }
 
   private void loadFilesUnderConstruction(int version, DataInputStream in, 
-                                  FSNamesystem fs) throws IOException {
-
+      FSNamesystem fs) throws IOException {
     FSDirectory fsDir = fs.dir;
     if (version > -13) // pre lease image version
       return;

+ 160 - 84
src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -64,6 +64,7 @@ import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.permission.*;
@@ -257,7 +258,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
   /**
    * Initialize FSNamesystem.
    */
-  private void initialize(Configuration conf, FSImage fsImage) throws IOException {
+  private void initialize(Configuration conf, FSImage fsImage)
+      throws IOException {
     this.systemStart = now();
     this.blockManager = new BlockManager(this, conf);
     setConfigurationParameters(conf);
@@ -631,15 +633,15 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
    * Set permissions for an existing file.
    * @throws IOException
    */
-  public synchronized void setPermission(String src, FsPermission permission
-      ) throws IOException {
+  public synchronized void setPermission(String src, FsPermission permission)
+      throws IOException, UnresolvedLinkException {
     if (isInSafeMode())
       throw new SafeModeException("Cannot set permission for " + src, safeMode);
     checkOwner(src);
     dir.setPermission(src, permission);
     getEditLog().logSync();
     if (auditLog.isInfoEnabled()) {
-      final HdfsFileStatus stat = dir.getFileInfo(src);
+      final HdfsFileStatus stat = dir.getFileInfo(src, false);
       logAuditEvent(UserGroupInformation.getCurrentUser(),
                     Server.getRemoteIp(),
                     "setPermission", src, null, stat);
@@ -650,8 +652,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
    * Set owner for an existing file.
    * @throws IOException
    */
-  public synchronized void setOwner(String src, String username, String group
-      ) throws IOException {
+  public synchronized void setOwner(String src, String username, String group)
+      throws IOException, UnresolvedLinkException {
     if (isInSafeMode())
         throw new SafeModeException("Cannot set owner for " + src, safeMode);
     FSPermissionChecker pc = checkOwner(src);
@@ -667,7 +669,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     dir.setOwner(src, username, group);
     getEditLog().logSync();
     if (auditLog.isInfoEnabled()) {
-      final HdfsFileStatus stat = dir.getFileInfo(src);
+      final HdfsFileStatus stat = dir.getFileInfo(src, false);
       logAuditEvent(UserGroupInformation.getCurrentUser(),
                     Server.getRemoteIp(),
                     "setOwner", src, null, stat);
@@ -679,7 +681,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
    * @see ClientProtocol#getBlockLocations(String, long, long)
    */
   LocatedBlocks getBlockLocations(String clientMachine, String src,
-      long offset, long length) throws IOException {
+      long offset, long length) throws IOException, UnresolvedLinkException {
     if (isPermissionEnabled) {
       checkPathAccess(src, FsAction.READ);
     }
@@ -702,7 +704,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
    * @throws FileNotFoundException
    */
   LocatedBlocks getBlockLocations(String src, long offset, long length,
-      boolean doAccessTime) throws IOException {
+      boolean doAccessTime) throws IOException, UnresolvedLinkException {
     if (offset < 0) {
       throw new IOException("Negative offset is not supported. File: " + src );
     }
@@ -722,11 +724,12 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
   private synchronized LocatedBlocks getBlockLocationsInternal(String src,
                                                        long offset, 
                                                        long length,
-                                                       boolean doAccessTime
-                                                       ) throws IOException {
+                                                       boolean doAccessTime)
+      throws IOException, UnresolvedLinkException {
     INodeFile inode = dir.getFileINode(src);
     if (inode == null)
       throw new FileNotFoundException(src);
+    assert !inode.isLink();
     if (doAccessTime && isAccessTimeSupported()) {
       dir.setTimes(src, inode, -1, now(), false);
     }
@@ -780,7 +783,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
    * @param srcs
    * @throws IOException
    */
-  public void concat(String target, String [] srcs) throws IOException{
+  public void concat(String target, String [] srcs) 
+    throws IOException, UnresolvedLinkException {
     FSNamesystem.LOG.debug("concat " + Arrays.toString(srcs) + " to " + target);
     // check safe mode
     if (isInSafeMode()) {
@@ -792,10 +796,10 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
       throw new IllegalArgumentException("concat: trg file name is empty");
     }
     if(srcs == null || srcs.length == 0) {
-      throw new IllegalArgumentException("concat:  srcs list is empty or null");
+      throw new IllegalArgumentException("concat: srcs list is empty or null");
     }
     
-    // curretnly we require all the files to be in the same dir
+    // currently we require all the files to be in the same dir
     String trgParent = 
       target.substring(0, target.lastIndexOf(Path.SEPARATOR_CHAR));
     for(String s : srcs) {
@@ -904,7 +908,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
    
     
     if (auditLog.isInfoEnabled()) {
-      final HdfsFileStatus stat = dir.getFileInfo(target);
+      final HdfsFileStatus stat = dir.getFileInfo(target, false);
       logAuditEvent(UserGroupInformation.getLoginUser(),
                     Server.getRemoteIp(),
                     "concat", Arrays.toString(srcs), target, stat);
@@ -917,7 +921,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
    * The access time is precise upto an hour. The transaction, if needed, is
    * written to the edits log but is not flushed.
    */
-  public synchronized void setTimes(String src, long mtime, long atime) throws IOException {
+  public synchronized void setTimes(String src, long mtime, long atime) 
+    throws IOException, UnresolvedLinkException {
     if (!isAccessTimeSupported() && atime != -1) {
       throw new IOException("Access time for hdfs is not configured. " +
                             " Please set dfs.support.accessTime configuration parameter.");
@@ -931,7 +936,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     if (inode != null) {
       dir.setTimes(src, inode, mtime, atime, true);
       if (auditLog.isInfoEnabled()) {
-        final HdfsFileStatus stat = dir.getFileInfo(src);
+        final HdfsFileStatus stat = dir.getFileInfo(src, false);
         logAuditEvent(UserGroupInformation.getCurrentUser(),
                       Server.getRemoteIp(),
                       "setTimes", src, null, stat);
@@ -941,11 +946,61 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     }
   }
 
+  /**
+   * Create a symbolic link.
+   */
+  public synchronized void createSymlink(String target, String link,
+      PermissionStatus dirPerms, boolean createParent) 
+      throws IOException, UnresolvedLinkException {
+    if (!createParent) {
+      verifyParentDir(link);
+    }
+    createSymlinkInternal(target, link, dirPerms, createParent);
+    getEditLog().logSync();
+    if (auditLog.isInfoEnabled()) {
+      final HdfsFileStatus stat = dir.getFileInfo(link, false);
+      logAuditEvent(UserGroupInformation.getCurrentUser(),
+                    Server.getRemoteIp(),
+                    "createSymlink", link, target, stat);
+    }
+  }
+
+  /**
+   * Create a symbolic link.
+   */
+  private synchronized void createSymlinkInternal(String target, String link,
+      PermissionStatus dirPerms, boolean createParent)
+      throws IOException, UnresolvedLinkException {
+    if (NameNode.stateChangeLog.isDebugEnabled()) {
+      NameNode.stateChangeLog.debug("DIR* NameSystem.createSymlink: target=" + 
+        target + " link=" + link);
+    }
+
+    if (isInSafeMode()) {
+      throw new SafeModeException("Cannot create symlink " + link, safeMode);
+    }
+    if (!DFSUtil.isValidName(link)) {
+      throw new IOException("Invalid file name: " + link);
+    }
+    if (!dir.isValidToCreate(link)) {
+      throw new IOException("failed to create link " + link 
+          +" either because the filename is invalid or the file exists");
+    }
+    if (isPermissionEnabled) {
+      checkAncestorAccess(link, FsAction.WRITE);
+    }
+    // validate that we have enough inodes.
+    checkFsObjectLimit();
+
+    // add symbolic link to namespace
+    dir.addSymlink(link, target, dirPerms, createParent);
+  }
+
   /**
    * Set replication for an existing file.
    * 
    * The NameNode sets new replication and schedules either replication of 
-   * under-replicated data blocks or removal of the eccessive block copies 
+   * under-replicated data blocks or removal of the excessive block copies 
    * if the blocks are over-replicated.
    * 
    * @see ClientProtocol#setReplication(String, short)
@@ -955,7 +1010,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
    *         false if file does not exist or is a directory
    */
   public boolean setReplication(String src, short replication) 
-                                throws IOException {
+    throws IOException, UnresolvedLinkException {
     boolean status = setReplicationInternal(src, replication);
     getEditLog().logSync();
     if (status && auditLog.isInfoEnabled()) {
@@ -966,9 +1021,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     return status;
   }
 
-  private synchronized boolean setReplicationInternal(String src, 
-                                             short replication
-                                             ) throws IOException {
+  private synchronized boolean setReplicationInternal(String src, short replication) 
+    throws IOException, UnresolvedLinkException {
     if (isInSafeMode())
       throw new SafeModeException("Cannot set replication for " + src, safeMode);
     blockManager.verifyReplication(src, replication, null);
@@ -1002,7 +1056,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     return true;
   }
     
-  long getPreferredBlockSize(String filename) throws IOException {
+  long getPreferredBlockSize(String filename) 
+    throws IOException, UnresolvedLinkException {
     if (isPermissionEnabled) {
       checkTraverse(filename);
     }
@@ -1010,17 +1065,18 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
   }
 
   /*
-   * Verify that parent dir exists
+   * Verify that parent directory of src exists.
    */
   private void verifyParentDir(String src) throws FileAlreadyExistsException,
-      FileNotFoundException {
+      FileNotFoundException, UnresolvedLinkException {
     Path parent = new Path(src).getParent();
     if (parent != null) {
       INode[] pathINodes = dir.getExistingPathINodes(parent.toString());
-      if (pathINodes[pathINodes.length - 1] == null) {
+      INode parentNode = pathINodes[pathINodes.length - 1];
+      if (parentNode == null) {
         throw new FileNotFoundException("Parent directory doesn't exist: "
             + parent.toString());
-      } else if (!pathINodes[pathINodes.length - 1].isDirectory()) {
+      } else if (!parentNode.isDirectory() && !parentNode.isLink()) {
         throw new FileAlreadyExistsException("Parent path is not a directory: "
             + parent.toString());
       }
@@ -1038,13 +1094,13 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
   void startFile(String src, PermissionStatus permissions,
                  String holder, String clientMachine,
                  EnumSet<CreateFlag> flag, boolean createParent, 
-                 short replication, long blockSize
-                ) throws IOException {
+                 short replication, long blockSize)
+      throws IOException, UnresolvedLinkException {
     startFileInternal(src, permissions, holder, clientMachine, flag,
         createParent, replication, blockSize);
     getEditLog().logSync();
     if (auditLog.isInfoEnabled()) {
-      final HdfsFileStatus stat = dir.getFileInfo(src);
+      final HdfsFileStatus stat = dir.getFileInfo(src, false);
       logAuditEvent(UserGroupInformation.getCurrentUser(),
                     Server.getRemoteIp(),
                     "create", src, null, stat);
@@ -1058,8 +1114,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
                                               EnumSet<CreateFlag> flag,
                                               boolean createParent,
                                               short replication,
-                                              long blockSize
-                                              ) throws IOException {
+                                              long blockSize)
+      throws IOException, UnresolvedLinkException {
     boolean overwrite = flag.contains(CreateFlag.OVERWRITE);
     boolean append = flag.contains(CreateFlag.APPEND);
     boolean create = flag.contains(CreateFlag.CREATE);
@@ -1230,8 +1286,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
   /**
    * Append to an existing file in the namespace.
    */
-  LocatedBlock appendFile(String src, String holder, String clientMachine
-      ) throws IOException {
+  LocatedBlock appendFile(String src, String holder, String clientMachine)
+    throws IOException, UnresolvedLinkException {
     if (supportAppends == false) {
       throw new IOException("Append to hdfs not supported." +
                             " Please refer to dfs.support.append configuration parameter.");
@@ -1314,7 +1370,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
                                          String clientName,
                                          Block previous,
                                          HashMap<Node, Node> excludedNodes
-                                         ) throws IOException {
+                                         ) 
+      throws IOException, UnresolvedLinkException {
     long fileLength, blockSize;
     int replication;
     DatanodeDescriptor clientNode = null;
@@ -1389,8 +1446,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
   /**
    * The client would like to let go of the given block
    */
-  public synchronized boolean abandonBlock(Block b, String src, String holder
-      ) throws IOException {
+  public synchronized boolean abandonBlock(Block b, String src, String holder)
+    throws IOException, UnresolvedLinkException {
     //
     // Remove the block from the pending creates list
     //
@@ -1406,7 +1463,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
   
   // make sure that we still have the lease on this file.
   private INodeFileUnderConstruction checkLease(String src, String holder) 
-                                                      throws IOException {
+    throws IOException, UnresolvedLinkException {
     INodeFile file = dir.getFileINode(src);
     checkLease(src, holder, file);
     return (INodeFileUnderConstruction)file;
@@ -1450,18 +1507,15 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     COMPLETE_SUCCESS
   }
   
-  public CompleteFileStatus completeFile(String src,
-                                         String holder,
-                                         Block last) throws IOException {
+  public CompleteFileStatus completeFile(String src, String holder, Block last) 
+    throws IOException, UnresolvedLinkException {
     CompleteFileStatus status = completeFileInternal(src, holder, last);
     getEditLog().logSync();
     return status;
   }
 
-  private synchronized CompleteFileStatus completeFileInternal(
-                                            String src, 
-                                            String holder,
-                                            Block last) throws IOException {
+  private synchronized CompleteFileStatus completeFileInternal(String src, 
+    String holder, Block last) throws IOException, UnresolvedLinkException {
     NameNode.stateChangeLog.debug("DIR* NameSystem.completeFile: " + src + " for " + holder);
     if (isInSafeMode())
       throw new SafeModeException("Cannot complete file " + src, safeMode);
@@ -1595,11 +1649,12 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
    * @deprecated Use {@link #renameTo(String, String, Options.Rename...)} instead.
    */
   @Deprecated
-  boolean renameTo(String src, String dst) throws IOException {
+  boolean renameTo(String src, String dst) 
+    throws IOException, UnresolvedLinkException {
     boolean status = renameToInternal(src, dst);
     getEditLog().logSync();
     if (status && auditLog.isInfoEnabled()) {
-      final HdfsFileStatus stat = dir.getFileInfo(dst);
+      final HdfsFileStatus stat = dir.getFileInfo(dst, false);
       logAuditEvent(UserGroupInformation.getCurrentUser(),
                     Server.getRemoteIp(),
                     "rename", src, dst, stat);
@@ -1609,8 +1664,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
 
   /** @deprecated See {@link #renameTo(String, String)} */
   @Deprecated
-  private synchronized boolean renameToInternal(String src, String dst
-      ) throws IOException {
+  private synchronized boolean renameToInternal(String src, String dst)
+    throws IOException, UnresolvedLinkException {
     NameNode.stateChangeLog.debug("DIR* NameSystem.renameTo: " + src + " to " + dst);
     if (isInSafeMode())
       throw new SafeModeException("Cannot rename " + src, safeMode);
@@ -1627,7 +1682,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
       checkAncestorAccess(actualdst, FsAction.WRITE);
     }
 
-    HdfsFileStatus dinfo = dir.getFileInfo(dst);
+    HdfsFileStatus dinfo = dir.getFileInfo(dst, false);
     if (dir.renameTo(src, dst)) {
       changeLease(src, dst, dinfo);     // update lease with new filename
       return true;
@@ -1638,7 +1693,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
 
   /** Rename src to dst */
   void renameTo(String src, String dst, Options.Rename... options)
-      throws IOException {
+      throws IOException, UnresolvedLinkException {
     renameToInternal(src, dst, options);
     getEditLog().logSync();
     if (auditLog.isInfoEnabled()) {
@@ -1646,14 +1701,14 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
       for (Rename option : options) {
         cmd.append(option.value()).append(" ");
       }
-      final HdfsFileStatus stat = dir.getFileInfo(dst);
+      final HdfsFileStatus stat = dir.getFileInfo(dst, false);
       logAuditEvent(UserGroupInformation.getCurrentUser(), Server.getRemoteIp(),
                     cmd.toString(), src, dst, stat);
     }
   }
 
   private synchronized void renameToInternal(String src, String dst,
-      Options.Rename... options) throws IOException {
+      Options.Rename... options) throws IOException, UnresolvedLinkException {
     if (NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("DIR* NameSystem.renameTo: with options - "
           + src + " to " + dst);
@@ -1669,7 +1724,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
       checkAncestorAccess(dst, FsAction.WRITE);
     }
 
-    HdfsFileStatus dinfo = dir.getFileInfo(dst);
+    HdfsFileStatus dinfo = dir.getFileInfo(dst, false);
     dir.renameTo(src, dst, options);
     changeLease(src, dst, dinfo); // update lease with new filename
   }
@@ -1678,7 +1733,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
    * Remove the indicated filename from namespace. If the filename 
    * is a directory (non empty) and recursive is set to false then throw exception.
    */
-    public boolean delete(String src, boolean recursive) throws IOException {
+    public boolean delete(String src, boolean recursive) 
+      throws IOException, UnresolvedLinkException {
       if ((!recursive) && (!dir.isDirEmpty(src))) {
         throw new IOException(src + " is non empty");
       }
@@ -1703,8 +1759,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
    * <p>
    * For small directory or file the deletion is done in one shot.
    */
-  private boolean deleteInternal(String src, 
-      boolean enforcePermission) throws IOException {
+  private boolean deleteInternal(String src, boolean enforcePermission) 
+    throws IOException, UnresolvedLinkException {
     boolean deleteNow = false;
     ArrayList<Block> collectedBlocks = new ArrayList<Block>();
     synchronized(this) {
@@ -1764,29 +1820,33 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
 
   /** Get the file info for a specific file.
    * @param src The string representation of the path to the file
+   * @param resolveLink whether to throw UnresolvedLinkException 
+   *        if src refers to a symlinks
    * @throws IOException if permission to access file is denied by the system 
+   * @throws UnresolvedLinkException if a symlink is encountered.
    * @return object containing information regarding the file
    *         or null if file not found
    */
-  HdfsFileStatus getFileInfo(String src) throws IOException {
+  HdfsFileStatus getFileInfo(String src, boolean resolveLink) 
+    throws IOException, UnresolvedLinkException {
     if (!DFSUtil.isValidName(src)) {
       throw new IOException("Invalid file name: " + src);
     }
     if (isPermissionEnabled) {
       checkTraverse(src);
     }
-    return dir.getFileInfo(src);
+    return dir.getFileInfo(src, resolveLink);
   }
 
   /**
    * Create all the necessary directories
    */
   public boolean mkdirs(String src, PermissionStatus permissions,
-      boolean createParent) throws IOException {
+      boolean createParent) throws IOException, UnresolvedLinkException {
     boolean status = mkdirsInternal(src, permissions, createParent);
     getEditLog().logSync();
     if (status && auditLog.isInfoEnabled()) {
-      final HdfsFileStatus stat = dir.getFileInfo(src);
+      final HdfsFileStatus stat = dir.getFileInfo(src, false);
       logAuditEvent(UserGroupInformation.getCurrentUser(),
                     Server.getRemoteIp(),
                     "mkdirs", src, null, stat);
@@ -1798,7 +1858,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
    * Create all the necessary directories
    */
   private synchronized boolean mkdirsInternal(String src,
-      PermissionStatus permissions, boolean createParent) throws IOException {
+      PermissionStatus permissions, boolean createParent) 
+      throws IOException, UnresolvedLinkException {
     NameNode.stateChangeLog.debug("DIR* NameSystem.mkdirs: " + src);
     if (isPermissionEnabled) {
       checkTraverse(src);
@@ -1832,7 +1893,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     return true;
   }
 
-  ContentSummary getContentSummary(String src) throws IOException {
+  ContentSummary getContentSummary(String src) 
+      throws IOException, UnresolvedLinkException {
     if (isPermissionEnabled) {
       checkPermission(src, false, null, null, null, FsAction.READ_EXECUTE);
     }
@@ -1844,7 +1906,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
    * See {@link ClientProtocol#setQuota(String, long, long)} for the 
    * contract.
    */
-  void setQuota(String path, long nsQuota, long dsQuota) throws IOException {
+  void setQuota(String path, long nsQuota, long dsQuota) 
+      throws IOException, UnresolvedLinkException {
     if (isInSafeMode())
       throw new SafeModeException("Cannot set quota on " + path, safeMode);
     if (isPermissionEnabled) {
@@ -1859,7 +1922,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
    * @param clientName The string representation of the client
    * @throws IOException if path does not exist
    */
-  void fsync(String src, String clientName) throws IOException {
+  void fsync(String src, String clientName) 
+      throws IOException, UnresolvedLinkException {
 
     NameNode.stateChangeLog.info("BLOCK* NameSystem.fsync: file "
                                   + src + " for " + clientName);
@@ -1885,10 +1949,9 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
    * @return true  if file has been successfully finalized and closed or 
    *         false if block recovery has been initiated
    */
-  boolean internalReleaseLease(
-      Lease lease, String src, String recoveryLeaseHolder)
-  throws AlreadyBeingCreatedException,
-         IOException {
+  boolean internalReleaseLease(Lease lease, String src, 
+      String recoveryLeaseHolder) throws AlreadyBeingCreatedException, 
+      IOException, UnresolvedLinkException {
     LOG.info("Recovering lease=" + lease + ", src=" + src);
 
     INodeFile iFile = dir.getFileINode(src);
@@ -2019,9 +2082,10 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
   }
 
 
-  private void finalizeINodeFileUnderConstruction(
-      String src,
-      INodeFileUnderConstruction pendingFile) throws IOException {
+  private void finalizeINodeFileUnderConstruction(String src, 
+      INodeFileUnderConstruction pendingFile) 
+      throws IOException, UnresolvedLinkException {
+      
     leaseManager.removeLease(pendingFile.getClientName(), src);
 
     // The file is no longer pending.
@@ -2037,8 +2101,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
 
   synchronized void commitBlockSynchronization(Block lastblock,
       long newgenerationstamp, long newlength,
-      boolean closeFile, boolean deleteblock, DatanodeID[] newtargets
-      ) throws IOException {
+      boolean closeFile, boolean deleteblock, DatanodeID[] newtargets)
+      throws IOException, UnresolvedLinkException {
     LOG.info("commitBlockSynchronization(lastblock=" + lastblock
           + ", newgenerationstamp=" + newgenerationstamp
           + ", newlength=" + newlength
@@ -2137,7 +2201,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
    * Get a listing of all files at 'src'.  The Object[] array
    * exists so we can return file attributes (soon to be implemented)
    */
-  public HdfsFileStatus[] getListing(String src) throws IOException {
+  public HdfsFileStatus[] getListing(String src) 
+    throws IOException, UnresolvedLinkException {
     if (isPermissionEnabled) {
       if (dir.isDir(src)) {
         checkPathAccess(src, FsAction.READ_EXECUTE);
@@ -3725,8 +3790,13 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     // Calculate number of blocks under construction
     long numUCBlocks = 0;
     for (Lease lease : leaseManager.getSortedLeases()) {
-      for(String path : lease.getPaths()) {
-        INode node = dir.getFileINode(path);
+      for (String path : lease.getPaths()) {
+        INode node; 
+        try {
+          node = dir.getFileINode(path);
+        } catch (UnresolvedLinkException e) {
+          throw new AssertionError("Lease files should reside on this FS");
+        }
         assert node != null : "Found a lease for nonexisting file.";
         assert node.isUnderConstruction() :
           "Found a lease for file that is not under construction.";
@@ -3860,27 +3930,28 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     return new PermissionStatus(fsOwner.getShortUserName(), supergroup, permission);
   }
 
-  private FSPermissionChecker checkOwner(String path) throws AccessControlException {
+  private FSPermissionChecker checkOwner(String path
+      ) throws AccessControlException, UnresolvedLinkException {
     return checkPermission(path, true, null, null, null, null);
   }
 
   private FSPermissionChecker checkPathAccess(String path, FsAction access
-      ) throws AccessControlException {
+      ) throws AccessControlException, UnresolvedLinkException {
     return checkPermission(path, false, null, null, access, null);
   }
 
   private FSPermissionChecker checkParentAccess(String path, FsAction access
-      ) throws AccessControlException {
+      ) throws AccessControlException, UnresolvedLinkException {
     return checkPermission(path, false, null, access, null, null);
   }
 
   private FSPermissionChecker checkAncestorAccess(String path, FsAction access
-      ) throws AccessControlException {
+      ) throws AccessControlException, UnresolvedLinkException {
     return checkPermission(path, false, access, null, null, null);
   }
 
   private FSPermissionChecker checkTraverse(String path
-      ) throws AccessControlException {
+      ) throws AccessControlException, UnresolvedLinkException {
     return checkPermission(path, false, null, null, null, null);
   }
 
@@ -3897,7 +3968,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
    */
   private FSPermissionChecker checkPermission(String path, boolean doCheckOwner,
       FsAction ancestorAccess, FsAction parentAccess, FsAction access,
-      FsAction subAccess) throws AccessControlException {
+      FsAction subAccess) throws AccessControlException, UnresolvedLinkException {
     FSPermissionChecker pc = new FSPermissionChecker(
         fsOwner.getShortUserName(), supergroup);
     if (!pc.isSuper) {
@@ -4216,7 +4287,12 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
       for (Lease lease : leaseManager.getSortedLeases()) {
         for(String path : lease.getPaths()) {
           // verify that path exists in namespace
-          INode node = dir.getFileINode(path);
+          INode node;
+          try {
+            node = dir.getFileINode(path);
+          } catch (UnresolvedLinkException e) {
+            throw new AssertionError("Lease files should reside on this FS");
+          }
           if (node == null) {
             throw new IOException("saveLeases found path " + path +
                                   " but no matching entry in namespace.");

+ 6 - 2
src/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java

@@ -25,6 +25,7 @@ import java.util.Stack;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.security.AccessControlException;
@@ -104,10 +105,12 @@ class FSPermissionChecker {
    * If path is not a directory, there is no effect.
    * @return a PermissionChecker object which caches data for later use.
    * @throws AccessControlException
+   * @throws UnresolvedLinkException
    */
   void checkPermission(String path, INodeDirectory root, boolean doCheckOwner,
       FsAction ancestorAccess, FsAction parentAccess, FsAction access,
-      FsAction subAccess) throws AccessControlException {
+      FsAction subAccess) 
+      throws AccessControlException, UnresolvedLinkException {
     if (LOG.isDebugEnabled()) {
       LOG.debug("ACCESS CHECK: " + this
           + ", doCheckOwner=" + doCheckOwner
@@ -118,7 +121,8 @@ class FSPermissionChecker {
     }
     // check if (parentAccess != null) && file exists, then check sb
     synchronized(root) {
-      INode[] inodes = root.getExistingPathINodes(path);
+      // Resolve symlinks, the check is performed on the link target.
+      INode[] inodes = root.getExistingPathINodes(path, true);
       int ancestorIndex = inodes.length - 2;
       for(; ancestorIndex >= 0 && inodes[ancestorIndex] == null;
           ancestorIndex--);

+ 25 - 2
src/java/org/apache/hadoop/hdfs/server/namenode/INode.java

@@ -179,6 +179,7 @@ abstract class INode implements Comparable<byte[]>, FSInodeInfo {
    * Check whether it's a directory
    */
   public abstract boolean isDirectory();
+
   /**
    * Collect all the blocks in all children of this INode.
    * Count and return the number of files in the sub tree.
@@ -215,7 +216,7 @@ abstract class INode implements Comparable<byte[]>, FSInodeInfo {
   }
   
   /**
-   * Adds total nubmer of names and total disk space taken under 
+   * Adds total number of names and total disk space taken under 
    * this tree to counts.
    * Returns updated counts object.
    */
@@ -318,6 +319,13 @@ abstract class INode implements Comparable<byte[]>, FSInodeInfo {
     return false;
   }
 
+  /**
+   * Check whether it's a symlink
+   */
+  public boolean isLink() {
+    return false;
+  }
+
   /**
    * Breaks file path into components.
    * @param path
@@ -351,11 +359,26 @@ abstract class INode implements Comparable<byte[]>, FSInodeInfo {
     return path.split(Path.SEPARATOR);
   }
 
+  /**
+   * Given some components, create a path name.
+   * @param components
+   * @return concatenated path
+   */
+  static String constructPath(byte[][] components, int start) {
+    StringBuilder buf = new StringBuilder();
+    for (int i = start; i < components.length; i++) {
+      buf.append(DFSUtil.bytes2String(components[i]));
+      if (i < components.length - 1) {
+        buf.append(Path.SEPARATOR);
+      }
+    }
+    return buf.toString();
+  }
+
   boolean removeNode() {
     if (parent == null) {
       return false;
     } else {
-      
       parent.removeChild(this);
       parent = null;
       return true;

+ 57 - 20
src/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java

@@ -22,11 +22,13 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
 
 /**
  * Directory INode class.
@@ -111,18 +113,22 @@ class INodeDirectory extends INode {
   }
 
   /**
+   * Return the INode of the last component in components, or null if the last
+   * component does not exist.
    */
-  private INode getNode(byte[][] components) {
+  private INode getNode(byte[][] components, boolean resolveLink) 
+    throws UnresolvedLinkException {
     INode[] inode  = new INode[1];
-    getExistingPathINodes(components, inode);
+    getExistingPathINodes(components, inode, resolveLink);
     return inode[0];
   }
 
   /**
    * This is the external interface
    */
-  INode getNode(String path) {
-    return getNode(getPathComponents(path));
+  INode getNode(String path, boolean resolveLink) 
+    throws UnresolvedLinkException {
+    return getNode(getPathComponents(path), resolveLink);
   }
 
   /**
@@ -131,7 +137,12 @@ class INodeDirectory extends INode {
    * will be stored starting from the root INode into existing[0]; if
    * existing is not big enough to store all path components, then only the
    * last existing and non existing INodes will be stored so that
-   * existing[existing.length-1] refers to the target INode.
+   * existing[existing.length-1] refers to the INode of the final component.
+   * 
+   * An UnresolvedPathException is always thrown when an intermediate path 
+   * component refers to a symbolic link. If the final path component refers 
+   * to a symbolic link then an UnresolvedPathException is only thrown if
+   * resolveLink is true.  
    * 
    * <p>
    * Example: <br>
@@ -155,11 +166,15 @@ class INodeDirectory extends INode {
    * the array with [rootINode,c1,c2,null], <br>
    * <code>getExistingPathINodes(["","c1","c2","c3"], [?,?,?,?])</code> should
    * fill the array with [rootINode,c1,c2,null]
+   * 
    * @param components array of path component name
-   * @param existing INode array to fill with existing INodes
+   * @param existing array to fill with existing INodes
+   * @param resolveLink indicates whether UnresolvedLinkException should
+   *        be thrown when the path refers to a symbolic link.
    * @return number of existing INodes in the path
    */
-  int getExistingPathINodes(byte[][] components, INode[] existing) {
+  int getExistingPathINodes(byte[][] components, INode[] existing, 
+      boolean resolveLink) throws UnresolvedLinkException {
     assert compareBytes(this.name, components[0]) == 0 :
       "Incorrect name " + getLocalName() + " expected " + 
       DFSUtil.bytes2String(components[0]);
@@ -167,17 +182,33 @@ class INodeDirectory extends INode {
     INode curNode = this;
     int count = 0;
     int index = existing.length - components.length;
-    if (index > 0)
+    if (index > 0) {
       index = 0;
-    while ((count < components.length) && (curNode != null)) {
-      if (index >= 0)
+    }
+    while (count < components.length && curNode != null) {
+      final boolean lastComp = (count == components.length - 1);      
+      if (index >= 0) {
         existing[index] = curNode;
-      if (!curNode.isDirectory() || (count == components.length - 1))
-        break; // no more child, stop here
+      }
+      if (curNode.isLink() && (!lastComp || (lastComp && resolveLink))) {
+        NameNode.stateChangeLog.debug("UnresolvedPathException " +
+           " count: " + count +
+           " componenent: " + DFSUtil.bytes2String(components[count]) +
+           " full path: " + constructPath(components, 0) +
+           " remaining path: " + constructPath(components, count+1) +
+           " symlink: " + ((INodeSymlink)curNode).getLinkValue());
+        final String linkTarget = ((INodeSymlink)curNode).getLinkValue();
+        throw new UnresolvedPathException(constructPath(components, 0),
+                                          constructPath(components, count+1),
+                                          linkTarget);
+      }
+      if (lastComp || !curNode.isDirectory()) {
+        break;
+      }
       INodeDirectory parentDir = (INodeDirectory)curNode;
       curNode = parentDir.getChildINode(components[count + 1]);
-      count += 1;
-      index += 1;
+      count++;
+      index++;
     }
     return count;
   }
@@ -187,6 +218,8 @@ class INodeDirectory extends INode {
    * always exist and is this INode.
    * 
    * @param path the path to explore
+   * @param resolveLink indicates whether UnresolvedLinkException should 
+   *        be thrown when the path refers to a symbolic link.
    * @return INodes array containing the existing INodes in the order they
    *         appear when following the path from the root INode to the
    *         deepest INodes. The array size will be the number of expected
@@ -195,11 +228,12 @@ class INodeDirectory extends INode {
    *         
    * @see #getExistingPathINodes(byte[][], INode[])
    */
-  INode[] getExistingPathINodes(String path) {
+  INode[] getExistingPathINodes(String path, boolean resolveLink) 
+    throws UnresolvedLinkException {
     byte[][] components = getPathComponents(path);
     INode[] inodes = new INode[components.length];
 
-    this.getExistingPathINodes(components, inodes);
+    this.getExistingPathINodes(components, inodes, resolveLink);
     
     return inodes;
   }
@@ -243,7 +277,8 @@ class INodeDirectory extends INode {
    * Equivalent to addNode(path, newNode, false).
    * @see #addNode(String, INode, boolean)
    */
-  <T extends INode> T addNode(String path, T newNode) throws FileNotFoundException {
+  <T extends INode> T addNode(String path, T newNode) 
+    throws FileNotFoundException, UnresolvedLinkException {
     return addNode(path, newNode, false);
   }
   /**
@@ -255,10 +290,11 @@ class INodeDirectory extends INode {
    * @param inheritPermission If true, copy the parent's permission to newNode.
    * @return null if the node already exists; inserted INode, otherwise
    * @throws FileNotFoundException if parent does not exist or 
+   * @throws UnresolvedLinkException if any path component is a symbolic link
    * is not a directory.
    */
   <T extends INode> T addNode(String path, T newNode, boolean inheritPermission
-      ) throws FileNotFoundException {
+      ) throws FileNotFoundException, UnresolvedLinkException  {
     if(addToParent(path, newNode, null, inheritPermission) == null)
       return null;
     return newNode;
@@ -278,7 +314,8 @@ class INodeDirectory extends INode {
                                       T newNode,
                                       INodeDirectory parent,
                                       boolean inheritPermission
-                                    ) throws FileNotFoundException {
+                                    ) throws FileNotFoundException, 
+                                             UnresolvedLinkException {
     byte[][] pathComponents = getPathComponents(path);
     assert pathComponents != null : "Incorrect path " + path;
     int pathLen = pathComponents.length;
@@ -287,7 +324,7 @@ class INodeDirectory extends INode {
     if(parent == null) {
       // Gets the parent INode
       INode[] inodes  = new INode[2];
-      getExistingPathINodes(pathComponents, inodes);
+      getExistingPathINodes(pathComponents, inodes, false);
       INode inode = inodes[0];
       if (inode == null) {
         throw new FileNotFoundException("Parent path does not exist: "+path);

+ 78 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java

@@ -0,0 +1,78 @@
+/**
+ * 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.hdfs.server.namenode;
+
+import java.util.List;
+
+import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.protocol.Block;
+
+/**
+ * An INode representing a symbolic link.
+ */
+public class INodeSymlink extends INode {
+  private byte[] symlink; // The target URI
+
+  INodeSymlink(String value, long modTime, long atime,
+               PermissionStatus permissions) {
+    super(permissions, modTime, atime);
+    assert value != null;
+    setLinkValue(value);
+    setModificationTimeForce(modTime);
+    setAccessTime(atime);
+  }
+
+  public boolean isLink() {
+    return true;
+  }
+  
+  void setLinkValue(String value) {
+    this.symlink = DFSUtil.string2Bytes(value);
+  }
+
+  public String getLinkValue() {
+    return DFSUtil.bytes2String(symlink);
+  }
+
+  public byte[] getSymlink() {
+    return symlink;
+  }
+
+  @Override
+  DirCounts spaceConsumedInTree(DirCounts counts) {
+    counts.nsCount += 1;
+    return counts;
+  }
+  
+  @Override
+  int collectSubtreeBlocksAndClear(List<Block> v) {
+    return 1;
+  }
+
+  @Override
+  long[] computeContentSummary(long[] summary) {
+    summary[1]++; // Increment the file count
+    return summary;
+  }
+
+  @Override
+  public boolean isDirectory() {
+    return false;
+  }
+}

+ 10 - 5
src/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java

@@ -30,6 +30,7 @@ import java.util.TreeSet;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 
 /**
@@ -157,8 +158,8 @@ public class LeaseManager {
   /**
    * Finds the pathname for the specified pendingFile
    */
-  synchronized String findPath(INodeFileUnderConstruction pendingFile
-      ) throws IOException {
+  synchronized String findPath(INodeFileUnderConstruction pendingFile)
+      throws IOException {
     Lease lease = getLease(pendingFile.getClientName());
     if (lease != null) {
       String src = lease.findPath(pendingFile);
@@ -220,10 +221,14 @@ public class LeaseManager {
      * @return the path associated with the pendingFile and null if not found.
      */
     private String findPath(INodeFileUnderConstruction pendingFile) {
-      for(String src : paths) {
-        if (fsnamesystem.dir.getFileINode(src) == pendingFile) {
-          return src;
+      try {
+        for (String src : paths) {
+          if (fsnamesystem.dir.getFileINode(src) == pendingFile) {
+            return src;
+          }
         }
+      } catch (UnresolvedLinkException e) {
+        throw new AssertionError("Lease files should reside on this FS");
       }
       return null;
     }

+ 108 - 44
src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java

@@ -36,6 +36,7 @@ import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Trash;
 import org.apache.hadoop.fs.Options;
+import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.HDFSPolicyProvider;
@@ -49,6 +50,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
+import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
 import org.apache.hadoop.hdfs.security.ExportedAccessKeys;
 import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
@@ -410,7 +412,8 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
     this(conf, NamenodeRole.ACTIVE);
   }
 
-  protected NameNode(Configuration conf, NamenodeRole role) throws IOException {
+  protected NameNode(Configuration conf, NamenodeRole role) 
+      throws IOException { 
     UserGroupInformation.setConfiguration(conf);
     DFSUtil.login(conf, 
         DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY,
@@ -569,9 +572,10 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
   }
   
   /** {@inheritDoc} */
-  public LocatedBlocks   getBlockLocations(String src, 
+  public LocatedBlocks getBlockLocations(String src, 
                                           long offset, 
-                                          long length) throws IOException {
+                                          long length) 
+      throws IOException {
     myMetrics.numGetBlockLocations.inc();
     return namesystem.getBlockLocations(getClientMachine(), 
                                         src, offset, length);
@@ -593,12 +597,11 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
   /** {@inheritDoc} */
   public void create(String src, 
                      FsPermission masked,
-                             String clientName, 
-                             EnumSetWritable<CreateFlag> flag,
-                             boolean createParent,
-                             short replication,
-                             long blockSize
-                             ) throws IOException {
+                     String clientName, 
+                     EnumSetWritable<CreateFlag> flag,
+                     boolean createParent,
+                     short replication,
+                     long blockSize) throws IOException {
     String clientMachine = getClientMachine();
     if (stateChangeLog.isDebugEnabled()) {
       stateChangeLog.debug("*DIR* NameNode.create: file "
@@ -617,7 +620,8 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
   }
 
   /** {@inheritDoc} */
-  public LocatedBlock append(String src, String clientName) throws IOException {
+  public LocatedBlock append(String src, String clientName) 
+      throws IOException {
     String clientMachine = getClientMachine();
     if (stateChangeLog.isDebugEnabled()) {
       stateChangeLog.debug("*DIR* NameNode.append: file "
@@ -629,21 +633,20 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
   }
 
   /** {@inheritDoc} */
-  public boolean setReplication(String src, 
-                                short replication
-                                ) throws IOException {
+  public boolean setReplication(String src, short replication) 
+    throws IOException {  
     return namesystem.setReplication(src, replication);
   }
     
   /** {@inheritDoc} */
-  public void setPermission(String src, FsPermission permissions
-      ) throws IOException {
+  public void setPermission(String src, FsPermission permissions)
+      throws IOException {
     namesystem.setPermission(src, permissions);
   }
 
   /** {@inheritDoc} */
-  public void setOwner(String src, String username, String groupname
-      ) throws IOException {
+  public void setOwner(String src, String username, String groupname)
+      throws IOException {
     namesystem.setOwner(src, username, groupname);
   }
 
@@ -651,8 +654,8 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
   public LocatedBlock addBlock(String src,
                                String clientName,
                                Block previous,
-                               DatanodeInfo[] excludedNodes
-                               ) throws IOException {
+                               DatanodeInfo[] excludedNodes)
+      throws IOException {
     stateChangeLog.debug("*BLOCK* NameNode.addBlock: file "
                          +src+" for "+clientName);
     HashMap<Node, Node> excludedNodesSet = null;
@@ -672,8 +675,8 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
   /**
    * The client needs to give up on the block.
    */
-  public void abandonBlock(Block b, String src, String holder
-      ) throws IOException {
+  public void abandonBlock(Block b, String src, String holder)
+      throws IOException {
     stateChangeLog.debug("*BLOCK* NameNode.abandonBlock: "
                          +b+" of file "+src);
     if (!namesystem.abandonBlock(b, src, holder)) {
@@ -682,8 +685,8 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
   }
 
   /** {@inheritDoc} */
-  public boolean complete(String src, String clientName,
-                          Block last) throws IOException {
+  public boolean complete(String src, String clientName, Block last)
+      throws IOException {
     stateChangeLog.debug("*DIR* NameNode.complete: " + src + " for " + clientName);
     CompleteFileStatus returnCode =
       namesystem.completeFile(src, clientName, last);
@@ -692,7 +695,8 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
     } else if (returnCode == CompleteFileStatus.COMPLETE_SUCCESS) {
       return true;
     } else {
-      throw new IOException("Could not complete write to file " + src + " by " + clientName);
+      throw new IOException("Could not complete write to file " + 
+                            src + " by " + clientName);
     }
   }
 
@@ -717,7 +721,7 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
   /** {@inheritDoc} */
   @Override
   public LocatedBlock updateBlockForPipeline(Block block, String clientName)
-  throws IOException {
+      throws IOException {
     return namesystem.updateBlockForPipeline(block, clientName);
   }
 
@@ -732,13 +736,14 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
   /** {@inheritDoc} */
   public void commitBlockSynchronization(Block block,
       long newgenerationstamp, long newlength,
-      boolean closeFile, boolean deleteblock, DatanodeID[] newtargets
-      ) throws IOException {
+      boolean closeFile, boolean deleteblock, DatanodeID[] newtargets)
+      throws IOException {
     namesystem.commitBlockSynchronization(block,
         newgenerationstamp, newlength, closeFile, deleteblock, newtargets);
   }
   
-  public long getPreferredBlockSize(String filename) throws IOException {
+  public long getPreferredBlockSize(String filename) 
+      throws IOException {
     return namesystem.getPreferredBlockSize(filename);
   }
     
@@ -767,7 +772,8 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
   
   /** {@inheritDoc} */
   @Override
-  public void rename(String src, String dst, Options.Rename... options) throws IOException {
+  public void rename(String src, String dst, Options.Rename... options)
+      throws IOException {
     stateChangeLog.debug("*DIR* NameNode.rename: " + src + " to " + dst);
     if (!checkPathLength(dst)) {
       throw new IOException("rename: Pathname too long.  Limit " 
@@ -809,7 +815,8 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
   }
     
   /** {@inheritDoc} */
-  public boolean mkdirs(String src, FsPermission masked, boolean createParent) throws IOException {
+  public boolean mkdirs(String src, FsPermission masked, boolean createParent)
+      throws IOException {
     stateChangeLog.debug("*DIR* NameNode.mkdirs: " + src);
     if (!checkPathLength(src)) {
       throw new IOException("mkdirs: Pathname too long.  Limit " 
@@ -840,15 +847,26 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
   /**
    * Get the file info for a specific file.
    * @param src The string representation of the path to the file
-   * @throws IOException if permission to access file is denied by the system
    * @return object containing information regarding the file
    *         or null if file not found
    */
   public HdfsFileStatus getFileInfo(String src)  throws IOException {
     myMetrics.numFileInfoOps.inc();
-    return namesystem.getFileInfo(src);
+    return namesystem.getFileInfo(src, true);
   }
 
+  /**
+   * Get the file info for a specific file. If the path refers to a 
+   * symlink then the FileStatus of the symlink is returned.
+   * @param src The string representation of the path to the file
+   * @return object containing information regarding the file
+   *         or null if file not found
+   */
+  public HdfsFileStatus getFileLinkInfo(String src) throws IOException { 
+    myMetrics.numFileInfoOps.inc();
+    return namesystem.getFileInfo(src, false);
+  }
+  
   /** @inheritDoc */
   public long[] getStats() {
     return namesystem.getStats();
@@ -857,7 +875,7 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
   /**
    */
   public DatanodeInfo[] getDatanodeReport(DatanodeReportType type)
-  throws IOException {
+      throws IOException {
     DatanodeInfo results[] = namesystem.datanodeReport(type);
     if (results == null ) {
       throw new IOException("Cannot find datanode report");
@@ -883,7 +901,8 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
    * @throws AccessControlException 
    * @inheritDoc
    */
-  public boolean restoreFailedStorage(String arg) throws AccessControlException {
+  public boolean restoreFailedStorage(String arg) 
+      throws AccessControlException {
     return namesystem.restoreFailedStorage(arg);
   }
 
@@ -931,8 +950,8 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
     namesystem.finalizeUpgrade();
   }
 
-  public UpgradeStatusReport distributedUpgradeProgress(UpgradeAction action
-                                                        ) throws IOException {
+  public UpgradeStatusReport distributedUpgradeProgress(UpgradeAction action)
+      throws IOException {
     return namesystem.distributedUpgradeProgress(action);
   }
 
@@ -950,7 +969,7 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
 
   /** {@inheritDoc} */
   public void setQuota(String path, long namespaceQuota, long diskspaceQuota) 
-                       throws IOException {
+      throws IOException {
     namesystem.setQuota(path, namespaceQuota, diskspaceQuota);
   }
   
@@ -960,17 +979,62 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
   }
 
   /** @inheritDoc */
-  public void setTimes(String src, long mtime, long atime) throws IOException {
+  public void setTimes(String src, long mtime, long atime) 
+      throws IOException {
     namesystem.setTimes(src, mtime, atime);
   }
 
+  /** @inheritDoc */
+  public void createSymlink(String target, String link, FsPermission dirPerms, 
+                            boolean createParent) 
+      throws IOException {
+    myMetrics.numcreateSymlinkOps.inc();
+    /* We enforce the MAX_PATH_LENGTH limit even though a symlink target 
+     * URI may refer to a non-HDFS file system. 
+     */
+    if (!checkPathLength(link)) {
+      throw new IOException("Symlink path exceeds " + MAX_PATH_LENGTH +
+                            " character limit");
+                            
+    }
+    if ("".equals(target)) {
+      throw new IOException("Invalid symlink target");
+    }
+    final UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+    namesystem.createSymlink(target, link,
+      new PermissionStatus(ugi.getUserName(), null, dirPerms), createParent);
+  }
+
+  /** @inheritDoc */
+  public String getLinkTarget(String path) throws IOException {
+    myMetrics.numgetLinkTargetOps.inc();
+    /* Resolves the first symlink in the given path, returning a
+     * new path consisting of the target of the symlink and any 
+     * remaining path components from the original path.
+     */
+    try {
+      HdfsFileStatus stat = namesystem.getFileInfo(path, false);
+      if (stat != null) {
+        // NB: getSymlink throws IOException if !stat.isSymlink() 
+        return stat.getSymlink();
+      }
+    } catch (UnresolvedPathException e) {
+      return e.getResolvedPath().toString();
+    } catch (UnresolvedLinkException e) {
+      // The NameNode should only throw an UnresolvedPathException
+      throw new AssertionError("UnresolvedLinkException thrown");
+    }
+    return null;
+  }
+
+
   ////////////////////////////////////////////////////////////////
   // DatanodeProtocol
   ////////////////////////////////////////////////////////////////
   /** 
    */
-  public DatanodeRegistration register(DatanodeRegistration nodeReg
-                                       ) throws IOException {
+  public DatanodeRegistration register(DatanodeRegistration nodeReg)
+      throws IOException {
     verifyVersion(nodeReg.getVersion());
     namesystem.registerDatanode(nodeReg);
       
@@ -1122,8 +1186,8 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
    * @throws IOException
    */
   private static boolean format(Configuration conf,
-                                boolean isConfirmationNeeded
-                                ) throws IOException {
+                                boolean isConfirmationNeeded)
+      throws IOException {
     Collection<URI> dirsToFormat = FSNamesystem.getNamespaceDirs(conf);
     Collection<URI> editDirsToFormat = 
                  FSNamesystem.getNamespaceEditsDirs(conf);
@@ -1236,8 +1300,8 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
                                           StartupOption.REGULAR.toString()));
   }
 
-  public static NameNode createNameNode(String argv[], 
-                                 Configuration conf) throws IOException {
+  public static NameNode createNameNode(String argv[], Configuration conf)
+      throws IOException {
     if (conf == null)
       conf = new HdfsConfiguration();
     StartupOption startOpt = parseArguments(argv);

+ 5 - 1
src/java/org/apache/hadoop/hdfs/server/namenode/metrics/NameNodeMetrics.java

@@ -67,6 +67,10 @@ public class NameNodeMetrics implements Updater {
                           new MetricsTimeVaryingInt("FileInfoOps", registry);
     public MetricsTimeVaryingInt numAddBlockOps = 
                           new MetricsTimeVaryingInt("AddBlockOps", registry);
+    public MetricsTimeVaryingInt numcreateSymlinkOps = 
+                          new MetricsTimeVaryingInt("CreateSymlinkOps", registry);
+    public MetricsTimeVaryingInt numgetLinkTargetOps = 
+                          new MetricsTimeVaryingInt("GetLinkTargetOps", registry);
 
     public MetricsTimeVaryingRate transactions =
                     new MetricsTimeVaryingRate("Transactions", registry, "Journal Transaction");
@@ -92,7 +96,7 @@ public class NameNodeMetrics implements Updater {
       String processName = nameNodeRole.toString();
       JvmMetrics.init(processName, sessionId);
 
-      // Now the Mbean for the name node - this alos registers the MBean
+      // Now the Mbean for the name node - this also registers the MBean
       namenodeActivityMBean = new NameNodeActivityMBean(registry);
       
       // Create a record for NameNode metrics

+ 11 - 5
src/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java

@@ -62,6 +62,7 @@ import org.apache.hadoop.io.WritableUtils;
  *      Username (String)
  *      Groupname (String)
  *      OctalPerms (short -> String)  // Modified in -19
+ *    Symlink (String) // added in -23
  * NumINodesUnderConstruction (int)
  * INodesUnderConstruction (count = NumINodesUnderConstruction)
  *  INodeUnderConstruction
@@ -96,7 +97,7 @@ import org.apache.hadoop.io.WritableUtils;
 class ImageLoaderCurrent implements ImageLoader {
   protected final DateFormat dateFormat = 
                                       new SimpleDateFormat("yyyy-MM-dd HH:mm");
-  private static int [] versions = {-16, -17, -18, -19, -20, -21, -22};
+  private static int [] versions = {-16, -17, -18, -19, -20, -21, -22, -23};
   private int imageVersion = 0;
 
   /* (non-Javadoc)
@@ -206,7 +207,8 @@ class ImageLoaderCurrent implements ImageLoader {
     v.visitEnclosingElement(ImageElement.BLOCKS,
                             ImageElement.NUM_BLOCKS, numBlocks);
     
-    if(numBlocks == -1) { // directory, no blocks to process
+    // directory or symlink, no blocks to process    
+    if(numBlocks == -1 || numBlocks == -2) { 
       v.leaveEnclosingElement(); // Blocks
       return;
     }
@@ -271,10 +273,14 @@ class ImageLoaderCurrent implements ImageLoader {
 
       processBlocks(in, v, numBlocks, skipBlocks);
 
-      if(numBlocks != 0) {
-        v.visit(ImageElement.NS_QUOTA, numBlocks <= 0 ? in.readLong() : -1);
+      // File or directory
+      if (numBlocks > 0 || numBlocks == -1) {
+        v.visit(ImageElement.NS_QUOTA, numBlocks == -1 ? in.readLong() : -1);
         if(imageVersion <= -18) // added in version -18
-          v.visit(ImageElement.DS_QUOTA, numBlocks <= 0 ? in.readLong() : -1);
+          v.visit(ImageElement.DS_QUOTA, numBlocks == -1 ? in.readLong() : -1);
+      }
+      if (imageVersion <= -23 && numBlocks == -2) {
+        v.visit(ImageElement.SYMLINK, Text.readString(in));
       }
 
       processPermission(in, v);

+ 1 - 0
src/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageVisitor.java

@@ -51,6 +51,7 @@ abstract class ImageVisitor {
     NS_QUOTA,
     DS_QUOTA,
     PERMISSIONS,
+    SYMLINK,
     NUM_INODES_UNDER_CONSTRUCTION,
     INODES_UNDER_CONSTRUCTION,
     INODE_UNDER_CONSTRUCTION,

+ 8 - 1
src/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/LsImageVisitor.java

@@ -44,6 +44,7 @@ class LsImageVisitor extends TextWriterImageVisitor {
   private long filesize;
   private String modTime;
   private String path;
+  private String linkTarget;
 
   private boolean inInode = false;
   final private StringBuilder sb = new StringBuilder();
@@ -62,7 +63,7 @@ class LsImageVisitor extends TextWriterImageVisitor {
    */
   private void newLine() {
     numBlocks = 0;
-    perms = username = group = path = "";
+    perms = username = group = path = linkTarget = "";
     filesize = 0l;
     replication = 0;
 
@@ -85,6 +86,9 @@ class LsImageVisitor extends TextWriterImageVisitor {
     sb.append(numBlocks < 0 ? "d" : "-");
     sb.append(perms);
 
+    if (0 != linkTarget.length()) {
+      path = path + " -> " + linkTarget; 
+    }
     formatter.format(lsStr, replication > 0 ? replication : "-",
                            username, group, filesize, modTime, path);
     sb.append("\n");
@@ -145,6 +149,9 @@ class LsImageVisitor extends TextWriterImageVisitor {
       case MODIFICATION_TIME:
         modTime = value;
         break;
+      case SYMLINK:
+        linkTarget = value;
+        break;
       default:
         // This is OK.  We're not looking for all the values.
         break;

+ 1 - 1
src/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewer.java

@@ -45,7 +45,7 @@ public class OfflineImageViewer {
     "\n" +
     "The oiv utility will attempt to parse correctly formed image files\n" +
     "and will abort fail with mal-formed image files. Currently the\n" +
-    "supports FSImage layout versions -16 through -22.\n" +
+    "supports FSImage layout versions -16 through -23.\n" +
     "\n" +
     "The tool works offline and does not require a running cluster in\n" +
     "order to process an image file.\n" +

+ 235 - 0
src/test/hdfs/org/apache/hadoop/fs/TestFcHdfsSymlink.java

@@ -0,0 +1,235 @@
+/**
+ * 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.*;
+import java.net.URI;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.FSConstants;
+import static org.junit.Assert.*;
+import org.junit.Test;
+import org.junit.BeforeClass;
+import org.junit.AfterClass;
+
+/**
+ * Test symbolic links using FileContext and Hdfs.
+ */
+public class TestFcHdfsSymlink extends FileContextSymlinkBaseTest {
+
+  private static MiniDFSCluster cluster;
+  
+  protected String getScheme() {
+    return "hdfs";
+  }
+
+  protected String testBaseDir1() {
+    return "/test1";
+  }
+  
+  protected String testBaseDir2() {
+    return "/test2";
+  }
+
+  protected URI testURI() {
+    return cluster.getURI();
+  }
+  
+  @BeforeClass
+  public static void testSetUp() throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, true);
+    conf.set(FsPermission.UMASK_LABEL, "000");
+    cluster = new MiniDFSCluster(conf, 1, true, null);
+    fc = FileContext.getFileContext(cluster.getURI());
+  }
+  
+  @AfterClass
+  public static void testTearDown() throws Exception {
+    cluster.shutdown();
+  }
+     
+  @Test
+  /** Link from Hdfs to LocalFs */
+  public void testLinkAcrossFileSystems() throws IOException {
+    Path localDir  = new Path("file:///tmp/test");
+    Path localFile = new Path("file:///tmp/test/file");
+    Path link      = new Path(testBaseDir1(), "linkToFile");
+    FileContext localFc = FileContext.getLocalFSFileContext();
+    localFc.delete(new Path("file:///tmp/test"), true);
+    localFc.mkdir(localDir, FileContext.DEFAULT_PERM, true);
+    localFc.setWorkingDirectory(localDir);
+    assertEquals(localDir, localFc.getWorkingDirectory());
+    createAndWriteFile(localFc, localFile);
+    fc.createSymlink(localFile, link, false);
+    readFile(link);
+    assertEquals(fileSize, fc.getFileStatus(link).getLen());
+  }
+  
+  @Test
+  /** Test access a symlink using AbstractFileSystem */
+  public void testAccessLinkFromAbstractFileSystem() throws IOException {
+    Path file = new Path(testBaseDir1(), "file");
+    Path link = new Path(testBaseDir1(), "linkToFile");
+    createAndWriteFile(file);
+    fc.createSymlink(file, link, false);
+    try {
+      AbstractFileSystem afs = fc.getDefaultFileSystem();
+      afs.open(link);
+      fail("Opened a link using AFS");
+    } catch (UnresolvedLinkException x) {
+      // Expected
+    }
+  }
+
+  @Test
+  /** Test create symlink to / */
+  public void testCreateLinkToSlash() throws IOException {
+    Path dir  = new Path(testBaseDir1());
+    Path file = new Path(testBaseDir1(), "file");
+    Path link = new Path(testBaseDir1(), "linkToSlash");
+    Path fileViaLink = new Path(testBaseDir1()+"/linkToSlash"+
+                                testBaseDir1()+"/file");
+    createAndWriteFile(file);
+    fc.setWorkingDirectory(dir);
+    fc.createSymlink(new Path("/"), link, false);
+    readFile(fileViaLink);
+    assertEquals(fileSize, fc.getFileStatus(fileViaLink).getLen());
+    // Ditto when using another file context since the file system
+    // for the slash is resolved according to the link's parent.
+    FileContext localFc = FileContext.getLocalFSFileContext();
+    Path linkQual = new Path(cluster.getURI().toString(), fileViaLink); 
+    assertEquals(fileSize, localFc.getFileStatus(linkQual).getLen());    
+  }
+  
+  
+  @Test
+  /** setPermission affects the target not the link */
+  public void testSetPermissionAffectsTarget() throws IOException {    
+    Path file       = new Path(testBaseDir1(), "file");
+    Path dir        = new Path(testBaseDir2());
+    Path linkToFile = new Path(testBaseDir1(), "linkToFile");
+    Path linkToDir  = new Path(testBaseDir1(), "linkToDir");
+    createAndWriteFile(file);
+    fc.createSymlink(file, linkToFile, false);
+    fc.createSymlink(dir, linkToDir, false);
+    
+    // Changing the permissions using the link does not modify
+    // the permissions of the link..
+    FsPermission perms = fc.getFileLinkStatus(linkToFile).getPermission();
+    fc.setPermission(linkToFile, new FsPermission((short)0664));
+    fc.setOwner(linkToFile, "user", "group");
+    assertEquals(perms, fc.getFileLinkStatus(linkToFile).getPermission());
+    // but the file's permissions were adjusted appropriately
+    FileStatus stat = fc.getFileStatus(file);
+    assertEquals(0664, stat.getPermission().toShort()); 
+    assertEquals("user", stat.getOwner());
+    assertEquals("group", stat.getGroup());
+    // Getting the file's permissions via the link is the same
+    // as getting the permissions directly.
+    assertEquals(stat.getPermission(), 
+                 fc.getFileStatus(linkToFile).getPermission());
+
+    // Ditto for a link to a directory
+    perms = fc.getFileLinkStatus(linkToDir).getPermission();
+    fc.setPermission(linkToDir, new FsPermission((short)0664));
+    fc.setOwner(linkToDir, "user", "group");
+    assertEquals(perms, fc.getFileLinkStatus(linkToDir).getPermission());
+    stat = fc.getFileStatus(dir);
+    assertEquals(0664, stat.getPermission().toShort()); 
+    assertEquals("user", stat.getOwner());
+    assertEquals("group", stat.getGroup());
+    assertEquals(stat.getPermission(), 
+                 fc.getFileStatus(linkToDir).getPermission());
+  }  
+
+  @Test
+  /** Create a symlink using a path with scheme but no authority */
+  public void testCreateWithPartQualPathFails() throws IOException {
+    Path fileWoAuth = new Path("hdfs:///test/file");
+    Path linkWoAuth = new Path("hdfs:///test/link");
+    try {
+      createAndWriteFile(fileWoAuth);
+      fail("HDFS requires URIs with schemes have an authority");
+    } catch (RuntimeException e) {
+      // Expected
+    }
+    try {
+      fc.createSymlink(new Path("foo"), linkWoAuth, false);
+      fail("HDFS requires URIs with schemes have an authority");
+    } catch (RuntimeException e) {
+      // Expected
+    }
+  }
+
+  @Test
+  /** setReplication affects the target not the link */  
+  public void testSetReplication() throws IOException {
+    Path file = new Path(testBaseDir1(), "file");
+    Path link = new Path(testBaseDir1(), "linkToFile");
+    createAndWriteFile(file);
+    fc.createSymlink(file, link, false);
+    fc.setReplication(link, (short)2);
+    assertEquals(0, fc.getFileLinkStatus(link).getReplication());
+    assertEquals(2, fc.getFileStatus(link).getReplication());      
+    assertEquals(2, fc.getFileStatus(file).getReplication());
+  }
+  
+  @Test
+  /** Test create symlink with a max len name */
+  public void testCreateLinkMaxPathLink() throws IOException {
+    Path dir  = new Path(testBaseDir1());
+    Path file = new Path(testBaseDir1(), "file");
+    final int maxPathLen = FSConstants.MAX_PATH_LENGTH;
+    final int dirLen     = dir.toString().length() + 1;
+    int   len            = maxPathLen - dirLen;
+    
+    // Build a MAX_PATH_LENGTH path
+    StringBuilder sb = new StringBuilder("");
+    for (int i = 0; i < (len / 10); i++) {
+      sb.append("0123456789");
+    }
+    for (int i = 0; i < (len % 10); i++) {
+      sb.append("x");
+    }
+    Path link = new Path(sb.toString());
+    assertEquals(maxPathLen, dirLen + link.toString().length()); 
+    
+    // Check that it works
+    createAndWriteFile(file);
+    fc.setWorkingDirectory(dir);
+    fc.createSymlink(file, link, false);
+    readFile(link);
+    
+    // Now modify the path so it's too large
+    link = new Path(sb.toString()+"x");
+    try {
+      fc.createSymlink(file, link, false);
+      fail("Path name should be too long");
+    } catch (IOException x) {
+      // Expected
+    }
+  } 
+}

+ 8 - 0
src/test/hdfs/org/apache/hadoop/hdfs/TestDFSClientRetries.java

@@ -250,6 +250,8 @@ public class TestDFSClientRetries extends TestCase {
 
     public HdfsFileStatus getFileInfo(String src) throws IOException { return null; }
 
+    public HdfsFileStatus getFileLinkInfo(String src) throws IOException { return null; }
+
     public ContentSummary getContentSummary(String path) throws IOException { return null; }
 
     public void setQuota(String path, long namespaceQuota, long diskspaceQuota) throws IOException {}
@@ -258,6 +260,12 @@ public class TestDFSClientRetries extends TestCase {
 
     public void setTimes(String src, long mtime, long atime) throws IOException {}
 
+    public void createSymlink(String target, String newpath, 
+        FsPermission dirPerm, boolean createPath) 
+        throws IOException, UnresolvedLinkException {}
+
+    public String getLinkTarget(String src) throws IOException { return null; }
+
     @Override public LocatedBlock updateBlockForPipeline(Block block, 
         String clientName) throws IOException { return null; }
 

+ 3 - 1
src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestFsck.java

@@ -378,7 +378,9 @@ public class TestFsck extends TestCase {
       DFSTestUtil.waitReplication(fs, filePath, (short)1);
       
       // intentionally corrupt NN data structure
-      INodeFile node = (INodeFile)cluster.getNamesystem().dir.rootDir.getNode(fileName);
+      INodeFile node = 
+        (INodeFile)cluster.getNamesystem().dir.rootDir.getNode(fileName,
+                                                               true);
       assertEquals(node.blocks.length, 1);
       node.blocks[0].setNumBytes(-1L);  // set the block length to be negative