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