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