1 /**
2  * Licensed to the Apache Software Foundation (ASF) under one
3  * or more contributor license agreements.  See the NOTICE file
4  * distributed with this work for additional information
5  * regarding copyright ownership.  The ASF licenses this file
6  * to you under the Apache License, Version 2.0 (the
7  * "License"); you may not use this file except in compliance
8  * with the License.  You may obtain a copy of the License at
9  *
10  *     http://www.apache.org/licenses/LICENSE-2.0
11  *
12  * Unless required by applicable law or agreed to in writing, software
13  * distributed under the License is distributed on an "AS IS" BASIS,
14  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15  * See the License for the specific language governing permissions and
16  * limitations under the License.
17  */
18 
19 package org.apache.hadoop.fs;
20 
21 import java.io.*;
22 import java.net.URI;
23 import java.net.URISyntaxException;
24 import java.util.EnumSet;
25 import java.util.List;
26 import java.util.Map;
27 
28 import org.apache.hadoop.classification.InterfaceAudience;
29 import org.apache.hadoop.classification.InterfaceStability;
30 import org.apache.hadoop.conf.Configuration;
31 import org.apache.hadoop.fs.permission.AclEntry;
32 import org.apache.hadoop.fs.permission.AclStatus;
33 import org.apache.hadoop.fs.permission.FsAction;
34 import org.apache.hadoop.fs.permission.FsPermission;
35 import org.apache.hadoop.fs.Options.ChecksumOpt;
36 import org.apache.hadoop.security.AccessControlException;
37 import org.apache.hadoop.util.Progressable;
38 
39 /****************************************************************
40  * A <code>FilterFileSystem</code> contains
41  * some other file system, which it uses as
42  * its  basic file system, possibly transforming
43  * the data along the way or providing  additional
44  * functionality. The class <code>FilterFileSystem</code>
45  * itself simply overrides all  methods of
46  * <code>FileSystem</code> with versions that
47  * pass all requests to the contained  file
48  * system. Subclasses of <code>FilterFileSystem</code>
49  * may further override some of  these methods
50  * and may also provide additional methods
51  * and fields.
52  *
53  *****************************************************************/
54 @InterfaceAudience.Public
55 @InterfaceStability.Stable
56 public class FilterFileSystem extends FileSystem {
57 
58   protected FileSystem fs;
59   protected String swapScheme;
60 
61   /*
62    * so that extending classes can define it
63    */
FilterFileSystem()64   public FilterFileSystem() {
65   }
66 
FilterFileSystem(FileSystem fs)67   public FilterFileSystem(FileSystem fs) {
68     this.fs = fs;
69     this.statistics = fs.statistics;
70   }
71 
72   /**
73    * Get the raw file system
74    * @return FileSystem being filtered
75    */
getRawFileSystem()76   public FileSystem getRawFileSystem() {
77     return fs;
78   }
79 
80   /** Called after a new FileSystem instance is constructed.
81    * @param name a uri whose authority section names the host, port, etc.
82    *   for this FileSystem
83    * @param conf the configuration
84    */
85   @Override
initialize(URI name, Configuration conf)86   public void initialize(URI name, Configuration conf) throws IOException {
87     super.initialize(name, conf);
88     // this is less than ideal, but existing filesystems sometimes neglect
89     // to initialize the embedded filesystem
90     if (fs.getConf() == null) {
91       fs.initialize(name, conf);
92     }
93     String scheme = name.getScheme();
94     if (!scheme.equals(fs.getUri().getScheme())) {
95       swapScheme = scheme;
96     }
97   }
98 
99   /** Returns a URI whose scheme and authority identify this FileSystem.*/
100   @Override
getUri()101   public URI getUri() {
102     return fs.getUri();
103   }
104 
105 
106   @Override
getCanonicalUri()107   protected URI getCanonicalUri() {
108     return fs.getCanonicalUri();
109   }
110 
111   @Override
canonicalizeUri(URI uri)112   protected URI canonicalizeUri(URI uri) {
113     return fs.canonicalizeUri(uri);
114   }
115 
116   /** Make sure that a path specifies a FileSystem. */
117   @Override
makeQualified(Path path)118   public Path makeQualified(Path path) {
119     Path fqPath = fs.makeQualified(path);
120     // swap in our scheme if the filtered fs is using a different scheme
121     if (swapScheme != null) {
122       try {
123         // NOTE: should deal with authority, but too much other stuff is broken
124         fqPath = new Path(
125             new URI(swapScheme, fqPath.toUri().getSchemeSpecificPart(), null)
126         );
127       } catch (URISyntaxException e) {
128         throw new IllegalArgumentException(e);
129       }
130     }
131     return fqPath;
132   }
133 
134   ///////////////////////////////////////////////////////////////
135   // FileSystem
136   ///////////////////////////////////////////////////////////////
137 
138   /** Check that a Path belongs to this FileSystem. */
139   @Override
checkPath(Path path)140   protected void checkPath(Path path) {
141     fs.checkPath(path);
142   }
143 
144   @Override
getFileBlockLocations(FileStatus file, long start, long len)145   public BlockLocation[] getFileBlockLocations(FileStatus file, long start,
146     long len) throws IOException {
147       return fs.getFileBlockLocations(file, start, len);
148   }
149 
150   @Override
resolvePath(final Path p)151   public Path resolvePath(final Path p) throws IOException {
152     return fs.resolvePath(p);
153   }
154   /**
155    * Opens an FSDataInputStream at the indicated Path.
156    * @param f the file name to open
157    * @param bufferSize the size of the buffer to be used.
158    */
159   @Override
open(Path f, int bufferSize)160   public FSDataInputStream open(Path f, int bufferSize) throws IOException {
161     return fs.open(f, bufferSize);
162   }
163 
164   @Override
append(Path f, int bufferSize, Progressable progress)165   public FSDataOutputStream append(Path f, int bufferSize,
166       Progressable progress) throws IOException {
167     return fs.append(f, bufferSize, progress);
168   }
169 
170   @Override
concat(Path f, Path[] psrcs)171   public void concat(Path f, Path[] psrcs) throws IOException {
172     fs.concat(f, psrcs);
173   }
174 
175   @Override
create(Path f, FsPermission permission, boolean overwrite, int bufferSize, short replication, long blockSize, Progressable progress)176   public FSDataOutputStream create(Path f, FsPermission permission,
177       boolean overwrite, int bufferSize, short replication, long blockSize,
178       Progressable progress) throws IOException {
179     return fs.create(f, permission,
180         overwrite, bufferSize, replication, blockSize, progress);
181   }
182 
183   @Override
create(Path f, FsPermission permission, EnumSet<CreateFlag> flags, int bufferSize, short replication, long blockSize, Progressable progress, ChecksumOpt checksumOpt)184   public FSDataOutputStream create(Path f,
185         FsPermission permission,
186         EnumSet<CreateFlag> flags,
187         int bufferSize,
188         short replication,
189         long blockSize,
190         Progressable progress,
191         ChecksumOpt checksumOpt) throws IOException {
192     return fs.create(f, permission,
193       flags, bufferSize, replication, blockSize, progress, checksumOpt);
194   }
195 
196   @Override
197   @Deprecated
createNonRecursive(Path f, FsPermission permission, EnumSet<CreateFlag> flags, int bufferSize, short replication, long blockSize, Progressable progress)198   public FSDataOutputStream createNonRecursive(Path f, FsPermission permission,
199       EnumSet<CreateFlag> flags, int bufferSize, short replication, long blockSize,
200       Progressable progress) throws IOException {
201 
202     return fs.createNonRecursive(f, permission, flags, bufferSize, replication, blockSize,
203         progress);
204   }
205 
206   /**
207    * Set replication for an existing file.
208    *
209    * @param src file name
210    * @param replication new replication
211    * @throws IOException
212    * @return true if successful;
213    *         false if file does not exist or is a directory
214    */
215   @Override
setReplication(Path src, short replication)216   public boolean setReplication(Path src, short replication) throws IOException {
217     return fs.setReplication(src, replication);
218   }
219 
220   /**
221    * Renames Path src to Path dst.  Can take place on local fs
222    * or remote DFS.
223    */
224   @Override
rename(Path src, Path dst)225   public boolean rename(Path src, Path dst) throws IOException {
226     return fs.rename(src, dst);
227   }
228 
229   @Override
truncate(Path f, final long newLength)230   public boolean truncate(Path f, final long newLength) throws IOException {
231     return fs.truncate(f, newLength);
232   }
233 
234   /** Delete a file */
235   @Override
delete(Path f, boolean recursive)236   public boolean delete(Path f, boolean recursive) throws IOException {
237     return fs.delete(f, recursive);
238   }
239 
240   /** List files in a directory. */
241   @Override
listStatus(Path f)242   public FileStatus[] listStatus(Path f) throws IOException {
243     return fs.listStatus(f);
244   }
245 
246   @Override
listCorruptFileBlocks(Path path)247   public RemoteIterator<Path> listCorruptFileBlocks(Path path)
248     throws IOException {
249     return fs.listCorruptFileBlocks(path);
250   }
251 
252   /** List files and its block locations in a directory. */
253   @Override
listLocatedStatus(Path f)254   public RemoteIterator<LocatedFileStatus> listLocatedStatus(Path f)
255   throws IOException {
256     return fs.listLocatedStatus(f);
257   }
258 
259   /** Return a remote iterator for listing in a directory */
260   @Override
listStatusIterator(Path f)261   public RemoteIterator<FileStatus> listStatusIterator(Path f)
262   throws IOException {
263     return fs.listStatusIterator(f);
264    }
265 
266   @Override
getHomeDirectory()267   public Path getHomeDirectory() {
268     return fs.getHomeDirectory();
269   }
270 
271 
272   /**
273    * Set the current working directory for the given file system. All relative
274    * paths will be resolved relative to it.
275    *
276    * @param newDir
277    */
278   @Override
setWorkingDirectory(Path newDir)279   public void setWorkingDirectory(Path newDir) {
280     fs.setWorkingDirectory(newDir);
281   }
282 
283   /**
284    * Get the current working directory for the given file system
285    *
286    * @return the directory pathname
287    */
288   @Override
getWorkingDirectory()289   public Path getWorkingDirectory() {
290     return fs.getWorkingDirectory();
291   }
292 
293   @Override
getInitialWorkingDirectory()294   protected Path getInitialWorkingDirectory() {
295     return fs.getInitialWorkingDirectory();
296   }
297 
298   @Override
getStatus(Path p)299   public FsStatus getStatus(Path p) throws IOException {
300     return fs.getStatus(p);
301   }
302 
303   @Override
mkdirs(Path f, FsPermission permission)304   public boolean mkdirs(Path f, FsPermission permission) throws IOException {
305     return fs.mkdirs(f, permission);
306   }
307 
308 
309   /**
310    * The src file is on the local disk.  Add it to FS at
311    * the given dst name.
312    * delSrc indicates if the source should be removed
313    */
314   @Override
copyFromLocalFile(boolean delSrc, Path src, Path dst)315   public void copyFromLocalFile(boolean delSrc, Path src, Path dst)
316     throws IOException {
317     fs.copyFromLocalFile(delSrc, src, dst);
318   }
319 
320   /**
321    * The src files are on the local disk.  Add it to FS at
322    * the given dst name.
323    * delSrc indicates if the source should be removed
324    */
325   @Override
copyFromLocalFile(boolean delSrc, boolean overwrite, Path[] srcs, Path dst)326   public void copyFromLocalFile(boolean delSrc, boolean overwrite,
327                                 Path[] srcs, Path dst)
328     throws IOException {
329     fs.copyFromLocalFile(delSrc, overwrite, srcs, dst);
330   }
331 
332   /**
333    * The src file is on the local disk.  Add it to FS at
334    * the given dst name.
335    * delSrc indicates if the source should be removed
336    */
337   @Override
copyFromLocalFile(boolean delSrc, boolean overwrite, Path src, Path dst)338   public void copyFromLocalFile(boolean delSrc, boolean overwrite,
339                                 Path src, Path dst)
340     throws IOException {
341     fs.copyFromLocalFile(delSrc, overwrite, src, dst);
342   }
343 
344   /**
345    * The src file is under FS, and the dst is on the local disk.
346    * Copy it from FS control to the local dst name.
347    * delSrc indicates if the src will be removed or not.
348    */
349   @Override
copyToLocalFile(boolean delSrc, Path src, Path dst)350   public void copyToLocalFile(boolean delSrc, Path src, Path dst)
351     throws IOException {
352     fs.copyToLocalFile(delSrc, src, dst);
353   }
354 
355   /**
356    * Returns a local File that the user can write output to.  The caller
357    * provides both the eventual FS target name and the local working
358    * file.  If the FS is local, we write directly into the target.  If
359    * the FS is remote, we write into the tmp local area.
360    */
361   @Override
startLocalOutput(Path fsOutputFile, Path tmpLocalFile)362   public Path startLocalOutput(Path fsOutputFile, Path tmpLocalFile)
363     throws IOException {
364     return fs.startLocalOutput(fsOutputFile, tmpLocalFile);
365   }
366 
367   /**
368    * Called when we're all done writing to the target.  A local FS will
369    * do nothing, because we've written to exactly the right place.  A remote
370    * FS will copy the contents of tmpLocalFile to the correct target at
371    * fsOutputFile.
372    */
373   @Override
completeLocalOutput(Path fsOutputFile, Path tmpLocalFile)374   public void completeLocalOutput(Path fsOutputFile, Path tmpLocalFile)
375     throws IOException {
376     fs.completeLocalOutput(fsOutputFile, tmpLocalFile);
377   }
378 
379   /** Return the total size of all files in the filesystem.*/
380   @Override
getUsed()381   public long getUsed() throws IOException{
382     return fs.getUsed();
383   }
384 
385   @Override
getDefaultBlockSize()386   public long getDefaultBlockSize() {
387     return fs.getDefaultBlockSize();
388   }
389 
390   @Override
getDefaultReplication()391   public short getDefaultReplication() {
392     return fs.getDefaultReplication();
393   }
394 
395   @Override
getServerDefaults()396   public FsServerDefaults getServerDefaults() throws IOException {
397     return fs.getServerDefaults();
398   }
399 
400   // path variants delegate to underlying filesystem
401   @Override
getDefaultBlockSize(Path f)402   public long getDefaultBlockSize(Path f) {
403     return fs.getDefaultBlockSize(f);
404   }
405 
406   @Override
getDefaultReplication(Path f)407   public short getDefaultReplication(Path f) {
408     return fs.getDefaultReplication(f);
409   }
410 
411   @Override
getServerDefaults(Path f)412   public FsServerDefaults getServerDefaults(Path f) throws IOException {
413     return fs.getServerDefaults(f);
414   }
415 
416   /**
417    * Get file status.
418    */
419   @Override
getFileStatus(Path f)420   public FileStatus getFileStatus(Path f) throws IOException {
421     return fs.getFileStatus(f);
422   }
423 
424   @Override
access(Path path, FsAction mode)425   public void access(Path path, FsAction mode) throws AccessControlException,
426       FileNotFoundException, IOException {
427     fs.access(path, mode);
428   }
429 
createSymlink(final Path target, final Path link, final boolean createParent)430   public void createSymlink(final Path target, final Path link,
431       final boolean createParent) throws AccessControlException,
432       FileAlreadyExistsException, FileNotFoundException,
433       ParentNotDirectoryException, UnsupportedFileSystemException,
434       IOException {
435     fs.createSymlink(target, link, createParent);
436   }
437 
getFileLinkStatus(final Path f)438   public FileStatus getFileLinkStatus(final Path f)
439       throws AccessControlException, FileNotFoundException,
440       UnsupportedFileSystemException, IOException {
441     return fs.getFileLinkStatus(f);
442   }
443 
supportsSymlinks()444   public boolean supportsSymlinks() {
445     return fs.supportsSymlinks();
446   }
447 
getLinkTarget(Path f)448   public Path getLinkTarget(Path f) throws IOException {
449     return fs.getLinkTarget(f);
450   }
451 
resolveLink(Path f)452   protected Path resolveLink(Path f) throws IOException {
453     return fs.resolveLink(f);
454   }
455 
456   @Override
getFileChecksum(Path f)457   public FileChecksum getFileChecksum(Path f) throws IOException {
458     return fs.getFileChecksum(f);
459   }
460 
461   @Override
getFileChecksum(Path f, long length)462   public FileChecksum getFileChecksum(Path f, long length) throws IOException {
463     return fs.getFileChecksum(f, length);
464   }
465 
466   @Override
setVerifyChecksum(boolean verifyChecksum)467   public void setVerifyChecksum(boolean verifyChecksum) {
468     fs.setVerifyChecksum(verifyChecksum);
469   }
470 
471   @Override
setWriteChecksum(boolean writeChecksum)472   public void setWriteChecksum(boolean writeChecksum) {
473     fs.setWriteChecksum(writeChecksum);
474   }
475 
476   @Override
getConf()477   public Configuration getConf() {
478     return fs.getConf();
479   }
480 
481   @Override
close()482   public void close() throws IOException {
483     super.close();
484     fs.close();
485   }
486 
487   @Override
setOwner(Path p, String username, String groupname )488   public void setOwner(Path p, String username, String groupname
489       ) throws IOException {
490     fs.setOwner(p, username, groupname);
491   }
492 
493   @Override
setTimes(Path p, long mtime, long atime )494   public void setTimes(Path p, long mtime, long atime
495       ) throws IOException {
496     fs.setTimes(p, mtime, atime);
497   }
498 
499   @Override
setPermission(Path p, FsPermission permission )500   public void setPermission(Path p, FsPermission permission
501       ) throws IOException {
502     fs.setPermission(p, permission);
503   }
504 
505   @Override
primitiveCreate(Path f, FsPermission absolutePermission, EnumSet<CreateFlag> flag, int bufferSize, short replication, long blockSize, Progressable progress, ChecksumOpt checksumOpt)506   protected FSDataOutputStream primitiveCreate(Path f,
507       FsPermission absolutePermission, EnumSet<CreateFlag> flag,
508       int bufferSize, short replication, long blockSize,
509       Progressable progress, ChecksumOpt checksumOpt)
510       throws IOException {
511     return fs.primitiveCreate(f, absolutePermission, flag,
512         bufferSize, replication, blockSize, progress, checksumOpt);
513   }
514 
515   @Override
516   @SuppressWarnings("deprecation")
primitiveMkdir(Path f, FsPermission abdolutePermission)517   protected boolean primitiveMkdir(Path f, FsPermission abdolutePermission)
518       throws IOException {
519     return fs.primitiveMkdir(f, abdolutePermission);
520   }
521 
522   @Override // FileSystem
getChildFileSystems()523   public FileSystem[] getChildFileSystems() {
524     return new FileSystem[]{fs};
525   }
526 
527   @Override // FileSystem
createSnapshot(Path path, String snapshotName)528   public Path createSnapshot(Path path, String snapshotName)
529       throws IOException {
530     return fs.createSnapshot(path, snapshotName);
531   }
532 
533   @Override // FileSystem
renameSnapshot(Path path, String snapshotOldName, String snapshotNewName)534   public void renameSnapshot(Path path, String snapshotOldName,
535       String snapshotNewName) throws IOException {
536     fs.renameSnapshot(path, snapshotOldName, snapshotNewName);
537   }
538 
539   @Override // FileSystem
deleteSnapshot(Path path, String snapshotName)540   public void deleteSnapshot(Path path, String snapshotName)
541       throws IOException {
542     fs.deleteSnapshot(path, snapshotName);
543   }
544 
545   @Override
modifyAclEntries(Path path, List<AclEntry> aclSpec)546   public void modifyAclEntries(Path path, List<AclEntry> aclSpec)
547       throws IOException {
548     fs.modifyAclEntries(path, aclSpec);
549   }
550 
551   @Override
removeAclEntries(Path path, List<AclEntry> aclSpec)552   public void removeAclEntries(Path path, List<AclEntry> aclSpec)
553       throws IOException {
554     fs.removeAclEntries(path, aclSpec);
555   }
556 
557   @Override
removeDefaultAcl(Path path)558   public void removeDefaultAcl(Path path) throws IOException {
559     fs.removeDefaultAcl(path);
560   }
561 
562   @Override
removeAcl(Path path)563   public void removeAcl(Path path) throws IOException {
564     fs.removeAcl(path);
565   }
566 
567   @Override
setAcl(Path path, List<AclEntry> aclSpec)568   public void setAcl(Path path, List<AclEntry> aclSpec) throws IOException {
569     fs.setAcl(path, aclSpec);
570   }
571 
572   @Override
getAclStatus(Path path)573   public AclStatus getAclStatus(Path path) throws IOException {
574     return fs.getAclStatus(path);
575   }
576 
577   @Override
setXAttr(Path path, String name, byte[] value)578   public void setXAttr(Path path, String name, byte[] value)
579       throws IOException {
580     fs.setXAttr(path, name, value);
581   }
582 
583   @Override
setXAttr(Path path, String name, byte[] value, EnumSet<XAttrSetFlag> flag)584   public void setXAttr(Path path, String name, byte[] value,
585       EnumSet<XAttrSetFlag> flag) throws IOException {
586     fs.setXAttr(path, name, value, flag);
587   }
588 
589   @Override
getXAttr(Path path, String name)590   public byte[] getXAttr(Path path, String name) throws IOException {
591     return fs.getXAttr(path, name);
592   }
593 
594   @Override
getXAttrs(Path path)595   public Map<String, byte[]> getXAttrs(Path path) throws IOException {
596     return fs.getXAttrs(path);
597   }
598 
599   @Override
getXAttrs(Path path, List<String> names)600   public Map<String, byte[]> getXAttrs(Path path, List<String> names)
601       throws IOException {
602     return fs.getXAttrs(path, names);
603   }
604 
605   @Override
listXAttrs(Path path)606   public List<String> listXAttrs(Path path) throws IOException {
607     return fs.listXAttrs(path);
608   }
609 
610   @Override
removeXAttr(Path path, String name)611   public void removeXAttr(Path path, String name) throws IOException {
612     fs.removeXAttr(path, name);
613   }
614 }
615