@@ -101,7 +101,42 @@ public void runJobWithMultipleRpcServices() throws Exception {
101
101
}
102
102
103
103
@ Test
104
- public void testHandleJobsWhenNotEnoughSlot () throws Exception {
104
+ public void testHandleStreamingJobsWhenNotEnoughSlot () throws Exception {
105
+ try {
106
+ setupAndRunHandleJobsWhenNotEnoughSlots (ScheduleMode .EAGER );
107
+ fail ("Job should fail." );
108
+ } catch (JobExecutionException e ) {
109
+ assertTrue (findThrowableWithMessage (e , "Job execution failed." ).isPresent ());
110
+ assertTrue (findThrowable (e , NoResourceAvailableException .class ).isPresent ());
111
+ assertTrue (findThrowableWithMessage (e , "Slots required: 2, slots allocated: 1" ).isPresent ());
112
+ }
113
+ }
114
+
115
+ @ Test
116
+ public void testHandleBatchJobsWhenNotEnoughSlot () throws Exception {
117
+ try {
118
+ setupAndRunHandleJobsWhenNotEnoughSlots (ScheduleMode .LAZY_FROM_SOURCES );
119
+ fail ("Job should fail." );
120
+ } catch (JobExecutionException e ) {
121
+ assertTrue (findThrowableWithMessage (e , "Job execution failed." ).isPresent ());
122
+ assertTrue (findThrowable (e , NoResourceAvailableException .class ).isPresent ());
123
+ assertTrue (findThrowableWithMessage (e , "Could not allocate enough slots" ).isPresent ());
124
+ }
125
+ }
126
+
127
+ private void setupAndRunHandleJobsWhenNotEnoughSlots (ScheduleMode scheduleMode ) throws Exception {
128
+ final JobVertex vertex = new JobVertex ("Test Vertex" );
129
+ vertex .setParallelism (2 );
130
+ vertex .setMaxParallelism (2 );
131
+ vertex .setInvokableClass (BlockingNoOpInvokable .class );
132
+
133
+ final JobGraph jobGraph = new JobGraph ("Test Job" , vertex );
134
+ jobGraph .setScheduleMode (scheduleMode );
135
+
136
+ runHandleJobsWhenNotEnoughSlots (jobGraph );
137
+ }
138
+
139
+ private void runHandleJobsWhenNotEnoughSlots (final JobGraph jobGraph ) throws Exception {
105
140
final Configuration configuration = getDefaultConfiguration ();
106
141
configuration .setLong (JobManagerOptions .SLOT_REQUEST_TIMEOUT , 100L );
107
142
@@ -114,24 +149,7 @@ public void testHandleJobsWhenNotEnoughSlot() throws Exception {
114
149
try (final MiniCluster miniCluster = new MiniCluster (cfg )) {
115
150
miniCluster .start ();
116
151
117
- final JobVertex vertex = new JobVertex ("Test Vertex" );
118
- vertex .setParallelism (2 );
119
- vertex .setMaxParallelism (2 );
120
- vertex .setInvokableClass (BlockingNoOpInvokable .class );
121
-
122
- final JobGraph jobGraph = new JobGraph ("Test Job" , vertex );
123
- jobGraph .setScheduleMode (ScheduleMode .EAGER );
124
-
125
- try {
126
- miniCluster .executeJobBlocking (jobGraph );
127
-
128
- fail ("Job should fail." );
129
- } catch (JobExecutionException e ) {
130
- assertTrue (findThrowableWithMessage (e , "Job execution failed." ).isPresent ());
131
-
132
- assertTrue (findThrowable (e , NoResourceAvailableException .class ).isPresent ());
133
- assertTrue (findThrowableWithMessage (e , "Slots required: 2, slots allocated: 1" ).isPresent ());
134
- }
152
+ miniCluster .executeJobBlocking (jobGraph );
135
153
}
136
154
}
137
155
0 commit comments