[
https://issues.apache.org/jira/browse/FLINK-7652?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16240172#comment-16240172
]
ASF GitHub Bot commented on FLINK-7652:
---------------------------------------
Github user tillrohrmann commented on a diff in the pull request:
https://github.com/apache/flink/pull/4734#discussion_r149057297
--- Diff:
flink-runtime/src/main/java/org/apache/flink/runtime/messages/webmonitor/JobStatusesWithIdsOverview.java
---
@@ -0,0 +1,235 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.messages.webmonitor;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.rest.messages.ResponseBody;
+
+import org.apache.flink.runtime.rest.messages.json.JobIDDeserializer;
+import org.apache.flink.runtime.rest.messages.json.JobIDSerializer;
+import
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
+import
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonDeserialize;
+import
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonSerialize;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * An overview of how many jobs are in which status.
+ */
+public class JobStatusesWithIdsOverview implements ResponseBody,
InfoMessage {
+
+ private static final long serialVersionUID = -3699051943490133183L;
+
+ public static final String FIELD_NAME_JOBS_CREATED_IDS = "jobs-created";
+ public static final String FIELD_NAME_JOBS_RUNNING_IDS = "jobs-running";
+ public static final String FIELD_NAME_JOBS_FINISHED_IDS =
"jobs-finished";
+ public static final String FIELD_NAME_JOBS_CANCELLING_IDS =
"jobs-cancelling";
+ public static final String FIELD_NAME_JOBS_CANCELLED_IDS =
"jobs-cancelled";
+ public static final String FIELD_NAME_JOBS_FAILING_IDS = "jobs-failing";
+ public static final String FIELD_NAME_JOBS_FAILED_IDS = "jobs-failed";
+ public static final String FIELD_NAME_JOBS_RESTARTING_IDS =
"jobs-restarting";
+ public static final String FIELD_NAME_JOBS_SUSPENDED_IDS =
"jobs-suspended";
+ public static final String FIELD_NAME_JOBS_RECONCILING_IDS =
"jobs-reconciling";
+
+ @JsonProperty(FIELD_NAME_JOBS_CREATED_IDS)
+ @JsonSerialize(contentUsing = JobIDSerializer.class)
+ private final List<JobID> jobsCreated;
+
+ @JsonProperty(FIELD_NAME_JOBS_RUNNING_IDS)
+ @JsonSerialize(contentUsing = JobIDSerializer.class)
+ private final List<JobID> jobsRunningOrPending;
+
+ @JsonProperty(FIELD_NAME_JOBS_FINISHED_IDS)
+ @JsonSerialize(contentUsing = JobIDSerializer.class)
+ private final List<JobID> jobsFinished;
+
+ @JsonProperty(FIELD_NAME_JOBS_CANCELLING_IDS)
+ @JsonSerialize(contentUsing = JobIDSerializer.class)
+ private final List<JobID> jobsCancelling;
+
+ @JsonProperty(FIELD_NAME_JOBS_CANCELLED_IDS)
+ @JsonSerialize(contentUsing = JobIDSerializer.class)
+ private final List<JobID> jobsCancelled;
+
+ @JsonProperty(FIELD_NAME_JOBS_FAILING_IDS)
+ @JsonSerialize(contentUsing = JobIDSerializer.class)
+ private final List<JobID> jobsFailing;
+
+ @JsonProperty(FIELD_NAME_JOBS_FAILED_IDS)
+ @JsonSerialize(contentUsing = JobIDSerializer.class)
+ private final List<JobID> jobsFailed;
+
+ @JsonProperty(FIELD_NAME_JOBS_RESTARTING_IDS)
+ @JsonSerialize(contentUsing = JobIDSerializer.class)
+ private final List<JobID> jobsRestarting;
+
+ @JsonProperty(FIELD_NAME_JOBS_SUSPENDED_IDS)
+ @JsonSerialize(contentUsing = JobIDSerializer.class)
+ private final List<JobID> jobsSuspended;
+
+ @JsonProperty(FIELD_NAME_JOBS_RECONCILING_IDS)
+ @JsonSerialize(contentUsing = JobIDSerializer.class)
+ private final List<JobID> jobsReconciling;
--- End diff --
I would actually not group the jobs here. Rather we could return a
collection of `JobIDs` with `JobStatus`. If you need to present a grouped
overview, then the client has to calculate it. That would simplify this class
and the handler considerably. What do you think?
> Port CurrentJobIdsHandler to new REST endpoint
> ----------------------------------------------
>
> Key: FLINK-7652
> URL: https://issues.apache.org/jira/browse/FLINK-7652
> Project: Flink
> Issue Type: Sub-task
> Components: REST, Webfrontend
> Reporter: Tzu-Li (Gordon) Tai
> Assignee: Tzu-Li (Gordon) Tai
> Labels: flip-6
> Fix For: 1.4.0
>
>
> Port existing {{CurrentJobIdsHandler}} to new REST endpoint
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)