8181import static org .apache .flink .runtime .scheduler .adaptive .WaitingForResourcesTest .assertNonNull ;
8282import static org .hamcrest .CoreMatchers .instanceOf ;
8383import static org .hamcrest .CoreMatchers .is ;
84+ import static org .hamcrest .CoreMatchers .not ;
8485import static org .hamcrest .CoreMatchers .notNullValue ;
8586import static org .junit .Assert .assertThat ;
8687
@@ -90,17 +91,60 @@ public class ExecutingTest extends TestLogger {
9091 @ Test
9192 public void testExecutionGraphDeploymentOnEnter () throws Exception {
9293 try (MockExecutingContext ctx = new MockExecutingContext ()) {
93- MockExecutionJobVertex mockExecutionJobVertex = new MockExecutionJobVertex ();
94+ MockExecutionJobVertex mockExecutionJobVertex =
95+ new MockExecutionJobVertex (MockExecutionVertex ::new );
96+ MockExecutionVertex mockExecutionVertex =
97+ (MockExecutionVertex ) mockExecutionJobVertex .getMockExecutionVertex ();
98+ mockExecutionVertex .setMockedExecutionState (ExecutionState .CREATED );
9499 ExecutionGraph executionGraph =
95100 new MockExecutionGraph (() -> Collections .singletonList (mockExecutionJobVertex ));
96101 Executing exec =
97102 new ExecutingStateBuilder ().setExecutionGraph (executionGraph ).build (ctx );
98103
99- assertThat (mockExecutionJobVertex . isExecutionDeployed (), is (true ));
104+ assertThat (mockExecutionVertex . isDeployCalled (), is (true ));
100105 assertThat (executionGraph .getState (), is (JobStatus .RUNNING ));
101106 }
102107 }
103108
109+ @ Test
110+ public void testNoDeploymentCallOnEnterWhenVertexRunning () throws Exception {
111+ try (MockExecutingContext ctx = new MockExecutingContext ()) {
112+ MockExecutionJobVertex mockExecutionJobVertex =
113+ new MockExecutionJobVertex (MockExecutionVertex ::new );
114+ ExecutionGraph executionGraph =
115+ new MockExecutionGraph (() -> Collections .singletonList (mockExecutionJobVertex ));
116+ executionGraph .transitionToRunning ();
117+ final MockExecutionVertex mockExecutionVertex =
118+ ((MockExecutionVertex ) mockExecutionJobVertex .getMockExecutionVertex ());
119+ mockExecutionVertex .setMockedExecutionState (ExecutionState .RUNNING );
120+
121+ new Executing (
122+ executionGraph ,
123+ getExecutionGraphHandler (executionGraph , ctx .getMainThreadExecutor ()),
124+ new TestingOperatorCoordinatorHandler (),
125+ log ,
126+ ctx ,
127+ ClassLoader .getSystemClassLoader ());
128+ assertThat (mockExecutionVertex .isDeployCalled (), is (false ));
129+ }
130+ }
131+
132+ @ Test (expected = IllegalStateException .class )
133+ public void testIllegalStateExceptionOnNotRunningExecutionGraph () throws Exception {
134+ try (MockExecutingContext ctx = new MockExecutingContext ()) {
135+ ExecutionGraph notRunningExecutionGraph = new StateTrackingMockExecutionGraph ();
136+ assertThat (notRunningExecutionGraph .getState (), is (not (JobStatus .RUNNING )));
137+
138+ new Executing (
139+ notRunningExecutionGraph ,
140+ getExecutionGraphHandler (notRunningExecutionGraph , ctx .getMainThreadExecutor ()),
141+ new TestingOperatorCoordinatorHandler (),
142+ log ,
143+ ctx ,
144+ ClassLoader .getSystemClassLoader ());
145+ }
146+ }
147+
104148 @ Test
105149 public void testDisposalOfOperatorCoordinatorsOnLeaveOfStateWithExecutionGraph ()
106150 throws Exception {
@@ -258,6 +302,23 @@ public void testFalseReportsViaUpdateTaskExecutionStateAreIgnored() throws Excep
258302 }
259303 }
260304
305+ @ Test
306+ public void testExecutionVertexMarkedAsFailedOnDeploymentFailure () throws Exception {
307+ try (MockExecutingContext ctx = new MockExecutingContext ()) {
308+ MockExecutionJobVertex mejv =
309+ new MockExecutionJobVertex (FailOnDeployMockExecutionVertex ::new );
310+ ExecutionGraph executionGraph =
311+ new MockExecutionGraph (() -> Collections .singletonList (mejv ));
312+ Executing exec =
313+ new ExecutingStateBuilder ().setExecutionGraph (executionGraph ).build (ctx );
314+
315+ assertThat (
316+ ((FailOnDeployMockExecutionVertex ) mejv .getMockExecutionVertex ())
317+ .getMarkedFailure (),
318+ is (instanceOf (JobException .class )));
319+ }
320+ }
321+
261322 @ Test
262323 public void testTransitionToStopWithSavepointState () throws Exception {
263324 try (MockExecutingContext ctx = new MockExecutingContext ()) {
@@ -389,31 +450,31 @@ public ExecutingStateBuilder setOperatorCoordinatorHandler(
389450
390451 private Executing build (MockExecutingContext ctx ) {
391452 executionGraph .transitionToRunning ();
392- final ExecutionGraphHandler executionGraphHandler =
393- new ExecutionGraphHandler (
394- executionGraph ,
395- log ,
396- ctx .getMainThreadExecutor (),
397- ctx .getMainThreadExecutor ());
398453
399454 return new Executing (
400455 executionGraph ,
401- executionGraphHandler ,
456+ getExecutionGraphHandler ( executionGraph , ctx . getMainThreadExecutor ()) ,
402457 operatorCoordinatorHandler ,
403458 log ,
404459 ctx ,
405460 ClassLoader .getSystemClassLoader ());
406461 }
407462 }
408463
464+ private ExecutionGraphHandler getExecutionGraphHandler (
465+ ExecutionGraph executionGraph , ComponentMainThreadExecutor mainThreadExecutor ) {
466+ return new ExecutionGraphHandler (
467+ executionGraph , log , mainThreadExecutor , mainThreadExecutor );
468+ }
469+
409470 private static class MockExecutingContext extends MockStateWithExecutionGraphContext
410471 implements Executing .Context {
411472
412473 private final StateValidator <FailingArguments > failingStateValidator =
413474 new StateValidator <>("failing" );
414475 private final StateValidator <RestartingArguments > restartingStateValidator =
415476 new StateValidator <>("restarting" );
416- private final StateValidator <ExecutingAndCancellingArguments > cancellingStateValidator =
477+ private final StateValidator <CancellingArguments > cancellingStateValidator =
417478 new StateValidator <>("cancelling" );
418479
419480 private Function <Throwable , Executing .FailureResult > howToHandleFailure ;
@@ -431,7 +492,7 @@ public void setExpectRestarting(Consumer<RestartingArguments> asserter) {
431492 restartingStateValidator .expectInput (asserter );
432493 }
433494
434- public void setExpectCancelling (Consumer <ExecutingAndCancellingArguments > asserter ) {
495+ public void setExpectCancelling (Consumer <CancellingArguments > asserter ) {
435496 cancellingStateValidator .expectInput (asserter );
436497 }
437498
@@ -455,7 +516,7 @@ public void goToCanceling(
455516 ExecutionGraphHandler executionGraphHandler ,
456517 OperatorCoordinatorHandler operatorCoordinatorHandler ) {
457518 cancellingStateValidator .validateInput (
458- new ExecutingAndCancellingArguments (
519+ new CancellingArguments (
459520 executionGraph , executionGraphHandler , operatorCoordinatorHandler ));
460521 hadStateTransition = true ;
461522 }
@@ -537,12 +598,12 @@ public void close() throws Exception {
537598 }
538599 }
539600
540- static class ExecutingAndCancellingArguments {
601+ static class CancellingArguments {
541602 private final ExecutionGraph executionGraph ;
542603 private final ExecutionGraphHandler executionGraphHandler ;
543604 private final OperatorCoordinatorHandler operatorCoordinatorHandle ;
544605
545- public ExecutingAndCancellingArguments (
606+ public CancellingArguments (
546607 ExecutionGraph executionGraph ,
547608 ExecutionGraphHandler executionGraphHandler ,
548609 OperatorCoordinatorHandler operatorCoordinatorHandle ) {
@@ -564,7 +625,7 @@ public OperatorCoordinatorHandler getOperatorCoordinatorHandle() {
564625 }
565626 }
566627
567- static class StopWithSavepointArguments extends ExecutingAndCancellingArguments {
628+ static class StopWithSavepointArguments extends CancellingArguments {
568629 private final CheckpointScheduling checkpointScheduling ;
569630 private final CompletableFuture <String > savepointFuture ;
570631
@@ -580,7 +641,7 @@ public StopWithSavepointArguments(
580641 }
581642 }
582643
583- static class RestartingArguments extends ExecutingAndCancellingArguments {
644+ static class RestartingArguments extends CancellingArguments {
584645 private final Duration backoffTime ;
585646
586647 public RestartingArguments (
@@ -597,7 +658,7 @@ public Duration getBackoffTime() {
597658 }
598659 }
599660
600- static class FailingArguments extends ExecutingAndCancellingArguments {
661+ static class FailingArguments extends CancellingArguments {
601662 private final Throwable failureCause ;
602663
603664 public FailingArguments (
@@ -614,7 +675,7 @@ public Throwable getFailureCause() {
614675 }
615676 }
616677
617- private static class MockExecutionGraph extends StateTrackingMockExecutionGraph {
678+ static class MockExecutionGraph extends StateTrackingMockExecutionGraph {
618679 private final boolean updateStateReturnValue ;
619680 private final Supplier <Iterable <ExecutionJobVertex >> getVerticesTopologicallySupplier ;
620681
@@ -623,10 +684,6 @@ private static class MockExecutionGraph extends StateTrackingMockExecutionGraph
623684 this (false , getVerticesTopologicallySupplier );
624685 }
625686
626- MockExecutionGraph (boolean updateStateReturnValue ) {
627- this (updateStateReturnValue , null );
628- }
629-
630687 private MockExecutionGraph (
631688 boolean updateStateReturnValue ,
632689 Supplier <Iterable <ExecutionJobVertex >> getVerticesTopologicallySupplier ) {
@@ -689,9 +746,11 @@ public Logger getLogger() {
689746 }
690747
691748 static class MockExecutionJobVertex extends ExecutionJobVertex {
692- private final MockExecutionVertex mockExecutionVertex ;
749+ private final ExecutionVertex mockExecutionVertex ;
693750
694- MockExecutionJobVertex () throws JobException {
751+ MockExecutionJobVertex (
752+ Function <ExecutionJobVertex , ExecutionVertex > executionVertexSupplier )
753+ throws JobException {
695754 super (
696755 new MockInternalExecutionGraphAccessor (),
697756 new JobVertex ("test" ),
@@ -700,37 +759,67 @@ static class MockExecutionJobVertex extends ExecutionJobVertex {
700759 1L ,
701760 new DefaultVertexParallelismInfo (1 , 1 , max -> Optional .empty ()),
702761 new DefaultSubtaskAttemptNumberStore (Collections .emptyList ()));
703- mockExecutionVertex = new MockExecutionVertex (this );
762+ mockExecutionVertex = executionVertexSupplier . apply (this );
704763 }
705764
706765 @ Override
707766 public ExecutionVertex [] getTaskVertices () {
708767 return new ExecutionVertex [] {mockExecutionVertex };
709768 }
710769
711- public MockExecutionVertex getMockExecutionVertex () {
770+ public ExecutionVertex getMockExecutionVertex () {
712771 return mockExecutionVertex ;
713772 }
773+ }
774+
775+ static class FailOnDeployMockExecutionVertex extends ExecutionVertex {
776+
777+ @ Nullable private Throwable markFailed = null ;
778+
779+ public FailOnDeployMockExecutionVertex (ExecutionJobVertex jobVertex ) {
780+ super (jobVertex , 1 , new IntermediateResult [] {}, Time .milliseconds (1L ), 1L , 1 , 0 );
781+ }
714782
715- public boolean isExecutionDeployed () {
716- return mockExecutionVertex .isDeployed ();
783+ @ Override
784+ public void deploy () throws JobException {
785+ throw new JobException ("Intentional Test exception" );
786+ }
787+
788+ @ Override
789+ public void markFailed (Throwable t ) {
790+ markFailed = t ;
791+ }
792+
793+ @ Nullable
794+ public Throwable getMarkedFailure () {
795+ return markFailed ;
717796 }
718797 }
719798
720799 static class MockExecutionVertex extends ExecutionVertex {
721- private boolean deployed = false ;
800+ private boolean deployCalled = false ;
801+ private ExecutionState mockedExecutionState = ExecutionState .RUNNING ;
722802
723803 MockExecutionVertex (ExecutionJobVertex jobVertex ) {
724804 super (jobVertex , 1 , new IntermediateResult [] {}, Time .milliseconds (1L ), 1L , 1 , 0 );
725805 }
726806
727807 @ Override
728808 public void deploy () throws JobException {
729- deployed = true ;
809+ deployCalled = true ;
810+ }
811+
812+ public boolean isDeployCalled () {
813+ return deployCalled ;
814+ }
815+
816+ @ Override
817+ public ExecutionState getExecutionState () {
818+ return mockedExecutionState ;
730819 }
731820
732- public boolean isDeployed ( ) {
733- return deployed ;
821+ public void setMockedExecutionState ( ExecutionState mockedExecutionState ) {
822+ this . mockedExecutionState = mockedExecutionState ;
734823 }
735824 }
736825
0 commit comments