You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by jl...@apache.org on 2013/07/30 16:43:00 UTC

svn commit: r1508471 - in /hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common: ./ src/main/java/org/apache/hadoop/fs/ src/main/java/org/apache/hadoop/fs/local/ src/main/java/org/apache/hadoop/util/ src/test/java/org/apache/hadoop/fs/

Author: jlowe
Date: Tue Jul 30 14:42:59 2013
New Revision: 1508471

URL: http://svn.apache.org/r1508471
Log:
Revert HADOOP-9652 from branch-2

Removed:
    hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Stat.java
    hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestStat.java
Modified:
    hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/CHANGES.txt
    hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DelegateToFileSystem.java
    hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HardLink.java
    hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
    hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/local/RawLocalFs.java
    hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
    hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFileSystem.java
    hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestSymlinkLocalFS.java

Modified: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/CHANGES.txt?rev=1508471&r1=1508470&r2=1508471&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/CHANGES.txt (original)
+++ hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/CHANGES.txt Tue Jul 30 14:42:59 2013
@@ -33,9 +33,6 @@ Release 2.3.0 - UNRELEASED
     HADOOP-9582. Non-existent file to "hadoop fs -conf" doesn't throw error
     (Ashwin Shankar via jlowe)
 
-    HADOOP-9652.  RawLocalFs#getFileLinkStatus does not fill in the link owner
-    and mode.  (Andrew Wang via Colin Patrick McCabe)
-
 Release 2.1.1-beta - UNRELEASED
 
   INCOMPATIBLE CHANGES

Modified: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DelegateToFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DelegateToFileSystem.java?rev=1508471&r1=1508470&r2=1508471&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DelegateToFileSystem.java (original)
+++ hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DelegateToFileSystem.java Tue Jul 30 14:42:59 2013
@@ -113,14 +113,7 @@ public abstract class DelegateToFileSyst
 
   @Override
   public FileStatus getFileLinkStatus(final Path f) throws IOException {
-    FileStatus status = fsImpl.getFileLinkStatus(f);
-    // FileSystem#getFileLinkStatus qualifies the link target
-    // AbstractFileSystem needs to return it plain since it's qualified
-    // in FileContext, so re-get and set the plain target
-    if (status.isSymlink()) {
-      status.setSymlink(fsImpl.getLinkTarget(f));
-    }
-    return status;
+    return getFileStatus(f);
   }
 
   @Override
@@ -206,18 +199,22 @@ public abstract class DelegateToFileSyst
 
   @Override
   public boolean supportsSymlinks() {
-    return fsImpl.supportsSymlinks();
+    return false;
   }  
   
   @Override
   public void createSymlink(Path target, Path link, boolean createParent) 
       throws IOException { 
-    fsImpl.createSymlink(target, link, createParent);
+    throw new IOException("File system does not support symlinks");
   } 
   
   @Override
   public Path getLinkTarget(final Path f) throws IOException {
-    return fsImpl.getLinkTarget(f);
+    /* We should never get here. Any file system that threw an 
+     * UnresolvedLinkException, causing this function to be called,
+     * should override getLinkTarget. 
+     */
+    throw new AssertionError();
   }
 
   @Override //AbstractFileSystem

Modified: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HardLink.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HardLink.java?rev=1508471&r1=1508470&r2=1508471&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HardLink.java (original)
+++ hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HardLink.java Tue Jul 30 14:42:59 2013
@@ -26,7 +26,6 @@ import java.io.InputStreamReader;
 import java.util.Arrays;
 
 import org.apache.hadoop.util.Shell;
