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 /** * 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()); }