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 */
018package org.apache.hadoop.fs.viewfs;
019
020import static org.apache.hadoop.fs.viewfs.Constants.PERMISSION_RRR;
021
022import java.io.FileNotFoundException;
023import java.io.IOException;
024import java.net.URI;
025import java.net.URISyntaxException;
026import java.util.Arrays;
027import java.util.HashSet;
028import java.util.List;
029import java.util.Set;
030import java.util.StringTokenizer;
031import java.util.Map.Entry;
032
033import org.apache.hadoop.classification.InterfaceAudience;
034import org.apache.hadoop.classification.InterfaceStability;
035import org.apache.hadoop.conf.Configuration;
036import org.apache.hadoop.fs.BlockLocation;
037import org.apache.hadoop.fs.ContentSummary;
038import org.apache.hadoop.fs.FSDataInputStream;
039import org.apache.hadoop.fs.FSDataOutputStream;
040import org.apache.hadoop.fs.FileAlreadyExistsException;
041import org.apache.hadoop.fs.FileChecksum;
042import org.apache.hadoop.fs.FileStatus;
043import org.apache.hadoop.fs.FileSystem;
044import org.apache.hadoop.fs.FsConstants;
045import org.apache.hadoop.fs.FsServerDefaults;
046import org.apache.hadoop.fs.InvalidPathException;
047import org.apache.hadoop.fs.Path;
048import org.apache.hadoop.fs.UnsupportedFileSystemException;
049import org.apache.hadoop.fs.permission.FsPermission;
050import org.apache.hadoop.fs.viewfs.InodeTree.INode;
051import org.apache.hadoop.fs.viewfs.InodeTree.INodeLink;
052import org.apache.hadoop.security.AccessControlException;
053import org.apache.hadoop.security.UserGroupInformation;
054import org.apache.hadoop.util.Progressable;
055
056/**
057 * ViewFileSystem (extends the FileSystem interface) implements a client-side
058 * mount table. Its spec and implementation is identical to {@link ViewFs}.
059 */
060
061@InterfaceAudience.Public
062@InterfaceStability.Evolving /*Evolving for a release,to be changed to Stable */
063public class ViewFileSystem extends FileSystem {
064  static AccessControlException readOnlyMountTable(final String operation,
065      final String p) {
066    return new AccessControlException( 
067        "InternalDir of ViewFileSystem is readonly; operation=" + operation + 
068        "Path=" + p);
069  }
070  static AccessControlException readOnlyMountTable(final String operation,
071      final Path p) {
072    return readOnlyMountTable(operation, p.toString());
073  }
074  
075  static public class MountPoint {
076    private Path src;       // the src of the mount
077    private URI[] targets; //  target of the mount; Multiple targets imply mergeMount
078    MountPoint(Path srcPath, URI[] targetURIs) {
079      src = srcPath;
080      targets = targetURIs;
081    }
082    Path getSrc() {
083      return src;
084    }
085    URI[] getTargets() {
086      return targets;
087    }
088  }
089  
090  final long creationTime; // of the the mount table
091  final UserGroupInformation ugi; // the user/group of user who created mtable
092  URI myUri;
093  private Path workingDir;
094  Configuration config;
095  InodeTree<FileSystem> fsState;  // the fs state; ie the mount table
096  Path homeDir = null;
097  
098  /**
099   * Prohibits names which contain a ".", "..", ":" or "/" 
100   */
101  private static boolean isValidName(final String src) {
102    // Check for ".." "." ":" "/"
103    final StringTokenizer tokens = new StringTokenizer(src, Path.SEPARATOR);
104    while(tokens.hasMoreTokens()) {
105      String element = tokens.nextToken();
106      if (element.equals("..") ||
107          element.equals(".")  ||
108          (element.indexOf(":") >= 0)) {
109        return false;
110      }
111    }
112    return true;
113  }
114  
115  /**
116   * Make the path Absolute and get the path-part of a pathname.
117   * Checks that URI matches this file system 
118   * and that the path-part is a valid name.
119   * 
120   * @param p path
121   * @return path-part of the Path p
122   */
123  private String getUriPath(final Path p) {
124    checkPath(p);
125    String s = makeAbsolute(p).toUri().getPath();
126    if (!isValidName(s)) {
127      throw new InvalidPathException("Path part " + s + " from URI" + p
128          + " is not a valid filename.");
129    }
130    return s;
131  }
132  
133  private Path makeAbsolute(final Path f) {
134    return f.isAbsolute() ? f : new Path(workingDir, f);
135  }
136  
137  /**
138   * This is the  constructor with the signature needed by
139   * {@link FileSystem#createFileSystem(URI, Configuration)}
140   * 
141   * After this constructor is called initialize() is called.
142   * @throws IOException 
143   */
144  public ViewFileSystem() throws IOException {
145    ugi = UserGroupInformation.getCurrentUser();
146    creationTime = System.currentTimeMillis();
147  }
148
149  /**
150   * Called after a new FileSystem instance is constructed.
151   * @param theUri a uri whose authority section names the host, port, etc. for
152   *          this FileSystem
153   * @param conf the configuration
154   */
155  public void initialize(final URI theUri, final Configuration conf)
156      throws IOException {
157    super.initialize(theUri, conf);
158    setConf(conf);
159    config = conf;
160    // Now build  client side view (i.e. client side mount table) from config.
161    final String authority = theUri.getAuthority();
162    try {
163      myUri = new URI(FsConstants.VIEWFS_SCHEME, authority, "/", null, null);
164      fsState = new InodeTree<FileSystem>(conf, authority) {
165
166        @Override
167        protected
168        FileSystem getTargetFileSystem(final URI uri)
169          throws URISyntaxException, IOException {
170            return new ChRootedFileSystem(uri, config);
171        }
172
173        @Override
174        protected
175        FileSystem getTargetFileSystem(final INodeDir<FileSystem> dir)
176          throws URISyntaxException {
177          return new InternalDirOfViewFs(dir, creationTime, ugi, myUri);
178        }
179
180        @Override
181        protected
182        FileSystem getTargetFileSystem(URI[] mergeFsURIList)
183            throws URISyntaxException, UnsupportedFileSystemException {
184          throw new UnsupportedFileSystemException("mergefs not implemented");
185          // return MergeFs.createMergeFs(mergeFsURIList, config);
186        }
187      };
188      workingDir = this.getHomeDirectory();
189    } catch (URISyntaxException e) {
190      throw new IOException("URISyntax exception: " + theUri);
191    }
192
193  }
194  
195  
196  /**
197   * Convenience Constructor for apps to call directly
198   * @param theUri which must be that of ViewFileSystem
199   * @param conf
200   * @throws IOException
201   */
202  ViewFileSystem(final URI theUri, final Configuration conf)
203    throws IOException {
204    this();
205    initialize(theUri, conf);
206  }
207  
208  /**
209   * Convenience Constructor for apps to call directly
210   * @param conf
211   * @throws IOException
212   */
213  public ViewFileSystem(final Configuration conf) throws IOException {
214    this(FsConstants.VIEWFS_URI, conf);
215  }
216  
217  public Path getTrashCanLocation(final Path f) throws FileNotFoundException {
218    final InodeTree.ResolveResult<FileSystem> res = 
219      fsState.resolve(getUriPath(f), true);
220    return res.isInternalDir() ? null : res.targetFileSystem.getHomeDirectory();
221  }
222  
223  @Override
224  public URI getUri() {
225    return myUri;
226  }
227  
228  @Override
229  public Path resolvePath(final Path f)
230      throws IOException {
231    final InodeTree.ResolveResult<FileSystem> res;
232      res = fsState.resolve(getUriPath(f), true);
233    if (res.isInternalDir()) {
234      return f;
235    }
236    return res.targetFileSystem.resolvePath(res.remainingPath);
237  }
238  
239  @Override
240  public Path getHomeDirectory() {
241    if (homeDir == null) {
242      String base = fsState.getHomeDirPrefixValue();
243      if (base == null) {
244        base = "/user";
245      }
246      homeDir = 
247        this.makeQualified(new Path(base + "/" + ugi.getShortUserName()));
248    }
249    return homeDir;
250  }
251  
252  @Override
253  public Path getWorkingDirectory() {
254    return workingDir;
255  }
256
257  @Override
258  public void setWorkingDirectory(final Path new_dir) {
259    getUriPath(new_dir); // this validates the path
260    workingDir = makeAbsolute(new_dir);
261  }
262  
263  @Override
264  public FSDataOutputStream append(final Path f, final int bufferSize,
265      final Progressable progress) throws IOException {
266    InodeTree.ResolveResult<FileSystem> res = 
267      fsState.resolve(getUriPath(f), true);
268    return res.targetFileSystem.append(res.remainingPath, bufferSize, progress);
269  }
270  
271  @Override
272  public FSDataOutputStream create(final Path f, final FsPermission permission,
273      final boolean overwrite, final int bufferSize, final short replication,
274      final long blockSize, final Progressable progress) throws IOException {
275    InodeTree.ResolveResult<FileSystem> res;
276    try {
277      res = fsState.resolve(getUriPath(f), false);
278    } catch (FileNotFoundException e) {
279        throw readOnlyMountTable("create", f);
280    }
281    assert(res.remainingPath != null);
282    return res.targetFileSystem.create(res.remainingPath, permission,
283         overwrite, bufferSize, replication, blockSize, progress);
284  }
285
286  
287  @Override
288  public boolean delete(final Path f, final boolean recursive)
289      throws AccessControlException, FileNotFoundException,
290      IOException {
291    InodeTree.ResolveResult<FileSystem> res = 
292      fsState.resolve(getUriPath(f), true);
293    // If internal dir or target is a mount link (ie remainingPath is Slash)
294    if (res.isInternalDir() || res.remainingPath == InodeTree.SlashPath) {
295      throw readOnlyMountTable("delete", f);
296    }
297    return res.targetFileSystem.delete(res.remainingPath, recursive);
298  }
299  
300  @Override
301  @SuppressWarnings("deprecation")
302  public boolean delete(final Path f)
303      throws AccessControlException, FileNotFoundException,
304      IOException {
305      return delete(f, true);
306  }
307  
308  @Override
309  public BlockLocation[] getFileBlockLocations(FileStatus fs, 
310      long start, long len) throws IOException {
311    final InodeTree.ResolveResult<FileSystem> res = 
312      fsState.resolve(getUriPath(fs.getPath()), true);
313    return res.targetFileSystem.getFileBlockLocations(
314          new ViewFsFileStatus(fs, res.remainingPath), start, len);
315  }
316
317  @Override
318  public FileChecksum getFileChecksum(final Path f)
319      throws AccessControlException, FileNotFoundException,
320      IOException {
321    InodeTree.ResolveResult<FileSystem> res = 
322      fsState.resolve(getUriPath(f), true);
323    return res.targetFileSystem.getFileChecksum(res.remainingPath);
324  }
325
326  @Override
327  public FileStatus getFileStatus(final Path f) throws AccessControlException,
328      FileNotFoundException, IOException {
329    InodeTree.ResolveResult<FileSystem> res = 
330      fsState.resolve(getUriPath(f), true);
331    
332    // FileStatus#getPath is a fully qualified path relative to the root of 
333    // target file system.
334    // We need to change it to viewfs URI - relative to root of mount table.
335    
336    // The implementors of RawLocalFileSystem were trying to be very smart.
337    // They implement FileStatus#getOwener lazily -- the object
338    // returned is really a RawLocalFileSystem that expect the
339    // FileStatus#getPath to be unchanged so that it can get owner when needed.
340    // Hence we need to interpose a new ViewFileSystemFileStatus that 
341    // works around.
342    FileStatus status =  res.targetFileSystem.getFileStatus(res.remainingPath);
343    return new ViewFsFileStatus(status, this.makeQualified(f));
344  }
345  
346  
347  @Override
348  public FileStatus[] listStatus(final Path f) throws AccessControlException,
349      FileNotFoundException, IOException {
350    InodeTree.ResolveResult<FileSystem> res =
351      fsState.resolve(getUriPath(f), true);
352    
353    FileStatus[] statusLst = res.targetFileSystem.listStatus(res.remainingPath);
354    if (!res.isInternalDir()) {
355      // We need to change the name in the FileStatus as described in
356      // {@link #getFileStatus }
357      ChRootedFileSystem targetFs;
358      targetFs = (ChRootedFileSystem) res.targetFileSystem;
359      int i = 0;
360      for (FileStatus status : statusLst) {
361          String suffix = targetFs.stripOutRoot(status.getPath());
362          statusLst[i++] = new ViewFsFileStatus(status, this.makeQualified(
363              suffix.length() == 0 ? f : new Path(res.resolvedPath, suffix)));
364      }
365    }
366    return statusLst;
367  }
368
369  @Override
370  public boolean mkdirs(final Path dir, final FsPermission permission)
371      throws IOException {
372    InodeTree.ResolveResult<FileSystem> res = 
373      fsState.resolve(getUriPath(dir), false);
374   return  res.targetFileSystem.mkdirs(res.remainingPath, permission);
375  }
376
377  @Override
378  public FSDataInputStream open(final Path f, final int bufferSize)
379      throws AccessControlException, FileNotFoundException,
380      IOException {
381    InodeTree.ResolveResult<FileSystem> res = 
382        fsState.resolve(getUriPath(f), true);
383    return res.targetFileSystem.open(res.remainingPath, bufferSize);
384  }
385
386  
387  @Override
388  public boolean rename(final Path src, final Path dst) throws IOException {
389    // passing resolveLastComponet as false to catch renaming a mount point to 
390    // itself. We need to catch this as an internal operation and fail.
391    InodeTree.ResolveResult<FileSystem> resSrc = 
392      fsState.resolve(getUriPath(src), false); 
393  
394    if (resSrc.isInternalDir()) {
395      throw readOnlyMountTable("rename", src);
396    }
397      
398    InodeTree.ResolveResult<FileSystem> resDst = 
399      fsState.resolve(getUriPath(dst), false);
400    if (resDst.isInternalDir()) {
401          throw readOnlyMountTable("rename", dst);
402    }
403    /**
404    // Alternate 1: renames within same file system - valid but we disallow
405    // Alternate 2: (as described in next para - valid but we have disallowed it
406    //
407    // Note we compare the URIs. the URIs include the link targets. 
408    // hence we allow renames across mount links as long as the mount links
409    // point to the same target.
410    if (!resSrc.targetFileSystem.getUri().equals(
411              resDst.targetFileSystem.getUri())) {
412      throw new IOException("Renames across Mount points not supported");
413    }
414    */
415    
416    //
417    // Alternate 3 : renames ONLY within the the same mount links.
418    //
419    if (resSrc.targetFileSystem !=resDst.targetFileSystem) {
420      throw new IOException("Renames across Mount points not supported");
421    }
422    return resSrc.targetFileSystem.rename(resSrc.remainingPath,
423        resDst.remainingPath);
424  }
425  
426  @Override
427  public void setOwner(final Path f, final String username,
428      final String groupname) throws AccessControlException,
429      FileNotFoundException,
430      IOException {
431    InodeTree.ResolveResult<FileSystem> res = 
432      fsState.resolve(getUriPath(f), true);
433    res.targetFileSystem.setOwner(res.remainingPath, username, groupname); 
434  }
435
436  @Override
437  public void setPermission(final Path f, final FsPermission permission)
438      throws AccessControlException, FileNotFoundException,
439      IOException {
440    InodeTree.ResolveResult<FileSystem> res = 
441      fsState.resolve(getUriPath(f), true);
442    res.targetFileSystem.setPermission(res.remainingPath, permission); 
443  }
444
445  @Override
446  public boolean setReplication(final Path f, final short replication)
447      throws AccessControlException, FileNotFoundException,
448      IOException {
449    InodeTree.ResolveResult<FileSystem> res = 
450      fsState.resolve(getUriPath(f), true);
451    return res.targetFileSystem.setReplication(res.remainingPath, replication);
452  }
453
454  @Override
455  public void setTimes(final Path f, final long mtime, final long atime)
456      throws AccessControlException, FileNotFoundException,
457      IOException {
458    InodeTree.ResolveResult<FileSystem> res = 
459      fsState.resolve(getUriPath(f), true);
460    res.targetFileSystem.setTimes(res.remainingPath, mtime, atime); 
461  }
462
463  @Override
464  public void setVerifyChecksum(final boolean verifyChecksum) { 
465    List<InodeTree.MountPoint<FileSystem>> mountPoints = 
466        fsState.getMountPoints();
467    for (InodeTree.MountPoint<FileSystem> mount : mountPoints) {
468      mount.target.targetFileSystem.setVerifyChecksum(verifyChecksum);
469    }
470  }
471  
472  @Override
473  public long getDefaultBlockSize() {
474    throw new NotInMountpointException("getDefaultBlockSize");
475  }
476
477  @Override
478  public short getDefaultReplication() {
479    throw new NotInMountpointException("getDefaultReplication");
480  }
481
482  @Override
483  public FsServerDefaults getServerDefaults() throws IOException {
484    throw new NotInMountpointException("getServerDefaults");
485  }
486
487  @Override
488  public long getDefaultBlockSize(Path f) {
489    try {
490      InodeTree.ResolveResult<FileSystem> res =
491        fsState.resolve(getUriPath(f), true);
492      return res.targetFileSystem.getDefaultBlockSize(res.remainingPath);
493    } catch (FileNotFoundException e) {
494      throw new NotInMountpointException(f, "getDefaultBlockSize"); 
495    }
496  }
497
498  @Override
499  public short getDefaultReplication(Path f) {
500    try {
501      InodeTree.ResolveResult<FileSystem> res =
502        fsState.resolve(getUriPath(f), true);
503      return res.targetFileSystem.getDefaultReplication(res.remainingPath);
504    } catch (FileNotFoundException e) {
505      throw new NotInMountpointException(f, "getDefaultReplication"); 
506    }
507  }
508
509  @Override
510  public FsServerDefaults getServerDefaults(Path f) throws IOException {
511    InodeTree.ResolveResult<FileSystem> res =
512      fsState.resolve(getUriPath(f), true);
513    return res.targetFileSystem.getServerDefaults(res.remainingPath);    
514  }
515
516  @Override
517  public ContentSummary getContentSummary(Path f) throws IOException {
518    InodeTree.ResolveResult<FileSystem> res = 
519      fsState.resolve(getUriPath(f), true);
520    return res.targetFileSystem.getContentSummary(res.remainingPath);
521  }
522
523  @Override
524  public void setWriteChecksum(final boolean writeChecksum) { 
525    List<InodeTree.MountPoint<FileSystem>> mountPoints = 
526        fsState.getMountPoints();
527    for (InodeTree.MountPoint<FileSystem> mount : mountPoints) {
528      mount.target.targetFileSystem.setWriteChecksum(writeChecksum);
529    }
530  }
531
532  @Override
533  public FileSystem[] getChildFileSystems() {
534    List<InodeTree.MountPoint<FileSystem>> mountPoints =
535        fsState.getMountPoints();
536    Set<FileSystem> children = new HashSet<FileSystem>();
537    for (InodeTree.MountPoint<FileSystem> mountPoint : mountPoints) {
538      FileSystem targetFs = mountPoint.target.targetFileSystem;
539      children.addAll(Arrays.asList(targetFs.getChildFileSystems()));
540    }
541    return children.toArray(new FileSystem[]{});
542  }
543  
544  public MountPoint[] getMountPoints() {
545    List<InodeTree.MountPoint<FileSystem>> mountPoints = 
546                  fsState.getMountPoints();
547    
548    MountPoint[] result = new MountPoint[mountPoints.size()];
549    for ( int i = 0; i < mountPoints.size(); ++i ) {
550      result[i] = new MountPoint(new Path(mountPoints.get(i).src), 
551                              mountPoints.get(i).target.targetDirLinkList);
552    }
553    return result;
554  }
555  
556  /*
557   * An instance of this class represents an internal dir of the viewFs 
558   * that is internal dir of the mount table.
559   * It is a read only mount tables and create, mkdir or delete operations
560   * are not allowed.
561   * If called on create or mkdir then this target is the parent of the
562   * directory in which one is trying to create or mkdir; hence
563   * in this case the path name passed in is the last component. 
564   * Otherwise this target is the end point of the path and hence
565   * the path name passed in is null. 
566   */
567  static class InternalDirOfViewFs extends FileSystem {
568    final InodeTree.INodeDir<FileSystem>  theInternalDir;
569    final long creationTime; // of the the mount table
570    final UserGroupInformation ugi; // the user/group of user who created mtable
571    final URI myUri;
572    
573    public InternalDirOfViewFs(final InodeTree.INodeDir<FileSystem> dir,
574        final long cTime, final UserGroupInformation ugi, URI uri)
575      throws URISyntaxException {
576      myUri = uri;
577      try {
578        initialize(myUri, new Configuration());
579      } catch (IOException e) {
580        throw new RuntimeException("Cannot occur");
581      }
582      theInternalDir = dir;
583      creationTime = cTime;
584      this.ugi = ugi;
585    }
586
587    static private void checkPathIsSlash(final Path f) throws IOException {
588      if (f != InodeTree.SlashPath) {
589        throw new IOException (
590        "Internal implementation error: expected file name to be /" );
591      }
592    }
593    
594    @Override
595    public URI getUri() {
596      return myUri;
597    }
598
599    @Override
600    public Path getWorkingDirectory() {
601      throw new RuntimeException (
602      "Internal impl error: getWorkingDir should not have been called" );
603    }
604
605    @Override
606    public void setWorkingDirectory(final Path new_dir) {
607      throw new RuntimeException (
608      "Internal impl error: getWorkingDir should not have been called" ); 
609    }
610
611    @Override
612    public FSDataOutputStream append(final Path f, final int bufferSize,
613        final Progressable progress) throws IOException {
614      throw readOnlyMountTable("append", f);
615    }
616
617    @Override
618    public FSDataOutputStream create(final Path f,
619        final FsPermission permission, final boolean overwrite,
620        final int bufferSize, final short replication, final long blockSize,
621        final Progressable progress) throws AccessControlException {
622      throw readOnlyMountTable("create", f);
623    }
624
625    @Override
626    public boolean delete(final Path f, final boolean recursive)
627        throws AccessControlException, IOException {
628      checkPathIsSlash(f);
629      throw readOnlyMountTable("delete", f);
630    }
631    
632    @Override
633    @SuppressWarnings("deprecation")
634    public boolean delete(final Path f)
635        throws AccessControlException, IOException {
636      return delete(f, true);
637    }
638
639    @Override
640    public BlockLocation[] getFileBlockLocations(final FileStatus fs,
641        final long start, final long len) throws 
642        FileNotFoundException, IOException {
643      checkPathIsSlash(fs.getPath());
644      throw new FileNotFoundException("Path points to dir not a file");
645    }
646
647    @Override
648    public FileChecksum getFileChecksum(final Path f)
649        throws FileNotFoundException, IOException {
650      checkPathIsSlash(f);
651      throw new FileNotFoundException("Path points to dir not a file");
652    }
653
654    @Override
655    public FileStatus getFileStatus(Path f) throws IOException {
656      checkPathIsSlash(f);
657      return new FileStatus(0, true, 0, 0, creationTime, creationTime,
658          PERMISSION_RRR, ugi.getUserName(), ugi.getGroupNames()[0],
659
660          new Path(theInternalDir.fullPath).makeQualified(
661              myUri, null));
662    }
663    
664
665    @Override
666    public FileStatus[] listStatus(Path f) throws AccessControlException,
667        FileNotFoundException, IOException {
668      checkPathIsSlash(f);
669      FileStatus[] result = new FileStatus[theInternalDir.children.size()];
670      int i = 0;
671      for (Entry<String, INode<FileSystem>> iEntry : 
672                                          theInternalDir.children.entrySet()) {
673        INode<FileSystem> inode = iEntry.getValue();
674        if (inode instanceof INodeLink ) {
675          INodeLink<FileSystem> link = (INodeLink<FileSystem>) inode;
676
677          result[i++] = new FileStatus(0, false, 0, 0,
678            creationTime, creationTime, PERMISSION_RRR,
679            ugi.getUserName(), ugi.getGroupNames()[0],
680            link.getTargetLink(),
681            new Path(inode.fullPath).makeQualified(
682                myUri, null));
683        } else {
684          result[i++] = new FileStatus(0, true, 0, 0,
685            creationTime, creationTime, PERMISSION_RRR,
686            ugi.getUserName(), ugi.getGroupNames()[0],
687            new Path(inode.fullPath).makeQualified(
688                myUri, null));
689        }
690      }
691      return result;
692    }
693
694    @Override
695    public boolean mkdirs(Path dir, FsPermission permission)
696        throws AccessControlException, FileAlreadyExistsException {
697      if (theInternalDir.isRoot && dir == null) {
698        throw new FileAlreadyExistsException("/ already exits");
699      }
700      // Note dir starts with /
701      if (theInternalDir.children.containsKey(dir.toString().substring(1))) {
702        return true; // this is the stupid semantics of FileSystem
703      }
704      throw readOnlyMountTable("mkdirs",  dir);
705    }
706
707    @Override
708    public FSDataInputStream open(Path f, int bufferSize)
709        throws AccessControlException, FileNotFoundException, IOException {
710      checkPathIsSlash(f);
711      throw new FileNotFoundException("Path points to dir not a file");
712    }
713
714    @Override
715    public boolean rename(Path src, Path dst) throws AccessControlException,
716        IOException {
717      checkPathIsSlash(src);
718      checkPathIsSlash(dst);
719      throw readOnlyMountTable("rename", src);     
720    }
721
722    @Override
723    public void setOwner(Path f, String username, String groupname)
724        throws AccessControlException, IOException {
725      checkPathIsSlash(f);
726      throw readOnlyMountTable("setOwner", f);
727    }
728
729    @Override
730    public void setPermission(Path f, FsPermission permission)
731        throws AccessControlException, IOException {
732      checkPathIsSlash(f);
733      throw readOnlyMountTable("setPermission", f);    
734    }
735
736    @Override
737    public boolean setReplication(Path f, short replication)
738        throws AccessControlException, IOException {
739      checkPathIsSlash(f);
740      throw readOnlyMountTable("setReplication", f);
741    }
742
743    @Override
744    public void setTimes(Path f, long mtime, long atime)
745        throws AccessControlException, IOException {
746      checkPathIsSlash(f);
747      throw readOnlyMountTable("setTimes", f);    
748    }
749
750    @Override
751    public void setVerifyChecksum(boolean verifyChecksum) {
752      // Noop for viewfs
753    }
754
755    @Override
756    public FsServerDefaults getServerDefaults(Path f) throws IOException {
757      throw new NotInMountpointException(f, "getServerDefaults");
758    }
759    
760    @Override
761    public long getDefaultBlockSize(Path f) {
762      throw new NotInMountpointException(f, "getDefaultBlockSize");
763    }
764
765    @Override
766    public short getDefaultReplication(Path f) {
767      throw new NotInMountpointException(f, "getDefaultReplication");
768    }
769  }
770}