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