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    
020    package org.apache.hadoop.fs;
021    
022    import com.google.common.annotations.VisibleForTesting;
023    
024    import java.io.BufferedOutputStream;
025    import java.io.DataOutput;
026    import java.io.EOFException;
027    import java.io.File;
028    import java.io.FileInputStream;
029    import java.io.FileNotFoundException;
030    import java.io.FileOutputStream;
031    import java.io.IOException;
032    import java.io.OutputStream;
033    import java.io.FileDescriptor;
034    import java.net.URI;
035    import java.nio.ByteBuffer;
036    import java.util.Arrays;
037    import java.util.EnumSet;
038    import java.util.StringTokenizer;
039    
040    import org.apache.hadoop.classification.InterfaceAudience;
041    import org.apache.hadoop.classification.InterfaceStability;
042    import org.apache.hadoop.conf.Configuration;
043    import org.apache.hadoop.fs.permission.FsPermission;
044    import org.apache.hadoop.io.nativeio.NativeIO;
045    import org.apache.hadoop.util.Progressable;
046    import org.apache.hadoop.util.Shell;
047    import 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
055    public 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    }