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        new LocalFSFileOutputStream(f, false), bufferSize), statistics);
272  }
273  
274  @Override
275  @Deprecated
276  public FSDataOutputStream createNonRecursive(Path f, FsPermission permission,
277      EnumSet<CreateFlag> flags, int bufferSize, short replication, long blockSize,
278      Progressable progress) throws IOException {
279    if (exists(f) && !flags.contains(CreateFlag.OVERWRITE)) {
280      throw new FileAlreadyExistsException("File already exists: " + f);
281    }
282    return new FSDataOutputStream(new BufferedOutputStream(
283        new LocalFSFileOutputStream(f, false), bufferSize), statistics);
284  }
285
286  @Override
287  public FSDataOutputStream create(Path f, FsPermission permission,
288    boolean overwrite, int bufferSize, short replication, long blockSize,
289    Progressable progress) throws IOException {
290
291    FSDataOutputStream out = create(f,
292        overwrite, bufferSize, replication, blockSize, progress);
293    setPermission(f, permission);
294    return out;
295  }
296
297  @Override
298  public FSDataOutputStream createNonRecursive(Path f, FsPermission permission,
299      boolean overwrite,
300      int bufferSize, short replication, long blockSize,
301      Progressable progress) throws IOException {
302    FSDataOutputStream out = create(f,
303        overwrite, false, bufferSize, replication, blockSize, progress);
304    setPermission(f, permission);
305    return out;
306  }
307
308  @Override
309  public boolean rename(Path src, Path dst) throws IOException {
310    // Attempt rename using Java API.
311    File srcFile = pathToFile(src);
312    File dstFile = pathToFile(dst);
313    if (srcFile.renameTo(dstFile)) {
314      return true;
315    }
316
317    // Enforce POSIX rename behavior that a source directory replaces an existing
318    // destination if the destination is an empty directory.  On most platforms,
319    // this is already handled by the Java API call above.  Some platforms
320    // (notably Windows) do not provide this behavior, so the Java API call above
321    // fails.  Delete destination and attempt rename again.
322    if (this.exists(dst)) {
323      FileStatus sdst = this.getFileStatus(dst);
324      if (sdst.isDirectory() && dstFile.list().length == 0) {
325        if (LOG.isDebugEnabled()) {
326          LOG.debug("Deleting empty destination and renaming " + src + " to " +
327            dst);
328        }
329        if (this.delete(dst, false) && srcFile.renameTo(dstFile)) {
330          return true;
331        }
332      }
333    }
334
335    // The fallback behavior accomplishes the rename by a full copy.
336    if (LOG.isDebugEnabled()) {
337      LOG.debug("Falling through to a copy of " + src + " to " + dst);
338    }
339    return FileUtil.copy(this, src, this, dst, true, getConf());
340  }
341  
342  /**
343   * Delete the given path to a file or directory.
344   * @param p the path to delete
345   * @param recursive to delete sub-directories
346   * @return true if the file or directory and all its contents were deleted
347   * @throws IOException if p is non-empty and recursive is false 
348   */
349  @Override
350  public boolean delete(Path p, boolean recursive) throws IOException {
351    File f = pathToFile(p);
352    if (!f.exists()) {
353      //no path, return false "nothing to delete"
354      return false;
355    }
356    if (f.isFile()) {
357      return f.delete();
358    } else if (!recursive && f.isDirectory() && 
359        (FileUtil.listFiles(f).length != 0)) {
360      throw new IOException("Directory " + f.toString() + " is not empty");
361    }
362    return FileUtil.fullyDelete(f);
363  }
364 
365  @Override
366  public FileStatus[] listStatus(Path f) throws IOException {
367    File localf = pathToFile(f);
368    FileStatus[] results;
369
370    if (!localf.exists()) {
371      throw new FileNotFoundException("File " + f + " does not exist");
372    }
373    if (localf.isFile()) {
374      if (!useDeprecatedFileStatus) {
375        return new FileStatus[] { getFileStatus(f) };
376      }
377      return new FileStatus[] {
378        new DeprecatedRawLocalFileStatus(localf, getDefaultBlockSize(f), this)};
379    }
380
381    String[] names = localf.list();
382    if (names == null) {
383      return null;
384    }
385    results = new FileStatus[names.length];
386    int j = 0;
387    for (int i = 0; i < names.length; i++) {
388      try {
389        // Assemble the path using the Path 3 arg constructor to make sure
390        // paths with colon are properly resolved on Linux
391        results[j] = getFileStatus(new Path(f, new Path(null, null, names[i])));
392        j++;
393      } catch (FileNotFoundException e) {
394        // ignore the files not found since the dir list may have have changed
395        // since the names[] list was generated.
396      }
397    }
398    if (j == names.length) {
399      return results;
400    }
401    return Arrays.copyOf(results, j);
402  }
403
404  /**
405   * Creates the specified directory hierarchy. Does not
406   * treat existence as an error.
407   */
408  @Override
409  public boolean mkdirs(Path f) throws IOException {
410    if(f == null) {
411      throw new IllegalArgumentException("mkdirs path arg is null");
412    }
413    Path parent = f.getParent();
414    File p2f = pathToFile(f);
415    if(parent != null) {
416      File parent2f = pathToFile(parent);
417      if(parent2f != null && parent2f.exists() && !parent2f.isDirectory()) {
418        throw new ParentNotDirectoryException("Parent path is not a directory: "
419            + parent);
420      }
421    }
422    if (p2f.exists() && !p2f.isDirectory()) {
423      throw new FileNotFoundException("Destination exists" +
424              " and is not a directory: " + p2f.getCanonicalPath());
425    }
426    return (parent == null || mkdirs(parent)) &&
427      (p2f.mkdir() || p2f.isDirectory());
428  }
429
430  @Override
431  public boolean mkdirs(Path f, FsPermission permission) throws IOException {
432    boolean b = mkdirs(f);
433    if(b) {
434      setPermission(f, permission);
435    }
436    return b;
437  }
438  
439
440  @Override
441  protected boolean primitiveMkdir(Path f, FsPermission absolutePermission)
442    throws IOException {
443    boolean b = mkdirs(f);
444    setPermission(f, absolutePermission);
445    return b;
446  }
447  
448  
449  @Override
450  public Path getHomeDirectory() {
451    return this.makeQualified(new Path(System.getProperty("user.home")));
452  }
453
454  /**
455   * Set the working directory to the given directory.
456   */
457  @Override
458  public void setWorkingDirectory(Path newDir) {
459    workingDir = makeAbsolute(newDir);
460    checkPath(workingDir);
461  }
462  
463  @Override
464  public Path getWorkingDirectory() {
465    return workingDir;
466  }
467  
468  @Override
469  protected Path getInitialWorkingDirectory() {
470    return this.makeQualified(new Path(System.getProperty("user.dir")));
471  }
472
473  @Override
474  public FsStatus getStatus(Path p) throws IOException {
475    File partition = pathToFile(p == null ? new Path("/") : p);
476    //File provides getUsableSpace() and getFreeSpace()
477    //File provides no API to obtain used space, assume used = total - free
478    return new FsStatus(partition.getTotalSpace(), 
479      partition.getTotalSpace() - partition.getFreeSpace(),
480      partition.getFreeSpace());
481  }
482  
483  // In the case of the local filesystem, we can just rename the file.
484  @Override
485  public void moveFromLocalFile(Path src, Path dst) throws IOException {
486    rename(src, dst);
487  }
488  
489  // We can write output directly to the final location
490  @Override
491  public Path startLocalOutput(Path fsOutputFile, Path tmpLocalFile)
492    throws IOException {
493    return fsOutputFile;
494  }
495  
496  // It's in the right place - nothing to do.
497  @Override
498  public void completeLocalOutput(Path fsWorkingFile, Path tmpLocalFile)
499    throws IOException {
500  }
501  
502  @Override
503  public void close() throws IOException {
504    super.close();
505  }
506  
507  @Override
508  public String toString() {
509    return "LocalFS";
510  }
511  
512  @Override
513  public FileStatus getFileStatus(Path f) throws IOException {
514    return getFileLinkStatusInternal(f, true);
515  }
516
517  @Deprecated
518  private FileStatus deprecatedGetFileStatus(Path f) throws IOException {
519    File path = pathToFile(f);
520    if (path.exists()) {
521      return new DeprecatedRawLocalFileStatus(pathToFile(f),
522          getDefaultBlockSize(f), this);
523    } else {
524      throw new FileNotFoundException("File " + f + " does not exist");
525    }
526  }
527
528  @Deprecated
529  static class DeprecatedRawLocalFileStatus extends FileStatus {
530    /* We can add extra fields here. It breaks at least CopyFiles.FilePair().
531     * We recognize if the information is already loaded by check if
532     * onwer.equals("").
533     */
534    private boolean isPermissionLoaded() {
535      return !super.getOwner().isEmpty(); 
536    }
537    
538    DeprecatedRawLocalFileStatus(File f, long defaultBlockSize, FileSystem fs) {
539      super(f.length(), f.isDirectory(), 1, defaultBlockSize,
540          f.lastModified(), new Path(f.getPath()).makeQualified(fs.getUri(),
541            fs.getWorkingDirectory()));
542    }
543    
544    @Override
545    public FsPermission getPermission() {
546      if (!isPermissionLoaded()) {
547        loadPermissionInfo();
548      }
549      return super.getPermission();
550    }
551
552    @Override
553    public String getOwner() {
554      if (!isPermissionLoaded()) {
555        loadPermissionInfo();
556      }
557      return super.getOwner();
558    }
559
560    @Override
561    public String getGroup() {
562      if (!isPermissionLoaded()) {
563        loadPermissionInfo();
564      }
565      return super.getGroup();
566    }
567
568    /// loads permissions, owner, and group from `ls -ld`
569    private void loadPermissionInfo() {
570      IOException e = null;
571      try {
572        String output = FileUtil.execCommand(new File(getPath().toUri()), 
573            Shell.getGetPermissionCommand());
574        StringTokenizer t =
575            new StringTokenizer(output, Shell.TOKEN_SEPARATOR_REGEX);
576        //expected format
577        //-rw-------    1 username groupname ...
578        String permission = t.nextToken();
579        if (permission.length() > FsPermission.MAX_PERMISSION_LENGTH) {
580          //files with ACLs might have a '+'
581          permission = permission.substring(0,
582            FsPermission.MAX_PERMISSION_LENGTH);
583        }
584        setPermission(FsPermission.valueOf(permission));
585        t.nextToken();
586
587        String owner = t.nextToken();
588        // If on windows domain, token format is DOMAIN\\user and we want to
589        // extract only the user name
590        if (Shell.WINDOWS) {
591          int i = owner.indexOf('\\');
592          if (i != -1)
593            owner = owner.substring(i + 1);
594        }
595        setOwner(owner);
596
597        setGroup(t.nextToken());
598      } catch (Shell.ExitCodeException ioe) {
599        if (ioe.getExitCode() != 1) {
600          e = ioe;
601        } else {
602          setPermission(null);
603          setOwner(null);
604          setGroup(null);
605        }
606      } catch (IOException ioe) {
607        e = ioe;
608      } finally {
609        if (e != null) {
610          throw new RuntimeException("Error while running command to get " +
611                                     "file permissions : " + 
612                                     StringUtils.stringifyException(e));
613        }
614      }
615    }
616
617    @Override
618    public void write(DataOutput out) throws IOException {
619      if (!isPermissionLoaded()) {
620        loadPermissionInfo();
621      }
622      super.write(out);
623    }
624  }
625
626  /**
627   * Use the command chown to set owner.
628   */
629  @Override
630  public void setOwner(Path p, String username, String groupname)
631    throws IOException {
632    FileUtil.setOwner(pathToFile(p), username, groupname);
633  }
634
635  /**
636   * Use the command chmod to set permission.
637   */
638  @Override
639  public void setPermission(Path p, FsPermission permission)
640    throws IOException {
641    if (NativeIO.isAvailable()) {
642      NativeIO.POSIX.chmod(pathToFile(p).getCanonicalPath(),
643                     permission.toShort());
644    } else {
645      String perm = String.format("%04o", permission.toShort());
646      Shell.execCommand(Shell.getSetPermissionCommand(perm, false,
647        FileUtil.makeShellPath(pathToFile(p), true)));
648    }
649  }
650 
651  /**
652   * Sets the {@link Path}'s last modified time <em>only</em> to the given
653   * valid time.
654   *
655   * @param mtime the modification time to set (only if greater than zero).
656   * @param atime currently ignored.
657   * @throws IOException if setting the last modified time fails.
658   */
659  @Override
660  public void setTimes(Path p, long mtime, long atime) throws IOException {
661    File f = pathToFile(p);
662    if(mtime >= 0) {
663      if(!f.setLastModified(mtime)) {
664        throw new IOException(
665          "couldn't set last-modified time to " +
666          mtime +
667          " for " +
668          f.getAbsolutePath());
669      }
670    }
671  }
672
673  @Override
674  public boolean supportsSymlinks() {
675    return true;
676  }
677
678  @SuppressWarnings("deprecation")
679  @Override
680  public void createSymlink(Path target, Path link, boolean createParent)
681      throws IOException {
682    if (!FileSystem.areSymlinksEnabled()) {
683      throw new UnsupportedOperationException("Symlinks not supported");
684    }
685    final String targetScheme = target.toUri().getScheme();
686    if (targetScheme != null && !"file".equals(targetScheme)) {
687      throw new IOException("Unable to create symlink to non-local file "+
688                            "system: "+target.toString());
689    }
690    if (createParent) {
691      mkdirs(link.getParent());
692    }
693
694    // NB: Use createSymbolicLink in java.nio.file.Path once available
695    int result = FileUtil.symLink(target.toString(),
696        makeAbsolute(link).toString());
697    if (result != 0) {
698      throw new IOException("Error " + result + " creating symlink " +
699          link + " to " + target);
700    }
701  }
702
703  /**
704   * Return a FileStatus representing the given path. If the path refers
705   * to a symlink return a FileStatus representing the link rather than
706   * the object the link refers to.
707   */
708  @Override
709  public FileStatus getFileLinkStatus(final Path f) throws IOException {
710    FileStatus fi = getFileLinkStatusInternal(f, false);
711    // getFileLinkStatus is supposed to return a symlink with a
712    // qualified path
713    if (fi.isSymlink()) {
714      Path targetQual = FSLinkResolver.qualifySymlinkTarget(this.getUri(),
715          fi.getPath(), fi.getSymlink());
716      fi.setSymlink(targetQual);
717    }
718    return fi;
719  }
720
721  /**
722   * Public {@link FileStatus} methods delegate to this function, which in turn
723   * either call the new {@link Stat} based implementation or the deprecated
724   * methods based on platform support.
725   * 
726   * @param f Path to stat
727   * @param dereference whether to dereference the final path component if a
728   *          symlink
729   * @return FileStatus of f
730   * @throws IOException
731   */
732  private FileStatus getFileLinkStatusInternal(final Path f,
733      boolean dereference) throws IOException {
734    if (!useDeprecatedFileStatus) {
735      return getNativeFileLinkStatus(f, dereference);
736    } else if (dereference) {
737      return deprecatedGetFileStatus(f);
738    } else {
739      return deprecatedGetFileLinkStatusInternal(f);
740    }
741  }
742
743  /**
744   * Deprecated. Remains for legacy support. Should be removed when {@link Stat}
745   * gains support for Windows and other operating systems.
746   */
747  @Deprecated
748  private FileStatus deprecatedGetFileLinkStatusInternal(final Path f)
749      throws IOException {
750    String target = FileUtil.readLink(new File(f.toString()));
751
752    try {
753      FileStatus fs = getFileStatus(f);
754      // If f refers to a regular file or directory
755      if (target.isEmpty()) {
756        return fs;
757      }
758      // Otherwise f refers to a symlink
759      return new FileStatus(fs.getLen(),
760          false,
761          fs.getReplication(),
762          fs.getBlockSize(),
763          fs.getModificationTime(),
764          fs.getAccessTime(),
765          fs.getPermission(),
766          fs.getOwner(),
767          fs.getGroup(),
768          new Path(target),
769          f);
770    } catch (FileNotFoundException e) {
771      /* The exists method in the File class returns false for dangling
772       * links so we can get a FileNotFoundException for links that exist.
773       * It's also possible that we raced with a delete of the link. Use
774       * the readBasicFileAttributes method in java.nio.file.attributes
775       * when available.
776       */
777      if (!target.isEmpty()) {
778        return new FileStatus(0, false, 0, 0, 0, 0, FsPermission.getDefault(),
779            "", "", new Path(target), f);
780      }
781      // f refers to a file or directory that does not exist
782      throw e;
783    }
784  }
785  /**
786   * Calls out to platform's native stat(1) implementation to get file metadata
787   * (permissions, user, group, atime, mtime, etc). This works around the lack
788   * of lstat(2) in Java 6.
789   * 
790   *  Currently, the {@link Stat} class used to do this only supports Linux
791   *  and FreeBSD, so the old {@link #deprecatedGetFileLinkStatusInternal(Path)}
792   *  implementation (deprecated) remains further OS support is added.
793   *
794   * @param f File to stat
795   * @param dereference whether to dereference symlinks
796   * @return FileStatus of f
797   * @throws IOException
798   */
799  private FileStatus getNativeFileLinkStatus(final Path f,
800      boolean dereference) throws IOException {
801    checkPath(f);
802    Stat stat = new Stat(f, getDefaultBlockSize(f), dereference, this);
803    FileStatus status = stat.getFileStatus();
804    return status;
805  }
806
807  @Override
808  public Path getLinkTarget(Path f) throws IOException {
809    FileStatus fi = getFileLinkStatusInternal(f, false);
810    // return an unqualified symlink target
811    return fi.getSymlink();
812  }
813}