Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[Failing Test]: Go test TestDebeziumIO_BasicRead is consistency failing in beam_PostCommit_XVR_Direct test suite #25223

Closed
1 of 15 tasks
chamikaramj opened this issue Jan 30, 2023 · 15 comments · Fixed by #25243
Closed
1 of 15 tasks
Assignees
Labels
bug done & done Issue has been reviewed after it was closed for verification, followups, etc. failing test go P1 permared tests

Comments

@chamikaramj
Copy link
Contributor

What happened?

https://ci-beam.apache.org/job/beam_PostCommit_XVR_Direct/

https://scans.gradle.com/s/lzqa6tpzxsbsc

https://scans.gradle.com/s/2qjaizetm44mk/console-log?page=13&task=:sdks:python:test-suites:direct:xlang:validatesCrossLanguageRunnerGoUsingJava#L12117

--- FAIL: TestDebeziumIO_BasicRead (4.18s)
panic: tried cross-language for beam:transform:org.apache.beam:debezium_read:v1 against localhost:41189 and failed
expanding external transform
expanding transform with ExpansionRequest: components:{} transform:{unique_name:"External" spec:{urn:"beam:transform:org.apache.beam:debezium_read:v1" payload:"\n\xb0\x01\n\x14\n\x0eConnectorClass\x1a\x02\x10\x07\n\x0e\n\x08Username\x1a\x02\x10\x07\n\x0e\n\x08Password\x1a\x02\x10\x07\n\n\n\x04Host\x1a\x02\x10\x07\n\n\n\x04Port\x1a\x02\x10\x07\n\x1a\n\x12MaxNumberOfRecords\x1a\x04\x08\x01\x10\x04\n\x1e\n\x14ConnectionProperties\x1a\x06\x1a\x04\n\x02\x10\x07\x12$3d418d51-2df7-4eda-bff9-7a21d5bf0570\x12\xa5\x01\x07\x00\nPostgreSQL\x08debezium\x03dbz\tlocalhost\x0536803\x01\x00\x00\x00\x04\x19database.dbname=inventory\x1edatabase.server.name=dbserver1\x1fdatabase.include.list=inventory\x1cinclude.schema.changes=false"} environment_id:"go"} namespace:"GmQdmXvlgv"
expansion failed
caused by:
rpc error: code = Unavailable desc = connection error: desc = "transport: Error while dialing dial tcp 127.0.0.1:41189: connect: connection refused" [recovered]
panic: tried cross-language for beam:transform:org.apache.beam:debezium_read:v1 against localhost:41189 and failed
expanding external transform
expanding transform with ExpansionRequest: components:{} transform:{unique_name:"External" spec:{urn:"beam:transform:org.apache.beam:debezium_read:v1" payload:"\n\xb0\x01\n\x14\n\x0eConnectorClass\x1a\x02\x10\x07\n\x0e\n\x08Username\x1a\x02\x10\x07\n\x0e\n\x08Password\x1a\x02\x10\x07\n\n\n\x04Host\x1a\x02\x10\x07\n\n\n\x04Port\x1a\x02\x10\x07\n\x1a\n\x12MaxNumberOfRecords\x1a\x04\x08\x01\x10\x04\n\x1e\n\x14ConnectionProperties\x1a\x06\x1a\x04\n\x02\x10\x07\x12$3d418d51-2df7-4eda-bff9-7a21d5bf0570\x12\xa5\x01\x07\x00\nPostgreSQL\x08debezium\x03dbz\tlocalhost\x0536803\x01\x00\x00\x00\x04\x19database.dbname=inventory\x1edatabase.server.name=dbserver1\x1fdatabase.include.list=inventory\x1cinclude.schema.changes=false"} environment_id:"go"} namespace:"GmQdmXvlgv"
expansion failed
caused by:
rpc error: code = Unavailable desc = connection error: desc = "transport: Error while dialing dial tcp 127.0.0.1:41189: connect: connection refused"

