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