-import org.apache.hadoop.util.Shell.OSType;
 
 /**
  * Class for creating hardlinks.
@@ -42,7 +41,15 @@ import org.apache.hadoop.util.Shell.OSTy
  */
 public class HardLink { 
 
-  public static OSType osType = Shell.osType;
+  public enum OSType {
+    OS_TYPE_UNIX,
+    OS_TYPE_WIN,
+    OS_TYPE_SOLARIS,
+    OS_TYPE_MAC,
+    OS_TYPE_FREEBSD
+  }
+  
+  public static OSType osType;
   private static HardLinkCommandGetter getHardLinkCommand;
   
   public final LinkStats linkStats; //not static
@@ -50,11 +57,12 @@ public class HardLink { 
   //initialize the command "getters" statically, so can use their 
   //methods without instantiating the HardLink object
   static { 
+    osType = getOSType();
     if (osType == OSType.OS_TYPE_WIN) {
       // Windows
       getHardLinkCommand = new HardLinkCGWin();
     } else {
-      // Unix or Linux
+      // Unix
       getHardLinkCommand = new HardLinkCGUnix();
       //override getLinkCountCommand for the particular Unix variant
       //Linux is already set as the default - {"stat","-c%h", null}
@@ -71,7 +79,27 @@ public class HardLink { 
   public HardLink() {
     linkStats = new LinkStats();
   }
-
+  
+  static private OSType getOSType() {
+    String osName = System.getProperty("os.name");
+    if (Shell.WINDOWS) {
+      return OSType.OS_TYPE_WIN;
+    }
+    else if (osName.contains("SunOS") 
+            || osName.contains("Solaris")) {
+       return OSType.OS_TYPE_SOLARIS;
+    }
+    else if (osName.contains("Mac")) {
+       return OSType.OS_TYPE_MAC;
+    }
+    else if (osName.contains("FreeBSD")) {
+       return OSType.OS_TYPE_FREEBSD;
+    }
+    else {
+      return OSType.OS_TYPE_UNIX;
+    }
+  }
+  
   /**
    * This abstract class bridges the OS-dependent implementations of the 
    * needed functionality for creating hardlinks and querying link counts.

Modified: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java?rev=1508471&r1=1508470&r2=1508471&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java (original)
+++ hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java Tue Jul 30 14:42:59 2013
@@ -379,9 +379,6 @@ public class RawLocalFileSystem extends 
       throw new FileNotFoundException("File " + f + " does not exist");
     }
     if (localf.isFile()) {
-      if (!Shell.WINDOWS) {
-        return new FileStatus[] { getFileStatus(f) };
-      }
       return new FileStatus[] {
         new RawLocalFileStatus(localf, getDefaultBlockSize(f), this) };
     }
@@ -513,10 +510,6 @@ public class RawLocalFileSystem extends 
   
   @Override
   public FileStatus getFileStatus(Path f) throws IOException {
-    if (!Shell.WINDOWS) {
-      return getFileLinkStatusInternal(f, true);
-    }
-
     File path = pathToFile(f);
     if (path.exists()) {
       return new RawLocalFileStatus(pathToFile(f), getDefaultBlockSize(f), this);
@@ -525,7 +518,6 @@ public class RawLocalFileSystem extends 
     }
   }
 
-  @Deprecated
   static class RawLocalFileStatus extends FileStatus {
     /* We can add extra fields here. It breaks at least CopyFiles.FilePair().
      * We recognize if the information is already loaded by check if
@@ -707,7 +699,6 @@ public class RawLocalFileSystem extends 
    * the given path does not refer to a symlink or there is an error
    * accessing the symlink.
    */
-  @Deprecated
   private String readLink(Path p) {
     /* NB: Use readSymbolicLink in java.nio.file.Path once available. Could
      * use getCanonicalPath in File to get the target of the symlink but that
@@ -728,12 +719,7 @@ public class RawLocalFileSystem extends 
    */
   @Override
   public FileStatus getFileLinkStatus(final Path f) throws IOException {
-    FileStatus fi;
-    if (!Shell.WINDOWS) {
-      fi = getFileLinkStatusInternal(f, false);
-    } else {
-      fi = getFileLinkStatusInternal(f);
-    }
+    FileStatus fi = getFileLinkStatusInternal(f);
     // getFileLinkStatus is supposed to return a symlink with a
     // qualified path
     if (fi.isSymlink()) {
@@ -744,12 +730,6 @@ public class RawLocalFileSystem extends 
     return fi;
   }
 
-  /**
-   * Deprecated. Remains for windows support. Should be removed in favor of
-   * {@link #getFileLinkStatusInternal(Path, boolean)} when {@link Stat} gains
-   * support for windows.
-   */
-  @Deprecated
   private FileStatus getFileLinkStatusInternal(final Path f) throws IOException {
     String target = readLink(f);
 
@@ -787,17 +767,9 @@ public class RawLocalFileSystem extends 
     }
   }
 
-  private FileStatus getFileLinkStatusInternal(final Path f,
-      boolean dereference) throws IOException {
-    checkPath(f);
-    Stat stat = new Stat(f, getDefaultBlockSize(f), dereference, this);
-    FileStatus status = stat.getFileStatus();
-    return status;
-  }
-
   @Override
   public Path getLinkTarget(Path f) throws IOException {
-    FileStatus fi = getFileLinkStatusInternal(f, false);
+    FileStatus fi = getFileLinkStatusInternal(f);
     // return an unqualified symlink target
     return fi.getSymlink();
   }

Modified: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/local/RawLocalFs.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/local/RawLocalFs.java?rev=1508471&r1=1508470&r2=1508471&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/local/RawLocalFs.java (original)
+++ hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/local/RawLocalFs.java Tue Jul 30 14:42:59 2013
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.fs.local;
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.URI;
 import java.net.URISyntaxException;
@@ -26,9 +27,13 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.AbstractFileSystem;
 import org.apache.hadoop.fs.DelegateToFileSystem;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FsConstants;
 import org.apache.hadoop.fs.FsServerDefaults;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RawLocalFileSystem;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.util.Shell;
 
 /**
  * The RawLocalFs implementation of AbstractFileSystem.
@@ -67,6 +72,92 @@ public class RawLocalFs extends Delegate
     return LocalConfigKeys.getServerDefaults();
   }
   
+  @Override
+  public boolean supportsSymlinks() {
+    return true;
+  }  
+  
+  @Override
+  public void createSymlink(Path target, Path link, boolean createParent) 
+      throws IOException {
+    final String targetScheme = target.toUri().getScheme();
+    if (targetScheme != null && !"file".equals(targetScheme)) {
+      throw new IOException("Unable to create symlink to non-local file "+
+                            "system: "+target.toString());
+    }
+    if (createParent) {
+      mkdir(link.getParent(), FsPermission.getDirDefault(), true);
+    }
+    // NB: Use createSymbolicLink in java.nio.file.Path once available
+    try {
+      Shell.execCommand(Shell.getSymlinkCommand(
+        Path.getPathWithoutSchemeAndAuthority(target).toString(),
+        Path.getPathWithoutSchemeAndAuthority(link).toString()));
+    } catch (IOException x) {
+      throw new IOException("Unable to create symlink: "+x.getMessage());
+    }
+  }
+
+  /** 
+   * Returns the target of the given symlink. Returns the empty string if  
+   * the given path does not refer to a symlink or there is an error 
+   * acessing the symlink.
+   */
+  private String readLink(Path p) {
+    /* NB: Use readSymbolicLink in java.nio.file.Path once available. Could
+     * use getCanonicalPath in File to get the target of the symlink but that 
+     * does not indicate if the given path refers to a symlink.
+     */
+    try {
+      final String path = p.toUri().getPath();
+      return Shell.execCommand(Shell.READ_LINK_COMMAND, path).trim(); 
+    } catch (IOException x) {
+      return "";
+    }
+  }
+  
+  /**
+   * Return a FileStatus representing the given path. If the path refers 
+   * to a symlink return a FileStatus representing the link rather than
+   * the object the link refers to.
+   */
+  @Override
+  public FileStatus getFileLinkStatus(final Path f) throws IOException {
+    String target = readLink(f);
+    try {
+      FileStatus fs = getFileStatus(f);
+      // If f refers to a regular file or directory      
+      if ("".equals(target)) {
+        return fs;
+      }
+      // Otherwise f refers to a symlink
+      return new FileStatus(fs.getLen(), 
+          false,
+          fs.getReplication(), 
+          fs.getBlockSize(),
+          fs.getModificationTime(),
+          fs.getAccessTime(),
+          fs.getPermission(),
+          fs.getOwner(),
+          fs.getGroup(),
+          new Path(target),
+          f);
+    } catch (FileNotFoundException e) {
+      /* The exists method in the File class returns false for dangling 
+       * links so we can get a FileNotFoundException for links that exist.
+       * It's also possible that we raced with a delete of the link. Use
+       * the readBasicFileAttributes method in java.nio.file.attributes 
+       * when available.
+       */
+      if (!"".equals(target)) {
+        return new FileStatus(0, false, 0, 0, 0, 0, FsPermission.getDefault(), 
+            "", "", new Path(target), f);        
+      }
+      // f refers to a file or directory that does not exist
+      throw e;
+    }
+  }
+  
    @Override
    public boolean isValidName(String src) {
      // Different local file systems have different validation rules.  Skip
@@ -74,4 +165,14 @@ public class RawLocalFs extends Delegate
      // RawLocalFileSystem.
      return true;
    }
+  
+  @Override
+  public Path getLinkTarget(Path f) throws IOException {
+    /* We should never get here. Valid local links are resolved transparently
+     * by the underlying local file system and accessing a dangling link will 
+     * result in an IOException, not an UnresolvedLinkException, so FileContext
+     * should never call this function.
+     */
+    throw new AssertionError();
+  }
 }

Modified: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java?rev=1508471&r1=1508470&r2=1508471&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java (original)
+++ hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java Tue Jul 30 14:42:59 2013
@@ -58,33 +58,6 @@ abstract public class Shell {
   /** Windows CreateProcess synchronization object */
   public static final Object WindowsProcessLaunchLock = new Object();
 
-  // OSType detection
-
-  public enum OSType {
-    OS_TYPE_LINUX,
-    OS_TYPE_WIN,
-    OS_TYPE_SOLARIS,
-    OS_TYPE_MAC,
-    OS_TYPE_FREEBSD
-  }
-
-  public static final OSType osType = getOSType();
-
-  static private OSType getOSType() {
-    String osName = System.getProperty("os.name");
-    if (Shell.WINDOWS) {
-      return OSType.OS_TYPE_WIN;
-    } else if (osName.contains("SunOS") || osName.contains("Solaris")) {
-      return OSType.OS_TYPE_SOLARIS;
-    } else if (osName.contains("Mac")) {
-      return OSType.OS_TYPE_MAC;
-    } else if (osName.contains("FreeBSD")) {
-      return OSType.OS_TYPE_FREEBSD;
-    } else {
-      return OSType.OS_TYPE_LINUX;
-    }
-  }
-
   /** a Unix command to get the current user's groups list */
   public static String[] getGroupsCommand() {
     return (WINDOWS)? new String[]{"cmd", "/c", "groups"}

Modified: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFileSystem.java?rev=1508471&r1=1508470&r2=1508471&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFileSystem.java (original)
+++ hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFileSystem.java Tue Jul 30 14:42:59 2013
@@ -358,12 +358,12 @@ public class TestLocalFileSystem {
 
     FileStatus status = fileSys.getFileStatus(path);
     assertTrue("check we're actually changing something", newModTime != status.getModificationTime());
-    long accessTime = status.getAccessTime();
+    assertEquals(0, status.getAccessTime());
 
     fileSys.setTimes(path, newModTime, -1);
     status = fileSys.getFileStatus(path);
     assertEquals(newModTime, status.getModificationTime());
-    assertEquals(accessTime, status.getAccessTime());
+    assertEquals(0, status.getAccessTime());
   }
 
   /**

Modified: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestSymlinkLocalFS.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestSymlinkLocalFS.java?rev=1508471&r1=1508470&r2=1508471&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestSymlinkLocalFS.java (original)
+++ hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestSymlinkLocalFS.java Tue Jul 30 14:42:59 2013
@@ -30,7 +30,6 @@ import java.net.URI;
 import java.net.URISyntaxException;
 
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.security.UserGroupInformation;
 import org.junit.Test;
 
 /**
@@ -124,7 +123,6 @@ abstract public class TestSymlinkLocalFS
     Path fileAbs  = new Path(testBaseDir1()+"/file");
     Path fileQual = new Path(testURI().toString(), fileAbs);
     Path link     = new Path(testBaseDir1()+"/linkToFile");
-    Path linkQual = new Path(testURI().toString(), link.toString());
     wrapper.createSymlink(fileAbs, link, false);
     // Deleting the link using FileContext currently fails because
     // resolve looks up LocalFs rather than RawLocalFs for the path 
@@ -142,15 +140,18 @@ abstract public class TestSymlinkLocalFS
       // Expected. File's exists method returns false for dangling links
     }
     // We can stat a dangling link
-    UserGroupInformation user = UserGroupInformation.getCurrentUser();
     FileStatus fsd = wrapper.getFileLinkStatus(link);
     assertEquals(fileQual, fsd.getSymlink());
     assertTrue(fsd.isSymlink());
     assertFalse(fsd.isDirectory());
-    assertEquals(user.getUserName(), fsd.getOwner());
-    // Compare against user's primary group
-    assertEquals(user.getGroupNames()[0], fsd.getGroup());
-    assertEquals(linkQual, fsd.getPath());
+    assertEquals("", fsd.getOwner());
+    assertEquals("", fsd.getGroup());
+    assertEquals(link, fsd.getPath());
+    assertEquals(0, fsd.getLen());
+    assertEquals(0, fsd.getBlockSize());
+    assertEquals(0, fsd.getReplication());
+    assertEquals(0, fsd.getAccessTime());
+    assertEquals(FsPermission.getDefault(), fsd.getPermission());
     // Accessing the link 
     try {
       readFile(link);