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 void setOwner(final Path f, final String username,
504      final String groupname) throws AccessControlException,
505      FileNotFoundException,
506      IOException {
507    InodeTree.ResolveResult<FileSystem> res = 
508      fsState.resolve(getUriPath(f), true);
509    res.targetFileSystem.setOwner(res.remainingPath, username, groupname); 
510  }
511
512  @Override
513  public void setPermission(final Path f, final FsPermission permission)
514      throws AccessControlException, FileNotFoundException,
515      IOException {
516    InodeTree.ResolveResult<FileSystem> res = 
517      fsState.resolve(getUriPath(f), true);
518    res.targetFileSystem.setPermission(res.remainingPath, permission); 
519  }
520
521  @Override
522  public boolean setReplication(final Path f, final short replication)
523      throws AccessControlException, FileNotFoundException,
524      IOException {
525    InodeTree.ResolveResult<FileSystem> res = 
526      fsState.resolve(getUriPath(f), true);
527    return res.targetFileSystem.setReplication(res.remainingPath, replication);
528  }
529
530  @Override
531  public void setTimes(final Path f, final long mtime, final long atime)
532      throws AccessControlException, FileNotFoundException,
533      IOException {
534    InodeTree.ResolveResult<FileSystem> res = 
535      fsState.resolve(getUriPath(f), true);
536    res.targetFileSystem.setTimes(res.remainingPath, mtime, atime); 
537  }
538
539  @Override
540  public void modifyAclEntries(Path path, List<AclEntry> aclSpec)
541      throws IOException {
542    InodeTree.ResolveResult<FileSystem> res = fsState.resolve(getUriPath(path),
543        true);
544    res.targetFileSystem.modifyAclEntries(res.remainingPath, aclSpec);
545  }
546
547  @Override
548  public void removeAclEntries(Path path, List<AclEntry> aclSpec)
549      throws IOException {
550    InodeTree.ResolveResult<FileSystem> res = fsState.resolve(getUriPath(path),
551        true);
552    res.targetFileSystem.removeAclEntries(res.remainingPath, aclSpec);
553  }
554
555  @Override
556  public void removeDefaultAcl(Path path)
557      throws IOException {
558    InodeTree.ResolveResult<FileSystem> res =
559      fsState.resolve(getUriPath(path), true);
560    res.targetFileSystem.removeDefaultAcl(res.remainingPath);
561  }
562
563  @Override
564  public void removeAcl(Path path)
565      throws IOException {
566    InodeTree.ResolveResult<FileSystem> res =
567      fsState.resolve(getUriPath(path), true);
568    res.targetFileSystem.removeAcl(res.remainingPath);
569  }
570
571  @Override
572  public void setAcl(Path path, List<AclEntry> aclSpec) throws IOException {
573    InodeTree.ResolveResult<FileSystem> res =
574      fsState.resolve(getUriPath(path), true);
575    res.targetFileSystem.setAcl(res.remainingPath, aclSpec);
576  }
577
578  @Override
579  public AclStatus getAclStatus(Path path) throws IOException {
580    InodeTree.ResolveResult<FileSystem> res =
581      fsState.resolve(getUriPath(path), true);
582    return res.targetFileSystem.getAclStatus(res.remainingPath);
583  }
584
585  @Override
586  public void setXAttr(Path path, String name, byte[] value,
587      EnumSet<XAttrSetFlag> flag) throws IOException {
588    InodeTree.ResolveResult<FileSystem> res =
589        fsState.resolve(getUriPath(path), true);
590    res.targetFileSystem.setXAttr(res.remainingPath, name, value, flag);
591  }
592
593  @Override
594  public byte[] getXAttr(Path path, String name) throws IOException {
595    InodeTree.ResolveResult<FileSystem> res =
596        fsState.resolve(getUriPath(path), true);
597    return res.targetFileSystem.getXAttr(res.remainingPath, name);
598  }
599
600  @Override
601  public Map<String, byte[]> getXAttrs(Path path) throws IOException {
602    InodeTree.ResolveResult<FileSystem> res =
603        fsState.resolve(getUriPath(path), true);
604    return res.targetFileSystem.getXAttrs(res.remainingPath);
605  }
606
607  @Override
608  public Map<String, byte[]> getXAttrs(Path path, List<String> names)
609      throws IOException {
610    InodeTree.ResolveResult<FileSystem> res =
611        fsState.resolve(getUriPath(path), true);
612    return res.targetFileSystem.getXAttrs(res.remainingPath, names);
613  }
614
615  @Override
616  public List<String> listXAttrs(Path path) throws IOException {
617    InodeTree.ResolveResult<FileSystem> res =
618      fsState.resolve(getUriPath(path), true);
619    return res.targetFileSystem.listXAttrs(res.remainingPath);
620  }
621
622  @Override
623  public void removeXAttr(Path path, String name) throws IOException {
624    InodeTree.ResolveResult<FileSystem> res = fsState.resolve(getUriPath(path),
625        true);
626    res.targetFileSystem.removeXAttr(res.remainingPath, name);
627  }
628
629  @Override
630  public void setVerifyChecksum(final boolean verifyChecksum) { 
631    List<InodeTree.MountPoint<FileSystem>> mountPoints = 
632        fsState.getMountPoints();
633    for (InodeTree.MountPoint<FileSystem> mount : mountPoints) {
634      mount.target.targetFileSystem.setVerifyChecksum(verifyChecksum);
635    }
636  }
637  
638  @Override
639  public long getDefaultBlockSize() {
640    throw new NotInMountpointException("getDefaultBlockSize");
641  }
642
643  @Override
644  public short getDefaultReplication() {
645    throw new NotInMountpointException("getDefaultReplication");
646  }
647
648  @Override
649  public FsServerDefaults getServerDefaults() throws IOException {
650    throw new NotInMountpointException("getServerDefaults");
651  }
652
653  @Override
654  public long getDefaultBlockSize(Path f) {
655    try {
656      InodeTree.ResolveResult<FileSystem> res =
657        fsState.resolve(getUriPath(f), true);
658      return res.targetFileSystem.getDefaultBlockSize(res.remainingPath);
659    } catch (FileNotFoundException e) {
660      throw new NotInMountpointException(f, "getDefaultBlockSize"); 
661    }
662  }
663
664  @Override
665  public short getDefaultReplication(Path f) {
666    try {
667      InodeTree.ResolveResult<FileSystem> res =
668        fsState.resolve(getUriPath(f), true);
669      return res.targetFileSystem.getDefaultReplication(res.remainingPath);
670    } catch (FileNotFoundException e) {
671      throw new NotInMountpointException(f, "getDefaultReplication"); 
672    }
673  }
674
675  @Override
676  public FsServerDefaults getServerDefaults(Path f) throws IOException {
677    InodeTree.ResolveResult<FileSystem> res =
678      fsState.resolve(getUriPath(f), true);
679    return res.targetFileSystem.getServerDefaults(res.remainingPath);    
680  }
681
682  @Override
683  public ContentSummary getContentSummary(Path f) throws IOException {
684    InodeTree.ResolveResult<FileSystem> res = 
685      fsState.resolve(getUriPath(f), true);
686    return res.targetFileSystem.getContentSummary(res.remainingPath);
687  }
688
689  @Override
690  public void setWriteChecksum(final boolean writeChecksum) { 
691    List<InodeTree.MountPoint<FileSystem>> mountPoints = 
692        fsState.getMountPoints();
693    for (InodeTree.MountPoint<FileSystem> mount : mountPoints) {
694      mount.target.targetFileSystem.setWriteChecksum(writeChecksum);
695    }
696  }
697
698  @Override
699  public FileSystem[] getChildFileSystems() {
700    List<InodeTree.MountPoint<FileSystem>> mountPoints =
701        fsState.getMountPoints();
702    Set<FileSystem> children = new HashSet<FileSystem>();
703    for (InodeTree.MountPoint<FileSystem> mountPoint : mountPoints) {
704      FileSystem targetFs = mountPoint.target.targetFileSystem;
705      children.addAll(Arrays.asList(targetFs.getChildFileSystems()));
706    }
707    return children.toArray(new FileSystem[]{});
708  }
709  
710  public MountPoint[] getMountPoints() {
711    List<InodeTree.MountPoint<FileSystem>> mountPoints = 
712                  fsState.getMountPoints();
713    
714    MountPoint[] result = new MountPoint[mountPoints.size()];
715    for ( int i = 0; i < mountPoints.size(); ++i ) {
716      result[i] = new MountPoint(new Path(mountPoints.get(i).src), 
717                              mountPoints.get(i).target.targetDirLinkList);
718    }
719    return result;
720  }
721  
722  /*
723   * An instance of this class represents an internal dir of the viewFs 
724   * that is internal dir of the mount table.
725   * It is a read only mount tables and create, mkdir or delete operations
726   * are not allowed.
727   * If called on create or mkdir then this target is the parent of the
728   * directory in which one is trying to create or mkdir; hence
729   * in this case the path name passed in is the last component. 
730   * Otherwise this target is the end point of the path and hence
731   * the path name passed in is null. 
732   */
733  static class InternalDirOfViewFs extends FileSystem {
734    final InodeTree.INodeDir<FileSystem>  theInternalDir;
735    final long creationTime; // of the the mount table
736    final UserGroupInformation ugi; // the user/group of user who created mtable
737    final URI myUri;
738    
739    public InternalDirOfViewFs(final InodeTree.INodeDir<FileSystem> dir,
740        final long cTime, final UserGroupInformation ugi, URI uri)
741      throws URISyntaxException {
742      myUri = uri;
743      try {
744        initialize(myUri, new Configuration());
745      } catch (IOException e) {
746        throw new RuntimeException("Cannot occur");
747      }
748      theInternalDir = dir;
749      creationTime = cTime;
750      this.ugi = ugi;
751    }
752
753    static private void checkPathIsSlash(final Path f) throws IOException {
754      if (f != InodeTree.SlashPath) {
755        throw new IOException (
756        "Internal implementation error: expected file name to be /" );
757      }
758    }
759    
760    @Override
761    public URI getUri() {
762      return myUri;
763    }
764
765    @Override
766    public Path getWorkingDirectory() {
767      throw new RuntimeException (
768      "Internal impl error: getWorkingDir should not have been called" );
769    }
770
771    @Override
772    public void setWorkingDirectory(final Path new_dir) {
773      throw new RuntimeException (
774      "Internal impl error: getWorkingDir should not have been called" ); 
775    }
776
777    @Override
778    public FSDataOutputStream append(final Path f, final int bufferSize,
779        final Progressable progress) throws IOException {
780      throw readOnlyMountTable("append", f);
781    }
782
783    @Override
784    public FSDataOutputStream create(final Path f,
785        final FsPermission permission, final boolean overwrite,
786        final int bufferSize, final short replication, final long blockSize,
787        final Progressable progress) throws AccessControlException {
788      throw readOnlyMountTable("create", f);
789    }
790
791    @Override
792    public boolean delete(final Path f, final boolean recursive)
793        throws AccessControlException, IOException {
794      checkPathIsSlash(f);
795      throw readOnlyMountTable("delete", f);
796    }
797    
798    @Override
799    @SuppressWarnings("deprecation")
800    public boolean delete(final Path f)
801        throws AccessControlException, IOException {
802      return delete(f, true);
803    }
804
805    @Override
806    public BlockLocation[] getFileBlockLocations(final FileStatus fs,
807        final long start, final long len) throws 
808        FileNotFoundException, IOException {
809      checkPathIsSlash(fs.getPath());
810      throw new FileNotFoundException("Path points to dir not a file");
811    }
812
813    @Override
814    public FileChecksum getFileChecksum(final Path f)
815        throws FileNotFoundException, IOException {
816      checkPathIsSlash(f);
817      throw new FileNotFoundException("Path points to dir not a file");
818    }
819
820    @Override
821    public FileStatus getFileStatus(Path f) throws IOException {
822      checkPathIsSlash(f);
823      return new FileStatus(0, true, 0, 0, creationTime, creationTime,
824          PERMISSION_555, ugi.getUserName(), ugi.getGroupNames()[0],
825
826          new Path(theInternalDir.fullPath).makeQualified(
827              myUri, ROOT_PATH));
828    }
829    
830
831    @Override
832    public FileStatus[] listStatus(Path f) throws AccessControlException,
833        FileNotFoundException, IOException {
834      checkPathIsSlash(f);
835      FileStatus[] result = new FileStatus[theInternalDir.children.size()];
836      int i = 0;
837      for (Entry<String, INode<FileSystem>> iEntry : 
838                                          theInternalDir.children.entrySet()) {
839        INode<FileSystem> inode = iEntry.getValue();
840        if (inode instanceof INodeLink ) {
841          INodeLink<FileSystem> link = (INodeLink<FileSystem>) inode;
842
843          result[i++] = new FileStatus(0, false, 0, 0,
844            creationTime, creationTime, PERMISSION_555,
845            ugi.getUserName(), ugi.getGroupNames()[0],
846            link.getTargetLink(),
847            new Path(inode.fullPath).makeQualified(
848                myUri, null));
849        } else {
850          result[i++] = new FileStatus(0, true, 0, 0,
851            creationTime, creationTime, PERMISSION_555,
852            ugi.getUserName(), ugi.getGroupNames()[0],
853            new Path(inode.fullPath).makeQualified(
854                myUri, null));
855        }
856      }
857      return result;
858    }
859
860    @Override
861    public boolean mkdirs(Path dir, FsPermission permission)
862        throws AccessControlException, FileAlreadyExistsException {
863      if (theInternalDir.isRoot && dir == null) {
864        throw new FileAlreadyExistsException("/ already exits");
865      }
866      // Note dir starts with /
867      if (theInternalDir.children.containsKey(dir.toString().substring(1))) {
868        return true; // this is the stupid semantics of FileSystem
869      }
870      throw readOnlyMountTable("mkdirs",  dir);
871    }
872
873    @Override
874    public FSDataInputStream open(Path f, int bufferSize)
875        throws AccessControlException, FileNotFoundException, IOException {
876      checkPathIsSlash(f);
877      throw new FileNotFoundException("Path points to dir not a file");
878    }
879
880    @Override
881    public boolean rename(Path src, Path dst) throws AccessControlException,
882        IOException {
883      checkPathIsSlash(src);
884      checkPathIsSlash(dst);
885      throw readOnlyMountTable("rename", src);     
886    }
887
888    @Override
889    public void setOwner(Path f, String username, String groupname)
890        throws AccessControlException, IOException {
891      checkPathIsSlash(f);
892      throw readOnlyMountTable("setOwner", f);
893    }
894
895    @Override
896    public void setPermission(Path f, FsPermission permission)
897        throws AccessControlException, IOException {
898      checkPathIsSlash(f);
899      throw readOnlyMountTable("setPermission", f);    
900    }
901
902    @Override
903    public boolean setReplication(Path f, short replication)
904        throws AccessControlException, IOException {
905      checkPathIsSlash(f);
906      throw readOnlyMountTable("setReplication", f);
907    }
908
909    @Override
910    public void setTimes(Path f, long mtime, long atime)
911        throws AccessControlException, IOException {
912      checkPathIsSlash(f);
913      throw readOnlyMountTable("setTimes", f);    
914    }
915
916    @Override
917    public void setVerifyChecksum(boolean verifyChecksum) {
918      // Noop for viewfs
919    }
920
921    @Override
922    public FsServerDefaults getServerDefaults(Path f) throws IOException {
923      throw new NotInMountpointException(f, "getServerDefaults");
924    }
925    
926    @Override
927    public long getDefaultBlockSize(Path f) {
928      throw new NotInMountpointException(f, "getDefaultBlockSize");
929    }
930
931    @Override
932    public short getDefaultReplication(Path f) {
933      throw new NotInMountpointException(f, "getDefaultReplication");
934    }
935
936    @Override
937    public void modifyAclEntries(Path path, List<AclEntry> aclSpec)
938        throws IOException {
939      checkPathIsSlash(path);
940      throw readOnlyMountTable("modifyAclEntries", path);
941    }
942
943    @Override
944    public void removeAclEntries(Path path, List<AclEntry> aclSpec)
945        throws IOException {
946      checkPathIsSlash(path);
947      throw readOnlyMountTable("removeAclEntries", path);
948    }
949
950    @Override
951    public void removeDefaultAcl(Path path) throws IOException {
952      checkPathIsSlash(path);
953      throw readOnlyMountTable("removeDefaultAcl", path);
954    }
955
956    @Override
957    public void removeAcl(Path path) throws IOException {
958      checkPathIsSlash(path);
959      throw readOnlyMountTable("removeAcl", path);
960    }
961
962    @Override
963    public void setAcl(Path path, List<AclEntry> aclSpec) throws IOException {
964      checkPathIsSlash(path);
965      throw readOnlyMountTable("setAcl", path);
966    }
967
968    @Override
969    public AclStatus getAclStatus(Path path) throws IOException {
970      checkPathIsSlash(path);
971      return new AclStatus.Builder().owner(ugi.getUserName())
972          .group(ugi.getGroupNames()[0])
973          .addEntries(AclUtil.getMinimalAcl(PERMISSION_555))
974          .stickyBit(false).build();
975    }
976
977    @Override
978    public void setXAttr(Path path, String name, byte[] value,
979                         EnumSet<XAttrSetFlag> flag) throws IOException {
980      checkPathIsSlash(path);
981      throw readOnlyMountTable("setXAttr", path);
982    }
983
984    @Override
985    public byte[] getXAttr(Path path, String name) throws IOException {
986      throw new NotInMountpointException(path, "getXAttr");
987    }
988
989    @Override
990    public Map<String, byte[]> getXAttrs(Path path) throws IOException {
991      throw new NotInMountpointException(path, "getXAttrs");
992    }
993
994    @Override
995    public Map<String, byte[]> getXAttrs(Path path, List<String> names)
996        throws IOException {
997      throw new NotInMountpointException(path, "getXAttrs");
998    }
999
1000    @Override
1001    public List<String> listXAttrs(Path path) throws IOException {
1002      throw new NotInMountpointException(path, "listXAttrs");
1003    }
1004
1005    @Override
1006    public void removeXAttr(Path path, String name) throws IOException {
1007      checkPathIsSlash(path);
1008      throw readOnlyMountTable("removeXAttr", path);
1009    }
1010  }
1011}