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.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      
601      
602      /*
603       * An instance of this class represents an internal dir of the viewFs 
604       * ie internal dir of the mount table.
605       * It is a ready only mount tbale and create, mkdir or delete operations
606       * are not allowed.
607       * If called on create or mkdir then this target is the parent of the
608       * directory in which one is trying to create or mkdir; hence
609       * in this case the path name passed in is the last component. 
610       * Otherwise this target is the end point of the path and hence
611       * the path name passed in is null. 
612       */
613      static class InternalDirOfViewFs extends AbstractFileSystem {
614        
615        final InodeTree.INodeDir<AbstractFileSystem>  theInternalDir;
616        final long creationTime; // of the the mount table
617        final UserGroupInformation ugi; // the user/group of user who created mtable
618        final URI myUri; // the URI of the outer ViewFs
619        
620        public InternalDirOfViewFs(final InodeTree.INodeDir<AbstractFileSystem> dir,
621            final long cTime, final UserGroupInformation ugi, final URI uri)
622          throws URISyntaxException {
623          super(FsConstants.VIEWFS_URI, FsConstants.VIEWFS_SCHEME, false, -1);
624          theInternalDir = dir;
625          creationTime = cTime;
626          this.ugi = ugi;
627          myUri = uri;
628        }
629    
630        static private void checkPathIsSlash(final Path f) throws IOException {
631          if (f != InodeTree.SlashPath) {
632            throw new IOException (
633            "Internal implementation error: expected file name to be /" );
634          }
635        }
636    
637        @Override
638        public FSDataOutputStream createInternal(final Path f,
639            final EnumSet<CreateFlag> flag, final FsPermission absolutePermission,
640            final int bufferSize, final short replication, final long blockSize,
641            final Progressable progress, final ChecksumOpt checksumOpt,
642            final boolean createParent) throws AccessControlException,
643            FileAlreadyExistsException, FileNotFoundException,
644            ParentNotDirectoryException, UnsupportedFileSystemException,
645            UnresolvedLinkException, IOException {
646          throw readOnlyMountTable("create", f);
647        }
648    
649        @Override
650        public boolean delete(final Path f, final boolean recursive)
651            throws AccessControlException, IOException {
652          checkPathIsSlash(f);
653          throw readOnlyMountTable("delete", f);
654        }
655    
656        @Override
657        public BlockLocation[] getFileBlockLocations(final Path f, final long start,
658            final long len) throws FileNotFoundException, IOException {
659          checkPathIsSlash(f);
660          throw new FileNotFoundException("Path points to dir not a file");
661        }
662    
663        @Override
664        public FileChecksum getFileChecksum(final Path f)
665            throws FileNotFoundException, IOException {
666          checkPathIsSlash(f);
667          throw new FileNotFoundException("Path points to dir not a file");
668        }
669    
670        @Override
671        public FileStatus getFileStatus(final Path f) throws IOException {
672          checkPathIsSlash(f);
673          return new FileStatus(0, true, 0, 0, creationTime, creationTime, 
674              PERMISSION_RRR, ugi.getUserName(), ugi.getGroupNames()[0],
675              new Path(theInternalDir.fullPath).makeQualified(
676                  myUri, null));
677        }
678        
679        @Override
680        public FileStatus getFileLinkStatus(final Path f)
681            throws FileNotFoundException {
682          // look up i internalDirs children - ignore first Slash
683          INode<AbstractFileSystem> inode =
684            theInternalDir.children.get(f.toUri().toString().substring(1)); 
685          if (inode == null) {
686            throw new FileNotFoundException(
687                "viewFs internal mount table - missing entry:" + f);
688          }
689          FileStatus result;
690          if (inode instanceof INodeLink) {
691            INodeLink<AbstractFileSystem> inodelink = 
692              (INodeLink<AbstractFileSystem>) inode;
693            result = new FileStatus(0, false, 0, 0, creationTime, creationTime,
694                PERMISSION_RRR, ugi.getUserName(), ugi.getGroupNames()[0],
695                inodelink.getTargetLink(),
696                new Path(inode.fullPath).makeQualified(
697                    myUri, null));
698          } else {
699            result = new FileStatus(0, true, 0, 0, creationTime, creationTime,
700              PERMISSION_RRR, ugi.getUserName(), ugi.getGroupNames()[0],
701              new Path(inode.fullPath).makeQualified(
702                  myUri, null));
703          }
704          return result;
705        }
706        
707        @Override
708        public FsStatus getFsStatus() {
709          return new FsStatus(0, 0, 0);
710        }
711    
712        @Override
713        public FsServerDefaults getServerDefaults() throws IOException {
714          throw new IOException("FsServerDefaults not implemented yet");
715        }
716    
717        @Override
718        public int getUriDefaultPort() {
719          return -1;
720        }
721    
722        @Override
723        public FileStatus[] listStatus(final Path f) throws AccessControlException,
724            IOException {
725          checkPathIsSlash(f);
726          FileStatus[] result = new FileStatus[theInternalDir.children.size()];
727          int i = 0;
728          for (Entry<String, INode<AbstractFileSystem>> iEntry : 
729                                              theInternalDir.children.entrySet()) {
730            INode<AbstractFileSystem> inode = iEntry.getValue();
731    
732            
733            if (inode instanceof INodeLink ) {
734              INodeLink<AbstractFileSystem> link = 
735                (INodeLink<AbstractFileSystem>) inode;
736    
737              result[i++] = new FileStatus(0, false, 0, 0,
738                creationTime, creationTime,
739                PERMISSION_RRR, ugi.getUserName(), ugi.getGroupNames()[0],
740                link.getTargetLink(),
741                new Path(inode.fullPath).makeQualified(
742                    myUri, null));
743            } else {
744              result[i++] = new FileStatus(0, true, 0, 0,
745                creationTime, creationTime,
746                PERMISSION_RRR, ugi.getUserName(), ugi.getGroupNames()[0],
747                new Path(inode.fullPath).makeQualified(
748                    myUri, null));
749            }
750          }
751          return result;
752        }
753    
754        @Override
755        public void mkdir(final Path dir, final FsPermission permission,
756            final boolean createParent) throws AccessControlException,
757            FileAlreadyExistsException {
758          if (theInternalDir.isRoot && dir == null) {
759            throw new FileAlreadyExistsException("/ already exits");
760          }
761          throw readOnlyMountTable("mkdir", dir);
762        }
763    
764        @Override
765        public FSDataInputStream open(final Path f, final int bufferSize)
766            throws FileNotFoundException, IOException {
767          checkPathIsSlash(f);
768          throw new FileNotFoundException("Path points to dir not a file");
769        }
770    
771        @Override
772        public void renameInternal(final Path src, final Path dst)
773            throws AccessControlException, IOException {
774          checkPathIsSlash(src);
775          checkPathIsSlash(dst);
776          throw readOnlyMountTable("rename", src);     
777        }
778    
779        @Override
780        public boolean supportsSymlinks() {
781          return true;
782        }
783        
784        @Override
785        public void createSymlink(final Path target, final Path link,
786            final boolean createParent) throws AccessControlException {
787          throw readOnlyMountTable("createSymlink", link);    
788        }
789    
790        @Override
791        public Path getLinkTarget(final Path f) throws FileNotFoundException,
792            IOException {
793          return getFileLinkStatus(f).getSymlink();
794        }
795    
796        @Override
797        public void setOwner(final Path f, final String username,
798            final String groupname) throws AccessControlException, IOException {
799          checkPathIsSlash(f);
800          throw readOnlyMountTable("setOwner", f);
801        }
802    
803        @Override
804        public void setPermission(final Path f, final FsPermission permission)
805            throws AccessControlException, IOException {
806          checkPathIsSlash(f);
807          throw readOnlyMountTable("setPermission", f);    
808        }
809    
810        @Override
811        public boolean setReplication(final Path f, final short replication)
812            throws AccessControlException, IOException {
813          checkPathIsSlash(f);
814          throw readOnlyMountTable("setReplication", f);
815        }
816    
817        @Override
818        public void setTimes(final Path f, final long mtime, final long atime)
819            throws AccessControlException, IOException {
820          checkPathIsSlash(f);
821          throw readOnlyMountTable("setTimes", f);    
822        }
823    
824        @Override
825        public void setVerifyChecksum(final boolean verifyChecksum)
826            throws AccessControlException {
827          throw readOnlyMountTable("setVerifyChecksum", "");   
828        }
829      }
830    }