|
21 | 21 | import org.apache.flink.annotation.Internal; |
22 | 22 | import org.apache.flink.annotation.VisibleForTesting; |
23 | 23 | import org.apache.flink.api.common.ApplicationID; |
| 24 | +import org.apache.flink.api.common.ApplicationState; |
24 | 25 | import org.apache.flink.api.common.JobID; |
25 | 26 | import org.apache.flink.api.common.JobStatus; |
26 | 27 | import org.apache.flink.api.java.tuple.Tuple2; |
|
72 | 73 | import org.apache.flink.runtime.jobmaster.JobResult; |
73 | 74 | import org.apache.flink.runtime.jobmaster.factories.DefaultJobManagerJobMetricGroupFactory; |
74 | 75 | import org.apache.flink.runtime.messages.Acknowledge; |
| 76 | +import org.apache.flink.runtime.messages.FlinkApplicationNotFoundException; |
| 77 | +import org.apache.flink.runtime.messages.FlinkApplicationTerminatedWithoutCancellationException; |
75 | 78 | import org.apache.flink.runtime.messages.FlinkJobNotFoundException; |
76 | 79 | import org.apache.flink.runtime.messages.FlinkJobTerminatedWithoutCancellationException; |
| 80 | +import org.apache.flink.runtime.messages.webmonitor.ApplicationDetails; |
| 81 | +import org.apache.flink.runtime.messages.webmonitor.ApplicationDetailsInfo; |
77 | 82 | import org.apache.flink.runtime.messages.webmonitor.ClusterOverview; |
78 | 83 | import org.apache.flink.runtime.messages.webmonitor.JobDetails; |
79 | 84 | import org.apache.flink.runtime.messages.webmonitor.JobsOverview; |
| 85 | +import org.apache.flink.runtime.messages.webmonitor.MultipleApplicationsDetails; |
80 | 86 | import org.apache.flink.runtime.messages.webmonitor.MultipleJobsDetails; |
81 | 87 | import org.apache.flink.runtime.metrics.MetricNames; |
82 | 88 | import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; |
@@ -902,6 +908,28 @@ public CompletableFuture<Acknowledge> cancelJob(JobID jobId, Duration timeout) { |
902 | 908 | return FutureUtils.completedExceptionally(new FlinkJobNotFoundException(jobId)); |
903 | 909 | } |
904 | 910 |
|
| 911 | + @Override |
| 912 | + public CompletableFuture<Acknowledge> cancelApplication( |
| 913 | + ApplicationID applicationId, Duration timeout) { |
| 914 | + if (!applications.containsKey(applicationId)) { |
| 915 | + return FutureUtils.completedExceptionally( |
| 916 | + new FlinkApplicationNotFoundException(applicationId)); |
| 917 | + } |
| 918 | + AbstractApplication application = applications.get(applicationId); |
| 919 | + ApplicationState current = application.getApplicationStatus(); |
| 920 | + if (current.isTerminalState()) { |
| 921 | + if (current == ApplicationState.CANCELED) { |
| 922 | + return CompletableFuture.completedFuture(Acknowledge.get()); |
| 923 | + } else { |
| 924 | + return FutureUtils.completedExceptionally( |
| 925 | + new FlinkApplicationTerminatedWithoutCancellationException( |
| 926 | + applicationId, current)); |
| 927 | + } |
| 928 | + } |
| 929 | + applications.get(applicationId).cancel(); |
| 930 | + return CompletableFuture.completedFuture(Acknowledge.get()); |
| 931 | + } |
| 932 | + |
905 | 933 | @Override |
906 | 934 | public CompletableFuture<ClusterOverview> requestClusterOverview(Duration timeout) { |
907 | 935 | CompletableFuture<ResourceOverview> taskManagerOverviewFuture = |
@@ -967,6 +995,114 @@ public CompletableFuture<MultipleJobsDetails> requestMultipleJobDetails(Duration |
967 | 995 | }); |
968 | 996 | } |
969 | 997 |
|
| 998 | + @Override |
| 999 | + public CompletableFuture<MultipleApplicationsDetails> requestMultipleApplicationDetails( |
| 1000 | + Duration timeout) { |
| 1001 | + List<CompletableFuture<ApplicationDetails>> applicationDetailsFutures = |
| 1002 | + applications.values().stream() |
| 1003 | + .map(application -> createApplicationDetails(application, timeout)) |
| 1004 | + .collect(Collectors.toList()); |
| 1005 | + return FutureUtils.combineAll(applicationDetailsFutures) |
| 1006 | + .thenCompose( |
| 1007 | + combinedApplicationDetails -> |
| 1008 | + CompletableFuture.completedFuture( |
| 1009 | + new MultipleApplicationsDetails( |
| 1010 | + combinedApplicationDetails))); |
| 1011 | + } |
| 1012 | + |
| 1013 | + private CompletableFuture<ApplicationDetails> createApplicationDetails( |
| 1014 | + AbstractApplication application, Duration timeout) { |
| 1015 | + ApplicationState applicationStatus = application.getApplicationStatus(); |
| 1016 | + long startTime = application.getStatusTimestamp(ApplicationState.RUNNING); |
| 1017 | + long endTime = |
| 1018 | + applicationStatus.isTerminalState() |
| 1019 | + ? application.getStatusTimestamp(applicationStatus) |
| 1020 | + : -1L; |
| 1021 | + long duration = (endTime >= 0L ? endTime : System.currentTimeMillis()) - startTime; |
| 1022 | + Map<String, Integer> jobInfo = new HashMap<>(); |
| 1023 | + List<CompletableFuture<JobStatus>> individualJobStatus = |
| 1024 | + application.getJobs().stream() |
| 1025 | + .map(jobId -> requestJobStatus(jobId, timeout)) |
| 1026 | + .collect(Collectors.toList()); |
| 1027 | + CompletableFuture<Collection<JobStatus>> combinedJobStatus = |
| 1028 | + FutureUtils.combineAll(individualJobStatus); |
| 1029 | + return combinedJobStatus.thenCompose( |
| 1030 | + jobStatuses -> { |
| 1031 | + for (JobStatus status : jobStatuses) { |
| 1032 | + if (status != null) { |
| 1033 | + jobInfo.compute( |
| 1034 | + status.name(), |
| 1035 | + (key, oldValue) -> (oldValue == null ? 1 : oldValue + 1)); |
| 1036 | + } |
| 1037 | + } |
| 1038 | + return CompletableFuture.completedFuture( |
| 1039 | + new ApplicationDetails( |
| 1040 | + application.getApplicationId(), |
| 1041 | + application.getName(), |
| 1042 | + startTime, |
| 1043 | + endTime, |
| 1044 | + duration, |
| 1045 | + applicationStatus.toString(), |
| 1046 | + jobInfo)); |
| 1047 | + }); |
| 1048 | + } |
| 1049 | + |
| 1050 | + @Override |
| 1051 | + public CompletableFuture<ApplicationDetailsInfo> requestApplication( |
| 1052 | + ApplicationID applicationId, Duration timeout) { |
| 1053 | + if (!applications.containsKey(applicationId)) { |
| 1054 | + return FutureUtils.completedExceptionally( |
| 1055 | + new FlinkApplicationNotFoundException(applicationId)); |
| 1056 | + } |
| 1057 | + AbstractApplication application = applications.get(applicationId); |
| 1058 | + ApplicationState applicationStatus = application.getApplicationStatus(); |
| 1059 | + long startTime = application.getStatusTimestamp(ApplicationState.RUNNING); |
| 1060 | + long endTime = |
| 1061 | + applicationStatus.isTerminalState() |
| 1062 | + ? application.getStatusTimestamp(applicationStatus) |
| 1063 | + : -1L; |
| 1064 | + long duration = (endTime >= 0L ? endTime : System.currentTimeMillis()) - startTime; |
| 1065 | + final Map<String, Long> timestamps = |
| 1066 | + CollectionUtil.newHashMapWithExpectedSize(ApplicationState.values().length); |
| 1067 | + for (ApplicationState status : ApplicationState.values()) { |
| 1068 | + timestamps.put(status.toString(), application.getStatusTimestamp(status)); |
| 1069 | + } |
| 1070 | + List<CompletableFuture<JobDetails>> jobDetailsFutures = |
| 1071 | + application.getJobs().stream() |
| 1072 | + .map(jobId -> requestJobDetails(jobId, timeout)) |
| 1073 | + .collect(Collectors.toList()); |
| 1074 | + return FutureUtils.combineAll(jobDetailsFutures) |
| 1075 | + .thenCompose( |
| 1076 | + combinedJobDetails -> |
| 1077 | + CompletableFuture.completedFuture( |
| 1078 | + new ApplicationDetailsInfo( |
| 1079 | + application.getApplicationId(), |
| 1080 | + application.getName(), |
| 1081 | + applicationStatus.toString(), |
| 1082 | + startTime, |
| 1083 | + endTime, |
| 1084 | + duration, |
| 1085 | + timestamps, |
| 1086 | + combinedJobDetails))); |
| 1087 | + } |
| 1088 | + |
| 1089 | + private CompletableFuture<JobDetails> requestJobDetails(JobID jobId, Duration timeout) { |
| 1090 | + Optional<JobManagerRunner> maybeJob = getJobManagerRunner(jobId); |
| 1091 | + return maybeJob.map(job -> job.requestJobDetails(timeout)) |
| 1092 | + .orElseGet( |
| 1093 | + () -> { |
| 1094 | + // is it a completed job? |
| 1095 | + final JobDetails jobDetails = |
| 1096 | + executionGraphInfoStore.getAvailableJobDetails(jobId); |
| 1097 | + if (jobDetails == null) { |
| 1098 | + return FutureUtils.completedExceptionally( |
| 1099 | + new FlinkJobNotFoundException(jobId)); |
| 1100 | + } else { |
| 1101 | + return CompletableFuture.completedFuture(jobDetails); |
| 1102 | + } |
| 1103 | + }); |
| 1104 | + } |
| 1105 | + |
970 | 1106 | @Override |
971 | 1107 | public CompletableFuture<JobStatus> requestJobStatus(JobID jobId, Duration timeout) { |
972 | 1108 | Optional<JobManagerRunner> maybeJob = getJobManagerRunner(jobId); |
|
0 commit comments