You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@beam.apache.org by Jan Lukavský <je...@seznam.cz> on 2020/05/05 21:41:21 UTC

[DISCUSS] Dealing with @Ignored tests

Hi,

it seems we are accumulating test cases (see discussion in [1]) that are 
marked as @Ignored (mostly due to flakiness), which is generally 
undesirable. Associated JIRAs seem to be open for a long time, and this 
might generally cause that we loose code coverage. Would anyone have 
idea on how to visualize these Ignored tests better? My first idea would 
be something similar to "Beam dependency check report", but that seems 
to be not the best example (which is completely different issue :)).

Jan

[1] https://github.com/apache/beam/pull/11614


Re: [DISCUSS] Dealing with @Ignored tests

Posted by Tyson Hamilton <ty...@google.com>.
A lot of these are good fixit tasks (shameless plug [1]),

  * archeology on the @Ignored tests to normalize them with proper labelling
  * checkstyle validation for @Ignored tests
  * spring cleaning on open JIRA issues

Having a Jira issue for the first 2 of these and adding the beam-fixit
label would be helpful. The third one is a bit trickier to coordinate and
may involve inventing some new process, or growing the group of dedicated
contributors for triaging/cleaning the backlog.

[1]:
https://lists.apache.org/thread.html/r9ddc77a8fee58ad02f68e2d9a7f054aab3e55717cc88ad1d5bc49311%40%3Cdev.beam.apache.org%3E

On Wed, May 20, 2020 at 4:22 PM Kyle Weaver <kc...@google.com> wrote:

