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/03/07 07:16:19 UTC

[GitHub] [flink] smattheis opened a new pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

smattheis opened a new pull request #18991:
URL: https://github.com/apache/flink/pull/18991


   ## What is the purpose of the change
   
   * Add mailbox delay/latency metric backed by a DescriptiveStatisticsHistogram and throughput metric backed by a MeterView.
   
   ## Brief change log
   
   * Added metrics to TaskIOMetricGroup
   * Extended StreamTask to schedule mailbox latency measurements using a timer
   * Extended MailboxProcessor to count processed mails for throughput measurement
   
   ## Verifying this change
   
   This change added tests and can be verified as follows:
   
   - Extended existing test method TaskMailboxProcessorTest#testRunDefaultActionAndMails to verify mail/throughput counting
   - Added test method StreamTaskTest#testMailboxMetricsLatency to verify timer-based latency measurement
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): (yes / __no__)
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (yes / __no__)
     - The serializers: (yes / __no__ / don't know)
     - The runtime per-record code paths (performance sensitive): (__yes__ / no / don't know)
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn, ZooKeeper: (yes / __no__ / don't know)
     - The S3 file system connector: (yes / __no__ / don't know)
   
   ## Documentation
   
     - Does this pull request introduce a new feature? (__yes__ / no)
     - If yes, how is the feature documented? (not applicable / __docs__ / __JavaDocs__ / not documented)
   
   __Note:__ This PR is based on https://github.com/apache/flink/pull/18984 which is a hotfix. 
   


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



[GitHub] [flink] flinkbot edited a comment on pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18991:
URL: https://github.com/apache/flink/pull/18991#issuecomment-1060268498


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32601",
       "triggerID" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5b9dea466c3984b5e0afef3502cf2f4addc829e6 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32601) 
   * de1583bca2dc4062ce5773f7dfaaeb532e1c1e27 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18991:
URL: https://github.com/apache/flink/pull/18991#issuecomment-1060268498


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32601",
       "triggerID" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "1063757856",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32828",
       "triggerID" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * de1583bca2dc4062ce5773f7dfaaeb532e1c1e27 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787) 
   * 7a222b87cf1721247dbc1c432107514efb5613b1 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32828) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18991:
URL: https://github.com/apache/flink/pull/18991#issuecomment-1060268498


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32601",
       "triggerID" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "1063757856",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32828",
       "triggerID" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32916",
       "triggerID" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2fab9dc17e3a008d69e9ef71548a26b3a7996dd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32999",
       "triggerID" : "2fab9dc17e3a008d69e9ef71548a26b3a7996dd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b70bc0e6248848030a86e60ccadfb071d92762c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33146",
       "triggerID" : "7b70bc0e6248848030a86e60ccadfb071d92762c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac60fac6097229637711be838e7dd586acb47e5f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33157",
       "triggerID" : "ac60fac6097229637711be838e7dd586acb47e5f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4f804908ad5239c06941496fc90faf98d1675723",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33194",
       "triggerID" : "4f804908ad5239c06941496fc90faf98d1675723",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ac60fac6097229637711be838e7dd586acb47e5f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33157) 
   * 4f804908ad5239c06941496fc90faf98d1675723 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33194) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] smattheis commented on a change in pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
smattheis commented on a change in pull request #18991:
URL: https://github.com/apache/flink/pull/18991#discussion_r820763673



##########
File path: docs/content.zh/docs/ops/metrics.md
##########
@@ -1428,6 +1428,16 @@ Note that the metrics are only available via reporters.
       <td>Maximum recorded duration of a single consecutive period of the task being in the hard back pressure state in the last sampling period. Please check softBackPressuredTimeMsPerSecond and hardBackPressuredTimeMsPerSecond for more information.</td>
       <td>Gauge</td>
     </tr>
+    <tr>

