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