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