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