001/* 002 * Copyright (c) 2007-2022 The Cascading Authors. All Rights Reserved. 003 * 004 * Project and contact information: https://cascading.wensel.net/ 005 * 006 * This file is part of the Cascading project. 007 * 008 * Licensed under the Apache License, Version 2.0 (the "License"); 009 * you may not use this file except in compliance with the License. 010 * You may obtain a copy of the License at 011 * 012 * http://www.apache.org/licenses/LICENSE-2.0 013 * 014 * Unless required by applicable law or agreed to in writing, software 015 * distributed under the License is distributed on an "AS IS" BASIS, 016 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 017 * See the License for the specific language governing permissions and 018 * limitations under the License. 019 */ 020 021package cascading.tap.hadoop; 022 023import java.beans.ConstructorProperties; 024import java.io.IOException; 025import java.net.URI; 026import java.net.URISyntaxException; 027import java.util.HashSet; 028import java.util.LinkedHashSet; 029import java.util.Set; 030 031import cascading.CascadingException; 032import cascading.flow.FlowProcess; 033import cascading.flow.FlowRuntimeProps; 034import cascading.flow.hadoop.util.HadoopUtil; 035import cascading.scheme.Scheme; 036import cascading.tap.SinkMode; 037import cascading.tap.Tap; 038import cascading.tap.TapException; 039import cascading.tap.hadoop.io.CombineFileRecordReaderWrapper; 040import cascading.tap.hadoop.io.HadoopTupleEntrySchemeCollector; 041import cascading.tap.hadoop.io.HadoopTupleEntrySchemeIterator; 042import cascading.tap.type.FileType; 043import cascading.tap.type.TapWith; 044import cascading.tuple.TupleEntryCollector; 045import cascading.tuple.TupleEntryIterator; 046import cascading.tuple.hadoop.TupleSerialization; 047import cascading.util.LazyIterable; 048import cascading.util.Util; 049import org.apache.hadoop.conf.Configurable; 050import org.apache.hadoop.conf.Configuration; 051import org.apache.hadoop.fs.FileStatus; 052import org.apache.hadoop.fs.FileSystem; 053import org.apache.hadoop.fs.Path; 054import org.apache.hadoop.fs.PathFilter; 055import org.apache.hadoop.mapred.FileInputFormat; 056import org.apache.hadoop.mapred.InputFormat; 057import org.apache.hadoop.mapred.InputSplit; 058import org.apache.hadoop.mapred.JobConf; 059import org.apache.hadoop.mapred.OutputCollector; 060import org.apache.hadoop.mapred.RecordReader; 061import org.apache.hadoop.mapred.Reporter; 062import org.apache.hadoop.mapred.lib.CombineFileInputFormat; 063import org.apache.hadoop.mapred.lib.CombineFileRecordReader; 064import org.apache.hadoop.mapred.lib.CombineFileSplit; 065import org.slf4j.Logger; 066import org.slf4j.LoggerFactory; 067 068/** 069 * Class Hfs is the base class for all Hadoop file system access. Hfs may only be used with the 070 * Hadoop {@link cascading.flow.FlowConnector} sub-classes when creating Hadoop executable {@link cascading.flow.Flow} 071 * instances. 072 * <p> 073 * Paths typically should point to a directory, where in turn all the "part" files immediately in that directory will 074 * be included. This is the practice Hadoop expects. Subdirectories are not included and typically result in a failure. 075 * <p> 076 * To include subdirectories, Hadoop supports "globing". Globing is a frustrating feature and is supported more 077 * robustly by {@link GlobHfs} and less so by Hfs. 078 * <p> 079 * Or in later versions of Hadoop (2.x+), set this property to {@code true}: 080 * {@code "mapreduce.input.fileinputformat.input.dir.recursive"} 081 * <p> 082 * Hfs will accept {@code /*} (wildcard) paths, but not all convenience methods like 083 * {@code jobConf.getSize} will behave properly or reliably. Nor can the Hfs instance 084 * with a wildcard path be used as a sink to write data. 085 * <p> 086 * In those cases use GlobHfs since it is a sub-class of {@link cascading.tap.MultiSourceTap}. 087 * <p> 088 * Optionally use {@link Dfs} or {@link Lfs} for resources specific to Hadoop Distributed file system or 089 * the Local file system, respectively. Using Hfs is the best practice when possible, Lfs and Dfs are conveniences. 090 * <p> 091 * Use the Hfs class if the 'kind' of resource is unknown at design time. To use, prefix a scheme to the 'stringPath'. Where 092 * {@code hdfs://...} will denote Dfs, and {@code file://...} will denote Lfs. 093 * <p> 094 * Call {@link HfsProps#setTemporaryDirectory(java.util.Map, String)} to use a different temporary file directory path 095 * other than the current Hadoop default path. 096 * <p> 097 * By default, Cascading on Hadoop will assume any source or sink Tap using the {@code file://} URI scheme 098 * intends to read files from the local client filesystem (for example when using the {@code Lfs} Tap) where the Hadoop 099 * job jar is started. Subsequently, Cascading will force any MapReduce jobs reading or writing to {@code file://} resources 100 * to run in Hadoop "standalone mode" so that the file can be read. 101 * <p> 102 * To change this behavior, {@link HfsProps#setLocalModeScheme(java.util.Map, String)} to set a different scheme value, 103 * or to "none" to disable entirely for the case the file to be read is available on every Hadoop processing node 104 * in the exact same path. 105 * <p> 106 * When using a MapReduce planner, Hfs can optionally combine multiple small files (or a series of small "blocks") into 107 * larger "splits". This reduces the number of resulting map tasks created by Hadoop and can improve application 108 * performance. 109 * <p> 110 * This is enabled by calling {@link HfsProps#setUseCombinedInput(boolean)} to {@code true}. By default, merging 111 * or combining splits into large ones is disabled. 112 * <p> 113 * Apache Tez planner does not require this setting, it is supported by default and enabled by the application manager. 114 * <p> 115 * If Hfs is overridden and sets multiple input paths directly against the configuration object, some methods will 116 * attempt to honor this property. But the primary path (returned by {@link #getIdentifier()}, must be a common directory 117 * to all the 'child' input paths added directly. 118 */ 119public class Hfs extends Tap<Configuration, RecordReader, OutputCollector> implements FileType<Configuration>, TapWith<Configuration, RecordReader, OutputCollector> 120 { 121 /** Field LOG */ 122 private static final Logger LOG = LoggerFactory.getLogger( Hfs.class ); 123 124 /** Field stringPath */ 125 protected String stringPath; 126 /** Field uriScheme */ 127 transient URI uriScheme; 128 /** Field path */ 129 transient Path path; 130 /** Field paths */ 131 private transient FileStatus[] statuses; // only used by getModifiedTime 132 133 private transient String cachedPath = null; 134 135 private static final PathFilter HIDDEN_FILES_FILTER = path -> 136 { 137 String name = path.getName(); 138 139 if( name.isEmpty() ) // should never happen 140 return true; 141 142 char first = name.charAt( 0 ); 143 144 return first != '_' && first != '.'; 145 }; 146 147 protected static String getLocalModeScheme( Configuration conf, String defaultValue ) 148 { 149 return conf.get( HfsProps.LOCAL_MODE_SCHEME, defaultValue ); 150 } 151 152 protected static boolean getUseCombinedInput( Configuration conf ) 153 { 154 boolean combineEnabled = conf.getBoolean( "cascading.hadoop.hfs.combine.files", false ); 155 156 if( conf.get( FlowRuntimeProps.COMBINE_SPLITS ) == null && !combineEnabled ) 157 return false; 158 159 if( !combineEnabled ) // enable if set in FlowRuntimeProps 160 combineEnabled = conf.getBoolean( FlowRuntimeProps.COMBINE_SPLITS, false ); 161 162 String platform = conf.get( "cascading.flow.platform", "" ); 163 164 // only supported by these platforms 165 if( platform.equals( "hadoop" ) || platform.equals( "hadoop2-mr1" ) ) 166 return combineEnabled; 167 168 // we are on a platform that supports combining, just not through the combiner 169 // do not enable it here locally 170 if( conf.get( FlowRuntimeProps.COMBINE_SPLITS ) != null ) 171 return false; 172 173 if( combineEnabled && !Boolean.getBoolean( "cascading.hadoop.hfs.combine.files.warned" ) ) 174 { 175 LOG.warn( "'cascading.hadoop.hfs.combine.files' has been set to true, but is unsupported by this platform: {}, will be ignored to prevent failures", platform ); 176 System.setProperty( "cascading.hadoop.hfs.combine.files.warned", "true" ); 177 } 178 179 return false; 180 } 181 182 protected static boolean getCombinedInputSafeMode( Configuration conf ) 183 { 184 return conf.getBoolean( "cascading.hadoop.hfs.combine.safemode", true ); 185 } 186 187 protected Hfs() 188 { 189 } 190 191 @ConstructorProperties({"scheme"}) 192 protected Hfs( Scheme<Configuration, RecordReader, OutputCollector, ?, ?> scheme ) 193 { 194 super( scheme ); 195 } 196 197 /** 198 * Constructor Hfs creates a new Hfs instance. 199 * 200 * @param scheme of type Scheme 201 * @param stringPath of type String 202 */ 203 @ConstructorProperties({"scheme", "stringPath"}) 204 public Hfs( Scheme<Configuration, RecordReader, OutputCollector, ?, ?> scheme, String stringPath ) 205 { 206 super( scheme ); 207 setStringPath( stringPath ); 208 } 209 210 /** 211 * Constructor Hfs creates a new Hfs instance. 212 * 213 * @param scheme of type Scheme 214 * @param stringPath of type String 215 * @param sinkMode of type SinkMode 216 */ 217 @ConstructorProperties({"scheme", "stringPath", "sinkMode"}) 218 public Hfs( Scheme<Configuration, RecordReader, OutputCollector, ?, ?> scheme, String stringPath, SinkMode sinkMode ) 219 { 220 super( scheme, sinkMode ); 221 setStringPath( stringPath ); 222 } 223 224 /** 225 * Constructor Hfs creates a new Hfs instance. 226 * 227 * @param scheme of type Scheme 228 * @param path of type Path 229 */ 230 @ConstructorProperties({"scheme", "path"}) 231 public Hfs( Scheme<Configuration, RecordReader, OutputCollector, ?, ?> scheme, Path path ) 232 { 233 super( scheme ); 234 setStringPath( path.toString() ); 235 } 236 237 /** 238 * Constructor Hfs creates a new Hfs instance. 239 * 240 * @param scheme of type Scheme 241 * @param path of type Path 242 * @param sinkMode of type SinkMode 243 */ 244 @ConstructorProperties({"scheme", "path", "sinkMode"}) 245 public Hfs( Scheme<Configuration, RecordReader, OutputCollector, ?, ?> scheme, Path path, SinkMode sinkMode ) 246 { 247 super( scheme, sinkMode ); 248 setStringPath( path.toString() ); 249 } 250 251 @Override 252 public TapWith<Configuration, RecordReader, OutputCollector> withChildIdentifier( String identifier ) 253 { 254 Path path = new Path( identifier ); 255 256 if( !path.toString().startsWith( getPath().toString() ) ) 257 path = new Path( getPath(), path ); 258 259 return create( getScheme(), path, getSinkMode() ); 260 } 261 262 @Override 263 public TapWith<Configuration, RecordReader, OutputCollector> withScheme( Scheme<Configuration, RecordReader, OutputCollector, ?, ?> scheme ) 264 { 265 return create( scheme, getPath(), getSinkMode() ); 266 } 267 268 @Override 269 public TapWith<Configuration, RecordReader, OutputCollector> withSinkMode( SinkMode sinkMode ) 270 { 271 return create( getScheme(), getPath(), sinkMode ); 272 } 273 274 protected TapWith<Configuration, RecordReader, OutputCollector> create( Scheme<Configuration, RecordReader, OutputCollector, ?, ?> scheme, Path path, SinkMode sinkMode ) 275 { 276 try 277 { 278 return Util.newInstance( getClass(), new Object[]{scheme, path, sinkMode} ); 279 } 280 catch( CascadingException exception ) 281 { 282 throw new TapException( "unable to create a new instance of: " + getClass().getName(), exception ); 283 } 284 } 285 286 protected void setStringPath( String stringPath ) 287 { 288 this.stringPath = Util.normalizeUrl( stringPath ); 289 } 290 291 protected void setUriScheme( URI uriScheme ) 292 { 293 this.uriScheme = uriScheme; 294 } 295 296 public URI getURIScheme( Configuration jobConf ) 297 { 298 if( uriScheme != null ) 299 return uriScheme; 300 301 uriScheme = makeURIScheme( jobConf ); 302 303 return uriScheme; 304 } 305 306 protected URI makeURIScheme( Configuration configuration ) 307 { 308 try 309 { 310 URI uriScheme; 311 312 LOG.debug( "handling path: {}", stringPath ); 313 314 URI uri = new Path( stringPath ).toUri(); // safer URI parsing 315 String schemeString = uri.getScheme(); 316 String authority = uri.getAuthority(); 317 318 LOG.debug( "found scheme: {}, authority: {}", schemeString, authority ); 319 320 if( schemeString != null && authority != null ) 321 uriScheme = new URI( schemeString + "://" + uri.getAuthority() ); 322 else if( schemeString != null ) 323 uriScheme = new URI( schemeString + ":///" ); 324 else 325 uriScheme = getDefaultFileSystemURIScheme( configuration ); 326 327 LOG.debug( "using uri scheme: {}", uriScheme ); 328 329 return uriScheme; 330 } 331 catch( URISyntaxException exception ) 332 { 333 throw new TapException( "could not determine scheme from path: " + getPath(), exception ); 334 } 335 } 336 337 /** 338 * Method getDefaultFileSystemURIScheme returns the URI scheme for the default Hadoop FileSystem. 339 * 340 * @param configuration of type JobConf 341 * @return URI 342 */ 343 public URI getDefaultFileSystemURIScheme( Configuration configuration ) 344 { 345 return getDefaultFileSystem( configuration ).getUri(); 346 } 347 348 protected FileSystem getDefaultFileSystem( Configuration configuration ) 349 { 350 try 351 { 352 return FileSystem.get( configuration ); 353 } 354 catch( IOException exception ) 355 { 356 throw new TapException( "unable to get handle to underlying filesystem", exception ); 357 } 358 } 359 360 protected FileSystem getFileSystem( Configuration configuration ) 361 { 362 URI scheme = getURIScheme( configuration ); 363 364 try 365 { 366 return FileSystem.get( scheme, configuration ); 367 } 368 catch( IOException exception ) 369 { 370 throw new TapException( "unable to get handle to get filesystem for: " + scheme.getScheme(), exception ); 371 } 372 } 373 374 @Override 375 public String getIdentifier() 376 { 377 if( cachedPath == null ) 378 cachedPath = getPath().toString(); 379 380 return cachedPath; 381 } 382 383 public Path getPath() 384 { 385 if( path != null ) 386 return path; 387 388 if( stringPath == null ) 389 throw new IllegalStateException( "path not initialized" ); 390 391 path = new Path( stringPath ); 392 393 return path; 394 } 395 396 @Override 397 public String getFullIdentifier( Configuration conf ) 398 { 399 return getPath().makeQualified( getFileSystem( conf ) ).toString(); 400 } 401 402 @Override 403 public void sourceConfInit( FlowProcess<? extends Configuration> process, Configuration conf ) 404 { 405 String fullIdentifier = getFullIdentifier( conf ); 406 407 applySourceConfInitIdentifiers( process, conf, fullIdentifier ); 408 409 verifyNoDuplicates( conf ); 410 } 411 412 protected static void verifyNoDuplicates( Configuration conf ) 413 { 414 Path[] inputPaths = getDeclaredInputPaths( conf ); 415 Set<Path> paths = new HashSet<Path>( (int) ( inputPaths.length / .75f ) ); 416 417 for( Path inputPath : inputPaths ) 418 { 419 if( !paths.add( inputPath ) ) 420 throw new TapException( "may not add duplicate paths, found: " + inputPath ); 421 } 422 } 423 424 protected static Path[] getDeclaredInputPaths( Configuration conf ) 425 { 426 return FileInputFormat.getInputPaths( HadoopUtil.asJobConfInstance( conf ) ); 427 } 428 429 protected void applySourceConfInitIdentifiers( FlowProcess<? extends Configuration> process, Configuration conf, final String... fullIdentifiers ) 430 { 431 sourceConfInitAddInputPaths( conf, new LazyIterable<String, Path>( fullIdentifiers ) 432 { 433 @Override 434 protected Path convert( String next ) 435 { 436 return new Path( next ); 437 } 438 } ); 439 440 sourceConfInitComplete( process, conf ); 441 } 442 443 protected void sourceConfInitAddInputPaths( Configuration conf, Iterable<Path> qualifiedPaths ) 444 { 445 HadoopUtil.addInputPaths( conf, qualifiedPaths ); 446 447 for( Path qualifiedPath : qualifiedPaths ) 448 { 449 boolean stop = !makeLocal( conf, qualifiedPath, "forcing job to stand-alone mode, via source: " ); 450 451 if( stop ) 452 break; 453 } 454 } 455 456 @Deprecated 457 protected void sourceConfInitAddInputPath( Configuration conf, Path qualifiedPath ) 458 { 459 HadoopUtil.addInputPath( conf, qualifiedPath ); 460 461 makeLocal( conf, qualifiedPath, "forcing job to stand-alone mode, via source: " ); 462 } 463 464 protected void sourceConfInitComplete( FlowProcess<? extends Configuration> process, Configuration conf ) 465 { 466 super.sourceConfInit( process, conf ); 467 468 TupleSerialization.setSerializations( conf ); // allows Hfs to be used independent of Flow 469 470 // use CombineFileInputFormat if that is enabled 471 handleCombineFileInputFormat( conf ); 472 } 473 474 /** 475 * Based on the configuration, handles and sets {@link CombineFileInputFormat} as the input 476 * format. 477 */ 478 private void handleCombineFileInputFormat( Configuration conf ) 479 { 480 // if combining files, override the configuration to use CombineFileInputFormat 481 if( !getUseCombinedInput( conf ) ) 482 return; 483 484 // get the prescribed individual input format from the underlying scheme so it can be used by CombinedInputFormat 485 String individualInputFormat = conf.get( "mapred.input.format.class" ); 486 487 if( individualInputFormat == null ) 488 throw new TapException( "input format is missing from the underlying scheme" ); 489 490 if( individualInputFormat.equals( CombinedInputFormat.class.getName() ) && 491 conf.get( CombineFileRecordReaderWrapper.INDIVIDUAL_INPUT_FORMAT ) == null ) 492 throw new TapException( "the input format class is already the combined input format but the underlying input format is missing" ); 493 494 // if safe mode is on (default) throw an exception if the InputFormat is not a FileInputFormat, otherwise log a 495 // warning and don't use the CombineFileInputFormat 496 boolean safeMode = getCombinedInputSafeMode( conf ); 497 498 if( !FileInputFormat.class.isAssignableFrom( conf.getClass( "mapred.input.format.class", null ) ) ) 499 { 500 if( safeMode ) 501 throw new TapException( "input format must be of type org.apache.hadoop.mapred.FileInputFormat, got: " + individualInputFormat ); 502 else 503 LOG.warn( "not combining input splits with CombineFileInputFormat, {} is not of type org.apache.hadoop.mapred.FileInputFormat.", individualInputFormat ); 504 } 505 else 506 { 507 // set the underlying individual input format 508 conf.set( CombineFileRecordReaderWrapper.INDIVIDUAL_INPUT_FORMAT, individualInputFormat ); 509 510 // override the input format class 511 conf.setClass( "mapred.input.format.class", CombinedInputFormat.class, InputFormat.class ); 512 } 513 } 514 515 @Override 516 public void sinkConfInit( FlowProcess<? extends Configuration> process, Configuration conf ) 517 { 518 Path qualifiedPath = new Path( getFullIdentifier( conf ) ); 519 520 HadoopUtil.setOutputPath( conf, qualifiedPath ); 521 super.sinkConfInit( process, conf ); 522 523 makeLocal( conf, qualifiedPath, "forcing job to stand-alone mode, via sink: " ); 524 525 TupleSerialization.setSerializations( conf ); // allows Hfs to be used independent of Flow 526 } 527 528 private boolean makeLocal( Configuration conf, Path qualifiedPath, String infoMessage ) 529 { 530 // don't change the conf or log any messages if running cluster side 531 if( HadoopUtil.isInflow( conf ) ) 532 return false; 533 534 String scheme = getLocalModeScheme( conf, "file" ); 535 536 if( !HadoopUtil.isLocal( conf ) && qualifiedPath.toUri().getScheme().equalsIgnoreCase( scheme ) ) 537 { 538 if( LOG.isInfoEnabled() ) 539 LOG.info( infoMessage + toString() ); 540 541 HadoopUtil.setLocal( conf ); // force job to run locally 542 543 return false; // only need to set local once 544 } 545 546 return true; 547 } 548 549 @Override 550 public TupleEntryIterator openForRead( FlowProcess<? extends Configuration> flowProcess, RecordReader input ) throws IOException 551 { 552 // input may be null when this method is called on the client side or cluster side when accumulating 553 // for a HashJoin 554 return new HadoopTupleEntrySchemeIterator( flowProcess, this, input ); 555 } 556 557 @Override 558 public TupleEntryCollector openForWrite( FlowProcess<? extends Configuration> flowProcess, OutputCollector output ) throws IOException 559 { 560 resetFileStatuses(); 561 562 // output may be null when this method is called on the client side or cluster side when creating 563 // side files with the PartitionTap 564 return new HadoopTupleEntrySchemeCollector( flowProcess, this, output ); 565 } 566 567 @Override 568 public boolean createResource( Configuration conf ) throws IOException 569 { 570 if( LOG.isDebugEnabled() ) 571 LOG.debug( "making dirs: {}", getFullIdentifier( conf ) ); 572 573 return getFileSystem( conf ).mkdirs( getPath() ); 574 } 575 576 @Override 577 public boolean deleteResource( Configuration conf ) throws IOException 578 { 579 String fullIdentifier = getFullIdentifier( conf ); 580 581 return deleteFullIdentifier( conf, fullIdentifier ); 582 } 583 584 private boolean deleteFullIdentifier( Configuration conf, String fullIdentifier ) throws IOException 585 { 586 if( LOG.isDebugEnabled() ) 587 LOG.debug( "deleting: {}", fullIdentifier ); 588 589 resetFileStatuses(); 590 591 Path fullPath = new Path( fullIdentifier ); 592 593 // do not delete the root directory 594 if( fullPath.depth() == 0 ) 595 return true; 596 597 FileSystem fileSystem = getFileSystem( conf ); 598 599 try 600 { 601 return fileSystem.delete( fullPath, true ); 602 } 603 catch( NullPointerException exception ) 604 { 605 // hack to get around npe thrown when fs reaches root directory 606 // removes coupling to the new aws hadoop artifacts that may not be deployed 607 if( !( fileSystem.getClass().getSimpleName().equals( "NativeS3FileSystem" ) ) ) 608 throw exception; 609 } 610 611 return true; 612 } 613 614 public boolean deleteChildResource( FlowProcess<? extends Configuration> flowProcess, String childIdentifier ) throws IOException 615 { 616 return deleteChildResource( flowProcess.getConfig(), childIdentifier ); 617 } 618 619 public boolean deleteChildResource( Configuration conf, String childIdentifier ) throws IOException 620 { 621 resetFileStatuses(); 622 623 Path childPath = new Path( childIdentifier ).makeQualified( getFileSystem( conf ) ); 624 625 if( !childPath.toString().startsWith( getFullIdentifier( conf ) ) ) 626 return false; 627 628 return deleteFullIdentifier( conf, childPath.toString() ); 629 } 630 631 @Override 632 public boolean resourceExists( Configuration conf ) throws IOException 633 { 634 // unfortunately getFileSystem( conf ).exists( getPath() ); does not account for "/*" etc 635 // nor is there an more efficient means to test for existence 636 FileStatus[] fileStatuses = getFileSystem( conf ).globStatus( getPath() ); 637 638 return fileStatuses != null && fileStatuses.length > 0; 639 } 640 641 @Override 642 public boolean isDirectory( FlowProcess<? extends Configuration> flowProcess ) throws IOException 643 { 644 return isDirectory( flowProcess.getConfig() ); 645 } 646 647 @Override 648 public boolean isDirectory( Configuration conf ) throws IOException 649 { 650 if( !resourceExists( conf ) ) 651 return false; 652 653 return getFileSystem( conf ).getFileStatus( getPath() ).isDir(); 654 } 655 656 @Override 657 public long getSize( FlowProcess<? extends Configuration> flowProcess ) throws IOException 658 { 659 return getSize( flowProcess.getConfig() ); 660 } 661 662 @Override 663 public long getSize( Configuration conf ) throws IOException 664 { 665 if( !resourceExists( conf ) ) 666 return 0; 667 668 FileStatus fileStatus = getFileStatus( conf ); 669 670 if( fileStatus.isDir() ) 671 return 0; 672 673 return getFileSystem( conf ).getFileStatus( getPath() ).getLen(); 674 } 675 676 /** 677 * Method getBlockSize returns the {@code blocksize} specified by the underlying file system for this resource. 678 * 679 * @param flowProcess 680 * @return long 681 * @throws IOException when 682 */ 683 public long getBlockSize( FlowProcess<? extends Configuration> flowProcess ) throws IOException 684 { 685 return getBlockSize( flowProcess.getConfig() ); 686 } 687 688 /** 689 * Method getBlockSize returns the {@code blocksize} specified by the underlying file system for this resource. 690 * 691 * @param conf of JobConf 692 * @return long 693 * @throws IOException when 694 */ 695 public long getBlockSize( Configuration conf ) throws IOException 696 { 697 if( !resourceExists( conf ) ) 698 return 0; 699 700 FileStatus fileStatus = getFileStatus( conf ); 701 702 if( fileStatus.isDir() ) 703 return 0; 704 705 return fileStatus.getBlockSize(); 706 } 707 708 /** 709 * Method getReplication returns the {@code replication} specified by the underlying file system for 710 * this resource. 711 * 712 * @param flowProcess 713 * @return int 714 * @throws IOException when 715 */ 716 public int getReplication( FlowProcess<? extends Configuration> flowProcess ) throws IOException 717 { 718 return getReplication( flowProcess.getConfig() ); 719 } 720 721 /** 722 * Method getReplication returns the {@code replication} specified by the underlying file system for 723 * this resource. 724 * 725 * @param conf of JobConf 726 * @return int 727 * @throws IOException when 728 */ 729 public int getReplication( Configuration conf ) throws IOException 730 { 731 if( !resourceExists( conf ) ) 732 return 0; 733 734 FileStatus fileStatus = getFileStatus( conf ); 735 736 if( fileStatus.isDir() ) 737 return 0; 738 739 return fileStatus.getReplication(); 740 } 741 742 @Override 743 public String[] getChildIdentifiers( FlowProcess<? extends Configuration> flowProcess ) throws IOException 744 { 745 return getChildIdentifiers( flowProcess.getConfig(), 1, false ); 746 } 747 748 @Override 749 public String[] getChildIdentifiers( Configuration conf ) throws IOException 750 { 751 return getChildIdentifiers( conf, 1, false ); 752 } 753 754 @Override 755 public String[] getChildIdentifiers( FlowProcess<? extends Configuration> flowProcess, int depth, boolean fullyQualified ) throws IOException 756 { 757 return getChildIdentifiers( flowProcess.getConfig(), depth, fullyQualified ); 758 } 759 760 @Override 761 public String[] getChildIdentifiers( Configuration conf, int depth, boolean fullyQualified ) throws IOException 762 { 763 if( !resourceExists( conf ) ) 764 return new String[ 0 ]; 765 766 if( depth == 0 && !fullyQualified ) 767 return new String[]{getIdentifier()}; 768 769 // we are assuming the identifier is a root of the declared input paths, if any 770 String rootIdentifier = getFullIdentifier( conf ); 771 int trim = fullyQualified ? 0 : rootIdentifier.length() + 1; 772 773 // if set, expected to be the children of the rootIdentifier 774 Path[] inputPaths = getDeclaredInputPaths( conf ); 775 776 if( inputPaths.length == 0 ) 777 inputPaths = new Path[]{new Path( rootIdentifier )}; 778 779 Set<String> results = new LinkedHashSet<>(); 780 781 for( Path identifier : inputPaths ) 782 getChildPaths( conf, results, trim, identifier, depth ); 783 784 return results.toArray( new String[ 0 ] ); 785 } 786 787 private void getChildPaths( Configuration conf, Set<String> results, int trim, Path path, int depth ) throws IOException 788 { 789 if( depth == 0 ) 790 { 791 String substring = path.toString().substring( trim ); 792 String identifier = getIdentifier(); 793 794 if( identifier == null || identifier.isEmpty() ) 795 results.add( new Path( substring ).toString() ); 796 else 797 results.add( new Path( identifier, substring ).toString() ); 798 799 return; 800 } 801 802 FileStatus[] statuses = getFileSystem( conf ).listStatus( path, HIDDEN_FILES_FILTER ); 803 804 if( statuses == null ) 805 return; 806 807 for( FileStatus fileStatus : statuses ) 808 getChildPaths( conf, results, trim, fileStatus.getPath(), depth - 1 ); 809 } 810 811 @Override 812 public long getModifiedTime( Configuration conf ) throws IOException 813 { 814 if( !resourceExists( conf ) ) 815 return 0; 816 817 FileStatus fileStatus = getFileStatus( conf ); 818 819 if( !fileStatus.isDir() ) 820 return fileStatus.getModificationTime(); 821 822 // todo: this should ignore the _temporary path, or not cache if found in the array 823 makeStatuses( conf ); 824 825 // statuses is empty, return 0 826 if( statuses == null || statuses.length == 0 ) 827 return 0; 828 829 long date = 0; 830 831 // filter out directories as we don't recurs into sub dirs 832 for( FileStatus status : statuses ) 833 { 834 if( !status.isDir() ) 835 date = Math.max( date, status.getModificationTime() ); 836 } 837 838 return date; 839 } 840 841 public FileStatus getFileStatus( Configuration conf ) throws IOException 842 { 843 return getFileSystem( conf ).getFileStatus( getPath() ); 844 } 845 846 public static Path getTempPath( Configuration conf ) 847 { 848 String tempDir = conf.get( HfsProps.TEMPORARY_DIRECTORY ); 849 850 if( tempDir == null ) 851 tempDir = conf.get( "hadoop.tmp.dir" ); 852 853 return new Path( tempDir ); 854 } 855 856 protected String makeTemporaryPathDirString( String name ) 857 { 858 // _ is treated as a hidden file, so wipe them out 859 name = name.replaceAll( "^[_\\W\\s]+", "" ); 860 861 if( name.isEmpty() ) 862 name = "temp-path"; 863 864 return name.replaceAll( "[\\W\\s]+", "_" ) + Util.createUniqueID(); 865 } 866 867 /** 868 * Given a file-system object, it makes an array of paths 869 * 870 * @param conf of type JobConf 871 * @throws IOException on failure 872 */ 873 private void makeStatuses( Configuration conf ) throws IOException 874 { 875 if( statuses != null ) 876 return; 877 878 statuses = getFileSystem( conf ).listStatus( getPath() ); 879 } 880 881 /** 882 * Method resetFileStatuses removes the status cache, if any. 883 */ 884 public void resetFileStatuses() 885 { 886 statuses = null; 887 } 888 889 /** Combined input format that uses the underlying individual input format to combine multiple files into a single split. */ 890 static class CombinedInputFormat extends CombineFileInputFormat implements Configurable 891 { 892 private Configuration conf; 893 894 public RecordReader getRecordReader( InputSplit split, JobConf job, Reporter reporter ) throws IOException 895 { 896 return new CombineFileRecordReader( job, (CombineFileSplit) split, reporter, CombineFileRecordReaderWrapper.class ); 897 } 898 899 @Override 900 public void setConf( Configuration conf ) 901 { 902 this.conf = conf; 903 904 // set the aliased property value, if zero, the super class will look up the hadoop property 905 setMaxSplitSize( conf.getLong( "cascading.hadoop.hfs.combine.max.size", 0 ) ); 906 } 907 908 @Override 909 public Configuration getConf() 910 { 911 return conf; 912 } 913 } 914 }