You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@beam.apache.org by "Emily Ye (Jira)" <ji...@apache.org> on 2021/07/31 01:59:00 UTC

[jira] [Commented] (BEAM-12699) Several streaming tests failing - PostCommit_Java_VR_Dataflow_V2_Streaming, Python PreCommit,

    [ https://issues.apache.org/jira/browse/BEAM-12699?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17390839#comment-17390839 ] 

Emily Ye commented on BEAM-12699:
---------------------------------

Pipeline against 205fbb10998c9e2a1c7842ab7efd88aef80828a6, the commit of the last passing run for the Java streaming PostCommit ([https://ci-beam.apache.org/job/beam_PostCommit_Java_VR_Dataflow_V2_Streaming/1036/)] has the same issue:

```
org.apache.beam.sdk.transforms.ParDoSchemaTest > testReadAndWrite FAILED
 java.lang.RuntimeException: Dataflow job 2021-07-30_17_26_44-14800110294544746882 terminated in state RUNNING but did not return a failure reason.
 at org.apache.beam.runners.dataflow.TestDataflowRunner.run(TestDataflowRunner.java:141)
 at org.apache.beam.runners.dataflow.TestDataflowRunner.run(TestDataflowRunner.java:93)
 at org.apache.beam.runners.dataflow.TestDataflowRunner.run(TestDataflowRunner.java:55)
 at org.apache.beam.sdk.Pipeline.run(Pipeline.java:323)
 at org.apache.beam.sdk.testing.TestPipeline.run(TestPipeline.java:398)
 at org.apache.beam.sdk.testing.TestPipeline.run(TestPipeline.java:334)
 at org.apache.beam.sdk.transforms.ParDoSchemaTest.testReadAndWrite(ParDoSchemaTest.java:168)
```
 https://console.cloud.google.com/dataflow/jobs/us-central1/2021-07-30_17_26_44-14800110294544746882?project=apache-beam-testing

Same with a test run against branch release-2.31.0: https://console.cloud.google.com/dataflow/jobs/us-central1/2021-07-30_18_37_43-7837911144308264390?project=apache-beam-testing

Test run:
 ./gradlew :runners:google-cloud-dataflow-java:validatesRunnerV2TestStreaming  --info

> Several streaming tests failing - PostCommit_Java_VR_Dataflow_V2_Streaming, Python PreCommit, 
> ----------------------------------------------------------------------------------------------
>
>                 Key: BEAM-12699
>                 URL: https://issues.apache.org/jira/browse/BEAM-12699
>             Project: Beam
>          Issue Type: Bug
>          Components: test-failures
>            Reporter: Emily Ye
>            Assignee: Slava Chernyak
>            Priority: P1
>              Labels: currently-failing
>
> Several streaming jobs are failing without having done any work.
> Seeing in INFO-level logs under `dataflow.googleapis.com/harness` logs: 
>  
> {code:java}
> Streaming engine endpoint to ipv4:209.85.200.95:443 closed unexpectedly with error code, NOT_FOUND, and will be retried if necessary. This may occur due to autoscaling events. Full status: NOT_FOUND: Requested entity was not found. === Source Location Trace: === ./third_party/grpc/google_specific/include/grpcpp/impl/codegen/status.h:97
> {code}
>  
> Java tests are ending with the following error messages: 
> {code:java}
> java.lang.RuntimeException: Dataflow job ... terminated in state RUNNING but did not return a failure reason.{code}
>  
> Python jobs have errors like:
> {code:java}
> ERROR apache_beam.io.gcp.tests.pubsub_matcher:pubsub_matcher.py:162 Timeout after 400 sec. Received 0 messages from projects/apache-beam-testing/subscriptions/wc_subscription_output39860e7e-f237-41fb-9f6b-4c2bd4783023.{code}
> This appears to be affecting many jobs:
> [https://ci-beam.apache.org/job/beam_PostCommit_Java_VR_Dataflow_V2_Streaming/1044/]
>  * Appears to be timing out/crashing, but ran a test locally to confirm: 
>  * Job: [https://console.cloud.google.com/dataflow/jobs/us-central1/2021-07-30_15_29_50-2692669480703228454?project=apache-beam-testing]
> Java PreCommit streaming test org.apache.beam.examples.WordCountIT.testE2EWordCount:
>  * [https://ci-beam.apache.org/job/beam_PreCommit_Java_Examples_Dataflow_Cron/3988/]
>  * [https://gradle.com/s/pha5hds4halho]
>  * Job: [https://console.cloud.google.com/dataflow/jobs/us-central1/2021-07-30_15_29_50-2692669480703228454?project=apache-beam-testing]
> Python PreCommits (apache_beam.examples.streaming_wordcount_it_test.StreamingWordCountIT.test_streaming_wordcount_it) 
>  * [https://ci-beam.apache.org/job/beam_PreCommit_Python_Cron/4485/] (first cron failure)
>  * [https://ci-beam.apache.org/job/beam_PreCommit_Python_Phrase/2680/]
>  * Scans: [https://scans.gradle.com/s/l4s4az3rolina, |https://scans.gradle.com/s/l4s4az3rolina] [https://gradle.com/s/zyzqaxlivyc7m]
>  * Job: [https://console.cloud.google.com/|https://console.cloud.google.com/dataflow/jobs/us-central1/2021-07-30_15_29_50-2692669480703228454?project=apache-beam-testing]dataflow/jobs/us-central1/2021-07-30_11_39_34-1628210491567694938?project=apache-beam-testing
> I highly suspect [https://ci-beam.apache.org/job/beam_PostCommit_Py_VR_Dataflow_V2/2292] is also failing because of this (timing matches and only :validatesRunnerStreamingTests is failing, not :validatesRunnerBatchTests) but the daemon is crashing and I have yet to confirm locally.
>  



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