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

   ### What happened?
   
   Some test suites are currently failing with errors like:
   
   ```
   RuntimeError: Pipeline construction environment and pipeline runtime 
environment are not compatible. If you use a custom container image, check that 
the Python interpreter minor version and the Apache Beam version in your image 
match the versions used at pipeline construction time. Submission environment: 
beam:version:sdk_base:apache/beam_python3.13_sdk:2.73.0.dev. Runtime 
environment: beam:version:sdk_base:apache/beam_python3.13_sdk:2.72.0.dev.
   ```
   
    https://github.com/apache/beam/pull/37665#issuecomment-3937738276
   
   
   Rootcause is that py3.13 wheel format 
apache_beam-2.73.0.dev0-cp313-cp313-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl
 doesn't match expected format: 
https://github.com/apache/beam/blob/9ff96dfd3299cd3604bb4a0c1aaeceb3bddb4d85/sdks/python/container/boot.go#L422
   
   so the staged whl gets ignored, hence we have a mismatch between SDK's 
version at submission (2.73.0.dev) and at runtime (2.72.0.dev).
   
   The logic in boot.go was influenced by the old assumptions that:
   - SDK is always staged to Dataflow worker (still the case for tests, no 
longer the case for prod)
   - When SDK package is staged its name must be predetermined, like 
https://github.com/apache/beam/blob/9ff96dfd3299cd3604bb4a0c1aaeceb3bddb4d85/sdks/python/container/boot.go#L72
 . 
   
   To fix, we should not assume that package name is predetermined or do any 
sort of wheel validation: we should trust that the wheel name passed via 
--sdk_location will be installable.
   
   We need to fix the go code in `/sdks/python/container`, then release a new 
beam-master container to fix this.
   
   ### Issue Priority
   
   Priority: 2 (default / most bugs should be filed as P2)
   
   ### Issue Components
   
   - [x] Component: Python SDK
   - [ ] 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: Infrastructure
   - [ ] 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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to