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