> > I think that after 2-3 years being ignored the tests might have already
> lost their relevance.
>
> This is true of all jira issues, and I think consensus on another thread
> was that we don't want to auto-close issues.
> https://lists.apache.org/thread.html/rb51dfffbc8caf40efe7e1d137402438a05d0375fd945bda8fd7e33d2%40%3Cdev.beam.apache.org%3E
>
> I think the best suggestion from that thread was that we do "spring
> cleaning" on open JIRA issues. I know some contributors are doing that
> already, but it'd be great if we could coordinate a wider-scale effort as
> opposed to just a handful of dedicated contributors trying to do everything.
>
> On Mon, May 18, 2020 at 4:21 AM Jan Lukavský <je...@seznam.cz> wrote:
>
>> Hi,
>>
>> +1 for creating a checkstyle validation that we have associated JIRA with
>> each Ignored test. But it seems to me we might need something more, because
>> some of the associated JIRAs are open for years. I think that after 2-3
>> years being ignored the tests might have already lost their relevance.
>>
>> Jan
>> On 5/15/20 10:20 PM, Luke Cwik wrote:
>>
>> For the ones without the label, someone would need to use blame and track
>> back to why it was sickbayed.
>>
>> On Fri, May 15, 2020 at 1:08 PM Kenneth Knowles <ke...@apache.org> wrote:
>>
>>> There are 101 instances of @Ignore, and I've listed them below. A few
>>> takeaways:
>>>
>>>  - highly concentrated in ZetaSQL, and then second tier in various state
>>> tests specific to a runner
>>>  - there are not that many overall, so I'm not sure a report will add
>>> much
>>>  - they do not all have Jiras
>>>  - they do not even all have any explanation at all (some don't leave
>>> out the string parameter, but have an empty string!)
>>>
>>> Having a checkstyle that there is a Jira attached seems nice. Then we
>>> could easily grep out the Jiras and not depend on the "sickbay" label.
>>>
>>> Raw data (to see the individual items, just do the grep and not the
>>> processing)
>>>
>>>   % grep --recursive --exclude-dir build '@Ignore' . | cut -d ' ' -f 1 |
>>> sort | uniq -c | sort -r
>>>   27
>>> ./sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLDialectSpecTest.java:
>>>   11
>>> ./runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkBroadcastStateInternalsTest.java:
>>>    7
>>> ./runners/spark/src/test/java/org/apache/beam/runners/spark/stateful/SparkStateInternalsTest.java:
>>>    7
>>> ./runners/apex/src/test/java/org/apache/beam/runners/apex/translation/utils/ApexStateInternalsTest.java:
>>>    4
>>> ./sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/QueryTest.java:
>>>    4
>>> ./runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/StructuredStreamingPipelineStateTest.java:
>>>    2
>>> ./sdks/java/io/xml/src/test/java/org/apache/beam/sdk/io/xml/XmlSourceTest.java:
>>>    2
>>> ./sdks/java/io/mqtt/src/test/java/org/apache/beam/sdk/io/mqtt/MqttIOTest.java:
>>>    2
>>> ./sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsub/PubsubJsonIT.java:
>>>    2
>>> ./sdks/java/extensions/sql/jdbc/src/test/java/org/apache/beam/sdk/extensions/sql/jdbc/BeamSqlLineIT.java:
>>>    2
>>> ./sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/ReduceByKeyTest.java:
>>>    2
>>> ./sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java:
>>>    2
>>> ./sdks/java/core/src/test/java/org/apache/beam/sdk/coders/PCollectionCustomCoderTest.java:
>>>    2
>>> ./runners/direct-java/src/test/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutorTest.java:
>>>    1
>>> ./sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/sources/UnboundedEventSourceTest.java:
>>>    1
>>> ./sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery5Test.java:
>>>    1
>>> ./sdks/java/io/kudu/src/test/java/org/apache/beam/sdk/io/kudu/KuduIOTest.java:
>>>    1
>>> ./sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java:
>>>    1
>>> ./sdks/java/io/hadoop-file-system/src/test/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemTest.java:
>>>    1
>>> ./sdks/java/io/clickhouse/src/test/java/org/apache/beam/sdk/io/clickhouse/ClickHouseIOTest.java:
>>>    1
>>> ./sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/dynamodb/DynamoDBIOTest.java:@Ignore
>>> ("[BEAM-7794]
>>>    1
>>> ./sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/dynamodb/DynamoDBIOTest.java:@Ignore
>>> ("[BEAM-7794]
>>>    1
>>> ./sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaCSVTableIT.java:@Ignore
>>> ("https://issues.apache.org/jira/projects/BEAM/issues/BEAM-7523")
>>>    1
>>> ./sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriverTest.java:
>>>    1
>>> ./sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlExplainTest.java:
>>>    1
>>> ./sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslSqlStdOperatorsTest.java:
>>>    1
>>> ./sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamComplexTypeTest.java:
>>>    1
>>> ./sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/JoinTest.java:
>>>    1
>>> ./sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/docs/DocumentationExamplesTest.java:
>>>    1
>>> ./sdks/java/core/src/test/java/org/apache/beam/sdk/values/PDoneTest.java:
>>>    1
>>> ./sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WatchTest.java:
>>>    1
>>> ./sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SplittableDoFnTest.java:
>>>    1
>>> ./sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java:
>>>    1
>>> ./sdks/java/core/src/test/java/org/apache/beam/sdk/coders/RowCoderTest.java:
>>>    1
>>> ./runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/streaming/SimpleSourceTest.java:
>>>    1
>>> ./runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/metrics/sink/SparkMetricsSinkTest.java:@Ignore
>>> ("Has
>>>    1
>>> ./runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BatchStatefulParDoOverridesTest.java:
>>>    1
>>> ./runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java:
>>>    1
>>> ./runners/apex/src/test/java/org/apache/beam/runners/apex/translation/ParDoTranslatorTest.java:
>>>
>>>
>>> Kenn
>>>
>>> On Tue, May 12, 2020 at 1:24 PM Mikhail Gryzykhin <
>>> gryzykhin.mikhail@gmail.com> wrote:
>>>
>>>> I wonder if we can add graph to community metrics showing ignored tests
>>>> by language/project/overall. That can be useful to see focus area.
>>>>
>>>> On Tue, May 12, 2020 at 12:28 PM Jan Lukavský <je...@seznam.cz> wrote:
>>>>
>>>>> +1, visualizing the number of ignored tests in a graph seems useful.
>>>>> Even better with some slices (e.g. per runner, module, ...).
>>>>> On 5/12/20 8:02 PM, Ahmet Altay wrote:
>>>>>
>>>>> +1 to generate a report instead of removing these tests. A report like
>>>>> this could help us with prioritization. It is easier to address issues when
>>>>> we can quantify how much of a problem it is.
>>>>>
>>>>> I am curious what we can do to incentivize reducing the number of
>>>>> flaky/ignored tests? A report itself might provide incentive, it is
>>>>> rewarding to see ignored tests numbers go down over time.
>>>>>
>>>>> On Mon, May 11, 2020 at 8:30 AM Luke Cwik <lc...@google.com> wrote:
>>>>>
>>>>>> Deleting ignored tests does lead us to losing the reason as to why
>>>>>> the test case was around so I would rather keep it around. I think it would
>>>>>> be more valuable to generate a report that goes on the website/wiki showing
>>>>>> stability of the modules (num tests, num passed, num skipped, num failed
>>>>>> (running averages over the past N runs)). We had discussed doing something
>>>>>> like this for ValidatesRunner so we could show which runner supports what
>>>>>> automatically.
>>>>>>
>>>>>> On Mon, May 11, 2020 at 12:53 AM Jan Lukavský <je...@seznam.cz>
>>>>>> wrote:
>>>>>>
>>>>>>> I think that we do have Jira issues for ignored test, there should
>>>>>>> be no problem with that. The questionable point is that when test gets
>>>>>>> Ignored, people might consider the problem as "less painful" and postpone
>>>>>>> the correct solution until ... forever. I'd just like to discuss if people
>>>>>>> see this as an issue. If yes, should we do something about that, or if no,
>>>>>>> maybe we can create a rule that test marked as Ignored for long time might
>>>>>>> be deleted, because apparently is only a dead code.
>>>>>>> On 5/6/20 6:30 PM, Kenneth Knowles wrote:
>>>>>>>
>>>>>>> Good point.
>>>>>>>
>>>>>>> The raw numbers are available in the test run output. See
>>>>>>> https://builds.apache.org/view/A-D/view/Beam/view/PostCommit/job/beam_PreCommit_Java_Cron/2718/testReport/ for
>>>>>>> the "skipped" column.
>>>>>>> And you get the same on console or Gradle Scan:
>>>>>>> https://scans.gradle.com/s/ml3jv5xctkrmg/tests?collapse-all
>>>>>>> This would be good to review periodically for obvious trouble spots.
>>>>>>>
>>>>>>> But I think you mean something more detailed. Some report with
>>>>>>> columns: Test Suite, Test Method, Jira, Date Ignored, Most Recent Update
>>>>>>>
>>>>>>> I think we can get most of this from Jira, if we just make sure that
>>>>>>> each ignored test has a Jira and they are all labeled in a consistent way.
>>>>>>> That would be the quickest way to get some result, even though it is not
>>>>>>> perfectly automated and audited.
>>>>>>>
>>>>>>> Kenn
>>>>>>>
>>>>>>> On Tue, May 5, 2020 at 2:41 PM Jan Lukavský <je...@seznam.cz> wrote:
>>>>>>>
>>>>>>>> Hi,
>>>>>>>>
>>>>>>>> it seems we are accumulating test cases (see discussion in [1])
>>>>>>>> that are
>>>>>>>> marked as @Ignored (mostly due to flakiness), which is generally
>>>>>>>> undesirable. Associated JIRAs seem to be open for a long time, and
>>>>>>>> this
>>>>>>>> might generally cause that we loose code coverage. Would anyone
>>>>>>>> have
>>>>>>>> idea on how to visualize these Ignored tests better? My first idea
>>>>>>>> would
>>>>>>>> be something similar to "Beam dependency check report", but that
>>>>>>>> seems
>>>>>>>> to be not the best example (which is completely different issue :)).
>>>>>>>>
>>>>>>>> Jan
>>>>>>>>
>>>>>>>> [1] https://github.com/apache/beam/pull/11614
>>>>>>>>
>>>>>>>>

Re: [DISCUSS] Dealing with @Ignored tests

Posted by Kyle Weaver <kc...@google.com>.
> I think that after 2-3 years being ignored the tests might have already
lost their relevance.

This is true of all jira issues, and I think consensus on another thread
was that we don't want to auto-close issues.
https://lists.apache.org/thread.html/rb51dfffbc8caf40efe7e1d137402438a05d0375fd945bda8fd7e33d2%40%3Cdev.beam.apache.org%3E

I think the best suggestion from that thread was that we do "spring
cleaning" on open JIRA issues. I know some contributors are doing that
already, but it'd be great if we could coordinate a wider-scale effort as
opposed to just a handful of dedicated contributors trying to do everything.

On Mon, May 18, 2020 at 4:21 AM Jan Lukavský <je...@seznam.cz> wrote:

> Hi,
>
> +1 for creating a checkstyle validation that we have associated JIRA with
> each Ignored test. But it seems to me we might need something more, because
> some of the associated JIRAs are open for years. I think that after 2-3
> years being ignored the tests might have already lost their relevance.
>
> Jan
> On 5/15/20 10:20 PM, Luke Cwik wrote:
>
> For the ones without the label, someone would need to use blame and track
> back to why it was sickbayed.
>
> On Fri, May 15, 2020 at 1:08 PM Kenneth Knowles <ke...@apache.org> wrote:
>
>> There are 101 instances of @Ignore, and I've listed them below. A few
>> takeaways:
>>
>>  - highly concentrated in ZetaSQL, and then second tier in various state
>> tests specific to a runner
>>  - there are not that many overall, so I'm not sure a report will add much
>>  - they do not all have Jiras
>>  - they do not even all have any explanation at all (some don't leave out
>> the string parameter, but have an empty string!)
>>
>> Having a checkstyle that there is a Jira attached seems nice. Then we
>> could easily grep out the Jiras and not depend on the "sickbay" label.
>>
>> Raw data (to see the individual items, just do the grep and not the
>> processing)
>>
>>   % grep --recursive --exclude-dir build '@Ignore' . | cut -d ' ' -f 1 |
>> sort | uniq -c | sort -r
>>   27
>> ./sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLDialectSpecTest.java:
>>   11
>> ./runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkBroadcastStateInternalsTest.java:
>>    7
>> ./runners/spark/src/test/java/org/apache/beam/runners/spark/stateful/SparkStateInternalsTest.java:
>>    7
>> ./runners/apex/src/test/java/org/apache/beam/runners/apex/translation/utils/ApexStateInternalsTest.java:
>>    4
>> ./sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/QueryTest.java:
>>    4
>> ./runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/StructuredStreamingPipelineStateTest.java:
>>    2
>> ./sdks/java/io/xml/src/test/java/org/apache/beam/sdk/io/xml/XmlSourceTest.java:
>>    2
>> ./sdks/java/io/mqtt/src/test/java/org/apache/beam/sdk/io/mqtt/MqttIOTest.java:
>>    2
>> ./sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsub/PubsubJsonIT.java:
>>    2
>> ./sdks/java/extensions/sql/jdbc/src/test/java/org/apache/beam/sdk/extensions/sql/jdbc/BeamSqlLineIT.java:
>>    2
>> ./sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/ReduceByKeyTest.java:
>>    2
>> ./sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java:
>>    2
>> ./sdks/java/core/src/test/java/org/apache/beam/sdk/coders/PCollectionCustomCoderTest.java:
>>    2
>> ./runners/direct-java/src/test/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutorTest.java:
>>    1
>> ./sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/sources/UnboundedEventSourceTest.java:
>>    1
>> ./sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery5Test.java:
>>    1
>> ./sdks/java/io/kudu/src/test/java/org/apache/beam/sdk/io/kudu/KuduIOTest.java:
>>    1
>> ./sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java:
>>    1
>> ./sdks/java/io/hadoop-file-system/src/test/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemTest.java:
>>    1
>> ./sdks/java/io/clickhouse/src/test/java/org/apache/beam/sdk/io/clickhouse/ClickHouseIOTest.java:
>>    1
>> ./sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/dynamodb/DynamoDBIOTest.java:@Ignore
>> ("[BEAM-7794]
>>    1
>> ./sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/dynamodb/DynamoDBIOTest.java:@Ignore
>> ("[BEAM-7794]
>>    1
>> ./sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaCSVTableIT.java:@Ignore
>> ("https://issues.apache.org/jira/projects/BEAM/issues/BEAM-7523")
>>    1
>> ./sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriverTest.java:
>>    1
>> ./sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlExplainTest.java:
>>    1
>> ./sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslSqlStdOperatorsTest.java:
>>    1
>> ./sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamComplexTypeTest.java:
>>    1
>> ./sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/JoinTest.java:
>>    1
>> ./sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/docs/DocumentationExamplesTest.java:
>>    1
>> ./sdks/java/core/src/test/java/org/apache/beam/sdk/values/PDoneTest.java:
>>    1
>> ./sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WatchTest.java:
>>    1
>> ./sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SplittableDoFnTest.java:
>>    1
>> ./sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java:
>>    1
>> ./sdks/java/core/src/test/java/org/apache/beam/sdk/coders/RowCoderTest.java:
>>    1
>> ./runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/streaming/SimpleSourceTest.java:
>>    1
>> ./runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/metrics/sink/SparkMetricsSinkTest.java:@Ignore
>> ("Has
>>    1
>> ./runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BatchStatefulParDoOverridesTest.java:
>>    1
>> ./runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java:
>>    1
>> ./runners/apex/src/test/java/org/apache/beam/runners/apex/translation/ParDoTranslatorTest.java:
>>
>>
>> Kenn
>>
>> On Tue, May 12, 2020 at 1:24 PM Mikhail Gryzykhin <
>> gryzykhin.mikhail@gmail.com> wrote:
>>
>>> I wonder if we can add graph to community metrics showing ignored tests
>>> by language/project/overall. That can be useful to see focus area.
>>>
>>> On Tue, May 12, 2020 at 12:28 PM Jan Lukavský <je...@seznam.cz> wrote:
>>>
>>>> +1, visualizing the number of ignored tests in a graph seems useful.
>>>> Even better with some slices (e.g. per runner, module, ...).
>>>> On 5/12/20 8:02 PM, Ahmet Altay wrote:
>>>>
>>>> +1 to generate a report instead of removing these tests. A report like
>>>> this could help us with prioritization. It is easier to address issues when
>>>> we can quantify how much of a problem it is.
>>>>
>>>> I am curious what we can do to incentivize reducing the number of
>>>> flaky/ignored tests? A report itself might provide incentive, it is
>>>> rewarding to see ignored tests numbers go down over time.
>>>>
>>>> On Mon, May 11, 2020 at 8:30 AM Luke Cwik <lc...@google.com> wrote:
>>>>
>>>>> Deleting ignored tests does lead us to losing the reason as to why the
>>>>> test case was around so I would rather keep it around. I think it would be
>>>>> more valuable to generate a report that goes on the website/wiki showing
>>>>> stability of the modules (num tests, num passed, num skipped, num failed
>>>>> (running averages over the past N runs)). We had discussed doing something
>>>>> like this for ValidatesRunner so we could show which runner supports what
>>>>> automatically.
>>>>>
>>>>> On Mon, May 11, 2020 at 12:53 AM Jan Lukavský <je...@seznam.cz> wrote:
>>>>>
>>>>>> I think that we do have Jira issues for ignored test, there should be
>>>>>> no problem with that. The questionable point is that when test gets
>>>>>> Ignored, people might consider the problem as "less painful" and postpone
>>>>>> the correct solution until ... forever. I'd just like to discuss if people
>>>>>> see this as an issue. If yes, should we do something about that, or if no,
>>>>>> maybe we can create a rule that test marked as Ignored for long time might
>>>>>> be deleted, because apparently is only a dead code.
>>>>>> On 5/6/20 6:30 PM, Kenneth Knowles wrote:
>>>>>>
>>>>>> Good point.
>>>>>>
>>>>>> The raw numbers are available in the test run output. See
>>>>>> https://builds.apache.org/view/A-D/view/Beam/view/PostCommit/job/beam_PreCommit_Java_Cron/2718/testReport/ for
>>>>>> the "skipped" column.
>>>>>> And you get the same on console or Gradle Scan:
>>>>>> https://scans.gradle.com/s/ml3jv5xctkrmg/tests?collapse-all
>>>>>> This would be good to review periodically for obvious trouble spots.
>>>>>>
>>>>>> But I think you mean something more detailed. Some report with
>>>>>> columns: Test Suite, Test Method, Jira, Date Ignored, Most Recent Update
>>>>>>
>>>>>> I think we can get most of this from Jira, if we just make sure that
>>>>>> each ignored test has a Jira and they are all labeled in a consistent way.
>>>>>> That would be the quickest way to get some result, even though it is not
>>>>>> perfectly automated and audited.
>>>>>>
>>>>>> Kenn
>>>>>>
>>>>>> On Tue, May 5, 2020 at 2:41 PM Jan Lukavský <je...@seznam.cz> wrote:
>>>>>>
>>>>>>> Hi,
>>>>>>>
>>>>>>> it seems we are accumulating test cases (see discussion in [1]) that
>>>>>>> are
>>>>>>> marked as @Ignored (mostly due to flakiness), which is generally
>>>>>>> undesirable. Associated JIRAs seem to be open for a long time, and
>>>>>>> this
>>>>>>> might generally cause that we loose code coverage. Would anyone have
>>>>>>> idea on how to visualize these Ignored tests better? My first idea
>>>>>>> would
>>>>>>> be something similar to "Beam dependency check report", but that
>>>>>>> seems
>>>>>>> to be not the best example (which is completely different issue :)).
>>>>>>>
>>>>>>> Jan
>>>>>>>
>>>>>>> [1] https://github.com/apache/beam/pull/11614
>>>>>>>
>>>>>>>

Re: [DISCUSS] Dealing with @Ignored tests

Posted by Jan Lukavský <je...@seznam.cz>.
Hi,

+1 for creating a checkstyle validation that we have associated JIRA 
with each Ignored test. But it seems to me we might need something more, 
because some of the associated JIRAs are open for years. I think that 
after 2-3 years being ignored the tests might have already lost their 
relevance.

Jan

On 5/15/20 10:20 PM, Luke Cwik wrote:
> For the ones without the label, someone would need to use blame and 
> track back to why it was sickbayed.
>
> On Fri, May 15, 2020 at 1:08 PM Kenneth Knowles <kenn@apache.org 
> <ma...@apache.org>> wrote:
>
>     There are 101 instances of @Ignore, and I've listed them below. A
>     few takeaways:
>
>      - highly concentrated in ZetaSQL, and then second tier in various
>     state tests specific to a runner
>      - there are not that many overall, so I'm not sure a report will
>     add much
>      - they do not all have Jiras
>      - they do not even all have any explanation at all (some don't
>     leave out the string parameter, but have an empty string!)
>
>     Having a checkstyle that there is a Jira attached seems nice. Then
>     we could easily grep out the Jiras and not depend on the "sickbay"
>     label.
>
>     Raw data (to see the individual items, just do the grep and not
>     the processing)
>
>       % grep --recursive --exclude-dir build '@Ignore' . | cut -d ' '
>     -f 1 | sort | uniq -c | sort -r
>       27
>     ./sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLDialectSpecTest.java:
>       11
>     ./runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkBroadcastStateInternalsTest.java:
>        7
>     ./runners/spark/src/test/java/org/apache/beam/runners/spark/stateful/SparkStateInternalsTest.java:
>        7
>     ./runners/apex/src/test/java/org/apache/beam/runners/apex/translation/utils/ApexStateInternalsTest.java:
>        4
>     ./sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/QueryTest.java:
>        4
>     ./runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/StructuredStreamingPipelineStateTest.java:
>        2
>     ./sdks/java/io/xml/src/test/java/org/apache/beam/sdk/io/xml/XmlSourceTest.java:
>        2
>     ./sdks/java/io/mqtt/src/test/java/org/apache/beam/sdk/io/mqtt/MqttIOTest.java:
>        2
>     ./sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsub/PubsubJsonIT.java:
>        2
>     ./sdks/java/extensions/sql/jdbc/src/test/java/org/apache/beam/sdk/extensions/sql/jdbc/BeamSqlLineIT.java:
>        2
>     ./sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/ReduceByKeyTest.java:
>        2
>     ./sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java:
>        2
>     ./sdks/java/core/src/test/java/org/apache/beam/sdk/coders/PCollectionCustomCoderTest.java:
>        2
>     ./runners/direct-java/src/test/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutorTest.java:
>        1
>     ./sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/sources/UnboundedEventSourceTest.java:
>        1
>     ./sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery5Test.java:
>        1
>     ./sdks/java/io/kudu/src/test/java/org/apache/beam/sdk/io/kudu/KuduIOTest.java:
>        1
>     ./sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java:
>        1
>     ./sdks/java/io/hadoop-file-system/src/test/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemTest.java:
>        1
>     ./sdks/java/io/clickhouse/src/test/java/org/apache/beam/sdk/io/clickhouse/ClickHouseIOTest.java:
>        1
>     ./sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/dynamodb/DynamoDBIOTest.java:@Ignore("[BEAM-7794]
>        1
>     ./sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/dynamodb/DynamoDBIOTest.java:@Ignore("[BEAM-7794]
>        1
>     ./sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaCSVTableIT.java:@Ignore("https://issues.apache.org/jira/projects/BEAM/issues/BEAM-7523")
>        1
>     ./sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriverTest.java:
>        1
>     ./sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlExplainTest.java:
>        1
>     ./sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslSqlStdOperatorsTest.java:
>        1
>     ./sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamComplexTypeTest.java:
>        1
>     ./sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/JoinTest.java:
>        1
>     ./sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/docs/DocumentationExamplesTest.java:
>        1
>     ./sdks/java/core/src/test/java/org/apache/beam/sdk/values/PDoneTest.java:
>        1
>     ./sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WatchTest.java:
>        1
>     ./sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SplittableDoFnTest.java:
>        1
>     ./sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java:
>        1
>     ./sdks/java/core/src/test/java/org/apache/beam/sdk/coders/RowCoderTest.java:
>        1
>     ./runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/streaming/SimpleSourceTest.java:
>        1
>     ./runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/metrics/sink/SparkMetricsSinkTest.java:@Ignore("Has
>        1
>     ./runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BatchStatefulParDoOverridesTest.java:
>        1
>     ./runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java:
>        1
>     ./runners/apex/src/test/java/org/apache/beam/runners/apex/translation/ParDoTranslatorTest.java:
>
>
>     Kenn
>
>     On Tue, May 12, 2020 at 1:24 PM Mikhail Gryzykhin
>     <gryzykhin.mikhail@gmail.com <ma...@gmail.com>>
>     wrote:
>
>         I wonder if we can add graph to community metrics showing
>         ignored tests by language/project/overall. That can be useful
>         to see focus area.
>
>         On Tue, May 12, 2020 at 12:28 PM Jan Lukavský <je.ik@seznam.cz
>         <ma...@seznam.cz>> wrote:
>
>             +1, visualizing the number of ignored tests in a graph
>             seems useful. Even better with some slices (e.g. per
>             runner, module, ...).
>
>             On 5/12/20 8:02 PM, Ahmet Altay wrote:
>>             +1 to generate a report instead of removing these tests.
>>             A report like this could help us with prioritization. It
>>             is easier to address issues when we can quantify how much
>>             of a problem it is.
>>
>>             I am curious what we can do to incentivize reducing the
>>             number of flaky/ignored tests? A report itself might
>>             provide incentive, it is rewarding to see ignored tests
>>             numbers go down over time.
>>
>>             On Mon, May 11, 2020 at 8:30 AM Luke Cwik
>>             <lcwik@google.com <ma...@google.com>> wrote:
>>
>>                 Deleting ignored tests does lead us to losing the
>>                 reason as to why the test case was around so I would
>>                 rather keep it around. I think it would be more
>>                 valuable to generate a report that goes on the
>>                 website/wiki showing stability of the modules (num
>>                 tests, num passed, num skipped, num failed (running
>>                 averages over the past N runs)). We had discussed
>>                 doing something like this for ValidatesRunner so we
>>                 could show which runner supports what automatically.
>>
>>                 On Mon, May 11, 2020 at 12:53 AM Jan Lukavský
>>                 <je.ik@seznam.cz <ma...@seznam.cz>> wrote:
>>
>>                     I think that we do have Jira issues for ignored
>>                     test, there should be no problem with that. The
>>                     questionable point is that when test gets
>>                     Ignored, people might consider the problem as
>>                     "less painful" and postpone the correct solution
>>                     until ... forever. I'd just like to discuss if
>>                     people see this as an issue. If yes, should we do
>>                     something about that, or if no, maybe we can
>>                     create a rule that test marked as Ignored for
>>                     long time might be deleted, because apparently is
>>                     only a dead code.
>>
>>                     On 5/6/20 6:30 PM, Kenneth Knowles wrote:
>>>                     Good point.
>>>
>>>                     The raw numbers are available in the test run
>>>                     output. See
>>>                     https://builds.apache.org/view/A-D/view/Beam/view/PostCommit/job/beam_PreCommit_Java_Cron/2718/testReport/ for
>>>                     the "skipped" column.
>>>                     And you get the same on console or Gradle Scan:
>>>                     https://scans.gradle.com/s/ml3jv5xctkrmg/tests?collapse-all
>>>                     This would be good to review periodically for
>>>                     obvious trouble spots.
>>>
>>>                     But I think you mean something more detailed.
>>>                     Some report with columns: Test Suite, Test
>>>                     Method, Jira, Date Ignored, Most Recent Update
>>>
>>>                     I think we can get most of this from Jira, if we
>>>                     just make sure that each ignored test has a Jira
>>>                     and they are all labeled in a consistent way.
>>>                     That would be the quickest way to get some
>>>                     result, even though it is not perfectly
>>>                     automated and audited.
>>>
>>>                     Kenn
>>>
>>>                     On Tue, May 5, 2020 at 2:41 PM Jan Lukavský
>>>                     <je.ik@seznam.cz <ma...@seznam.cz>> wrote:
>>>
>>>                         Hi,
>>>
>>>                         it seems we are accumulating test cases (see
>>>                         discussion in [1]) that are
>>>                         marked as @Ignored (mostly due to
>>>                         flakiness), which is generally
>>>                         undesirable. Associated JIRAs seem to be
>>>                         open for a long time, and this
>>>                         might generally cause that we loose code
>>>                         coverage. Would anyone have
>>>                         idea on how to visualize these Ignored tests
>>>                         better? My first idea would
>>>                         be something similar to "Beam dependency
>>>                         check report", but that seems
>>>                         to be not the best example (which is
>>>                         completely different issue :)).
>>>
>>>                         Jan
>>>
>>>                         [1] https://github.com/apache/beam/pull/11614
>>>

Re: [DISCUSS] Dealing with @Ignored tests

Posted by Luke Cwik <lc...@google.com>.
For the ones without the label, someone would need to use blame and track
back to why it was sickbayed.

On Fri, May 15, 2020 at 1:08 PM Kenneth Knowles <ke...@apache.org> wrote:

> There are 101 instances of @Ignore, and I've listed them below. A few
> takeaways:
>
>  - highly concentrated in ZetaSQL, and then second tier in various state
> tests specific to a runner
>  - there are not that many overall, so I'm not sure a report will add much
>  - they do not all have Jiras
>  - they do not even all have any explanation at all (some don't leave out
> the string parameter, but have an empty string!)
>
> Having a checkstyle that there is a Jira attached seems nice. Then we
> could easily grep out the Jiras and not depend on the "sickbay" label.
>
> Raw data (to see the individual items, just do the grep and not the
> processing)
>
>   % grep --recursive --exclude-dir build '@Ignore' . | cut -d ' ' -f 1 |
> sort | uniq -c | sort -r
>   27
> ./sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLDialectSpecTest.java:
>   11
> ./runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkBroadcastStateInternalsTest.java:
>    7
> ./runners/spark/src/test/java/org/apache/beam/runners/spark/stateful/SparkStateInternalsTest.java:
>    7
> ./runners/apex/src/test/java/org/apache/beam/runners/apex/translation/utils/ApexStateInternalsTest.java:
>    4
> ./sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/QueryTest.java:
>    4
> ./runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/StructuredStreamingPipelineStateTest.java:
>    2
> ./sdks/java/io/xml/src/test/java/org/apache/beam/sdk/io/xml/XmlSourceTest.java:
>    2
> ./sdks/java/io/mqtt/src/test/java/org/apache/beam/sdk/io/mqtt/MqttIOTest.java:
>    2
> ./sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsub/PubsubJsonIT.java:
>    2
> ./sdks/java/extensions/sql/jdbc/src/test/java/org/apache/beam/sdk/extensions/sql/jdbc/BeamSqlLineIT.java:
>    2
> ./sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/ReduceByKeyTest.java:
>    2
> ./sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java:
>    2
> ./sdks/java/core/src/test/java/org/apache/beam/sdk/coders/PCollectionCustomCoderTest.java:
>    2
> ./runners/direct-java/src/test/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutorTest.java:
>    1
> ./sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/sources/UnboundedEventSourceTest.java:
>    1
> ./sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery5Test.java:
>    1
> ./sdks/java/io/kudu/src/test/java/org/apache/beam/sdk/io/kudu/KuduIOTest.java:
>    1
> ./sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java:
>    1
> ./sdks/java/io/hadoop-file-system/src/test/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemTest.java:
>    1
> ./sdks/java/io/clickhouse/src/test/java/org/apache/beam/sdk/io/clickhouse/ClickHouseIOTest.java:
>    1
> ./sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/dynamodb/DynamoDBIOTest.java:@Ignore
> ("[BEAM-7794]
>    1
> ./sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/dynamodb/DynamoDBIOTest.java:@Ignore
> ("[BEAM-7794]
>    1
> ./sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaCSVTableIT.java:@Ignore
> ("https://issues.apache.org/jira/projects/BEAM/issues/BEAM-7523")
>    1
> ./sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriverTest.java:
>    1
> ./sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlExplainTest.java:
>    1
> ./sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslSqlStdOperatorsTest.java:
>    1
> ./sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamComplexTypeTest.java:
>    1
> ./sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/JoinTest.java:
>    1
> ./sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/docs/DocumentationExamplesTest.java:
>    1
> ./sdks/java/core/src/test/java/org/apache/beam/sdk/values/PDoneTest.java:
>    1
> ./sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WatchTest.java:
>    1
> ./sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SplittableDoFnTest.java:
>    1
> ./sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java:
>    1
> ./sdks/java/core/src/test/java/org/apache/beam/sdk/coders/RowCoderTest.java:
>    1
> ./runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/streaming/SimpleSourceTest.java:
>    1
> ./runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/metrics/sink/SparkMetricsSinkTest.java:@Ignore
> ("Has
>    1
> ./runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BatchStatefulParDoOverridesTest.java:
>    1
> ./runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java:
>    1
> ./runners/apex/src/test/java/org/apache/beam/runners/apex/translation/ParDoTranslatorTest.java:
>
> Kenn
>
> On Tue, May 12, 2020 at 1:24 PM Mikhail Gryzykhin <
> gryzykhin.mikhail@gmail.com> wrote:
>
>> I wonder if we can add graph to community metrics showing ignored tests
>> by language/project/overall. That can be useful to see focus area.
>>
>> On Tue, May 12, 2020 at 12:28 PM Jan Lukavský <je...@seznam.cz> wrote:
>>
>>> +1, visualizing the number of ignored tests in a graph seems useful.
>>> Even better with some slices (e.g. per runner, module, ...).
>>> On 5/12/20 8:02 PM, Ahmet Altay wrote:
>>>
>>> +1 to generate a report instead of removing these tests. A report like
>>> this could help us with prioritization. It is easier to address issues when
>>> we can quantify how much of a problem it is.
>>>
>>> I am curious what we can do to incentivize reducing the number of
>>> flaky/ignored tests? A report itself might provide incentive, it is
>>> rewarding to see ignored tests numbers go down over time.
>>>
>>> On Mon, May 11, 2020 at 8:30 AM Luke Cwik <lc...@google.com> wrote:
>>>
>>>> Deleting ignored tests does lead us to losing the reason as to why the
>>>> test case was around so I would rather keep it around. I think it would be
>>>> more valuable to generate a report that goes on the website/wiki showing
>>>> stability of the modules (num tests, num passed, num skipped, num failed
>>>> (running averages over the past N runs)). We had discussed doing something
>>>> like this for ValidatesRunner so we could show which runner supports what
>>>> automatically.
>>>>
>>>> On Mon, May 11, 2020 at 12:53 AM Jan Lukavský <je...@seznam.cz> wrote:
>>>>
>>>>> I think that we do have Jira issues for ignored test, there should be
>>>>> no problem with that. The questionable point is that when test gets
>>>>> Ignored, people might consider the problem as "less painful" and postpone
>>>>> the correct solution until ... forever. I'd just like to discuss if people
>>>>> see this as an issue. If yes, should we do something about that, or if no,
>>>>> maybe we can create a rule that test marked as Ignored for long time might
>>>>> be deleted, because apparently is only a dead code.
>>>>> On 5/6/20 6:30 PM, Kenneth Knowles wrote:
>>>>>
>>>>> Good point.
>>>>>
>>>>> The raw numbers are available in the test run output. See
>>>>> https://builds.apache.org/view/A-D/view/Beam/view/PostCommit/job/beam_PreCommit_Java_Cron/2718/testReport/ for
>>>>> the "skipped" column.
>>>>> And you get the same on console or Gradle Scan:
>>>>> https://scans.gradle.com/s/ml3jv5xctkrmg/tests?collapse-all
>>>>> This would be good to review periodically for obvious trouble spots.
>>>>>
>>>>> But I think you mean something more detailed. Some report with
>>>>> columns: Test Suite, Test Method, Jira, Date Ignored, Most Recent Update
>>>>>
>>>>> I think we can get most of this from Jira, if we just make sure that
>>>>> each ignored test has a Jira and they are all labeled in a consistent way.
>>>>> That would be the quickest way to get some result, even though it is not
>>>>> perfectly automated and audited.
>>>>>
>>>>> Kenn
>>>>>
>>>>> On Tue, May 5, 2020 at 2:41 PM Jan Lukavský <je...@seznam.cz> wrote:
>>>>>
>>>>>> Hi,
>>>>>>
>>>>>> it seems we are accumulating test cases (see discussion in [1]) that
>>>>>> are
>>>>>> marked as @Ignored (mostly due to flakiness), which is generally
>>>>>> undesirable. Associated JIRAs seem to be open for a long time, and
>>>>>> this
>>>>>> might generally cause that we loose code coverage. Would anyone have
>>>>>> idea on how to visualize these Ignored tests better? My first idea
>>>>>> would
>>>>>> be something similar to "Beam dependency check report", but that
>>>>>> seems
>>>>>> to be not the best example (which is completely different issue :)).
>>>>>>
>>>>>> Jan
>>>>>>
>>>>>> [1] https://github.com/apache/beam/pull/11614
>>>>>>
>>>>>>

Re: [DISCUSS] Dealing with @Ignored tests

Posted by Kenneth Knowles <ke...@apache.org>.
There are 101 instances of @Ignore, and I've listed them below. A few
takeaways:

 - highly concentrated in ZetaSQL, and then second tier in various state
tests specific to a runner
 - there are not that many overall, so I'm not sure a report will add much
 - they do not all have Jiras
 - they do not even all have any explanation at all (some don't leave out
the string parameter, but have an empty string!)

Having a checkstyle that there is a Jira attached seems nice. Then we could
easily grep out the Jiras and not depend on the "sickbay" label.

Raw data (to see the individual items, just do the grep and not the
processing)

  % grep --recursive --exclude-dir build '@Ignore' . | cut -d ' ' -f 1 |
sort | uniq -c | sort -r
  27
./sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLDialectSpecTest.java:
  11
./runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkBroadcastStateInternalsTest.java:
   7
./runners/spark/src/test/java/org/apache/beam/runners/spark/stateful/SparkStateInternalsTest.java:
   7
./runners/apex/src/test/java/org/apache/beam/runners/apex/translation/utils/ApexStateInternalsTest.java:
   4
./sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/QueryTest.java:
   4
./runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/StructuredStreamingPipelineStateTest.java:
   2
./sdks/java/io/xml/src/test/java/org/apache/beam/sdk/io/xml/XmlSourceTest.java:
   2
./sdks/java/io/mqtt/src/test/java/org/apache/beam/sdk/io/mqtt/MqttIOTest.java:
   2
./sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsub/PubsubJsonIT.java:
   2
./sdks/java/extensions/sql/jdbc/src/test/java/org/apache/beam/sdk/extensions/sql/jdbc/BeamSqlLineIT.java:
   2
./sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/ReduceByKeyTest.java:
   2
./sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java:
   2
./sdks/java/core/src/test/java/org/apache/beam/sdk/coders/PCollectionCustomCoderTest.java:
   2
./runners/direct-java/src/test/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutorTest.java:
   1
./sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/sources/UnboundedEventSourceTest.java:
   1
./sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery5Test.java:
   1
./sdks/java/io/kudu/src/test/java/org/apache/beam/sdk/io/kudu/KuduIOTest.java:
   1
./sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java:
   1
./sdks/java/io/hadoop-file-system/src/test/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemTest.java:
   1
./sdks/java/io/clickhouse/src/test/java/org/apache/beam/sdk/io/clickhouse/ClickHouseIOTest.java:
   1
./sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/dynamodb/DynamoDBIOTest.java:@Ignore
("[BEAM-7794]
   1
./sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/dynamodb/DynamoDBIOTest.java:@Ignore
("[BEAM-7794]
   1
./sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaCSVTableIT.java:@Ignore
("https://issues.apache.org/jira/projects/BEAM/issues/BEAM-7523")
   1
./sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriverTest.java:
   1
./sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlExplainTest.java:
   1
./sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslSqlStdOperatorsTest.java:
   1
./sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamComplexTypeTest.java:
   1
./sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/JoinTest.java:
   1
./sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/docs/DocumentationExamplesTest.java:
   1
./sdks/java/core/src/test/java/org/apache/beam/sdk/values/PDoneTest.java:
   1
./sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WatchTest.java:
   1
./sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SplittableDoFnTest.java:
   1
./sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java:
   1
./sdks/java/core/src/test/java/org/apache/beam/sdk/coders/RowCoderTest.java:
   1
./runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/streaming/SimpleSourceTest.java:
   1
./runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/metrics/sink/SparkMetricsSinkTest.java:@Ignore
("Has
   1
./runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BatchStatefulParDoOverridesTest.java:
   1
./runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java:
   1
./runners/apex/src/test/java/org/apache/beam/runners/apex/translation/ParDoTranslatorTest.java:

Kenn

On Tue, May 12, 2020 at 1:24 PM Mikhail Gryzykhin <
gryzykhin.mikhail@gmail.com> wrote:

> I wonder if we can add graph to community metrics showing ignored tests by
> language/project/overall. That can be useful to see focus area.
>
> On Tue, May 12, 2020 at 12:28 PM Jan Lukavský <je...@seznam.cz> wrote:
>
>> +1, visualizing the number of ignored tests in a graph seems useful. Even
>> better with some slices (e.g. per runner, module, ...).
>> On 5/12/20 8:02 PM, Ahmet Altay wrote:
>>
>> +1 to generate a report instead of removing these tests. A report like
>> this could help us with prioritization. It is easier to address issues when
>> we can quantify how much of a problem it is.
>>
>> I am curious what we can do to incentivize reducing the number of
>> flaky/ignored tests? A report itself might provide incentive, it is
>> rewarding to see ignored tests numbers go down over time.
>>
>> On Mon, May 11, 2020 at 8:30 AM Luke Cwik <lc...@google.com> wrote:
>>
>>> Deleting ignored tests does lead us to losing the reason as to why the
>>> test case was around so I would rather keep it around. I think it would be
>>> more valuable to generate a report that goes on the website/wiki showing
>>> stability of the modules (num tests, num passed, num skipped, num failed
>>> (running averages over the past N runs)). We had discussed doing something
>>> like this for ValidatesRunner so we could show which runner supports what
>>> automatically.
>>>
>>> On Mon, May 11, 2020 at 12:53 AM Jan Lukavský <je...@seznam.cz> wrote:
>>>
>>>> I think that we do have Jira issues for ignored test, there should be
>>>> no problem with that. The questionable point is that when test gets
>>>> Ignored, people might consider the problem as "less painful" and postpone
>>>> the correct solution until ... forever. I'd just like to discuss if people
>>>> see this as an issue. If yes, should we do something about that, or if no,
>>>> maybe we can create a rule that test marked as Ignored for long time might
>>>> be deleted, because apparently is only a dead code.
>>>> On 5/6/20 6:30 PM, Kenneth Knowles wrote:
>>>>
>>>> Good point.
>>>>
>>>> The raw numbers are available in the test run output. See
>>>> https://builds.apache.org/view/A-D/view/Beam/view/PostCommit/job/beam_PreCommit_Java_Cron/2718/testReport/ for
>>>> the "skipped" column.
>>>> And you get the same on console or Gradle Scan:
>>>> https://scans.gradle.com/s/ml3jv5xctkrmg/tests?collapse-all
>>>> This would be good to review periodically for obvious trouble spots.
>>>>
>>>> But I think you mean something more detailed. Some report with columns:
>>>> Test Suite, Test Method, Jira, Date Ignored, Most Recent Update
>>>>
>>>> I think we can get most of this from Jira, if we just make sure that
>>>> each ignored test has a Jira and they are all labeled in a consistent way.
>>>> That would be the quickest way to get some result, even though it is not
>>>> perfectly automated and audited.
>>>>
>>>> Kenn
>>>>
>>>> On Tue, May 5, 2020 at 2:41 PM Jan Lukavský <je...@seznam.cz> wrote:
>>>>
>>>>> Hi,
>>>>>
>>>>> it seems we are accumulating test cases (see discussion in [1]) that
>>>>> are
>>>>> marked as @Ignored (mostly due to flakiness), which is generally
>>>>> undesirable. Associated JIRAs seem to be open for a long time, and
>>>>> this
>>>>> might generally cause that we loose code coverage. Would anyone have
>>>>> idea on how to visualize these Ignored tests better? My first idea
>>>>> would
>>>>> be something similar to "Beam dependency check report", but that seems
>>>>> to be not the best example (which is completely different issue :)).
>>>>>
>>>>> Jan
>>>>>
>>>>> [1] https://github.com/apache/beam/pull/11614
>>>>>
>>>>>

Re: [DISCUSS] Dealing with @Ignored tests

Posted by Mikhail Gryzykhin <gr...@gmail.com>.
I wonder if we can add graph to community metrics showing ignored tests by
language/project/overall. That can be useful to see focus area.

On Tue, May 12, 2020 at 12:28 PM Jan Lukavský <je...@seznam.cz> wrote:

> +1, visualizing the number of ignored tests in a graph seems useful. Even
> better with some slices (e.g. per runner, module, ...).
> On 5/12/20 8:02 PM, Ahmet Altay wrote:
>
> +1 to generate a report instead of removing these tests. A report like
> this could help us with prioritization. It is easier to address issues when
> we can quantify how much of a problem it is.
>
> I am curious what we can do to incentivize reducing the number of
> flaky/ignored tests? A report itself might provide incentive, it is
> rewarding to see ignored tests numbers go down over time.
>
> On Mon, May 11, 2020 at 8:30 AM Luke Cwik <lc...@google.com> wrote:
>
>> Deleting ignored tests does lead us to losing the reason as to why the
>> test case was around so I would rather keep it around. I think it would be
>> more valuable to generate a report that goes on the website/wiki showing
>> stability of the modules (num tests, num passed, num skipped, num failed
>> (running averages over the past N runs)). We had discussed doing something
>> like this for ValidatesRunner so we could show which runner supports what
>> automatically.
>>
>> On Mon, May 11, 2020 at 12:53 AM Jan Lukavský <je...@seznam.cz> wrote:
>>
>>> I think that we do have Jira issues for ignored test, there should be no
>>> problem with that. The questionable point is that when test gets Ignored,
>>> people might consider the problem as "less painful" and postpone the
>>> correct solution until ... forever. I'd just like to discuss if people see
>>> this as an issue. If yes, should we do something about that, or if no,
>>> maybe we can create a rule that test marked as Ignored for long time might
>>> be deleted, because apparently is only a dead code.
>>> On 5/6/20 6:30 PM, Kenneth Knowles wrote:
>>>
>>> Good point.
>>>
>>> The raw numbers are available in the test run output. See
>>> https://builds.apache.org/view/A-D/view/Beam/view/PostCommit/job/beam_PreCommit_Java_Cron/2718/testReport/ for
>>> the "skipped" column.
>>> And you get the same on console or Gradle Scan:
>>> https://scans.gradle.com/s/ml3jv5xctkrmg/tests?collapse-all
>>> This would be good to review periodically for obvious trouble spots.
>>>
>>> But I think you mean something more detailed. Some report with columns:
>>> Test Suite, Test Method, Jira, Date Ignored, Most Recent Update
>>>
>>> I think we can get most of this from Jira, if we just make sure that
>>> each ignored test has a Jira and they are all labeled in a consistent way.
>>> That would be the quickest way to get some result, even though it is not
>>> perfectly automated and audited.
>>>
>>> Kenn
>>>
>>> On Tue, May 5, 2020 at 2:41 PM Jan Lukavský <je...@seznam.cz> wrote:
>>>
>>>> Hi,
>>>>
>>>> it seems we are accumulating test cases (see discussion in [1]) that
>>>> are
>>>> marked as @Ignored (mostly due to flakiness), which is generally
>>>> undesirable. Associated JIRAs seem to be open for a long time, and this
>>>> might generally cause that we loose code coverage. Would anyone have
>>>> idea on how to visualize these Ignored tests better? My first idea
>>>> would
>>>> be something similar to "Beam dependency check report", but that seems
>>>> to be not the best example (which is completely different issue :)).
>>>>
>>>> Jan
>>>>
>>>> [1] https://github.com/apache/beam/pull/11614
>>>>
>>>>

Re: [DISCUSS] Dealing with @Ignored tests

Posted by Jan Lukavský <je...@seznam.cz>.
+1, visualizing the number of ignored tests in a graph seems useful. 
Even better with some slices (e.g. per runner, module, ...).

On 5/12/20 8:02 PM, Ahmet Altay wrote:
> +1 to generate a report instead of removing these tests. A report like 
> this could help us with prioritization. It is easier to address issues 
> when we can quantify how much of a problem it is.
>
> I am curious what we can do to incentivize reducing the number of 
> flaky/ignored tests? A report itself might provide incentive, it is 
> rewarding to see ignored tests numbers go down over time.
>
> On Mon, May 11, 2020 at 8:30 AM Luke Cwik <lcwik@google.com 
> <ma...@google.com>> wrote:
>
>     Deleting ignored tests does lead us to losing the reason as to why
>     the test case was around so I would rather keep it around. I think
>     it would be more valuable to generate a report that goes on the
>     website/wiki showing stability of the modules (num tests, num
>     passed, num skipped, num failed (running averages over the past N
>     runs)). We had discussed doing something like this for
>     ValidatesRunner so we could show which runner supports what
>     automatically.
>
>     On Mon, May 11, 2020 at 12:53 AM Jan Lukavský <je.ik@seznam.cz
>     <ma...@seznam.cz>> wrote:
>
>         I think that we do have Jira issues for ignored test, there
>         should be no problem with that. The questionable point is that
>         when test gets Ignored, people might consider the problem as
>         "less painful" and postpone the correct solution until ...
>         forever. I'd just like to discuss if people see this as an
>         issue. If yes, should we do something about that, or if no,
>         maybe we can create a rule that test marked as Ignored for
>         long time might be deleted, because apparently is only a dead
>         code.
>
>         On 5/6/20 6:30 PM, Kenneth Knowles wrote:
>>         Good point.
>>
>>         The raw numbers are available in the test run output. See
>>         https://builds.apache.org/view/A-D/view/Beam/view/PostCommit/job/beam_PreCommit_Java_Cron/2718/testReport/ for
>>         the "skipped" column.
>>         And you get the same on console or Gradle Scan:
>>         https://scans.gradle.com/s/ml3jv5xctkrmg/tests?collapse-all
>>         This would be good to review periodically for obvious trouble
>>         spots.
>>
>>         But I think you mean something more detailed. Some report
>>         with columns: Test Suite, Test Method, Jira, Date Ignored,
>>         Most Recent Update
>>
>>         I think we can get most of this from Jira, if we just make
>>         sure that each ignored test has a Jira and they are all
>>         labeled in a consistent way. That would be the quickest way
>>         to get some result, even though it is not perfectly automated
>>         and audited.
>>
>>         Kenn
>>
>>         On Tue, May 5, 2020 at 2:41 PM Jan Lukavský <je.ik@seznam.cz
>>         <ma...@seznam.cz>> wrote:
>>
>>             Hi,
>>
>>             it seems we are accumulating test cases (see discussion
>>             in [1]) that are
>>             marked as @Ignored (mostly due to flakiness), which is
>>             generally
>>             undesirable. Associated JIRAs seem to be open for a long
>>             time, and this
>>             might generally cause that we loose code coverage. Would
>>             anyone have
>>             idea on how to visualize these Ignored tests better? My
>>             first idea would
>>             be something similar to "Beam dependency check report",
>>             but that seems
>>             to be not the best example (which is completely different
>>             issue :)).
>>
>>             Jan
>>
>>             [1] https://github.com/apache/beam/pull/11614
>>

Re: [DISCUSS] Dealing with @Ignored tests

Posted by Ahmet Altay <al...@google.com>.
+1 to generate a report instead of removing these tests. A report like this
could help us with prioritization. It is easier to address issues when we
can quantify how much of a problem it is.

I am curious what we can do to incentivize reducing the number of
flaky/ignored tests? A report itself might provide incentive, it is
rewarding to see ignored tests numbers go down over time.

On Mon, May 11, 2020 at 8:30 AM Luke Cwik <lc...@google.com> wrote:

> Deleting ignored tests does lead us to losing the reason as to why the
> test case was around so I would rather keep it around. I think it would be
> more valuable to generate a report that goes on the website/wiki showing
> stability of the modules (num tests, num passed, num skipped, num failed
> (running averages over the past N runs)). We had discussed doing something
> like this for ValidatesRunner so we could show which runner supports what
> automatically.
>
> On Mon, May 11, 2020 at 12:53 AM Jan Lukavský <je...@seznam.cz> wrote:
>
>> I think that we do have Jira issues for ignored test, there should be no
>> problem with that. The questionable point is that when test gets Ignored,
>> people might consider the problem as "less painful" and postpone the
>> correct solution until ... forever. I'd just like to discuss if people see
>> this as an issue. If yes, should we do something about that, or if no,
>> maybe we can create a rule that test marked as Ignored for long time might
>> be deleted, because apparently is only a dead code.
>> On 5/6/20 6:30 PM, Kenneth Knowles wrote:
>>
>> Good point.
>>
>> The raw numbers are available in the test run output. See
>> https://builds.apache.org/view/A-D/view/Beam/view/PostCommit/job/beam_PreCommit_Java_Cron/2718/testReport/ for
>> the "skipped" column.
>> And you get the same on console or Gradle Scan:
>> https://scans.gradle.com/s/ml3jv5xctkrmg/tests?collapse-all
>> This would be good to review periodically for obvious trouble spots.
>>
>> But I think you mean something more detailed. Some report with columns:
>> Test Suite, Test Method, Jira, Date Ignored, Most Recent Update
>>
>> I think we can get most of this from Jira, if we just make sure that each
>> ignored test has a Jira and they are all labeled in a consistent way. That
>> would be the quickest way to get some result, even though it is not
>> perfectly automated and audited.
>>
>> Kenn
>>
>> On Tue, May 5, 2020 at 2:41 PM Jan Lukavský <je...@seznam.cz> wrote:
>>
>>> Hi,
>>>
>>> it seems we are accumulating test cases (see discussion in [1]) that are
>>> marked as @Ignored (mostly due to flakiness), which is generally
>>> undesirable. Associated JIRAs seem to be open for a long time, and this
>>> might generally cause that we loose code coverage. Would anyone have
>>> idea on how to visualize these Ignored tests better? My first idea would
>>> be something similar to "Beam dependency check report", but that seems
>>> to be not the best example (which is completely different issue :)).
>>>
>>> Jan
>>>
>>> [1] https://github.com/apache/beam/pull/11614
>>>
>>>

Re: [DISCUSS] Dealing with @Ignored tests

Posted by Luke Cwik <lc...@google.com>.
Deleting ignored tests does lead us to losing the reason as to why the test
case was around so I would rather keep it around. I think it would be more
valuable to generate a report that goes on the website/wiki showing
stability of the modules (num tests, num passed, num skipped, num failed
(running averages over the past N runs)). We had discussed doing something
like this for ValidatesRunner so we could show which runner supports what
automatically.

On Mon, May 11, 2020 at 12:53 AM Jan Lukavský <je...@seznam.cz> wrote:

> I think that we do have Jira issues for ignored test, there should be no
> problem with that. The questionable point is that when test gets Ignored,
> people might consider the problem as "less painful" and postpone the
> correct solution until ... forever. I'd just like to discuss if people see
> this as an issue. If yes, should we do something about that, or if no,
> maybe we can create a rule that test marked as Ignored for long time might
> be deleted, because apparently is only a dead code.
> On 5/6/20 6:30 PM, Kenneth Knowles wrote:
>
> Good point.
>
> The raw numbers are available in the test run output. See
> https://builds.apache.org/view/A-D/view/Beam/view/PostCommit/job/beam_PreCommit_Java_Cron/2718/testReport/ for
> the "skipped" column.
> And you get the same on console or Gradle Scan:
> https://scans.gradle.com/s/ml3jv5xctkrmg/tests?collapse-all
> This would be good to review periodically for obvious trouble spots.
>
> But I think you mean something more detailed. Some report with columns:
> Test Suite, Test Method, Jira, Date Ignored, Most Recent Update
>
> I think we can get most of this from Jira, if we just make sure that each
> ignored test has a Jira and they are all labeled in a consistent way. That
> would be the quickest way to get some result, even though it is not
> perfectly automated and audited.
>
> Kenn
>
> On Tue, May 5, 2020 at 2:41 PM Jan Lukavský <je...@seznam.cz> wrote:
>
>> Hi,
>>
>> it seems we are accumulating test cases (see discussion in [1]) that are
>> marked as @Ignored (mostly due to flakiness), which is generally
>> undesirable. Associated JIRAs seem to be open for a long time, and this
>> might generally cause that we loose code coverage. Would anyone have
>> idea on how to visualize these Ignored tests better? My first idea would
>> be something similar to "Beam dependency check report", but that seems
>> to be not the best example (which is completely different issue :)).
>>
>> Jan
>>
>> [1] https://github.com/apache/beam/pull/11614
>>
>>

Re: [DISCUSS] Dealing with @Ignored tests

Posted by Jan Lukavský <je...@seznam.cz>.
I think that we do have Jira issues for ignored test, there should be no 
problem with that. The questionable point is that when test gets 
Ignored, people might consider the problem as "less painful" and 
postpone the correct solution until ... forever. I'd just like to 
discuss if people see this as an issue. If yes, should we do something 
about that, or if no, maybe we can create a rule that test marked as 
Ignored for long time might be deleted, because apparently is only a 
dead code.

On 5/6/20 6:30 PM, Kenneth Knowles wrote:
> Good point.
>
> The raw numbers are available in the test run output. See 
> https://builds.apache.org/view/A-D/view/Beam/view/PostCommit/job/beam_PreCommit_Java_Cron/2718/testReport/ for 
> the "skipped" column.
> And you get the same on console or Gradle Scan: 
> https://scans.gradle.com/s/ml3jv5xctkrmg/tests?collapse-all
> This would be good to review periodically for obvious trouble spots.
>
> But I think you mean something more detailed. Some report with 
> columns: Test Suite, Test Method, Jira, Date Ignored, Most Recent Update
>
> I think we can get most of this from Jira, if we just make sure that 
> each ignored test has a Jira and they are all labeled in a consistent 
> way. That would be the quickest way to get some result, even though it 
> is not perfectly automated and audited.
>
> Kenn
>
> On Tue, May 5, 2020 at 2:41 PM Jan Lukavský <je.ik@seznam.cz 
> <ma...@seznam.cz>> wrote:
>
>     Hi,
>
>     it seems we are accumulating test cases (see discussion in [1])
>     that are
>     marked as @Ignored (mostly due to flakiness), which is generally
>     undesirable. Associated JIRAs seem to be open for a long time, and
>     this
>     might generally cause that we loose code coverage. Would anyone have
>     idea on how to visualize these Ignored tests better? My first idea
>     would
>     be something similar to "Beam dependency check report", but that
>     seems
>     to be not the best example (which is completely different issue :)).
>
>     Jan
>
>     [1] https://github.com/apache/beam/pull/11614
>

Re: [DISCUSS] Dealing with @Ignored tests

Posted by Kenneth Knowles <ke...@apache.org>.
Good point.

The raw numbers are available in the test run output. See
https://builds.apache.org/view/A-D/view/Beam/view/PostCommit/job/beam_PreCommit_Java_Cron/2718/testReport/
for
the "skipped" column.
And you get the same on console or Gradle Scan:
https://scans.gradle.com/s/ml3jv5xctkrmg/tests?collapse-all
This would be good to review periodically for obvious trouble spots.

But I think you mean something more detailed. Some report with columns:
Test Suite, Test Method, Jira, Date Ignored, Most Recent Update

I think we can get most of this from Jira, if we just make sure that each
ignored test has a Jira and they are all labeled in a consistent way. That
would be the quickest way to get some result, even though it is not
perfectly automated and audited.

Kenn

On Tue, May 5, 2020 at 2:41 PM Jan Lukavský <je...@seznam.cz> wrote:

> Hi,
>
> it seems we are accumulating test cases (see discussion in [1]) that are
> marked as @Ignored (mostly due to flakiness), which is generally
> undesirable. Associated JIRAs seem to be open for a long time, and this
> might generally cause that we loose code coverage. Would anyone have
> idea on how to visualize these Ignored tests better? My first idea would
> be something similar to "Beam dependency check report", but that seems
> to be not the best example (which is completely different issue :)).
>
> Jan
>
> [1] https://github.com/apache/beam/pull/11614
>
>