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    package org.apache.hadoop.fs.viewfs;
019    
020    import static org.apache.hadoop.fs.viewfs.Constants.PERMISSION_RRR;
021    
022    import java.io.FileNotFoundException;
023    import java.io.IOException;
024    import java.net.URI;
025    import java.net.URISyntaxException;
026    import java.util.Arrays;
027    import java.util.HashSet;
028    import java.util.List;
029    import java.util.Set;
030    import java.util.StringTokenizer;
031    import java.util.Map.Entry;
032    
033    import org.apache.hadoop.classification.InterfaceAudience;
034    import org.apache.hadoop.classification.InterfaceStability;
035    import org.apache.hadoop.conf.Configuration;
036    import org.apache.hadoop.fs.BlockLocation;
037    import org.apache.hadoop.fs.ContentSummary;
038    import org.apache.hadoop.fs.FSDataInputStream;
039    import org.apache.hadoop.fs.FSDataOutputStream;
040    import org.apache.hadoop.fs.FileAlreadyExistsException;
041    import org.apache.hadoop.fs.FileChecksum;
042    import org.apache.hadoop.fs.FileStatus;
043    import org.apache.hadoop.fs.FileSystem;
044    import org.apache.hadoop.fs.FsConstants;
045    import org.apache.hadoop.fs.FsServerDefaults;
046    import org.apache.hadoop.fs.InvalidPathException;
047    import org.apache.hadoop.fs.Path;
048    import org.apache.hadoop.fs.UnsupportedFileSystemException;
049    import org.apache.hadoop.fs.permission.FsPermission;
050    import org.apache.hadoop.fs.viewfs.InodeTree.INode;
051    import org.apache.hadoop.fs.viewfs.InodeTree.INodeLink;
052    import org.apache.hadoop.security.AccessControlException;
053    import org.apache.hadoop.security.UserGroupInformation;
054    import 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 */
063    public 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    }