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