You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/04/08 09:11:11 UTC

[GitHub] [flink] smattheis commented on a diff in pull request #19398: [FLINK-26864][metrics] Fix performance regression from mailbox latency measuement

smattheis commented on code in PR #19398:
URL: https://github.com/apache/flink/pull/19398#discussion_r845908392


##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxMetricsController.java:
##########
@@ -0,0 +1,154 @@
+/*
+ * 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.streaming.runtime.tasks.mailbox;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.operators.MailboxExecutor;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.Histogram;
+import org.apache.flink.streaming.runtime.tasks.TimerService;
+import org.apache.flink.util.clock.SystemClock;
+
+import javax.annotation.Nullable;
+
+/**
+ * Mailbox metrics controller class. The use of mailbox metrics, in particular scheduling latency
+ * measurements that require a {@link TimerService}, induce (cyclic) dependencies between {@link
+ * MailboxProcessor} and {@link org.apache.flink.streaming.runtime.tasks.StreamTask}. An instance of
+ * this class contains and gives control over these dependencies.
+ */
+@Internal
+public class MailboxMetricsController {
+    /** Default timer interval in milliseconds for triggering mailbox latency measurement. */
+    public final int defaultLatencyMeasurementInterval = 1000;
+
+    private final Histogram latencyHistogram;
+    private final Counter mailCounter;
+
+    @Nullable private TimerService timerService;
+    @Nullable private MailboxExecutor mailboxExecutor;
+    private int measurementInterval = defaultLatencyMeasurementInterval;
+    private boolean started = false;
+
+    /**
+     * Creates instance of {@link MailboxMetricsController} with references to metrics provided as
+     * parameters.
+     *
+     * @param latencyHistogram Histogram of mailbox latency measurements.
+     * @param mailCounter Counter for number of mails processed.
+     */
+    public MailboxMetricsController(Histogram latencyHistogram, Counter mailCounter) {
+        this.timerService = null;
+        this.mailboxExecutor = null;
+        this.latencyHistogram = latencyHistogram;
+        this.mailCounter = mailCounter;
+    }
+
+    /**
+     * Sets up latency measurement with required {@link TimerService} and {@link MailboxExecutor}.
+     *
+     * <p>Note: For each instance, latency measurement can be set up only once.
+     *
+     * @param timerService {@link TimerService} used for latency measurement.
+     * @param mailboxExecutor {@link MailboxExecutor} used for latency measurement.
+     */
+    public void setupLatencyMeasurement(
+            TimerService timerService, MailboxExecutor mailboxExecutor) {
+        if (isLatencyMeasurementSetup()) {
+            throw new IllegalStateException(
+                    "latency measurement has already been setup and cannot be setup twice");
+        }

Review Comment:
   Is it okay like this:
   ``` java
       import static org.apache.flink.util.Preconditions.checkState;
   ...
       public void startLatencyMeasurement() {
           checkState(isLatencyMeasurementStarted(), "latency measurement has already been started");
           checkState(
                   !isLatencyMeasurementSetup(),
                   "timer service and mailbox executor must be setup for latency measurement");
           scheduleLatencyMeasurement();
           started = true;
       }
   ```



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org