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

   ### What happened?
   
   This is especially painful as the default Beam Java runner can't run 
cross-language transforms, so one is forced to use Docker when running on local 
runners such as Prism or the Python Universal Local Runner.
   
   To reproduce, try running any of the examples at 
https://github.com/apache/beam/blob/master/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonExternalTransform.java
 with the Prism runner. E.g.
   
   ```
   PortablePipelineOptions options = 
PipelineOptionsFactory.create().as(PortablePipelineOptions.class);
   options.setDefaultEnvironmentType("LOOPBACK");
   
   Pipeline p = Pipeline.create(options);
   p.apply(Create.of(KV.of("A", "x"), KV.of("A", "y"), KV.of("B", "z")))
     .apply(PythonExternalTransform
                       .<PCollection<KV<String, String>>, 
PCollection<KV<String, Iterable<String>>>>
                           from("apache_beam.GroupByKey"))
   ```
   
   results in 
   
   ```
   java.lang.IllegalArgumentException: External service address must not be 
empty (set it using '--environmentOptions=external_service_address=...'?).
        at 
org.apache.beam.sdk.util.construction.Environments.createExternalEnvironment(Environments.java:224)
        at 
org.apache.beam.sdk.util.construction.Environments.createOrGetDefaultEnvironment(Environments.java:193)
        at 
org.apache.beam.sdk.util.construction.SdkComponents.create(SdkComponents.java:109)
        at 
org.apache.beam.sdk.util.construction.External$ExpandableTransform.expand(External.java:221)
   ```
   
   We should allow creating a placeholder environment of this type and late 
binding it to the loopback environment once that is created. (Possibly we 
should do this for all environments during construction and only bind them when 
we go to run them.)
   
   ### Issue Priority
   
   Priority: 2 (default / most bugs should be filed as P2)
   
   ### 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: 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: github-unsubscr...@beam.apache.org.apache.org

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

Reply via email to