001    /**
002     * Licensed to the Apache Software Foundation (ASF) under one
003     * or more contributor license agreements.  See the NOTICE file
004     * distributed with this work for additional information
005     * regarding copyright ownership.  The ASF licenses this file
006     * to you under the Apache License, Version 2.0 (the
007     * "License"); you may not use this file except in compliance
008     * with the License.  You may obtain a copy of the License at
009     *
010     *     http://www.apache.org/licenses/LICENSE-2.0
011     *
012     * Unless required by applicable law or agreed to in writing, software
013     * distributed under the License is distributed on an "AS IS" BASIS,
014     * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015     * See the License for the specific language governing permissions and
016     * limitations under the License.
017     */
018    package org.apache.hadoop.fs.viewfs;
019    
020    import static org.apache.hadoop.fs.viewfs.Constants.PERMISSION_555;
021    
022    import java.io.FileNotFoundException;
023    import java.io.IOException;
024    import java.net.URI;
025    import java.net.URISyntaxException;
026    import java.util.Arrays;
027    import java.util.EnumSet;
028    import java.util.HashSet;
029    import java.util.List;
030    import java.util.Map;
031    import java.util.Set;
032    import java.util.Map.Entry;
033    
034    import org.apache.hadoop.classification.InterfaceAudience;
035    import org.apache.hadoop.classification.InterfaceStability;
036    import org.apache.hadoop.conf.Configuration;
037    import org.apache.hadoop.fs.BlockLocation;
038    import org.apache.hadoop.fs.ContentSummary;
039    import org.apache.hadoop.fs.CreateFlag;
040    import org.apache.hadoop.fs.FSDataInputStream;
041    import org.apache.hadoop.fs.FSDataOutputStream;
042    import org.apache.hadoop.fs.FileAlreadyExistsException;
043    import org.apache.hadoop.fs.FileChecksum;
044    import org.apache.hadoop.fs.FileStatus;
045    import org.apache.hadoop.fs.FileSystem;
046    import org.apache.hadoop.fs.FsConstants;
047    import org.apache.hadoop.fs.FsServerDefaults;
048    import org.apache.hadoop.fs.Path;
049    import org.apache.hadoop.fs.UnsupportedFileSystemException;
050    import org.apache.hadoop.fs.XAttrSetFlag;
051    import org.apache.hadoop.fs.permission.AclEntry;
052    import org.apache.hadoop.fs.permission.AclStatus;
053    import org.apache.hadoop.fs.permission.FsPermission;
054    import org.apache.hadoop.fs.viewfs.InodeTree.INode;
055    import org.apache.hadoop.fs.viewfs.InodeTree.INodeLink;
056    import org.apache.hadoop.security.AccessControlException;
057    import org.apache.hadoop.security.UserGroupInformation;
058    import org.apache.hadoop.util.Progressable;
059    import org.apache.hadoop.util.Time;
060    
061    /**
062     * ViewFileSystem (extends the FileSystem interface) implements a client-side
063     * mount table. Its spec and implementation is identical to {@link ViewFs}.
064     */
065    
066    @InterfaceAudience.Public
067    @InterfaceStability.Evolving /*Evolving for a release,to be changed to Stable */
068    public class ViewFileSystem extends FileSystem {
069    
070      private static final Path ROOT_PATH = new Path(Path.SEPARATOR);
071    
072      static AccessControlException readOnlyMountTable(final String operation,
073          final String p) {
074        return new AccessControlException( 
075            "InternalDir of ViewFileSystem is readonly; operation=" + operation + 
076            "Path=" + p);
077      }
078      static AccessControlException readOnlyMountTable(final String operation,
079          final Path p) {
080        return readOnlyMountTable(operation, p.toString());
081      }
082      
083      static public class MountPoint {
084        private Path src;       // the src of the mount
085        private URI[] targets; //  target of the mount; Multiple targets imply mergeMount
086        MountPoint(Path srcPath, URI[] targetURIs) {
087          src = srcPath;
088          targets = targetURIs;
089        }
090        Path getSrc() {
091          return src;
092        }
093        URI[] getTargets() {
094          return targets;
095        }
096      }
097      
098      final long creationTime; // of the the mount table
099      final UserGroupInformation ugi; // the user/group of user who created mtable
100      URI myUri;
101      private Path workingDir;
102      Configuration config;
103      InodeTree<FileSystem> fsState;  // the fs state; ie the mount table
104      Path homeDir = null;
105      
106      /**
107       * Make the path Absolute and get the path-part of a pathname.
108       * Checks that URI matches this file system 
109       * and that the path-part is a valid name.
110       * 
111       * @param p path
112       * @return path-part of the Path p
113       */
114      private String getUriPath(final Path p) {
115        checkPath(p);
116        String s = makeAbsolute(p).toUri().getPath();
117        return s;
118      }
119      
120      private Path makeAbsolute(final Path f) {
121        return f.isAbsolute() ? f : new Path(workingDir, f);
122      }
123      
124      /**
125       * This is the  constructor with the signature needed by
126       * {@link FileSystem#createFileSystem(URI, Configuration)}
127       * 
128       * After this constructor is called initialize() is called.
129       * @throws IOException 
130       */
131      public ViewFileSystem() throws IOException {
132        ugi = UserGroupInformation.getCurrentUser();
133        creationTime = Time.now();
134      }
135    
136      /**
137       * Return the protocol scheme for the FileSystem.
138       * <p/>
139       *
140       * @return <code>viewfs</code>
141       */
142      @Override
143      public String getScheme() {
144        return "viewfs";
145      }
146    
147      /**
148       * Called after a new FileSystem instance is constructed.
149       * @param theUri a uri whose authority section names the host, port, etc. for
150       *          this FileSystem
151       * @param conf the configuration
152       */
153      @Override
154      public void initialize(final URI theUri, final Configuration conf)
155          throws IOException {
156        super.initialize(theUri, conf);
157        setConf(conf);
158        config = conf;
159        // Now build  client side view (i.e. client side mount table) from config.
160        final String authority = theUri.getAuthority();
161        try {
162          myUri = new URI(FsConstants.VIEWFS_SCHEME, authority, "/", null, null);
163          fsState = new InodeTree<FileSystem>(conf, authority) {
164    
165            @Override
166            protected
167            FileSystem getTargetFileSystem(final URI uri)
168              throws URISyntaxException, IOException {
169                return new ChRootedFileSystem(uri, config);
170            }
171    
172            @Override
173            protected
174            FileSystem getTargetFileSystem(final INodeDir<FileSystem> dir)
175              throws URISyntaxException {
176              return new InternalDirOfViewFs(dir, creationTime, ugi, myUri);
177            }
178    
179            @Override
180            protected
181            FileSystem getTargetFileSystem(URI[] mergeFsURIList)
182                throws URISyntaxException, UnsupportedFileSystemException {
183              throw new UnsupportedFileSystemException("mergefs not implemented");
184              // return MergeFs.createMergeFs(mergeFsURIList, config);
185            }
186          };
187          workingDir = this.getHomeDirectory();
188        } catch (URISyntaxException e) {
189          throw new IOException("URISyntax exception: " + theUri);
190        }
191    
192      }
193      
194      
195      /**
196       * Convenience Constructor for apps to call directly
197       * @param theUri which must be that of ViewFileSystem
198       * @param conf
199       * @throws IOException
200       */
201      ViewFileSystem(final URI theUri, final Configuration conf)
202        throws IOException {
203        this();
204        initialize(theUri, conf);
205      }
206      
207      /**
208       * Convenience Constructor for apps to call directly
209       * @param conf
210       * @throws IOException
211       */
212      public ViewFileSystem(final Configuration conf) throws IOException {
213        this(FsConstants.VIEWFS_URI, conf);
214      }
215      
216      public Path getTrashCanLocation(final Path f) throws FileNotFoundException {
217        final InodeTree.ResolveResult<FileSystem> res = 
218          fsState.resolve(getUriPath(f), true);
219        return res.isInternalDir() ? null : res.targetFileSystem.getHomeDirectory();
220      }
221      
222      @Override
223      public URI getUri() {
224        return myUri;
225      }
226      
227      @Override
228      public Path resolvePath(final Path f)
229          throws IOException {
230        final InodeTree.ResolveResult<FileSystem> res;
231          res = fsState.resolve(getUriPath(f), true);
232        if (res.isInternalDir()) {
233          return f;
234        }
235        return res.targetFileSystem.resolvePath(res.remainingPath);
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 = (base.equals("/") ? 
246              this.makeQualified(new Path(base + ugi.getShortUserName())):
247              this.makeQualified(new Path(base + "/" + ugi.getShortUserName())));
248        }
249        return homeDir;
250      }
251      
252      @Override
253      public Path getWorkingDirectory() {
254        return workingDir;
255      }
256    
257      @Override
258      public void setWorkingDirectory(final Path new_dir) {
259        getUriPath(new_dir); // this validates the path
260        workingDir = makeAbsolute(new_dir);
261      }
262      
263      @Override
264      public FSDataOutputStream append(final Path f, final int bufferSize,
265          final Progressable progress) throws IOException {
266        InodeTree.ResolveResult<FileSystem> res = 
267          fsState.resolve(getUriPath(f), true);
268        return res.targetFileSystem.append(res.remainingPath, bufferSize, progress);
269      }
270      
271      @Override
272      public FSDataOutputStream createNonRecursive(Path f, FsPermission permission,
273          EnumSet<CreateFlag> flags, int bufferSize, short replication, long blockSize,
274          Progressable progress) throws IOException {
275        InodeTree.ResolveResult<FileSystem> res;
276        try {
277          res = fsState.resolve(getUriPath(f), false);
278        } catch (FileNotFoundException e) {
279            throw readOnlyMountTable("create", f);
280        }
281        assert(res.remainingPath != null);
282        return res.targetFileSystem.createNonRecursive(res.remainingPath, permission,
283             flags, bufferSize, replication, blockSize, progress);
284      }
285      
286      @Override
287      public FSDataOutputStream create(final Path f, final FsPermission permission,
288          final boolean overwrite, final int bufferSize, final short replication,
289          final long blockSize, final Progressable progress) throws IOException {
290        InodeTree.ResolveResult<FileSystem> res;
291        try {
292          res = fsState.resolve(getUriPath(f), false);
293        } catch (FileNotFoundException e) {
294            throw readOnlyMountTable("create", f);
295        }
296        assert(res.remainingPath != null);
297        return res.targetFileSystem.create(res.remainingPath, permission,
298             overwrite, bufferSize, replication, blockSize, progress);
299      }
300    
301      
302      @Override
303      public boolean delete(final Path f, final boolean recursive)
304          throws AccessControlException, FileNotFoundException,
305          IOException {
306        InodeTree.ResolveResult<FileSystem> res = 
307          fsState.resolve(getUriPath(f), true);
308        // If internal dir or target is a mount link (ie remainingPath is Slash)
309        if (res.isInternalDir() || res.remainingPath == InodeTree.SlashPath) {
310          throw readOnlyMountTable("delete", f);
311        }
312        return res.targetFileSystem.delete(res.remainingPath, recursive);
313      }
314      
315      @Override
316      @SuppressWarnings("deprecation")
317      public boolean delete(final Path f)
318          throws AccessControlException, FileNotFoundException,
319          IOException {
320          return delete(f, true);
321      }
322      
323      @Override
324      public BlockLocation[] getFileBlockLocations(FileStatus fs, 
325          long start, long len) throws IOException {
326        final InodeTree.ResolveResult<FileSystem> res = 
327          fsState.resolve(getUriPath(fs.getPath()), true);
328        return res.targetFileSystem.getFileBlockLocations(
329              new ViewFsFileStatus(fs, res.remainingPath), start, len);
330      }
331    
332      @Override
333      public FileChecksum getFileChecksum(final Path f)
334          throws AccessControlException, FileNotFoundException,
335          IOException {
336        InodeTree.ResolveResult<FileSystem> res = 
337          fsState.resolve(getUriPath(f), true);
338        return res.targetFileSystem.getFileChecksum(res.remainingPath);
339      }
340    
341      @Override
342      public FileStatus getFileStatus(final Path f) throws AccessControlException,
343          FileNotFoundException, IOException {
344        InodeTree.ResolveResult<FileSystem> res = 
345          fsState.resolve(getUriPath(f), true);
346        
347        // FileStatus#getPath is a fully qualified path relative to the root of 
348        // target file system.
349        // We need to change it to viewfs URI - relative to root of mount table.
350        
351        // The implementors of RawLocalFileSystem were trying to be very smart.
352        // They implement FileStatus#getOwener lazily -- the object
353        // returned is really a RawLocalFileSystem that expect the
354        // FileStatus#getPath to be unchanged so that it can get owner when needed.
355        // Hence we need to interpose a new ViewFileSystemFileStatus that 
356        // works around.
357        FileStatus status =  res.targetFileSystem.getFileStatus(res.remainingPath);
358        return new ViewFsFileStatus(status, this.makeQualified(f));
359      }
360      
361      
362      @Override
363      public FileStatus[] listStatus(final Path f) throws AccessControlException,
364          FileNotFoundException, IOException {
365        InodeTree.ResolveResult<FileSystem> res =
366          fsState.resolve(getUriPath(f), true);
367        
368        FileStatus[] statusLst = res.targetFileSystem.listStatus(res.remainingPath);
369        if (!res.isInternalDir()) {
370          // We need to change the name in the FileStatus as described in
371          // {@link #getFileStatus }
372          ChRootedFileSystem targetFs;
373          targetFs = (ChRootedFileSystem) res.targetFileSystem;
374          int i = 0;
375          for (FileStatus status : statusLst) {
376              String suffix = targetFs.stripOutRoot(status.getPath());
377              statusLst[i++] = new ViewFsFileStatus(status, this.makeQualified(
378                  suffix.length() == 0 ? f : new Path(res.resolvedPath, suffix)));
379          }
380        }
381        return statusLst;
382      }
383    
384      @Override
385      public boolean mkdirs(final Path dir, final FsPermission permission)
386          throws IOException {
387        InodeTree.ResolveResult<FileSystem> res = 
388          fsState.resolve(getUriPath(dir), false);
389       return  res.targetFileSystem.mkdirs(res.remainingPath, permission);
390      }
391    
392      @Override
393      public FSDataInputStream open(final Path f, final int bufferSize)
394          throws AccessControlException, FileNotFoundException,
395          IOException {
396        InodeTree.ResolveResult<FileSystem> res = 
397            fsState.resolve(getUriPath(f), true);
398        return res.targetFileSystem.open(res.remainingPath, bufferSize);
399      }
400    
401      
402      @Override
403      public boolean rename(final Path src, final Path dst) throws IOException {
404        // passing resolveLastComponet as false to catch renaming a mount point to 
405        // itself. We need to catch this as an internal operation and fail.
406        InodeTree.ResolveResult<FileSystem> resSrc = 
407          fsState.resolve(getUriPath(src), false); 
408      
409        if (resSrc.isInternalDir()) {
410          throw readOnlyMountTable("rename", src);
411        }
412          
413        InodeTree.ResolveResult<FileSystem> resDst = 
414          fsState.resolve(getUriPath(dst), false);
415        if (resDst.isInternalDir()) {
416              throw readOnlyMountTable("rename", dst);
417        }
418        /**
419        // Alternate 1: renames within same file system - valid but we disallow
420        // Alternate 2: (as described in next para - valid but we have disallowed it
421        //
422        // Note we compare the URIs. the URIs include the link targets. 
423        // hence we allow renames across mount links as long as the mount links
424        // point to the same target.
425        if (!resSrc.targetFileSystem.getUri().equals(
426                  resDst.targetFileSystem.getUri())) {
427          throw new IOException("Renames across Mount points not supported");
428        }
429        */
430        
431        //
432        // Alternate 3 : renames ONLY within the the same mount links.
433        //
434        if (resSrc.targetFileSystem !=resDst.targetFileSystem) {
435          throw new IOException("Renames across Mount points not supported");
436        }
437        return resSrc.targetFileSystem.rename(resSrc.remainingPath,
438            resDst.remainingPath);
439      }
440      
441      @Override
442      public void setOwner(final Path f, final String username,
443          final String groupname) throws AccessControlException,
444          FileNotFoundException,
445          IOException {
446        InodeTree.ResolveResult<FileSystem> res = 
447          fsState.resolve(getUriPath(f), true);
448        res.targetFileSystem.setOwner(res.remainingPath, username, groupname); 
449      }
450    
451      @Override
452      public void setPermission(final Path f, final FsPermission permission)
453          throws AccessControlException, FileNotFoundException,
454          IOException {
455        InodeTree.ResolveResult<FileSystem> res = 
456          fsState.resolve(getUriPath(f), true);
457        res.targetFileSystem.setPermission(res.remainingPath, permission); 
458      }
459    
460      @Override
461      public boolean setReplication(final Path f, final short replication)
462          throws AccessControlException, FileNotFoundException,
463          IOException {
464        InodeTree.ResolveResult<FileSystem> res = 
465          fsState.resolve(getUriPath(f), true);
466        return res.targetFileSystem.setReplication(res.remainingPath, replication);
467      }
468    
469      @Override
470      public void setTimes(final Path f, final long mtime, final long atime)
471          throws AccessControlException, FileNotFoundException,
472          IOException {
473        InodeTree.ResolveResult<FileSystem> res = 
474          fsState.resolve(getUriPath(f), true);
475        res.targetFileSystem.setTimes(res.remainingPath, mtime, atime); 
476      }
477    
478      @Override
479      public void modifyAclEntries(Path path, List<AclEntry> aclSpec)
480          throws IOException {
481        InodeTree.ResolveResult<FileSystem> res = fsState.resolve(getUriPath(path),
482            true);
483        res.targetFileSystem.modifyAclEntries(res.remainingPath, aclSpec);
484      }
485    
486      @Override
487      public void removeAclEntries(Path path, List<AclEntry> aclSpec)
488          throws IOException {
489        InodeTree.ResolveResult<FileSystem> res = fsState.resolve(getUriPath(path),
490            true);
491        res.targetFileSystem.removeAclEntries(res.remainingPath, aclSpec);
492      }
493    
494      @Override
495      public void removeDefaultAcl(Path path)
496          throws IOException {
497        InodeTree.ResolveResult<FileSystem> res =
498          fsState.resolve(getUriPath(path), true);
499        res.targetFileSystem.removeDefaultAcl(res.remainingPath);
500      }
501    
502      @Override
503      public void removeAcl(Path path)
504          throws IOException {
505        InodeTree.ResolveResult<FileSystem> res =
506          fsState.resolve(getUriPath(path), true);
507        res.targetFileSystem.removeAcl(res.remainingPath);
508      }
509    
510      @Override
511      public void setAcl(Path path, List<AclEntry> aclSpec) throws IOException {
512        InodeTree.ResolveResult<FileSystem> res =
513          fsState.resolve(getUriPath(path), true);
514        res.targetFileSystem.setAcl(res.remainingPath, aclSpec);
515      }
516    
517      @Override
518      public AclStatus getAclStatus(Path path) throws IOException {
519        InodeTree.ResolveResult<FileSystem> res =
520          fsState.resolve(getUriPath(path), true);
521        return res.targetFileSystem.getAclStatus(res.remainingPath);
522      }
523    
524      @Override
525      public void setXAttr(Path path, String name, byte[] value,
526          EnumSet<XAttrSetFlag> flag) throws IOException {
527        InodeTree.ResolveResult<FileSystem> res =
528            fsState.resolve(getUriPath(path), true);
529        res.targetFileSystem.setXAttr(res.remainingPath, name, value, flag);
530      }
531    
532      @Override
533      public byte[] getXAttr(Path path, String name) throws IOException {
534        InodeTree.ResolveResult<FileSystem> res =
535            fsState.resolve(getUriPath(path), true);
536        return res.targetFileSystem.getXAttr(res.remainingPath, name);
537      }
538    
539      @Override
540      public Map<String, byte[]> getXAttrs(Path path) throws IOException {
541        InodeTree.ResolveResult<FileSystem> res =
542            fsState.resolve(getUriPath(path), true);
543        return res.targetFileSystem.getXAttrs(res.remainingPath);
544      }
545    
546      @Override
547      public Map<String, byte[]> getXAttrs(Path path, List<String> names)
548          throws IOException {
549        InodeTree.ResolveResult<FileSystem> res =
550            fsState.resolve(getUriPath(path), true);
551        return res.targetFileSystem.getXAttrs(res.remainingPath, names);
552      }
553    
554      @Override
555      public List<String> listXAttrs(Path path) throws IOException {
556        InodeTree.ResolveResult<FileSystem> res =
557          fsState.resolve(getUriPath(path), true);
558        return res.targetFileSystem.listXAttrs(res.remainingPath);
559      }
560    
561      @Override
562      public void removeXAttr(Path path, String name) throws IOException {
563        InodeTree.ResolveResult<FileSystem> res = fsState.resolve(getUriPath(path),
564            true);
565        res.targetFileSystem.removeXAttr(res.remainingPath, name);
566      }
567    
568      @Override
569      public void setVerifyChecksum(final boolean verifyChecksum) { 
570        List<InodeTree.MountPoint<FileSystem>> mountPoints = 
571            fsState.getMountPoints();
572        for (InodeTree.MountPoint<FileSystem> mount : mountPoints) {
573          mount.target.targetFileSystem.setVerifyChecksum(verifyChecksum);
574        }
575      }
576      
577      @Override
578      public long getDefaultBlockSize() {
579        throw new NotInMountpointException("getDefaultBlockSize");
580      }
581    
582      @Override
583      public short getDefaultReplication() {
584        throw new NotInMountpointException("getDefaultReplication");
585      }
586    
587      @Override
588      public FsServerDefaults getServerDefaults() throws IOException {
589        throw new NotInMountpointException("getServerDefaults");
590      }
591    
592      @Override
593      public long getDefaultBlockSize(Path f) {
594        try {
595          InodeTree.ResolveResult<FileSystem> res =
596            fsState.resolve(getUriPath(f), true);
597          return res.targetFileSystem.getDefaultBlockSize(res.remainingPath);
598        } catch (FileNotFoundException e) {
599          throw new NotInMountpointException(f, "getDefaultBlockSize"); 
600        }
601      }
602    
603      @Override
604      public short getDefaultReplication(Path f) {
605        try {
606          InodeTree.ResolveResult<FileSystem> res =
607            fsState.resolve(getUriPath(f), true);
608          return res.targetFileSystem.getDefaultReplication(res.remainingPath);
609        } catch (FileNotFoundException e) {
610          throw new NotInMountpointException(f, "getDefaultReplication"); 
611        }
612      }
613    
614      @Override
615      public FsServerDefaults getServerDefaults(Path f) throws IOException {
616        InodeTree.ResolveResult<FileSystem> res =
617          fsState.resolve(getUriPath(f), true);
618        return res.targetFileSystem.getServerDefaults(res.remainingPath);    
619      }
620    
621      @Override
622      public ContentSummary getContentSummary(Path f) throws IOException {
623        InodeTree.ResolveResult<FileSystem> res = 
624          fsState.resolve(getUriPath(f), true);
625        return res.targetFileSystem.getContentSummary(res.remainingPath);
626      }
627    
628      @Override
629      public void setWriteChecksum(final boolean writeChecksum) { 
630        List<InodeTree.MountPoint<FileSystem>> mountPoints = 
631            fsState.getMountPoints();
632        for (InodeTree.MountPoint<FileSystem> mount : mountPoints) {
633          mount.target.targetFileSystem.setWriteChecksum(writeChecksum);
634        }
635      }
636    
637      @Override
638      public FileSystem[] getChildFileSystems() {
639        List<InodeTree.MountPoint<FileSystem>> mountPoints =
640            fsState.getMountPoints();
641        Set<FileSystem> children = new HashSet<FileSystem>();
642        for (InodeTree.MountPoint<FileSystem> mountPoint : mountPoints) {
643          FileSystem targetFs = mountPoint.target.targetFileSystem;
644          children.addAll(Arrays.asList(targetFs.getChildFileSystems()));
645        }
646        return children.toArray(new FileSystem[]{});
647      }
648      
649      public MountPoint[] getMountPoints() {
650        List<InodeTree.MountPoint<FileSystem>> mountPoints = 
651                      fsState.getMountPoints();
652        
653        MountPoint[] result = new MountPoint[mountPoints.size()];
654        for ( int i = 0; i < mountPoints.size(); ++i ) {
655          result[i] = new MountPoint(new Path(mountPoints.get(i).src), 
656                                  mountPoints.get(i).target.targetDirLinkList);
657        }
658        return result;
659      }
660      
661      /*
662       * An instance of this class represents an internal dir of the viewFs 
663       * that is internal dir of the mount table.
664       * It is a read only mount tables and create, mkdir or delete operations
665       * are not allowed.
666       * If called on create or mkdir then this target is the parent of the
667       * directory in which one is trying to create or mkdir; hence
668       * in this case the path name passed in is the last component. 
669       * Otherwise this target is the end point of the path and hence
670       * the path name passed in is null. 
671       */
672      static class InternalDirOfViewFs extends FileSystem {
673        final InodeTree.INodeDir<FileSystem>  theInternalDir;
674        final long creationTime; // of the the mount table
675        final UserGroupInformation ugi; // the user/group of user who created mtable
676        final URI myUri;
677        
678        public InternalDirOfViewFs(final InodeTree.INodeDir<FileSystem> dir,
679            final long cTime, final UserGroupInformation ugi, URI uri)
680          throws URISyntaxException {
681          myUri = uri;
682          try {
683            initialize(myUri, new Configuration());
684          } catch (IOException e) {
685            throw new RuntimeException("Cannot occur");
686          }
687          theInternalDir = dir;
688          creationTime = cTime;
689          this.ugi = ugi;
690        }
691    
692        static private void checkPathIsSlash(final Path f) throws IOException {
693          if (f != InodeTree.SlashPath) {
694            throw new IOException (
695            "Internal implementation error: expected file name to be /" );
696          }
697        }
698        
699        @Override
700        public URI getUri() {
701          return myUri;
702        }
703    
704        @Override
705        public Path getWorkingDirectory() {
706          throw new RuntimeException (
707          "Internal impl error: getWorkingDir should not have been called" );
708        }
709    
710        @Override
711        public void setWorkingDirectory(final Path new_dir) {
712          throw new RuntimeException (
713          "Internal impl error: getWorkingDir should not have been called" ); 
714        }
715    
716        @Override
717        public FSDataOutputStream append(final Path f, final int bufferSize,
718            final Progressable progress) throws IOException {
719          throw readOnlyMountTable("append", f);
720        }
721    
722        @Override
723        public FSDataOutputStream create(final Path f,
724            final FsPermission permission, final boolean overwrite,
725            final int bufferSize, final short replication, final long blockSize,
726            final Progressable progress) throws AccessControlException {
727          throw readOnlyMountTable("create", f);
728        }
729    
730        @Override
731        public boolean delete(final Path f, final boolean recursive)
732            throws AccessControlException, IOException {
733          checkPathIsSlash(f);
734          throw readOnlyMountTable("delete", f);
735        }
736        
737        @Override
738        @SuppressWarnings("deprecation")
739        public boolean delete(final Path f)
740            throws AccessControlException, IOException {
741          return delete(f, true);
742        }
743    
744        @Override
745        public BlockLocation[] getFileBlockLocations(final FileStatus fs,
746            final long start, final long len) throws 
747            FileNotFoundException, IOException {
748          checkPathIsSlash(fs.getPath());
749          throw new FileNotFoundException("Path points to dir not a file");
750        }
751    
752        @Override
753        public FileChecksum getFileChecksum(final Path f)
754            throws FileNotFoundException, IOException {
755          checkPathIsSlash(f);
756          throw new FileNotFoundException("Path points to dir not a file");
757        }
758    
759        @Override
760        public FileStatus getFileStatus(Path f) throws IOException {
761          checkPathIsSlash(f);
762          return new FileStatus(0, true, 0, 0, creationTime, creationTime,
763              PERMISSION_555, ugi.getUserName(), ugi.getGroupNames()[0],
764    
765              new Path(theInternalDir.fullPath).makeQualified(
766                  myUri, ROOT_PATH));
767        }
768        
769    
770        @Override
771        public FileStatus[] listStatus(Path f) throws AccessControlException,
772            FileNotFoundException, IOException {
773          checkPathIsSlash(f);
774          FileStatus[] result = new FileStatus[theInternalDir.children.size()];
775          int i = 0;
776          for (Entry<String, INode<FileSystem>> iEntry : 
777                                              theInternalDir.children.entrySet()) {
778            INode<FileSystem> inode = iEntry.getValue();
779            if (inode instanceof INodeLink ) {
780              INodeLink<FileSystem> link = (INodeLink<FileSystem>) inode;
781    
782              result[i++] = new FileStatus(0, false, 0, 0,
783                creationTime, creationTime, PERMISSION_555,
784                ugi.getUserName(), ugi.getGroupNames()[0],
785                link.getTargetLink(),
786                new Path(inode.fullPath).makeQualified(
787                    myUri, null));
788            } else {
789              result[i++] = new FileStatus(0, true, 0, 0,
790                creationTime, creationTime, PERMISSION_555,
791                ugi.getUserName(), ugi.getGroupNames()[0],
792                new Path(inode.fullPath).makeQualified(
793                    myUri, null));
794            }
795          }
796          return result;
797        }
798    
799        @Override
800        public boolean mkdirs(Path dir, FsPermission permission)
801            throws AccessControlException, FileAlreadyExistsException {
802          if (theInternalDir.isRoot && dir == null) {
803            throw new FileAlreadyExistsException("/ already exits");
804          }
805          // Note dir starts with /
806          if (theInternalDir.children.containsKey(dir.toString().substring(1))) {
807            return true; // this is the stupid semantics of FileSystem
808          }
809          throw readOnlyMountTable("mkdirs",  dir);
810        }
811    
812        @Override
813        public FSDataInputStream open(Path f, int bufferSize)
814            throws AccessControlException, FileNotFoundException, IOException {
815          checkPathIsSlash(f);
816          throw new FileNotFoundException("Path points to dir not a file");
817        }
818    
819        @Override
820        public boolean rename(Path src, Path dst) throws AccessControlException,
821            IOException {
822          checkPathIsSlash(src);
823          checkPathIsSlash(dst);
824          throw readOnlyMountTable("rename", src);     
825        }
826    
827        @Override
828        public void setOwner(Path f, String username, String groupname)
829            throws AccessControlException, IOException {
830          checkPathIsSlash(f);
831          throw readOnlyMountTable("setOwner", f);
832        }
833    
834        @Override
835        public void setPermission(Path f, FsPermission permission)
836            throws AccessControlException, IOException {
837          checkPathIsSlash(f);
838          throw readOnlyMountTable("setPermission", f);    
839        }
840    
841        @Override
842        public boolean setReplication(Path f, short replication)
843            throws AccessControlException, IOException {
844          checkPathIsSlash(f);
845          throw readOnlyMountTable("setReplication", f);
846        }
847    
848        @Override
849        public void setTimes(Path f, long mtime, long atime)
850            throws AccessControlException, IOException {
851          checkPathIsSlash(f);
852          throw readOnlyMountTable("setTimes", f);    
853        }
854    
855        @Override
856        public void setVerifyChecksum(boolean verifyChecksum) {
857          // Noop for viewfs
858        }
859    
860        @Override
861        public FsServerDefaults getServerDefaults(Path f) throws IOException {
862          throw new NotInMountpointException(f, "getServerDefaults");
863        }
864        
865        @Override
866        public long getDefaultBlockSize(Path f) {
867          throw new NotInMountpointException(f, "getDefaultBlockSize");
868        }
869    
870        @Override
871        public short getDefaultReplication(Path f) {
872          throw new NotInMountpointException(f, "getDefaultReplication");
873        }
874      }
875    }