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