You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2022/08/29 21:14:39 UTC

[GitHub] [beam] Abacn opened a new issue, #22944: [Bug]: Kafka Performance Tests actually failing but not marked as failed on Jenkins

Abacn opened a new issue, #22944:
URL: https://github.com/apache/beam/issues/22944

   ### What happened?
   
   Grafana dashboard shows zero read time of kafkaIO | 1 GB:
   <img width="807" alt="image" src="https://user-images.githubusercontent.com/8010435/187299139-03c26834-e6c0-427d-9a84-f97503559094.png">
   Taking a closer look to the log, the Dataflow job actually failed:
   ```
   Aug 29, 2022 1:06:01 PM org.apache.beam.runners.dataflow.util.MonitoringUtil$LoggingHandler process
   09:06:01     SEVERE: 2022-08-29T13:06:00.196Z: Workflow failed. Causes: S01:Generate records+Measure write time+Write to Kafka/Kafka ProducerRecord/Map+Write to Kafka/KafkaIO.WriteRecords/ParDo(KafkaWriter) failed., The job failed because a work item has failed 4 times. Look in previous log entries for the cause of each one of the 4 failures. For more information, see https://cloud.google.com/dataflow/docs/guides/common-errors. The work item was attempted on these ****s: 
   09:06:01 
   09:06:01       kafkaioit0testkafkaioread-08290559-7f48-harness-wvf0
   09:06:01           Root cause: Work item failed.,
   09:06:01 
   09:06:01       kafkaioit0testkafkaioread-08290559-7f48-harness-4gm7
   09:06:01           Root cause: Work item failed.,
   09:06:01 
   09:06:01       kafkaioit0testkafkaioread-08290559-7f48-harness-m1h9
   09:06:01           Root cause: Work item failed.,
   09:06:01 
   09:06:01       kafkaioit0testkafkaioread-08290559-7f48-harness-t0qh
   09:06:01           Root cause: Work item failed.
   09:06:01     Aug 29, 2022 1:06:01 PM org.apache.beam.runners.dataflow.util.MonitoringUtil$LoggingHandler process
   09:06:01     INFO: 2022-08-29T13:06:00.262Z: Cleaning up.
   09:06:01     Aug 29, 2022 1:06:01 PM org.apache.beam.runners.dataflow.util.MonitoringUtil$LoggingHandler process
   09:06:01     INFO: 2022-08-29T13:06:00.350Z: Stopping **** pool...
   09:08:16     Aug 29, 2022 1:08:16 PM org.apache.beam.runners.dataflow.util.MonitoringUtil$LoggingHandler process
   09:08:16     INFO: 2022-08-29T13:08:15.939Z: Autoscaling: Resized **** pool from 5 to 0.
   09:08:16     Aug 29, 2022 1:08:16 PM org.apache.beam.runners.dataflow.util.MonitoringUtil$LoggingHandler process
   09:08:16     INFO: 2022-08-29T13:08:15.989Z: Worker pool stopped.
   09:08:29     Aug 29, 2022 1:08:29 PM org.apache.beam.runners.dataflow.DataflowPipelineJob logTerminalState
   09:08:29     INFO: Job 2022-08-29_05_59_45-207371049498193911 failed with status FAILED.
   
   ...
   
   Kafka/KafkaIO.Read.ReadFromKafkaViaUnbounded/Read(KafkaUnboundedSource)/Reshuffle/Reshuffle/GroupByKey/Write failed., The job failed because a work item has failed 4 times. Look in previous log entries for the cause of each one of the 4 failures. For more information, see https://cloud.google.com/dataflow/docs/guides/common-errors. The work item was attempted on these ****s: 
   09:10:35 
   09:10:35       kafkaioit0testkafkaioread-08290608-wu0a-harness-1nt9
   09:10:35           Root cause: Work item failed.,
   09:10:35 
   09:10:35       kafkaioit0testkafkaioread-08290608-wu0a-harness-1nt9
   09:10:35           Root cause: Work item failed.,
   09:10:35 
   09:10:35       kafkaioit0testkafkaioread-08290608-wu0a-harness-mmtp
   09:10:35           Root cause: Work item failed.,
   09:10:35 
   09:10:35       kafkaioit0testkafkaioread-08290608-wu0a-harness-mmtp
   09:10:35           Root cause: Work item failed.
   09:10:35     Aug 29, 2022 1:10:35 PM org.apache.beam.runners.dataflow.util.MonitoringUtil$LoggingHandler process
   09:10:35     INFO: 2022-08-29T13:10:34.131Z: Cleaning up.
   09:10:35     Aug 29, 2022 1:10:35 PM org.apache.beam.runners.dataflow.util.MonitoringUtil$LoggingHandler process
   09:10:35     INFO: 2022-08-29T13:10:34.248Z: Stopping **** pool...
   09:12:43     Aug 29, 2022 1:12:43 PM org.apache.beam.runners.dataflow.util.MonitoringUtil$LoggingHandler process
   09:12:43     INFO: 2022-08-29T13:12:41.822Z: Autoscaling: Resized **** pool from 5 to 0.
   09:12:43     Aug 29, 2022 1:12:43 PM org.apache.beam.runners.dataflow.util.MonitoringUtil$LoggingHandler process
   09:12:43     INFO: 2022-08-29T13:12:41.930Z: Worker pool stopped.
   09:13:19     Aug 29, 2022 1:13:19 PM org.apache.beam.runners.dataflow.DataflowPipelineJob logTerminalState
   09:13:19     INFO: Job 2022-08-29_06_08_39-15433867759668129710 failed with status FAILED.
   09:13:19 
   09:13:19 org.apache.beam.sdk.io.kafka.KafkaIOIT > testKafkaIOReadsAndWritesCorrectlyInBatch STANDARD_OUT
   09:13:19     Load test results for test (ID): 1cfab208-e4d1-4262-9a47-a8cfa54e8d8a and timestamp: 2022-08-29T12:59:31.874000000Z:
   09:13:19                      Metric:                    Value:
   09:13:19                    read_time                       0.0
   09:13:19                   write_time                    13.829
   09:13:19                     run_time                    13.829
   09:13:20 
   09:13:20 Gradle Test Executor 1 finished executing tests.
   ```
   
   It can be seen that write has failed and read never happens, which explains the zero read_time. However, the jenkins job still marked as Pass.
   
   Taking a look at the source code, `writeResult.waitUntilFinish();` does not follow an assert of job succeeded. Gradle then does not detect job failing.
   
   Other performance test jobs may have the same issue.
   
   ### Issue Priority
   
   Priority: 2
   
   ### Issue Component
   
   Component: test-failures


