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