You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "Piotr Nowojski (Jira)" <ji...@apache.org> on 2019/09/25 15:19:00 UTC

[jira] [Updated] (FLINK-14214) Performance regression in TwoInputStreamOperator

     [ https://issues.apache.org/jira/browse/FLINK-14214?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Piotr Nowojski updated FLINK-14214:
-----------------------------------
    Description: 
https://issues.apache.org/jira/browse/FLINK-13051 introduced a performance regression visible for example [here|http://codespeed.dak8s.net:8000/timeline/#/?exe=1,3&ben=twoInputMapSink&env=2&revs=200&equid=off&quarts=off&extr=off] as a drop on August 30th (note that the later performance improvement is caused by serialisation improvement, which is unrelated to the previous slow down). 

Probable suspect is the following {{.isDone()}} check executed inside {{StreamTwoInputProcessor}} once per record:


{code:java}
// to avoid starvation, if the input selection is ALL and availableInputsMask is not ALL,
// always try to check and set the availability of another input
// TODO: because this can be a costly operation (checking volatile inside CompletableFuture`
//  this might be optimized to only check once per processed NetworkBuffer
if (inputSelectionHandler.shouldSetAvailableForAnotherInput()) {
	checkAndSetAvailable(1 - readingInputIndex);
}
{code}


  was:
https://issues.apache.org/jira/browse/FLINK-13051 introduced a performance regression visible for example [here|http://codespeed.dak8s.net:8000/timeline/#/?exe=1,3&ben=twoInputMapSink&env=2&revs=200&equid=off&quarts=off&extr=off] as a drop on August 30th. 

Probable suspect is the following {{.isDone()}} check executed inside {{StreamTwoInputProcessor}} once per record:


{code:java}
// to avoid starvation, if the input selection is ALL and availableInputsMask is not ALL,
// always try to check and set the availability of another input
// TODO: because this can be a costly operation (checking volatile inside CompletableFuture`
//  this might be optimized to only check once per processed NetworkBuffer
if (inputSelectionHandler.shouldSetAvailableForAnotherInput()) {
	checkAndSetAvailable(1 - readingInputIndex);
}
{code}



> Performance regression in TwoInputStreamOperator
> ------------------------------------------------
>
>                 Key: FLINK-14214
>                 URL: https://issues.apache.org/jira/browse/FLINK-14214
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Task
>    Affects Versions: 1.10.0
>            Reporter: Piotr Nowojski
>            Assignee: Piotr Nowojski
>            Priority: Major
>
> https://issues.apache.org/jira/browse/FLINK-13051 introduced a performance regression visible for example [here|http://codespeed.dak8s.net:8000/timeline/#/?exe=1,3&ben=twoInputMapSink&env=2&revs=200&equid=off&quarts=off&extr=off] as a drop on August 30th (note that the later performance improvement is caused by serialisation improvement, which is unrelated to the previous slow down). 
> Probable suspect is the following {{.isDone()}} check executed inside {{StreamTwoInputProcessor}} once per record:
> {code:java}
> // to avoid starvation, if the input selection is ALL and availableInputsMask is not ALL,
> // always try to check and set the availability of another input
> // TODO: because this can be a costly operation (checking volatile inside CompletableFuture`
> //  this might be optimized to only check once per processed NetworkBuffer
> if (inputSelectionHandler.shouldSetAvailableForAnotherInput()) {
> 	checkAndSetAvailable(1 - readingInputIndex);
> }
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)