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  /**
753   * Retrieve the storage policy for a given file or directory.
754   *
755   * @param src file or directory path.
756   * @return storage policy for give file.
757   * @throws IOException
758   */
759  public BlockStoragePolicySpi getStoragePolicy(final Path src)
760      throws IOException {
761    InodeTree.ResolveResult<AbstractFileSystem> res =
762        fsState.resolve(getUriPath(src), true);
763    return res.targetFileSystem.getStoragePolicy(res.remainingPath);
764  }
765
766  /*
767   * An instance of this class represents an internal dir of the viewFs 
768   * ie internal dir of the mount table.
769   * It is a ready only mount tbale and create, mkdir or delete operations
770   * are not allowed.
771   * If called on create or mkdir then this target is the parent of the
772   * directory in which one is trying to create or mkdir; hence
773   * in this case the path name passed in is the last component. 
774   * Otherwise this target is the end point of the path and hence
775   * the path name passed in is null. 
776   */
777  static class InternalDirOfViewFs extends AbstractFileSystem {
778    
779    final InodeTree.INodeDir<AbstractFileSystem>  theInternalDir;
780    final long creationTime; // of the the mount table
781    final UserGroupInformation ugi; // the user/group of user who created mtable
782    final URI myUri; // the URI of the outer ViewFs
783    
784    public InternalDirOfViewFs(final InodeTree.INodeDir<AbstractFileSystem> dir,
785        final long cTime, final UserGroupInformation ugi, final URI uri)
786      throws URISyntaxException {
787      super(FsConstants.VIEWFS_URI, FsConstants.VIEWFS_SCHEME, false, -1);
788      theInternalDir = dir;
789      creationTime = cTime;
790      this.ugi = ugi;
791      myUri = uri;
792    }
793
794    static private void checkPathIsSlash(final Path f) throws IOException {
795      if (f != InodeTree.SlashPath) {
796        throw new IOException (
797        "Internal implementation error: expected file name to be /" );
798      }
799    }
800
801    @Override
802    public FSDataOutputStream createInternal(final Path f,
803        final EnumSet<CreateFlag> flag, final FsPermission absolutePermission,
804        final int bufferSize, final short replication, final long blockSize,
805        final Progressable progress, final ChecksumOpt checksumOpt,
806        final boolean createParent) throws AccessControlException,
807        FileAlreadyExistsException, FileNotFoundException,
808        ParentNotDirectoryException, UnsupportedFileSystemException,
809        UnresolvedLinkException, IOException {
810      throw readOnlyMountTable("create", f);
811    }
812
813    @Override
814    public boolean delete(final Path f, final boolean recursive)
815        throws AccessControlException, IOException {
816      checkPathIsSlash(f);
817      throw readOnlyMountTable("delete", f);
818    }
819
820    @Override
821    public BlockLocation[] getFileBlockLocations(final Path f, final long start,
822        final long len) throws FileNotFoundException, IOException {
823      checkPathIsSlash(f);
824      throw new FileNotFoundException("Path points to dir not a file");
825    }
826
827    @Override
828    public FileChecksum getFileChecksum(final Path f)
829        throws FileNotFoundException, IOException {
830      checkPathIsSlash(f);
831      throw new FileNotFoundException("Path points to dir not a file");
832    }
833
834    @Override
835    public FileStatus getFileStatus(final Path f) throws IOException {
836      checkPathIsSlash(f);
837      return new FileStatus(0, true, 0, 0, creationTime, creationTime,
838          PERMISSION_555, ugi.getUserName(), ugi.getGroupNames()[0],
839          new Path(theInternalDir.fullPath).makeQualified(
840              myUri, null));
841    }
842    
843    @Override
844    public FileStatus getFileLinkStatus(final Path f)
845        throws FileNotFoundException {
846      // look up i internalDirs children - ignore first Slash
847      INode<AbstractFileSystem> inode =
848        theInternalDir.children.get(f.toUri().toString().substring(1)); 
849      if (inode == null) {
850        throw new FileNotFoundException(
851            "viewFs internal mount table - missing entry:" + f);
852      }
853      FileStatus result;
854      if (inode instanceof INodeLink) {
855        INodeLink<AbstractFileSystem> inodelink = 
856          (INodeLink<AbstractFileSystem>) inode;
857        result = new FileStatus(0, false, 0, 0, creationTime, creationTime,
858            PERMISSION_555, ugi.getUserName(), ugi.getGroupNames()[0],
859            inodelink.getTargetLink(),
860            new Path(inode.fullPath).makeQualified(
861                myUri, null));
862      } else {
863        result = new FileStatus(0, true, 0, 0, creationTime, creationTime,
864          PERMISSION_555, ugi.getUserName(), ugi.getGroupNames()[0],
865          new Path(inode.fullPath).makeQualified(
866              myUri, null));
867      }
868      return result;
869    }
870    
871    @Override
872    public FsStatus getFsStatus() {
873      return new FsStatus(0, 0, 0);
874    }
875
876    @Override
877    public FsServerDefaults getServerDefaults() throws IOException {
878      throw new IOException("FsServerDefaults not implemented yet");
879    }
880
881    @Override
882    public int getUriDefaultPort() {
883      return -1;
884    }
885
886    @Override
887    public FileStatus[] listStatus(final Path f) throws AccessControlException,
888        IOException {
889      checkPathIsSlash(f);
890      FileStatus[] result = new FileStatus[theInternalDir.children.size()];
891      int i = 0;
892      for (Entry<String, INode<AbstractFileSystem>> iEntry : 
893                                          theInternalDir.children.entrySet()) {
894        INode<AbstractFileSystem> inode = iEntry.getValue();
895
896        
897        if (inode instanceof INodeLink ) {
898          INodeLink<AbstractFileSystem> link = 
899            (INodeLink<AbstractFileSystem>) inode;
900
901          result[i++] = new FileStatus(0, false, 0, 0,
902            creationTime, creationTime,
903            PERMISSION_555, ugi.getUserName(), ugi.getGroupNames()[0],
904            link.getTargetLink(),
905            new Path(inode.fullPath).makeQualified(
906                myUri, null));
907        } else {
908          result[i++] = new FileStatus(0, true, 0, 0,
909            creationTime, creationTime,
910            PERMISSION_555, ugi.getUserName(), ugi.getGroupNames()[0],
911            new Path(inode.fullPath).makeQualified(
912                myUri, null));
913        }
914      }
915      return result;
916    }
917
918    @Override
919    public void mkdir(final Path dir, final FsPermission permission,
920        final boolean createParent) throws AccessControlException,
921        FileAlreadyExistsException {
922      if (theInternalDir.isRoot && dir == null) {
923        throw new FileAlreadyExistsException("/ already exits");
924      }
925      throw readOnlyMountTable("mkdir", dir);
926    }
927
928    @Override
929    public FSDataInputStream open(final Path f, final int bufferSize)
930        throws FileNotFoundException, IOException {
931      checkPathIsSlash(f);
932      throw new FileNotFoundException("Path points to dir not a file");
933    }
934
935    @Override
936    public boolean truncate(final Path f, final long newLength)
937        throws FileNotFoundException, IOException {
938      checkPathIsSlash(f);
939      throw readOnlyMountTable("truncate", f);
940    }
941
942    @Override
943    public void renameInternal(final Path src, final Path dst)
944        throws AccessControlException, IOException {
945      checkPathIsSlash(src);
946      checkPathIsSlash(dst);
947      throw readOnlyMountTable("rename", src);     
948    }
949
950    @Override
951    public boolean supportsSymlinks() {
952      return true;
953    }
954    
955    @Override
956    public void createSymlink(final Path target, final Path link,
957        final boolean createParent) throws AccessControlException {
958      throw readOnlyMountTable("createSymlink", link);    
959    }
960
961    @Override
962    public Path getLinkTarget(final Path f) throws FileNotFoundException,
963        IOException {
964      return getFileLinkStatus(f).getSymlink();
965    }
966
967    @Override
968    public void setOwner(final Path f, final String username,
969        final String groupname) throws AccessControlException, IOException {
970      checkPathIsSlash(f);
971      throw readOnlyMountTable("setOwner", f);
972    }
973
974    @Override
975    public void setPermission(final Path f, final FsPermission permission)
976        throws AccessControlException, IOException {
977      checkPathIsSlash(f);
978      throw readOnlyMountTable("setPermission", f);    
979    }
980
981    @Override
982    public boolean setReplication(final Path f, final short replication)
983        throws AccessControlException, IOException {
984      checkPathIsSlash(f);
985      throw readOnlyMountTable("setReplication", f);
986    }
987
988    @Override
989    public void setTimes(final Path f, final long mtime, final long atime)
990        throws AccessControlException, IOException {
991      checkPathIsSlash(f);
992      throw readOnlyMountTable("setTimes", f);    
993    }
994
995    @Override
996    public void setVerifyChecksum(final boolean verifyChecksum)
997        throws AccessControlException {
998      throw readOnlyMountTable("setVerifyChecksum", "");   
999    }
1000
1001    @Override
1002    public void modifyAclEntries(Path path, List<AclEntry> aclSpec)
1003        throws IOException {
1004      checkPathIsSlash(path);
1005      throw readOnlyMountTable("modifyAclEntries", path);
1006    }
1007
1008    @Override
1009    public void removeAclEntries(Path path, List<AclEntry> aclSpec)
1010        throws IOException {
1011      checkPathIsSlash(path);
1012      throw readOnlyMountTable("removeAclEntries", path);
1013    }
1014
1015    @Override
1016    public void removeDefaultAcl(Path path) throws IOException {
1017      checkPathIsSlash(path);
1018      throw readOnlyMountTable("removeDefaultAcl", path);
1019    }
1020
1021    @Override
1022    public void removeAcl(Path path) throws IOException {
1023      checkPathIsSlash(path);
1024      throw readOnlyMountTable("removeAcl", path);
1025    }
1026
1027    @Override
1028    public void setAcl(Path path, List<AclEntry> aclSpec) throws IOException {
1029      checkPathIsSlash(path);
1030      throw readOnlyMountTable("setAcl", path);
1031    }
1032
1033    @Override
1034    public AclStatus getAclStatus(Path path) throws IOException {
1035      checkPathIsSlash(path);
1036      return new AclStatus.Builder().owner(ugi.getUserName())
1037          .group(ugi.getGroupNames()[0])
1038          .addEntries(AclUtil.getMinimalAcl(PERMISSION_555))
1039          .stickyBit(false).build();
1040    }
1041
1042    @Override
1043    public void setXAttr(Path path, String name, byte[] value,
1044                         EnumSet<XAttrSetFlag> flag) throws IOException {
1045      checkPathIsSlash(path);
1046      throw readOnlyMountTable("setXAttr", path);
1047    }
1048
1049    @Override
1050    public byte[] getXAttr(Path path, String name) throws IOException {
1051      throw new NotInMountpointException(path, "getXAttr");
1052    }
1053
1054    @Override
1055    public Map<String, byte[]> getXAttrs(Path path) throws IOException {
1056      throw new NotInMountpointException(path, "getXAttrs");
1057    }
1058
1059    @Override
1060    public Map<String, byte[]> getXAttrs(Path path, List<String> names)
1061        throws IOException {
1062      throw new NotInMountpointException(path, "getXAttrs");
1063    }
1064
1065    @Override
1066    public List<String> listXAttrs(Path path) throws IOException {
1067      throw new NotInMountpointException(path, "listXAttrs");
1068    }
1069
1070    @Override
1071    public void removeXAttr(Path path, String name) throws IOException {
1072      checkPathIsSlash(path);
1073      throw readOnlyMountTable("removeXAttr", path);
1074    }
1075
1076    @Override
1077    public Path createSnapshot(Path path, String snapshotName)
1078        throws IOException {
1079      checkPathIsSlash(path);
1080      throw readOnlyMountTable("createSnapshot", path);
1081    }
1082
1083    @Override
1084    public void renameSnapshot(Path path, String snapshotOldName,
1085        String snapshotNewName) throws IOException {
1086      checkPathIsSlash(path);
1087      throw readOnlyMountTable("renameSnapshot", path);
1088    }
1089
1090    @Override
1091    public void deleteSnapshot(Path path, String snapshotName)
1092        throws IOException {
1093      checkPathIsSlash(path);
1094      throw readOnlyMountTable("deleteSnapshot", path);
1095    }
1096
1097    @Override
1098    public void setStoragePolicy(Path path, String policyName)
1099        throws IOException {
1100      throw readOnlyMountTable("setStoragePolicy", path);
1101    }
1102  }
1103}