Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id A65B6200D16 for ; Tue, 10 Oct 2017 15:02:52 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id A4A23160BE0; Tue, 10 Oct 2017 13:02:52 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id C365F1609E5 for ; Tue, 10 Oct 2017 15:02:51 +0200 (CEST) Received: (qmail 84337 invoked by uid 500); 10 Oct 2017 13:02:51 -0000 Mailing-List: contact issues-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@flink.apache.org Delivered-To: mailing list issues@flink.apache.org Received: (qmail 84328 invoked by uid 99); 10 Oct 2017 13:02:50 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd1-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 10 Oct 2017 13:02:50 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd1-us-west.apache.org (ASF Mail Server at spamd1-us-west.apache.org) with ESMTP id 2BD06D89B0 for ; Tue, 10 Oct 2017 13:02:50 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd1-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -3.221 X-Spam-Level: X-Spam-Status: No, score=-3.221 tagged_above=-999 required=6.31 tests=[KAM_ASCII_DIVIDERS=0.8, KAM_LAZY_DOMAIN_SECURITY=1, RCVD_IN_DNSWL_HI=-5, RCVD_IN_MSPIKE_H3=-0.01, RCVD_IN_MSPIKE_WL=-0.01, RP_MATCHES_RCVD=-0.001] autolearn=disabled Received: from mx1-lw-us.apache.org ([10.40.0.8]) by localhost (spamd1-us-west.apache.org [10.40.0.7]) (amavisd-new, port 10024) with ESMTP id FczdTv2yHI13 for ; Tue, 10 Oct 2017 13:02:48 +0000 (UTC) Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx1-lw-us.apache.org (ASF Mail Server at mx1-lw-us.apache.org) with SMTP id 9FD515F3FE for ; Tue, 10 Oct 2017 13:02:47 +0000 (UTC) Received: (qmail 84019 invoked by uid 99); 10 Oct 2017 13:02:47 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 10 Oct 2017 13:02:47 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 1D517F5C1F; Tue, 10 Oct 2017 13:02:47 +0000 (UTC) From: tillrohrmann To: issues@flink.incubator.apache.org Reply-To: issues@flink.incubator.apache.org References: In-Reply-To: Subject: [GitHub] flink pull request #4734: [FLINK-7652] [flip6] Port CurrentJobIdsHandler to ... Content-Type: text/plain Message-Id: <20171010130247.1D517F5C1F@git1-us-west.apache.org> Date: Tue, 10 Oct 2017 13:02:47 +0000 (UTC) archived-at: Tue, 10 Oct 2017 13:02:52 -0000 Github user tillrohrmann commented on a diff in the pull request: https://github.com/apache/flink/pull/4734#discussion_r143718904 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/messages/webmonitor/JobStatusesWithIdsOverview.java --- @@ -0,0 +1,368 @@ +/* + * 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 com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.SerializerProvider; +import com.fasterxml.jackson.databind.annotation.JsonDeserialize; +import com.fasterxml.jackson.databind.annotation.JsonSerialize; +import com.fasterxml.jackson.databind.deser.std.StdDeserializer; +import com.fasterxml.jackson.databind.ser.std.StdSerializer; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.LinkedList; +import java.util.List; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * An overview of how many jobs are in which status. + */ +@JsonSerialize(using = JobStatusesWithIdsOverview.JobStatusesWithIdsOverviewSerializer.class) +@JsonDeserialize(using = JobStatusesWithIdsOverview.JobStatusesWithIdsOverviewDeserializer.class) +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"; + + private final List jobsCreated; + private final List jobsRunningOrPending; + private final List jobsFinished; + private final List jobsCancelling; + private final List jobsCancelled; + private final List jobsFailing; + private final List jobsFailed; + private final List jobsRestarting; + private final List jobsSuspended; + private final List jobsReconciling; + + public JobStatusesWithIdsOverview( + List jobsCreated, + List jobsRunningOrPending, + List jobsFinished, + List jobsCancelling, + List jobsCancelled, + List jobsFailing, + List jobsFailed, + List jobsRestarting, + List jobsSuspended, + List jobsReconciling) { + + this.jobsCreated = checkNotNull(jobsCreated); + this.jobsRunningOrPending = checkNotNull(jobsRunningOrPending); + this.jobsFinished = checkNotNull(jobsFinished); + this.jobsCancelling = checkNotNull(jobsCancelling); + this.jobsCancelled = checkNotNull(jobsCancelled); + this.jobsFailing = checkNotNull(jobsFailing); + this.jobsFailed = checkNotNull(jobsFailed); + this.jobsRestarting = checkNotNull(jobsRestarting); + this.jobsSuspended = checkNotNull(jobsSuspended); + this.jobsReconciling = checkNotNull(jobsReconciling); + } + + public JobStatusesWithIdsOverview(JobStatusesWithIdsOverview first, JobStatusesWithIdsOverview second) { + this.jobsCreated = combine(first.getJobsCreated(), second.getJobsCreated()); + this.jobsRunningOrPending = combine(first.getJobsRunningOrPending(), second.getJobsRunningOrPending()); + this.jobsFinished = combine(first.getJobsFinished(), second.getJobsFinished()); + this.jobsCancelling = combine(first.getJobsCancelling(), second.getJobsCancelling()); + this.jobsCancelled = combine(first.getJobsCancelled(), second.getJobsCancelled()); + this.jobsFailing = combine(first.getJobsFailing(), second.getJobsFailing()); + this.jobsFailed = combine(first.getJobsFailed(), second.getJobsFailed()); + this.jobsRestarting = combine(first.getJobsRestarting(), second.getJobsRestarting()); + this.jobsSuspended = combine(first.getJobsSuspended(), second.getJobsSuspended()); + this.jobsReconciling = combine(first.getJobsReconciling(), second.getJobsReconciling()); + } + + public List getJobsCreated() { + return jobsCreated; + } + + public List getJobsRunningOrPending() { + return jobsRunningOrPending; + } + + public List getJobsFinished() { + return jobsFinished; + } + + public List getJobsCancelling() { + return jobsCancelling; + } + + public List getJobsCancelled() { + return jobsCancelled; + } + + public List getJobsFailing() { + return jobsFailing; + } + + public List getJobsFailed() { + return jobsFailed; + } + + public List getJobsRestarting() { + return jobsRestarting; + } + + public List getJobsSuspended() { + return jobsSuspended; + } + + public List getJobsReconciling() { + return jobsReconciling; + } + + // ------------------------------------------------------------------------ + + + @Override + public int hashCode() { + return jobsCreated.hashCode() ^ + jobsRunningOrPending.hashCode() ^ + jobsFinished.hashCode() ^ + jobsCancelling.hashCode() ^ + jobsCancelled.hashCode() ^ + jobsFailing.hashCode() ^ + jobsFailed.hashCode() ^ + jobsRestarting.hashCode() ^ + jobsSuspended.hashCode() ^ + jobsReconciling.hashCode(); + } + + @Override + public boolean equals(Object obj) { + if (obj == this) { + return true; + } + else if (obj instanceof JobStatusesWithIdsOverview) { + JobStatusesWithIdsOverview that = (JobStatusesWithIdsOverview) obj; + return this.jobsCreated.equals(that.jobsCreated) && + this.jobsRunningOrPending.equals(that.jobsRunningOrPending) && + this.jobsFinished.equals(that.jobsFinished) && + this.jobsCancelling.equals(that.jobsCancelling) && + this.jobsCancelled.equals(that.jobsCancelled) && + this.jobsFailing.equals(that.jobsFailing) && + this.jobsFailed.equals(that.jobsFailed) && + this.jobsRestarting.equals(that.jobsRestarting) && + this.jobsSuspended.equals(that.jobsSuspended) && + this.jobsReconciling.equals(that.jobsReconciling); + } + else { + return false; + } + } + + @Override + public String toString() { + return "JobStatusesWithIdsOverview {" + + "createdJobs=" + jobsCreated + + ", runningOrPendingJobs=" + jobsRunningOrPending + + ", finishedJobs=" + jobsFinished + + ", cancellingJobs=" + jobsCancelling + + ", cancelledJobs=" + jobsCancelled + + ", failingJobs=" + jobsFailing + + ", failedJobs=" + jobsFailed + + ", restartingJobs=" + jobsRestarting + + ", suspendedJobs=" + jobsSuspended + + ", reconcilingJobs=" + jobsReconciling + + '}'; + } + + // ------------------------------------------------------------------------ + // Message serializers + // ------------------------------------------------------------------------ + + public static final class JobStatusesWithIdsOverviewSerializer extends StdSerializer { --- End diff -- Just out of curiosity, why did you define a (de)serializer for this class? The structure looks as if this could have been done automatically by the `ObjectMapper`. ---