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              return new ChRootedFs(
209                  AbstractFileSystem.createFileSystem(uri, config),
210                  new Path(uri.getPath()));
211          }
212    
213          @Override
214          protected
215          AbstractFileSystem getTargetFileSystem(
216              final INodeDir<AbstractFileSystem> dir) throws URISyntaxException {
217            return new InternalDirOfViewFs(dir, creationTime, ugi, getUri());
218          }
219    
220          @Override
221          protected
222          AbstractFileSystem getTargetFileSystem(URI[] mergeFsURIList)
223              throws URISyntaxException, UnsupportedFileSystemException {
224            throw new UnsupportedFileSystemException("mergefs not implemented yet");
225            // return MergeFs.createMergeFs(mergeFsURIList, config);
226          }
227        };
228      }
229    
230      @Override
231      public FsServerDefaults getServerDefaults() throws IOException {
232        return LocalConfigKeys.getServerDefaults(); 
233      }
234    
235      @Override
236      public int getUriDefaultPort() {
237        return -1;
238      }
239     
240      @Override
241      public Path getHomeDirectory() {
242        if (homeDir == null) {
243          String base = fsState.getHomeDirPrefixValue();
244          if (base == null) {
245            base = "/user";
246          }
247          homeDir = 
248            this.makeQualified(new Path(base + "/" + ugi.getShortUserName()));
249        }
250        return homeDir;
251      }
252      
253      @Override
254      public Path resolvePath(final Path f) throws FileNotFoundException,
255              AccessControlException, UnresolvedLinkException, IOException {
256        final InodeTree.ResolveResult<AbstractFileSystem> res;
257          res = fsState.resolve(getUriPath(f), true);
258        if (res.isInternalDir()) {
259          return f;
260        }
261        return res.targetFileSystem.resolvePath(res.remainingPath);
262    
263      }
264      
265      @Override
266      public FSDataOutputStream createInternal(final Path f,
267          final EnumSet<CreateFlag> flag, final FsPermission absolutePermission,
268          final int bufferSize, final short replication, final long blockSize,
269          final Progressable progress, final ChecksumOpt checksumOpt,
270          final boolean createParent) throws AccessControlException,
271          FileAlreadyExistsException, FileNotFoundException,
272          ParentNotDirectoryException, UnsupportedFileSystemException,
273          UnresolvedLinkException, IOException {
274        InodeTree.ResolveResult<AbstractFileSystem> res;
275        try {
276          res = fsState.resolve(getUriPath(f), false);
277        } catch (FileNotFoundException e) {
278          if (createParent) {
279            throw readOnlyMountTable("create", f);
280          } else {
281            throw e;
282          }
283        }
284        assert(res.remainingPath != null);
285        return res.targetFileSystem.createInternal(res.remainingPath, flag,
286            absolutePermission, bufferSize, replication,
287            blockSize, progress, checksumOpt,
288            createParent);
289      }
290    
291      @Override
292      public boolean delete(final Path f, final boolean recursive)
293          throws AccessControlException, FileNotFoundException,
294          UnresolvedLinkException, IOException {
295        InodeTree.ResolveResult<AbstractFileSystem> res = 
296          fsState.resolve(getUriPath(f), true);
297        // If internal dir or target is a mount link (ie remainingPath is Slash)
298        if (res.isInternalDir() || res.remainingPath == InodeTree.SlashPath) {
299          throw new AccessControlException(
300              "Cannot delete internal mount table directory: " + f);
301        }
302        return res.targetFileSystem.delete(res.remainingPath, recursive);
303      }
304    
305      @Override
306      public BlockLocation[] getFileBlockLocations(final Path f, final long start,
307          final long len) throws AccessControlException, FileNotFoundException,
308          UnresolvedLinkException, IOException {
309        InodeTree.ResolveResult<AbstractFileSystem> res = 
310          fsState.resolve(getUriPath(f), true);
311        return
312          res.targetFileSystem.getFileBlockLocations(res.remainingPath, start, len);
313      }
314    
315      @Override
316      public FileChecksum getFileChecksum(final Path f)
317          throws AccessControlException, FileNotFoundException,
318          UnresolvedLinkException, IOException {
319        InodeTree.ResolveResult<AbstractFileSystem> res = 
320          fsState.resolve(getUriPath(f), true);
321        return res.targetFileSystem.getFileChecksum(res.remainingPath);
322      }
323    
324      @Override
325      public FileStatus getFileStatus(final Path f) throws AccessControlException,
326          FileNotFoundException, UnresolvedLinkException, IOException {
327        InodeTree.ResolveResult<AbstractFileSystem> res = 
328          fsState.resolve(getUriPath(f), true);
329    
330        //  FileStatus#getPath is a fully qualified path relative to the root of 
331        // target file system.
332        // We need to change it to viewfs URI - relative to root of mount table.
333        
334        // The implementors of RawLocalFileSystem were trying to be very smart.
335        // They implement FileStatus#getOwener lazily -- the object
336        // returned is really a RawLocalFileSystem that expect the
337        // FileStatus#getPath to be unchanged so that it can get owner when needed.
338        // Hence we need to interpose a new ViewFsFileStatus that works around.
339        
340        
341        FileStatus status =  res.targetFileSystem.getFileStatus(res.remainingPath);
342        return new ViewFsFileStatus(status, this.makeQualified(f));
343      }
344    
345      @Override
346      public FileStatus getFileLinkStatus(final Path f)
347         throws AccessControlException, FileNotFoundException,
348         UnsupportedFileSystemException, IOException {
349        InodeTree.ResolveResult<AbstractFileSystem> res = 
350          fsState.resolve(getUriPath(f), false); // do not follow mount link
351        return res.targetFileSystem.getFileLinkStatus(res.remainingPath);
352      }
353      
354      @Override
355      public FsStatus getFsStatus() throws AccessControlException,
356          FileNotFoundException, IOException {
357        return new FsStatus(0, 0, 0);
358      }
359    
360      @Override
361      public RemoteIterator<FileStatus> listStatusIterator(final Path f)
362        throws AccessControlException, FileNotFoundException,
363        UnresolvedLinkException, IOException {
364        final InodeTree.ResolveResult<AbstractFileSystem> res =
365          fsState.resolve(getUriPath(f), true);
366        final RemoteIterator<FileStatus> fsIter =
367          res.targetFileSystem.listStatusIterator(res.remainingPath);
368        if (res.isInternalDir()) {
369          return fsIter;
370        }
371        
372        return new RemoteIterator<FileStatus>() {
373          final RemoteIterator<FileStatus> myIter;
374          final ChRootedFs targetFs;
375          { // Init
376              myIter = fsIter;
377              targetFs = (ChRootedFs) res.targetFileSystem;
378          }
379          
380          @Override
381          public boolean hasNext() throws IOException {
382            return myIter.hasNext();
383          }
384          
385          @Override
386          public FileStatus next() throws IOException {
387            FileStatus status =  myIter.next();
388            String suffix = targetFs.stripOutRoot(status.getPath());
389            return new ViewFsFileStatus(status, makeQualified(
390                suffix.length() == 0 ? f : new Path(res.resolvedPath, suffix)));
391          }
392        };
393      }
394      
395      @Override
396      public FileStatus[] listStatus(final Path f) throws AccessControlException,
397          FileNotFoundException, UnresolvedLinkException, IOException {
398        InodeTree.ResolveResult<AbstractFileSystem> res =
399          fsState.resolve(getUriPath(f), true);
400        
401        FileStatus[] statusLst = res.targetFileSystem.listStatus(res.remainingPath);
402        if (!res.isInternalDir()) {
403          // We need to change the name in the FileStatus as described in
404          // {@link #getFileStatus }
405          ChRootedFs targetFs;
406          targetFs = (ChRootedFs) res.targetFileSystem;
407          int i = 0;
408          for (FileStatus status : statusLst) {
409              String suffix = targetFs.stripOutRoot(status.getPath());
410              statusLst[i++] = new ViewFsFileStatus(status, this.makeQualified(
411                  suffix.length() == 0 ? f : new Path(res.resolvedPath, suffix)));
412          }
413        }
414        return statusLst;
415      }
416    
417      @Override
418      public void mkdir(final Path dir, final FsPermission permission,
419          final boolean createParent) throws AccessControlException,
420          FileAlreadyExistsException,
421          FileNotFoundException, UnresolvedLinkException, IOException {
422        InodeTree.ResolveResult<AbstractFileSystem> res = 
423          fsState.resolve(getUriPath(dir), false);
424        res.targetFileSystem.mkdir(res.remainingPath, permission, createParent);
425      }
426    
427      @Override
428      public FSDataInputStream open(final Path f, final int bufferSize)
429          throws AccessControlException, FileNotFoundException,
430          UnresolvedLinkException, IOException {
431        InodeTree.ResolveResult<AbstractFileSystem> res = 
432            fsState.resolve(getUriPath(f), true);
433        return res.targetFileSystem.open(res.remainingPath, bufferSize);
434      }
435    
436      
437      @Override
438      public void renameInternal(final Path src, final Path dst,
439          final boolean overwrite) throws IOException, UnresolvedLinkException {
440        // passing resolveLastComponet as false to catch renaming a mount point 
441        // itself we need to catch this as an internal operation and fail.
442        InodeTree.ResolveResult<AbstractFileSystem> resSrc = 
443          fsState.resolve(getUriPath(src), false); 
444      
445        if (resSrc.isInternalDir()) {
446          throw new AccessControlException(
447              "Cannot Rename within internal dirs of mount table: it is readOnly");
448        }
449          
450        InodeTree.ResolveResult<AbstractFileSystem> resDst = 
451                                    fsState.resolve(getUriPath(dst), false);
452        if (resDst.isInternalDir()) {
453          throw new AccessControlException(
454              "Cannot Rename within internal dirs of mount table: it is readOnly");
455        }
456        
457        /**
458        // Alternate 1: renames within same file system - valid but we disallow
459        // Alternate 2: (as described in next para - valid but we have disallowed it
460        //
461        // Note we compare the URIs. the URIs include the link targets. 
462        // hence we allow renames across mount links as long as the mount links
463        // point to the same target.
464        if (!resSrc.targetFileSystem.getUri().equals(
465                  resDst.targetFileSystem.getUri())) {
466          throw new IOException("Renames across Mount points not supported");
467        }
468        */
469        
470        //
471        // Alternate 3 : renames ONLY within the the same mount links.
472        //
473    
474        if (resSrc.targetFileSystem !=resDst.targetFileSystem) {
475          throw new IOException("Renames across Mount points not supported");
476        }
477        
478        resSrc.targetFileSystem.renameInternal(resSrc.remainingPath,
479          resDst.remainingPath, overwrite);
480      }
481    
482      @Override
483      public void renameInternal(final Path src, final Path dst)
484          throws AccessControlException, FileAlreadyExistsException,
485          FileNotFoundException, ParentNotDirectoryException,
486          UnresolvedLinkException, IOException {
487        renameInternal(src, dst, false);
488      }
489      
490      @Override
491      public boolean supportsSymlinks() {
492        return true;
493      }
494      
495      @Override
496      public void createSymlink(final Path target, final Path link,
497          final boolean createParent) throws IOException, UnresolvedLinkException {
498        InodeTree.ResolveResult<AbstractFileSystem> res;
499        try {
500          res = fsState.resolve(getUriPath(link), false);
501        } catch (FileNotFoundException e) {
502          if (createParent) {
503            throw readOnlyMountTable("createSymlink", link);
504          } else {
505            throw e;
506          }
507        }
508        assert(res.remainingPath != null);
509        res.targetFileSystem.createSymlink(target, res.remainingPath,
510            createParent);  
511      }
512    
513      @Override
514      public Path getLinkTarget(final Path f) throws IOException {
515        InodeTree.ResolveResult<AbstractFileSystem> res = 
516          fsState.resolve(getUriPath(f), false); // do not follow mount link
517        return res.targetFileSystem.getLinkTarget(res.remainingPath);
518      }
519    
520      @Override
521      public void setOwner(final Path f, final String username,
522          final String groupname) throws AccessControlException,
523          FileNotFoundException, UnresolvedLinkException, IOException {
524        InodeTree.ResolveResult<AbstractFileSystem> res = 
525          fsState.resolve(getUriPath(f), true);
526        res.targetFileSystem.setOwner(res.remainingPath, username, groupname); 
527      }
528    
529      @Override
530      public void setPermission(final Path f, final FsPermission permission)
531          throws AccessControlException, FileNotFoundException,
532          UnresolvedLinkException, IOException {
533        InodeTree.ResolveResult<AbstractFileSystem> res = 
534          fsState.resolve(getUriPath(f), true);
535        res.targetFileSystem.setPermission(res.remainingPath, permission); 
536        
537      }
538    
539      @Override
540      public boolean setReplication(final Path f, final short replication)
541          throws AccessControlException, FileNotFoundException,
542          UnresolvedLinkException, IOException {
543        InodeTree.ResolveResult<AbstractFileSystem> res = 
544          fsState.resolve(getUriPath(f), true);
545        return res.targetFileSystem.setReplication(res.remainingPath, replication);
546      }
547    
548      @Override
549      public void setTimes(final Path f, final long mtime, final long atime)
550          throws AccessControlException, FileNotFoundException,
551          UnresolvedLinkException, IOException {
552        InodeTree.ResolveResult<AbstractFileSystem> res = 
553          fsState.resolve(getUriPath(f), true);
554        res.targetFileSystem.setTimes(res.remainingPath, mtime, atime); 
555      }
556    
557      @Override
558      public void setVerifyChecksum(final boolean verifyChecksum)
559          throws AccessControlException, IOException {
560        // This is a file system level operations, however ViewFs 
561        // points to many file systems. Noop for ViewFs. 
562      }
563      
564      public MountPoint[] getMountPoints() {
565        List<InodeTree.MountPoint<AbstractFileSystem>> mountPoints = 
566                      fsState.getMountPoints();
567        
568        MountPoint[] result = new MountPoint[mountPoints.size()];
569        for ( int i = 0; i < mountPoints.size(); ++i ) {
570          result[i] = new MountPoint(new Path(mountPoints.get(i).src), 
571                                  mountPoints.get(i).target.targetDirLinkList);
572        }
573        return result;
574      }
575      
576      @Override
577      public List<Token<?>> getDelegationTokens(String renewer) throws IOException {
578        List<InodeTree.MountPoint<AbstractFileSystem>> mountPoints = 
579                    fsState.getMountPoints();
580        int initialListSize  = 0;
581        for (InodeTree.MountPoint<AbstractFileSystem> im : mountPoints) {
582          initialListSize += im.target.targetDirLinkList.length; 
583        }
584        List<Token<?>> result = new ArrayList<Token<?>>(initialListSize);
585        for ( int i = 0; i < mountPoints.size(); ++i ) {
586          List<Token<?>> tokens = 
587            mountPoints.get(i).target.targetFileSystem.getDelegationTokens(renewer);
588          if (tokens != null) {
589            result.addAll(tokens);
590          }
591        }
592        return result;
593      }
594    
595      
596      
597      /*
598       * An instance of this class represents an internal dir of the viewFs 
599       * ie internal dir of the mount table.
600       * It is a ready only mount tbale and create, mkdir or delete operations
601       * are not allowed.
602       * If called on create or mkdir then this target is the parent of the
603       * directory in which one is trying to create or mkdir; hence
604       * in this case the path name passed in is the last component. 
605       * Otherwise this target is the end point of the path and hence
606       * the path name passed in is null. 
607       */
608      static class InternalDirOfViewFs extends AbstractFileSystem {
609        
610        final InodeTree.INodeDir<AbstractFileSystem>  theInternalDir;
611        final long creationTime; // of the the mount table
612        final UserGroupInformation ugi; // the user/group of user who created mtable
613        final URI myUri; // the URI of the outer ViewFs
614        
615        public InternalDirOfViewFs(final InodeTree.INodeDir<AbstractFileSystem> dir,
616            final long cTime, final UserGroupInformation ugi, final URI uri)
617          throws URISyntaxException {
618          super(FsConstants.VIEWFS_URI, FsConstants.VIEWFS_SCHEME, false, -1);
619          theInternalDir = dir;
620          creationTime = cTime;
621          this.ugi = ugi;
622          myUri = uri;
623        }
624    
625        static private void checkPathIsSlash(final Path f) throws IOException {
626          if (f != InodeTree.SlashPath) {
627            throw new IOException (
628            "Internal implementation error: expected file name to be /" );
629          }
630        }
631    
632        @Override
633        public FSDataOutputStream createInternal(final Path f,
634            final EnumSet<CreateFlag> flag, final FsPermission absolutePermission,
635            final int bufferSize, final short replication, final long blockSize,
636            final Progressable progress, final ChecksumOpt checksumOpt,
637            final boolean createParent) throws AccessControlException,
638            FileAlreadyExistsException, FileNotFoundException,
639            ParentNotDirectoryException, UnsupportedFileSystemException,
640            UnresolvedLinkException, IOException {
641          throw readOnlyMountTable("create", f);
642        }
643    
644        @Override
645        public boolean delete(final Path f, final boolean recursive)
646            throws AccessControlException, IOException {
647          checkPathIsSlash(f);
648          throw readOnlyMountTable("delete", f);
649        }
650    
651        @Override
652        public BlockLocation[] getFileBlockLocations(final Path f, final long start,
653            final long len) throws FileNotFoundException, IOException {
654          checkPathIsSlash(f);
655          throw new FileNotFoundException("Path points to dir not a file");
656        }
657    
658        @Override
659        public FileChecksum getFileChecksum(final Path f)
660            throws FileNotFoundException, IOException {
661          checkPathIsSlash(f);
662          throw new FileNotFoundException("Path points to dir not a file");
663        }
664    
665        @Override
666        public FileStatus getFileStatus(final Path f) throws IOException {
667          checkPathIsSlash(f);
668          return new FileStatus(0, true, 0, 0, creationTime, creationTime, 
669              PERMISSION_RRR, ugi.getUserName(), ugi.getGroupNames()[0],
670              new Path(theInternalDir.fullPath).makeQualified(
671                  myUri, null));
672        }
673        
674        @Override
675        public FileStatus getFileLinkStatus(final Path f)
676            throws FileNotFoundException {
677          // look up i internalDirs children - ignore first Slash
678          INode<AbstractFileSystem> inode =
679            theInternalDir.children.get(f.toUri().toString().substring(1)); 
680          if (inode == null) {
681            throw new FileNotFoundException(
682                "viewFs internal mount table - missing entry:" + f);
683          }
684          FileStatus result;
685          if (inode instanceof INodeLink) {
686            INodeLink<AbstractFileSystem> inodelink = 
687              (INodeLink<AbstractFileSystem>) inode;
688            result = new FileStatus(0, false, 0, 0, creationTime, creationTime,
689                PERMISSION_RRR, ugi.getUserName(), ugi.getGroupNames()[0],
690                inodelink.getTargetLink(),
691                new Path(inode.fullPath).makeQualified(
692                    myUri, null));
693          } else {
694            result = new FileStatus(0, true, 0, 0, creationTime, creationTime,
695              PERMISSION_RRR, ugi.getUserName(), ugi.getGroupNames()[0],
696              new Path(inode.fullPath).makeQualified(
697                  myUri, null));
698          }
699          return result;
700        }
701        
702        @Override
703        public FsStatus getFsStatus() {
704          return new FsStatus(0, 0, 0);
705        }
706    
707        @Override
708        public FsServerDefaults getServerDefaults() throws IOException {
709          throw new IOException("FsServerDefaults not implemented yet");
710        }
711    
712        @Override
713        public int getUriDefaultPort() {
714          return -1;
715        }
716    
717        @Override
718        public FileStatus[] listStatus(final Path f) throws AccessControlException,
719            IOException {
720          checkPathIsSlash(f);
721          FileStatus[] result = new FileStatus[theInternalDir.children.size()];
722          int i = 0;
723          for (Entry<String, INode<AbstractFileSystem>> iEntry : 
724                                              theInternalDir.children.entrySet()) {
725            INode<AbstractFileSystem> inode = iEntry.getValue();
726    
727            
728            if (inode instanceof INodeLink ) {
729              INodeLink<AbstractFileSystem> link = 
730                (INodeLink<AbstractFileSystem>) inode;
731    
732              result[i++] = new FileStatus(0, false, 0, 0,
733                creationTime, creationTime,
734                PERMISSION_RRR, ugi.getUserName(), ugi.getGroupNames()[0],
735                link.getTargetLink(),
736                new Path(inode.fullPath).makeQualified(
737                    myUri, null));
738            } else {
739              result[i++] = new FileStatus(0, true, 0, 0,
740                creationTime, creationTime,
741                PERMISSION_RRR, ugi.getUserName(), ugi.getGroupNames()[0],
742                new Path(inode.fullPath).makeQualified(
743                    myUri, null));
744            }
745          }
746          return result;
747        }
748    
749        @Override
750        public void mkdir(final Path dir, final FsPermission permission,
751            final boolean createParent) throws AccessControlException,
752            FileAlreadyExistsException {
753          if (theInternalDir.isRoot & dir == null) {
754            throw new FileAlreadyExistsException("/ already exits");
755          }
756          throw readOnlyMountTable("mkdir", dir);
757        }
758    
759        @Override
760        public FSDataInputStream open(final Path f, final int bufferSize)
761            throws FileNotFoundException, IOException {
762          checkPathIsSlash(f);
763          throw new FileNotFoundException("Path points to dir not a file");
764        }
765    
766        @Override
767        public void renameInternal(final Path src, final Path dst)
768            throws AccessControlException, IOException {
769          checkPathIsSlash(src);
770          checkPathIsSlash(dst);
771          throw readOnlyMountTable("rename", src);     
772        }
773    
774        @Override
775        public boolean supportsSymlinks() {
776          return true;
777        }
778        
779        @Override
780        public void createSymlink(final Path target, final Path link,
781            final boolean createParent) throws AccessControlException {
782          throw readOnlyMountTable("createSymlink", link);    
783        }
784    
785        @Override
786        public Path getLinkTarget(final Path f) throws FileNotFoundException,
787            IOException {
788          return getFileLinkStatus(f).getSymlink();
789        }
790    
791        @Override
792        public void setOwner(final Path f, final String username,
793            final String groupname) throws AccessControlException, IOException {
794          checkPathIsSlash(f);
795          throw readOnlyMountTable("setOwner", f);
796        }
797    
798        @Override
799        public void setPermission(final Path f, final FsPermission permission)
800            throws AccessControlException, IOException {
801          checkPathIsSlash(f);
802          throw readOnlyMountTable("setPermission", f);    
803        }
804    
805        @Override
806        public boolean setReplication(final Path f, final short replication)
807            throws AccessControlException, IOException {
808          checkPathIsSlash(f);
809          throw readOnlyMountTable("setReplication", f);
810        }
811    
812        @Override
813        public void setTimes(final Path f, final long mtime, final long atime)
814            throws AccessControlException, IOException {
815          checkPathIsSlash(f);
816          throw readOnlyMountTable("setTimes", f);    
817        }
818    
819        @Override
820        public void setVerifyChecksum(final boolean verifyChecksum)
821            throws AccessControlException {
822          throw readOnlyMountTable("setVerifyChecksum", "");   
823        }
824      }
825    }