private Map<ExecutionAttemptID, Execution> setupExecution( JobVertex v1, int dop1, JobVertex v2, int dop2) throws Exception { final JobID jobId = new JobID(); v1.setParallelism(dop1); v2.setParallelism(dop2); v1.setInvokableClass(BatchTask.class); v2.setInvokableClass(BatchTask.class); // execution graph that executes actions synchronously ExecutionGraph eg = new ExecutionGraph( TestingUtils.directExecutionContext(), jobId, "some job", new Configuration(), new SerializedValue<>(new ExecutionConfig()), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); eg.setQueuedSchedulingAllowed(false); List<JobVertex> ordered = Arrays.asList(v1, v2); eg.attachJobGraph(ordered); Scheduler scheduler = new Scheduler(TestingUtils.defaultExecutionContext()); for (int i = 0; i < dop1 + dop2; i++) { scheduler.newInstanceAvailable( ExecutionGraphTestUtils.getInstance( new ExecutionGraphTestUtils.SimpleActorGateway( TestingUtils.directExecutionContext()))); } assertEquals(dop1 + dop2, scheduler.getNumberOfAvailableSlots()); // schedule, this triggers mock deployment eg.scheduleForExecution(scheduler); Map<ExecutionAttemptID, Execution> executions = eg.getRegisteredExecutions(); assertEquals(dop1 + dop2, executions.size()); return executions; }
@Test public void testBuildDeploymentDescriptor() { try { final JobID jobId = new JobID(); final JobVertexID jid1 = new JobVertexID(); final JobVertexID jid2 = new JobVertexID(); final JobVertexID jid3 = new JobVertexID(); final JobVertexID jid4 = new JobVertexID(); JobVertex v1 = new JobVertex("v1", jid1); JobVertex v2 = new JobVertex("v2", jid2); JobVertex v3 = new JobVertex("v3", jid3); JobVertex v4 = new JobVertex("v4", jid4); v1.setParallelism(10); v2.setParallelism(10); v3.setParallelism(10); v4.setParallelism(10); v1.setInvokableClass(BatchTask.class); v2.setInvokableClass(BatchTask.class); v3.setInvokableClass(BatchTask.class); v4.setInvokableClass(BatchTask.class); v2.connectNewDataSetAsInput(v1, DistributionPattern.ALL_TO_ALL); v3.connectNewDataSetAsInput(v2, DistributionPattern.ALL_TO_ALL); v4.connectNewDataSetAsInput(v2, DistributionPattern.ALL_TO_ALL); ExecutionGraph eg = new ExecutionGraph( TestingUtils.defaultExecutionContext(), jobId, "some job", new Configuration(), new SerializedValue<>(new ExecutionConfig()), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); List<JobVertex> ordered = Arrays.asList(v1, v2, v3, v4); eg.attachJobGraph(ordered); ExecutionJobVertex ejv = eg.getAllVertices().get(jid2); ExecutionVertex vertex = ejv.getTaskVertices()[3]; ExecutionGraphTestUtils.SimpleActorGateway instanceGateway = new ExecutionGraphTestUtils.SimpleActorGateway(TestingUtils.directExecutionContext()); final Instance instance = getInstance(instanceGateway); final SimpleSlot slot = instance.allocateSimpleSlot(jobId); assertEquals(ExecutionState.CREATED, vertex.getExecutionState()); vertex.deployToSlot(slot); assertEquals(ExecutionState.DEPLOYING, vertex.getExecutionState()); TaskDeploymentDescriptor descr = instanceGateway.lastTDD; assertNotNull(descr); assertEquals(jobId, descr.getJobID()); assertEquals(jid2, descr.getVertexID()); assertEquals(3, descr.getIndexInSubtaskGroup()); assertEquals(10, descr.getNumberOfSubtasks()); assertEquals(BatchTask.class.getName(), descr.getInvokableClassName()); assertEquals("v2", descr.getTaskName()); List<ResultPartitionDeploymentDescriptor> producedPartitions = descr.getProducedPartitions(); List<InputGateDeploymentDescriptor> consumedPartitions = descr.getInputGates(); assertEquals(2, producedPartitions.size()); assertEquals(1, consumedPartitions.size()); assertEquals(10, producedPartitions.get(0).getNumberOfSubpartitions()); assertEquals(10, producedPartitions.get(1).getNumberOfSubpartitions()); assertEquals(10, consumedPartitions.get(0).getInputChannelDeploymentDescriptors().length); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); } }
@Test /** * Tests that a blocking batch job fails if there are not enough resources left to schedule the * succeeding tasks. This test case is related to [FLINK-4296] where finished producing tasks * swallow the fail exception when scheduling a consumer task. */ public void testNoResourceAvailableFailure() throws Exception { final JobID jobId = new JobID(); JobVertex v1 = new JobVertex("source"); JobVertex v2 = new JobVertex("sink"); int dop1 = 1; int dop2 = 1; v1.setParallelism(dop1); v2.setParallelism(dop2); v1.setInvokableClass(BatchTask.class); v2.setInvokableClass(BatchTask.class); v2.connectNewDataSetAsInput( v1, DistributionPattern.POINTWISE, ResultPartitionType.BLOCKING, false); // execution graph that executes actions synchronously ExecutionGraph eg = new ExecutionGraph( TestingUtils.directExecutionContext(), jobId, "failing test job", new Configuration(), new SerializedValue<>(new ExecutionConfig()), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); eg.setQueuedSchedulingAllowed(false); List<JobVertex> ordered = Arrays.asList(v1, v2); eg.attachJobGraph(ordered); Scheduler scheduler = new Scheduler(TestingUtils.directExecutionContext()); for (int i = 0; i < dop1; i++) { scheduler.newInstanceAvailable( ExecutionGraphTestUtils.getInstance( new ExecutionGraphTestUtils.SimpleActorGateway( TestingUtils.directExecutionContext()))); } assertEquals(dop1, scheduler.getNumberOfAvailableSlots()); // schedule, this triggers mock deployment eg.scheduleForExecution(scheduler); ExecutionAttemptID attemptID = eg.getJobVertex(v1.getID()) .getTaskVertices()[0] .getCurrentExecutionAttempt() .getAttemptId(); eg.updateState(new TaskExecutionState(jobId, attemptID, ExecutionState.RUNNING)); eg.updateState( new TaskExecutionState( jobId, attemptID, ExecutionState.FINISHED, null, new AccumulatorSnapshot( jobId, attemptID, new HashMap<AccumulatorRegistry.Metric, Accumulator<?, ?>>(), new HashMap<String, Accumulator<?, ?>>()))); assertEquals(JobStatus.FAILED, eg.getState()); }
/* * Test setup: * - v1 is isolated, no slot sharing * - v2 and v3 (not connected) share slots * - v4 and v5 (connected) share slots */ @Test public void testAssignSlotSharingGroup() { try { JobVertex v1 = new JobVertex("v1"); JobVertex v2 = new JobVertex("v2"); JobVertex v3 = new JobVertex("v3"); JobVertex v4 = new JobVertex("v4"); JobVertex v5 = new JobVertex("v5"); v1.setParallelism(4); v2.setParallelism(5); v3.setParallelism(7); v4.setParallelism(1); v5.setParallelism(11); v2.connectNewDataSetAsInput(v1, DistributionPattern.POINTWISE); v5.connectNewDataSetAsInput(v4, DistributionPattern.POINTWISE); SlotSharingGroup jg1 = new SlotSharingGroup(); v2.setSlotSharingGroup(jg1); v3.setSlotSharingGroup(jg1); SlotSharingGroup jg2 = new SlotSharingGroup(); v4.setSlotSharingGroup(jg2); v5.setSlotSharingGroup(jg2); List<JobVertex> vertices = new ArrayList<JobVertex>(Arrays.asList(v1, v2, v3, v4, v5)); ExecutionGraph eg = new ExecutionGraph( TestingUtils.defaultExecutionContext(), new JobID(), "test job", new Configuration(), ExecutionConfigTest.getSerializedConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); eg.attachJobGraph(vertices); // verify that the vertices are all in the same slot sharing group SlotSharingGroup group1 = null; SlotSharingGroup group2 = null; // verify that v1 tasks have no slot sharing group assertNull(eg.getJobVertex(v1.getID()).getSlotSharingGroup()); // v2 and v3 are shared group1 = eg.getJobVertex(v2.getID()).getSlotSharingGroup(); assertNotNull(group1); assertEquals(group1, eg.getJobVertex(v3.getID()).getSlotSharingGroup()); assertEquals(2, group1.getJobVertexIds().size()); assertTrue(group1.getJobVertexIds().contains(v2.getID())); assertTrue(group1.getJobVertexIds().contains(v3.getID())); // v4 and v5 are shared group2 = eg.getJobVertex(v4.getID()).getSlotSharingGroup(); assertNotNull(group2); assertEquals(group2, eg.getJobVertex(v5.getID()).getSlotSharingGroup()); assertEquals(2, group1.getJobVertexIds().size()); assertTrue(group2.getJobVertexIds().contains(v4.getID())); assertTrue(group2.getJobVertexIds().contains(v5.getID())); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); } }