You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by "Abacn (via GitHub)" <gi...@apache.org> on 2024/04/16 15:18:56 UTC

[I] [Bug]: Uber jar not work with Dataflow runner v2 due to `UnknownCoderWrapper` for Beam 2.55 [beam]

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

   ### What happened?
   
   Minimum reproduce:
   
   A minimum pipeline:
   
   ```
   PipelineOptions options = PipelineOptionsFactory.fromArgs(argv).create();
   Pipeline p = Pipeline.create(options);
   p.apply(Create.of(1, 2, 3, 4, 5));
   p.run().waitUntilFinish();
   ```
   
   And build the uber jar with command ./gradlew :beamtest:shadowJar. Submit the job to Dataflow with
   
   ```
   ./gradlew :beamtest:run --args='--project=<...> --region=us-central1 \
   --tempLocation=gs://<...>/tmp --runner=DataflowRunner \
   --filesToStage=/<...>/build/libs/beamtest-1.0-all.jar'
   ```
   
   The job fails with error
   
   ```
   "org.apache.beam.sdk.coders.CoderException: `UnknownCoderWrapper` was used to perform an actual decoding in the Java SDK. Potentially a Java transform is being followed by a cross-language transform thatuses a coder that is not available in the Java SDK. Please make sure that Python transforms at the multi-language boundary use Beam portable coders.
   	at org.apache.beam.sdk.util.construction.UnknownCoderWrapper.decode(UnknownCoderWrapper.java:55)
   	at org.apache.beam.sdk.fn.data.BeamFnDataInboundObserver.multiplexElements(BeamFnDataInboundObserver.java:158)
   	at org.apache.beam.fn.harness.control.ProcessBundleHandler.processBundle(ProcessBundleHandler.java:537)
   	at org.apache.beam.fn.harness.control.BeamFnControlClient.delegateOnInstructionRequestType(BeamFnControlClient.java:150)
   	at org.apache.beam.fn.harness.control.BeamFnControlClient$InboundObserver.lambda$onNext$0(BeamFnControlClient.java:115)
   	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
   	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
   	at org.apache.beam.sdk.util.UnboundedScheduledExecutorService$ScheduledFutureTask.run(UnboundedScheduledExecutorService.java:163)
   	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
   	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
   	at java.lang.Thread.run(Thread.java:750)
   ```
   
   The same pipeline succeeded in Beam 2.54.0, 2.54.0 under Dataflow runner v2
   
   ### Issue Priority
   
   Priority: 1 (data loss / total loss of function)
   
   ### Issue Components
   
   - [ ] Component: Python SDK
   - [X] Component: Java SDK
   - [ ] Component: Go SDK
   - [ ] Component: Typescript SDK
   - [ ] Component: IO connector
   - [ ] Component: Beam YAML
   - [ ] 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


-- 
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


Re: [I] [Bug]: Uber jar not work with Dataflow runner v2 due to `UnknownCoderWrapper` for Beam 2.55 [beam]

Posted by "robertwb (via GitHub)" <gi...@apache.org>.
robertwb commented on issue #30994:
URL: https://github.com/apache/beam/issues/30994#issuecomment-2060139753

   Presumably this used to work and doesn't now? Or do we need better instructions on creating an uberjar that correctly preserves all the registration information? 


-- 
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


Re: [I] [Bug]: Uber jar not work with Dataflow runner v2 due to `UnknownCoderWrapper` for Beam 2.55 [beam]

Posted by "liferoad (via GitHub)" <gi...@apache.org>.
liferoad commented on issue #30994:
URL: https://github.com/apache/beam/issues/30994#issuecomment-2061115420

   Please create a doc about how to build an uber jar. We have been getting couple of customer issues related to this.


-- 
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


Re: [I] [Bug]: Uber jar not register coders causing `UnknownCoderWrapper` error [beam]

Posted by "chamikaramj (via GitHub)" <gi...@apache.org>.
chamikaramj commented on issue #30994:
URL: https://github.com/apache/beam/issues/30994#issuecomment-2061688287

   > Presumably this used to work and doesn't now? Or do we need better instructions on creating an uberjar that correctly preserves all the registration information?
   
   Yeah, I'm not sure what resulted in the regression. Might be the core-construction merge (but I haven't verified). +1 for updating instructions as a workaround while we figure out the root cause.


-- 
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


Re: [I] [Bug]: Uber jar not register coders causing `UnknownCoderWrapper` error [beam]

Posted by "robertwb (via GitHub)" <gi...@apache.org>.
robertwb commented on issue #30994:
URL: https://github.com/apache/beam/issues/30994#issuecomment-2061613599

   Long term, should we try to move away from autoservice for built in components (at least if standard uberjar building tools do not do the right thing with them)? Is this more possible now with the merging of runners core? @kennknowles 


-- 
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


Re: [I] [Bug]: Uber jar not work with Dataflow runner v2 due to `UnknownCoderWrapper` for Beam 2.55 [beam]

Posted by "chamikaramj (via GitHub)" <gi...@apache.org>.
chamikaramj commented on issue #30994:
URL: https://github.com/apache/beam/issues/30994#issuecomment-2060077026

   I don't think this is necessarily a bug but a limitation due the uber jar not properly registering coders. I tried a job with more logs and seems like we are simply not able to find a translator for the URN "beam:coder:length_prefix:v1" (and `UnknownCoderWrapper ` ends up being the fallback).
   
   We register LengthPrefixCoder here: https://github.com/apache/beam/blob/6bca71070e96b56b781600e8833a72cea329b1a1/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ModelCoderRegistrar.java#L48
   
   So seems like this registration is not being performed for the uber-jar.
   
   I'm not getting the error when I stage the `beam-sdks-java-core` jar (which performs the above registration) along with the uber jar.
   
   `--filesToStage=./build/libs/unknown_coder_error-1.0-all.jar,beam-sdks-java-core-2.56.0-SNAPSHOT.jar`
   
   I'm not sure if this is necessarily a release blocker. 
   
   


-- 
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


Re: [I] [Bug]: Uber jar not work with Dataflow runner v2 due to `UnknownCoderWrapper` for Beam 2.55 [beam]

Posted by "Abacn (via GitHub)" <gi...@apache.org>.
Abacn commented on issue #30994:
URL: https://github.com/apache/beam/issues/30994#issuecomment-2061431242

   I see, so
   
   - the underlying issue---coder register auto service not working in uber jar packed in this way---always exist
   - some change on 2.55.0 makes prefix coder involved in beam.Create, and expose this issue
   
   Having the understanding that the underlying issue always exist, and the action item is more like a documentation request (proper way to packaging uber jar), I agree this is isn't a release blocker. Adjust the priority tag accordingly


-- 
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


Re: [I] [Bug]: Uber jar not register coders causing `UnknownCoderWrapper` error [beam]

Posted by "robertwb (via GitHub)" <gi...@apache.org>.
robertwb commented on issue #30994:
URL: https://github.com/apache/beam/issues/30994#issuecomment-2064563142

   I created https://github.com/apache/beam/pull/31042 which should give a clearer error. Perhaps that is worth cherry-picking.


-- 
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