goroutine 27 [running]:
testing.tRunner.func1.2({0xf0a5c0, 0xc000111a00})
/home/jenkins/sdk/go1.19.3/src/testing/testing.go:1396 +0x24e
testing.tRunner.func1()
/home/jenkins/sdk/go1.19.3/src/testing/testing.go:1399 +0x39f
panic({0xf0a5c0, 0xc000111a00})
/home/jenkins/sdk/go1.19.3/src/runtime/panic.go:884 +0x212
github.com/apache/beam/sdks/v2/go/pkg/beam.CrossLanguage({0xc000660c40?, 0xc000114730?}, {0x10724c9, 0x2f}, {0xc00013a6e0?, 0x0?, 0x2000000000000?}, {0xc0000430d0, 0xf}, 0x0, ...)
/home/jenkins/jenkins-slave/workspace/beam_PostCommit_XVR_Direct/src/sdks/go/pkg/beam/xlang.go:165 +0x136
github.com/apache/beam/sdks/v2/go/pkg/beam/io/xlang/debeziumio.Read({0xc000660c40?, 0xc000114730?}, {0x10435e1, 0x8}, {0x103dfe5, 0x3}, {0x10449cc, 0x9}, {0xc0003ad870, 0x5}, ...)
/home/jenkins/jenkins-slave/workspace/beam_PostCommit_XVR_Direct/src/sdks/go/pkg/beam/io/xlang/debeziumio/debezium.go:125 +0x3be
github.com/apache/beam/sdks/v2/go/test/integration/io/xlang/debezium.ReadPipeline({0xc0000430d0, 0xf}, {0x10435e1, 0x8}, {0x103dfe5, 0x3}, {0xc000491f60?, 0x5290e5?}, {0x10449cc, 0x9}, ...)
/home/jenkins/jenkins-slave/workspace/beam_PostCommit_XVR_Direct/src/sdks/go/test/integration/io/xlang/debezium/debezium.go:30 +0x3dd
github.com/apache/beam/sdks/v2/go/test/integration/io/xlang/debezium.TestDebeziumIO_BasicRead(0xc00048fa00)
/home/jenkins/jenkins-slave/workspace/beam_PostCommit_XVR_Direct/src/sdks/go/test/integration/io/xlang/debezium/debezium_test.go:88 +0x1cc
testing.tRunner(0xc00048fa00, 0x10ab1c0)
/home/jenkins/sdk/go1.19.3/src/testing/testing.go:1446 +0x10b
created by testing.(*T).Run
/home/jenkins/sdk/go1.19.3/src/testing/testing.go:1493 +0x35f
FAIL github.com/apache/beam/sdks/v2/go/test/integration/io/xlang/debezium 7.221s

Issue Failure

Failure: Test is continually failing

Issue Priority

Priority: 1 (unhealthy code / failing or flaky postcommit so we cannot be sure the product is healthy)

Issue Components

  • Component: Python SDK
  • Component: Java SDK
  • Component: Go SDK
  • Component: Typescript SDK
  • Component: IO connector
  • Component: Beam examples
  • Component: Beam playground
  • Component: Beam katas
  • Component: Website
  • Component: Spark Runner
  • Component: Flink Runner
  • Component: Samza Runner
  • Component: Twister2 Runner
  • Component: Hazelcast Jet Runner
  • Component: Google Cloud Dataflow Runner
@chamikaramj
Copy link
Contributor Author

@lostluck any idea ?

@lostluck
Copy link
Contributor

First, title is bad: Doesn't specify which Direct Runner. Go pipelines can't run on the Java Direct runner. Vacuously, nothing but python can run on the Python Direct runner, since that's kinda what distinguishes direct runners from portable runners.

Cross language doesn't execute on the Go direct runner, so I assume that's not the runner in question either.

Go pipelines can run on the Python Portable runner however, so I'll assume that's the case.

In this case, it looks like however the Java Expansion Service is being spun up, it's not yetpermitting external connections.

"transport: Error while dialing dial tcp 127.0.0.1:41189: connect: connection refused"

That likely means it's not yet ready.

Looking back in history, it looks like this does pass sometimes, (3666 is an example), so why?


Given that the tests fail in ~5-6 seconds when it does fail, it feels like the connection/request stage isn't retrying/backing off since the container may not yet be ready.

So that means this line (or the one below):

err = errors.Wrapf(err, "expansion failed")
is what's failing, and we don't do a retry/backoff. Reading error, it's probably the connection isn't ready/waiting yet, which is expected/default behavior. The connection happens asynchronously before the first request, which is why we don't see it fail here:
conn, err := grpc.Dial(p.Config, grpc.WithInsecure())

So the fix is a simple re-try + backoff to some timeout in that area, which aligns with the Bruno Conjecture that we don't have resilient enough tests/infra/frameworks implemented.

@lostluck
Copy link
Contributor

I will note that it's not always the Go Debezium target. Sometimes it's one of the Python cross language targets, probably for a similar cause.

https://ci-beam.apache.org/job/beam_PostCommit_XVR_Direct/3665 ->

org.apache.beam.sdk.extensions.python.PythonExternalTransformTest > trivialPythonTransform FAILED
    java.lang.RuntimeException at PythonExternalTransformTest.java:58
        Caused by: java.util.concurrent.TimeoutException at PythonExternalTransformTest.java:58

@chamikaramj
Copy link
Contributor Author

Yes, Python portable DirectRunner.

For Python failures, we submitted #25176. Recent failures of the test suite seems to be due to the Go issue.

Also, probably we should split this test suite SDK-wise to give a better signal.

@chamikaramj chamikaramj changed the title [Failing Test]: Go test TestDebeziumIO_BasicRead consistency failing for DirectRunner [Failing Test]: Go test TestDebeziumIO_BasicRead is consistency failing in beam_PostCommit_XVR_Direct test suite Jan 31, 2023
@lostluck
Copy link
Contributor

