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 cm...@apache.org on 2013/07/27 01:09:56 UTC

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

Author: cmccabe
Date: Fri Jul 26 23:09:55 2013
New Revision: 1507509

URL: http://svn.apache.org/r1507509
Log:
HADOOP-9652.  RawLocalFs#getFileLinkStatus does not fill in the link owner and mode.  (Andrew Wang via Colin Patrick McCabe)

Added:
    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/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/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=1507509&r1=1507508&r2=1507509&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 Fri Jul 26 23:09:55 2013
@@ -113,7 +113,14 @@ public abstract class DelegateToFileSyst
 
   @Override
   public FileStatus getFileLinkStatus(final Path f) throws IOException {
-    return getFileStatus(f);
+    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;
   }
 
   @Override
@@ -199,22 +206,18 @@ public abstract class DelegateToFileSyst
 
   @Override
   public boolean supportsSymlinks() {
-    return false;
+    return fsImpl.supportsSymlinks();
   }  
   
   @Override
   public void createSymlink(Path target, Path link, boolean createParent) 
       throws IOException { 
-    throw new IOException("File system does not support symlinks");
+    fsImpl.createSymlink(target, link, createParent);
   } 
   
   @Override
   public Path getLinkTarget(final Path f) throws IOException {
-    /* We should never get here. Any file system that threw an 
-     * UnresolvedLinkException, causing this function to be called,
-     * should override getLinkTarget. 
-     */
-    throw new AssertionError();
+    return fsImpl.getLinkTarget(f);
   }
 
   @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=1507509&r1=1507508&r2=1507509&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 Fri Jul 26 23:09:55 2013
@@ -26,6 +26,7 @@ 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.
@@ -41,15 +42,7 @@ import org.apache.hadoop.util.Shell;
  */
 public class HardLink { 
 
-  public enum OSType {
-    OS_TYPE_UNIX,
-    OS_TYPE_WIN,
-    OS_TYPE_SOLARIS,
-    OS_TYPE_MAC,
-    OS_TYPE_FREEBSD
-  }
-  
-  public static OSType osType;
+  public static OSType osType = Shell.osType;
   private static HardLinkCommandGetter getHardLinkCommand;
   
   public final LinkStats linkStats; //not static
@@ -57,12 +50,11 @@ 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
+      // Unix or Linux
       getHardLinkCommand = new HardLinkCGUnix();
       //override getLinkCountCommand for the particular Unix variant
       //Linux is already set as the default - {"stat","-c%h", null}
@@ -79,27 +71,7 @@ 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=1507509&r1=1507508&r2=1507509&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 Fri Jul 26 23:09:55 2013
@@ -379,6 +379,9 @@ 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) };
     }
@@ -510,6 +513,10 @@ 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);
@@ -518,6 +525,7 @@ 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
@@ -699,6 +707,7 @@ 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
@@ -719,7 +728,12 @@ public class RawLocalFileSystem extends 
    */
   @Override
   public FileStatus getFileLinkStatus(final Path f) throws IOException {
-    FileStatus fi = getFileLinkStatusInternal(f);
+    FileStatus fi;
+    if (!Shell.WINDOWS) {
+      fi = getFileLinkStatusInternal(f, false);
+    } else {
+      fi = getFileLinkStatusInternal(f);
+    }
     // getFileLinkStatus is supposed to return a symlink with a
     // qualified path
     if (fi.isSymlink()) {
@@ -730,6 +744,12 @@ 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);
 
@@ -767,9 +787,17 @@ 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);
+    FileStatus fi = getFileLinkStatusInternal(f, false);
     // return an unqualified symlink target
     return fi.getSymlink();
   }

