001 /* 002 * Copyright (c) 2007-2014 Concurrent, Inc. All Rights Reserved. 003 * 004 * Project and contact information: http://www.cascading.org/ 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 021 package cascading.flow.hadoop.util; 022 023 import java.io.IOException; 024 import java.io.UnsupportedEncodingException; 025 import java.lang.reflect.Field; 026 import java.net.URI; 027 import java.net.URL; 028 import java.util.HashMap; 029 import java.util.HashSet; 030 import java.util.List; 031 import java.util.Map; 032 import java.util.Properties; 033 import java.util.Set; 034 import java.util.jar.Attributes; 035 import java.util.jar.Manifest; 036 037 import cascading.flow.FlowException; 038 import cascading.flow.hadoop.HadoopFlowProcess; 039 import cascading.flow.planner.PlatformInfo; 040 import cascading.scheme.hadoop.TextLine; 041 import cascading.tap.SinkMode; 042 import cascading.tap.hadoop.Hfs; 043 import cascading.tap.hadoop.Lfs; 044 import cascading.tuple.Fields; 045 import cascading.tuple.Tuple; 046 import cascading.tuple.TupleEntryCollector; 047 import cascading.tuple.TupleEntryIterator; 048 import cascading.util.Util; 049 import org.apache.commons.codec.binary.Base64; 050 import org.apache.hadoop.conf.Configurable; 051 import org.apache.hadoop.conf.Configuration; 052 import org.apache.hadoop.filecache.DistributedCache; 053 import org.apache.hadoop.fs.FileSystem; 054 import org.apache.hadoop.fs.LocalFileSystem; 055 import org.apache.hadoop.fs.Path; 056 import org.apache.hadoop.mapred.JobConf; 057 import org.slf4j.Logger; 058 import org.slf4j.LoggerFactory; 059 060 /** 061 * 062 */ 063 public class HadoopUtil 064 { 065 private static final Logger LOG = LoggerFactory.getLogger( HadoopUtil.class ); 066 private static final String ENCODING = "US-ASCII"; 067 private static final Class<?> DEFAULT_OBJECT_SERIALIZER = JavaObjectSerializer.class; 068 private static PlatformInfo platformInfo; 069 070 public static void initLog4j( JobConf jobConf ) 071 { 072 String values = jobConf.get( "log4j.logger", null ); 073 074 if( values == null || values.length() == 0 ) 075 return; 076 077 if( !Util.hasClass( "org.apache.log4j.Logger" ) ) 078 { 079 LOG.info( "org.apache.log4j.Logger is not in the current CLASSPATH, not setting log4j.logger properties" ); 080 return; 081 } 082 083 String[] elements = values.split( "," ); 084 085 for( String element : elements ) 086 setLogLevel( element.split( "=" ) ); 087 } 088 089 private static void setLogLevel( String[] logger ) 090 { 091 // removing logj4 dependency 092 // org.apache.log4j.Logger.getLogger( logger[ 0 ] ).setLevel( org.apache.log4j.Level.toLevel( logger[ 1 ] ) ); 093 094 Object loggerObject = Util.invokeStaticMethod( "org.apache.log4j.Logger", "getLogger", 095 new Object[]{logger[ 0 ]}, new Class[]{String.class} ); 096 097 Object levelObject = Util.invokeStaticMethod( "org.apache.log4j.Level", "toLevel", 098 new Object[]{logger[ 1 ]}, new Class[]{String.class} ); 099 100 Util.invokeInstanceMethod( loggerObject, "setLevel", 101 new Object[]{levelObject}, new Class[]{levelObject.getClass()} ); 102 } 103 104 public static JobConf createJobConf( Map<Object, Object> properties, JobConf defaultJobconf ) 105 { 106 JobConf jobConf = defaultJobconf == null ? new JobConf() : new JobConf( defaultJobconf ); 107 108 if( properties == null ) 109 return jobConf; 110 111 Set<Object> keys = new HashSet<Object>( properties.keySet() ); 112 113 // keys will only be grabbed if both key/value are String, so keep orig keys 114 if( properties instanceof Properties ) 115 keys.addAll( ( (Properties) properties ).stringPropertyNames() ); 116 117 for( Object key : keys ) 118 { 119 Object value = properties.get( key ); 120 121 if( value == null && properties instanceof Properties && key instanceof String ) 122 value = ( (Properties) properties ).getProperty( (String) key ); 123 124 if( value == null ) // don't stuff null values 125 continue; 126 127 // don't let these objects pass, even though toString is called below. 128 if( value instanceof Class || value instanceof JobConf ) 129 continue; 130 131 jobConf.set( key.toString(), value.toString() ); 132 } 133 134 return jobConf; 135 } 136 137 public static Map<Object, Object> createProperties( Configuration jobConf ) 138 { 139 Map<Object, Object> properties = new HashMap<Object, Object>(); 140 141 if( jobConf == null ) 142 return properties; 143 144 for( Map.Entry<String, String> entry : jobConf ) 145 properties.put( entry.getKey(), entry.getValue() ); 146 147 return properties; 148 } 149 150 public static Thread getHDFSShutdownHook() 151 { 152 Exception caughtException; 153 154 try 155 { 156 // we must init the FS so the finalizer is registered 157 FileSystem.getLocal( new JobConf() ); 158 159 Field field = FileSystem.class.getDeclaredField( "clientFinalizer" ); 160 field.setAccessible( true ); 161 162 Thread finalizer = (Thread) field.get( null ); 163 164 if( finalizer != null ) 165 Runtime.getRuntime().removeShutdownHook( finalizer ); 166 167 return finalizer; 168 } 169 catch( NoSuchFieldException exception ) 170 { 171 caughtException = exception; 172 } 173 catch( IllegalAccessException exception ) 174 { 175 caughtException = exception; 176 } 177 catch( IOException exception ) 178 { 179 caughtException = exception; 180 } 181 182 LOG.debug( "unable to find and remove client hdfs shutdown hook, received exception: {}", caughtException.getClass().getName() ); 183 184 return null; 185 } 186 187 public static String encodeBytes( byte[] bytes ) 188 { 189 try 190 { 191 return new String( Base64.encodeBase64( bytes ), ENCODING ); 192 } 193 catch( UnsupportedEncodingException exception ) 194 { 195 throw new RuntimeException( exception ); 196 } 197 } 198 199 public static byte[] decodeBytes( String string ) 200 { 201 try 202 { 203 byte[] bytes = string.getBytes( ENCODING ); 204 return Base64.decodeBase64( bytes ); 205 } 206 catch( UnsupportedEncodingException exception ) 207 { 208 throw new RuntimeException( exception ); 209 } 210 } 211 212 public static <T> ObjectSerializer instantiateSerializer( Configuration conf, Class<T> type ) throws ClassNotFoundException 213 { 214 Class<ObjectSerializer> flowSerializerClass; 215 216 String serializerClassName = conf.get( ObjectSerializer.OBJECT_SERIALIZER_PROPERTY ); 217 218 if( serializerClassName == null || serializerClassName.length() == 0 ) 219 flowSerializerClass = (Class<ObjectSerializer>) DEFAULT_OBJECT_SERIALIZER; 220 else 221 flowSerializerClass = (Class<ObjectSerializer>) Class.forName( serializerClassName ); 222 223 ObjectSerializer objectSerializer; 224 225 try 226 { 227 objectSerializer = flowSerializerClass.newInstance(); 228 229 if( objectSerializer instanceof Configurable ) 230 ( (Configurable) objectSerializer ).setConf( conf ); 231 } 232 catch( Exception exception ) 233 { 234 exception.printStackTrace(); 235 throw new IllegalArgumentException( "Unable to instantiate serializer \"" 236 + flowSerializerClass.getName() 237 + "\" for class: " 238 + type.getName() ); 239 } 240 241 if( !objectSerializer.accepts( type ) ) 242 throw new IllegalArgumentException( serializerClassName + " won't accept objects of class " + type.toString() ); 243 244 return objectSerializer; 245 } 246 247 public static <T> String serializeBase64( T object, JobConf conf ) throws IOException 248 { 249 return serializeBase64( object, conf, true ); 250 } 251 252 public static <T> String serializeBase64( T object, JobConf conf, boolean compress ) throws IOException 253 { 254 ObjectSerializer objectSerializer; 255 256 try 257 { 258 objectSerializer = instantiateSerializer( conf, object.getClass() ); 259 } 260 catch( ClassNotFoundException exception ) 261 { 262 throw new IOException( exception ); 263 } 264 265 return encodeBytes( objectSerializer.serialize( object, compress ) ); 266 } 267 268 /** 269 * This method deserializes the Base64 encoded String into an Object instance. 270 * 271 * @param string 272 * @return an Object 273 */ 274 public static <T> T deserializeBase64( String string, Configuration conf, Class<T> type ) throws IOException 275 { 276 return deserializeBase64( string, conf, type, true ); 277 } 278 279 public static <T> T deserializeBase64( String string, Configuration conf, Class<T> type, boolean decompress ) throws IOException 280 { 281 if( string == null || string.length() == 0 ) 282 return null; 283 284 ObjectSerializer objectSerializer; 285 286 try 287 { 288 objectSerializer = instantiateSerializer( conf, type ); 289 } 290 catch( ClassNotFoundException exception ) 291 { 292 throw new IOException( exception ); 293 } 294 295 return objectSerializer.deserialize( decodeBytes( string ), type, decompress ); 296 } 297 298 public static Class findMainClass( Class defaultType ) 299 { 300 StackTraceElement[] stackTrace = Thread.currentThread().getStackTrace(); 301 302 for( StackTraceElement stackTraceElement : stackTrace ) 303 { 304 if( stackTraceElement.getMethodName().equals( "main" ) && !stackTraceElement.getClassName().startsWith( "org.apache.hadoop" ) ) 305 { 306 try 307 { 308 LOG.info( "resolving application jar from found main method on: {}", stackTraceElement.getClassName() ); 309 310 return Thread.currentThread().getContextClassLoader().loadClass( stackTraceElement.getClassName() ); 311 } 312 catch( ClassNotFoundException exception ) 313 { 314 LOG.warn( "unable to load class while discovering application jar: {}", stackTraceElement.getClassName(), exception ); 315 } 316 } 317 } 318 319 LOG.info( "using default application jar, may cause class not found exceptions on the cluster" ); 320 321 return defaultType; 322 } 323 324 public static Map<String, String> getConfig( JobConf defaultConf, JobConf updatedConf ) 325 { 326 Map<String, String> configs = new HashMap<String, String>(); 327 328 for( Map.Entry<String, String> entry : updatedConf ) 329 configs.put( entry.getKey(), entry.getValue() ); 330 331 for( Map.Entry<String, String> entry : defaultConf ) 332 { 333 if( entry.getValue() == null ) 334 continue; 335 336 String updatedValue = configs.get( entry.getKey() ); 337 338 // if both null, lets purge from map to save space 339 if( updatedValue == null && entry.getValue() == null ) 340 configs.remove( entry.getKey() ); 341 342 // if the values are the same, lets also purge from map to save space 343 if( updatedValue != null && updatedValue.equals( entry.getValue() ) ) 344 configs.remove( entry.getKey() ); 345 346 configs.remove( "mapred.working.dir" ); 347 configs.remove( "mapreduce.job.working.dir" ); // hadoop2 348 } 349 350 return configs; 351 } 352 353 public static JobConf[] getJobConfs( JobConf job, List<Map<String, String>> configs ) 354 { 355 JobConf[] jobConfs = new JobConf[ configs.size() ]; 356 357 for( int i = 0; i < jobConfs.length; i++ ) 358 jobConfs[ i ] = mergeConf( job, configs.get( i ), false ); 359 360 return jobConfs; 361 } 362 363 public static JobConf mergeConf( JobConf job, Map<String, String> config, boolean directly ) 364 { 365 JobConf currentConf = directly ? job : new JobConf( job ); 366 367 for( String key : config.keySet() ) 368 { 369 LOG.debug( "merging key: {} value: {}", key, config.get( key ) ); 370 371 currentConf.set( key, config.get( key ) ); 372 } 373 374 return currentConf; 375 } 376 377 public static JobConf removePropertiesFrom( JobConf jobConf, String... keys ) 378 { 379 Map<Object, Object> properties = createProperties( jobConf ); 380 381 for( String key : keys ) 382 properties.remove( key ); 383 384 return createJobConf( properties, null ); 385 } 386 387 public static boolean removeStateFromDistCache( JobConf conf, String path ) throws IOException 388 { 389 return new Hfs( new TextLine(), path ).deleteResource( conf ); 390 } 391 392 public static String writeStateToDistCache( JobConf conf, String id, String stepState ) 393 { 394 LOG.info( "writing step state to dist cache, too large for job conf, size: {}", stepState.length() ); 395 396 String statePath = Hfs.getTempPath( conf ) + "/step-state-" + id; 397 398 Hfs temp = new Hfs( new TextLine(), statePath, SinkMode.REPLACE ); 399 400 try 401 { 402 TupleEntryCollector writer = temp.openForWrite( new HadoopFlowProcess( conf ) ); 403 404 writer.add( new Tuple( stepState ) ); 405 406 writer.close(); 407 } 408 catch( IOException exception ) 409 { 410 throw new FlowException( "unable to write step state to Hadoop FS: " + temp.getIdentifier() ); 411 } 412 413 URI uri = new Path( statePath ).toUri(); 414 DistributedCache.addCacheFile( uri, conf ); 415 416 LOG.info( "using step state path: {}", uri ); 417 418 return statePath; 419 } 420 421 public static String readStateFromDistCache( JobConf jobConf, String id ) throws IOException 422 { 423 Path[] files = DistributedCache.getLocalCacheFiles( jobConf ); 424 425 Path stepStatePath = null; 426 427 for( Path file : files ) 428 { 429 if( !file.toString().contains( "step-state-" + id ) ) 430 continue; 431 432 stepStatePath = file; 433 break; 434 } 435 436 if( stepStatePath == null ) 437 throw new FlowException( "unable to find step state from distributed cache" ); 438 439 LOG.info( "reading step state from local path: {}", stepStatePath ); 440 441 Hfs temp = new Lfs( new TextLine( new Fields( "line" ) ), stepStatePath.toString() ); 442 443 TupleEntryIterator reader = null; 444 445 try 446 { 447 reader = temp.openForRead( new HadoopFlowProcess( jobConf ) ); 448 449 if( !reader.hasNext() ) 450 throw new FlowException( "step state path is empty: " + temp.getIdentifier() ); 451 452 return reader.next().getString( 0 ); 453 } 454 catch( IOException exception ) 455 { 456 throw new FlowException( "unable to find state path: " + temp.getIdentifier(), exception ); 457 } 458 finally 459 { 460 try 461 { 462 if( reader != null ) 463 reader.close(); 464 } 465 catch( IOException exception ) 466 { 467 LOG.warn( "error closing state path reader", exception ); 468 } 469 } 470 } 471 472 public static PlatformInfo getPlatformInfo() 473 { 474 if( platformInfo == null ) 475 platformInfo = getPlatformInfoInternal(); 476 477 return platformInfo; 478 } 479 480 private static PlatformInfo getPlatformInfoInternal() 481 { 482 URL url = JobConf.class.getResource( JobConf.class.getSimpleName() + ".class" ); 483 484 if( url == null || !url.toString().startsWith( "jar" ) ) 485 return new PlatformInfo( "Hadoop", null, null ); 486 487 String path = url.toString(); 488 String manifestPath = path.substring( 0, path.lastIndexOf( "!" ) + 1 ) + "/META-INF/MANIFEST.MF"; 489 490 Manifest manifest; 491 492 try 493 { 494 manifest = new Manifest( new URL( manifestPath ).openStream() ); 495 } 496 catch( IOException exception ) 497 { 498 LOG.warn( "unable to get manifest from {}", manifestPath, exception ); 499 500 return new PlatformInfo( "Hadoop", null, null ); 501 } 502 503 Attributes attributes = manifest.getAttributes( "org/apache/hadoop" ); 504 505 if( attributes == null ) 506 { 507 LOG.debug( "unable to get Hadoop manifest attributes" ); 508 return new PlatformInfo( "Hadoop", null, null ); 509 } 510 511 String vendor = attributes.getValue( "Implementation-Vendor" ); 512 String version = attributes.getValue( "Implementation-Version" ); 513 514 return new PlatformInfo( "Hadoop", vendor, version ); 515 } 516 517 /** 518 * Add to class path. 519 * 520 * @param config the config 521 * @param classpath the classpath 522 */ 523 public static Map<Path, Path> addToClassPath( JobConf config, List<String> classpath ) 524 { 525 if( classpath == null ) 526 return null; 527 528 // given to fully qualified 529 Map<String, Path> localPaths = new HashMap<String, Path>(); 530 Map<String, Path> remotePaths = new HashMap<String, Path>(); 531 532 resolvePaths( config, classpath, localPaths, remotePaths ); 533 534 try 535 { 536 LocalFileSystem localFS = getLocalFS( config ); 537 538 for( String path : localPaths.keySet() ) 539 { 540 // only add local if no remote 541 if( remotePaths.containsKey( path ) ) 542 continue; 543 544 Path artifact = localPaths.get( path ); 545 546 DistributedCache.addFileToClassPath( artifact.makeQualified( localFS ), config ); 547 } 548 549 FileSystem defaultFS = getDefaultFS( config ); 550 551 for( String path : remotePaths.keySet() ) 552 { 553 // always add remote 554 Path artifact = remotePaths.get( path ); 555 556 DistributedCache.addFileToClassPath( artifact.makeQualified( defaultFS ), config ); 557 } 558 } 559 catch( IOException exception ) 560 { 561 throw new FlowException( "unable to set distributed cache paths", exception ); 562 } 563 564 return getCommonPaths( localPaths, remotePaths ); 565 } 566 567 public static void syncPaths( JobConf config, Map<Path, Path> commonPaths ) 568 { 569 if( commonPaths == null ) 570 return; 571 572 Map<Path, Path> copyPaths = getCopyPaths( config, commonPaths ); 573 574 FileSystem remoteFS = getDefaultFS( config ); 575 576 for( Map.Entry<Path, Path> entry : copyPaths.entrySet() ) 577 { 578 Path localPath = entry.getKey(); 579 Path remotePath = entry.getValue(); 580 581 try 582 { 583 LOG.info( "copying from: {}, to: {}", localPath, remotePath ); 584 remoteFS.copyFromLocalFile( localPath, remotePath ); 585 } 586 catch( IOException exception ) 587 { 588 throw new FlowException( "unable to copy local: " + localPath + " to remote: " + remotePath ); 589 } 590 } 591 } 592 593 private static Map<Path, Path> getCommonPaths( Map<String, Path> localPaths, Map<String, Path> remotePaths ) 594 { 595 Map<Path, Path> commonPaths = new HashMap<Path, Path>(); 596 597 for( Map.Entry<String, Path> entry : localPaths.entrySet() ) 598 { 599 if( remotePaths.containsKey( entry.getKey() ) ) 600 commonPaths.put( entry.getValue(), remotePaths.get( entry.getKey() ) ); 601 } 602 return commonPaths; 603 } 604 605 private static Map<Path, Path> getCopyPaths( JobConf config, Map<Path, Path> commonPaths ) 606 { 607 Map<Path, Path> copyPaths = new HashMap<Path, Path>(); 608 609 FileSystem remoteFS = getDefaultFS( config ); 610 FileSystem localFS = getLocalFS( config ); 611 612 for( Map.Entry<Path, Path> entry : commonPaths.entrySet() ) 613 { 614 Path localPath = entry.getKey(); 615 Path remotePath = entry.getValue(); 616 617 try 618 { 619 boolean localExists = localFS.exists( localPath ); 620 boolean remoteExist = remoteFS.exists( remotePath ); 621 622 if( localExists && !remoteExist ) 623 { 624 copyPaths.put( localPath, remotePath ); 625 } 626 else if( localExists ) 627 { 628 long localModTime = localFS.getFileStatus( localPath ).getModificationTime(); 629 long remoteModTime = remoteFS.getFileStatus( remotePath ).getModificationTime(); 630 631 if( localModTime > remoteModTime ) 632 copyPaths.put( localPath, remotePath ); 633 } 634 } 635 catch( IOException exception ) 636 { 637 throw new FlowException( "unable to get handle to underlying filesystem", exception ); 638 } 639 } 640 641 return copyPaths; 642 } 643 644 private static void resolvePaths( JobConf config, List<String> classpath, Map<String, Path> localPaths, Map<String, Path> remotePaths ) 645 { 646 FileSystem defaultFS = getDefaultFS( config ); 647 FileSystem localFS = getLocalFS( config ); 648 649 boolean defaultIsLocal = defaultFS.equals( localFS ); 650 651 for( String stringPath : classpath ) 652 { 653 URI uri = URI.create( stringPath ); // fails if invalid uri 654 Path path = new Path( uri.toString() ); 655 656 if( uri.getScheme() == null && !defaultIsLocal ) // we want to sync 657 { 658 Path localPath = path.makeQualified( localFS ); 659 660 if( !exists( localFS, localPath ) ) 661 throw new FlowException( "path not found: " + localPath ); 662 663 localPaths.put( stringPath, localPath ); 664 remotePaths.put( stringPath, path.makeQualified( defaultFS ) ); 665 } 666 else if( localFS.equals( getFileSystem( config, path ) ) ) 667 { 668 if( !exists( localFS, path ) ) 669 throw new FlowException( "path not found: " + path ); 670 671 localPaths.put( stringPath, path ); 672 } 673 else 674 { 675 if( !exists( defaultFS, path ) ) 676 throw new FlowException( "path not found: " + path ); 677 678 remotePaths.put( stringPath, path ); 679 } 680 } 681 } 682 683 private static boolean exists( FileSystem fileSystem, Path path ) 684 { 685 try 686 { 687 return fileSystem.exists( path ); 688 } 689 catch( IOException exception ) 690 { 691 throw new FlowException( "could not test file exists: " + path ); 692 } 693 } 694 695 private static FileSystem getFileSystem( JobConf config, Path path ) 696 { 697 try 698 { 699 return path.getFileSystem( config ); 700 } 701 catch( IOException exception ) 702 { 703 throw new FlowException( "unable to get handle to underlying filesystem", exception ); 704 } 705 } 706 707 private static LocalFileSystem getLocalFS( JobConf config ) 708 { 709 try 710 { 711 return FileSystem.getLocal( config ); 712 } 713 catch( IOException exception ) 714 { 715 throw new FlowException( "unable to get handle to underlying filesystem", exception ); 716 } 717 } 718 719 private static FileSystem getDefaultFS( JobConf config ) 720 { 721 try 722 { 723 return FileSystem.get( config ); 724 } 725 catch( IOException exception ) 726 { 727 throw new FlowException( "unable to get handle to underlying filesystem", exception ); 728 } 729 } 730 731 public static boolean isLocal( JobConf conf ) 732 { 733 // hadoop 1.0 and 2.0 use different properties to define local mode: we check the new YARN 734 // property first 735 String frameworkName = conf.get( "mapreduce.framework.name" ); 736 737 // we are running on hadoop 2.0 (YARN) 738 if( frameworkName != null ) 739 return frameworkName.equals( "local" ); 740 741 // hadoop 1.0: use the old property to determine the local mode 742 return conf.get( "mapred.job.tracker" ).equals( "local" ); 743 } 744 745 public static void setLocal( JobConf conf ) 746 { 747 // set both properties to local 748 conf.set( "mapred.job.tracker", "local" ); 749 conf.set( "mapreduce.framework.name", "local" ); 750 } 751 }