Przeglądaj źródła

revert HADOOP-9652

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1508248 13f79535-47bb-0310-9956-ffa450edef68
Colin McCabe 12 lat temu
rodzic
commit
f593d57296

+ 0 - 3
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -310,9 +310,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

+ 8 - 11
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DelegateToFileSystem.java

@@ -113,14 +113,7 @@ public abstract class DelegateToFileSystem extends AbstractFileSystem {
 
   @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 DelegateToFileSystem extends AbstractFileSystem {
 
   @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

+ 32 - 4
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HardLink.java

@@ -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.OSType;
  */
 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.

+ 2 - 30
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java

@@ -385,9 +385,6 @@ public class RawLocalFileSystem extends FileSystem {
       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) };
     }
@@ -519,10 +516,6 @@ public class RawLocalFileSystem extends FileSystem {
   
   @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);
@@ -531,7 +524,6 @@ public class RawLocalFileSystem extends FileSystem {
     }
   }
 
-  @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
@@ -705,7 +697,6 @@ public class RawLocalFileSystem extends FileSystem {
    * 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
@@ -726,12 +717,7 @@ public class RawLocalFileSystem extends FileSystem {
    */
   @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()) {
@@ -742,12 +728,6 @@ public class RawLocalFileSystem extends FileSystem {
     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);
 
@@ -785,17 +765,9 @@ public class RawLocalFileSystem extends FileSystem {
     }
   }
 
-  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();
   }

+ 0 - 159
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Stat.java

@@ -1,159 +0,0 @@
-/**
- * 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);
-    }
-  }
-}

+ 106 - 5
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/local/RawLocalFs.java

@@ -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.InterfaceStability;
 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.
@@ -66,12 +71,108 @@ public class RawLocalFs extends DelegateToFileSystem {
   public FsServerDefaults getServerDefaults() throws IOException {
     return LocalConfigKeys.getServerDefaults();
   }
-
+  
   @Override
-  public boolean isValidName(String src) {
-    // Different local file systems have different validation rules. Skip
-    // validation here and just let the OS handle it. This is consistent with
-    // RawLocalFileSystem.
+  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 (target.isEmpty()) {
+        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 (!target.isEmpty()) {
+        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
+     // validation here and just let the OS handle it.  This is consistent with
+     // 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();
   }
 }

+ 0 - 27
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java

@@ -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"}

+ 2 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFileSystem.java

@@ -363,12 +363,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());
   }
 
   /**

+ 0 - 112
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestStat.java

@@ -1,112 +0,0 @@
-/**
- * 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
-    }
-  }
-}

+ 8 - 7
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestSymlinkLocalFS.java

@@ -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 extends SymlinkBaseTest {
     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 extends SymlinkBaseTest {
       // 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);