|
37 | 37 | import org.apache.flink.runtime.client.JobSubmissionException;
|
38 | 38 | import org.apache.flink.runtime.clusterframework.ApplicationStatus;
|
39 | 39 | import org.apache.flink.runtime.dispatcher.DispatcherGateway;
|
| 40 | +import org.apache.flink.runtime.execution.ExecutionState; |
| 41 | +import org.apache.flink.runtime.instance.SlotSharingGroupId; |
40 | 42 | import org.apache.flink.runtime.io.network.partition.DataSetMetaInfo;
|
41 | 43 | import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
|
42 | 44 | import org.apache.flink.runtime.jobgraph.JobGraph;
|
43 | 45 | import org.apache.flink.runtime.jobgraph.JobGraphTestUtils;
|
| 46 | +import org.apache.flink.runtime.jobgraph.JobVertexID; |
44 | 47 | import org.apache.flink.runtime.jobgraph.OperatorID;
|
45 | 48 | import org.apache.flink.runtime.jobmaster.JobResult;
|
46 | 49 | import org.apache.flink.runtime.messages.Acknowledge;
|
|
71 | 74 | import org.apache.flink.runtime.rest.messages.JobCancellationHeaders;
|
72 | 75 | import org.apache.flink.runtime.rest.messages.JobCancellationMessageParameters;
|
73 | 76 | import org.apache.flink.runtime.rest.messages.JobMessageParameters;
|
| 77 | +import org.apache.flink.runtime.rest.messages.JobPlanInfo; |
74 | 78 | import org.apache.flink.runtime.rest.messages.JobsOverviewHeaders;
|
75 | 79 | import org.apache.flink.runtime.rest.messages.MessageHeaders;
|
76 | 80 | import org.apache.flink.runtime.rest.messages.MessageParameters;
|
|
86 | 90 | import org.apache.flink.runtime.rest.messages.dataset.ClusterDataSetIdPathParameter;
|
87 | 91 | import org.apache.flink.runtime.rest.messages.dataset.ClusterDataSetListHeaders;
|
88 | 92 | import org.apache.flink.runtime.rest.messages.dataset.ClusterDataSetListResponseBody;
|
| 93 | +import org.apache.flink.runtime.rest.messages.job.JobDetailsHeaders; |
| 94 | +import org.apache.flink.runtime.rest.messages.job.JobDetailsInfo; |
89 | 95 | import org.apache.flink.runtime.rest.messages.job.JobExecutionResultHeaders;
|
90 | 96 | import org.apache.flink.runtime.rest.messages.job.JobExecutionResultResponseBody;
|
91 | 97 | import org.apache.flink.runtime.rest.messages.job.JobStatusInfoHeaders;
|
|
96 | 102 | import org.apache.flink.runtime.rest.messages.job.coordination.ClientCoordinationMessageParameters;
|
97 | 103 | import org.apache.flink.runtime.rest.messages.job.coordination.ClientCoordinationRequestBody;
|
98 | 104 | import org.apache.flink.runtime.rest.messages.job.coordination.ClientCoordinationResponseBody;
|
| 105 | +import org.apache.flink.runtime.rest.messages.job.metrics.IOMetricsInfo; |
99 | 106 | import org.apache.flink.runtime.rest.messages.job.savepoints.SavepointDisposalRequest;
|
100 | 107 | import org.apache.flink.runtime.rest.messages.job.savepoints.SavepointDisposalStatusHeaders;
|
101 | 108 | import org.apache.flink.runtime.rest.messages.job.savepoints.SavepointDisposalStatusMessageParameters;
|
@@ -1236,6 +1243,58 @@ void testNotShowSuspendedJobStatus() throws Exception {
|
1236 | 1243 | }
|
1237 | 1244 | }
|
1238 | 1245 |
|
| 1246 | + @Test |
| 1247 | + void testJobDetailsContainsSlotSharingGroupId() throws Exception { |
| 1248 | + final IOMetricsInfo jobVertexMetrics = |
| 1249 | + new IOMetricsInfo(0, false, 0, false, 0, false, 0, false, 0, 0, 0); |
| 1250 | + SlotSharingGroupId slotSharingGroupId = new SlotSharingGroupId(); |
| 1251 | + final Collection<JobDetailsInfo.JobVertexDetailsInfo> jobVertexDetailsInfos = |
| 1252 | + Collections.singletonList( |
| 1253 | + new JobDetailsInfo.JobVertexDetailsInfo( |
| 1254 | + new JobVertexID(), |
| 1255 | + slotSharingGroupId, |
| 1256 | + "jobVertex1", |
| 1257 | + 2, |
| 1258 | + 1, |
| 1259 | + ExecutionState.RUNNING, |
| 1260 | + 1, |
| 1261 | + 2, |
| 1262 | + 1, |
| 1263 | + Collections.singletonMap(ExecutionState.RUNNING, 0), |
| 1264 | + jobVertexMetrics)); |
| 1265 | + final JobDetailsInfo jobDetailsInfo = |
| 1266 | + new JobDetailsInfo( |
| 1267 | + jobId, |
| 1268 | + "foobar", |
| 1269 | + false, |
| 1270 | + JobStatus.RUNNING, |
| 1271 | + 1, |
| 1272 | + 2, |
| 1273 | + 1, |
| 1274 | + 2, |
| 1275 | + 10, |
| 1276 | + Collections.singletonMap(JobStatus.RUNNING, 1L), |
| 1277 | + jobVertexDetailsInfos, |
| 1278 | + Collections.singletonMap(ExecutionState.RUNNING, 1), |
| 1279 | + new JobPlanInfo.RawJson("{\"id\":\"1234\"}")); |
| 1280 | + final TestJobDetailsInfoHandler jobDetailsInfoHandler = |
| 1281 | + new TestJobDetailsInfoHandler(jobDetailsInfo); |
| 1282 | + |
| 1283 | + try (TestRestServerEndpoint restServerEndpoint = |
| 1284 | + createRestServerEndpoint(jobDetailsInfoHandler)) { |
| 1285 | + try (RestClusterClient<?> restClusterClient = |
| 1286 | + createRestClusterClient(restServerEndpoint.getServerAddress().getPort())) { |
| 1287 | + final CompletableFuture<JobDetailsInfo> jobDetailsInfoFuture = |
| 1288 | + restClusterClient.getJobDetails(jobId); |
| 1289 | + Collection<JobDetailsInfo.JobVertexDetailsInfo> jobVertexInfos = |
| 1290 | + jobDetailsInfoFuture.get().getJobVertexInfos(); |
| 1291 | + assertThat(jobVertexInfos).hasSize(1); |
| 1292 | + assertThat(jobVertexInfos.iterator().next().getSlotSharingGroupId()) |
| 1293 | + .isEqualTo(slotSharingGroupId); |
| 1294 | + } |
| 1295 | + } |
| 1296 | + } |
| 1297 | + |
1239 | 1298 | private class TestClientCoordinationHandler
|
1240 | 1299 | extends TestHandler<
|
1241 | 1300 | ClientCoordinationRequestBody,
|
@@ -1402,6 +1461,25 @@ protected CompletableFuture<JobStatusInfo> handleRequest(
|
1402 | 1461 | }
|
1403 | 1462 | }
|
1404 | 1463 |
|
| 1464 | + private class TestJobDetailsInfoHandler |
| 1465 | + extends TestHandler<EmptyRequestBody, JobDetailsInfo, JobMessageParameters> { |
| 1466 | + |
| 1467 | + private final JobDetailsInfo jobDetailsInfo; |
| 1468 | + |
| 1469 | + private TestJobDetailsInfoHandler(@Nonnull JobDetailsInfo jobDetailsInfo) { |
| 1470 | + super(JobDetailsHeaders.getInstance()); |
| 1471 | + this.jobDetailsInfo = checkNotNull(jobDetailsInfo); |
| 1472 | + } |
| 1473 | + |
| 1474 | + @Override |
| 1475 | + protected CompletableFuture<JobDetailsInfo> handleRequest( |
| 1476 | + @Nonnull HandlerRequest<EmptyRequestBody> request, |
| 1477 | + @Nonnull DispatcherGateway gateway) |
| 1478 | + throws RestHandlerException { |
| 1479 | + return CompletableFuture.completedFuture(jobDetailsInfo); |
| 1480 | + } |
| 1481 | + } |
| 1482 | + |
1405 | 1483 | private abstract class TestHandler<
|
1406 | 1484 | R extends RequestBody, P extends ResponseBody, M extends MessageParameters>
|
1407 | 1485 | extends AbstractRestHandler<DispatcherGateway, R, P, M> {
|
|
0 commit comments