Review comment:
       That would see it as meaningful. I didn't want to extend the scope too much as the issue's scope is already extended with the throughput but I would be +1 for it. Although I need to check if this feasible in an easy way. Counting (which methods need to inc and dec as it's not fully trivial with the createBatch semantics) vs. size method (what interval? how to do that with a metrics class? also note that size() is marked @VisibleForTesting and acquires a lock).
   
   What do you think should we invest the effort now?




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



[GitHub] [flink] smattheis commented on pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
smattheis commented on pull request #18991:
URL: https://github.com/apache/flink/pull/18991#issuecomment-1063757856


   @flinkbot run azure


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



[GitHub] [flink] pnowojski commented on a change in pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #18991:
URL: https://github.com/apache/flink/pull/18991#discussion_r824689000



##########
File path: docs/content.zh/docs/ops/metrics.md
##########
@@ -1443,6 +1443,11 @@ Note that the metrics are only available via reporters.
       <td>The latency is the time that actions spend waiting in the task's mailbox before being processed. The metric is a statistic of the latency in milliseconds that is measured approximately once every second and includes the last 60 measurements.</td>
       <td>Histogram</td>
     </tr>
+    <tr>
+      <td>mailboxSize</td>

Review comment:
       `mailboxQueueSize`?

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxProcessor.java
##########
@@ -175,6 +198,7 @@ public void close() {
     public void drain() throws Exception {
         for (final Mail mail : mailbox.drain()) {
             mail.run();
+            numMailsProcessed.inc();

Review comment:
       Nit, extract 
   ```
   mail.run();
   numMailsProcessed.inc();
   ``` 
   to a helper method to deduplicate the code?

##########
File path: flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
##########
@@ -1763,6 +1765,66 @@ public long calculateThroughput() {
         }
     }
 
+    /**
+     * Tests mailbox metrics latency and queue size and verifies that (1) latency measurement is
+     * executed initially once and at least once triggered by timer, (2) latency mean value is
+     * greater than zero and (3) mailbox size is greater than zero for some time and eventually
+     * equals to zero.
+     *
+     * @throws Exception on {@link MockEnvironmentBuilder#build()} failure.
+     */
+    @Test
+    public void testMailboxMetrics() throws Exception {

Review comment:
       Are you sure this test is going to be stable? It looks like 20ms freeze of the metrics thread would cause it to fail. 
   
   Can you take a look at the buffer debloating tests and `StreamTask#debloat` method in particular? I think using `StreamTaskMailboxTestHarnessBuilder` you would be able to implement a precise and reliable test. 
   
   1. Unlike `StreamTask#debloat` method, yours `StreamTask#meassureMailboxLatency` would look like:
   ```
   StreamTask#meassureMailboxLatency() {
                       long startTime = SystemClock.getInstance().relativeTimeMillis();
                       mainMailboxExecutor.execute(
                               () -> {
                                   long endTime = SystemClock.getInstance().relativeTimeMillis();
                                   long latency = endTime - startTime;
                                   environment
                                           .getMetricGroup()
                                           .getIOMetricGroup()
                                           .getMailboxLatency()
                                           .update(latency);
                                   scheduleMailboxMetrics();
                               },
                               "Measure mailbox latency metric");
                   });
   }
   ```
   2. You could enqueue 10 mails.
   3. Setup to sleep X ms (`20ms`?)  per record.
   4. Call `StreamTask#meassureMailboxLatency`
   5. Assert `mailboxQueueSize == 11`.
   6. Call `testHarness.processAll()`
   7. Assert `maxLatency >= X ms`.

##########
File path: flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxProcessorTest.java
##########
@@ -117,6 +119,8 @@ public void runDefaultAction(Controller controller) throws Exception {
         MailboxProcessor mailboxProcessor = start(mailboxThread);
         mailboxProcessor.getMailboxExecutor(DEFAULT_PRIORITY).execute(() -> stop.set(true), "stop");
         mailboxThread.join();
+        Assert.assertTrue(counter.get() > 0);

Review comment:
       `assertEquals(1, counter.get())`?
   
   
   note, I'm recommending both the condition to be more strict and dropping vague `assertTrue`. `assertTrue` would give a very not helpful error in case of a failure.

##########
File path: flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxProcessorTest.java
##########
@@ -117,6 +119,8 @@ public void runDefaultAction(Controller controller) throws Exception {
         MailboxProcessor mailboxProcessor = start(mailboxThread);
         mailboxProcessor.getMailboxExecutor(DEFAULT_PRIORITY).execute(() -> stop.set(true), "stop");
         mailboxThread.join();
+        Assert.assertTrue(counter.get() > 0);
+        Assert.assertTrue(mailboxProcessor.getNumMailsProcessedCounter().getCount() > 0);

Review comment:
       `assertGE(mailboxProcessor.getNumMailsProcessedCounter().getCount(), counter.get())`?
   




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



[GitHub] [flink] pnowojski commented on a change in pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #18991:
URL: https://github.com/apache/flink/pull/18991#discussion_r827994494



##########
File path: flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
##########
@@ -1763,6 +1765,108 @@ public long calculateThroughput() {
         }
     }
 
+    /**
+     * Tests mailbox metrics latency and queue size and verifies that (1) latency measurement is
+     * executed initially once and at least once triggered by timer, (2) latency max value is
+     * greater than zero and (3) mailbox size is greater than zero for some time and eventually
+     * equals to zero.
+     *
+     * <p>Note: This test uses a timeout which, if exceeded, indicates that the minimum number of
+     * latency measurements has not reached the specified minimum within specified time.
+     *
+     * @throws Exception on {@link MockEnvironmentBuilder#build()} failure.
+     */
+    @Test(timeout = 4000)

Review comment:
       I would remove this timeout. Our general best practise (learnt the hard way) is to avoid timeouts inside the tests. If there is a deadlock/livelock, it's actually better for mvn watchdog to detect that, and kill the build with printing thread dump.
   
   Apart of that 4s is most likely too short. We have seen occasional 10-15sec freezes in the CI. 




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



[GitHub] [flink] flinkbot edited a comment on pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18991:
URL: https://github.com/apache/flink/pull/18991#issuecomment-1060268498


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32601",
       "triggerID" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "1063757856",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32828",
       "triggerID" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32916",
       "triggerID" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2fab9dc17e3a008d69e9ef71548a26b3a7996dd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32999",
       "triggerID" : "2fab9dc17e3a008d69e9ef71548a26b3a7996dd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b70bc0e6248848030a86e60ccadfb071d92762c",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33146",
       "triggerID" : "7b70bc0e6248848030a86e60ccadfb071d92762c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7b70bc0e6248848030a86e60ccadfb071d92762c Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33146) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18991:
URL: https://github.com/apache/flink/pull/18991#issuecomment-1060268498


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32601",
       "triggerID" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "1063757856",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32828",
       "triggerID" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32916",
       "triggerID" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2fab9dc17e3a008d69e9ef71548a26b3a7996dd3",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2fab9dc17e3a008d69e9ef71548a26b3a7996dd3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 195dd50ec12230bbc8c7a57e58b1dee43f16be33 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32916) 
   * 2fab9dc17e3a008d69e9ef71548a26b3a7996dd3 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot commented on pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #18991:
URL: https://github.com/apache/flink/pull/18991#issuecomment-1060268498


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5b9dea466c3984b5e0afef3502cf2f4addc829e6 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18991:
URL: https://github.com/apache/flink/pull/18991#issuecomment-1060268498


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32601",
       "triggerID" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "1063757856",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * de1583bca2dc4062ce5773f7dfaaeb532e1c1e27 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18991:
URL: https://github.com/apache/flink/pull/18991#issuecomment-1060268498


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32601",
       "triggerID" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "1063757856",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32828",
       "triggerID" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32916",
       "triggerID" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2fab9dc17e3a008d69e9ef71548a26b3a7996dd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32999",
       "triggerID" : "2fab9dc17e3a008d69e9ef71548a26b3a7996dd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b70bc0e6248848030a86e60ccadfb071d92762c",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33146",
       "triggerID" : "7b70bc0e6248848030a86e60ccadfb071d92762c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac60fac6097229637711be838e7dd586acb47e5f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ac60fac6097229637711be838e7dd586acb47e5f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7b70bc0e6248848030a86e60ccadfb071d92762c Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33146) 
   * ac60fac6097229637711be838e7dd586acb47e5f UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18991:
URL: https://github.com/apache/flink/pull/18991#issuecomment-1060268498


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32601",
       "triggerID" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "1063757856",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32828",
       "triggerID" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32916",
       "triggerID" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2fab9dc17e3a008d69e9ef71548a26b3a7996dd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32999",
       "triggerID" : "2fab9dc17e3a008d69e9ef71548a26b3a7996dd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b70bc0e6248848030a86e60ccadfb071d92762c",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33146",
       "triggerID" : "7b70bc0e6248848030a86e60ccadfb071d92762c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac60fac6097229637711be838e7dd586acb47e5f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ac60fac6097229637711be838e7dd586acb47e5f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7b70bc0e6248848030a86e60ccadfb071d92762c Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33146) 
   * ac60fac6097229637711be838e7dd586acb47e5f UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] smattheis commented on a change in pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
smattheis commented on a change in pull request #18991:
URL: https://github.com/apache/flink/pull/18991#discussion_r825751243



##########
File path: flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
##########
@@ -1763,6 +1765,66 @@ public long calculateThroughput() {
         }
     }
 
+    /**
+     * Tests mailbox metrics latency and queue size and verifies that (1) latency measurement is
+     * executed initially once and at least once triggered by timer, (2) latency mean value is
+     * greater than zero and (3) mailbox size is greater than zero for some time and eventually
+     * equals to zero.
+     *
+     * @throws Exception on {@link MockEnvironmentBuilder#build()} failure.
+     */
+    @Test
+    public void testMailboxMetrics() throws Exception {

Review comment:
       1. The test is stable in the way that any additional delay will not cause the test to fail. Note that the assertions ask for a minimum delay and minimum number of latency measurements (>= 2) and `Thread.sleep(iteration == 2 ? 10 : 1);` is only to not let the test sleep/run for too long, e.g., total execution time is currently ~34ms. (The test would be work the same way if there was only `Thread.sleep(10) or `Thread.sleep(20)`.
   2. I don't see the point of testing with testHarness.processMail because 1) it does not call the StreamTask#invoke method and, hence, won't perform the initial call of the scheduleMailboxMetrics and 2) I don't see/understand how the testHarness uses/implements timer-triggered execution of actions which is relevant part of the code to be tested.




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



[GitHub] [flink] flinkbot edited a comment on pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18991:
URL: https://github.com/apache/flink/pull/18991#issuecomment-1060268498


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32601",
       "triggerID" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "1063757856",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32828",
       "triggerID" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32916",
       "triggerID" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2fab9dc17e3a008d69e9ef71548a26b3a7996dd3",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32999",
       "triggerID" : "2fab9dc17e3a008d69e9ef71548a26b3a7996dd3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 195dd50ec12230bbc8c7a57e58b1dee43f16be33 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32916) 
   * 2fab9dc17e3a008d69e9ef71548a26b3a7996dd3 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32999) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] pnowojski merged pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
pnowojski merged pull request #18991:
URL: https://github.com/apache/flink/pull/18991


   


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



[GitHub] [flink] flinkbot edited a comment on pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18991:
URL: https://github.com/apache/flink/pull/18991#issuecomment-1060268498


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32601",
       "triggerID" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "1063757856",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32828",
       "triggerID" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32916",
       "triggerID" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2fab9dc17e3a008d69e9ef71548a26b3a7996dd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32999",
       "triggerID" : "2fab9dc17e3a008d69e9ef71548a26b3a7996dd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b70bc0e6248848030a86e60ccadfb071d92762c",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33146",
       "triggerID" : "7b70bc0e6248848030a86e60ccadfb071d92762c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac60fac6097229637711be838e7dd586acb47e5f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ac60fac6097229637711be838e7dd586acb47e5f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7b70bc0e6248848030a86e60ccadfb071d92762c Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33146) 
   * ac60fac6097229637711be838e7dd586acb47e5f UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18991:
URL: https://github.com/apache/flink/pull/18991#issuecomment-1060268498


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32601",
       "triggerID" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "1063757856",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32828",
       "triggerID" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32916",
       "triggerID" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2fab9dc17e3a008d69e9ef71548a26b3a7996dd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32999",
       "triggerID" : "2fab9dc17e3a008d69e9ef71548a26b3a7996dd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b70bc0e6248848030a86e60ccadfb071d92762c",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33146",
       "triggerID" : "7b70bc0e6248848030a86e60ccadfb071d92762c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7b70bc0e6248848030a86e60ccadfb071d92762c Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33146) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] smattheis commented on a change in pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
smattheis commented on a change in pull request #18991:
URL: https://github.com/apache/flink/pull/18991#discussion_r820768065



##########
File path: docs/content.zh/docs/ops/metrics.md
##########
@@ -1428,6 +1428,16 @@ Note that the metrics are only available via reporters.
       <td>Maximum recorded duration of a single consecutive period of the task being in the hard back pressure state in the last sampling period. Please check softBackPressuredTimeMsPerSecond and hardBackPressuredTimeMsPerSecond for more information.</td>
       <td>Gauge</td>
     </tr>
+    <tr>
+      <td>mailboxThroughputPerSecond</td>
+      <td>The number of actions processed from the task's mailbox per second over all actions which includes, e.g., checkpointing, timer, or cancellation actions.</td>
+      <td>Meter</td>
+    </tr>
+    <tr>
+      <td>mailboxThroughputLatencyUs</td>
+      <td>The latency in processing actions from the task's mailbox, i.e., an action's waiting time in the mailbox, in microseconds for actions with standard (default) priority, e.g., checkpointing actions.</td>
+      <td>Histogram</td>

Review comment:
       Only in the IDE integrated markdown renderer. There it was totally fine. (Otherwise I would have not noticed that I need to extend the rowspan.)




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



[GitHub] [flink] flinkbot edited a comment on pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18991:
URL: https://github.com/apache/flink/pull/18991#issuecomment-1060268498


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32601",
       "triggerID" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "1063757856",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32828",
       "triggerID" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32916",
       "triggerID" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7a222b87cf1721247dbc1c432107514efb5613b1 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32828) 
   * 195dd50ec12230bbc8c7a57e58b1dee43f16be33 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32916) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18991:
URL: https://github.com/apache/flink/pull/18991#issuecomment-1060268498


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32601",
       "triggerID" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "1063757856",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32828",
       "triggerID" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32916",
       "triggerID" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 195dd50ec12230bbc8c7a57e58b1dee43f16be33 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32916) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18991:
URL: https://github.com/apache/flink/pull/18991#issuecomment-1060268498


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32601",
       "triggerID" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5b9dea466c3984b5e0afef3502cf2f4addc829e6 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32601) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] smattheis commented on a change in pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
smattheis commented on a change in pull request #18991:
URL: https://github.com/apache/flink/pull/18991#discussion_r820765598



##########
File path: docs/content.zh/docs/ops/metrics.md
##########
@@ -1428,6 +1428,16 @@ Note that the metrics are only available via reporters.
       <td>Maximum recorded duration of a single consecutive period of the task being in the hard back pressure state in the last sampling period. Please check softBackPressuredTimeMsPerSecond and hardBackPressuredTimeMsPerSecond for more information.</td>
       <td>Gauge</td>
     </tr>
+    <tr>
+      <td>mailboxThroughputPerSecond</td>
+      <td>The number of actions processed from the task's mailbox per second over all actions which includes, e.g., checkpointing, timer, or cancellation actions.</td>
+      <td>Meter</td>
+    </tr>
+    <tr>
+      <td>mailboxThroughputLatencyUs</td>

Review comment:
       The latency is often below milliseconds and nanoseconds gives large numbers ... I can change it. Would do you prefer? If we need to change it, I would go for milliseconds because below or above zero is still a good metric.




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



[GitHub] [flink] flinkbot edited a comment on pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18991:
URL: https://github.com/apache/flink/pull/18991#issuecomment-1060268498


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32601",
       "triggerID" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "1063757856",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32828",
       "triggerID" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32916",
       "triggerID" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2fab9dc17e3a008d69e9ef71548a26b3a7996dd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32999",
       "triggerID" : "2fab9dc17e3a008d69e9ef71548a26b3a7996dd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b70bc0e6248848030a86e60ccadfb071d92762c",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33146",
       "triggerID" : "7b70bc0e6248848030a86e60ccadfb071d92762c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7b70bc0e6248848030a86e60ccadfb071d92762c Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33146) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18991:
URL: https://github.com/apache/flink/pull/18991#issuecomment-1060268498


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32601",
       "triggerID" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "1063757856",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32828",
       "triggerID" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32916",
       "triggerID" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2fab9dc17e3a008d69e9ef71548a26b3a7996dd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32999",
       "triggerID" : "2fab9dc17e3a008d69e9ef71548a26b3a7996dd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b70bc0e6248848030a86e60ccadfb071d92762c",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33146",
       "triggerID" : "7b70bc0e6248848030a86e60ccadfb071d92762c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac60fac6097229637711be838e7dd586acb47e5f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33157",
       "triggerID" : "ac60fac6097229637711be838e7dd586acb47e5f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7b70bc0e6248848030a86e60ccadfb071d92762c Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33146) 
   * ac60fac6097229637711be838e7dd586acb47e5f Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33157) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18991:
URL: https://github.com/apache/flink/pull/18991#issuecomment-1060268498


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32601",
       "triggerID" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5b9dea466c3984b5e0afef3502cf2f4addc829e6 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32601) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] smattheis commented on a change in pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
smattheis commented on a change in pull request #18991:
URL: https://github.com/apache/flink/pull/18991#discussion_r820776221



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/TaskIOMetricGroup.java
##########
@@ -100,6 +104,10 @@ public TaskIOMetricGroup(TaskMetricGroup parent) {
                         hardBackPressuredTimePerSecond::getMaxSingleMeasurement);
 
         this.busyTimePerSecond = gauge(MetricNames.TASK_BUSY_TIME, this::getBusyTimePerSecond);
+
+        this.mailboxThroughput = meter(MetricNames.MAILBOX_THROUGHPUT, new MeterView(60));
+        this.mailboxLatency =
+                histogram(MetricNames.MAILBOX_LATENCY, new DescriptiveStatisticsHistogram(60));

Review comment:
       In fact, it is the default parameter. The odd thing is that the constructor of MeterView requires either new SimpleCounter() or timeSpanInSeconds as parameter. So it would be necessary to add std constructors to both MeterView and DescrHistogram. However, I thought this is the best place to set the value and have it consistent for both mailbox metrics even though it's hard coded.

##########
File path: docs/content.zh/docs/ops/metrics.md
##########
@@ -1328,7 +1328,7 @@ Note that the metrics are only available via reporters.
       <td>Histogram</td>
     </tr>
     <tr>
-      <th rowspan="20"><strong>Task</strong></th>
+      <th rowspan="22"><strong>Task</strong></th>

Review comment:
       There is the TaskMetricGroup which contains the TaskMetricIOGroup. However, the TaskMetricGroup seemed to me as the place used by the application developer to add custom metrics. Though I'm not sure about it.
   If TaskMetricIOGroup is not the right place, it seems that we need to add new MetricGroup to TaskMetricGroup. What do you think?




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



[GitHub] [flink] flinkbot edited a comment on pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18991:
URL: https://github.com/apache/flink/pull/18991#issuecomment-1060268498


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32601",
       "triggerID" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "1063757856",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32828",
       "triggerID" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32916",
       "triggerID" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2fab9dc17e3a008d69e9ef71548a26b3a7996dd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32999",
       "triggerID" : "2fab9dc17e3a008d69e9ef71548a26b3a7996dd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b70bc0e6248848030a86e60ccadfb071d92762c",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33146",
       "triggerID" : "7b70bc0e6248848030a86e60ccadfb071d92762c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac60fac6097229637711be838e7dd586acb47e5f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ac60fac6097229637711be838e7dd586acb47e5f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7b70bc0e6248848030a86e60ccadfb071d92762c Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33146) 
   * ac60fac6097229637711be838e7dd586acb47e5f UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] smattheis commented on a change in pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
smattheis commented on a change in pull request #18991:
URL: https://github.com/apache/flink/pull/18991#discussion_r824852079



##########
File path: flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxProcessorTest.java
##########
@@ -117,6 +119,8 @@ public void runDefaultAction(Controller controller) throws Exception {
         MailboxProcessor mailboxProcessor = start(mailboxThread);
         mailboxProcessor.getMailboxExecutor(DEFAULT_PRIORITY).execute(() -> stop.set(true), "stop");
         mailboxThread.join();
+        Assert.assertTrue(counter.get() > 0);

Review comment:
       Equals 1 won't work as it's time-dependent. I will change to hamcrest assertThat(greater ...) - I was kind of on the track that we are restricted to Junit which is not true. My bad, stupid me.

##########
File path: flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxProcessorTest.java
##########
@@ -117,6 +119,8 @@ public void runDefaultAction(Controller controller) throws Exception {
         MailboxProcessor mailboxProcessor = start(mailboxThread);
         mailboxProcessor.getMailboxExecutor(DEFAULT_PRIORITY).execute(() -> stop.set(true), "stop");
         mailboxThread.join();
+        Assert.assertTrue(counter.get() > 0);
+        Assert.assertTrue(mailboxProcessor.getNumMailsProcessedCounter().getCount() > 0);

Review comment:
       counter.get() is not correct here as counter is the number of default action executions, not the number of mails. At the moment, the number of mails processed includes only the poison mail and manual the stop mail. So it would be 2. However, if there are changes in future that some house-keeping mails are generated and processed, one would need to adjust this test assertion. The > 0 proves that the counter is working in general. I can change it to > 1 which is IMO here better than =2.




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



[GitHub] [flink] flinkbot edited a comment on pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18991:
URL: https://github.com/apache/flink/pull/18991#issuecomment-1060268498


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32601",
       "triggerID" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "1063757856",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32828",
       "triggerID" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32916",
       "triggerID" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2fab9dc17e3a008d69e9ef71548a26b3a7996dd3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32999",
       "triggerID" : "2fab9dc17e3a008d69e9ef71548a26b3a7996dd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b70bc0e6248848030a86e60ccadfb071d92762c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7b70bc0e6248848030a86e60ccadfb071d92762c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2fab9dc17e3a008d69e9ef71548a26b3a7996dd3 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32999) 
   * 7b70bc0e6248848030a86e60ccadfb071d92762c UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18991:
URL: https://github.com/apache/flink/pull/18991#issuecomment-1060268498


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32601",
       "triggerID" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "1063757856",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32828",
       "triggerID" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32916",
       "triggerID" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2fab9dc17e3a008d69e9ef71548a26b3a7996dd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32999",
       "triggerID" : "2fab9dc17e3a008d69e9ef71548a26b3a7996dd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b70bc0e6248848030a86e60ccadfb071d92762c",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33146",
       "triggerID" : "7b70bc0e6248848030a86e60ccadfb071d92762c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac60fac6097229637711be838e7dd586acb47e5f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ac60fac6097229637711be838e7dd586acb47e5f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7b70bc0e6248848030a86e60ccadfb071d92762c Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33146) 
   * ac60fac6097229637711be838e7dd586acb47e5f UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] pnowojski commented on a change in pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #18991:
URL: https://github.com/apache/flink/pull/18991#discussion_r827989506



##########
File path: flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
##########
@@ -1763,6 +1765,108 @@ public long calculateThroughput() {
         }
     }
 
+    /**
+     * Tests mailbox metrics latency and queue size and verifies that (1) latency measurement is
+     * executed initially once and at least once triggered by timer, (2) latency max value is
+     * greater than zero and (3) mailbox size is greater than zero for some time and eventually
+     * equals to zero.
+     *
+     * <p>Note: This test uses a timeout which, if exceeded, indicates that the minimum number of
+     * latency measurements has not reached the specified minimum within specified time.
+     *
+     * @throws Exception on {@link MockEnvironmentBuilder#build()} failure.
+     */
+    @Test(timeout = 4000)
+    public void testMailboxMetrics() throws Exception {
+        try (MockEnvironment mockEnvironment = new MockEnvironmentBuilder().build()) {
+            Gauge<Integer> mailboxSizeMetric =
+                    mockEnvironment.getMetricGroup().getIOMetricGroup().getMailboxSize();
+            Histogram mailboxLatencyMetric =
+                    mockEnvironment.getMetricGroup().getIOMetricGroup().getMailboxLatency();
+            AtomicInteger maxMailboxSize = new AtomicInteger(-1);
+            final int minMeasurements = 2;
+            SupplierWithException<StreamTask, Exception> task =
+                    () ->
+                            new StreamTask<Object, StreamOperator<Object>>(mockEnvironment) {
+                                @Override
+                                protected void init() {
+                                    this.mailboxMetricsInterval = 2;
+                                }
+
+                                @Override
+                                protected void processInput(
+                                        MailboxDefaultAction.Controller controller)
+                                        throws Exception {
+                                    if (mailboxLatencyMetric.getCount() < minMeasurements) {
+                                        mailboxProcessor
+                                                .getMainMailboxExecutor()
+                                                .execute(() -> {}, "mail");
+                                        // The actual delay here is irrelevant for the test but
+                                        // delay should be at least once 10 ms to reach a measurable
+                                        // delay >~ 8 ms.
+                                        Thread.sleep(mailboxLatencyMetric.getCount() == 0 ? 10 : 1);
+                                    } else {
+                                        controller.suspendDefaultAction();
+                                        mailboxProcessor.suspend();
+                                    }
+                                    maxMailboxSize.set(
+                                            Math.max(
+                                                    maxMailboxSize.get(),
+                                                    mailboxSizeMetric.getValue()));
+                                }
+                            };
+
+            runTask(task::get).waitForTaskCompletion(false);
+
+            assertThat(
+                    mailboxLatencyMetric.getCount(),
+                    greaterThanOrEqualTo(new Long(minMeasurements)));
+            assertThat(mailboxLatencyMetric.getStatistics().getMax(), greaterThan(0L));
+            assertThat(maxMailboxSize.get(), greaterThan(0));
+            assertThat(mailboxSizeMetric.getValue(), equalTo(0));
+        }
+    }
+
+    @Test
+    public void testMailboxMetricsWithTestHarness() throws Exception {
+        final int numMails = 10;
+        StreamTaskMailboxTestHarnessBuilder<Integer> builder =
+                new StreamTaskMailboxTestHarnessBuilder<>(
+                                OneInputStreamTask::new, BasicTypeInfo.INT_TYPE_INFO)
+                        .addInput(BasicTypeInfo.INT_TYPE_INFO)
+                        .setupOutputForSingletonOperatorChain(
+                                new TestBoundedOneInputStreamOperator());
+        try (StreamTaskMailboxTestHarness<Integer> harness = builder.build()) {
+            Histogram mailboxLatencyMetric =
+                    harness.streamTask
+                            .getEnvironment()
+                            .getMetricGroup()
+                            .getIOMetricGroup()
+                            .getMailboxLatency();
+            Gauge<Integer> mailboxSizeMetric =
+                    harness.streamTask
+                            .getEnvironment()
+                            .getMetricGroup()
+                            .getIOMetricGroup()
+                            .getMailboxSize();
+
+            harness.streamTask.measureMailboxLatency();
+            for (int i = 0; i < numMails; ++i) {
+                harness.streamTask.mainMailboxExecutor.execute(() -> Thread.sleep(5), "add value");
+            }
+            harness.streamTask.measureMailboxLatency();
+
+            assertThat(mailboxSizeMetric.getValue(), greaterThanOrEqualTo(numMails));
+            assertThat(mailboxLatencyMetric.getCount(), equalTo(0L));
+
+            harness.processAll();
+
+            assertThat(mailboxSizeMetric.getValue(), equalTo(0));
+            assertThat(mailboxLatencyMetric.getCount(), equalTo(2L));
+            assertThat(mailboxLatencyMetric.getStatistics().getMax(), greaterThan(0L));

Review comment:
       `greaterThan(5 * numMails)` and less than time diff between `harness.streamTask.measureMailboxLatency();` and `harness.processAll();`?

##########
File path: flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
##########
@@ -1763,6 +1765,108 @@ public long calculateThroughput() {
         }
     }
 
+    /**
+     * Tests mailbox metrics latency and queue size and verifies that (1) latency measurement is
+     * executed initially once and at least once triggered by timer, (2) latency max value is
+     * greater than zero and (3) mailbox size is greater than zero for some time and eventually
+     * equals to zero.
+     *
+     * <p>Note: This test uses a timeout which, if exceeded, indicates that the minimum number of
+     * latency measurements has not reached the specified minimum within specified time.
+     *
+     * @throws Exception on {@link MockEnvironmentBuilder#build()} failure.
+     */
+    @Test(timeout = 4000)

Review comment:
       I would remove this timeout. Our general practise is to avoid timeouts inside the tests. If there is a deadlock/livelock, it's actually better for mvn watchdog to detect that, and kill the build with printing thread dump.
   
   Apart of that 4s is most likely too short. We have seen occasional 10-15sec freezes in the CI. 

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxProcessor.java
##########
@@ -175,6 +198,7 @@ public void close() {
     public void drain() throws Exception {
         for (final Mail mail : mailbox.drain()) {
             mail.run();
+            numMailsProcessed.inc();

Review comment:
       nit: bump, have you missed this?




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



[GitHub] [flink] flinkbot edited a comment on pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18991:
URL: https://github.com/apache/flink/pull/18991#issuecomment-1060268498


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32601",
       "triggerID" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "1063757856",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32828",
       "triggerID" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32916",
       "triggerID" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2fab9dc17e3a008d69e9ef71548a26b3a7996dd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32999",
       "triggerID" : "2fab9dc17e3a008d69e9ef71548a26b3a7996dd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b70bc0e6248848030a86e60ccadfb071d92762c",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33146",
       "triggerID" : "7b70bc0e6248848030a86e60ccadfb071d92762c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7b70bc0e6248848030a86e60ccadfb071d92762c Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33146) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18991:
URL: https://github.com/apache/flink/pull/18991#issuecomment-1060268498


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32601",
       "triggerID" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "1063757856",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32828",
       "triggerID" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32916",
       "triggerID" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2fab9dc17e3a008d69e9ef71548a26b3a7996dd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32999",
       "triggerID" : "2fab9dc17e3a008d69e9ef71548a26b3a7996dd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b70bc0e6248848030a86e60ccadfb071d92762c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33146",
       "triggerID" : "7b70bc0e6248848030a86e60ccadfb071d92762c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac60fac6097229637711be838e7dd586acb47e5f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33157",
       "triggerID" : "ac60fac6097229637711be838e7dd586acb47e5f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ac60fac6097229637711be838e7dd586acb47e5f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33157) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18991:
URL: https://github.com/apache/flink/pull/18991#issuecomment-1060268498


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32601",
       "triggerID" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5b9dea466c3984b5e0afef3502cf2f4addc829e6 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32601) 
   * de1583bca2dc4062ce5773f7dfaaeb532e1c1e27 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18991:
URL: https://github.com/apache/flink/pull/18991#issuecomment-1060268498


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32601",
       "triggerID" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "1063757856",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * de1583bca2dc4062ce5773f7dfaaeb532e1c1e27 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787) 
   * 7a222b87cf1721247dbc1c432107514efb5613b1 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] smattheis commented on a change in pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
smattheis commented on a change in pull request #18991:
URL: https://github.com/apache/flink/pull/18991#discussion_r820893233



##########
File path: docs/content.zh/docs/ops/metrics.md
##########
@@ -1428,6 +1428,16 @@ Note that the metrics are only available via reporters.
       <td>Maximum recorded duration of a single consecutive period of the task being in the hard back pressure state in the last sampling period. Please check softBackPressuredTimeMsPerSecond and hardBackPressuredTimeMsPerSecond for more information.</td>
       <td>Gauge</td>
     </tr>
+    <tr>
+      <td>mailboxThroughputPerSecond</td>
+      <td>The number of actions processed from the task's mailbox per second over all actions which includes, e.g., checkpointing, timer, or cancellation actions.</td>
+      <td>Meter</td>
+    </tr>
+    <tr>
+      <td>mailboxThroughputLatencyUs</td>

Review comment:
       Agreed on `mailboxLatencyMS` and `mailboxMailsPerSecond`.




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



[GitHub] [flink] flinkbot edited a comment on pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18991:
URL: https://github.com/apache/flink/pull/18991#issuecomment-1060268498


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32601",
       "triggerID" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "1063757856",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32828",
       "triggerID" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7a222b87cf1721247dbc1c432107514efb5613b1 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32828) 
   * 195dd50ec12230bbc8c7a57e58b1dee43f16be33 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18991:
URL: https://github.com/apache/flink/pull/18991#issuecomment-1060268498


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32601",
       "triggerID" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "1063757856",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32828",
       "triggerID" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32916",
       "triggerID" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2fab9dc17e3a008d69e9ef71548a26b3a7996dd3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32999",
       "triggerID" : "2fab9dc17e3a008d69e9ef71548a26b3a7996dd3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2fab9dc17e3a008d69e9ef71548a26b3a7996dd3 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32999) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] pnowojski commented on a change in pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #18991:
URL: https://github.com/apache/flink/pull/18991#discussion_r825998098



##########
File path: flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
##########
@@ -1763,6 +1765,66 @@ public long calculateThroughput() {
         }
     }
 
+    /**
+     * Tests mailbox metrics latency and queue size and verifies that (1) latency measurement is
+     * executed initially once and at least once triggered by timer, (2) latency mean value is
+     * greater than zero and (3) mailbox size is greater than zero for some time and eventually
+     * equals to zero.
+     *
+     * @throws Exception on {@link MockEnvironmentBuilder#build()} failure.
+     */
+    @Test
+    public void testMailboxMetrics() throws Exception {

Review comment:
       With this simple modification:
   ```
       private void scheduleMailboxMetrics() {
           systemTimerService.registerTimer(
                   systemTimerService.getCurrentProcessingTime() + mailboxMetricsInterval,
                   timestamp -> {
                       long startTime = SystemClock.getInstance().relativeTimeMillis();
                       Thread.sleep(2000);  // <<<<< added sleep
                       mainMailboxExecutor.execute(...);
       }
   ```
   the test is failing so you can see it won't be stable in case of JVM hiccups.
   
   > it does not call the StreamTask#invoke method
   
   Yes, that's the downside. If needed we can have detailed correctness test (a) implemented with the mailbox test harness, where we can do proper assertions easily complemented with a basic test (b) that performs some trivial assertion that `scheduleMailboxMetrics()` has been called in the `StreamTask#invoke`. (a) would be as I described above. (b) could be a simplified version of this test, that just do a while true loop as long as reported latency is > 0ms. 




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



[GitHub] [flink] flinkbot edited a comment on pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18991:
URL: https://github.com/apache/flink/pull/18991#issuecomment-1060268498


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32601",
       "triggerID" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "1063757856",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32828",
       "triggerID" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32916",
       "triggerID" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2fab9dc17e3a008d69e9ef71548a26b3a7996dd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32999",
       "triggerID" : "2fab9dc17e3a008d69e9ef71548a26b3a7996dd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b70bc0e6248848030a86e60ccadfb071d92762c",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33146",
       "triggerID" : "7b70bc0e6248848030a86e60ccadfb071d92762c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7b70bc0e6248848030a86e60ccadfb071d92762c Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33146) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18991:
URL: https://github.com/apache/flink/pull/18991#issuecomment-1060268498


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32601",
       "triggerID" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "1063757856",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32828",
       "triggerID" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32916",
       "triggerID" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2fab9dc17e3a008d69e9ef71548a26b3a7996dd3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32999",
       "triggerID" : "2fab9dc17e3a008d69e9ef71548a26b3a7996dd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b70bc0e6248848030a86e60ccadfb071d92762c",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33146",
       "triggerID" : "7b70bc0e6248848030a86e60ccadfb071d92762c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac60fac6097229637711be838e7dd586acb47e5f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ac60fac6097229637711be838e7dd586acb47e5f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2fab9dc17e3a008d69e9ef71548a26b3a7996dd3 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32999) 
   * 7b70bc0e6248848030a86e60ccadfb071d92762c Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33146) 
   * ac60fac6097229637711be838e7dd586acb47e5f UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18991:
URL: https://github.com/apache/flink/pull/18991#issuecomment-1060268498


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32601",
       "triggerID" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "1063757856",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * de1583bca2dc4062ce5773f7dfaaeb532e1c1e27 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] pnowojski commented on a change in pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #18991:
URL: https://github.com/apache/flink/pull/18991#discussion_r824687883



##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxProcessor.java
##########
@@ -175,6 +199,7 @@ public void close() {
     public void drain() throws Exception {
         for (final Mail mail : mailbox.drain()) {
             mail.run();
+            mailboxThroughputMeter.markEvent();

Review comment:
       > Well, with a quick look at SumCounter, I actually don't understand why this needs to extend the SimpleCounter ... looks a bit hacky to me. I assume it should rather implement the Counter interface ... not sure though.
   
   You can try it out and refactor it if you want/think it's worth it.
   
   I was thinking about one of the following ideas:
   1. Pass `SimpleCounter`, not `Counter` to the `MailboxProcessor`. 
   2. Defining `public static final class MailboxLengthCounter extends SimpleCounter {}` (or sth like that).
   
   However on a second thought, it shouldn't matter. Every enqueued mail has to go through threads synchronisation while being enqueued, so an extra virtual call here or there should be negligible in comparison. Let's keep it as it is.




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



[GitHub] [flink] smattheis commented on pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
smattheis commented on pull request #18991:
URL: https://github.com/apache/flink/pull/18991#issuecomment-1063359093


   @pnowojski The changes have been revised as discussed and are ready for 2nd review.


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



[GitHub] [flink] smattheis commented on a change in pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
smattheis commented on a change in pull request #18991:
URL: https://github.com/apache/flink/pull/18991#discussion_r820800932



##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxProcessor.java
##########
@@ -175,6 +199,7 @@ public void close() {
     public void drain() throws Exception {
         for (final Mail mail : mailbox.drain()) {
             mail.run();
+            mailboxThroughputMeter.markEvent();

Review comment:
       Well, with a quick look at SumCounter, I actually don't understand why this needs to extend the SimpleCounter ... looks a bit hacky to me. I assume it should rather implement the Counter interface ... not sure though.




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



[GitHub] [flink] pnowojski commented on a change in pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #18991:
URL: https://github.com/apache/flink/pull/18991#discussion_r820731984



##########
File path: docs/content.zh/docs/ops/metrics.md
##########
@@ -1428,6 +1428,16 @@ Note that the metrics are only available via reporters.
       <td>Maximum recorded duration of a single consecutive period of the task being in the hard back pressure state in the last sampling period. Please check softBackPressuredTimeMsPerSecond and hardBackPressuredTimeMsPerSecond for more information.</td>
       <td>Gauge</td>
     </tr>
+    <tr>

Review comment:
       what about the mailbox queue length? Should we add it as well?

##########
File path: docs/content.zh/docs/ops/metrics.md
##########
@@ -1428,6 +1428,16 @@ Note that the metrics are only available via reporters.
       <td>Maximum recorded duration of a single consecutive period of the task being in the hard back pressure state in the last sampling period. Please check softBackPressuredTimeMsPerSecond and hardBackPressuredTimeMsPerSecond for more information.</td>
       <td>Gauge</td>
     </tr>
+    <tr>
+      <td>mailboxThroughputPerSecond</td>
+      <td>The number of actions processed from the task's mailbox per second over all actions which includes, e.g., checkpointing, timer, or cancellation actions.</td>
+      <td>Meter</td>
+    </tr>
+    <tr>
+      <td>mailboxThroughputLatencyUs</td>
+      <td>The latency in processing actions from the task's mailbox, i.e., an action's waiting time in the mailbox, in microseconds for actions with standard (default) priority, e.g., checkpointing actions.</td>
+      <td>Histogram</td>

Review comment:
       Have you checked how is it being reported/displayed, especially in the WebUI?

##########
File path: docs/content.zh/docs/ops/metrics.md
##########
@@ -1428,6 +1428,16 @@ Note that the metrics are only available via reporters.
       <td>Maximum recorded duration of a single consecutive period of the task being in the hard back pressure state in the last sampling period. Please check softBackPressuredTimeMsPerSecond and hardBackPressuredTimeMsPerSecond for more information.</td>
       <td>Gauge</td>
     </tr>
+    <tr>
+      <td>mailboxThroughputPerSecond</td>

Review comment:
       `mailsPerSecond` would be more consistent with existing metrics? `taskThreadActionsPerSecond`?

##########
File path: docs/content.zh/docs/ops/metrics.md
##########
@@ -1428,6 +1428,16 @@ Note that the metrics are only available via reporters.
       <td>Maximum recorded duration of a single consecutive period of the task being in the hard back pressure state in the last sampling period. Please check softBackPressuredTimeMsPerSecond and hardBackPressuredTimeMsPerSecond for more information.</td>
       <td>Gauge</td>
     </tr>
+    <tr>
+      <td>mailboxThroughputPerSecond</td>
+      <td>The number of actions processed from the task's mailbox per second over all actions which includes, e.g., checkpointing, timer, or cancellation actions.</td>
+      <td>Meter</td>
+    </tr>
+    <tr>
+      <td>mailboxThroughputLatencyUs</td>

Review comment:
       `mailboxLatencyUs`? `taskThreadLatencyUs`?

##########
File path: docs/content.zh/docs/ops/metrics.md
##########
@@ -1428,6 +1428,16 @@ Note that the metrics are only available via reporters.
       <td>Maximum recorded duration of a single consecutive period of the task being in the hard back pressure state in the last sampling period. Please check softBackPressuredTimeMsPerSecond and hardBackPressuredTimeMsPerSecond for more information.</td>
       <td>Gauge</td>
     </tr>
+    <tr>
+      <td>mailboxThroughputPerSecond</td>
+      <td>The number of actions processed from the task's mailbox per second over all actions which includes, e.g., checkpointing, timer, or cancellation actions.</td>
+      <td>Meter</td>
+    </tr>
+    <tr>
+      <td>mailboxThroughputLatencyUs</td>

Review comment:
       Why microseconds? Everything else is either in seconds, milliseconds or nanoseconds. 

##########
File path: docs/content.zh/docs/ops/metrics.md
##########
@@ -1328,7 +1328,7 @@ Note that the metrics are only available via reporters.
       <td>Histogram</td>
     </tr>
     <tr>
-      <th rowspan="20"><strong>Task</strong></th>
+      <th rowspan="22"><strong>Task</strong></th>

Review comment:
       Do we have a better place for those metrics? They are not very "task io" related

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/TaskIOMetricGroup.java
##########
@@ -100,6 +104,10 @@ public TaskIOMetricGroup(TaskMetricGroup parent) {
                         hardBackPressuredTimePerSecond::getMaxSingleMeasurement);
 
         this.busyTimePerSecond = gauge(MetricNames.TASK_BUSY_TIME, this::getBusyTimePerSecond);
+
+        this.mailboxThroughput = meter(MetricNames.MAILBOX_THROUGHPUT, new MeterView(60));
+        this.mailboxLatency =
+                histogram(MetricNames.MAILBOX_LATENCY, new DescriptiveStatisticsHistogram(60));

Review comment:
       Why are you defining `60s`  as `timeSpanInSeconds` here? Why not using a default parameter?

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxProcessor.java
##########
@@ -175,6 +199,7 @@ public void close() {
     public void drain() throws Exception {
         for (final Mail mail : mailbox.drain()) {
             mail.run();
+            mailboxThroughputMeter.markEvent();

Review comment:
       I would be slightly worried about adding extra virtual calls overhead. I don't think this is a confirmed concern here, but since we could quite easily avoid it, wouldn't it be better to have here a simple `long counter++` here?

##########
File path: docs/content.zh/docs/ops/metrics.md
##########
@@ -1428,6 +1428,16 @@ Note that the metrics are only available via reporters.
       <td>Maximum recorded duration of a single consecutive period of the task being in the hard back pressure state in the last sampling period. Please check softBackPressuredTimeMsPerSecond and hardBackPressuredTimeMsPerSecond for more information.</td>
       <td>Gauge</td>
     </tr>
+    <tr>
+      <td>mailboxThroughputPerSecond</td>
+      <td>The number of actions processed from the task's mailbox per second over all actions which includes, e.g., checkpointing, timer, or cancellation actions.</td>
+      <td>Meter</td>
+    </tr>
+    <tr>
+      <td>mailboxThroughputLatencyUs</td>
+      <td>The latency in processing actions from the task's mailbox, i.e., an action's waiting time in the mailbox, in microseconds for actions with standard (default) priority, e.g., checkpointing actions.</td>

Review comment:
       I would mention that it is being sampled periodically, so it might not be fully accurate. 




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



[GitHub] [flink] smattheis commented on a change in pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
smattheis commented on a change in pull request #18991:
URL: https://github.com/apache/flink/pull/18991#discussion_r820766909



##########
File path: docs/content.zh/docs/ops/metrics.md
##########
@@ -1428,6 +1428,16 @@ Note that the metrics are only available via reporters.
       <td>Maximum recorded duration of a single consecutive period of the task being in the hard back pressure state in the last sampling period. Please check softBackPressuredTimeMsPerSecond and hardBackPressuredTimeMsPerSecond for more information.</td>
       <td>Gauge</td>
     </tr>
+    <tr>
+      <td>mailboxThroughputPerSecond</td>

Review comment:
       I would keep the prefix with `mailbox`. Do you think `mailboxMailsPerSecond` is okay?




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



[GitHub] [flink] smattheis commented on a change in pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
smattheis commented on a change in pull request #18991:
URL: https://github.com/apache/flink/pull/18991#discussion_r820765598



##########
File path: docs/content.zh/docs/ops/metrics.md
##########
@@ -1428,6 +1428,16 @@ Note that the metrics are only available via reporters.
       <td>Maximum recorded duration of a single consecutive period of the task being in the hard back pressure state in the last sampling period. Please check softBackPressuredTimeMsPerSecond and hardBackPressuredTimeMsPerSecond for more information.</td>
       <td>Gauge</td>
     </tr>
+    <tr>
+      <td>mailboxThroughputPerSecond</td>
+      <td>The number of actions processed from the task's mailbox per second over all actions which includes, e.g., checkpointing, timer, or cancellation actions.</td>
+      <td>Meter</td>
+    </tr>
+    <tr>
+      <td>mailboxThroughputLatencyUs</td>

Review comment:
       The latency is often below milliseconds and nanoseconds gives large numbers ... I can change it. What do you prefer? If we need to change it, I would go for milliseconds because below or above zero is still a good metric.




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



[GitHub] [flink] smattheis commented on a change in pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
smattheis commented on a change in pull request #18991:
URL: https://github.com/apache/flink/pull/18991#discussion_r820793763



##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxProcessor.java
##########
@@ -175,6 +199,7 @@ public void close() {
     public void drain() throws Exception {
         for (final Mail mail : mailbox.drain()) {
             mail.run();
+            mailboxThroughputMeter.markEvent();

Review comment:
       That means to not use an existing Metrics implementation and to come up with some implementation that calculates the throughput.
   How about making MeterView#markEvent and SimpleCounter#inc final? (I think that would avoid virtual method dispatching, right?)
   
   ``` java
       @Override
       public void markEvent() {
           this.counter.inc();
       }
   ```
   
   ``` java
       @Override
       public void inc() {
           count++;
       }
   ``` 
   
   Final should be fine for both methods since (1) no class inherits from MeterView and (2) the only class that inherits from SimpleCounter, which is SumCounter, does not override inc method.




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



[GitHub] [flink] flinkbot edited a comment on pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18991:
URL: https://github.com/apache/flink/pull/18991#issuecomment-1060268498


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32601",
       "triggerID" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * de1583bca2dc4062ce5773f7dfaaeb532e1c1e27 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18991:
URL: https://github.com/apache/flink/pull/18991#issuecomment-1060268498


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32601",
       "triggerID" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "1063757856",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32828",
       "triggerID" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7a222b87cf1721247dbc1c432107514efb5613b1 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32828) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18991:
URL: https://github.com/apache/flink/pull/18991#issuecomment-1060268498


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32601",
       "triggerID" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "1063757856",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32828",
       "triggerID" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32916",
       "triggerID" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2fab9dc17e3a008d69e9ef71548a26b3a7996dd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32999",
       "triggerID" : "2fab9dc17e3a008d69e9ef71548a26b3a7996dd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b70bc0e6248848030a86e60ccadfb071d92762c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33146",
       "triggerID" : "7b70bc0e6248848030a86e60ccadfb071d92762c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac60fac6097229637711be838e7dd586acb47e5f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33157",
       "triggerID" : "ac60fac6097229637711be838e7dd586acb47e5f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4f804908ad5239c06941496fc90faf98d1675723",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33194",
       "triggerID" : "4f804908ad5239c06941496fc90faf98d1675723",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4f804908ad5239c06941496fc90faf98d1675723 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33194) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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



[GitHub] [flink] flinkbot edited a comment on pull request #18991: [FLINK-26279][runtime] Add mailbox delay and throughput metrics

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18991:
URL: https://github.com/apache/flink/pull/18991#issuecomment-1060268498


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32601",
       "triggerID" : "5b9dea466c3984b5e0afef3502cf2f4addc829e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "de1583bca2dc4062ce5773f7dfaaeb532e1c1e27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32787",
       "triggerID" : "1063757856",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32828",
       "triggerID" : "7a222b87cf1721247dbc1c432107514efb5613b1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32916",
       "triggerID" : "195dd50ec12230bbc8c7a57e58b1dee43f16be33",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2fab9dc17e3a008d69e9ef71548a26b3a7996dd3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32999",
       "triggerID" : "2fab9dc17e3a008d69e9ef71548a26b3a7996dd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b70bc0e6248848030a86e60ccadfb071d92762c",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33146",
       "triggerID" : "7b70bc0e6248848030a86e60ccadfb071d92762c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2fab9dc17e3a008d69e9ef71548a26b3a7996dd3 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=32999) 
   * 7b70bc0e6248848030a86e60ccadfb071d92762c Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=33146) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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