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.flow.tez; 022 023import java.io.File; 024import java.io.IOException; 025import java.util.ArrayList; 026import java.util.Collection; 027import java.util.HashMap; 028import java.util.HashSet; 029import java.util.Iterator; 030import java.util.LinkedList; 031import java.util.List; 032import java.util.Map; 033import java.util.Set; 034 035import cascading.CascadingException; 036import cascading.flow.FlowElement; 037import cascading.flow.FlowElements; 038import cascading.flow.FlowException; 039import cascading.flow.FlowNode; 040import cascading.flow.FlowProcess; 041import cascading.flow.FlowRuntimeProps; 042import cascading.flow.hadoop.ConfigurationSetter; 043import cascading.flow.hadoop.util.HadoopUtil; 044import cascading.flow.planner.BaseFlowStep; 045import cascading.flow.planner.FlowStepJob; 046import cascading.flow.planner.graph.ElementGraph; 047import cascading.flow.planner.process.FlowNodeGraph; 048import cascading.flow.planner.process.ProcessEdge; 049import cascading.flow.stream.annotations.StreamMode; 050import cascading.flow.tez.planner.Hadoop2TezFlowStepJob; 051import cascading.flow.tez.util.TezUtil; 052import cascading.management.state.ClientState; 053import cascading.pipe.Boundary; 054import cascading.pipe.CoGroup; 055import cascading.pipe.Group; 056import cascading.pipe.GroupBy; 057import cascading.pipe.Merge; 058import cascading.pipe.Splice; 059import cascading.property.AppProps; 060import cascading.tap.CompositeTaps; 061import cascading.tap.Tap; 062import cascading.tap.hadoop.Hfs; 063import cascading.tap.hadoop.PartitionTap; 064import cascading.tap.hadoop.util.Hadoop18TapUtil; 065import cascading.tuple.Fields; 066import cascading.tuple.hadoop.TupleSerialization; 067import cascading.tuple.hadoop.util.GroupingSortingComparator; 068import cascading.tuple.hadoop.util.ReverseGroupingSortingComparator; 069import cascading.tuple.hadoop.util.ReverseTupleComparator; 070import cascading.tuple.hadoop.util.TupleComparator; 071import cascading.tuple.io.KeyTuple; 072import cascading.tuple.io.TuplePair; 073import cascading.tuple.io.ValueTuple; 074import cascading.tuple.tez.util.GroupingSortingPartitioner; 075import cascading.tuple.tez.util.TuplePartitioner; 076import cascading.util.Util; 077import cascading.util.Version; 078import org.apache.hadoop.conf.Configuration; 079import org.apache.hadoop.fs.FileSystem; 080import org.apache.hadoop.fs.Path; 081import org.apache.hadoop.io.serializer.Serialization; 082import org.apache.hadoop.mapred.JobConf; 083import org.apache.hadoop.mapreduce.JobContext; 084import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat; 085import org.apache.hadoop.yarn.api.records.LocalResource; 086import org.apache.hadoop.yarn.api.records.LocalResourceType; 087import org.apache.tez.common.TezUtils; 088import org.apache.tez.dag.api.DAG; 089import org.apache.tez.dag.api.DataSinkDescriptor; 090import org.apache.tez.dag.api.DataSourceDescriptor; 091import org.apache.tez.dag.api.Edge; 092import org.apache.tez.dag.api.EdgeProperty; 093import org.apache.tez.dag.api.GroupInputEdge; 094import org.apache.tez.dag.api.InputDescriptor; 095import org.apache.tez.dag.api.OutputDescriptor; 096import org.apache.tez.dag.api.ProcessorDescriptor; 097import org.apache.tez.dag.api.TezConfiguration; 098import org.apache.tez.dag.api.UserPayload; 099import org.apache.tez.dag.api.Vertex; 100import org.apache.tez.dag.api.VertexGroup; 101import org.apache.tez.mapreduce.input.MRInput; 102import org.apache.tez.mapreduce.output.MROutput; 103import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; 104import org.apache.tez.runtime.library.input.ConcatenatedMergedKeyValueInput; 105import org.apache.tez.runtime.library.input.OrderedGroupedKVInput; 106import org.apache.tez.runtime.library.input.OrderedGroupedMergedKVInput; 107import org.apache.tez.runtime.library.input.UnorderedKVInput; 108import org.apache.tez.runtime.library.output.OrderedPartitionedKVOutput; 109import org.apache.tez.runtime.library.output.UnorderedKVOutput; 110import org.apache.tez.runtime.library.output.UnorderedPartitionedKVOutput; 111import org.slf4j.Logger; 112import org.slf4j.LoggerFactory; 113 114import static cascading.flow.hadoop.util.HadoopUtil.*; 115import static cascading.flow.tez.util.TezUtil.addToClassPath; 116import static cascading.tap.hadoop.DistCacheTap.CASCADING_LOCAL_RESOURCES; 117import static cascading.tap.hadoop.DistCacheTap.CASCADING_REMOTE_RESOURCES; 118import static cascading.util.Util.getFirst; 119import static java.util.Collections.singletonList; 120import static org.apache.hadoop.yarn.api.records.LocalResourceType.ARCHIVE; 121import static org.apache.hadoop.yarn.api.records.LocalResourceType.FILE; 122 123/** 124 * 125 */ 126public class Hadoop2TezFlowStep extends BaseFlowStep<TezConfiguration> 127 { 128 private static final Logger LOG = LoggerFactory.getLogger( Hadoop2TezFlowStep.class ); 129 130 private Map<String, LocalResource> allLocalResources = new HashMap<>(); 131 private Map<Path, Path> syncPaths = new HashMap<>(); 132 private Map<String, String> environment = new HashMap<>(); 133 134 public Hadoop2TezFlowStep( ElementGraph elementGraph, FlowNodeGraph flowNodeGraph ) 135 { 136 super( elementGraph, flowNodeGraph ); 137 } 138 139 @Override 140 public Map<Object, Object> getConfigAsProperties() 141 { 142 return HadoopUtil.createProperties( getConfig() ); 143 } 144 145 @Override 146 public TezConfiguration createInitializedConfig( FlowProcess<TezConfiguration> flowProcess, TezConfiguration parentConfig ) 147 { 148 TezConfiguration stepConf = parentConfig == null ? new TezConfiguration() : new TezConfiguration( parentConfig ); 149 150 Set<String> serializations = getFieldDeclaredSerializations( Serialization.class ); 151 152 TupleSerialization.setSerializations( stepConf, serializations ); 153 154 String versionString = Version.getRelease(); 155 156 if( versionString != null ) 157 stepConf.set( "cascading.version", versionString ); 158 159 stepConf.set( CASCADING_FLOW_STEP_ID, getID() ); 160 stepConf.set( "cascading.flow.step.num", Integer.toString( getOrdinal() ) ); 161 162 String flowStagingPath = ( (Hadoop2TezFlow) getFlow() ).getFlowStagingPath(); 163 List<String> classPath = ( (Hadoop2TezFlow) getFlow() ).getClassPath(); 164 165 // is updated in addToClassPath method 166 Map<String, LocalResource> dagResources = new HashMap<>(); 167 168 if( !classPath.isEmpty() ) 169 { 170 // jars in the root will be in the remote CLASSPATH, no need to add to the environment 171 Map<Path, Path> dagClassPath = addToClassPath( stepConf, flowStagingPath, null, classPath, FILE, dagResources, null ); 172 173 syncPaths.putAll( dagClassPath ); 174 } 175 176 String appJarPath = stepConf.get( AppProps.APP_JAR_PATH ); 177 178 if( appJarPath != null ) 179 { 180 // the PATTERN represents the insides of the app jar, those elements must be added to the remote CLASSPATH 181 List<String> classpath = singletonList( appJarPath ); 182 Map<Path, Path> pathMap = addToClassPath( stepConf, flowStagingPath, null, classpath, ARCHIVE, dagResources, environment ); 183 184 syncPaths.putAll( pathMap ); 185 186 // AM does not support environments like containers do, so the classpath has to be passed via configuration. 187 String fileName = new File( appJarPath ).getName(); 188 stepConf.set( TezConfiguration.TEZ_CLUSTER_ADDITIONAL_CLASSPATH_PREFIX, 189 "$PWD/" + fileName + "/:$PWD/" + fileName + "/classes/:$PWD/" + fileName + "/lib/*:" ); 190 } 191 192 allLocalResources.putAll( dagResources ); 193 194 initFromStepConfigDef( stepConf ); 195 196 return stepConf; 197 } 198 199 @Override 200 protected FlowStepJob createFlowStepJob( ClientState clientState, FlowProcess<TezConfiguration> flowProcess, TezConfiguration initializedStepConfig ) 201 { 202 DAG dag = createDAG( flowProcess, initializedStepConfig ); 203 204 return new Hadoop2TezFlowStepJob( clientState, this, initializedStepConfig, dag ); 205 } 206 207 private DAG createDAG( FlowProcess<TezConfiguration> flowProcess, TezConfiguration initializedConfig ) 208 { 209 FlowNodeGraph nodeGraph = getFlowNodeGraph(); 210 Map<FlowNode, Vertex> vertexMap = new HashMap<>(); 211 DAG dag = DAG.create( getStepDisplayName( initializedConfig.getInt( "cascading.display.id.truncate", Util.ID_LENGTH ) ) ); 212 213 dag.addTaskLocalFiles( allLocalResources ); 214 215 Iterator<FlowNode> iterator = nodeGraph.getOrderedTopologicalIterator(); // ordering of nodes for consistent remote debugging 216 217 while( iterator.hasNext() ) 218 { 219 FlowNode flowNode = iterator.next(); 220 221 Vertex vertex = createVertex( flowProcess, initializedConfig, flowNode ); 222 dag.addVertex( vertex ); 223 224 vertexMap.put( flowNode, vertex ); 225 } 226 227 LinkedList<ProcessEdge> processedEdges = new LinkedList<>(); 228 229 for( ProcessEdge processEdge : nodeGraph.edgeSet() ) 230 { 231 if( processedEdges.contains( processEdge ) ) 232 continue; 233 234 FlowNode edgeTargetFlowNode = nodeGraph.getEdgeTarget( processEdge ); 235 236 FlowElement flowElement = processEdge.getFlowElement(); 237 List<FlowNode> sourceNodes = nodeGraph.getElementSourceProcesses( flowElement ); 238 239 EdgeProperty edgeProperty = createEdgeProperty( initializedConfig, processEdge ); 240 241 Vertex targetVertex = vertexMap.get( edgeTargetFlowNode ); 242 243 if( sourceNodes.size() == 1 || flowElement instanceof CoGroup || flowElement instanceof Boundary ) // todo: create group vertices around incoming ordinal 244 { 245 FlowNode edgeSourceFlowNode = nodeGraph.getEdgeSource( processEdge ); 246 Vertex sourceVertex = vertexMap.get( edgeSourceFlowNode ); 247 248 LOG.debug( "adding edge between: {} and {}", sourceVertex, targetVertex ); 249 250 dag.addEdge( Edge.create( sourceVertex, targetVertex, edgeProperty ) ); 251 } 252 else if( flowElement instanceof GroupBy || flowElement instanceof Merge ) // merge - source nodes > 1 253 { 254 List<String> sourceVerticesIDs = new ArrayList<>(); 255 List<Vertex> sourceVertices = new ArrayList<>(); 256 257 for( FlowNode edgeSourceFlowNode : sourceNodes ) 258 { 259 sourceVerticesIDs.add( edgeSourceFlowNode.getID() ); 260 sourceVertices.add( vertexMap.get( edgeSourceFlowNode ) ); 261 processedEdges.add( nodeGraph.getEdge( edgeSourceFlowNode, edgeTargetFlowNode ) ); 262 } 263 264 VertexGroup vertexGroup = dag.createVertexGroup( edgeTargetFlowNode.getID(), sourceVertices.toArray( new Vertex[ sourceVertices.size() ] ) ); 265 266 String inputClassName = flowElement instanceof Group ? OrderedGroupedMergedKVInput.class.getName() : ConcatenatedMergedKeyValueInput.class.getName(); 267 268 InputDescriptor inputDescriptor = InputDescriptor.create( inputClassName ).setUserPayload( edgeProperty.getEdgeDestination().getUserPayload() ); 269 270 String type = ( (Splice) flowElement ).isMerge() ? "merged" : "grouped"; 271 LOG.info( "adding {} edge between: {} and {}", type, Util.join( sourceVerticesIDs, "," ), targetVertex.getName() ); 272 dag.addEdge( GroupInputEdge.create( vertexGroup, targetVertex, edgeProperty, inputDescriptor ) ); 273 } 274 else 275 { 276 throw new UnsupportedOperationException( "can't make edge for: " + flowElement ); 277 } 278 } 279 280 return dag; 281 } 282 283 private EdgeProperty createEdgeProperty( TezConfiguration config, ProcessEdge processEdge ) 284 { 285 FlowElement flowElement = processEdge.getFlowElement(); 286 287 EdgeValues edgeValues = new EdgeValues( new TezConfiguration( config ), processEdge ); 288 289 edgeValues.keyClassName = KeyTuple.class.getName(); // TEZ_RUNTIME_INTERMEDIATE_OUTPUT_KEY_CLASS 290 edgeValues.valueClassName = ValueTuple.class.getName(); // TEZ_RUNTIME_INTERMEDIATE_OUTPUT_VALUE_CLASS 291 edgeValues.keyComparatorClassName = TupleComparator.class.getName(); 292 edgeValues.keyPartitionerClassName = TuplePartitioner.class.getName(); 293 edgeValues.outputClassName = null; 294 edgeValues.inputClassName = null; 295 edgeValues.movementType = null; 296 edgeValues.sourceType = null; 297 edgeValues.schedulingType = null; 298 299 if( flowElement instanceof Group ) 300 applyGroup( edgeValues ); 301 else if( ( flowElement instanceof Boundary || flowElement instanceof Merge ) && processEdge.getSinkAnnotations().contains( StreamMode.Accumulated ) ) 302 applyBoundaryMergeAccumulated( edgeValues ); 303 else if( flowElement instanceof Boundary || flowElement instanceof Merge ) 304 applyBoundaryMerge( edgeValues ); 305 else 306 throw new IllegalStateException( "unsupported flow element: " + flowElement.getClass().getCanonicalName() ); 307 308 applyEdgeAnnotations( processEdge, edgeValues ); 309 310 return createEdgeProperty( edgeValues ); 311 } 312 313 private void applyEdgeAnnotations( ProcessEdge processEdge, EdgeValues edgeValues ) 314 { 315 processEdge.addEdgeAnnotation( edgeValues.movementType ); 316 processEdge.addEdgeAnnotation( edgeValues.sourceType ); 317 processEdge.addEdgeAnnotation( edgeValues.schedulingType ); 318 } 319 320 private EdgeValues applyBoundaryMerge( EdgeValues edgeValues ) 321 { 322 // todo: support for one to one 323 edgeValues.outputClassName = UnorderedPartitionedKVOutput.class.getName(); 324 edgeValues.inputClassName = UnorderedKVInput.class.getName(); 325 326 edgeValues.movementType = EdgeProperty.DataMovementType.SCATTER_GATHER; 327 edgeValues.sourceType = EdgeProperty.DataSourceType.PERSISTED; 328 edgeValues.schedulingType = EdgeProperty.SchedulingType.SEQUENTIAL; 329 330 return edgeValues; 331 } 332 333 private EdgeValues applyBoundaryMergeAccumulated( EdgeValues edgeValues ) 334 { 335 edgeValues.outputClassName = UnorderedKVOutput.class.getName(); 336 edgeValues.inputClassName = UnorderedKVInput.class.getName(); 337 338 edgeValues.movementType = EdgeProperty.DataMovementType.BROADCAST; 339 edgeValues.sourceType = EdgeProperty.DataSourceType.PERSISTED; 340 edgeValues.schedulingType = EdgeProperty.SchedulingType.SEQUENTIAL; 341 342 return edgeValues; 343 } 344 345 private EdgeValues applyGroup( EdgeValues edgeValues ) 346 { 347 Group group = (Group) edgeValues.flowElement; 348 349 if( group.isSortReversed() ) 350 edgeValues.keyComparatorClassName = ReverseTupleComparator.class.getName(); 351 352 int ordinal = getFirst( edgeValues.ordinals ); 353 354 addComparators( edgeValues.config, "cascading.group.comparator", group.getKeySelectors(), edgeValues.getResolvedKeyFieldsMap().get( ordinal ) ); 355 356 if( !group.isGroupBy() ) 357 { 358 edgeValues.outputClassName = OrderedPartitionedKVOutput.class.getName(); 359 edgeValues.inputClassName = OrderedGroupedKVInput.class.getName(); 360 361 edgeValues.movementType = EdgeProperty.DataMovementType.SCATTER_GATHER; 362 edgeValues.sourceType = EdgeProperty.DataSourceType.PERSISTED; 363 edgeValues.schedulingType = EdgeProperty.SchedulingType.SEQUENTIAL; 364 } 365 else 366 { 367 addComparators( edgeValues.config, "cascading.sort.comparator", group.getSortingSelectors(), edgeValues.getResolvedSortFieldsMap().get( ordinal ) ); 368 369 edgeValues.outputClassName = OrderedPartitionedKVOutput.class.getName(); 370 edgeValues.inputClassName = OrderedGroupedKVInput.class.getName(); 371 372 edgeValues.movementType = EdgeProperty.DataMovementType.SCATTER_GATHER; 373 edgeValues.sourceType = EdgeProperty.DataSourceType.PERSISTED; 374 edgeValues.schedulingType = EdgeProperty.SchedulingType.SEQUENTIAL; 375 } 376 377 if( group.isSorted() ) 378 { 379 edgeValues.keyClassName = TuplePair.class.getName(); 380 edgeValues.keyPartitionerClassName = GroupingSortingPartitioner.class.getName(); 381 382 if( group.isSortReversed() ) 383 edgeValues.keyComparatorClassName = ReverseGroupingSortingComparator.class.getName(); 384 else 385 edgeValues.keyComparatorClassName = GroupingSortingComparator.class.getName(); 386 } 387 388 return edgeValues; 389 } 390 391 private EdgeProperty createEdgeProperty( EdgeValues edgeValues ) 392 { 393 TezConfiguration outputConfig = new TezConfiguration( edgeValues.getConfig() ); 394 outputConfig.set( "cascading.node.sink", FlowElements.id( edgeValues.getFlowElement() ) ); 395 outputConfig.set( "cascading.node.ordinals", Util.join( edgeValues.getOrdinals(), "," ) ); 396 addFields( outputConfig, "cascading.node.key.fields", edgeValues.getResolvedKeyFieldsMap() ); 397 addFields( outputConfig, "cascading.node.sort.fields", edgeValues.getResolvedSortFieldsMap() ); 398 addFields( outputConfig, "cascading.node.value.fields", edgeValues.getResolvedValueFieldsMap() ); 399 400 UserPayload outputPayload = createIntermediatePayloadOutput( outputConfig, edgeValues ); 401 402 TezConfiguration inputConfig = new TezConfiguration( edgeValues.getConfig() ); 403 inputConfig.set( "cascading.node.source", FlowElements.id( edgeValues.getFlowElement() ) ); 404 inputConfig.set( "cascading.node.ordinals", Util.join( edgeValues.getOrdinals(), "," ) ); 405 addFields( inputConfig, "cascading.node.key.fields", edgeValues.getResolvedKeyFieldsMap() ); 406 addFields( inputConfig, "cascading.node.sort.fields", edgeValues.getResolvedSortFieldsMap() ); 407 addFields( inputConfig, "cascading.node.value.fields", edgeValues.getResolvedValueFieldsMap() ); 408 409 UserPayload inputPayload = createIntermediatePayloadInput( inputConfig, edgeValues ); 410 411 return EdgeProperty.create( 412 edgeValues.getMovementType(), 413 edgeValues.getSourceType(), 414 edgeValues.getSchedulingType(), 415 OutputDescriptor.create( edgeValues.getOutputClassName() ).setUserPayload( outputPayload ), 416 InputDescriptor.create( edgeValues.getInputClassName() ).setUserPayload( inputPayload ) 417 ); 418 } 419 420 private UserPayload createIntermediatePayloadOutput( TezConfiguration config, EdgeValues edgeValues ) 421 { 422 config.set( TezRuntimeConfiguration.TEZ_RUNTIME_KEY_CLASS, edgeValues.keyClassName ); 423 config.set( TezRuntimeConfiguration.TEZ_RUNTIME_VALUE_CLASS, edgeValues.valueClassName ); 424 config.set( TezRuntimeConfiguration.TEZ_RUNTIME_KEY_COMPARATOR_CLASS, edgeValues.keyComparatorClassName ); 425 config.set( TezRuntimeConfiguration.TEZ_RUNTIME_PARTITIONER_CLASS, edgeValues.keyPartitionerClassName ); 426 427 setWorkingDirectory( config ); 428 429 return getPayload( config ); 430 } 431 432 private UserPayload createIntermediatePayloadInput( TezConfiguration config, EdgeValues edgeValues ) 433 { 434 config.set( TezRuntimeConfiguration.TEZ_RUNTIME_KEY_CLASS, edgeValues.keyClassName ); 435 config.set( TezRuntimeConfiguration.TEZ_RUNTIME_VALUE_CLASS, edgeValues.valueClassName ); 436 config.set( TezRuntimeConfiguration.TEZ_RUNTIME_KEY_COMPARATOR_CLASS, edgeValues.keyComparatorClassName ); 437 config.set( TezRuntimeConfiguration.TEZ_RUNTIME_PARTITIONER_CLASS, edgeValues.keyPartitionerClassName ); 438 439 setWorkingDirectory( config ); 440 441 return getPayload( config ); 442 } 443 444 private static void setWorkingDirectory( Configuration conf ) 445 { 446 String name = conf.get( JobContext.WORKING_DIR ); 447 448 if( name != null ) 449 return; 450 451 try 452 { 453 Path dir = FileSystem.get( conf ).getWorkingDirectory(); 454 conf.set( JobContext.WORKING_DIR, dir.toString() ); 455 } 456 catch( IOException exception ) 457 { 458 throw new RuntimeException( exception ); 459 } 460 } 461 462 public Vertex createVertex( FlowProcess<TezConfiguration> flowProcess, TezConfiguration initializedConfig, FlowNode flowNode ) 463 { 464 JobConf conf = new JobConf( initializedConfig ); 465 466 addInputOutputMapping( conf, flowNode ); 467 468 conf.setBoolean( "mapred.used.genericoptionsparser", true ); 469 470 Map<String, LocalResource> taskLocalResources = new HashMap<>(); 471 472 Map<FlowElement, Configuration> sourceConfigs = initFromSources( flowNode, flowProcess, conf, taskLocalResources ); 473 Map<FlowElement, Configuration> sinkConfigs = initFromSinks( flowNode, flowProcess, conf ); 474 475 initFromTraps( flowNode, flowProcess, conf ); 476 477 initFromNodeConfigDef( flowNode, conf ); 478 479 // force step to local mode if any tap is local 480 setLocalMode( initializedConfig, conf, null ); 481 482 conf.set( "cascading.flow.node.num", Integer.toString( flowNode.getOrdinal() ) ); 483 484 HadoopUtil.setIsInflow( conf ); // must be called after all taps configurations have been retrieved 485 486 int parallelism = getParallelism( flowNode, conf ); 487 488 if( parallelism == 0 ) 489 throw new FlowException( getName(), "the default number of gather partitions must be set, see cascading.flow.FlowRuntimeProps" ); 490 491 flowNode.addProcessAnnotation( FlowRuntimeProps.GATHER_PARTITIONS, Integer.toString( parallelism ) ); 492 493 Vertex vertex = newVertex( flowNode, conf, parallelism ); 494 495 if( !taskLocalResources.isEmpty() ) 496 vertex.addTaskLocalFiles( taskLocalResources ); 497 498 for( FlowElement flowElement : sourceConfigs.keySet() ) 499 { 500 if( !( flowElement instanceof Tap ) ) 501 continue; 502 503 Configuration sourceConf = sourceConfigs.get( flowElement ); 504 505 // not setting the new-api value could result in failures if not set by the Scheme 506 if( sourceConf.get( "mapred.mapper.new-api" ) == null ) 507 HadoopUtil.setNewApi( sourceConf, sourceConf.get( "mapred.input.format.class", sourceConf.get( "mapreduce.job.inputformat.class" ) ) ); 508 509 // unfortunately we cannot just load the input format and set it on the builder with also pulling all other 510 // values out of the configuration. 511 MRInput.MRInputConfigBuilder configBuilder = MRInput.createConfigBuilder( sourceConf, null ); 512 513 // the default in Tez is true, this overrides 514 if( conf.get( FlowRuntimeProps.COMBINE_SPLITS ) != null ) 515 configBuilder.groupSplits( conf.getBoolean( FlowRuntimeProps.COMBINE_SPLITS, true ) ); 516 517 // grouping splits loses file name info, breaking partition tap default impl 518 if( !CompositeTaps.unwindNarrow( PartitionTap.class, (Tap) flowElement ).isEmpty() ) // todo: generify 519 configBuilder.groupSplits( false ); 520 521 DataSourceDescriptor dataSourceDescriptor = configBuilder.build(); 522 523 vertex.addDataSource( FlowElements.id( flowElement ), dataSourceDescriptor ); 524 } 525 526 for( FlowElement flowElement : sinkConfigs.keySet() ) 527 { 528 if( !( flowElement instanceof Tap ) ) 529 continue; 530 531 Configuration sinkConf = sinkConfigs.get( flowElement ); 532 533 Class outputFormatClass; 534 String outputPath; 535 536 // we have to set sane defaults if not set by the tap 537 // typically the case of MultiSinkTap 538 String formatClassName = sinkConf.get( "mapred.output.format.class", sinkConf.get( "mapreduce.job.outputformat.class" ) ); 539 540 if( formatClassName == null ) 541 { 542 outputFormatClass = TextOutputFormat.class; // unused, use "new" api, its the default 543 outputPath = Hfs.getTempPath( sinkConf ).toString(); // unused 544 } 545 else 546 { 547 outputFormatClass = Util.loadClass( formatClassName ); 548 outputPath = getOutputPath( sinkConf ); 549 } 550 551 if( outputPath == null && getOutputPath( sinkConf ) == null && isFileOutputFormat( outputFormatClass ) ) 552 outputPath = Hfs.getTempPath( sinkConf ).toString(); // unused 553 554 MROutput.MROutputConfigBuilder configBuilder = MROutput.createConfigBuilder( sinkConf, outputFormatClass, outputPath ); 555 556 DataSinkDescriptor dataSinkDescriptor = configBuilder.build(); 557 558 vertex.addDataSink( FlowElements.id( flowElement ), dataSinkDescriptor ); 559 } 560 561 addRemoteDebug( flowNode, vertex ); 562 addRemoteProfiling( flowNode, vertex ); 563 564 if( vertex.getTaskLaunchCmdOpts() != null ) 565 flowNode.addProcessAnnotation( TezConfiguration.TEZ_TASK_LAUNCH_CMD_OPTS, vertex.getTaskLaunchCmdOpts() ); 566 567 return vertex; 568 } 569 570 protected String getOutputPath( Configuration sinkConf ) 571 { 572 return sinkConf.get( "mapred.output.dir", sinkConf.get( "mapreduce.output.fileoutputformat.outputdir" ) ); 573 } 574 575 protected boolean isFileOutputFormat( Class outputFormatClass ) 576 { 577 return org.apache.hadoop.mapred.FileOutputFormat.class.isAssignableFrom( outputFormatClass ) || 578 org.apache.hadoop.mapreduce.lib.output.FileOutputFormat.class.isAssignableFrom( outputFormatClass ); 579 } 580 581 protected int getParallelism( FlowNode flowNode, JobConf conf ) 582 { 583 // only count streamed taps, accumulated taps are always annotated 584 HashSet<Tap> sourceStreamedTaps = new HashSet<>( flowNode.getSourceTaps() ); 585 586 sourceStreamedTaps.removeAll( flowNode.getSourceElements( StreamMode.Accumulated ) ); 587 588 if( sourceStreamedTaps.size() != 0 ) 589 return -1; 590 591 int parallelism = Integer.MAX_VALUE; 592 593 for( Tap tap : flowNode.getSinkTaps() ) 594 { 595 int numSinkParts = tap.getScheme().getNumSinkParts(); 596 597 if( numSinkParts == 0 ) 598 continue; 599 600 if( parallelism != Integer.MAX_VALUE ) 601 LOG.info( "multiple sink taps in flow node declaring numSinkParts, choosing lowest value. see cascading.flow.FlowRuntimeProps for broader control." ); 602 603 parallelism = Math.min( parallelism, numSinkParts ); 604 } 605 606 if( parallelism != Integer.MAX_VALUE ) 607 return parallelism; 608 609 return conf.getInt( FlowRuntimeProps.GATHER_PARTITIONS, 0 ); 610 } 611 612 private void addInputOutputMapping( JobConf conf, FlowNode flowNode ) 613 { 614 FlowNodeGraph flowNodeGraph = getFlowNodeGraph(); 615 Set<ProcessEdge> incomingEdges = flowNodeGraph.incomingEdgesOf( flowNode ); 616 617 for( ProcessEdge processEdge : incomingEdges ) 618 conf.set( "cascading.node.source." + processEdge.getFlowElementID(), processEdge.getSourceProcessID() ); 619 620 Set<ProcessEdge> outgoingEdges = flowNodeGraph.outgoingEdgesOf( flowNode ); 621 622 for( ProcessEdge processEdge : outgoingEdges ) 623 conf.set( "cascading.node.sink." + processEdge.getFlowElementID(), processEdge.getSinkProcessID() ); 624 } 625 626 protected Map<FlowElement, Configuration> initFromSources( FlowNode flowNode, FlowProcess<TezConfiguration> flowProcess, 627 Configuration conf, Map<String, LocalResource> taskLocalResources ) 628 { 629 Set<? extends FlowElement> accumulatedSources = flowNode.getSourceElements( StreamMode.Accumulated ); 630 631 for( FlowElement element : accumulatedSources ) 632 { 633 if( element instanceof Tap ) 634 { 635 JobConf current = new JobConf( conf ); 636 Tap tap = (Tap) element; 637 638 if( tap.getIdentifier() == null ) 639 throw new IllegalStateException( "tap may not have null identifier: " + tap.toString() ); 640 641 tap.sourceConfInit( flowProcess, current ); 642 643 Collection<String> paths = current.getStringCollection( CASCADING_LOCAL_RESOURCES + Tap.id( tap ) ); 644 645 if( !paths.isEmpty() ) 646 { 647 String flowStagingPath = ( (Hadoop2TezFlow) getFlow() ).getFlowStagingPath(); 648 String resourceSubPath = Tap.id( tap ); 649 Map<Path, Path> pathMap = TezUtil.addToClassPath( current, flowStagingPath, resourceSubPath, paths, LocalResourceType.FILE, taskLocalResources, null ); 650 651 current.setStrings( CASCADING_REMOTE_RESOURCES + Tap.id( tap ), taskLocalResources.keySet().toArray( new String[ taskLocalResources.size() ] ) ); 652 653 allLocalResources.putAll( taskLocalResources ); 654 syncPaths.putAll( pathMap ); 655 } 656 657 Map<String, String> map = flowProcess.diffConfigIntoMap( new TezConfiguration( conf ), new TezConfiguration( current ) ); 658 conf.set( "cascading.node.accumulated.source.conf." + Tap.id( tap ), pack( map, conf ) ); 659 660 setLocalMode( conf, current, tap ); 661 } 662 } 663 664 Set<FlowElement> sources = new HashSet<>( flowNode.getSourceElements() ); 665 666 sources.removeAll( accumulatedSources ); 667 668 if( sources.isEmpty() ) 669 throw new IllegalStateException( "all sources marked as accumulated" ); 670 671 Map<FlowElement, Configuration> configs = new HashMap<>(); 672 673 for( FlowElement element : sources ) 674 { 675 JobConf current = new JobConf( conf ); 676 677 String id = FlowElements.id( element ); 678 679 current.set( "cascading.node.source", id ); 680 681 if( element instanceof Tap ) 682 { 683 Tap tap = (Tap) element; 684 685 if( tap.getIdentifier() == null ) 686 throw new IllegalStateException( "tap may not have null identifier: " + tap.toString() ); 687 688 tap.sourceConfInit( flowProcess, current ); 689 690 setLocalMode( conf, current, tap ); 691 } 692 693 configs.put( element, current ); 694 } 695 696 return configs; 697 } 698 699 protected Map<FlowElement, Configuration> initFromSinks( FlowNode flowNode, FlowProcess<? extends Configuration> flowProcess, Configuration conf ) 700 { 701 Set<FlowElement> sinks = flowNode.getSinkElements(); 702 Map<FlowElement, Configuration> configs = new HashMap<>(); 703 704 for( FlowElement element : sinks ) 705 { 706 JobConf current = new JobConf( conf ); 707 708 if( element instanceof Tap ) 709 { 710 Tap tap = (Tap) element; 711 712 if( tap.getIdentifier() == null ) 713 throw new IllegalStateException( "tap may not have null identifier: " + element.toString() ); 714 715 tap.sinkConfInit( flowProcess, current ); 716 717 setLocalMode( conf, current, tap ); 718 } 719 720 String id = FlowElements.id( element ); 721 722 current.set( "cascading.node.sink", id ); 723 724 configs.put( element, current ); 725 } 726 727 return configs; 728 } 729 730 private void initFromNodeConfigDef( FlowNode flowNode, Configuration conf ) 731 { 732 initConfFromNodeConfigDef( flowNode.getElementGraph(), new ConfigurationSetter( conf ) ); 733 } 734 735 private void initFromStepConfigDef( Configuration conf ) 736 { 737 initConfFromStepConfigDef( new ConfigurationSetter( conf ) ); 738 } 739 740 protected void initFromTraps( FlowNode flowNode, FlowProcess<? extends Configuration> flowProcess, Configuration conf ) 741 { 742 Map<String, Tap> traps = flowNode.getTrapMap(); 743 744 if( !traps.isEmpty() ) 745 { 746 JobConf trapConf = new JobConf( conf ); 747 748 for( Tap tap : traps.values() ) 749 { 750 tap.sinkConfInit( flowProcess, trapConf ); 751 setLocalMode( conf, trapConf, tap ); 752 } 753 } 754 } 755 756 private Vertex newVertex( FlowNode flowNode, Configuration conf, int parallelism ) 757 { 758 conf.set( FlowNode.CASCADING_FLOW_NODE, pack( flowNode, conf ) ); // todo: pack into payload directly 759 760 ProcessorDescriptor descriptor = ProcessorDescriptor.create( FlowProcessor.class.getName() ); 761 762 descriptor.setUserPayload( getPayload( conf ) ); 763 764 Vertex vertex = Vertex.create( flowNode.getID(), descriptor, parallelism ); 765 766 if( environment != null ) 767 vertex.setTaskEnvironment( environment ); 768 769 return vertex; 770 } 771 772 private UserPayload getPayload( Configuration conf ) 773 { 774 try 775 { 776 return TezUtils.createUserPayloadFromConf( conf ); 777 } 778 catch( IOException exception ) 779 { 780 throw new CascadingException( exception ); 781 } 782 } 783 784 private String pack( Object object, Configuration conf ) 785 { 786 try 787 { 788 return serializeBase64( object, conf, true ); 789 } 790 catch( IOException exception ) 791 { 792 throw new FlowException( "unable to pack object: " + object.getClass().getCanonicalName(), exception ); 793 } 794 } 795 796 @Override 797 public void clean( TezConfiguration config ) 798 { 799 for( Tap sink : getSinkTaps() ) 800 { 801 if( sink.isTemporary() && ( getFlow().getFlowStats().isSuccessful() || getFlow().getRunID() == null ) ) 802 { 803 try 804 { 805 sink.deleteResource( config ); 806 } 807 catch( Exception exception ) 808 { 809 // sink all exceptions, don't fail app 810 logWarn( "unable to remove temporary file: " + sink, exception ); 811 } 812 } 813 else 814 { 815 cleanTapMetaData( config, sink ); 816 } 817 } 818 819 for( Tap tap : getTraps() ) 820 cleanTapMetaData( config, tap ); 821 } 822 823 private void cleanTapMetaData( TezConfiguration config, Tap tap ) 824 { 825 try 826 { 827 Hadoop18TapUtil.cleanupTapMetaData( config, tap ); 828 } 829 catch( IOException exception ) 830 { 831 // ignore exception 832 } 833 } 834 835 public void syncArtifacts() 836 { 837 // this may not be strictly necessary, but there is a condition where setting the access time 838 // fails, so there may be one were setting the modification time fails. if so, we can compensate. 839 Map<String, Long> timestamps = HadoopUtil.syncPaths( getConfig(), syncPaths, true ); 840 841 for( Map.Entry<String, Long> entry : timestamps.entrySet() ) 842 { 843 LocalResource localResource = allLocalResources.get( entry.getKey() ); 844 845 if( localResource != null ) 846 localResource.setTimestamp( entry.getValue() ); 847 } 848 } 849 850 private void setLocalMode( Configuration parent, JobConf current, Tap tap ) 851 { 852 // force step to local mode 853 if( !HadoopUtil.isLocal( current ) ) 854 return; 855 856 if( tap != null ) 857 logInfo( "tap forcing step to tez local mode: " + tap.getIdentifier() ); 858 859 HadoopUtil.setLocal( parent ); 860 } 861 862 private void addRemoteDebug( FlowNode flowNode, Vertex vertex ) 863 { 864 String value = System.getProperty( "test.debug.node", null ); 865 866 if( Util.isEmpty( value ) ) 867 return; 868 869 if( !flowNode.getSourceElementNames().contains( value ) && asInt( value ) != flowNode.getOrdinal() ) 870 return; 871 872 LOG.warn( "remote debugging enabled with property: {}, on node: {}, with node id: {}", "test.debug.node", value, flowNode.getID() ); 873 874 String opts = vertex.getTaskLaunchCmdOpts(); 875 876 if( opts == null ) 877 opts = ""; 878 879 String address = System.getProperty( "test.debug.address", "localhost:5005" ).trim(); 880 881 opts += " -agentlib:jdwp=transport=dt_socket,server=n,address=" + address + ",suspend=y"; 882 883 vertex.setTaskLaunchCmdOpts( opts ); 884 } 885 886 private void addRemoteProfiling( FlowNode flowNode, Vertex vertex ) 887 { 888 String value = System.getProperty( "test.profile.node", null ); 889 890 if( Util.isEmpty( value ) ) 891 return; 892 893 if( !flowNode.getSourceElementNames().contains( value ) && asInt( value ) != flowNode.getOrdinal() ) 894 return; 895 896 LOG.warn( "remote profiling enabled with property: {}, on node: {}, with node id: {}", "test.profile.node", value, flowNode.getID() ); 897 898 String opts = vertex.getTaskLaunchCmdOpts(); 899 900 if( opts == null ) 901 opts = ""; 902 903 String path = System.getProperty( "test.profile.path", "/tmp/jfr/" ); 904 905 if( !path.endsWith( "/" ) ) 906 path += "/"; 907 908 LOG.warn( "remote profiling property: {}, logging to path: {}", "test.profile.path", path ); 909 910 opts += String.format( " -XX:+UnlockCommercialFeatures -XX:+FlightRecorder -XX:FlightRecorderOptions=defaultrecording=true,dumponexit=true,dumponexitpath=%1$s%2$s,disk=true,repository=%1$s%2$s", path, flowNode.getID() ); 911 912 vertex.setTaskLaunchCmdOpts( opts ); 913 } 914 915 private int asInt( String value ) 916 { 917 try 918 { 919 return Integer.parseInt( value ); 920 } 921 catch( NumberFormatException exception ) 922 { 923 return -1; 924 } 925 } 926 927 public Map<String, LocalResource> getAllLocalResources() 928 { 929 return allLocalResources; 930 } 931 932 private static class EdgeValues 933 { 934 FlowElement flowElement; 935 TezConfiguration config; 936 Set<Integer> ordinals; 937 String keyClassName; 938 String valueClassName; 939 String keyComparatorClassName; 940 String keyPartitionerClassName; 941 String outputClassName; 942 String inputClassName; 943 EdgeProperty.DataMovementType movementType; 944 EdgeProperty.DataSourceType sourceType; 945 EdgeProperty.SchedulingType schedulingType; 946 947 Map<Integer, Fields> resolvedKeyFieldsMap; 948 Map<Integer, Fields> resolvedSortFieldsMap; 949 Map<Integer, Fields> resolvedValueFieldsMap; 950 951 private EdgeValues( TezConfiguration config, ProcessEdge processEdge ) 952 { 953 this.config = config; 954 this.flowElement = processEdge.getFlowElement(); 955 this.ordinals = processEdge.getSourceProvidedOrdinals(); 956 957 this.resolvedKeyFieldsMap = processEdge.getResolvedKeyFields(); 958 this.resolvedSortFieldsMap = processEdge.getResolvedSortFields(); 959 this.resolvedValueFieldsMap = processEdge.getResolvedValueFields(); 960 } 961 962 public FlowElement getFlowElement() 963 { 964 return flowElement; 965 } 966 967 public TezConfiguration getConfig() 968 { 969 return config; 970 } 971 972 public Set getOrdinals() 973 { 974 return ordinals; 975 } 976 977 public String getKeyClassName() 978 { 979 return keyClassName; 980 } 981 982 public String getValueClassName() 983 { 984 return valueClassName; 985 } 986 987 public String getKeyComparatorClassName() 988 { 989 return keyComparatorClassName; 990 } 991 992 public String getKeyPartitionerClassName() 993 { 994 return keyPartitionerClassName; 995 } 996 997 public String getOutputClassName() 998 { 999 return outputClassName; 1000 } 1001 1002 public String getInputClassName() 1003 { 1004 return inputClassName; 1005 } 1006 1007 public EdgeProperty.DataMovementType getMovementType() 1008 { 1009 return movementType; 1010 } 1011 1012 public EdgeProperty.DataSourceType getSourceType() 1013 { 1014 return sourceType; 1015 } 1016 1017 public EdgeProperty.SchedulingType getSchedulingType() 1018 { 1019 return schedulingType; 1020 } 1021 1022 public Map<Integer, Fields> getResolvedKeyFieldsMap() 1023 { 1024 return resolvedKeyFieldsMap; 1025 } 1026 1027 public Map<Integer, Fields> getResolvedSortFieldsMap() 1028 { 1029 return resolvedSortFieldsMap; 1030 } 1031 1032 public Map<Integer, Fields> getResolvedValueFieldsMap() 1033 { 1034 return resolvedValueFieldsMap; 1035 } 1036 } 1037 }