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