private DAGPlan createDAG() { // Create a plan with 3 vertices: A, B, C. Group(A,B)->C Configuration conf = new Configuration(false); int dummyTaskCount = 1; Resource dummyTaskResource = Resource.newInstance(1, 1); org.apache.tez.dag.api.Vertex v1 = new org.apache.tez.dag.api.Vertex( "vertex1", new ProcessorDescriptor("Processor").setHistoryText("vertex1 Processor HistoryText"), dummyTaskCount, dummyTaskResource); v1.addInput( "input1", new InputDescriptor("input.class").setHistoryText("input HistoryText"), null); org.apache.tez.dag.api.Vertex v2 = new org.apache.tez.dag.api.Vertex( "vertex2", new ProcessorDescriptor("Processor").setHistoryText("vertex2 Processor HistoryText"), dummyTaskCount, dummyTaskResource); org.apache.tez.dag.api.Vertex v3 = new org.apache.tez.dag.api.Vertex( "vertex3", new ProcessorDescriptor("Processor").setHistoryText("vertex3 Processor HistoryText"), dummyTaskCount, dummyTaskResource); DAG dag = new DAG("testDag"); String groupName1 = "uv12"; org.apache.tez.dag.api.VertexGroup uv12 = dag.createVertexGroup(groupName1, v1, v2); OutputDescriptor outDesc = new OutputDescriptor("output.class").setHistoryText("uvOut HistoryText"); uv12.addOutput("uvOut", outDesc, OutputCommitter.class); v3.addOutput("uvOut", outDesc, OutputCommitter.class); GroupInputEdge e1 = new GroupInputEdge( uv12, v3, new EdgeProperty( DataMovementType.SCATTER_GATHER, DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, new OutputDescriptor("dummy output class").setHistoryText("Dummy History Text"), new InputDescriptor("dummy input class").setHistoryText("Dummy History Text")), new InputDescriptor("merge.class").setHistoryText("Merge HistoryText")); dag.addVertex(v1); dag.addVertex(v2); dag.addVertex(v3); dag.addEdge(e1); return dag.createDag(conf); }
@Test(timeout = 10000) public void testBasicSpeculationPerVertexConf() throws Exception { DAG dag = DAG.create("test"); String vNameNoSpec = "A"; String vNameSpec = "B"; Vertex vA = Vertex.create(vNameNoSpec, ProcessorDescriptor.create("Proc.class"), 5); Vertex vB = Vertex.create(vNameSpec, ProcessorDescriptor.create("Proc.class"), 5); vA.setConf(TezConfiguration.TEZ_AM_SPECULATION_ENABLED, "false"); dag.addVertex(vA); dag.addVertex(vB); // min/max src fraction is set to 1. So vertices will run sequentially dag.addEdge( Edge.create( vA, vB, EdgeProperty.create( DataMovementType.SCATTER_GATHER, DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, OutputDescriptor.create("O"), InputDescriptor.create("I")))); MockTezClient tezClient = createTezSession(); DAGClient dagClient = tezClient.submitDAG(dag); DAGImpl dagImpl = (DAGImpl) mockApp.getContext().getCurrentDAG(); TezVertexID vertexId = dagImpl.getVertex(vNameSpec).getVertexId(); TezVertexID vertexIdNoSpec = dagImpl.getVertex(vNameNoSpec).getVertexId(); // original attempt is killed and speculative one is successful TezTaskAttemptID killedTaId = TezTaskAttemptID.getInstance(TezTaskID.getInstance(vertexId, 0), 0); TezTaskAttemptID noSpecTaId = TezTaskAttemptID.getInstance(TezTaskID.getInstance(vertexIdNoSpec, 0), 0); // cause speculation trigger for both mockLauncher.setStatusUpdatesForTask(killedTaId, 100); mockLauncher.setStatusUpdatesForTask(noSpecTaId, 100); mockLauncher.startScheduling(true); dagClient.waitForCompletion(); Assert.assertEquals(DAGStatus.State.SUCCEEDED, dagClient.getDAGStatus(null).getState()); org.apache.tez.dag.app.dag.Vertex vSpec = dagImpl.getVertex(vertexId); org.apache.tez.dag.app.dag.Vertex vNoSpec = dagImpl.getVertex(vertexIdNoSpec); // speculation for vA but not for vB Assert.assertTrue( vSpec.getAllCounters().findCounter(TaskCounter.NUM_SPECULATIONS).getValue() > 0); Assert.assertEquals( 0, vNoSpec.getAllCounters().findCounter(TaskCounter.NUM_SPECULATIONS).getValue()); tezClient.stop(); }
public DAG createDAG( FileSystem remoteFs, Configuration conf, Path remoteStagingDir, int numMapper, int numReducer, int iReduceStagesCount, int numIReducer, long mapSleepTime, int mapSleepCount, long reduceSleepTime, int reduceSleepCount, long iReduceSleepTime, int iReduceSleepCount, boolean writeSplitsToDFS, boolean generateSplitsInAM) throws IOException, YarnException { Configuration mapStageConf = new JobConf(conf); mapStageConf.setInt(MRJobConfig.NUM_MAPS, numMapper); mapStageConf.setLong(MAP_SLEEP_TIME, mapSleepTime); mapStageConf.setLong(REDUCE_SLEEP_TIME, reduceSleepTime); mapStageConf.setLong(IREDUCE_SLEEP_TIME, iReduceSleepTime); mapStageConf.setInt(MAP_SLEEP_COUNT, mapSleepCount); mapStageConf.setInt(REDUCE_SLEEP_COUNT, reduceSleepCount); mapStageConf.setInt(IREDUCE_SLEEP_COUNT, iReduceSleepCount); mapStageConf.setInt(IREDUCE_STAGES_COUNT, iReduceStagesCount); mapStageConf.setInt(IREDUCE_TASKS_COUNT, numIReducer); mapStageConf.set(MRJobConfig.MAP_CLASS_ATTR, SleepMapper.class.getName()); mapStageConf.set(MRJobConfig.INPUT_FORMAT_CLASS_ATTR, SleepInputFormat.class.getName()); if (numIReducer == 0 && numReducer == 0) { mapStageConf.set(MRJobConfig.OUTPUT_FORMAT_CLASS_ATTR, NullOutputFormat.class.getName()); } MRHelpers.translateVertexConfToTez(mapStageConf); Configuration[] intermediateReduceStageConfs = null; if (iReduceStagesCount > 0 && numIReducer > 0) { intermediateReduceStageConfs = new JobConf[iReduceStagesCount]; for (int i = 1; i <= iReduceStagesCount; ++i) { JobConf iReduceStageConf = new JobConf(conf); iReduceStageConf.setLong(MRRSleepJob.REDUCE_SLEEP_TIME, iReduceSleepTime); iReduceStageConf.setInt(MRRSleepJob.REDUCE_SLEEP_COUNT, iReduceSleepCount); iReduceStageConf.setInt(MRJobConfig.NUM_REDUCES, numIReducer); iReduceStageConf.set(MRJobConfig.REDUCE_CLASS_ATTR, ISleepReducer.class.getName()); iReduceStageConf.set(MRJobConfig.MAP_OUTPUT_KEY_CLASS, IntWritable.class.getName()); iReduceStageConf.set(MRJobConfig.MAP_OUTPUT_VALUE_CLASS, IntWritable.class.getName()); iReduceStageConf.set( MRJobConfig.PARTITIONER_CLASS_ATTR, MRRSleepJobPartitioner.class.getName()); MRHelpers.translateVertexConfToTez(iReduceStageConf); intermediateReduceStageConfs[i - 1] = iReduceStageConf; } } Configuration finalReduceConf = null; if (numReducer > 0) { finalReduceConf = new JobConf(conf); finalReduceConf.setLong(MRRSleepJob.REDUCE_SLEEP_TIME, reduceSleepTime); finalReduceConf.setInt(MRRSleepJob.REDUCE_SLEEP_COUNT, reduceSleepCount); finalReduceConf.setInt(MRJobConfig.NUM_REDUCES, numReducer); finalReduceConf.set(MRJobConfig.REDUCE_CLASS_ATTR, SleepReducer.class.getName()); finalReduceConf.set(MRJobConfig.MAP_OUTPUT_KEY_CLASS, IntWritable.class.getName()); finalReduceConf.set(MRJobConfig.MAP_OUTPUT_VALUE_CLASS, IntWritable.class.getName()); finalReduceConf.set(MRJobConfig.OUTPUT_FORMAT_CLASS_ATTR, NullOutputFormat.class.getName()); MRHelpers.translateVertexConfToTez(finalReduceConf); } MRHelpers.doJobClientMagic(mapStageConf); if (iReduceStagesCount > 0 && numIReducer > 0) { for (int i = 0; i < iReduceStagesCount; ++i) { MRHelpers.doJobClientMagic(intermediateReduceStageConfs[i]); } } if (numReducer > 0) { MRHelpers.doJobClientMagic(finalReduceConf); } InputSplitInfo inputSplitInfo = null; if (!generateSplitsInAM) { if (writeSplitsToDFS) { LOG.info("Writing splits to DFS"); try { inputSplitInfo = MRHelpers.generateInputSplits(mapStageConf, remoteStagingDir); } catch (InterruptedException e) { throw new TezUncheckedException("Could not generate input splits", e); } catch (ClassNotFoundException e) { throw new TezUncheckedException("Failed to generate input splits", e); } } else { try { LOG.info("Creating in-mem splits"); inputSplitInfo = MRHelpers.generateInputSplitsToMem(mapStageConf); } catch (ClassNotFoundException e) { throw new TezUncheckedException("Could not generate input splits", e); } catch (InterruptedException e) { throw new TezUncheckedException("Could not generate input splits", e); } } if (inputSplitInfo.getCredentials() != null) { this.credentials.addAll(inputSplitInfo.getCredentials()); } } DAG dag = new DAG("MRRSleepJob"); String jarPath = ClassUtil.findContainingJar(getClass()); if (jarPath == null) { throw new TezUncheckedException( "Could not find any jar containing" + " MRRSleepJob.class in the classpath"); } Path remoteJarPath = remoteFs.makeQualified(new Path(remoteStagingDir, "dag_job.jar")); remoteFs.copyFromLocalFile(new Path(jarPath), remoteJarPath); FileStatus jarFileStatus = remoteFs.getFileStatus(remoteJarPath); TokenCache.obtainTokensForNamenodes(this.credentials, new Path[] {remoteJarPath}, mapStageConf); Map<String, LocalResource> commonLocalResources = new HashMap<String, LocalResource>(); LocalResource dagJarLocalRsrc = LocalResource.newInstance( ConverterUtils.getYarnUrlFromPath(remoteJarPath), LocalResourceType.FILE, LocalResourceVisibility.APPLICATION, jarFileStatus.getLen(), jarFileStatus.getModificationTime()); commonLocalResources.put("dag_job.jar", dagJarLocalRsrc); List<Vertex> vertices = new ArrayList<Vertex>(); byte[] mapInputPayload = null; byte[] mapUserPayload = MRHelpers.createUserPayloadFromConf(mapStageConf); if (writeSplitsToDFS || generateSplitsInAM) { mapInputPayload = MRHelpers.createMRInputPayload(mapUserPayload, null); } else { mapInputPayload = MRHelpers.createMRInputPayload(mapUserPayload, inputSplitInfo.getSplitsProto()); } int numTasks = generateSplitsInAM ? -1 : numMapper; Vertex mapVertex = new Vertex( "map", new ProcessorDescriptor(MapProcessor.class.getName()).setUserPayload(mapUserPayload), numTasks, MRHelpers.getMapResource(mapStageConf)); if (!generateSplitsInAM) { mapVertex.setTaskLocationsHint(inputSplitInfo.getTaskLocationHints()); } if (writeSplitsToDFS) { Map<String, LocalResource> mapLocalResources = new HashMap<String, LocalResource>(); mapLocalResources.putAll(commonLocalResources); MRHelpers.updateLocalResourcesForInputSplits(remoteFs, inputSplitInfo, mapLocalResources); mapVertex.setTaskLocalFiles(mapLocalResources); } else { mapVertex.setTaskLocalFiles(commonLocalResources); } if (generateSplitsInAM) { MRHelpers.addMRInput(mapVertex, mapInputPayload, MRInputAMSplitGenerator.class); } else { if (writeSplitsToDFS) { MRHelpers.addMRInput(mapVertex, mapInputPayload, null); } else { MRHelpers.addMRInput(mapVertex, mapInputPayload, MRInputSplitDistributor.class); } } vertices.add(mapVertex); if (iReduceStagesCount > 0 && numIReducer > 0) { for (int i = 0; i < iReduceStagesCount; ++i) { Configuration iconf = intermediateReduceStageConfs[i]; byte[] iReduceUserPayload = MRHelpers.createUserPayloadFromConf(iconf); Vertex ivertex = new Vertex( "ireduce" + (i + 1), new ProcessorDescriptor(ReduceProcessor.class.getName()) .setUserPayload(iReduceUserPayload), numIReducer, MRHelpers.getReduceResource(iconf)); ivertex.setTaskLocalFiles(commonLocalResources); vertices.add(ivertex); } } Vertex finalReduceVertex = null; if (numReducer > 0) { byte[] reducePayload = MRHelpers.createUserPayloadFromConf(finalReduceConf); finalReduceVertex = new Vertex( "reduce", new ProcessorDescriptor(ReduceProcessor.class.getName()) .setUserPayload(reducePayload), numReducer, MRHelpers.getReduceResource(finalReduceConf)); finalReduceVertex.setTaskLocalFiles(commonLocalResources); MRHelpers.addMROutputLegacy(finalReduceVertex, reducePayload); vertices.add(finalReduceVertex); } else { // Map only job MRHelpers.addMROutputLegacy(mapVertex, mapUserPayload); } Configuration partitionerConf = new Configuration(false); partitionerConf.set(MRJobConfig.PARTITIONER_CLASS_ATTR, MRRSleepJobPartitioner.class.getName()); OrderedPartitionedKVEdgeConfigurer edgeConf = OrderedPartitionedKVEdgeConfigurer.newBuilder( IntWritable.class.getName(), IntWritable.class.getName(), MRPartitioner.class.getName(), partitionerConf) .configureInput() .useLegacyInput() .done() .build(); for (int i = 0; i < vertices.size(); ++i) { dag.addVertex(vertices.get(i)); if (i != 0) { dag.addEdge( new Edge(vertices.get(i - 1), vertices.get(i), edgeConf.createDefaultEdgeProperty())); } } return dag; }