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