Added: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Stat.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Stat.java?rev=1507509&view=auto
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Stat.java (added)
+++ hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Stat.java Fri Jul 26 23:09:55 2013
@@ -0,0 +1,159 @@
+/**
+ * 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.BufferedReader;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.NoSuchElementException;
+import java.util.StringTokenizer;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.util.Shell.OSType;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Wrapper for the Unix stat(1) command. Used to workaround the lack of 
+ * lstat(2) in Java 6.
+ */
+@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
+@InterfaceStability.Evolving
+public class Stat extends Shell {
+
+  private final Path original;
+  private final Path qualified;
+  private final Path path;
+  private final long blockSize;
+  private final boolean dereference;
+
+  private FileStatus stat;
+  
+  public Stat(Path path, long blockSize, boolean deref, FileSystem fs)
+      throws IOException {
+    super();
+    // Original path
+    this.original = path;
+    // Qualify with working directory and scheme/auth
+    this.qualified = original.makeQualified(
+        fs.getUri(), fs.getWorkingDirectory());
+    // Strip back down to a plain path
+    this.path = new Path(qualified.toUri().getPath());
+    this.blockSize = blockSize;
+    this.dereference = deref;
+  }
+
+  public FileStatus getFileStatus() throws IOException {
+    run();
+    return stat;
+  }
+
+  @VisibleForTesting
+  FileStatus getFileStatusForTesting() {
+    return stat;
+  }
+
+  @Override
+  protected String[] getExecString() {
+    String derefFlag = "-";
+    if (dereference) {
+      derefFlag = "-L";
+    }
+    if (osType == OSType.OS_TYPE_LINUX) {
+      return new String[] {
+          "bash", "-c",
+          "exec 'stat' '" + derefFlag + "c' '%s,%F,%Y,%X,%a,%U,%G,%N' '"
+              + path + "' 2>&1" };
+    } else if (osType == OSType.OS_TYPE_FREEBSD) {
+      return new String[] {
+          "bash", "-c",
+          "exec 'stat' '" + derefFlag + "f' '%z,%HT,%m,%a,%Op,%Su,%Sg,`link\' -> `%Y\'' '"
+              + path + "' 2>&1" };
+    } else {
+      throw new UnsupportedOperationException(
+          "stat is not supported on this platform");
+    }
+  }
+
+  @Override
+  protected void parseExecResult(BufferedReader lines) throws IOException {
+    // Reset stat
+    stat = null;
+
+    String line = lines.readLine();
+    if (line == null) {
+      throw new IOException("Unable to stat path: " + original);
+    }
+    if (line.endsWith("No such file or directory") ||
+        line.endsWith("Not a directory")) {
+      throw new FileNotFoundException("File " + original + " does not exist");
+    }
+    if (line.endsWith("Too many levels of symbolic links")) {
+      throw new IOException("Possible cyclic loop while following symbolic" +
+          " link " + original);
+    }
+    // 6,symbolic link,6,1373584236,1373584236,lrwxrwxrwx,andrew,andrew,`link' -> `target'
+    StringTokenizer tokens = new StringTokenizer(line, ",");
+    try {
+      long length = Long.parseLong(tokens.nextToken());
+      boolean isDir = tokens.nextToken().equalsIgnoreCase("directory") ? true
+          : false;
+      // Convert from seconds to milliseconds
+      long modTime = Long.parseLong(tokens.nextToken())*1000;
+      long accessTime = Long.parseLong(tokens.nextToken())*1000;
+      // FsPermissions only supports exactly 3 octal digits
+      // Need to pad up and trim down
+      String octalPerms = tokens.nextToken();
+      while (octalPerms.length() < 3) {
+        octalPerms = "0" + octalPerms;
+      }
+      octalPerms = octalPerms.substring(octalPerms.length()-3);
+      FsPermission perms = new FsPermission(octalPerms);
+      String owner = tokens.nextToken();
+      String group = tokens.nextToken();
+      String symStr = tokens.nextToken();
+      // 'notalink'
+      // 'link' -> `target'
+      // '' -> ''
+      Path symlink = null;
+      StringTokenizer symTokens = new StringTokenizer(symStr, "`");
+      symTokens.nextToken();
+      try {
+        String target = symTokens.nextToken();
+        target = target.substring(0, target.length()-1);
+        if (!target.isEmpty()) {
+          symlink = new Path(target);
+        }
+      } catch (NoSuchElementException e) {
+        // null if not a symlink
+      }
+      // Set stat
+      stat = new FileStatus(length, isDir, 1, blockSize, modTime, accessTime,
+          perms, owner, group, symlink, qualified);
+      System.out.println(line);
+      System.out.println(stat.toString());
+    } catch (NumberFormatException e) {
+      throw new IOException("Unexpected stat output: " + line, e);
+    } catch (NoSuchElementException e) {
+      throw new IOException("Unexpected stat output: " + line, e);
+    }
+  }
+}

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=1507509&r1=1507508&r2=1507509&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 Fri Jul 26 23:09:55 2013
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.fs.local;
 
-import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.URI;
 import java.net.URISyntaxException;
