rkhachatryan commented on a change in pull request #14526: URL: https://github.com/apache/flink/pull/14526#discussion_r551353719
########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/TimeGauge.java ########## @@ -0,0 +1,80 @@ +/* + * 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. + */ + +// ---------------------------------------------------------------------------- +// This class is largely adapted from "com.google.common.base.Preconditions", +// which is part of the "Guava" library. +// +// Because of frequent issues with dependency conflicts, this class was +// added to the Flink code base to reduce dependency on Guava. +// ---------------------------------------------------------------------------- + +package org.apache.flink.runtime.metrics.groups; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.View; + +/** + * {@link TimeGauge} encapsulate logic of measuring time rates (like idleTime ms/s). In particularly + * it takes care of the case, when single measurement (as defined by {@link #markStart()} and {@link + * #markEnd()}), crosses the boundary of single {@link #update()} interval. Thanks to the {@link + * #markStart()} {@link TimeGauge} can correctly calculate next value in {@link #update()}, even if + * this measurement hasn't not ended ({@link #markEnd()}). + */ +public class TimeGauge implements Gauge<Long>, View { + private long previousCount; + private long currentCount; + private long lastDifference; + private long currentMeasurementStart; + + public synchronized void markStart() { + if (currentMeasurementStart == 0) { + currentMeasurementStart = System.currentTimeMillis(); + } + } + + public synchronized void markEnd() { + if (currentMeasurementStart != 0) { + currentCount += System.currentTimeMillis() - currentMeasurementStart; + currentMeasurementStart = 0; + } + } + + @Override + public synchronized void update() { + if (currentMeasurementStart != 0) { + long now = System.currentTimeMillis(); + currentCount += now - currentMeasurementStart; + currentMeasurementStart = now; + } + lastDifference = (currentCount - previousCount) / UPDATE_INTERVAL_SECONDS; Review comment: Do we have to rely on `UPDATE_INTERVAL_SECONDS`? Why not to record previous update time (`currentTimeMillis`) instead? ########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java ########## @@ -400,9 +404,23 @@ protected void processInput(MailboxDefaultAction.Controller controller) throws E controller.allActionsCompleted(); return; } - CompletableFuture<?> jointFuture = getInputOutputJointFuture(status); - MailboxDefaultAction.Suspension suspendedDefaultAction = controller.suspendDefaultAction(); - assertNoException(jointFuture.thenRun(suspendedDefaultAction::resume)); + + final Suspension suspension = controller.suspendDefaultAction(); Review comment: This method (`processInput`) became quite long now. Can we extract `getBackPressuredGauge` or `onOutputUnavailable`? ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/TaskIOMetricGroup.java ########## @@ -98,10 +105,26 @@ public Counter getNumBuffersOutCounter() { return numBuffersOut; } - public Meter getIdleTimeMsPerSecond() { + public TimeGauge getIdleTimeMsPerSecond() { return idleTimePerSecond; } + public TimeGauge getBackPressuredTimePerSecond() { + return backPressuredTimePerSecond; + } + + public void setEnableBusyTime(boolean enabled) { + busyTimeEnabled = enabled; + } + + private double getBusyTimePerSecond() { + double v = Review comment: nit: can be inlined ########## File path: flink-runtime-web/web-dashboard/src/app/share/common/dagre/node.component.ts ########## @@ -31,8 +31,18 @@ export class NodeComponent { operatorStrategy: string | null; parallelism: number | null; lowWatermark: number | null | undefined; + backPressuredPercentage: number | undefined = NaN; + busyPercentage: number | undefined = NaN; + backgroundColor: string | undefined; + borderColor: string | undefined; height = 0; id: string; + lightRed = '#ee6464'; + lightBlue = '#5db1ff'; + lightBlack = '#888888'; + red = '#ee2222'; + blue = '#1890ff'; + black = '#000000'; Review comment: I think it's better to name these colors semantically, e.g. back-pressuring instead of red, back-pressured instead of black. ########## File path: flink-runtime-web/web-dashboard/src/app/share/common/dagre/node.component.ts ########## @@ -9,7 +9,7 @@ * * http://www.apache.org/licenses/LICENSE-2.0 * - * Unless required by applicable law or agreed to in writing, software + * Unless requilightRed by applicable law or agreed to in writing, software Review comment: Typo? ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/TimeGauge.java ########## @@ -0,0 +1,80 @@ +/* + * 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. + */ + +// ---------------------------------------------------------------------------- +// This class is largely adapted from "com.google.common.base.Preconditions", +// which is part of the "Guava" library. +// +// Because of frequent issues with dependency conflicts, this class was +// added to the Flink code base to reduce dependency on Guava. +// ---------------------------------------------------------------------------- + +package org.apache.flink.runtime.metrics.groups; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.View; + +/** + * {@link TimeGauge} encapsulate logic of measuring time rates (like idleTime ms/s). In particularly + * it takes care of the case, when single measurement (as defined by {@link #markStart()} and {@link + * #markEnd()}), crosses the boundary of single {@link #update()} interval. Thanks to the {@link + * #markStart()} {@link TimeGauge} can correctly calculate next value in {@link #update()}, even if + * this measurement hasn't not ended ({@link #markEnd()}). + */ +public class TimeGauge implements Gauge<Long>, View { + private long previousCount; + private long currentCount; + private long lastDifference; + private long currentMeasurementStart; + + public synchronized void markStart() { + if (currentMeasurementStart == 0) { + currentMeasurementStart = System.currentTimeMillis(); Review comment: IIUC, the same `TimeGauge` object is now used by `SteamTask` and `BufferWritingResultPartition`. If that's true, one of the `markStart` calls will be ignored, and "wrong" `markEnd` can record `currentCount`. ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org