001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018
019
020package org.apache.hadoop.fs;
021
022import com.google.common.annotations.VisibleForTesting;
023
024import java.io.BufferedOutputStream;
025import java.io.DataOutput;
026import java.io.EOFException;
027import java.io.File;
028import java.io.FileInputStream;
029import java.io.FileNotFoundException;
030import java.io.FileOutputStream;
031import java.io.IOException;
032import java.io.OutputStream;
033import java.io.FileDescriptor;
034import java.net.URI;
035import java.nio.ByteBuffer;
036import java.util.Arrays;
037import java.util.EnumSet;
038import java.util.StringTokenizer;
039
040import org.apache.hadoop.classification.InterfaceAudience;
041import org.apache.hadoop.classification.InterfaceStability;
042import org.apache.hadoop.conf.Configuration;
043import org.apache.hadoop.fs.permission.FsPermission;
044import org.apache.hadoop.io.nativeio.NativeIO;
045import org.apache.hadoop.util.Progressable;
046import org.apache.hadoop.util.Shell;
047import org.apache.hadoop.util.StringUtils;
048
049/****************************************************************
050 * Implement the FileSystem API for the raw local filesystem.
051 *
052 *****************************************************************/
053@InterfaceAudience.Public
054@InterfaceStability.Stable
055public class RawLocalFileSystem extends FileSystem {
056  static final URI NAME = URI.create("file:///");
057  private Path workingDir;
058  // Temporary workaround for HADOOP-9652.
059  private static boolean useDeprecatedFileStatus = true;
060
061  @VisibleForTesting
062  public static void useStatIfAvailable() {
063    useDeprecatedFileStatus = !Stat.isAvailable();
064  }
065  
066  public RawLocalFileSystem() {
067    workingDir = getInitialWorkingDirectory();
068  }
069  
070  private Path makeAbsolute(Path f) {
071    if (f.isAbsolute()) {
072      return f;
073    } else {
074      return new Path(workingDir, f);
075    }
076  }
077  
078  /** Convert a path to a File. */
079  public File pathToFile(Path path) {
080    checkPath(path);
081    if (!path.isAbsolute()) {
082      path = new Path(getWorkingDirectory(), path);
083    }
084    return new File(path.toUri().getPath());
085  }
086
087  @Override
088  public URI getUri() { return NAME; }
089  
090  @Override
091  public void initialize(URI uri, Configuration conf) throws IOException {
092    super.initialize(uri, conf);
093    setConf(conf);
094  }
095  
096  /*******************************************************
097   * For open()'s FSInputStream.
098   *******************************************************/
099  class LocalFSFileInputStream extends FSInputStream implements HasFileDescriptor {
100    private FileInputStream fis;
101    private long position;
102
103    public LocalFSFileInputStream(Path f) throws IOException {
104      fis = new FileInputStream(pathToFile(f));
105    }
106    
107    @Override
108    public void seek(long pos) throws IOException {
109      if (pos < 0) {
110        throw new EOFException(
111          FSExceptionMessages.NEGATIVE_SEEK);
112      }
113      fis.getChannel().position(pos);
114      this.position = pos;
115    }
116    
117    @Override
118    public long getPos() throws IOException {
119      return this.position;
120    }
121    
122    @Override
123    public boolean seekToNewSource(long targetPos) throws IOException {
124      return false;
125    }
126    
127    /*
128     * Just forward to the fis
129     */
130    @Override
131    public int available() throws IOException { return fis.available(); }
132    @Override
133    public void close() throws IOException { fis.close(); }
134    @Override
135    public boolean markSupported() { return false; }
136    
137    @Override
138    public int read() throws IOException {
139      try {
140        int value = fis.read();
141        if (value >= 0) {
142          this.position++;
143          statistics.incrementBytesRead(1);
144        }
145        return value;
146      } catch (IOException e) {                 // unexpected exception
147        throw new FSError(e);                   // assume native fs error
148      }
149    }
150    
151    @Override
152    public int read(byte[] b, int off, int len) throws IOException {
153      try {
154        int value = fis.read(b, off, len);
155        if (value > 0) {
156          this.position += value;
157          statistics.incrementBytesRead(value);
158        }
159        return value;
160      } catch (IOException e) {                 // unexpected exception
161        throw new FSError(e);                   // assume native fs error
162      }
163    }
164    
165    @Override
166    public int read(long position, byte[] b, int off, int len)
167      throws IOException {
168      ByteBuffer bb = ByteBuffer.wrap(b, off, len);
169      try {
170        int value = fis.getChannel().read(bb, position);
171        if (value > 0) {
172          statistics.incrementBytesRead(value);
173        }
174        return value;
175      } catch (IOException e) {
176        throw new FSError(e);
177      }
178    }
179    
180    @Override
181    public long skip(long n) throws IOException {
182      long value = fis.skip(n);
183      if (value > 0) {
184        this.position += value;
185      }
186      return value;
187    }
188
189    @Override
190    public FileDescriptor getFileDescriptor() throws IOException {
191      return fis.getFD();
192    }
193  }
194  
195  @Override
196  public FSDataInputStream open(Path f, int bufferSize) throws IOException {
197    if (!exists(f)) {
198      throw new FileNotFoundException(f.toString());
199    }
200    return new FSDataInputStream(new BufferedFSInputStream(
201        new LocalFSFileInputStream(f), bufferSize));
202  }
203  
204  /*********************************************************
205   * For create()'s FSOutputStream.
206   *********************************************************/
207  class LocalFSFileOutputStream extends OutputStream {
208    private FileOutputStream fos;
209    
210    private LocalFSFileOutputStream(Path f, boolean append) throws IOException {
211      this.fos = new FileOutputStream(pathToFile(f), append);
212    }
213    
214    /*
215     * Just forward to the fos
216     */
217    @Override
218    public void close() throws IOException { fos.close(); }
219    @Override
220    public void flush() throws IOException { fos.flush(); }
221    @Override
222    public void write(byte[] b, int off, int len) throws IOException {
223      try {
224        fos.write(b, off, len);
225      } catch (IOException e) {                // unexpected exception
226        throw new FSError(e);                  // assume native fs error
227      }
228    }
229    
230    @Override
231    public void write(int b) throws IOException {
232      try {
233        fos.write(b);
234      } catch (IOException e) {              // unexpected exception
235        throw new FSError(e);                // assume native fs error
236      }
237    }
238  }
239
240  @Override
241  public FSDataOutputStream append(Path f, int bufferSize,
242      Progressable progress) throws IOException {
243    if (!exists(f)) {
244      throw new FileNotFoundException("File " + f + " not found");
245    }
246    if (getFileStatus(f).isDirectory()) {
247      throw new IOException("Cannot append to a diretory (=" + f + " )");
248    }
249    return new FSDataOutputStream(new BufferedOutputStream(
250        new LocalFSFileOutputStream(f, true), bufferSize), statistics);
251  }
252
253  @Override
254  public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize,
255    short replication, long blockSize, Progressable progress)
256    throws IOException {
257    return create(f, overwrite, true, bufferSize, replication, blockSize, progress);
258  }
259
260  private FSDataOutputStream create(Path f, boolean overwrite,
261      boolean createParent, int bufferSize, short replication, long blockSize,
262      Progressable progress) throws IOException {
263    if (exists(f) && !overwrite) {
264      throw new FileAlreadyExistsException("File already exists: " + f);
265    }
266    Path parent = f.getParent();
267    if (parent != null && !mkdirs(parent)) {
268      throw new IOException("Mkdirs failed to create " + parent.toString());
269    }
270    return new FSDataOutputStream(new BufferedOutputStream(
271        createOutputStream(f, false), bufferSize), statistics);
272  }
273  
274  protected OutputStream createOutputStream(Path f, boolean append) 
275      throws IOException {
276    return new LocalFSFileOutputStream(f, append); 
277  }
278  
279  @Override
280  @Deprecated
281  public FSDataOutputStream createNonRecursive(Path f, FsPermission permission,
282      EnumSet<CreateFlag> flags, int bufferSize, short replication, long blockSize,
283      Progressable progress) throws IOException {
284    if (exists(f) && !flags.contains(CreateFlag.OVERWRITE)) {
285      throw new FileAlreadyExistsException("File already exists: " + f);
286    }
287    return new FSDataOutputStream(new BufferedOutputStream(
288        new LocalFSFileOutputStream(f, false), bufferSize), statistics);
289  }
290
291  @Override
292  public FSDataOutputStream create(Path f, FsPermission permission,
293    boolean overwrite, int bufferSize, short replication, long blockSize,
294    Progressable progress) throws IOException {
295
296    FSDataOutputStream out = create(f,
297        overwrite, bufferSize, replication, blockSize, progress);
298    setPermission(f, permission);
299    return out;
300  }
301
302  @Override
303  public FSDataOutputStream createNonRecursive(Path f, FsPermission permission,
304      boolean overwrite,
305      int bufferSize, short replication, long blockSize,
306      Progressable progress) throws IOException {
307    FSDataOutputStream out = create(f,
308        overwrite, false, bufferSize, replication, blockSize, progress);
309    setPermission(f, permission);
310    return out;
311  }
312
313  @Override
314  public boolean rename(Path src, Path dst) throws IOException {
315    // Attempt rename using Java API.
316    File srcFile = pathToFile(src);
317    File dstFile = pathToFile(dst);
318    if (srcFile.renameTo(dstFile)) {
319      return true;
320    }
321
322    // Enforce POSIX rename behavior that a source directory replaces an existing
323    // destination if the destination is an empty directory.  On most platforms,
324    // this is already handled by the Java API call above.  Some platforms
325    // (notably Windows) do not provide this behavior, so the Java API call above
326    // fails.  Delete destination and attempt rename again.
327    if (this.exists(dst)) {
328      FileStatus sdst = this.getFileStatus(dst);
329      if (sdst.isDirectory() && dstFile.list().length == 0) {
330        if (LOG.isDebugEnabled()) {
331          LOG.debug("Deleting empty destination and renaming " + src + " to " +
332            dst);
333        }
334        if (this.delete(dst, false) && srcFile.renameTo(dstFile)) {
335          return true;
336        }
337      }
338    }
339
340    // The fallback behavior accomplishes the rename by a full copy.
341    if (LOG.isDebugEnabled()) {
342      LOG.debug("Falling through to a copy of " + src + " to " + dst);
343    }
344    return FileUtil.copy(this, src, this, dst, true, getConf());
345  }
346  
347  /**
348   * Delete the given path to a file or directory.
349   * @param p the path to delete
350   * @param recursive to delete sub-directories
351   * @return true if the file or directory and all its contents were deleted
352   * @throws IOException if p is non-empty and recursive is false 
353   */
354  @Override
355  public boolean delete(Path p, boolean recursive) throws IOException {
356    File f = pathToFile(p);
357    if (!f.exists()) {
358      //no path, return false "nothing to delete"
359      return false;
360    }
361    if (f.isFile()) {
362      return f.delete();
363    } else if (!recursive && f.isDirectory() && 
364        (FileUtil.listFiles(f).length != 0)) {
365      throw new IOException("Directory " + f.toString() + " is not empty");
366    }
367    return FileUtil.fullyDelete(f);
368  }
369 
370  @Override
371  public FileStatus[] listStatus(Path f) throws IOException {
372    File localf = pathToFile(f);
373    FileStatus[] results;
374
375    if (!localf.exists()) {
376      throw new FileNotFoundException("File " + f + " does not exist");
377    }
378    if (localf.isFile()) {
379      if (!useDeprecatedFileStatus) {
380        return new FileStatus[] { getFileStatus(f) };
381      }
382      return new FileStatus[] {
383        new DeprecatedRawLocalFileStatus(localf, getDefaultBlockSize(f), this)};
384    }
385
386    String[] names = localf.list();
387    if (names == null) {
388      return null;
389    }
390    results = new FileStatus[names.length];
391    int j = 0;
392    for (int i = 0; i < names.length; i++) {
393      try {
394        // Assemble the path using the Path 3 arg constructor to make sure
395        // paths with colon are properly resolved on Linux
396        results[j] = getFileStatus(new Path(f, new Path(null, null, names[i])));
397        j++;
398      } catch (FileNotFoundException e) {
399        // ignore the files not found since the dir list may have have changed
400        // since the names[] list was generated.
401      }
402    }
403    if (j == names.length) {
404      return results;
405    }
406    return Arrays.copyOf(results, j);
407  }
408  
409  protected boolean mkOneDir(File p2f) throws IOException {
410    return p2f.mkdir();
411  }
412
413  /**
414   * Creates the specified directory hierarchy. Does not
415   * treat existence as an error.
416   */
417  @Override
418  public boolean mkdirs(Path f) throws IOException {
419    if(f == null) {
420      throw new IllegalArgumentException("mkdirs path arg is null");
421    }
422    Path parent = f.getParent();
423    File p2f = pathToFile(f);
424    File parent2f = null;
425    if(parent != null) {
426      parent2f = pathToFile(parent);
427      if(parent2f != null && parent2f.exists() && !parent2f.isDirectory()) {
428        throw new ParentNotDirectoryException("Parent path is not a directory: "
429            + parent);
430      }
431    }
432    if (p2f.exists() && !p2f.isDirectory()) {
433      throw new FileNotFoundException("Destination exists" +
434              " and is not a directory: " + p2f.getCanonicalPath());
435    }
436    return (parent == null || parent2f.exists() || mkdirs(parent)) &&
437      (mkOneDir(p2f) || p2f.isDirectory());
438  }
439
440  @Override
441  public boolean mkdirs(Path f, FsPermission permission) throws IOException {
442    boolean b = mkdirs(f);
443    if(b) {
444      setPermission(f, permission);
445    }
446    return b;
447  }
448  
449
450  @Override
451  protected boolean primitiveMkdir(Path f, FsPermission absolutePermission)
452    throws IOException {
453    boolean b = mkdirs(f);
454    setPermission(f, absolutePermission);
455    return b;
456  }
457  
458  
459  @Override
460  public Path getHomeDirectory() {
461    return this.makeQualified(new Path(System.getProperty("user.home")));
462  }
463
464  /**
465   * Set the working directory to the given directory.
466   */
467  @Override
468  public void setWorkingDirectory(Path newDir) {
469    workingDir = makeAbsolute(newDir);
470    checkPath(workingDir);
471  }
472  
473  @Override
474  public Path getWorkingDirectory() {
475    return workingDir;
476  }
477  
478  @Override
479  protected Path getInitialWorkingDirectory() {
480    return this.makeQualified(new Path(System.getProperty("user.dir")));
481  }
482
483  @Override
484  public FsStatus getStatus(Path p) throws IOException {
485    File partition = pathToFile(p == null ? new Path("/") : p);
486    //File provides getUsableSpace() and getFreeSpace()
487    //File provides no API to obtain used space, assume used = total - free
488    return new FsStatus(partition.getTotalSpace(), 
489      partition.getTotalSpace() - partition.getFreeSpace(),
490      partition.getFreeSpace());
491  }
492  
493  // In the case of the local filesystem, we can just rename the file.
494  @Override
495  public void moveFromLocalFile(Path src, Path dst) throws IOException {
496    rename(src, dst);
497  }
498  
499  // We can write output directly to the final location
500  @Override
501  public Path startLocalOutput(Path fsOutputFile, Path tmpLocalFile)
502    throws IOException {
503    return fsOutputFile;
504  }
505  
506  // It's in the right place - nothing to do.
507  @Override
508  public void completeLocalOutput(Path fsWorkingFile, Path tmpLocalFile)
509    throws IOException {
510  }
511  
512  @Override
513  public void close() throws IOException {
514    super.close();
515  }
516  
517  @Override
518  public String toString() {
519    return "LocalFS";
520  }
521  
522  @Override
523  public FileStatus getFileStatus(Path f) throws IOException {
524    return getFileLinkStatusInternal(f, true);
525  }
526
527  @Deprecated
528  private FileStatus deprecatedGetFileStatus(Path f) throws IOException {
529    File path = pathToFile(f);
530    if (path.exists()) {
531      return new DeprecatedRawLocalFileStatus(pathToFile(f),
532          getDefaultBlockSize(f), this);
533    } else {
534      throw new FileNotFoundException("File " + f + " does not exist");
535    }
536  }
537
538  @Deprecated
539  static class DeprecatedRawLocalFileStatus extends FileStatus {
540    /* We can add extra fields here. It breaks at least CopyFiles.FilePair().
541     * We recognize if the information is already loaded by check if
542     * onwer.equals("").
543     */
544    private boolean isPermissionLoaded() {
545      return !super.getOwner().isEmpty(); 
546    }
547    
548    DeprecatedRawLocalFileStatus(File f, long defaultBlockSize, FileSystem fs) {
549      super(f.length(), f.isDirectory(), 1, defaultBlockSize,
550          f.lastModified(), new Path(f.getPath()).makeQualified(fs.getUri(),
551            fs.getWorkingDirectory()));
552    }
553    
554    @Override
555    public FsPermission getPermission() {
556      if (!isPermissionLoaded()) {
557        loadPermissionInfo();
558      }
559      return super.getPermission();
560    }
561
562    @Override
563    public String getOwner() {
564      if (!isPermissionLoaded()) {
565        loadPermissionInfo();
566      }
567      return super.getOwner();
568    }
569
570    @Override
571    public String getGroup() {
572      if (!isPermissionLoaded()) {
573        loadPermissionInfo();
574      }
575      return super.getGroup();
576    }
577
578    /// loads permissions, owner, and group from `ls -ld`
579    private void loadPermissionInfo() {
580      IOException e = null;
581      try {
582        String output = FileUtil.execCommand(new File(getPath().toUri()), 
583            Shell.getGetPermissionCommand());
584        StringTokenizer t =
585            new StringTokenizer(output, Shell.TOKEN_SEPARATOR_REGEX);
586        //expected format
587        //-rw-------    1 username groupname ...
588        String permission = t.nextToken();
589        if (permission.length() > FsPermission.MAX_PERMISSION_LENGTH) {
590          //files with ACLs might have a '+'
591          permission = permission.substring(0,
592            FsPermission.MAX_PERMISSION_LENGTH);
593        }
594        setPermission(FsPermission.valueOf(permission));
595        t.nextToken();
596
597        String owner = t.nextToken();
598        // If on windows domain, token format is DOMAIN\\user and we want to
599        // extract only the user name
600        if (Shell.WINDOWS) {
601          int i = owner.indexOf('\\');
602          if (i != -1)
603            owner = owner.substring(i + 1);
604        }
605        setOwner(owner);
606
607        setGroup(t.nextToken());
608      } catch (Shell.ExitCodeException ioe) {
609        if (ioe.getExitCode() != 1) {
610          e = ioe;
611        } else {
612          setPermission(null);
613          setOwner(null);
614          setGroup(null);
615        }
616      } catch (IOException ioe) {
617        e = ioe;
618      } finally {
619        if (e != null) {
620          throw new RuntimeException("Error while running command to get " +
621                                     "file permissions : " + 
622                                     StringUtils.stringifyException(e));
623        }
624      }
625    }
626
627    @Override
628    public void write(DataOutput out) throws IOException {
629      if (!isPermissionLoaded()) {
630        loadPermissionInfo();
631      }
632      super.write(out);
633    }
634  }
635
636  /**
637   * Use the command chown to set owner.
638   */
639  @Override
640  public void setOwner(Path p, String username, String groupname)
641    throws IOException {
642    FileUtil.setOwner(pathToFile(p), username, groupname);
643  }
644
645  /**
646   * Use the command chmod to set permission.
647   */
648  @Override
649  public void setPermission(Path p, FsPermission permission)
650    throws IOException {
651    if (NativeIO.isAvailable()) {
652      NativeIO.POSIX.chmod(pathToFile(p).getCanonicalPath(),
653                     permission.toShort());
654    } else {
655      String perm = String.format("%04o", permission.toShort());
656      Shell.execCommand(Shell.getSetPermissionCommand(perm, false,
657        FileUtil.makeShellPath(pathToFile(p), true)));
658    }
659  }
660 
661  /**
662   * Sets the {@link Path}'s last modified time <em>only</em> to the given
663   * valid time.
664   *
665   * @param mtime the modification time to set (only if greater than zero).
666   * @param atime currently ignored.
667   * @throws IOException if setting the last modified time fails.
668   */
669  @Override
670  public void setTimes(Path p, long mtime, long atime) throws IOException {
671    File f = pathToFile(p);
672    if(mtime >= 0) {
673      if(!f.setLastModified(mtime)) {
674        throw new IOException(
675          "couldn't set last-modified time to " +
676          mtime +
677          " for " +
678          f.getAbsolutePath());
679      }
680    }
681  }
682
683  @Override
684  public boolean supportsSymlinks() {
685    return true;
686  }
687
688  @SuppressWarnings("deprecation")
689  @Override
690  public void createSymlink(Path target, Path link, boolean createParent)
691      throws IOException {
692    if (!FileSystem.areSymlinksEnabled()) {
693      throw new UnsupportedOperationException("Symlinks not supported");
694    }
695    final String targetScheme = target.toUri().getScheme();
696    if (targetScheme != null && !"file".equals(targetScheme)) {
697      throw new IOException("Unable to create symlink to non-local file "+
698                            "system: "+target.toString());
699    }
700    if (createParent) {
701      mkdirs(link.getParent());
702    }
703
704    // NB: Use createSymbolicLink in java.nio.file.Path once available
705    int result = FileUtil.symLink(target.toString(),
706        makeAbsolute(link).toString());
707    if (result != 0) {
708      throw new IOException("Error " + result + " creating symlink " +
709          link + " to " + target);
710    }
711  }
712
713  /**
714   * Return a FileStatus representing the given path. If the path refers
715   * to a symlink return a FileStatus representing the link rather than
716   * the object the link refers to.
717   */
718  @Override
719  public FileStatus getFileLinkStatus(final Path f) throws IOException {
720    FileStatus fi = getFileLinkStatusInternal(f, false);
721    // getFileLinkStatus is supposed to return a symlink with a
722    // qualified path
723    if (fi.isSymlink()) {
724      Path targetQual = FSLinkResolver.qualifySymlinkTarget(this.getUri(),
725          fi.getPath(), fi.getSymlink());
726      fi.setSymlink(targetQual);
727    }
728    return fi;
729  }
730
731  /**
732   * Public {@link FileStatus} methods delegate to this function, which in turn
733   * either call the new {@link Stat} based implementation or the deprecated
734   * methods based on platform support.
735   * 
736   * @param f Path to stat
737   * @param dereference whether to dereference the final path component if a
738   *          symlink
739   * @return FileStatus of f
740   * @throws IOException
741   */
742  private FileStatus getFileLinkStatusInternal(final Path f,
743      boolean dereference) throws IOException {
744    if (!useDeprecatedFileStatus) {
745      return getNativeFileLinkStatus(f, dereference);
746    } else if (dereference) {
747      return deprecatedGetFileStatus(f);
748    } else {
749      return deprecatedGetFileLinkStatusInternal(f);
750    }
751  }
752
753  /**
754   * Deprecated. Remains for legacy support. Should be removed when {@link Stat}
755   * gains support for Windows and other operating systems.
756   */
757  @Deprecated
758  private FileStatus deprecatedGetFileLinkStatusInternal(final Path f)
759      throws IOException {
760    String target = FileUtil.readLink(new File(f.toString()));
761
762    try {
763      FileStatus fs = getFileStatus(f);
764      // If f refers to a regular file or directory
765      if (target.isEmpty()) {
766        return fs;
767      }
768      // Otherwise f refers to a symlink
769      return new FileStatus(fs.getLen(),
770          false,
771          fs.getReplication(),
772          fs.getBlockSize(),
773          fs.getModificationTime(),
774          fs.getAccessTime(),
775          fs.getPermission(),
776          fs.getOwner(),
777          fs.getGroup(),
778          new Path(target),
779          f);
780    } catch (FileNotFoundException e) {
781      /* The exists method in the File class returns false for dangling
782       * links so we can get a FileNotFoundException for links that exist.
783       * It's also possible that we raced with a delete of the link. Use
784       * the readBasicFileAttributes method in java.nio.file.attributes
785       * when available.
786       */
787      if (!target.isEmpty()) {
788        return new FileStatus(0, false, 0, 0, 0, 0, FsPermission.getDefault(),
789            "", "", new Path(target), f);
790      }
791      // f refers to a file or directory that does not exist
792      throw e;
793    }
794  }
795  /**
796   * Calls out to platform's native stat(1) implementation to get file metadata
797   * (permissions, user, group, atime, mtime, etc). This works around the lack
798   * of lstat(2) in Java 6.
799   * 
800   *  Currently, the {@link Stat} class used to do this only supports Linux
801   *  and FreeBSD, so the old {@link #deprecatedGetFileLinkStatusInternal(Path)}
802   *  implementation (deprecated) remains further OS support is added.
803   *
804   * @param f File to stat
805   * @param dereference whether to dereference symlinks
806   * @return FileStatus of f
807   * @throws IOException
808   */
809  private FileStatus getNativeFileLinkStatus(final Path f,
810      boolean dereference) throws IOException {
811    checkPath(f);
812    Stat stat = new Stat(f, getDefaultBlockSize(f), dereference, this);
813    FileStatus status = stat.getFileStatus();
814    return status;
815  }
816
817  @Override
818  public Path getLinkTarget(Path f) throws IOException {
819    FileStatus fi = getFileLinkStatusInternal(f, false);
820    // return an unqualified symlink target
821    return fi.getSymlink();
822  }
823}