-- 
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: github-unsubscribe@beam.apache.org.apache.org

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


[GitHub] [beam] Abacn commented on issue #22944: [Bug]: Kafka Performance Tests actually failing but not marked as failed on Jenkins

Posted by GitBox <gi...@apache.org>.
Abacn commented on issue #22944:
URL: https://github.com/apache/beam/issues/22944#issuecomment-1242743938

   False positive eliminated in #23027, kafka performance test now fails. Leave this issue open for the test fix.


-- 
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: github-unsubscribe@beam.apache.org

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


[GitHub] [beam] Abacn commented on issue #22944: [Bug]: Kafka Performance Tests actually failing but not marked as failed on Jenkins

Posted by GitBox <gi...@apache.org>.
Abacn commented on issue #22944:
URL: https://github.com/apache/beam/issues/22944#issuecomment-1248821539

   Current status: Pipeline now runs, though get assertion error: "Expected: "0f12c27c9a7672e14775594be66cad9a". but: was "966684bbc4d071c54466678005c86eec"


-- 
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: github-unsubscribe@beam.apache.org

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


[GitHub] [beam] johnjcasey closed issue #22944: [Bug]: Kafka Performance Tests actually failing but not marked as failed on Jenkins

Posted by GitBox <gi...@apache.org>.
johnjcasey closed issue #22944: [Bug]: Kafka Performance Tests actually failing but not marked as failed on Jenkins
URL: https://github.com/apache/beam/issues/22944


-- 
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: github-unsubscribe@beam.apache.org

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


[GitHub] [beam] Abacn commented on issue #22944: [Bug]: Kafka Performance Tests actually failing but not marked as failed on Jenkins

Posted by GitBox <gi...@apache.org>.
Abacn commented on issue #22944:
URL: https://github.com/apache/beam/issues/22944#issuecomment-1230880373

   For example: 
   https://github.com/apache/beam/blob/4ae54b2e1e28096f2b173d5f5574910e8cfd80c9/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java#L241
   https://github.com/apache/beam/blob/4ae54b2e1e28096f2b173d5f5574910e8cfd80c9/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOIT.java#L140
   https://github.com/apache/beam/blob/4ae54b2e1e28096f2b173d5f5574910e8cfd80c9/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/text/TextIOIT.java#L153
   
   These tests are used as performance tests but the job status is not checked. Jenkins task could still success even if dataflow job fails.


-- 
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: github-unsubscribe@beam.apache.org

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


[GitHub] [beam] Abacn commented on issue #22944: [Bug]: Kafka Performance Tests actually failing but not marked as failed on Jenkins

Posted by GitBox <gi...@apache.org>.
Abacn commented on issue #22944:
URL: https://github.com/apache/beam/issues/22944#issuecomment-1231977571

   .take-issue


-- 
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: github-unsubscribe@beam.apache.org

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