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; 019 020import java.io.FileNotFoundException; 021import java.io.IOException; 022import java.io.InputStream; 023import java.io.OutputStream; 024import java.net.URI; 025import java.security.PrivilegedExceptionAction; 026import java.util.ArrayList; 027import java.util.Collection; 028import java.util.EnumSet; 029import java.util.HashSet; 030import java.util.IdentityHashMap; 031import java.util.List; 032import java.util.Map; 033import java.util.Set; 034import java.util.Stack; 035import java.util.TreeSet; 036import java.util.Map.Entry; 037 038import org.apache.commons.logging.Log; 039import org.apache.commons.logging.LogFactory; 040import org.apache.hadoop.HadoopIllegalArgumentException; 041import org.apache.hadoop.classification.InterfaceAudience; 042import org.apache.hadoop.classification.InterfaceStability; 043import org.apache.hadoop.conf.Configuration; 044import org.apache.hadoop.fs.FileSystem.Statistics; 045import org.apache.hadoop.fs.Options.CreateOpts; 046import org.apache.hadoop.fs.permission.AclEntry; 047import org.apache.hadoop.fs.permission.AclStatus; 048import org.apache.hadoop.fs.permission.FsAction; 049import org.apache.hadoop.fs.permission.FsPermission; 050import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY; 051import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_DEFAULT; 052 053import org.apache.hadoop.io.IOUtils; 054import org.apache.hadoop.ipc.RpcClientException; 055import org.apache.hadoop.ipc.RpcServerException; 056import org.apache.hadoop.ipc.UnexpectedServerException; 057import org.apache.hadoop.fs.InvalidPathException; 058import org.apache.hadoop.security.AccessControlException; 059import org.apache.hadoop.security.UserGroupInformation; 060import org.apache.hadoop.security.token.Token; 061import org.apache.hadoop.util.ShutdownHookManager; 062 063import com.google.common.base.Preconditions; 064 065/** 066 * The FileContext class provides an interface for users of the Hadoop 067 * file system. It exposes a number of file system operations, e.g. create, 068 * open, list. 069 * 070 * <h2>Path Names</h2> 071 * 072 * The Hadoop file system supports a URI namespace and URI names. This enables 073 * multiple types of file systems to be referenced using fully-qualified URIs. 074 * Two common Hadoop file system implementations are 075 * <ul> 076 * <li>the local file system: file:///path 077 * <li>the HDFS file system: hdfs://nnAddress:nnPort/path 078 * </ul> 079 * 080 * The Hadoop file system also supports additional naming schemes besides URIs. 081 * Hadoop has the concept of a <i>default file system</i>, which implies a 082 * default URI scheme and authority. This enables <i>slash-relative names</i> 083 * relative to the default FS, which are more convenient for users and 084 * application writers. The default FS is typically set by the user's 085 * environment, though it can also be manually specified. 086 * <p> 087 * 088 * Hadoop also supports <i>working-directory-relative</i> names, which are paths 089 * relative to the current working directory (similar to Unix). The working 090 * directory can be in a different file system than the default FS. 091 * <p> 092 * Thus, Hadoop path names can be specified as one of the following: 093 * <ul> 094 * <li>a fully-qualified URI: scheme://authority/path (e.g. 095 * hdfs://nnAddress:nnPort/foo/bar) 096 * <li>a slash-relative name: path relative to the default file system (e.g. 097 * /foo/bar) 098 * <li>a working-directory-relative name: path relative to the working dir (e.g. 099 * foo/bar) 100 * </ul> 101 * Relative paths with scheme (scheme:foo/bar) are illegal. 102 * 103 * <h2>Role of FileContext and Configuration Defaults</h2> 104 * 105 * The FileContext is the analogue of per-process file-related state in Unix. It 106 * contains two properties: 107 * 108 * <ul> 109 * <li>the default file system (for resolving slash-relative names) 110 * <li>the umask (for file permissions) 111 * </ul> 112 * In general, these properties are obtained from the default configuration file 113 * in the user's environment (see {@link Configuration}). 114 * 115 * Further file system properties are specified on the server-side. File system 116 * operations default to using these server-side defaults unless otherwise 117 * specified. 118 * <p> 119 * The file system related server-side defaults are: 120 * <ul> 121 * <li> the home directory (default is "/user/userName") 122 * <li> the initial wd (only for local fs) 123 * <li> replication factor 124 * <li> block size 125 * <li> buffer size 126 * <li> encryptDataTransfer 127 * <li> checksum option. (checksumType and bytesPerChecksum) 128 * </ul> 129 * 130 * <h2>Example Usage</h2> 131 * 132 * Example 1: use the default config read from the $HADOOP_CONFIG/core.xml. 133 * Unspecified values come from core-defaults.xml in the release jar. 134 * <ul> 135 * <li> myFContext = FileContext.getFileContext(); // uses the default config 136 * // which has your default FS 137 * <li> myFContext.create(path, ...); 138 * <li> myFContext.setWorkingDir(path); 139 * <li> myFContext.open (path, ...); 140 * <li>... 141 * </ul> 142 * Example 2: Get a FileContext with a specific URI as the default FS 143 * <ul> 144 * <li> myFContext = FileContext.getFileContext(URI); 145 * <li> myFContext.create(path, ...); 146 * <li>... 147 * </ul> 148 * Example 3: FileContext with local file system as the default 149 * <ul> 150 * <li> myFContext = FileContext.getLocalFSFileContext(); 151 * <li> myFContext.create(path, ...); 152 * <li> ... 153 * </ul> 154 * Example 4: Use a specific config, ignoring $HADOOP_CONFIG 155 * Generally you should not need use a config unless you are doing 156 * <ul> 157 * <li> configX = someConfigSomeOnePassedToYou; 158 * <li> myFContext = getFileContext(configX); // configX is not changed, 159 * // is passed down 160 * <li> myFContext.create(path, ...); 161 * <li>... 162 * </ul> 163 * 164 */ 165 166@InterfaceAudience.Public 167@InterfaceStability.Evolving /*Evolving for a release,to be changed to Stable */ 168public class FileContext { 169 170 public static final Log LOG = LogFactory.getLog(FileContext.class); 171 /** 172 * Default permission for directory and symlink 173 * In previous versions, this default permission was also used to 174 * create files, so files created end up with ugo+x permission. 175 * See HADOOP-9155 for detail. 176 * Two new constants are added to solve this, please use 177 * {@link FileContext#DIR_DEFAULT_PERM} for directory, and use 178 * {@link FileContext#FILE_DEFAULT_PERM} for file. 179 * This constant is kept for compatibility. 180 */ 181 public static final FsPermission DEFAULT_PERM = FsPermission.getDefault(); 182 /** 183 * Default permission for directory 184 */ 185 public static final FsPermission DIR_DEFAULT_PERM = FsPermission.getDirDefault(); 186 /** 187 * Default permission for file 188 */ 189 public static final FsPermission FILE_DEFAULT_PERM = FsPermission.getFileDefault(); 190 191 /** 192 * Priority of the FileContext shutdown hook. 193 */ 194 public static final int SHUTDOWN_HOOK_PRIORITY = 20; 195 196 /** 197 * List of files that should be deleted on JVM shutdown. 198 */ 199 static final Map<FileContext, Set<Path>> DELETE_ON_EXIT = 200 new IdentityHashMap<FileContext, Set<Path>>(); 201 202 /** JVM shutdown hook thread. */ 203 static final FileContextFinalizer FINALIZER = 204 new FileContextFinalizer(); 205 206 private static final PathFilter DEFAULT_FILTER = new PathFilter() { 207 @Override 208 public boolean accept(final Path file) { 209 return true; 210 } 211 }; 212 213 /** 214 * The FileContext is defined by. 215 * 1) defaultFS (slash) 216 * 2) wd 217 * 3) umask 218 */ 219 private final AbstractFileSystem defaultFS; //default FS for this FileContext. 220 private Path workingDir; // Fully qualified 221 private FsPermission umask; 222 private final Configuration conf; 223 private final UserGroupInformation ugi; 224 final boolean resolveSymlinks; 225 226 private FileContext(final AbstractFileSystem defFs, 227 final FsPermission theUmask, final Configuration aConf) { 228 defaultFS = defFs; 229 umask = FsPermission.getUMask(aConf); 230 conf = aConf; 231 try { 232 ugi = UserGroupInformation.getCurrentUser(); 233 } catch (IOException e) { 234 LOG.error("Exception in getCurrentUser: ",e); 235 throw new RuntimeException("Failed to get the current user " + 236 "while creating a FileContext", e); 237 } 238 /* 239 * Init the wd. 240 * WorkingDir is implemented at the FileContext layer 241 * NOT at the AbstractFileSystem layer. 242 * If the DefaultFS, such as localFilesystem has a notion of 243 * builtin WD, we use that as the initial WD. 244 * Otherwise the WD is initialized to the home directory. 245 */ 246 workingDir = defaultFS.getInitialWorkingDirectory(); 247 if (workingDir == null) { 248 workingDir = defaultFS.getHomeDirectory(); 249 } 250 resolveSymlinks = conf.getBoolean( 251 CommonConfigurationKeys.FS_CLIENT_RESOLVE_REMOTE_SYMLINKS_KEY, 252 CommonConfigurationKeys.FS_CLIENT_RESOLVE_REMOTE_SYMLINKS_DEFAULT); 253 util = new Util(); // for the inner class 254 } 255 256 /* 257 * Remove relative part - return "absolute": 258 * If input is relative path ("foo/bar") add wd: ie "/<workingDir>/foo/bar" 259 * A fully qualified uri ("hdfs://nn:p/foo/bar") or a slash-relative path 260 * ("/foo/bar") are returned unchanged. 261 * 262 * Applications that use FileContext should use #makeQualified() since 263 * they really want a fully qualified URI. 264 * Hence this method is not called makeAbsolute() and 265 * has been deliberately declared private. 266 */ 267 Path fixRelativePart(Path p) { 268 Preconditions.checkNotNull(p, "path cannot be null"); 269 if (p.isUriPathAbsolute()) { 270 return p; 271 } else { 272 return new Path(workingDir, p); 273 } 274 } 275 276 /** 277 * Delete all the paths that were marked as delete-on-exit. 278 */ 279 static void processDeleteOnExit() { 280 synchronized (DELETE_ON_EXIT) { 281 Set<Entry<FileContext, Set<Path>>> set = DELETE_ON_EXIT.entrySet(); 282 for (Entry<FileContext, Set<Path>> entry : set) { 283 FileContext fc = entry.getKey(); 284 Set<Path> paths = entry.getValue(); 285 for (Path path : paths) { 286 try { 287 fc.delete(path, true); 288 } catch (IOException e) { 289 LOG.warn("Ignoring failure to deleteOnExit for path " + path); 290 } 291 } 292 } 293 DELETE_ON_EXIT.clear(); 294 } 295 } 296 297 /** 298 * Get the file system of supplied path. 299 * 300 * @param absOrFqPath - absolute or fully qualified path 301 * @return the file system of the path 302 * 303 * @throws UnsupportedFileSystemException If the file system for 304 * <code>absOrFqPath</code> is not supported. 305 * @throws IOExcepton If the file system for <code>absOrFqPath</code> could 306 * not be instantiated. 307 */ 308 protected AbstractFileSystem getFSofPath(final Path absOrFqPath) 309 throws UnsupportedFileSystemException, IOException { 310 absOrFqPath.checkNotSchemeWithRelative(); 311 absOrFqPath.checkNotRelative(); 312 313 try { 314 // Is it the default FS for this FileContext? 315 defaultFS.checkPath(absOrFqPath); 316 return defaultFS; 317 } catch (Exception e) { // it is different FileSystem 318 return getAbstractFileSystem(ugi, absOrFqPath.toUri(), conf); 319 } 320 } 321 322 private static AbstractFileSystem getAbstractFileSystem( 323 UserGroupInformation user, final URI uri, final Configuration conf) 324 throws UnsupportedFileSystemException, IOException { 325 try { 326 return user.doAs(new PrivilegedExceptionAction<AbstractFileSystem>() { 327 @Override 328 public AbstractFileSystem run() throws UnsupportedFileSystemException { 329 return AbstractFileSystem.get(uri, conf); 330 } 331 }); 332 } catch (InterruptedException ex) { 333 LOG.error(ex); 334 throw new IOException("Failed to get the AbstractFileSystem for path: " 335 + uri, ex); 336 } 337 } 338 339 /** 340 * Protected Static Factory methods for getting a FileContexts 341 * that take a AbstractFileSystem as input. To be used for testing. 342 */ 343 344 /** 345 * Create a FileContext with specified FS as default using the specified 346 * config. 347 * 348 * @param defFS 349 * @param aConf 350 * @return new FileContext with specified FS as default. 351 */ 352 public static FileContext getFileContext(final AbstractFileSystem defFS, 353 final Configuration aConf) { 354 return new FileContext(defFS, FsPermission.getUMask(aConf), aConf); 355 } 356 357 /** 358 * Create a FileContext for specified file system using the default config. 359 * 360 * @param defaultFS 361 * @return a FileContext with the specified AbstractFileSystem 362 * as the default FS. 363 */ 364 protected static FileContext getFileContext( 365 final AbstractFileSystem defaultFS) { 366 return getFileContext(defaultFS, new Configuration()); 367 } 368 369 /** 370 * Static Factory methods for getting a FileContext. 371 * Note new file contexts are created for each call. 372 * The only singleton is the local FS context using the default config. 373 * 374 * Methods that use the default config: the default config read from the 375 * $HADOOP_CONFIG/core.xml, 376 * Unspecified key-values for config are defaulted from core-defaults.xml 377 * in the release jar. 378 * 379 * The keys relevant to the FileContext layer are extracted at time of 380 * construction. Changes to the config after the call are ignore 381 * by the FileContext layer. 382 * The conf is passed to lower layers like AbstractFileSystem and HDFS which 383 * pick up their own config variables. 384 */ 385 386 /** 387 * Create a FileContext using the default config read from the 388 * $HADOOP_CONFIG/core.xml, Unspecified key-values for config are defaulted 389 * from core-defaults.xml in the release jar. 390 * 391 * @throws UnsupportedFileSystemException If the file system from the default 392 * configuration is not supported 393 */ 394 public static FileContext getFileContext() 395 throws UnsupportedFileSystemException { 396 return getFileContext(new Configuration()); 397 } 398 399 /** 400 * @return a FileContext for the local file system using the default config. 401 * @throws UnsupportedFileSystemException If the file system for 402 * {@link FsConstants#LOCAL_FS_URI} is not supported. 403 */ 404 public static FileContext getLocalFSFileContext() 405 throws UnsupportedFileSystemException { 406 return getFileContext(FsConstants.LOCAL_FS_URI); 407 } 408 409 /** 410 * Create a FileContext for specified URI using the default config. 411 * 412 * @param defaultFsUri 413 * @return a FileContext with the specified URI as the default FS. 414 * 415 * @throws UnsupportedFileSystemException If the file system for 416 * <code>defaultFsUri</code> is not supported 417 */ 418 public static FileContext getFileContext(final URI defaultFsUri) 419 throws UnsupportedFileSystemException { 420 return getFileContext(defaultFsUri, new Configuration()); 421 } 422 423 /** 424 * Create a FileContext for specified default URI using the specified config. 425 * 426 * @param defaultFsUri 427 * @param aConf 428 * @return new FileContext for specified uri 429 * @throws UnsupportedFileSystemException If the file system with specified is 430 * not supported 431 * @throws RuntimeException If the file system specified is supported but 432 * could not be instantiated, or if login fails. 433 */ 434 public static FileContext getFileContext(final URI defaultFsUri, 435 final Configuration aConf) throws UnsupportedFileSystemException { 436 UserGroupInformation currentUser = null; 437 AbstractFileSystem defaultAfs = null; 438 if (defaultFsUri.getScheme() == null) { 439 return getFileContext(aConf); 440 } 441 try { 442 currentUser = UserGroupInformation.getCurrentUser(); 443 defaultAfs = getAbstractFileSystem(currentUser, defaultFsUri, aConf); 444 } catch (UnsupportedFileSystemException ex) { 445 throw ex; 446 } catch (IOException ex) { 447 LOG.error(ex); 448 throw new RuntimeException(ex); 449 } 450 return getFileContext(defaultAfs, aConf); 451 } 452 453 /** 454 * Create a FileContext using the passed config. Generally it is better to use 455 * {@link #getFileContext(URI, Configuration)} instead of this one. 456 * 457 * 458 * @param aConf 459 * @return new FileContext 460 * @throws UnsupportedFileSystemException If file system in the config 461 * is not supported 462 */ 463 public static FileContext getFileContext(final Configuration aConf) 464 throws UnsupportedFileSystemException { 465 final URI defaultFsUri = URI.create(aConf.get(FS_DEFAULT_NAME_KEY, 466 FS_DEFAULT_NAME_DEFAULT)); 467 if ( defaultFsUri.getScheme() != null 468 && !defaultFsUri.getScheme().trim().isEmpty()) { 469 return getFileContext(defaultFsUri, aConf); 470 } 471 throw new UnsupportedFileSystemException(String.format( 472 "%s: URI configured via %s carries no scheme", 473 defaultFsUri, FS_DEFAULT_NAME_KEY)); 474 } 475 476 /** 477 * @param aConf - from which the FileContext is configured 478 * @return a FileContext for the local file system using the specified config. 479 * 480 * @throws UnsupportedFileSystemException If default file system in the config 481 * is not supported 482 * 483 */ 484 public static FileContext getLocalFSFileContext(final Configuration aConf) 485 throws UnsupportedFileSystemException { 486 return getFileContext(FsConstants.LOCAL_FS_URI, aConf); 487 } 488 489 /* This method is needed for tests. */ 490 @InterfaceAudience.Private 491 @InterfaceStability.Unstable /* return type will change to AFS once 492 HADOOP-6223 is completed */ 493 public AbstractFileSystem getDefaultFileSystem() { 494 return defaultFS; 495 } 496 497 /** 498 * Set the working directory for wd-relative names (such a "foo/bar"). Working 499 * directory feature is provided by simply prefixing relative names with the 500 * working dir. Note this is different from Unix where the wd is actually set 501 * to the inode. Hence setWorkingDir does not follow symlinks etc. This works 502 * better in a distributed environment that has multiple independent roots. 503 * {@link #getWorkingDirectory()} should return what setWorkingDir() set. 504 * 505 * @param newWDir new working directory 506 * @throws IOException 507 * <br> 508 * NewWdir can be one of: 509 * <ul> 510 * <li>relative path: "foo/bar";</li> 511 * <li>absolute without scheme: "/foo/bar"</li> 512 * <li>fully qualified with scheme: "xx://auth/foo/bar"</li> 513 * </ul> 514 * <br> 515 * Illegal WDs: 516 * <ul> 517 * <li>relative with scheme: "xx:foo/bar"</li> 518 * <li>non existent directory</li> 519 * </ul> 520 */ 521 public void setWorkingDirectory(final Path newWDir) throws IOException { 522 newWDir.checkNotSchemeWithRelative(); 523 /* wd is stored as a fully qualified path. We check if the given 524 * path is not relative first since resolve requires and returns 525 * an absolute path. 526 */ 527 final Path newWorkingDir = new Path(workingDir, newWDir); 528 FileStatus status = getFileStatus(newWorkingDir); 529 if (status.isFile()) { 530 throw new FileNotFoundException("Cannot setWD to a file"); 531 } 532 workingDir = newWorkingDir; 533 } 534 535 /** 536 * Gets the working directory for wd-relative names (such a "foo/bar"). 537 */ 538 public Path getWorkingDirectory() { 539 return workingDir; 540 } 541 542 /** 543 * Gets the ugi in the file-context 544 * @return UserGroupInformation 545 */ 546 public UserGroupInformation getUgi() { 547 return ugi; 548 } 549 550 /** 551 * Return the current user's home directory in this file system. 552 * The default implementation returns "/user/$USER/". 553 * @return the home directory 554 */ 555 public Path getHomeDirectory() { 556 return defaultFS.getHomeDirectory(); 557 } 558 559 /** 560 * 561 * @return the umask of this FileContext 562 */ 563 public FsPermission getUMask() { 564 return umask; 565 } 566 567 /** 568 * Set umask to the supplied parameter. 569 * @param newUmask the new umask 570 */ 571 public void setUMask(final FsPermission newUmask) { 572 umask = newUmask; 573 } 574 575 576 /** 577 * Resolve the path following any symlinks or mount points 578 * @param f to be resolved 579 * @return fully qualified resolved path 580 * 581 * @throws FileNotFoundException If <code>f</code> does not exist 582 * @throws AccessControlException if access denied 583 * @throws IOException If an IO Error occurred 584 * 585 * Exceptions applicable to file systems accessed over RPC: 586 * @throws RpcClientException If an exception occurred in the RPC client 587 * @throws RpcServerException If an exception occurred in the RPC server 588 * @throws UnexpectedServerException If server implementation throws 589 * undeclared exception to RPC server 590 * 591 * RuntimeExceptions: 592 * @throws InvalidPathException If path <code>f</code> is not valid 593 */ 594 public Path resolvePath(final Path f) throws FileNotFoundException, 595 UnresolvedLinkException, AccessControlException, IOException { 596 return resolve(f); 597 } 598 599 /** 600 * Make the path fully qualified if it is isn't. 601 * A Fully-qualified path has scheme and authority specified and an absolute 602 * path. 603 * Use the default file system and working dir in this FileContext to qualify. 604 * @param path 605 * @return qualified path 606 */ 607 public Path makeQualified(final Path path) { 608 return path.makeQualified(defaultFS.getUri(), getWorkingDirectory()); 609 } 610 611 /** 612 * Create or overwrite file on indicated path and returns an output stream for 613 * writing into the file. 614 * 615 * @param f the file name to open 616 * @param createFlag gives the semantics of create; see {@link CreateFlag} 617 * @param opts file creation options; see {@link Options.CreateOpts}. 618 * <ul> 619 * <li>Progress - to report progress on the operation - default null 620 * <li>Permission - umask is applied against permission: default is 621 * FsPermissions:getDefault() 622 * 623 * <li>CreateParent - create missing parent path; default is to not 624 * to create parents 625 * <li>The defaults for the following are SS defaults of the file 626 * server implementing the target path. Not all parameters make sense 627 * for all kinds of file system - eg. localFS ignores Blocksize, 628 * replication, checksum 629 * <ul> 630 * <li>BufferSize - buffersize used in FSDataOutputStream 631 * <li>Blocksize - block size for file blocks 632 * <li>ReplicationFactor - replication for blocks 633 * <li>ChecksumParam - Checksum parameters. server default is used 634 * if not specified. 635 * </ul> 636 * </ul> 637 * 638 * @return {@link FSDataOutputStream} for created file 639 * 640 * @throws AccessControlException If access is denied 641 * @throws FileAlreadyExistsException If file <code>f</code> already exists 642 * @throws FileNotFoundException If parent of <code>f</code> does not exist 643 * and <code>createParent</code> is false 644 * @throws ParentNotDirectoryException If parent of <code>f</code> is not a 645 * directory. 646 * @throws UnsupportedFileSystemException If file system for <code>f</code> is 647 * not supported 648 * @throws IOException If an I/O error occurred 649 * 650 * Exceptions applicable to file systems accessed over RPC: 651 * @throws RpcClientException If an exception occurred in the RPC client 652 * @throws RpcServerException If an exception occurred in the RPC server 653 * @throws UnexpectedServerException If server implementation throws 654 * undeclared exception to RPC server 655 * 656 * RuntimeExceptions: 657 * @throws InvalidPathException If path <code>f</code> is not valid 658 */ 659 public FSDataOutputStream create(final Path f, 660 final EnumSet<CreateFlag> createFlag, Options.CreateOpts... opts) 661 throws AccessControlException, FileAlreadyExistsException, 662 FileNotFoundException, ParentNotDirectoryException, 663 UnsupportedFileSystemException, IOException { 664 Path absF = fixRelativePart(f); 665 666 // If one of the options is a permission, extract it & apply umask 667 // If not, add a default Perms and apply umask; 668 // AbstractFileSystem#create 669 670 CreateOpts.Perms permOpt = CreateOpts.getOpt(CreateOpts.Perms.class, opts); 671 FsPermission permission = (permOpt != null) ? permOpt.getValue() : 672 FILE_DEFAULT_PERM; 673 permission = permission.applyUMask(umask); 674 675 final CreateOpts[] updatedOpts = 676 CreateOpts.setOpt(CreateOpts.perms(permission), opts); 677 return new FSLinkResolver<FSDataOutputStream>() { 678 @Override 679 public FSDataOutputStream next(final AbstractFileSystem fs, final Path p) 680 throws IOException { 681 return fs.create(p, createFlag, updatedOpts); 682 } 683 }.resolve(this, absF); 684 } 685 686 /** 687 * Make(create) a directory and all the non-existent parents. 688 * 689 * @param dir - the dir to make 690 * @param permission - permissions is set permission&~umask 691 * @param createParent - if true then missing parent dirs are created if false 692 * then parent must exist 693 * 694 * @throws AccessControlException If access is denied 695 * @throws FileAlreadyExistsException If directory <code>dir</code> already 696 * exists 697 * @throws FileNotFoundException If parent of <code>dir</code> does not exist 698 * and <code>createParent</code> is false 699 * @throws ParentNotDirectoryException If parent of <code>dir</code> is not a 700 * directory 701 * @throws UnsupportedFileSystemException If file system for <code>dir</code> 702 * is not supported 703 * @throws IOException If an I/O error occurred 704 * 705 * Exceptions applicable to file systems accessed over RPC: 706 * @throws RpcClientException If an exception occurred in the RPC client 707 * @throws UnexpectedServerException If server implementation throws 708 * undeclared exception to RPC server 709 * 710 * RuntimeExceptions: 711 * @throws InvalidPathException If path <code>dir</code> is not valid 712 */ 713 public void mkdir(final Path dir, final FsPermission permission, 714 final boolean createParent) throws AccessControlException, 715 FileAlreadyExistsException, FileNotFoundException, 716 ParentNotDirectoryException, UnsupportedFileSystemException, 717 IOException { 718 final Path absDir = fixRelativePart(dir); 719 final FsPermission absFerms = (permission == null ? 720 FsPermission.getDirDefault() : permission).applyUMask(umask); 721 new FSLinkResolver<Void>() { 722 @Override 723 public Void next(final AbstractFileSystem fs, final Path p) 724 throws IOException, UnresolvedLinkException { 725 fs.mkdir(p, absFerms, createParent); 726 return null; 727 } 728 }.resolve(this, absDir); 729 } 730 731 /** 732 * Delete a file. 733 * @param f the path to delete. 734 * @param recursive if path is a directory and set to 735 * true, the directory is deleted else throws an exception. In 736 * case of a file the recursive can be set to either true or false. 737 * 738 * @throws AccessControlException If access is denied 739 * @throws FileNotFoundException If <code>f</code> does not exist 740 * @throws UnsupportedFileSystemException If file system for <code>f</code> is 741 * not supported 742 * @throws IOException If an I/O error occurred 743 * 744 * Exceptions applicable to file systems accessed over RPC: 745 * @throws RpcClientException If an exception occurred in the RPC client 746 * @throws RpcServerException If an exception occurred in the RPC server 747 * @throws UnexpectedServerException If server implementation throws 748 * undeclared exception to RPC server 749 * 750 * RuntimeExceptions: 751 * @throws InvalidPathException If path <code>f</code> is invalid 752 */ 753 public boolean delete(final Path f, final boolean recursive) 754 throws AccessControlException, FileNotFoundException, 755 UnsupportedFileSystemException, IOException { 756 Path absF = fixRelativePart(f); 757 return new FSLinkResolver<Boolean>() { 758 @Override 759 public Boolean next(final AbstractFileSystem fs, final Path p) 760 throws IOException, UnresolvedLinkException { 761 return Boolean.valueOf(fs.delete(p, recursive)); 762 } 763 }.resolve(this, absF); 764 } 765 766 /** 767 * Opens an FSDataInputStream at the indicated Path using 768 * default buffersize. 769 * @param f the file name to open 770 * 771 * @throws AccessControlException If access is denied 772 * @throws FileNotFoundException If file <code>f</code> does not exist 773 * @throws UnsupportedFileSystemException If file system for <code>f</code> 774 * is not supported 775 * @throws IOException If an I/O error occurred 776 * 777 * Exceptions applicable to file systems accessed over RPC: 778 * @throws RpcClientException If an exception occurred in the RPC client 779 * @throws RpcServerException If an exception occurred in the RPC server 780 * @throws UnexpectedServerException If server implementation throws 781 * undeclared exception to RPC server 782 */ 783 public FSDataInputStream open(final Path f) throws AccessControlException, 784 FileNotFoundException, UnsupportedFileSystemException, IOException { 785 final Path absF = fixRelativePart(f); 786 return new FSLinkResolver<FSDataInputStream>() { 787 @Override 788 public FSDataInputStream next(final AbstractFileSystem fs, final Path p) 789 throws IOException, UnresolvedLinkException { 790 return fs.open(p); 791 } 792 }.resolve(this, absF); 793 } 794 795 /** 796 * Opens an FSDataInputStream at the indicated Path. 797 * 798 * @param f the file name to open 799 * @param bufferSize the size of the buffer to be used. 800 * 801 * @throws AccessControlException If access is denied 802 * @throws FileNotFoundException If file <code>f</code> does not exist 803 * @throws UnsupportedFileSystemException If file system for <code>f</code> is 804 * not supported 805 * @throws IOException If an I/O error occurred 806 * 807 * Exceptions applicable to file systems accessed over RPC: 808 * @throws RpcClientException If an exception occurred in the RPC client 809 * @throws RpcServerException If an exception occurred in the RPC server 810 * @throws UnexpectedServerException If server implementation throws 811 * undeclared exception to RPC server 812 */ 813 public FSDataInputStream open(final Path f, final int bufferSize) 814 throws AccessControlException, FileNotFoundException, 815 UnsupportedFileSystemException, IOException { 816 final Path absF = fixRelativePart(f); 817 return new FSLinkResolver<FSDataInputStream>() { 818 @Override 819 public FSDataInputStream next(final AbstractFileSystem fs, final Path p) 820 throws IOException, UnresolvedLinkException { 821 return fs.open(p, bufferSize); 822 } 823 }.resolve(this, absF); 824 } 825 826 /** 827 * Truncate the file in the indicated path to the indicated size. 828 * <ul> 829 * <li>Fails if path is a directory. 830 * <li>Fails if path does not exist. 831 * <li>Fails if path is not closed. 832 * <li>Fails if new size is greater than current size. 833 * </ul> 834 * @param f The path to the file to be truncated 835 * @param newLength The size the file is to be truncated to 836 * 837 * @return <code>true</code> if the file has been truncated to the desired 838 * <code>newLength</code> and is immediately available to be reused for 839 * write operations such as <code>append</code>, or 840 * <code>false</code> if a background process of adjusting the length of 841 * the last block has been started, and clients should wait for it to 842 * complete before proceeding with further file updates. 843 * 844 * @throws AccessControlException If access is denied 845 * @throws FileNotFoundException If file <code>f</code> does not exist 846 * @throws UnsupportedFileSystemException If file system for <code>f</code> is 847 * not supported 848 * @throws IOException If an I/O error occurred 849 * 850 * Exceptions applicable to file systems accessed over RPC: 851 * @throws RpcClientException If an exception occurred in the RPC client 852 * @throws RpcServerException If an exception occurred in the RPC server 853 * @throws UnexpectedServerException If server implementation throws 854 * undeclared exception to RPC server 855 */ 856 public boolean truncate(final Path f, final long newLength) 857 throws AccessControlException, FileNotFoundException, 858 UnsupportedFileSystemException, IOException { 859 final Path absF = fixRelativePart(f); 860 return new FSLinkResolver<Boolean>() { 861 @Override 862 public Boolean next(final AbstractFileSystem fs, final Path p) 863 throws IOException, UnresolvedLinkException { 864 return fs.truncate(p, newLength); 865 } 866 }.resolve(this, absF); 867 } 868 869 /** 870 * Set replication for an existing file. 871 * 872 * @param f file name 873 * @param replication new replication 874 * 875 * @return true if successful 876 * 877 * @throws AccessControlException If access is denied 878 * @throws FileNotFoundException If file <code>f</code> does not exist 879 * @throws IOException If an I/O error occurred 880 * 881 * Exceptions applicable to file systems accessed over RPC: 882 * @throws RpcClientException If an exception occurred in the RPC client 883 * @throws RpcServerException If an exception occurred in the RPC server 884 * @throws UnexpectedServerException If server implementation throws 885 * undeclared exception to RPC server 886 */ 887 public boolean setReplication(final Path f, final short replication) 888 throws AccessControlException, FileNotFoundException, 889 IOException { 890 final Path absF = fixRelativePart(f); 891 return new FSLinkResolver<Boolean>() { 892 @Override 893 public Boolean next(final AbstractFileSystem fs, final Path p) 894 throws IOException, UnresolvedLinkException { 895 return Boolean.valueOf(fs.setReplication(p, replication)); 896 } 897 }.resolve(this, absF); 898 } 899 900 /** 901 * Renames Path src to Path dst 902 * <ul> 903 * <li 904 * <li>Fails if src is a file and dst is a directory. 905 * <li>Fails if src is a directory and dst is a file. 906 * <li>Fails if the parent of dst does not exist or is a file. 907 * </ul> 908 * <p> 909 * If OVERWRITE option is not passed as an argument, rename fails if the dst 910 * already exists. 911 * <p> 912 * If OVERWRITE option is passed as an argument, rename overwrites the dst if 913 * it is a file or an empty directory. Rename fails if dst is a non-empty 914 * directory. 915 * <p> 916 * Note that atomicity of rename is dependent on the file system 917 * implementation. Please refer to the file system documentation for details 918 * <p> 919 * 920 * @param src path to be renamed 921 * @param dst new path after rename 922 * 923 * @throws AccessControlException If access is denied 924 * @throws FileAlreadyExistsException If <code>dst</code> already exists and 925 * <code>options</options> has {@link Options.Rename#OVERWRITE} 926 * option false. 927 * @throws FileNotFoundException If <code>src</code> does not exist 928 * @throws ParentNotDirectoryException If parent of <code>dst</code> is not a 929 * directory 930 * @throws UnsupportedFileSystemException If file system for <code>src</code> 931 * and <code>dst</code> is not supported 932 * @throws IOException If an I/O error occurred 933 * 934 * Exceptions applicable to file systems accessed over RPC: 935 * @throws RpcClientException If an exception occurred in the RPC client 936 * @throws RpcServerException If an exception occurred in the RPC server 937 * @throws UnexpectedServerException If server implementation throws 938 * undeclared exception to RPC server 939 */ 940 public void rename(final Path src, final Path dst, 941 final Options.Rename... options) throws AccessControlException, 942 FileAlreadyExistsException, FileNotFoundException, 943 ParentNotDirectoryException, UnsupportedFileSystemException, 944 IOException { 945 final Path absSrc = fixRelativePart(src); 946 final Path absDst = fixRelativePart(dst); 947 AbstractFileSystem srcFS = getFSofPath(absSrc); 948 AbstractFileSystem dstFS = getFSofPath(absDst); 949 if(!srcFS.getUri().equals(dstFS.getUri())) { 950 throw new IOException("Renames across AbstractFileSystems not supported"); 951 } 952 try { 953 srcFS.rename(absSrc, absDst, options); 954 } catch (UnresolvedLinkException e) { 955 /* We do not know whether the source or the destination path 956 * was unresolved. Resolve the source path up until the final 957 * path component, then fully resolve the destination. 958 */ 959 final Path source = resolveIntermediate(absSrc); 960 new FSLinkResolver<Void>() { 961 @Override 962 public Void next(final AbstractFileSystem fs, final Path p) 963 throws IOException, UnresolvedLinkException { 964 fs.rename(source, p, options); 965 return null; 966 } 967 }.resolve(this, absDst); 968 } 969 } 970 971 /** 972 * Set permission of a path. 973 * @param f 974 * @param permission - the new absolute permission (umask is not applied) 975 * 976 * @throws AccessControlException If access is denied 977 * @throws FileNotFoundException If <code>f</code> does not exist 978 * @throws UnsupportedFileSystemException If file system for <code>f</code> 979 * is not supported 980 * @throws IOException If an I/O error occurred 981 * 982 * Exceptions applicable to file systems accessed over RPC: 983 * @throws RpcClientException If an exception occurred in the RPC client 984 * @throws RpcServerException If an exception occurred in the RPC server 985 * @throws UnexpectedServerException If server implementation throws 986 * undeclared exception to RPC server 987 */ 988 public void setPermission(final Path f, final FsPermission permission) 989 throws AccessControlException, FileNotFoundException, 990 UnsupportedFileSystemException, IOException { 991 final Path absF = fixRelativePart(f); 992 new FSLinkResolver<Void>() { 993 @Override 994 public Void next(final AbstractFileSystem fs, final Path p) 995 throws IOException, UnresolvedLinkException { 996 fs.setPermission(p, permission); 997 return null; 998 } 999 }.resolve(this, absF); 1000 } 1001 1002 /** 1003 * Set owner of a path (i.e. a file or a directory). The parameters username 1004 * and groupname cannot both be null. 1005 * 1006 * @param f The path 1007 * @param username If it is null, the original username remains unchanged. 1008 * @param groupname If it is null, the original groupname remains unchanged. 1009 * 1010 * @throws AccessControlException If access is denied 1011 * @throws FileNotFoundException If <code>f</code> does not exist 1012 * @throws UnsupportedFileSystemException If file system for <code>f</code> is 1013 * not supported 1014 * @throws IOException If an I/O error occurred 1015 * 1016 * Exceptions applicable to file systems accessed over RPC: 1017 * @throws RpcClientException If an exception occurred in the RPC client 1018 * @throws RpcServerException If an exception occurred in the RPC server 1019 * @throws UnexpectedServerException If server implementation throws 1020 * undeclared exception to RPC server 1021 * 1022 * RuntimeExceptions: 1023 * @throws HadoopIllegalArgumentException If <code>username</code> or 1024 * <code>groupname</code> is invalid. 1025 */ 1026 public void setOwner(final Path f, final String username, 1027 final String groupname) throws AccessControlException, 1028 UnsupportedFileSystemException, FileNotFoundException, 1029 IOException { 1030 if ((username == null) && (groupname == null)) { 1031 throw new HadoopIllegalArgumentException( 1032 "username and groupname cannot both be null"); 1033 } 1034 final Path absF = fixRelativePart(f); 1035 new FSLinkResolver<Void>() { 1036 @Override 1037 public Void next(final AbstractFileSystem fs, final Path p) 1038 throws IOException, UnresolvedLinkException { 1039 fs.setOwner(p, username, groupname); 1040 return null; 1041 } 1042 }.resolve(this, absF); 1043 } 1044 1045 /** 1046 * Set access time of a file. 1047 * @param f The path 1048 * @param mtime Set the modification time of this file. 1049 * The number of milliseconds since epoch (Jan 1, 1970). 1050 * A value of -1 means that this call should not set modification time. 1051 * @param atime Set the access time of this file. 1052 * The number of milliseconds since Jan 1, 1970. 1053 * A value of -1 means that this call should not set access time. 1054 * 1055 * @throws AccessControlException If access is denied 1056 * @throws FileNotFoundException If <code>f</code> does not exist 1057 * @throws UnsupportedFileSystemException If file system for <code>f</code> is 1058 * not supported 1059 * @throws IOException If an I/O error occurred 1060 * 1061 * Exceptions applicable to file systems accessed over RPC: 1062 * @throws RpcClientException If an exception occurred in the RPC client 1063 * @throws RpcServerException If an exception occurred in the RPC server 1064 * @throws UnexpectedServerException If server implementation throws 1065 * undeclared exception to RPC server 1066 */ 1067 public void setTimes(final Path f, final long mtime, final long atime) 1068 throws AccessControlException, FileNotFoundException, 1069 UnsupportedFileSystemException, IOException { 1070 final Path absF = fixRelativePart(f); 1071 new FSLinkResolver<Void>() { 1072 @Override 1073 public Void next(final AbstractFileSystem fs, final Path p) 1074 throws IOException, UnresolvedLinkException { 1075 fs.setTimes(p, mtime, atime); 1076 return null; 1077 } 1078 }.resolve(this, absF); 1079 } 1080 1081 /** 1082 * Get the checksum of a file. 1083 * 1084 * @param f file path 1085 * 1086 * @return The file checksum. The default return value is null, 1087 * which indicates that no checksum algorithm is implemented 1088 * in the corresponding FileSystem. 1089 * 1090 * @throws AccessControlException If access is denied 1091 * @throws FileNotFoundException If <code>f</code> does not exist 1092 * @throws IOException If an I/O error occurred 1093 * 1094 * Exceptions applicable to file systems accessed over RPC: 1095 * @throws RpcClientException If an exception occurred in the RPC client 1096 * @throws RpcServerException If an exception occurred in the RPC server 1097 * @throws UnexpectedServerException If server implementation throws 1098 * undeclared exception to RPC server 1099 */ 1100 public FileChecksum getFileChecksum(final Path f) 1101 throws AccessControlException, FileNotFoundException, 1102 IOException { 1103 final Path absF = fixRelativePart(f); 1104 return new FSLinkResolver<FileChecksum>() { 1105 @Override 1106 public FileChecksum next(final AbstractFileSystem fs, final Path p) 1107 throws IOException, UnresolvedLinkException { 1108 return fs.getFileChecksum(p); 1109 } 1110 }.resolve(this, absF); 1111 } 1112 1113 /** 1114 * Set the verify checksum flag for the file system denoted by the path. 1115 * This is only applicable if the 1116 * corresponding FileSystem supports checksum. By default doesn't do anything. 1117 * @param verifyChecksum 1118 * @param f set the verifyChecksum for the Filesystem containing this path 1119 * 1120 * @throws AccessControlException If access is denied 1121 * @throws FileNotFoundException If <code>f</code> does not exist 1122 * @throws UnsupportedFileSystemException If file system for <code>f</code> is 1123 * not supported 1124 * @throws IOException If an I/O error occurred 1125 * 1126 * Exceptions applicable to file systems accessed over RPC: 1127 * @throws RpcClientException If an exception occurred in the RPC client 1128 * @throws RpcServerException If an exception occurred in the RPC server 1129 * @throws UnexpectedServerException If server implementation throws 1130 * undeclared exception to RPC server 1131 */ 1132 public void setVerifyChecksum(final boolean verifyChecksum, final Path f) 1133 throws AccessControlException, FileNotFoundException, 1134 UnsupportedFileSystemException, IOException { 1135 final Path absF = resolve(fixRelativePart(f)); 1136 getFSofPath(absF).setVerifyChecksum(verifyChecksum); 1137 } 1138 1139 /** 1140 * Return a file status object that represents the path. 1141 * @param f The path we want information from 1142 * 1143 * @return a FileStatus object 1144 * 1145 * @throws AccessControlException If access is denied 1146 * @throws FileNotFoundException If <code>f</code> does not exist 1147 * @throws UnsupportedFileSystemException If file system for <code>f</code> is 1148 * not supported 1149 * @throws IOException If an I/O error occurred 1150 * 1151 * Exceptions applicable to file systems accessed over RPC: 1152 * @throws RpcClientException If an exception occurred in the RPC client 1153 * @throws RpcServerException If an exception occurred in the RPC server 1154 * @throws UnexpectedServerException If server implementation throws 1155 * undeclared exception to RPC server 1156 */ 1157 public FileStatus getFileStatus(final Path f) throws AccessControlException, 1158 FileNotFoundException, UnsupportedFileSystemException, IOException { 1159 final Path absF = fixRelativePart(f); 1160 return new FSLinkResolver<FileStatus>() { 1161 @Override 1162 public FileStatus next(final AbstractFileSystem fs, final Path p) 1163 throws IOException, UnresolvedLinkException { 1164 return fs.getFileStatus(p); 1165 } 1166 }.resolve(this, absF); 1167 } 1168 1169 /** 1170 * Checks if the user can access a path. The mode specifies which access 1171 * checks to perform. If the requested permissions are granted, then the 1172 * method returns normally. If access is denied, then the method throws an 1173 * {@link AccessControlException}. 1174 * <p/> 1175 * The default implementation of this method calls {@link #getFileStatus(Path)} 1176 * and checks the returned permissions against the requested permissions. 1177 * Note that the getFileStatus call will be subject to authorization checks. 1178 * Typically, this requires search (execute) permissions on each directory in 1179 * the path's prefix, but this is implementation-defined. Any file system 1180 * that provides a richer authorization model (such as ACLs) may override the 1181 * default implementation so that it checks against that model instead. 1182 * <p> 1183 * In general, applications should avoid using this method, due to the risk of 1184 * time-of-check/time-of-use race conditions. The permissions on a file may 1185 * change immediately after the access call returns. Most applications should 1186 * prefer running specific file system actions as the desired user represented 1187 * by a {@link UserGroupInformation}. 1188 * 1189 * @param path Path to check 1190 * @param mode type of access to check 1191 * @throws AccessControlException if access is denied 1192 * @throws FileNotFoundException if the path does not exist 1193 * @throws UnsupportedFileSystemException if file system for <code>path</code> 1194 * is not supported 1195 * @throws IOException see specific implementation 1196 * 1197 * Exceptions applicable to file systems accessed over RPC: 1198 * @throws RpcClientException If an exception occurred in the RPC client 1199 * @throws RpcServerException If an exception occurred in the RPC server 1200 * @throws UnexpectedServerException If server implementation throws 1201 * undeclared exception to RPC server 1202 */ 1203 @InterfaceAudience.LimitedPrivate({"HDFS", "Hive"}) 1204 public void access(final Path path, final FsAction mode) 1205 throws AccessControlException, FileNotFoundException, 1206 UnsupportedFileSystemException, IOException { 1207 final Path absPath = fixRelativePart(path); 1208 new FSLinkResolver<Void>() { 1209 @Override 1210 public Void next(AbstractFileSystem fs, Path p) throws IOException, 1211 UnresolvedLinkException { 1212 fs.access(p, mode); 1213 return null; 1214 } 1215 }.resolve(this, absPath); 1216 } 1217 1218 /** 1219 * Return a file status object that represents the path. If the path 1220 * refers to a symlink then the FileStatus of the symlink is returned. 1221 * The behavior is equivalent to #getFileStatus() if the underlying 1222 * file system does not support symbolic links. 1223 * @param f The path we want information from. 1224 * @return A FileStatus object 1225 * 1226 * @throws AccessControlException If access is denied 1227 * @throws FileNotFoundException If <code>f</code> does not exist 1228 * @throws UnsupportedFileSystemException If file system for <code>f</code> is 1229 * not supported 1230 * @throws IOException If an I/O error occurred 1231 */ 1232 public FileStatus getFileLinkStatus(final Path f) 1233 throws AccessControlException, FileNotFoundException, 1234 UnsupportedFileSystemException, IOException { 1235 final Path absF = fixRelativePart(f); 1236 return new FSLinkResolver<FileStatus>() { 1237 @Override 1238 public FileStatus next(final AbstractFileSystem fs, final Path p) 1239 throws IOException, UnresolvedLinkException { 1240 FileStatus fi = fs.getFileLinkStatus(p); 1241 if (fi.isSymlink()) { 1242 fi.setSymlink(FSLinkResolver.qualifySymlinkTarget(fs.getUri(), p, 1243 fi.getSymlink())); 1244 } 1245 return fi; 1246 } 1247 }.resolve(this, absF); 1248 } 1249 1250 /** 1251 * Returns the target of the given symbolic link as it was specified 1252 * when the link was created. Links in the path leading up to the 1253 * final path component are resolved transparently. 1254 * 1255 * @param f the path to return the target of 1256 * @return The un-interpreted target of the symbolic link. 1257 * 1258 * @throws AccessControlException If access is denied 1259 * @throws FileNotFoundException If path <code>f</code> does not exist 1260 * @throws UnsupportedFileSystemException If file system for <code>f</code> is 1261 * not supported 1262 * @throws IOException If the given path does not refer to a symlink 1263 * or an I/O error occurred 1264 */ 1265 public Path getLinkTarget(final Path f) throws AccessControlException, 1266 FileNotFoundException, UnsupportedFileSystemException, IOException { 1267 final Path absF = fixRelativePart(f); 1268 return new FSLinkResolver<Path>() { 1269 @Override 1270 public Path next(final AbstractFileSystem fs, final Path p) 1271 throws IOException, UnresolvedLinkException { 1272 FileStatus fi = fs.getFileLinkStatus(p); 1273 return fi.getSymlink(); 1274 } 1275 }.resolve(this, absF); 1276 } 1277 1278 /** 1279 * Return blockLocation of the given file for the given offset and len. 1280 * For a nonexistent file or regions, null will be returned. 1281 * 1282 * This call is most helpful with DFS, where it returns 1283 * hostnames of machines that contain the given file. 1284 * 1285 * @param f - get blocklocations of this file 1286 * @param start position (byte offset) 1287 * @param len (in bytes) 1288 * 1289 * @return block locations for given file at specified offset of len 1290 * 1291 * @throws AccessControlException If access is denied 1292 * @throws FileNotFoundException If <code>f</code> does not exist 1293 * @throws UnsupportedFileSystemException If file system for <code>f</code> is 1294 * not supported 1295 * @throws IOException If an I/O error occurred 1296 * 1297 * Exceptions applicable to file systems accessed over RPC: 1298 * @throws RpcClientException If an exception occurred in the RPC client 1299 * @throws RpcServerException If an exception occurred in the RPC server 1300 * @throws UnexpectedServerException If server implementation throws 1301 * undeclared exception to RPC server 1302 * 1303 * RuntimeExceptions: 1304 * @throws InvalidPathException If path <code>f</code> is invalid 1305 */ 1306 @InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"}) 1307 @InterfaceStability.Evolving 1308 public BlockLocation[] getFileBlockLocations(final Path f, final long start, 1309 final long len) throws AccessControlException, FileNotFoundException, 1310 UnsupportedFileSystemException, IOException { 1311 final Path absF = fixRelativePart(f); 1312 return new FSLinkResolver<BlockLocation[]>() { 1313 @Override 1314 public BlockLocation[] next(final AbstractFileSystem fs, final Path p) 1315 throws IOException, UnresolvedLinkException { 1316 return fs.getFileBlockLocations(p, start, len); 1317 } 1318 }.resolve(this, absF); 1319 } 1320 1321 /** 1322 * Returns a status object describing the use and capacity of the 1323 * file system denoted by the Parh argument p. 1324 * If the file system has multiple partitions, the 1325 * use and capacity of the partition pointed to by the specified 1326 * path is reflected. 1327 * 1328 * @param f Path for which status should be obtained. null means the 1329 * root partition of the default file system. 1330 * 1331 * @return a FsStatus object 1332 * 1333 * @throws AccessControlException If access is denied 1334 * @throws FileNotFoundException If <code>f</code> does not exist 1335 * @throws UnsupportedFileSystemException If file system for <code>f</code> is 1336 * not supported 1337 * @throws IOException If an I/O error occurred 1338 * 1339 * Exceptions applicable to file systems accessed over RPC: 1340 * @throws RpcClientException If an exception occurred in the RPC client 1341 * @throws RpcServerException If an exception occurred in the RPC server 1342 * @throws UnexpectedServerException If server implementation throws 1343 * undeclared exception to RPC server 1344 */ 1345 public FsStatus getFsStatus(final Path f) throws AccessControlException, 1346 FileNotFoundException, UnsupportedFileSystemException, IOException { 1347 if (f == null) { 1348 return defaultFS.getFsStatus(); 1349 } 1350 final Path absF = fixRelativePart(f); 1351 return new FSLinkResolver<FsStatus>() { 1352 @Override 1353 public FsStatus next(final AbstractFileSystem fs, final Path p) 1354 throws IOException, UnresolvedLinkException { 1355 return fs.getFsStatus(p); 1356 } 1357 }.resolve(this, absF); 1358 } 1359 1360 /** 1361 * Creates a symbolic link to an existing file. An exception is thrown if 1362 * the symlink exits, the user does not have permission to create symlink, 1363 * or the underlying file system does not support symlinks. 1364 * 1365 * Symlink permissions are ignored, access to a symlink is determined by 1366 * the permissions of the symlink target. 1367 * 1368 * Symlinks in paths leading up to the final path component are resolved 1369 * transparently. If the final path component refers to a symlink some 1370 * functions operate on the symlink itself, these are: 1371 * - delete(f) and deleteOnExit(f) - Deletes the symlink. 1372 * - rename(src, dst) - If src refers to a symlink, the symlink is 1373 * renamed. If dst refers to a symlink, the symlink is over-written. 1374 * - getLinkTarget(f) - Returns the target of the symlink. 1375 * - getFileLinkStatus(f) - Returns a FileStatus object describing 1376 * the symlink. 1377 * Some functions, create() and mkdir(), expect the final path component 1378 * does not exist. If they are given a path that refers to a symlink that 1379 * does exist they behave as if the path referred to an existing file or 1380 * directory. All other functions fully resolve, ie follow, the symlink. 1381 * These are: open, setReplication, setOwner, setTimes, setWorkingDirectory, 1382 * setPermission, getFileChecksum, setVerifyChecksum, getFileBlockLocations, 1383 * getFsStatus, getFileStatus, exists, and listStatus. 1384 * 1385 * Symlink targets are stored as given to createSymlink, assuming the 1386 * underlying file system is capable of storing a fully qualified URI. 1387 * Dangling symlinks are permitted. FileContext supports four types of 1388 * symlink targets, and resolves them as follows 1389 * <pre> 1390 * Given a path referring to a symlink of form: 1391 * 1392 * <---X---> 1393 * fs://host/A/B/link 1394 * <-----Y-----> 1395 * 1396 * In this path X is the scheme and authority that identify the file system, 1397 * and Y is the path leading up to the final path component "link". If Y is 1398 * a symlink itself then let Y' be the target of Y and X' be the scheme and 1399 * authority of Y'. Symlink targets may: 1400 * 1401 * 1. Fully qualified URIs 1402 * 1403 * fs://hostX/A/B/file Resolved according to the target file system. 1404 * 1405 * 2. Partially qualified URIs (eg scheme but no host) 1406 * 1407 * fs:///A/B/file Resolved according to the target file system. Eg resolving 1408 * a symlink to hdfs:///A results in an exception because 1409 * HDFS URIs must be fully qualified, while a symlink to 1410 * file:///A will not since Hadoop's local file systems 1411 * require partially qualified URIs. 1412 * 1413 * 3. Relative paths 1414 * 1415 * path Resolves to [Y'][path]. Eg if Y resolves to hdfs://host/A and path 1416 * is "../B/file" then [Y'][path] is hdfs://host/B/file 1417 * 1418 * 4. Absolute paths 1419 * 1420 * path Resolves to [X'][path]. Eg if Y resolves hdfs://host/A/B and path 1421 * is "/file" then [X][path] is hdfs://host/file 1422 * </pre> 1423 * 1424 * @param target the target of the symbolic link 1425 * @param link the path to be created that points to target 1426 * @param createParent if true then missing parent dirs are created if 1427 * false then parent must exist 1428 * 1429 * 1430 * @throws AccessControlException If access is denied 1431 * @throws FileAlreadyExistsException If file <code>linkcode> already exists 1432 * @throws FileNotFoundException If <code>target</code> does not exist 1433 * @throws ParentNotDirectoryException If parent of <code>link</code> is not a 1434 * directory. 1435 * @throws UnsupportedFileSystemException If file system for 1436 * <code>target</code> or <code>link</code> is not supported 1437 * @throws IOException If an I/O error occurred 1438 */ 1439 @SuppressWarnings("deprecation") 1440 public void createSymlink(final Path target, final Path link, 1441 final boolean createParent) throws AccessControlException, 1442 FileAlreadyExistsException, FileNotFoundException, 1443 ParentNotDirectoryException, UnsupportedFileSystemException, 1444 IOException { 1445 if (!FileSystem.areSymlinksEnabled()) { 1446 throw new UnsupportedOperationException("Symlinks not supported"); 1447 } 1448 final Path nonRelLink = fixRelativePart(link); 1449 new FSLinkResolver<Void>() { 1450 @Override 1451 public Void next(final AbstractFileSystem fs, final Path p) 1452 throws IOException, UnresolvedLinkException { 1453 fs.createSymlink(target, p, createParent); 1454 return null; 1455 } 1456 }.resolve(this, nonRelLink); 1457 } 1458 1459 /** 1460 * List the statuses of the files/directories in the given path if the path is 1461 * a directory. 1462 * 1463 * @param f is the path 1464 * 1465 * @return an iterator that traverses statuses of the files/directories 1466 * in the given path 1467 * 1468 * @throws AccessControlException If access is denied 1469 * @throws FileNotFoundException If <code>f</code> does not exist 1470 * @throws UnsupportedFileSystemException If file system for <code>f</code> is 1471 * not supported 1472 * @throws IOException If an I/O error occurred 1473 * 1474 * Exceptions applicable to file systems accessed over RPC: 1475 * @throws RpcClientException If an exception occurred in the RPC client 1476 * @throws RpcServerException If an exception occurred in the RPC server 1477 * @throws UnexpectedServerException If server implementation throws 1478 * undeclared exception to RPC server 1479 */ 1480 public RemoteIterator<FileStatus> listStatus(final Path f) throws 1481 AccessControlException, FileNotFoundException, 1482 UnsupportedFileSystemException, IOException { 1483 final Path absF = fixRelativePart(f); 1484 return new FSLinkResolver<RemoteIterator<FileStatus>>() { 1485 @Override 1486 public RemoteIterator<FileStatus> next( 1487 final AbstractFileSystem fs, final Path p) 1488 throws IOException, UnresolvedLinkException { 1489 return fs.listStatusIterator(p); 1490 } 1491 }.resolve(this, absF); 1492 } 1493 1494 /** 1495 * @return an iterator over the corrupt files under the given path 1496 * (may contain duplicates if a file has more than one corrupt block) 1497 * @throws IOException 1498 */ 1499 public RemoteIterator<Path> listCorruptFileBlocks(Path path) 1500 throws IOException { 1501 final Path absF = fixRelativePart(path); 1502 return new FSLinkResolver<RemoteIterator<Path>>() { 1503 @Override 1504 public RemoteIterator<Path> next(final AbstractFileSystem fs, 1505 final Path p) 1506 throws IOException, UnresolvedLinkException { 1507 return fs.listCorruptFileBlocks(p); 1508 } 1509 }.resolve(this, absF); 1510 } 1511 1512 /** 1513 * List the statuses of the files/directories in the given path if the path is 1514 * a directory. 1515 * Return the file's status and block locations If the path is a file. 1516 * 1517 * If a returned status is a file, it contains the file's block locations. 1518 * 1519 * @param f is the path 1520 * 1521 * @return an iterator that traverses statuses of the files/directories 1522 * in the given path 1523 * If any IO exception (for example the input directory gets deleted while 1524 * listing is being executed), next() or hasNext() of the returned iterator 1525 * may throw a RuntimeException with the io exception as the cause. 1526 * 1527 * @throws AccessControlException If access is denied 1528 * @throws FileNotFoundException If <code>f</code> does not exist 1529 * @throws UnsupportedFileSystemException If file system for <code>f</code> is 1530 * not supported 1531 * @throws IOException If an I/O error occurred 1532 * 1533 * Exceptions applicable to file systems accessed over RPC: 1534 * @throws RpcClientException If an exception occurred in the RPC client 1535 * @throws RpcServerException If an exception occurred in the RPC server 1536 * @throws UnexpectedServerException If server implementation throws 1537 * undeclared exception to RPC server 1538 */ 1539 public RemoteIterator<LocatedFileStatus> listLocatedStatus( 1540 final Path f) throws 1541 AccessControlException, FileNotFoundException, 1542 UnsupportedFileSystemException, IOException { 1543 final Path absF = fixRelativePart(f); 1544 return new FSLinkResolver<RemoteIterator<LocatedFileStatus>>() { 1545 @Override 1546 public RemoteIterator<LocatedFileStatus> next( 1547 final AbstractFileSystem fs, final Path p) 1548 throws IOException, UnresolvedLinkException { 1549 return fs.listLocatedStatus(p); 1550 } 1551 }.resolve(this, absF); 1552 } 1553 1554 /** 1555 * Mark a path to be deleted on JVM shutdown. 1556 * 1557 * @param f the existing path to delete. 1558 * 1559 * @return true if deleteOnExit is successful, otherwise false. 1560 * 1561 * @throws AccessControlException If access is denied 1562 * @throws UnsupportedFileSystemException If file system for <code>f</code> is 1563 * not supported 1564 * @throws IOException If an I/O error occurred 1565 * 1566 * Exceptions applicable to file systems accessed over RPC: 1567 * @throws RpcClientException If an exception occurred in the RPC client 1568 * @throws RpcServerException If an exception occurred in the RPC server 1569 * @throws UnexpectedServerException If server implementation throws 1570 * undeclared exception to RPC server 1571 */ 1572 public boolean deleteOnExit(Path f) throws AccessControlException, 1573 IOException { 1574 if (!this.util().exists(f)) { 1575 return false; 1576 } 1577 synchronized (DELETE_ON_EXIT) { 1578 if (DELETE_ON_EXIT.isEmpty()) { 1579 ShutdownHookManager.get().addShutdownHook(FINALIZER, SHUTDOWN_HOOK_PRIORITY); 1580 } 1581 1582 Set<Path> set = DELETE_ON_EXIT.get(this); 1583 if (set == null) { 1584 set = new TreeSet<Path>(); 1585 DELETE_ON_EXIT.put(this, set); 1586 } 1587 set.add(f); 1588 } 1589 return true; 1590 } 1591 1592 private final Util util; 1593 public Util util() { 1594 return util; 1595 } 1596 1597 1598 /** 1599 * Utility/library methods built over the basic FileContext methods. 1600 * Since this are library functions, the oprtation are not atomic 1601 * and some of them may partially complete if other threads are making 1602 * changes to the same part of the name space. 1603 */ 1604 public class Util { 1605 /** 1606 * Does the file exist? 1607 * Note: Avoid using this method if you already have FileStatus in hand. 1608 * Instead reuse the FileStatus 1609 * @param f the file or dir to be checked 1610 * 1611 * @throws AccessControlException If access is denied 1612 * @throws IOException If an I/O error occurred 1613 * @throws UnsupportedFileSystemException If file system for <code>f</code> is 1614 * not supported 1615 * 1616 * Exceptions applicable to file systems accessed over RPC: 1617 * @throws RpcClientException If an exception occurred in the RPC client 1618 * @throws RpcServerException If an exception occurred in the RPC server 1619 * @throws UnexpectedServerException If server implementation throws 1620 * undeclared exception to RPC server 1621 */ 1622 public boolean exists(final Path f) throws AccessControlException, 1623 UnsupportedFileSystemException, IOException { 1624 try { 1625 FileStatus fs = FileContext.this.getFileStatus(f); 1626 assert fs != null; 1627 return true; 1628 } catch (FileNotFoundException e) { 1629 return false; 1630 } 1631 } 1632 1633 /** 1634 * Return the {@link ContentSummary} of path f. 1635 * @param f path 1636 * 1637 * @return the {@link ContentSummary} of path f. 1638 * 1639 * @throws AccessControlException If access is denied 1640 * @throws FileNotFoundException If <code>f</code> does not exist 1641 * @throws UnsupportedFileSystemException If file system for 1642 * <code>f</code> is not supported 1643 * @throws IOException If an I/O error occurred 1644 * 1645 * Exceptions applicable to file systems accessed over RPC: 1646 * @throws RpcClientException If an exception occurred in the RPC client 1647 * @throws RpcServerException If an exception occurred in the RPC server 1648 * @throws UnexpectedServerException If server implementation throws 1649 * undeclared exception to RPC server 1650 */ 1651 public ContentSummary getContentSummary(Path f) 1652 throws AccessControlException, FileNotFoundException, 1653 UnsupportedFileSystemException, IOException { 1654 FileStatus status = FileContext.this.getFileStatus(f); 1655 if (status.isFile()) { 1656 long length = status.getLen(); 1657 return new ContentSummary.Builder().length(length). 1658 fileCount(1).directoryCount(0).spaceConsumed(length). 1659 build(); 1660 } 1661 long[] summary = {0, 0, 1}; 1662 RemoteIterator<FileStatus> statusIterator = 1663 FileContext.this.listStatus(f); 1664 while(statusIterator.hasNext()) { 1665 FileStatus s = statusIterator.next(); 1666 long length = s.getLen(); 1667 ContentSummary c = s.isDirectory() ? getContentSummary(s.getPath()) : 1668 new ContentSummary.Builder().length(length).fileCount(1). 1669 directoryCount(0).spaceConsumed(length).build(); 1670 summary[0] += c.getLength(); 1671 summary[1] += c.getFileCount(); 1672 summary[2] += c.getDirectoryCount(); 1673 } 1674 return new ContentSummary.Builder().length(summary[0]). 1675 fileCount(summary[1]).directoryCount(summary[2]). 1676 spaceConsumed(summary[0]).build(); 1677 } 1678 1679 /** 1680 * See {@link #listStatus(Path[], PathFilter)} 1681 */ 1682 public FileStatus[] listStatus(Path[] files) throws AccessControlException, 1683 FileNotFoundException, IOException { 1684 return listStatus(files, DEFAULT_FILTER); 1685 } 1686 1687 /** 1688 * Filter files/directories in the given path using the user-supplied path 1689 * filter. 1690 * 1691 * @param f is the path name 1692 * @param filter is the user-supplied path filter 1693 * 1694 * @return an array of FileStatus objects for the files under the given path 1695 * after applying the filter 1696 * 1697 * @throws AccessControlException If access is denied 1698 * @throws FileNotFoundException If <code>f</code> does not exist 1699 * @throws UnsupportedFileSystemException If file system for 1700 * <code>pathPattern</code> is not supported 1701 * @throws IOException If an I/O error occurred 1702 * 1703 * Exceptions applicable to file systems accessed over RPC: 1704 * @throws RpcClientException If an exception occurred in the RPC client 1705 * @throws RpcServerException If an exception occurred in the RPC server 1706 * @throws UnexpectedServerException If server implementation throws 1707 * undeclared exception to RPC server 1708 */ 1709 public FileStatus[] listStatus(Path f, PathFilter filter) 1710 throws AccessControlException, FileNotFoundException, 1711 UnsupportedFileSystemException, IOException { 1712 ArrayList<FileStatus> results = new ArrayList<FileStatus>(); 1713 listStatus(results, f, filter); 1714 return results.toArray(new FileStatus[results.size()]); 1715 } 1716 1717 /** 1718 * Filter files/directories in the given list of paths using user-supplied 1719 * path filter. 1720 * 1721 * @param files is a list of paths 1722 * @param filter is the filter 1723 * 1724 * @return a list of statuses for the files under the given paths after 1725 * applying the filter 1726 * 1727 * @throws AccessControlException If access is denied 1728 * @throws FileNotFoundException If a file in <code>files</code> does not 1729 * exist 1730 * @throws IOException If an I/O error occurred 1731 * 1732 * Exceptions applicable to file systems accessed over RPC: 1733 * @throws RpcClientException If an exception occurred in the RPC client 1734 * @throws RpcServerException If an exception occurred in the RPC server 1735 * @throws UnexpectedServerException If server implementation throws 1736 * undeclared exception to RPC server 1737 */ 1738 public FileStatus[] listStatus(Path[] files, PathFilter filter) 1739 throws AccessControlException, FileNotFoundException, IOException { 1740 ArrayList<FileStatus> results = new ArrayList<FileStatus>(); 1741 for (int i = 0; i < files.length; i++) { 1742 listStatus(results, files[i], filter); 1743 } 1744 return results.toArray(new FileStatus[results.size()]); 1745 } 1746 1747 /* 1748 * Filter files/directories in the given path using the user-supplied path 1749 * filter. Results are added to the given array <code>results</code>. 1750 */ 1751 private void listStatus(ArrayList<FileStatus> results, Path f, 1752 PathFilter filter) throws AccessControlException, 1753 FileNotFoundException, IOException { 1754 FileStatus[] listing = listStatus(f); 1755 if (listing != null) { 1756 for (int i = 0; i < listing.length; i++) { 1757 if (filter.accept(listing[i].getPath())) { 1758 results.add(listing[i]); 1759 } 1760 } 1761 } 1762 } 1763 1764 /** 1765 * List the statuses of the files/directories in the given path 1766 * if the path is a directory. 1767 * 1768 * @param f is the path 1769 * 1770 * @return an array that contains statuses of the files/directories 1771 * in the given path 1772 * 1773 * @throws AccessControlException If access is denied 1774 * @throws FileNotFoundException If <code>f</code> does not exist 1775 * @throws UnsupportedFileSystemException If file system for <code>f</code> is 1776 * not supported 1777 * @throws IOException If an I/O error occurred 1778 * 1779 * Exceptions applicable to file systems accessed over RPC: 1780 * @throws RpcClientException If an exception occurred in the RPC client 1781 * @throws RpcServerException If an exception occurred in the RPC server 1782 * @throws UnexpectedServerException If server implementation throws 1783 * undeclared exception to RPC server 1784 */ 1785 public FileStatus[] listStatus(final Path f) throws AccessControlException, 1786 FileNotFoundException, UnsupportedFileSystemException, 1787 IOException { 1788 final Path absF = fixRelativePart(f); 1789 return new FSLinkResolver<FileStatus[]>() { 1790 @Override 1791 public FileStatus[] next(final AbstractFileSystem fs, final Path p) 1792 throws IOException, UnresolvedLinkException { 1793 return fs.listStatus(p); 1794 } 1795 }.resolve(FileContext.this, absF); 1796 } 1797 1798 /** 1799 * List the statuses and block locations of the files in the given path. 1800 * 1801 * If the path is a directory, 1802 * if recursive is false, returns files in the directory; 1803 * if recursive is true, return files in the subtree rooted at the path. 1804 * The subtree is traversed in the depth-first order. 1805 * If the path is a file, return the file's status and block locations. 1806 * Files across symbolic links are also returned. 1807 * 1808 * @param f is the path 1809 * @param recursive if the subdirectories need to be traversed recursively 1810 * 1811 * @return an iterator that traverses statuses of the files 1812 * If any IO exception (for example a sub-directory gets deleted while 1813 * listing is being executed), next() or hasNext() of the returned iterator 1814 * may throw a RuntimeException with the IO exception as the cause. 1815 * 1816 * @throws AccessControlException If access is denied 1817 * @throws FileNotFoundException If <code>f</code> does not exist 1818 * @throws UnsupportedFileSystemException If file system for <code>f</code> 1819 * is not supported 1820 * @throws IOException If an I/O error occurred 1821 * 1822 * Exceptions applicable to file systems accessed over RPC: 1823 * @throws RpcClientException If an exception occurred in the RPC client 1824 * @throws RpcServerException If an exception occurred in the RPC server 1825 * @throws UnexpectedServerException If server implementation throws 1826 * undeclared exception to RPC server 1827 */ 1828 public RemoteIterator<LocatedFileStatus> listFiles( 1829 final Path f, final boolean recursive) throws AccessControlException, 1830 FileNotFoundException, UnsupportedFileSystemException, 1831 IOException { 1832 return new RemoteIterator<LocatedFileStatus>() { 1833 private Stack<RemoteIterator<LocatedFileStatus>> itors = 1834 new Stack<RemoteIterator<LocatedFileStatus>>(); 1835 RemoteIterator<LocatedFileStatus> curItor = listLocatedStatus(f); 1836 LocatedFileStatus curFile; 1837 1838 /** 1839 * Returns <tt>true</tt> if the iterator has more files. 1840 * 1841 * @return <tt>true</tt> if the iterator has more files. 1842 * @throws AccessControlException if not allowed to access next 1843 * file's status or locations 1844 * @throws FileNotFoundException if next file does not exist any more 1845 * @throws UnsupportedFileSystemException if next file's 1846 * fs is unsupported 1847 * @throws IOException for all other IO errors 1848 * for example, NameNode is not avaialbe or 1849 * NameNode throws IOException due to an error 1850 * while getting the status or block locations 1851 */ 1852 @Override 1853 public boolean hasNext() throws IOException { 1854 while (curFile == null) { 1855 if (curItor.hasNext()) { 1856 handleFileStat(curItor.next()); 1857 } else if (!itors.empty()) { 1858 curItor = itors.pop(); 1859 } else { 1860 return false; 1861 } 1862 } 1863 return true; 1864 } 1865 1866 /** 1867 * Process the input stat. 1868 * If it is a file, return the file stat. 1869 * If it is a directory, traverse the directory if recursive is true; 1870 * ignore it if recursive is false. 1871 * If it is a symlink, resolve the symlink first and then process it 1872 * depending on if it is a file or directory. 1873 * @param stat input status 1874 * @throws AccessControlException if access is denied 1875 * @throws FileNotFoundException if file is not found 1876 * @throws UnsupportedFileSystemException if fs is not supported 1877 * @throws IOException for all other IO errors 1878 */ 1879 private void handleFileStat(LocatedFileStatus stat) 1880 throws IOException { 1881 if (stat.isFile()) { // file 1882 curFile = stat; 1883 } else if (stat.isSymlink()) { // symbolic link 1884 // resolve symbolic link 1885 FileStatus symstat = FileContext.this.getFileStatus( 1886 stat.getSymlink()); 1887 if (symstat.isFile() || (recursive && symstat.isDirectory())) { 1888 itors.push(curItor); 1889 curItor = listLocatedStatus(stat.getPath()); 1890 } 1891 } else if (recursive) { // directory 1892 itors.push(curItor); 1893 curItor = listLocatedStatus(stat.getPath()); 1894 } 1895 } 1896 1897 /** 1898 * Returns the next file's status with its block locations 1899 * 1900 * @throws AccessControlException if not allowed to access next 1901 * file's status or locations 1902 * @throws FileNotFoundException if next file does not exist any more 1903 * @throws UnsupportedFileSystemException if next file's 1904 * fs is unsupported 1905 * @throws IOException for all other IO errors 1906 * for example, NameNode is not avaialbe or 1907 * NameNode throws IOException due to an error 1908 * while getting the status or block locations 1909 */ 1910 @Override 1911 public LocatedFileStatus next() throws IOException { 1912 if (hasNext()) { 1913 LocatedFileStatus result = curFile; 1914 curFile = null; 1915 return result; 1916 } 1917 throw new java.util.NoSuchElementException("No more entry in " + f); 1918 } 1919 }; 1920 } 1921 1922 /** 1923 * <p>Return all the files that match filePattern and are not checksum 1924 * files. Results are sorted by their names. 1925 * 1926 * <p> 1927 * A filename pattern is composed of <i>regular</i> characters and 1928 * <i>special pattern matching</i> characters, which are: 1929 * 1930 * <dl> 1931 * <dd> 1932 * <dl> 1933 * <p> 1934 * <dt> <tt> ? </tt> 1935 * <dd> Matches any single character. 1936 * 1937 * <p> 1938 * <dt> <tt> * </tt> 1939 * <dd> Matches zero or more characters. 1940 * 1941 * <p> 1942 * <dt> <tt> [<i>abc</i>] </tt> 1943 * <dd> Matches a single character from character set 1944 * <tt>{<i>a,b,c</i>}</tt>. 1945 * 1946 * <p> 1947 * <dt> <tt> [<i>a</i>-<i>b</i>] </tt> 1948 * <dd> Matches a single character from the character range 1949 * <tt>{<i>a...b</i>}</tt>. Note: character <tt><i>a</i></tt> must be 1950 * lexicographically less than or equal to character <tt><i>b</i></tt>. 1951 * 1952 * <p> 1953 * <dt> <tt> [^<i>a</i>] </tt> 1954 * <dd> Matches a single char that is not from character set or range 1955 * <tt>{<i>a</i>}</tt>. Note that the <tt>^</tt> character must occur 1956 * immediately to the right of the opening bracket. 1957 * 1958 * <p> 1959 * <dt> <tt> \<i>c</i> </tt> 1960 * <dd> Removes (escapes) any special meaning of character <i>c</i>. 1961 * 1962 * <p> 1963 * <dt> <tt> {ab,cd} </tt> 1964 * <dd> Matches a string from the string set <tt>{<i>ab, cd</i>} </tt> 1965 * 1966 * <p> 1967 * <dt> <tt> {ab,c{de,fh}} </tt> 1968 * <dd> Matches a string from string set <tt>{<i>ab, cde, cfh</i>}</tt> 1969 * 1970 * </dl> 1971 * </dd> 1972 * </dl> 1973 * 1974 * @param pathPattern a regular expression specifying a pth pattern 1975 * 1976 * @return an array of paths that match the path pattern 1977 * 1978 * @throws AccessControlException If access is denied 1979 * @throws UnsupportedFileSystemException If file system for 1980 * <code>pathPattern</code> is not supported 1981 * @throws IOException If an I/O error occurred 1982 * 1983 * Exceptions applicable to file systems accessed over RPC: 1984 * @throws RpcClientException If an exception occurred in the RPC client 1985 * @throws RpcServerException If an exception occurred in the RPC server 1986 * @throws UnexpectedServerException If server implementation throws 1987 * undeclared exception to RPC server 1988 */ 1989 public FileStatus[] globStatus(Path pathPattern) 1990 throws AccessControlException, UnsupportedFileSystemException, 1991 IOException { 1992 return new Globber(FileContext.this, pathPattern, DEFAULT_FILTER).glob(); 1993 } 1994 1995 /** 1996 * Return an array of FileStatus objects whose path names match pathPattern 1997 * and is accepted by the user-supplied path filter. Results are sorted by 1998 * their path names. 1999 * Return null if pathPattern has no glob and the path does not exist. 2000 * Return an empty array if pathPattern has a glob and no path matches it. 2001 * 2002 * @param pathPattern regular expression specifying the path pattern 2003 * @param filter user-supplied path filter 2004 * 2005 * @return an array of FileStatus objects 2006 * 2007 * @throws AccessControlException If access is denied 2008 * @throws UnsupportedFileSystemException If file system for 2009 * <code>pathPattern</code> is not supported 2010 * @throws IOException If an I/O error occurred 2011 * 2012 * Exceptions applicable to file systems accessed over RPC: 2013 * @throws RpcClientException If an exception occurred in the RPC client 2014 * @throws RpcServerException If an exception occurred in the RPC server 2015 * @throws UnexpectedServerException If server implementation throws 2016 * undeclared exception to RPC server 2017 */ 2018 public FileStatus[] globStatus(final Path pathPattern, 2019 final PathFilter filter) throws AccessControlException, 2020 UnsupportedFileSystemException, IOException { 2021 return new Globber(FileContext.this, pathPattern, filter).glob(); 2022 } 2023 2024 /** 2025 * Copy file from src to dest. See 2026 * {@link #copy(Path, Path, boolean, boolean)} 2027 */ 2028 public boolean copy(final Path src, final Path dst) 2029 throws AccessControlException, FileAlreadyExistsException, 2030 FileNotFoundException, ParentNotDirectoryException, 2031 UnsupportedFileSystemException, IOException { 2032 return copy(src, dst, false, false); 2033 } 2034 2035 /** 2036 * Copy from src to dst, optionally deleting src and overwriting dst. 2037 * @param src 2038 * @param dst 2039 * @param deleteSource - delete src if true 2040 * @param overwrite overwrite dst if true; throw IOException if dst exists 2041 * and overwrite is false. 2042 * 2043 * @return true if copy is successful 2044 * 2045 * @throws AccessControlException If access is denied 2046 * @throws FileAlreadyExistsException If <code>dst</code> already exists 2047 * @throws FileNotFoundException If <code>src</code> does not exist 2048 * @throws ParentNotDirectoryException If parent of <code>dst</code> is not 2049 * a directory 2050 * @throws UnsupportedFileSystemException If file system for 2051 * <code>src</code> or <code>dst</code> is not supported 2052 * @throws IOException If an I/O error occurred 2053 * 2054 * Exceptions applicable to file systems accessed over RPC: 2055 * @throws RpcClientException If an exception occurred in the RPC client 2056 * @throws RpcServerException If an exception occurred in the RPC server 2057 * @throws UnexpectedServerException If server implementation throws 2058 * undeclared exception to RPC server 2059 * 2060 * RuntimeExceptions: 2061 * @throws InvalidPathException If path <code>dst</code> is invalid 2062 */ 2063 public boolean copy(final Path src, final Path dst, boolean deleteSource, 2064 boolean overwrite) throws AccessControlException, 2065 FileAlreadyExistsException, FileNotFoundException, 2066 ParentNotDirectoryException, UnsupportedFileSystemException, 2067 IOException { 2068 src.checkNotSchemeWithRelative(); 2069 dst.checkNotSchemeWithRelative(); 2070 Path qSrc = makeQualified(src); 2071 Path qDst = makeQualified(dst); 2072 checkDest(qSrc.getName(), qDst, overwrite); 2073 FileStatus fs = FileContext.this.getFileStatus(qSrc); 2074 if (fs.isDirectory()) { 2075 checkDependencies(qSrc, qDst); 2076 mkdir(qDst, FsPermission.getDirDefault(), true); 2077 FileStatus[] contents = listStatus(qSrc); 2078 for (FileStatus content : contents) { 2079 copy(makeQualified(content.getPath()), makeQualified(new Path(qDst, 2080 content.getPath().getName())), deleteSource, overwrite); 2081 } 2082 } else { 2083 InputStream in=null; 2084 OutputStream out = null; 2085 try { 2086 in = open(qSrc); 2087 EnumSet<CreateFlag> createFlag = overwrite ? EnumSet.of( 2088 CreateFlag.CREATE, CreateFlag.OVERWRITE) : 2089 EnumSet.of(CreateFlag.CREATE); 2090 out = create(qDst, createFlag); 2091 IOUtils.copyBytes(in, out, conf, true); 2092 } finally { 2093 IOUtils.closeStream(out); 2094 IOUtils.closeStream(in); 2095 } 2096 } 2097 if (deleteSource) { 2098 return delete(qSrc, true); 2099 } else { 2100 return true; 2101 } 2102 } 2103 } 2104 2105 /** 2106 * Check if copying srcName to dst would overwrite an existing 2107 * file or directory. 2108 * @param srcName File or directory to be copied. 2109 * @param dst Destination to copy srcName to. 2110 * @param overwrite Whether it's ok to overwrite an existing file. 2111 * @throws AccessControlException If access is denied. 2112 * @throws IOException If dst is an existing directory, or dst is an 2113 * existing file and the overwrite option is not passed. 2114 */ 2115 private void checkDest(String srcName, Path dst, boolean overwrite) 2116 throws AccessControlException, IOException { 2117 try { 2118 FileStatus dstFs = getFileStatus(dst); 2119 if (dstFs.isDirectory()) { 2120 if (null == srcName) { 2121 throw new IOException("Target " + dst + " is a directory"); 2122 } 2123 // Recurse to check if dst/srcName exists. 2124 checkDest(null, new Path(dst, srcName), overwrite); 2125 } else if (!overwrite) { 2126 throw new IOException("Target " + new Path(dst, srcName) 2127 + " already exists"); 2128 } 2129 } catch (FileNotFoundException e) { 2130 // dst does not exist - OK to copy. 2131 } 2132 } 2133 2134 // 2135 // If the destination is a subdirectory of the source, then 2136 // generate exception 2137 // 2138 private static void checkDependencies(Path qualSrc, Path qualDst) 2139 throws IOException { 2140 if (isSameFS(qualSrc, qualDst)) { 2141 String srcq = qualSrc.toString() + Path.SEPARATOR; 2142 String dstq = qualDst.toString() + Path.SEPARATOR; 2143 if (dstq.startsWith(srcq)) { 2144 if (srcq.length() == dstq.length()) { 2145 throw new IOException("Cannot copy " + qualSrc + " to itself."); 2146 } else { 2147 throw new IOException("Cannot copy " + qualSrc + 2148 " to its subdirectory " + qualDst); 2149 } 2150 } 2151 } 2152 } 2153 2154 /** 2155 * Are qualSrc and qualDst of the same file system? 2156 * @param qualPath1 - fully qualified path 2157 * @param qualPath2 - fully qualified path 2158 * @return 2159 */ 2160 private static boolean isSameFS(Path qualPath1, Path qualPath2) { 2161 URI srcUri = qualPath1.toUri(); 2162 URI dstUri = qualPath2.toUri(); 2163 return (srcUri.getScheme().equals(dstUri.getScheme()) && 2164 !(srcUri.getAuthority() != null && dstUri.getAuthority() != null && srcUri 2165 .getAuthority().equals(dstUri.getAuthority()))); 2166 } 2167 2168 /** 2169 * Deletes all the paths in deleteOnExit on JVM shutdown. 2170 */ 2171 static class FileContextFinalizer implements Runnable { 2172 @Override 2173 public synchronized void run() { 2174 processDeleteOnExit(); 2175 } 2176 } 2177 2178 /** 2179 * Resolves all symbolic links in the specified path. 2180 * Returns the new path object. 2181 */ 2182 protected Path resolve(final Path f) throws FileNotFoundException, 2183 UnresolvedLinkException, AccessControlException, IOException { 2184 return new FSLinkResolver<Path>() { 2185 @Override 2186 public Path next(final AbstractFileSystem fs, final Path p) 2187 throws IOException, UnresolvedLinkException { 2188 return fs.resolvePath(p); 2189 } 2190 }.resolve(this, f); 2191 } 2192 2193 /** 2194 * Resolves all symbolic links in the specified path leading up 2195 * to, but not including the final path component. 2196 * @param f path to resolve 2197 * @return the new path object. 2198 */ 2199 protected Path resolveIntermediate(final Path f) throws IOException { 2200 return new FSLinkResolver<FileStatus>() { 2201 @Override 2202 public FileStatus next(final AbstractFileSystem fs, final Path p) 2203 throws IOException, UnresolvedLinkException { 2204 return fs.getFileLinkStatus(p); 2205 } 2206 }.resolve(this, f).getPath(); 2207 } 2208 2209 /** 2210 * Returns the list of AbstractFileSystems accessed in the path. The list may 2211 * contain more than one AbstractFileSystems objects in case of symlinks. 2212 * 2213 * @param f 2214 * Path which needs to be resolved 2215 * @return List of AbstractFileSystems accessed in the path 2216 * @throws IOException 2217 */ 2218 Set<AbstractFileSystem> resolveAbstractFileSystems(final Path f) 2219 throws IOException { 2220 final Path absF = fixRelativePart(f); 2221 final HashSet<AbstractFileSystem> result 2222 = new HashSet<AbstractFileSystem>(); 2223 new FSLinkResolver<Void>() { 2224 @Override 2225 public Void next(final AbstractFileSystem fs, final Path p) 2226 throws IOException, UnresolvedLinkException { 2227 result.add(fs); 2228 fs.getFileStatus(p); 2229 return null; 2230 } 2231 }.resolve(this, absF); 2232 return result; 2233 } 2234 2235 /** 2236 * Get the statistics for a particular file system 2237 * 2238 * @param uri 2239 * the uri to lookup the statistics. Only scheme and authority part 2240 * of the uri are used as the key to store and lookup. 2241 * @return a statistics object 2242 */ 2243 public static Statistics getStatistics(URI uri) { 2244 return AbstractFileSystem.getStatistics(uri); 2245 } 2246 2247 /** 2248 * Clears all the statistics stored in AbstractFileSystem, for all the file 2249 * systems. 2250 */ 2251 public static void clearStatistics() { 2252 AbstractFileSystem.clearStatistics(); 2253 } 2254 2255 /** 2256 * Prints the statistics to standard output. File System is identified by the 2257 * scheme and authority. 2258 */ 2259 public static void printStatistics() { 2260 AbstractFileSystem.printStatistics(); 2261 } 2262 2263 /** 2264 * @return Map of uri and statistics for each filesystem instantiated. The uri 2265 * consists of scheme and authority for the filesystem. 2266 */ 2267 public static Map<URI, Statistics> getAllStatistics() { 2268 return AbstractFileSystem.getAllStatistics(); 2269 } 2270 2271 /** 2272 * Get delegation tokens for the file systems accessed for a given 2273 * path. 2274 * @param p Path for which delegations tokens are requested. 2275 * @param renewer the account name that is allowed to renew the token. 2276 * @return List of delegation tokens. 2277 * @throws IOException 2278 */ 2279 @InterfaceAudience.LimitedPrivate( { "HDFS", "MapReduce" }) 2280 public List<Token<?>> getDelegationTokens( 2281 Path p, String renewer) throws IOException { 2282 Set<AbstractFileSystem> afsSet = resolveAbstractFileSystems(p); 2283 List<Token<?>> tokenList = 2284 new ArrayList<Token<?>>(); 2285 for (AbstractFileSystem afs : afsSet) { 2286 List<Token<?>> afsTokens = afs.getDelegationTokens(renewer); 2287 tokenList.addAll(afsTokens); 2288 } 2289 return tokenList; 2290 } 2291 2292 /** 2293 * Modifies ACL entries of files and directories. This method can add new ACL 2294 * entries or modify the permissions on existing ACL entries. All existing 2295 * ACL entries that are not specified in this call are retained without 2296 * changes. (Modifications are merged into the current ACL.) 2297 * 2298 * @param path Path to modify 2299 * @param aclSpec List<AclEntry> describing modifications 2300 * @throws IOException if an ACL could not be modified 2301 */ 2302 public void modifyAclEntries(final Path path, final List<AclEntry> aclSpec) 2303 throws IOException { 2304 Path absF = fixRelativePart(path); 2305 new FSLinkResolver<Void>() { 2306 @Override 2307 public Void next(final AbstractFileSystem fs, final Path p) 2308 throws IOException { 2309 fs.modifyAclEntries(p, aclSpec); 2310 return null; 2311 } 2312 }.resolve(this, absF); 2313 } 2314 2315 /** 2316 * Removes ACL entries from files and directories. Other ACL entries are 2317 * retained. 2318 * 2319 * @param path Path to modify 2320 * @param aclSpec List<AclEntry> describing entries to remove 2321 * @throws IOException if an ACL could not be modified 2322 */ 2323 public void removeAclEntries(final Path path, final List<AclEntry> aclSpec) 2324 throws IOException { 2325 Path absF = fixRelativePart(path); 2326 new FSLinkResolver<Void>() { 2327 @Override 2328 public Void next(final AbstractFileSystem fs, final Path p) 2329 throws IOException { 2330 fs.removeAclEntries(p, aclSpec); 2331 return null; 2332 } 2333 }.resolve(this, absF); 2334 } 2335 2336 /** 2337 * Removes all default ACL entries from files and directories. 2338 * 2339 * @param path Path to modify 2340 * @throws IOException if an ACL could not be modified 2341 */ 2342 public void removeDefaultAcl(Path path) 2343 throws IOException { 2344 Path absF = fixRelativePart(path); 2345 new FSLinkResolver<Void>() { 2346 @Override 2347 public Void next(final AbstractFileSystem fs, final Path p) 2348 throws IOException { 2349 fs.removeDefaultAcl(p); 2350 return null; 2351 } 2352 }.resolve(this, absF); 2353 } 2354 2355 /** 2356 * Removes all but the base ACL entries of files and directories. The entries 2357 * for user, group, and others are retained for compatibility with permission 2358 * bits. 2359 * 2360 * @param path Path to modify 2361 * @throws IOException if an ACL could not be removed 2362 */ 2363 public void removeAcl(Path path) throws IOException { 2364 Path absF = fixRelativePart(path); 2365 new FSLinkResolver<Void>() { 2366 @Override 2367 public Void next(final AbstractFileSystem fs, final Path p) 2368 throws IOException { 2369 fs.removeAcl(p); 2370 return null; 2371 } 2372 }.resolve(this, absF); 2373 } 2374 2375 /** 2376 * Fully replaces ACL of files and directories, discarding all existing 2377 * entries. 2378 * 2379 * @param path Path to modify 2380 * @param aclSpec List<AclEntry> describing modifications, must include entries 2381 * for user, group, and others for compatibility with permission bits. 2382 * @throws IOException if an ACL could not be modified 2383 */ 2384 public void setAcl(Path path, final List<AclEntry> aclSpec) 2385 throws IOException { 2386 Path absF = fixRelativePart(path); 2387 new FSLinkResolver<Void>() { 2388 @Override 2389 public Void next(final AbstractFileSystem fs, final Path p) 2390 throws IOException { 2391 fs.setAcl(p, aclSpec); 2392 return null; 2393 } 2394 }.resolve(this, absF); 2395 } 2396 2397 /** 2398 * Gets the ACLs of files and directories. 2399 * 2400 * @param path Path to get 2401 * @return RemoteIterator<AclStatus> which returns each AclStatus 2402 * @throws IOException if an ACL could not be read 2403 */ 2404 public AclStatus getAclStatus(Path path) throws IOException { 2405 Path absF = fixRelativePart(path); 2406 return new FSLinkResolver<AclStatus>() { 2407 @Override 2408 public AclStatus next(final AbstractFileSystem fs, final Path p) 2409 throws IOException { 2410 return fs.getAclStatus(p); 2411 } 2412 }.resolve(this, absF); 2413 } 2414 2415 /** 2416 * Set an xattr of a file or directory. 2417 * The name must be prefixed with the namespace followed by ".". For example, 2418 * "user.attr". 2419 * <p/> 2420 * Refer to the HDFS extended attributes user documentation for details. 2421 * 2422 * @param path Path to modify 2423 * @param name xattr name. 2424 * @param value xattr value. 2425 * @throws IOException 2426 */ 2427 public void setXAttr(Path path, String name, byte[] value) 2428 throws IOException { 2429 setXAttr(path, name, value, EnumSet.of(XAttrSetFlag.CREATE, 2430 XAttrSetFlag.REPLACE)); 2431 } 2432 2433 /** 2434 * Set an xattr of a file or directory. 2435 * The name must be prefixed with the namespace followed by ".". For example, 2436 * "user.attr". 2437 * <p/> 2438 * Refer to the HDFS extended attributes user documentation for details. 2439 * 2440 * @param path Path to modify 2441 * @param name xattr name. 2442 * @param value xattr value. 2443 * @param flag xattr set flag 2444 * @throws IOException 2445 */ 2446 public void setXAttr(Path path, final String name, final byte[] value, 2447 final EnumSet<XAttrSetFlag> flag) throws IOException { 2448 final Path absF = fixRelativePart(path); 2449 new FSLinkResolver<Void>() { 2450 @Override 2451 public Void next(final AbstractFileSystem fs, final Path p) 2452 throws IOException { 2453 fs.setXAttr(p, name, value, flag); 2454 return null; 2455 } 2456 }.resolve(this, absF); 2457 } 2458 2459 /** 2460 * Get an xattr for a file or directory. 2461 * The name must be prefixed with the namespace followed by ".". For example, 2462 * "user.attr". 2463 * <p/> 2464 * Refer to the HDFS extended attributes user documentation for details. 2465 * 2466 * @param path Path to get extended attribute 2467 * @param name xattr name. 2468 * @return byte[] xattr value. 2469 * @throws IOException 2470 */ 2471 public byte[] getXAttr(Path path, final String name) throws IOException { 2472 final Path absF = fixRelativePart(path); 2473 return new FSLinkResolver<byte[]>() { 2474 @Override 2475 public byte[] next(final AbstractFileSystem fs, final Path p) 2476 throws IOException { 2477 return fs.getXAttr(p, name); 2478 } 2479 }.resolve(this, absF); 2480 } 2481 2482 /** 2483 * Get all of the xattrs for a file or directory. 2484 * Only those xattrs for which the logged-in user has permissions to view 2485 * are returned. 2486 * <p/> 2487 * Refer to the HDFS extended attributes user documentation for details. 2488 * 2489 * @param path Path to get extended attributes 2490 * @return Map<String, byte[]> describing the XAttrs of the file or directory 2491 * @throws IOException 2492 */ 2493 public Map<String, byte[]> getXAttrs(Path path) throws IOException { 2494 final Path absF = fixRelativePart(path); 2495 return new FSLinkResolver<Map<String, byte[]>>() { 2496 @Override 2497 public Map<String, byte[]> next(final AbstractFileSystem fs, final Path p) 2498 throws IOException { 2499 return fs.getXAttrs(p); 2500 } 2501 }.resolve(this, absF); 2502 } 2503 2504 /** 2505 * Get all of the xattrs for a file or directory. 2506 * Only those xattrs for which the logged-in user has permissions to view 2507 * are returned. 2508 * <p/> 2509 * Refer to the HDFS extended attributes user documentation for details. 2510 * 2511 * @param path Path to get extended attributes 2512 * @param names XAttr names. 2513 * @return Map<String, byte[]> describing the XAttrs of the file or directory 2514 * @throws IOException 2515 */ 2516 public Map<String, byte[]> getXAttrs(Path path, final List<String> names) 2517 throws IOException { 2518 final Path absF = fixRelativePart(path); 2519 return new FSLinkResolver<Map<String, byte[]>>() { 2520 @Override 2521 public Map<String, byte[]> next(final AbstractFileSystem fs, final Path p) 2522 throws IOException { 2523 return fs.getXAttrs(p, names); 2524 } 2525 }.resolve(this, absF); 2526 } 2527 2528 /** 2529 * Remove an xattr of a file or directory. 2530 * The name must be prefixed with the namespace followed by ".". For example, 2531 * "user.attr". 2532 * <p/> 2533 * Refer to the HDFS extended attributes user documentation for details. 2534 * 2535 * @param path Path to remove extended attribute 2536 * @param name xattr name 2537 * @throws IOException 2538 */ 2539 public void removeXAttr(Path path, final String name) throws IOException { 2540 final Path absF = fixRelativePart(path); 2541 new FSLinkResolver<Void>() { 2542 @Override 2543 public Void next(final AbstractFileSystem fs, final Path p) 2544 throws IOException { 2545 fs.removeXAttr(p, name); 2546 return null; 2547 } 2548 }.resolve(this, absF); 2549 } 2550 2551 /** 2552 * Get all of the xattr names for a file or directory. 2553 * Only those xattr names which the logged-in user has permissions to view 2554 * are returned. 2555 * <p/> 2556 * Refer to the HDFS extended attributes user documentation for details. 2557 * 2558 * @param path Path to get extended attributes 2559 * @return List<String> of the XAttr names of the file or directory 2560 * @throws IOException 2561 */ 2562 public List<String> listXAttrs(Path path) throws IOException { 2563 final Path absF = fixRelativePart(path); 2564 return new FSLinkResolver<List<String>>() { 2565 @Override 2566 public List<String> next(final AbstractFileSystem fs, final Path p) 2567 throws IOException { 2568 return fs.listXAttrs(p); 2569 } 2570 }.resolve(this, absF); 2571 } 2572 2573 /** 2574 * Create a snapshot with a default name. 2575 * 2576 * @param path The directory where snapshots will be taken. 2577 * @return the snapshot path. 2578 * 2579 * @throws IOException If an I/O error occurred 2580 * 2581 * <p>Exceptions applicable to file systems accessed over RPC: 2582 * @throws RpcClientException If an exception occurred in the RPC client 2583 * @throws RpcServerException If an exception occurred in the RPC server 2584 * @throws UnexpectedServerException If server implementation throws 2585 * undeclared exception to RPC server 2586 */ 2587 public final Path createSnapshot(Path path) throws IOException { 2588 return createSnapshot(path, null); 2589 } 2590 2591 /** 2592 * Create a snapshot. 2593 * 2594 * @param path The directory where snapshots will be taken. 2595 * @param snapshotName The name of the snapshot 2596 * @return the snapshot path. 2597 * 2598 * @throws IOException If an I/O error occurred 2599 * 2600 * <p>Exceptions applicable to file systems accessed over RPC: 2601 * @throws RpcClientException If an exception occurred in the RPC client 2602 * @throws RpcServerException If an exception occurred in the RPC server 2603 * @throws UnexpectedServerException If server implementation throws 2604 * undeclared exception to RPC server 2605 */ 2606 public Path createSnapshot(final Path path, final String snapshotName) 2607 throws IOException { 2608 final Path absF = fixRelativePart(path); 2609 return new FSLinkResolver<Path>() { 2610 2611 @Override 2612 public Path next(final AbstractFileSystem fs, final Path p) 2613 throws IOException { 2614 return fs.createSnapshot(p, snapshotName); 2615 } 2616 }.resolve(this, absF); 2617 } 2618 2619 /** 2620 * Rename a snapshot. 2621 * 2622 * @param path The directory path where the snapshot was taken 2623 * @param snapshotOldName Old name of the snapshot 2624 * @param snapshotNewName New name of the snapshot 2625 * 2626 * @throws IOException If an I/O error occurred 2627 * 2628 * <p>Exceptions applicable to file systems accessed over RPC: 2629 * @throws RpcClientException If an exception occurred in the RPC client 2630 * @throws RpcServerException If an exception occurred in the RPC server 2631 * @throws UnexpectedServerException If server implementation throws 2632 * undeclared exception to RPC server 2633 */ 2634 public void renameSnapshot(final Path path, final String snapshotOldName, 2635 final String snapshotNewName) throws IOException { 2636 final Path absF = fixRelativePart(path); 2637 new FSLinkResolver<Void>() { 2638 @Override 2639 public Void next(final AbstractFileSystem fs, final Path p) 2640 throws IOException { 2641 fs.renameSnapshot(p, snapshotOldName, snapshotNewName); 2642 return null; 2643 } 2644 }.resolve(this, absF); 2645 } 2646 2647 /** 2648 * Delete a snapshot of a directory. 2649 * 2650 * @param path The directory that the to-be-deleted snapshot belongs to 2651 * @param snapshotName The name of the snapshot 2652 * 2653 * @throws IOException If an I/O error occurred 2654 * 2655 * <p>Exceptions applicable to file systems accessed over RPC: 2656 * @throws RpcClientException If an exception occurred in the RPC client 2657 * @throws RpcServerException If an exception occurred in the RPC server 2658 * @throws UnexpectedServerException If server implementation throws 2659 * undeclared exception to RPC server 2660 */ 2661 public void deleteSnapshot(final Path path, final String snapshotName) 2662 throws IOException { 2663 final Path absF = fixRelativePart(path); 2664 new FSLinkResolver<Void>() { 2665 @Override 2666 public Void next(final AbstractFileSystem fs, final Path p) 2667 throws IOException { 2668 fs.deleteSnapshot(p, snapshotName); 2669 return null; 2670 } 2671 }.resolve(this, absF); 2672 } 2673 2674 /** 2675 * Set the storage policy for a given file or directory. 2676 * 2677 * @param path file or directory path. 2678 * @param policyName the name of the target storage policy. The list 2679 * of supported Storage policies can be retrieved 2680 * via {@link #getAllStoragePolicies}. 2681 */ 2682 public void setStoragePolicy(final Path path, final String policyName) 2683 throws IOException { 2684 final Path absF = fixRelativePart(path); 2685 new FSLinkResolver<Void>() { 2686 @Override 2687 public Void next(final AbstractFileSystem fs, final Path p) 2688 throws IOException { 2689 fs.setStoragePolicy(path, policyName); 2690 return null; 2691 } 2692 }.resolve(this, absF); 2693 } 2694 2695 /** 2696 * Query the effective storage policy ID for the given file or directory. 2697 * 2698 * @param src file or directory path. 2699 * @return storage policy for give file. 2700 * @throws IOException 2701 */ 2702 public BlockStoragePolicySpi getStoragePolicy(Path path) throws IOException { 2703 final Path absF = fixRelativePart(path); 2704 return new FSLinkResolver<BlockStoragePolicySpi>() { 2705 @Override 2706 public BlockStoragePolicySpi next(final AbstractFileSystem fs, 2707 final Path p) 2708 throws IOException { 2709 return fs.getStoragePolicy(p); 2710 } 2711 }.resolve(this, absF); 2712 } 2713 2714 /** 2715 * Retrieve all the storage policies supported by this file system. 2716 * 2717 * @return all storage policies supported by this filesystem. 2718 * @throws IOException 2719 */ 2720 public Collection<? extends BlockStoragePolicySpi> getAllStoragePolicies() 2721 throws IOException { 2722 return defaultFS.getAllStoragePolicies(); 2723 } 2724}