@@ -27,13 +26,9 @@ 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.
@@ -72,92 +67,6 @@ 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
@@ -165,14 +74,4 @@ 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=1507509&r1=1507508&r2=1507509&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 Fri Jul 26 23:09:55 2013
@@ -58,6 +58,33 @@ 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=1507509&r1=1507508&r2=1507509&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 Fri Jul 26 23:09:55 2013
@@ -358,12 +358,12 @@ public class TestLocalFileSystem {
 
     FileStatus status = fileSys.getFileStatus(path);
     assertTrue("check we're actually changing something", newModTime != status.getModificationTime());
-    assertEquals(0, status.getAccessTime());
+    long accessTime = status.getAccessTime();
 
     fileSys.setTimes(path, newModTime, -1);
     status = fileSys.getFileStatus(path);
     assertEquals(newModTime, status.getModificationTime());
-    assertEquals(0, status.getAccessTime());
+    assertEquals(accessTime, status.getAccessTime());
   }
 
   /**

Added: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestStat.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestStat.java?rev=1507509&view=auto
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestStat.java (added)
+++ hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestStat.java Fri Jul 26 23:09:55 2013
@@ -0,0 +1,112 @@
+/**
+ * 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 static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.BufferedReader;
+import java.io.FileNotFoundException;
+import java.io.StringReader;
+
+import org.apache.hadoop.conf.Configuration;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestStat {
+
+  private static Stat stat;
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    stat = new Stat(new Path("/dummypath"),
+        4096l, false, FileSystem.get(new Configuration()));
+  }
+
+  private class StatOutput {
+    final String doesNotExist;
+    final String directory;
+    final String file;
+    final String symlink;
+
+    StatOutput(String doesNotExist, String directory, String file,
+        String symlink) {
+      this.doesNotExist = doesNotExist;
+      this.directory = directory;
+      this.file = file;
+      this.symlink = symlink;
+    }
+
+    void test() throws Exception {
+      BufferedReader br;
+      FileStatus status;
+
+      try {
+        br = new BufferedReader(new StringReader(doesNotExist));
+        stat.parseExecResult(br);
+      } catch (FileNotFoundException e) {
+        // expected
+      }
+
+      br = new BufferedReader(new StringReader(directory));
+      stat.parseExecResult(br);
+      status = stat.getFileStatusForTesting();
+      assertTrue(status.isDirectory());
+
+      br = new BufferedReader(new StringReader(file));
+      stat.parseExecResult(br);
+      status = stat.getFileStatusForTesting();
+      assertTrue(status.isFile());
+
+      br = new BufferedReader(new StringReader(symlink));
+      stat.parseExecResult(br);
+      status = stat.getFileStatusForTesting();
+      assertTrue(status.isSymlink());
+    }
+  }
+
+  @Test(timeout=10000)
+  public void testStatLinux() throws Exception {
+    StatOutput linux = new StatOutput(
+        "stat: cannot stat `watermelon': No such file or directory",
+        "4096,directory,1373584236,1373586485,755,andrew,root,`.'",
+        "0,regular empty file,1373584228,1373584228,644,andrew,andrew,`target'",
+        "6,symbolic link,1373584236,1373584236,777,andrew,andrew,`link' -> `target'");
+    linux.test();
+  }
+
+  @Test(timeout=10000)
+  public void testStatFreeBSD() throws Exception {
+    StatOutput freebsd = new StatOutput(
+        "stat: symtest/link: stat: No such file or directory",
+        "512,Directory,1373583695,1373583669,40755,awang,awang,`link' -> `'",
+        "0,Regular File,1373508937,1373508937,100644,awang,awang,`link' -> `'",
+        "6,Symbolic Link,1373508941,1373508941,120755,awang,awang,`link' -> `target'");
+    freebsd.test();
+  }
+
+  @Test(timeout=10000)
+  public void testStatFileNotFound() throws Exception {
+    try {
+      stat.getFileStatus();
+      fail("Expected FileNotFoundException");
+    } catch (FileNotFoundException e) {
+      // expected
+    }
+  }
+}

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=1507509&r1=1507508&r2=1507509&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 Fri Jul 26 23:09:55 2013
@@ -30,6 +30,7 @@ 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;
 
 /**
@@ -123,6 +124,7 @@ 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 
@@ -140,18 +142,15 @@ 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("", 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());
+    assertEquals(user.getUserName(), fsd.getOwner());
+    // Compare against user's primary group
+    assertEquals(user.getGroupNames()[0], fsd.getGroup());
+    assertEquals(linkQual, fsd.getPath());
     // Accessing the link 
     try {
       readFile(link);