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