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