21
21
import org .apache .flink .api .common .JobStatus ;
22
22
import org .apache .flink .client .program .rest .RestClusterClient ;
23
23
import org .apache .flink .configuration .Configuration ;
24
- import org .apache .flink .runtime .client . JobStatusMessage ;
24
+ import org .apache .flink .runtime .execution . ExecutionState ;
25
25
import org .apache .flink .runtime .highavailability .nonha .standalone .StandaloneClientHAServices ;
26
+ import org .apache .flink .runtime .messages .webmonitor .JobDetails ;
27
+ import org .apache .flink .runtime .messages .webmonitor .MultipleJobsDetails ;
26
28
import org .apache .flink .runtime .rest .messages .ConfigurationInfo ;
27
29
import org .apache .flink .runtime .rest .messages .JobMessageParameters ;
30
+ import org .apache .flink .runtime .rest .messages .JobsOverviewHeaders ;
28
31
import org .apache .flink .runtime .rest .messages .MessageHeaders ;
29
32
import org .apache .flink .runtime .rest .messages .MessageParameters ;
30
33
import org .apache .flink .runtime .rest .messages .RequestBody ;
38
41
import javax .annotation .Nullable ;
39
42
40
43
import java .time .Duration ;
41
- import java .time .Instant ;
42
- import java .util .Collection ;
43
- import java .util .List ;
44
+ import java .util .Arrays ;
44
45
import java .util .Map ;
45
46
import java .util .concurrent .CompletableFuture ;
46
47
import java .util .concurrent .TimeoutException ;
47
48
import java .util .concurrent .atomic .AtomicLong ;
49
+ import java .util .stream .Collectors ;
48
50
49
51
import static org .assertj .core .api .Assertions .assertThat ;
50
52
import static org .assertj .core .api .Assertions .assertThatThrownBy ;
@@ -56,12 +58,9 @@ class FlinkClusterJobListFetcherTest {
56
58
/** Test whether the job list and confs are expected. */
57
59
@ Test
58
60
void testFetchJobListAndConfigurationInfo () throws Exception {
59
- var job1 =
60
- new JobStatusMessage (
61
- new JobID (), "" , JobStatus .RUNNING , Instant .now ().toEpochMilli ());
62
- var job2 =
63
- new JobStatusMessage (
64
- new JobID (), "" , JobStatus .CANCELLING , Instant .now ().toEpochMilli ());
61
+ var job1 = new JobID ();
62
+ var job2 = new JobID ();
63
+ var jobs = Map .of (job1 , JobStatus .RUNNING , job2 , JobStatus .CANCELLING );
65
64
66
65
Configuration expectedConf1 = new Configuration ();
67
66
expectedConf1 .setString ("option_key1" , "option_value1" );
@@ -70,18 +69,17 @@ void testFetchJobListAndConfigurationInfo() throws Exception {
70
69
expectedConf2 .setString ("option_key2" , "option_value2" );
71
70
expectedConf2 .setString ("option_key3" , "option_value3" );
72
71
73
- var jobs = Map .of (job1 .getJobId (), job1 , job2 .getJobId (), job2 );
74
- var configurations = Map .of (job1 .getJobId (), expectedConf1 , job2 .getJobId (), expectedConf2 );
72
+ var configurations = Map .of (job1 , expectedConf1 , job2 , expectedConf2 );
75
73
var closeCounter = new AtomicLong ();
76
74
FlinkClusterJobListFetcher jobListFetcher =
77
75
new FlinkClusterJobListFetcher (
78
76
getRestClusterClient (
79
- Either .Left (List . of ( job1 , job2 ) ),
77
+ Either .Left (jobs ),
80
78
Either .Left (
81
79
Map .of (
82
- job1 . getJobId () ,
80
+ job1 ,
83
81
ConfigurationInfo .from (expectedConf1 ),
84
- job2 . getJobId () ,
82
+ job2 ,
85
83
ConfigurationInfo .from (expectedConf2 ))),
86
84
closeCounter ),
87
85
Duration .ofSeconds (10 ));
@@ -94,11 +92,9 @@ void testFetchJobListAndConfigurationInfo() throws Exception {
94
92
95
93
assertThat (fetchedJobList ).hasSize (2 );
96
94
for (var jobContext : fetchedJobList ) {
97
- JobStatusMessage expectedJobStatusMessage = jobs .get (jobContext .getJobID ());
95
+ var expectedJobState = jobs .get (jobContext .getJobID ());
98
96
Configuration expectedConf = configurations .get (jobContext .getJobID ());
99
- assertThat (expectedJobStatusMessage ).isNotNull ();
100
- assertThat (jobContext .getJobStatus ())
101
- .isEqualTo (expectedJobStatusMessage .getJobState ());
97
+ assertThat (jobContext .getJobStatus ()).isEqualTo (expectedJobState );
102
98
assertThat (jobContext .getConfiguration ()).isNotNull ().isEqualTo (expectedConf );
103
99
}
104
100
}
@@ -130,16 +126,13 @@ void testFetchJobListException() {
130
126
*/
131
127
@ Test
132
128
void testFetchConfigurationException () {
133
- var job1 =
134
- new JobStatusMessage (
135
- new JobID (), "" , JobStatus .RUNNING , Instant .now ().toEpochMilli ());
136
129
var expectedException = new RuntimeException ("Expected exception." );
137
130
var closeCounter = new AtomicLong ();
138
131
139
132
FlinkClusterJobListFetcher jobListFetcher =
140
133
new FlinkClusterJobListFetcher (
141
134
getRestClusterClient (
142
- Either .Left (List .of (job1 )),
135
+ Either .Left (Map .of (new JobID (), JobStatus . RUNNING )),
143
136
Either .Right (expectedException ),
144
137
closeCounter ),
145
138
Duration .ofSeconds (10 ));
@@ -171,14 +164,12 @@ void testFetchJobListTimeout() {
171
164
*/
172
165
@ Test
173
166
void testFetchConfigurationTimeout () {
174
- var job1 =
175
- new JobStatusMessage (
176
- new JobID (), "" , JobStatus .RUNNING , Instant .now ().toEpochMilli ());
177
167
CompletableFuture <Void > closeFuture = new CompletableFuture <>();
178
168
179
169
FlinkClusterJobListFetcher jobListFetcher =
180
170
new FlinkClusterJobListFetcher (
181
- getTimeoutableRestClusterClient (List .of (job1 ), null , closeFuture ),
171
+ getTimeoutableRestClusterClient (
172
+ Map .of (new JobID (), JobStatus .RUNNING ), null , closeFuture ),
182
173
Duration .ofSeconds (2 ));
183
174
184
175
assertThat (closeFuture ).isNotDone ();
@@ -189,16 +180,16 @@ void testFetchConfigurationTimeout() {
189
180
}
190
181
191
182
/**
192
- * @param jobListOrException When listJobs is called, return jobList if Either is left, return
193
- * failedFuture if Either is right.
183
+ * @param jobsOrException When the jobs overview is called, return jobList if Either is left,
184
+ * return failedFuture if Either is right.
194
185
* @param configurationsOrException When fetch job conf, return configuration if Either is left,
195
186
* return failedFuture if Either is right.
196
187
* @param closeCounter Increment the count each time the {@link RestClusterClient#close} is
197
188
* called
198
189
*/
199
190
private static FunctionWithException <Configuration , RestClusterClient <String >, Exception >
200
191
getRestClusterClient (
201
- Either <Collection < JobStatusMessage >, Throwable > jobListOrException ,
192
+ Either <Map < JobID , JobStatus >, Throwable > jobsOrException ,
202
193
Either <Map <JobID , ConfigurationInfo >, Throwable > configurationsOrException ,
203
194
AtomicLong closeCounter ) {
204
195
return conf ->
@@ -207,14 +198,6 @@ void testFetchConfigurationTimeout() {
207
198
"test-cluster" ,
208
199
(c , e ) -> new StandaloneClientHAServices ("localhost" )) {
209
200
210
- @ Override
211
- public CompletableFuture <Collection <JobStatusMessage >> listJobs () {
212
- if (jobListOrException .isLeft ()) {
213
- return CompletableFuture .completedFuture (jobListOrException .left ());
214
- }
215
- return CompletableFuture .failedFuture (jobListOrException .right ());
216
- }
217
-
218
201
@ Override
219
202
public <
220
203
M extends MessageHeaders <R , P , U >,
@@ -231,6 +214,22 @@ CompletableFuture<P> sendRequest(M h, U p, R r) {
231
214
return (CompletableFuture <P >)
232
215
CompletableFuture .completedFuture (
233
216
configurationsOrException .left ().get (jobID ));
217
+ } else if (h instanceof JobsOverviewHeaders ) {
218
+ if (jobsOrException .isLeft ()) {
219
+ return (CompletableFuture <P >)
220
+ CompletableFuture .completedFuture (
221
+ new MultipleJobsDetails (
222
+ jobsOrException .left ().entrySet ().stream ()
223
+ .map (
224
+ entry ->
225
+ generateJobDetails (
226
+ entry
227
+ .getKey (),
228
+ entry
229
+ .getValue ()))
230
+ .collect (Collectors .toList ())));
231
+ }
232
+ return CompletableFuture .failedFuture (jobsOrException .right ());
234
233
}
235
234
fail ("Unknown request" );
236
235
return null ;
@@ -245,15 +244,15 @@ public void close() {
245
244
}
246
245
247
246
/**
248
- * @param jobList When listJobs is called, return jobList if it's not null, don't complete
247
+ * @param jobs When the jobs overview is called, return jobList if it's not null, don't complete
249
248
* future if it's null.
250
249
* @param configuration When fetch job conf, return configuration if it's not null, don't
251
250
* complete future if it's null.
252
251
* @param closeFuture Complete this closeFuture when {@link RestClusterClient#close} is called.
253
252
*/
254
253
private static FunctionWithException <Configuration , RestClusterClient <String >, Exception >
255
254
getTimeoutableRestClusterClient (
256
- @ Nullable Collection < JobStatusMessage > jobList ,
255
+ @ Nullable Map < JobID , JobStatus > jobs ,
257
256
@ Nullable ConfigurationInfo configuration ,
258
257
CompletableFuture <Void > closeFuture ) {
259
258
return conf ->
@@ -262,14 +261,6 @@ public void close() {
262
261
"test-cluster" ,
263
262
(c , e ) -> new StandaloneClientHAServices ("localhost" )) {
264
263
265
- @ Override
266
- public CompletableFuture <Collection <JobStatusMessage >> listJobs () {
267
- if (jobList == null ) {
268
- return new CompletableFuture <>();
269
- }
270
- return CompletableFuture .completedFuture (jobList );
271
- }
272
-
273
264
@ Override
274
265
public <
275
266
M extends MessageHeaders <R , P , U >,
@@ -283,6 +274,21 @@ CompletableFuture<P> sendRequest(M h, U p, R r) {
283
274
}
284
275
return (CompletableFuture <P >)
285
276
CompletableFuture .completedFuture (configuration );
277
+ } else if (h instanceof JobsOverviewHeaders ) {
278
+ if (jobs == null ) {
279
+ return new CompletableFuture <>();
280
+ }
281
+ return (CompletableFuture <P >)
282
+ CompletableFuture .completedFuture (
283
+ new MultipleJobsDetails (
284
+ jobs .entrySet ().stream ()
285
+ .map (
286
+ entry ->
287
+ generateJobDetails (
288
+ entry .getKey (),
289
+ entry
290
+ .getValue ()))
291
+ .collect (Collectors .toList ())));
286
292
}
287
293
fail ("Unknown request" );
288
294
return null ;
@@ -295,4 +301,25 @@ public void close() {
295
301
}
296
302
};
297
303
}
304
+
305
+ private static JobDetails generateJobDetails (JobID jobID , JobStatus jobStatus ) {
306
+ int [] countPerState = new int [ExecutionState .values ().length ];
307
+ if (jobStatus == JobStatus .RUNNING ) {
308
+ countPerState [ExecutionState .RUNNING .ordinal ()] = 5 ;
309
+ countPerState [ExecutionState .FINISHED .ordinal ()] = 2 ;
310
+ } else if (jobStatus == JobStatus .CANCELLING ) {
311
+ countPerState [ExecutionState .CANCELING .ordinal ()] = 7 ;
312
+ }
313
+ int numTasks = Arrays .stream (countPerState ).sum ();
314
+ return new JobDetails (
315
+ jobID ,
316
+ "test-job" ,
317
+ System .currentTimeMillis (),
318
+ -1 ,
319
+ 0 ,
320
+ jobStatus ,
321
+ System .currentTimeMillis (),
322
+ countPerState ,
323
+ numTasks );
324
+ }
298
325
}
0 commit comments