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