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.ArrayList;
027    import java.util.EnumSet;
028    import java.util.List;
029    import java.util.Map.Entry;
030    
031    import org.apache.hadoop.classification.InterfaceAudience;
032    import org.apache.hadoop.classification.InterfaceStability;
033    import org.apache.hadoop.conf.Configuration;
034    import org.apache.hadoop.fs.AbstractFileSystem;
035    import org.apache.hadoop.fs.BlockLocation;
036    import org.apache.hadoop.fs.CreateFlag;
037    import org.apache.hadoop.fs.FSDataInputStream;
038    import org.apache.hadoop.fs.FSDataOutputStream;
039    import org.apache.hadoop.fs.FileAlreadyExistsException;
040    import org.apache.hadoop.fs.FileChecksum;
041    import org.apache.hadoop.fs.FileStatus;
042    import org.apache.hadoop.fs.FsConstants;
043    import org.apache.hadoop.fs.FsServerDefaults;
044    import org.apache.hadoop.fs.FsStatus;
045    import org.apache.hadoop.fs.Options.ChecksumOpt;
046    import org.apache.hadoop.fs.ParentNotDirectoryException;
047    import org.apache.hadoop.fs.Path;
048    import org.apache.hadoop.fs.RemoteIterator;
049    import org.apache.hadoop.fs.UnresolvedLinkException;
050    import org.apache.hadoop.fs.UnsupportedFileSystemException;
051    import org.apache.hadoop.fs.local.LocalConfigKeys;
052    import org.apache.hadoop.fs.permission.FsPermission;
053    import org.apache.hadoop.fs.viewfs.InodeTree.INode;
054    import org.apache.hadoop.fs.viewfs.InodeTree.INodeLink;
055    import org.apache.hadoop.security.AccessControlException;
056    import org.apache.hadoop.security.UserGroupInformation;
057    import org.apache.hadoop.security.token.Token;
058    import org.apache.hadoop.util.Progressable;
059    import org.apache.hadoop.util.Time;
060    
061    
062    /**
063     * ViewFs (extends the AbstractFileSystem interface) implements a client-side
064     * mount table. The viewFs file system is implemented completely in memory on
065     * the client side. The client-side mount table allows a client to provide a 
066     * customized view of a file system namespace that is composed from 
067     * one or more individual file systems (a localFs or Hdfs, S3fs, etc).
068     * For example one could have a mount table that provides links such as
069     * <ul>
070     * <li>  /user          -> hdfs://nnContainingUserDir/user
071     * <li>  /project/foo   -> hdfs://nnProject1/projects/foo
072     * <li>  /project/bar   -> hdfs://nnProject2/projects/bar
073     * <li>  /tmp           -> hdfs://nnTmp/privateTmpForUserXXX
074     * </ul> 
075     * 
076     * ViewFs is specified with the following URI: <b>viewfs:///</b> 
077     * <p>
078     * To use viewfs one would typically set the default file system in the
079     * config  (i.e. fs.default.name< = viewfs:///) along with the
080     * mount table config variables as described below. 
081     * 
082     * <p>
083     * <b> ** Config variables to specify the mount table entries ** </b>
084     * <p>
085     * 
086     * The file system is initialized from the standard Hadoop config through
087     * config variables.
088     * See {@link FsConstants} for URI and Scheme constants; 
089     * See {@link Constants} for config var constants; 
090     * see {@link ConfigUtil} for convenient lib.
091     * 
092     * <p>
093     * All the mount table config entries for view fs are prefixed by 
094     * <b>fs.viewfs.mounttable.</b>
095     * For example the above example can be specified with the following
096     *  config variables:
097     *  <ul>
098     *  <li> fs.viewfs.mounttable.default.link./user=
099     *  hdfs://nnContainingUserDir/user
100     *  <li> fs.viewfs.mounttable.default.link./project/foo=
101     *  hdfs://nnProject1/projects/foo
102     *  <li> fs.viewfs.mounttable.default.link./project/bar=
103     *  hdfs://nnProject2/projects/bar
104     *  <li> fs.viewfs.mounttable.default.link./tmp=
105     *  hdfs://nnTmp/privateTmpForUserXXX
106     *  </ul>
107     *  
108     * The default mount table (when no authority is specified) is 
109     * from config variables prefixed by <b>fs.viewFs.mounttable.default </b>
110     * The authority component of a URI can be used to specify a different mount
111     * table. For example,
112     * <ul>
113     * <li>  viewfs://sanjayMountable/
114     * </ul>
115     * is initialized from fs.viewFs.mounttable.sanjayMountable.* config variables.
116     * 
117     *  <p> 
118     *  <b> **** Merge Mounts **** </b>(NOTE: merge mounts are not implemented yet.)
119     *  <p>
120     *  
121     *   One can also use "MergeMounts" to merge several directories (this is
122     *   sometimes  called union-mounts or junction-mounts in the literature.
123     *   For example of the home directories are stored on say two file systems
124     *   (because they do not fit on one) then one could specify a mount
125     *   entry such as following merges two dirs:
126     *   <ul>
127     *   <li> /user -> hdfs://nnUser1/user,hdfs://nnUser2/user
128     *   </ul>
129     *  Such a mergeLink can be specified with the following config var where ","
130     *  is used as the separator for each of links to be merged:
131     *  <ul>
132     *  <li> fs.viewfs.mounttable.default.linkMerge./user=
133     *  hdfs://nnUser1/user,hdfs://nnUser1/user
134     *  </ul>
135     *   A special case of the merge mount is where mount table's root is merged
136     *   with the root (slash) of another file system:
137     *   <ul>
138     *   <li>    fs.viewfs.mounttable.default.linkMergeSlash=hdfs://nn99/
139     *   </ul>
140     *   In this cases the root of the mount table is merged with the root of
141     *            <b>hdfs://nn99/ </b> 
142     */
143    
144    @InterfaceAudience.Public
145    @InterfaceStability.Evolving /*Evolving for a release,to be changed to Stable */
146    public class ViewFs extends AbstractFileSystem {
147      final long creationTime; // of the the mount table
148      final UserGroupInformation ugi; // the user/group of user who created mtable
149      final Configuration config;
150      InodeTree<AbstractFileSystem> fsState;  // the fs state; ie the mount table
151      Path homeDir = null;
152      
153      static AccessControlException readOnlyMountTable(final String operation,
154          final String p) {
155        return new AccessControlException( 
156            "InternalDir of ViewFileSystem is readonly; operation=" + operation + 
157            "Path=" + p);
158      }
159      static AccessControlException readOnlyMountTable(final String operation,
160          final Path p) {
161        return readOnlyMountTable(operation, p.toString());
162      }
163      
164      
165      static public class MountPoint {
166        private Path src;       // the src of the mount
167        private URI[] targets; //  target of the mount; Multiple targets imply mergeMount
168        MountPoint(Path srcPath, URI[] targetURIs) {
169          src = srcPath;
170          targets = targetURIs;
171        }
172        Path getSrc() {
173          return src;
174        }
175        URI[] getTargets() {
176          return targets;
177        }
178      }
179      
180      public ViewFs(final Configuration conf) throws IOException,
181          URISyntaxException {
182        this(FsConstants.VIEWFS_URI, conf);
183      }
184      
185      /**
186       * This constructor has the signature needed by
187       * {@link AbstractFileSystem#createFileSystem(URI, Configuration)}.
188       * 
189       * @param theUri which must be that of ViewFs
190       * @param conf
191       * @throws IOException
192       * @throws URISyntaxException 
193       */
194      ViewFs(final URI theUri, final Configuration conf) throws IOException,
195          URISyntaxException {
196        super(theUri, FsConstants.VIEWFS_SCHEME, false, -1);
197        creationTime = Time.now();
198        ugi = UserGroupInformation.getCurrentUser();
199        config = conf;
200        // Now build  client side view (i.e. client side mount table) from config.
201        String authority = theUri.getAuthority();
202        fsState = new InodeTree<AbstractFileSystem>(conf, authority) {
203    
204          @Override
205          protected
206          AbstractFileSystem getTargetFileSystem(final URI uri)
207            throws URISyntaxException, UnsupportedFileSystemException {
208              String pathString = uri.getPath();
209              if (pathString.isEmpty()) {
210                pathString = "/";
211              }
212              return new ChRootedFs(
213                  AbstractFileSystem.createFileSystem(uri, config),
214                  new Path(pathString));
215          }
216    
217          @Override
218          protected
219          AbstractFileSystem getTargetFileSystem(
220              final INodeDir<AbstractFileSystem> dir) throws URISyntaxException {
221            return new InternalDirOfViewFs(dir, creationTime, ugi, getUri());
222          }
223    
224          @Override
225          protected
226          AbstractFileSystem getTargetFileSystem(URI[] mergeFsURIList)
227              throws URISyntaxException, UnsupportedFileSystemException {
228            throw new UnsupportedFileSystemException("mergefs not implemented yet");
229            // return MergeFs.createMergeFs(mergeFsURIList, config);
230          }
231        };
232      }
233    
234      @Override
235      public FsServerDefaults getServerDefaults() throws IOException {
236        return LocalConfigKeys.getServerDefaults(); 
237      }
238    
239      @Override
240      public int getUriDefaultPort() {
241        return -1;
242      }
243     
244      @Override
245      public Path getHomeDirectory() {
246        if (homeDir == null) {
247          String base = fsState.getHomeDirPrefixValue();
248          if (base == null) {
249            base = "/user";
250          }
251          homeDir = (base.equals("/") ? 
252            this.makeQualified(new Path(base + ugi.getShortUserName())):
253            this.makeQualified(new Path(base + "/" + ugi.getShortUserName())));
254        }
255        return homeDir;
256      }
257      
258      @Override
259      public Path resolvePath(final Path f) throws FileNotFoundException,
260              AccessControlException, UnresolvedLinkException, IOException {
261        final InodeTree.ResolveResult<AbstractFileSystem> res;
262          res = fsState.resolve(getUriPath(f), true);
263        if (res.isInternalDir()) {
264          return f;
265        }
266        return res.targetFileSystem.resolvePath(res.remainingPath);
267    
268      }
269      
270      @Override
271      public FSDataOutputStream createInternal(final Path f,
272          final EnumSet<CreateFlag> flag, final FsPermission absolutePermission,
273          final int bufferSize, final short replication, final long blockSize,
274          final Progressable progress, final ChecksumOpt checksumOpt,
275          final boolean createParent) throws AccessControlException,
276          FileAlreadyExistsException, FileNotFoundException,
277          ParentNotDirectoryException, UnsupportedFileSystemException,
278          UnresolvedLinkException, IOException {
279        InodeTree.ResolveResult<AbstractFileSystem> res;
280        try {
281          res = fsState.resolve(getUriPath(f), false);
282        } catch (FileNotFoundException e) {
283          if (createParent) {
284            throw readOnlyMountTable("create", f);
285          } else {
286            throw e;
287          }
288        }
289        assert(res.remainingPath != null);
290        return res.targetFileSystem.createInternal(res.remainingPath, flag,
291            absolutePermission, bufferSize, replication,
292            blockSize, progress, checksumOpt,
293            createParent);
294      }
295    
296      @Override
297      public boolean delete(final Path f, final boolean recursive)
298          throws AccessControlException, FileNotFoundException,
299          UnresolvedLinkException, IOException {
300        InodeTree.ResolveResult<AbstractFileSystem> res = 
301          fsState.resolve(getUriPath(f), true);
302        // If internal dir or target is a mount link (ie remainingPath is Slash)
303        if (res.isInternalDir() || res.remainingPath == InodeTree.SlashPath) {
304          throw new AccessControlException(
305              "Cannot delete internal mount table directory: " + f);
306        }
307        return res.targetFileSystem.delete(res.remainingPath, recursive);
308      }
309    
310      @Override
311      public BlockLocation[] getFileBlockLocations(final Path f, final long start,
312          final long len) throws AccessControlException, FileNotFoundException,
313          UnresolvedLinkException, IOException {
314        InodeTree.ResolveResult<AbstractFileSystem> res = 
315          fsState.resolve(getUriPath(f), true);
316        return
317          res.targetFileSystem.getFileBlockLocations(res.remainingPath, start, len);
318      }
319    
320      @Override
321      public FileChecksum getFileChecksum(final Path f)
322          throws AccessControlException, FileNotFoundException,
323          UnresolvedLinkException, IOException {
324        InodeTree.ResolveResult<AbstractFileSystem> res = 
325          fsState.resolve(getUriPath(f), true);
326        return res.targetFileSystem.getFileChecksum(res.remainingPath);
327      }
328    
329      @Override
330      public FileStatus getFileStatus(final Path f) throws AccessControlException,
331          FileNotFoundException, UnresolvedLinkException, IOException {
332        InodeTree.ResolveResult<AbstractFileSystem> res = 
333          fsState.resolve(getUriPath(f), true);
334    
335        //  FileStatus#getPath is a fully qualified path relative to the root of 
336        // target file system.
337        // We need to change it to viewfs URI - relative to root of mount table.
338        
339        // The implementors of RawLocalFileSystem were trying to be very smart.
340        // They implement FileStatus#getOwener lazily -- the object
341        // returned is really a RawLocalFileSystem that expect the
342        // FileStatus#getPath to be unchanged so that it can get owner when needed.
343        // Hence we need to interpose a new ViewFsFileStatus that works around.
344        
345        
346        FileStatus status =  res.targetFileSystem.getFileStatus(res.remainingPath);
347        return new ViewFsFileStatus(status, this.makeQualified(f));
348      }
349    
350      @Override
351      public FileStatus getFileLinkStatus(final Path f)
352         throws AccessControlException, FileNotFoundException,
353         UnsupportedFileSystemException, IOException {
354        InodeTree.ResolveResult<AbstractFileSystem> res = 
355          fsState.resolve(getUriPath(f), false); // do not follow mount link
356        return res.targetFileSystem.getFileLinkStatus(res.remainingPath);
357      }
358      
359      @Override
360      public FsStatus getFsStatus() throws AccessControlException,
361          FileNotFoundException, IOException {
362        return new FsStatus(0, 0, 0);
363      }
364    
365      @Override
366      public RemoteIterator<FileStatus> listStatusIterator(final Path f)
367        throws AccessControlException, FileNotFoundException,
368        UnresolvedLinkException, IOException {
369        final InodeTree.ResolveResult<AbstractFileSystem> res =
370          fsState.resolve(getUriPath(f), true);
371        final RemoteIterator<FileStatus> fsIter =
372          res.targetFileSystem.listStatusIterator(res.remainingPath);
373        if (res.isInternalDir()) {
374          return fsIter;
375        }
376        
377        return new RemoteIterator<FileStatus>() {
378          final RemoteIterator<FileStatus> myIter;
379          final ChRootedFs targetFs;
380          { // Init
381              myIter = fsIter;
382              targetFs = (ChRootedFs) res.targetFileSystem;
383          }
384          
385          @Override
386          public boolean hasNext() throws IOException {
387            return myIter.hasNext();
388          }
389          
390          @Override
391          public FileStatus next() throws IOException {
392            FileStatus status =  myIter.next();
393            String suffix = targetFs.stripOutRoot(status.getPath());
394            return new ViewFsFileStatus(status, makeQualified(
395                suffix.length() == 0 ? f : new Path(res.resolvedPath, suffix)));
396          }
397        };
398      }
399      
400      @Override
401      public FileStatus[] listStatus(final Path f) throws AccessControlException,
402          FileNotFoundException, UnresolvedLinkException, IOException {
403        InodeTree.ResolveResult<AbstractFileSystem> res =
404          fsState.resolve(getUriPath(f), true);
405        
406        FileStatus[] statusLst = res.targetFileSystem.listStatus(res.remainingPath);
407        if (!res.isInternalDir()) {
408          // We need to change the name in the FileStatus as described in
409          // {@link #getFileStatus }
410          ChRootedFs targetFs;
411          targetFs = (ChRootedFs) res.targetFileSystem;
412          int i = 0;
413          for (FileStatus status : statusLst) {
414              String suffix = targetFs.stripOutRoot(status.getPath());
415              statusLst[i++] = new ViewFsFileStatus(status, this.makeQualified(
416                  suffix.length() == 0 ? f : new Path(res.resolvedPath, suffix)));
417          }
418        }
419        return statusLst;
420      }
421    
422      @Override
423      public void mkdir(final Path dir, final FsPermission permission,
424          final boolean createParent) throws AccessControlException,
425          FileAlreadyExistsException,
426          FileNotFoundException, UnresolvedLinkException, IOException {
427        InodeTree.ResolveResult<AbstractFileSystem> res = 
428          fsState.resolve(getUriPath(dir), false);
429        res.targetFileSystem.mkdir(res.remainingPath, permission, createParent);
430      }
431    
432      @Override
433      public FSDataInputStream open(final Path f, final int bufferSize)
434          throws AccessControlException, FileNotFoundException,
435          UnresolvedLinkException, IOException {
436        InodeTree.ResolveResult<AbstractFileSystem> res = 
437            fsState.resolve(getUriPath(f), true);
438        return res.targetFileSystem.open(res.remainingPath, bufferSize);
439      }
440    
441      
442      @Override
443      public void renameInternal(final Path src, final Path dst,
444          final boolean overwrite) throws IOException, UnresolvedLinkException {
445        // passing resolveLastComponet as false to catch renaming a mount point 
446        // itself we need to catch this as an internal operation and fail.
447        InodeTree.ResolveResult<AbstractFileSystem> resSrc = 
448          fsState.resolve(getUriPath(src), false); 
449      
450        if (resSrc.isInternalDir()) {
451          throw new AccessControlException(
452              "Cannot Rename within internal dirs of mount table: it is readOnly");
453        }
454          
455        InodeTree.ResolveResult<AbstractFileSystem> resDst = 
456                                    fsState.resolve(getUriPath(dst), false);
457        if (resDst.isInternalDir()) {
458          throw new AccessControlException(
459              "Cannot Rename within internal dirs of mount table: it is readOnly");
460        }
461        
462        /**
463        // Alternate 1: renames within same file system - valid but we disallow
464        // Alternate 2: (as described in next para - valid but we have disallowed it
465        //
466        // Note we compare the URIs. the URIs include the link targets. 
467        // hence we allow renames across mount links as long as the mount links
468        // point to the same target.
469        if (!resSrc.targetFileSystem.getUri().equals(
470                  resDst.targetFileSystem.getUri())) {
471          throw new IOException("Renames across Mount points not supported");
472        }
473        */
474        
475        //
476        // Alternate 3 : renames ONLY within the the same mount links.
477        //
478    
479        if (resSrc.targetFileSystem !=resDst.targetFileSystem) {
480          throw new IOException("Renames across Mount points not supported");
481        }
482        
483        resSrc.targetFileSystem.renameInternal(resSrc.remainingPath,
484          resDst.remainingPath, overwrite);
485      }
486    
487      @Override
488      public void renameInternal(final Path src, final Path dst)
489          throws AccessControlException, FileAlreadyExistsException,
490          FileNotFoundException, ParentNotDirectoryException,
491          UnresolvedLinkException, IOException {
492        renameInternal(src, dst, false);
493      }
494      
495      @Override
496      public boolean supportsSymlinks() {
497        return true;
498      }
499      
500      @Override
501      public void createSymlink(final Path target, final Path link,
502          final boolean createParent) throws IOException, UnresolvedLinkException {
503        InodeTree.ResolveResult<AbstractFileSystem> res;
504        try {
505          res = fsState.resolve(getUriPath(link), false);
506        } catch (FileNotFoundException e) {
507          if (createParent) {
508            throw readOnlyMountTable("createSymlink", link);
509          } else {
510            throw e;
511          }
512        }
513        assert(res.remainingPath != null);
514        res.targetFileSystem.createSymlink(target, res.remainingPath,
515            createParent);  
516      }
517    
518      @Override
519      public Path getLinkTarget(final Path f) throws IOException {
520        InodeTree.ResolveResult<AbstractFileSystem> res = 
521          fsState.resolve(getUriPath(f), false); // do not follow mount link
522        return res.targetFileSystem.getLinkTarget(res.remainingPath);
523      }
524    
525      @Override
526      public void setOwner(final Path f, final String username,
527          final String groupname) throws AccessControlException,
528          FileNotFoundException, UnresolvedLinkException, IOException {
529        InodeTree.ResolveResult<AbstractFileSystem> res = 
530          fsState.resolve(getUriPath(f), true);
531        res.targetFileSystem.setOwner(res.remainingPath, username, groupname); 
532      }
533    
534      @Override
535      public void setPermission(final Path f, final FsPermission permission)
536          throws AccessControlException, FileNotFoundException,
537          UnresolvedLinkException, IOException {
538        InodeTree.ResolveResult<AbstractFileSystem> res = 
539          fsState.resolve(getUriPath(f), true);
540        res.targetFileSystem.setPermission(res.remainingPath, permission); 
541        
542      }
543    
544      @Override
545      public boolean setReplication(final Path f, final short replication)
546          throws AccessControlException, FileNotFoundException,
547          UnresolvedLinkException, IOException {
548        InodeTree.ResolveResult<AbstractFileSystem> res = 
549          fsState.resolve(getUriPath(f), true);
550        return res.targetFileSystem.setReplication(res.remainingPath, replication);
551      }
552    
553      @Override
554      public void setTimes(final Path f, final long mtime, final long atime)
555          throws AccessControlException, FileNotFoundException,
556          UnresolvedLinkException, IOException {
557        InodeTree.ResolveResult<AbstractFileSystem> res = 
558          fsState.resolve(getUriPath(f), true);
559        res.targetFileSystem.setTimes(res.remainingPath, mtime, atime); 
560      }
561    
562      @Override
563      public void setVerifyChecksum(final boolean verifyChecksum)
564          throws AccessControlException, IOException {
565        // This is a file system level operations, however ViewFs 
566        // points to many file systems. Noop for ViewFs. 
567      }
568      
569      public MountPoint[] getMountPoints() {
570        List<InodeTree.MountPoint<AbstractFileSystem>> mountPoints = 
571                      fsState.getMountPoints();
572        
573        MountPoint[] result = new MountPoint[mountPoints.size()];
574        for ( int i = 0; i < mountPoints.size(); ++i ) {
575          result[i] = new MountPoint(new Path(mountPoints.get(i).src), 
576                                  mountPoints.get(i).target.targetDirLinkList);
577        }
578        return result;
579      }
580      
581      @Override
582      public List<Token<?>> getDelegationTokens(String renewer) throws IOException {
583        List<InodeTree.MountPoint<AbstractFileSystem>> mountPoints = 
584                    fsState.getMountPoints();
585        int initialListSize  = 0;
586        for (InodeTree.MountPoint<AbstractFileSystem> im : mountPoints) {
587          initialListSize += im.target.targetDirLinkList.length; 
588        }
589        List<Token<?>> result = new ArrayList<Token<?>>(initialListSize);
590        for ( int i = 0; i < mountPoints.size(); ++i ) {
591          List<Token<?>> tokens = 
592            mountPoints.get(i).target.targetFileSystem.getDelegationTokens(renewer);
593          if (tokens != null) {
594            result.addAll(tokens);
595          }
596        }
597        return result;
598      }
599    
600      @Override
601      public boolean isValidName(String src) {
602        // Prefix validated at mount time and rest of path validated by mount target.
603        return true;
604      }
605    
606      
607      
608      /*
609       * An instance of this class represents an internal dir of the viewFs 
610       * ie internal dir of the mount table.
611       * It is a ready only mount tbale and create, mkdir or delete operations
612       * are not allowed.
613       * If called on create or mkdir then this target is the parent of the
614       * directory in which one is trying to create or mkdir; hence
615       * in this case the path name passed in is the last component. 
616       * Otherwise this target is the end point of the path and hence
617       * the path name passed in is null. 
618       */
619      static class InternalDirOfViewFs extends AbstractFileSystem {
620        
621        final InodeTree.INodeDir<AbstractFileSystem>  theInternalDir;
622        final long creationTime; // of the the mount table
623        final UserGroupInformation ugi; // the user/group of user who created mtable
624        final URI myUri; // the URI of the outer ViewFs
625        
626        public InternalDirOfViewFs(final InodeTree.INodeDir<AbstractFileSystem> dir,
627            final long cTime, final UserGroupInformation ugi, final URI uri)
628          throws URISyntaxException {
629          super(FsConstants.VIEWFS_URI, FsConstants.VIEWFS_SCHEME, false, -1);
630          theInternalDir = dir;
631          creationTime = cTime;
632          this.ugi = ugi;
633          myUri = uri;
634        }
635    
636        static private void checkPathIsSlash(final Path f) throws IOException {
637          if (f != InodeTree.SlashPath) {
638            throw new IOException (
639            "Internal implementation error: expected file name to be /" );
640          }
641        }
642    
643        @Override
644        public FSDataOutputStream createInternal(final Path f,
645            final EnumSet<CreateFlag> flag, final FsPermission absolutePermission,
646            final int bufferSize, final short replication, final long blockSize,
647            final Progressable progress, final ChecksumOpt checksumOpt,
648            final boolean createParent) throws AccessControlException,
649            FileAlreadyExistsException, FileNotFoundException,
650            ParentNotDirectoryException, UnsupportedFileSystemException,
651            UnresolvedLinkException, IOException {
652          throw readOnlyMountTable("create", f);
653        }
654    
655        @Override
656        public boolean delete(final Path f, final boolean recursive)
657            throws AccessControlException, IOException {
658          checkPathIsSlash(f);
659          throw readOnlyMountTable("delete", f);
660        }
661    
662        @Override
663        public BlockLocation[] getFileBlockLocations(final Path f, final long start,
664            final long len) throws FileNotFoundException, IOException {
665          checkPathIsSlash(f);
666          throw new FileNotFoundException("Path points to dir not a file");
667        }
668    
669        @Override
670        public FileChecksum getFileChecksum(final Path f)
671            throws FileNotFoundException, IOException {
672          checkPathIsSlash(f);
673          throw new FileNotFoundException("Path points to dir not a file");
674        }
675    
676        @Override
677        public FileStatus getFileStatus(final Path f) throws IOException {
678          checkPathIsSlash(f);
679          return new FileStatus(0, true, 0, 0, creationTime, creationTime,
680              PERMISSION_555, ugi.getUserName(), ugi.getGroupNames()[0],
681              new Path(theInternalDir.fullPath).makeQualified(
682                  myUri, null));
683        }
684        
685        @Override
686        public FileStatus getFileLinkStatus(final Path f)
687            throws FileNotFoundException {
688          // look up i internalDirs children - ignore first Slash
689          INode<AbstractFileSystem> inode =
690            theInternalDir.children.get(f.toUri().toString().substring(1)); 
691          if (inode == null) {
692            throw new FileNotFoundException(
693                "viewFs internal mount table - missing entry:" + f);
694          }
695          FileStatus result;
696          if (inode instanceof INodeLink) {
697            INodeLink<AbstractFileSystem> inodelink = 
698              (INodeLink<AbstractFileSystem>) inode;
699            result = new FileStatus(0, false, 0, 0, creationTime, creationTime,
700                PERMISSION_555, ugi.getUserName(), ugi.getGroupNames()[0],
701                inodelink.getTargetLink(),
702                new Path(inode.fullPath).makeQualified(
703                    myUri, null));
704          } else {
705            result = new FileStatus(0, true, 0, 0, creationTime, creationTime,
706              PERMISSION_555, ugi.getUserName(), ugi.getGroupNames()[0],
707              new Path(inode.fullPath).makeQualified(
708                  myUri, null));
709          }
710          return result;
711        }
712        
713        @Override
714        public FsStatus getFsStatus() {
715          return new FsStatus(0, 0, 0);
716        }
717    
718        @Override
719        public FsServerDefaults getServerDefaults() throws IOException {
720          throw new IOException("FsServerDefaults not implemented yet");
721        }
722    
723        @Override
724        public int getUriDefaultPort() {
725          return -1;
726        }
727    
728        @Override
729        public FileStatus[] listStatus(final Path f) throws AccessControlException,
730            IOException {
731          checkPathIsSlash(f);
732          FileStatus[] result = new FileStatus[theInternalDir.children.size()];
733          int i = 0;
734          for (Entry<String, INode<AbstractFileSystem>> iEntry : 
735                                              theInternalDir.children.entrySet()) {
736            INode<AbstractFileSystem> inode = iEntry.getValue();
737    
738            
739            if (inode instanceof INodeLink ) {
740              INodeLink<AbstractFileSystem> link = 
741                (INodeLink<AbstractFileSystem>) inode;
742    
743              result[i++] = new FileStatus(0, false, 0, 0,
744                creationTime, creationTime,
745                PERMISSION_555, ugi.getUserName(), ugi.getGroupNames()[0],
746                link.getTargetLink(),
747                new Path(inode.fullPath).makeQualified(
748                    myUri, null));
749            } else {
750              result[i++] = new FileStatus(0, true, 0, 0,
751                creationTime, creationTime,
752                PERMISSION_555, ugi.getUserName(), ugi.getGroupNames()[0],
753                new Path(inode.fullPath).makeQualified(
754                    myUri, null));
755            }
756          }
757          return result;
758        }
759    
760        @Override
761        public void mkdir(final Path dir, final FsPermission permission,
762            final boolean createParent) throws AccessControlException,
763            FileAlreadyExistsException {
764          if (theInternalDir.isRoot && dir == null) {
765            throw new FileAlreadyExistsException("/ already exits");
766          }
767          throw readOnlyMountTable("mkdir", dir);
768        }
769    
770        @Override
771        public FSDataInputStream open(final Path f, final int bufferSize)
772            throws FileNotFoundException, IOException {
773          checkPathIsSlash(f);
774          throw new FileNotFoundException("Path points to dir not a file");
775        }
776    
777        @Override
778        public void renameInternal(final Path src, final Path dst)
779            throws AccessControlException, IOException {
780          checkPathIsSlash(src);
781          checkPathIsSlash(dst);
782          throw readOnlyMountTable("rename", src);     
783        }
784    
785        @Override
786        public boolean supportsSymlinks() {
787          return true;
788        }
789        
790        @Override
791        public void createSymlink(final Path target, final Path link,
792            final boolean createParent) throws AccessControlException {
793          throw readOnlyMountTable("createSymlink", link);    
794        }
795    
796        @Override
797        public Path getLinkTarget(final Path f) throws FileNotFoundException,
798            IOException {
799          return getFileLinkStatus(f).getSymlink();
800        }
801    
802        @Override
803        public void setOwner(final Path f, final String username,
804            final String groupname) throws AccessControlException, IOException {
805          checkPathIsSlash(f);
806          throw readOnlyMountTable("setOwner", f);
807        }
808    
809        @Override
810        public void setPermission(final Path f, final FsPermission permission)
811            throws AccessControlException, IOException {
812          checkPathIsSlash(f);
813          throw readOnlyMountTable("setPermission", f);    
814        }
815    
816        @Override
817        public boolean setReplication(final Path f, final short replication)
818            throws AccessControlException, IOException {
819          checkPathIsSlash(f);
820          throw readOnlyMountTable("setReplication", f);
821        }
822    
823        @Override
824        public void setTimes(final Path f, final long mtime, final long atime)
825            throws AccessControlException, IOException {
826          checkPathIsSlash(f);
827          throw readOnlyMountTable("setTimes", f);    
828        }
829    
830        @Override
831        public void setVerifyChecksum(final boolean verifyChecksum)
832            throws AccessControlException {
833          throw readOnlyMountTable("setVerifyChecksum", "");   
834        }
835      }
836    }