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