I maintain that a runner can't both be "Direct" and "Portable" :P. One either has two different runners, or it's unnecessarily complicated.

If it's going through the FnAPI/Portability framework, it's not direct execution and vice versa. I'm not sure what the purpose would be of a "hybrid" runner that executes some transforms "direct" and others via portability. It's a state that's not equivalent to any production set up, invalidating the benefits it might be providing as a test runner.

@chamikaramj
Copy link
Contributor Author

We might have to evolve that term given that all runners are expected to be FnAPI/Portability-based eventually :)

I take "Direct" as directly available via Beam, without using an external distributed system. But others may have a different definitions.

@lostluck
Copy link
Contributor

Given Python automatically starts flink and spark...

@Abacn
Copy link
Contributor

Abacn commented Jan 31, 2023

related to #25189 fails to communicate with same debezium expansion service

@Abacn
Copy link
Contributor

Abacn commented Jan 31, 2023

other XVR test (e.g. samza) also affected.

@chamikaramj
Copy link
Contributor Author

cc: @lukecwik

https://github.com/apache/beam/pull/25104/files seems to be a change to SDK harness channels though so I'm not sure how that will cause a failure in the expansion time. But I might be missing something.

@lostluck
Copy link
Contributor

Agreed.

@Abacn
Copy link
Contributor

Abacn commented Jan 31, 2023

Try to run the expansion service locally and see the following error:

java -jar /Users/yathu/beam/sdks/java/io/debezium/expansion-service/build/libs/beam-sdks-java-io-debezium-expansion-service-2.46.0-SNAPSHOT.jar 33291 --filesToStage=/Users/yathu/beam/sdks/java/io/debezium/expansion-service/build/libs/beam-sdks-java-io-debezium-expansion-service-2.46.0-SNAPSHOT.jar
Starting expansion service at localhost:33291
Jan 31, 2023 2:14:51 PM org.apache.beam.sdk.expansion.service.ExpansionService loadRegisteredTransforms
INFO: Registering external transforms: [beam:transform:org.apache.beam:debezium_read:v1, beam:external:java:generate_sequence:v1]
Exception in thread "main" java.util.ServiceConfigurationError: org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider: Provider org.apache.beam.io.debezium.DebeziumReadSchemaTransformProvider could not be instantiated
	at java.util.ServiceLoader.fail(ServiceLoader.java:232)
	at java.util.ServiceLoader.access$100(ServiceLoader.java:185)
	at java.util.ServiceLoader$LazyIterator.nextService(ServiceLoader.java:384)
	at java.util.ServiceLoader$LazyIterator.next(ServiceLoader.java:404)
	at java.util.ServiceLoader$1.next(ServiceLoader.java:480)
	at org.apache.beam.sdk.expansion.service.ExpansionServiceSchemaTransformProvider.<init>(ExpansionServiceSchemaTransformProvider.java:53)
	at org.apache.beam.sdk.expansion.service.ExpansionServiceSchemaTransformProvider.of(ExpansionServiceSchemaTransformProvider.java:69)
	at org.apache.beam.sdk.expansion.service.ExpansionService.getRegisteredSchemaTransforms(ExpansionService.java:444)
	at org.apache.beam.sdk.expansion.service.ExpansionService.main(ExpansionService.java:684)
Caused by: java.lang.IllegalAccessException: Class java.util.ServiceLoader$LazyIterator can not access a member of class org.apache.beam.io.debezium.DebeziumReadSchemaTransformProvider with modifiers ""
	at sun.reflect.Reflection.ensureMemberAccess(Reflection.java:102)
	at java.lang.Class.newInstance(Class.java:436)
	at java.util.ServiceLoader$LazyIterator.nextService(ServiceLoader.java:380)
	... 6 more

did a bisect, found the culprit is actually #25107 (before it the command succeeded), CC: @pabloem sorry CC: @lukecwik it wasn't #25104

@Abacn
Copy link
Contributor

Abacn commented Jan 31, 2023

Looks like the added schematransform auto service is somehow conflict with expansion service? Or they cannot live in same shadowJar?

Note that google-cloud-platform expansion service is experiencing same problem

@chamikaramj
Copy link
Contributor Author

Can we try a revert ?

@Abacn
Copy link
Contributor

Abacn commented Jan 31, 2023

Can we try a revert ?

I got a simple forward fix. It is just because @AutoService annotated class does not have public constructor. WIll open a PR

@github-actions github-actions bot added this to the 2.46.0 Release milestone Jan 31, 2023
@lostluck lostluck added the done & done Issue has been reviewed after it was closed for verification, followups, etc. label Feb 7, 2023
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
bug done & done Issue has been reviewed after it was closed for verification, followups, etc. failing test go P1 permared tests
Projects
None yet
Development

Successfully merging a pull request may close this issue.

3 participants