Awesome, I am happy to make the update. One more question, I noticed that expansion service is currently being brought up as part of job server in Java [1], if this is the preferred approach, does it mean, we should update ServerConfiguration to include such configurations, like expansion server port [2]
Best, Ke [1] https://github.com/apache/beam/blob/master/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/JobServerDriver.java#L58 <https://github.com/apache/beam/blob/master/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/JobServerDriver.java#L58> [2] https://github.com/apache/beam/blob/master/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/JobServerDriver.java#L88 <https://github.com/apache/beam/blob/master/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/JobServerDriver.java#L88> > On Dec 9, 2021, at 4:34 PM, Robert Bradshaw <rober...@google.com> wrote: > > An expansion service should be able to specify its own environment, as > is done in Python, not blindly inherit it from the caller. Java should > be updated to have the same flexibility. > > On Thu, Dec 9, 2021 at 4:27 PM Ke Wu <ke.wu...@gmail.com> wrote: >> >> Hi Robert, >> >> After some more digging and exploration, I realized that it is not clean and >> straightforward to update PipelineOption to support such. >> >> Therefore, I spent more time exploring the external transform & expansion >> service route. However, I noticed that this approach has the same limitation >> where we cannot configure environments ourselves. >> >> This is because when ExpansionService register environment [1], it uses the >> pipeline options of the original pipeline. This limitation not only restrict >> our capability to configure a different external environment in external >> mode but also restrict Docker mode to specify docker image for different >> SDKs but use defaults. >> >> Is this expected or do you think ExpansionService should be updated to be >> configurable of the environment it registers ? >> >> Best, >> Ke >> >> >> [1] >> >> Java: >> https://github.com/apache/beam/blob/master/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java#L526 >> >> Python: >> https://github.com/apache/beam/blob/master/sdks/python/apache_beam/runners/portability/expansion_service.py#L39 >> >> >> >> On Nov 17, 2021, at 1:02 PM, Robert Bradshaw <rober...@google.com> wrote: >> >> Could you give a concrete example of what such a specification would look >> like? >> >> On Wed, Nov 17, 2021 at 11:05 AM Ke Wu <ke.wu...@gmail.com> wrote: >> >> >> Hi Robert, >> >> Thanks for the pointer, using expansion service hack seems to work! >> >> On the other hand, since PipelineOptions is the place to configure external >> service address anyway, do you think it makes sense to expand it so it is >> capable of specifying multiple external environment to external service >> address mapping? >> >> Best, >> Ke >> >> On Oct 6, 2021, at 2:09 PM, Robert Bradshaw <rober...@google.com> wrote: >> >> On Wed, Oct 6, 2021 at 1:12 PM Ke Wu <ke.wu...@gmail.com> wrote: >> >> >> I have a quick follow up questions. >> >> When using multiple external environments, is there a way to configure the >> multiple external service address? It looks like the current PipelineOptions >> only supports specifying one external address. >> >> >> PipelineOptions wasn't really built with the idea of multiple distinct >> environments in mind. One hack you could do is put one of the >> environments in an expansion service with its own environment (as if >> it were written in a different language) and configure that >> environment separately. >> >> On Oct 4, 2021, at 4:12 PM, Ke Wu <ke.wu...@gmail.com> wrote: >> >> This is great, let me try it out. >> >> Best, >> Ke >> >> On Sep 30, 2021, at 6:06 PM, Robert Bradshaw <rober...@google.com> wrote: >> >> On Thu, Sep 30, 2021 at 6:00 PM Ke Wu <ke.wu...@gmail.com> wrote: >> >> >> I am able to annotate/mark a java transform by setting its resource hints >> [1] as well, which resulted in a different environment id, e.g. >> >> beam:env:docker:v1 VS beam:env:docker:v11 >> >> Is this on the right track? >> >> >> Yep. >> >> If Yes, I suppose then I need to configure job bundle factory to be able to >> understand multiple environments and configure them separately as well. >> >> >> It should already do the right thing here. That's how multi-language works. >> >> [1] >> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java#L218 >> >> On Sep 30, 2021, at 10:34 AM, Robert Bradshaw <rober...@google.com> wrote: >> >> On Thu, Sep 30, 2021 at 9:25 AM Ke Wu <ke.wu...@gmail.com> wrote: >> >> >> Ideally, we do not want to expose anything directly to users and we, as the >> framework and platform provider, separate things out under the hood. >> >> I would expect users to author their DoFn(s) in the same way as they do >> right now, but we expect to change the DoFn(s) that we provide, will be >> annotated/marked so that it can be recognized during runtime. >> >> In our use case, application is executed in Kubernetes environment >> therefore, we are expecting to directly use different docker image to >> isolate dependencies. >> >> e.g. we have docker image A, which is beam core, that is used to start job >> server and runner process. We have a docker image B, which contains DoFn(s) >> that platform provides to serve as a external worker pool service to execute >> platform provided DoFn(s), last but not least, users would have their own >> docker image represent their application, which will be used to start the >> external worker pool service to handle their own UDF execution. >> >> Does this make sense ? >> >> >> In Python it's pretty trivial to annotate transforms (e.g. the >> "platform" transforms) which could be used to mark their environments >> prior to optimization (e.g. fusion). As mentioned, you could use >> resource hints (even a "dummy" hint like >> "use_platform_environment=True") to force these into a separate docker >> image as well. >> >> On Sep 29, 2021, at 1:09 PM, Luke Cwik <lc...@google.com> wrote: >> >> That sounds neat. I think that before you try to figure out how to change >> Beam to fit this usecase is to think about what would be the best way for >> users to specify these requirements when they are constructing the pipeline. >> Once you have some samples that you could share the community would probably >> be able to give you more pointed advice. >> For example will they be running one application with a complicated class >> loader setup, if so then we could probably do away with multiple >> environments and try to have DoFn's recognize their specific class loader >> configuration and replicate it on the SDK harness side. >> >> Also, for performance reasons users may want to resolve their dependency >> issues to create a maximally fused graph to limit performance impact due to >> the encoding/decoding boundaries at the edges of those fused graphs. >> >> Finally, this could definitely apply to languages like Python and Go (now >> that Go has support for modules) as dependency issues are a common problem. >> >> >> On Wed, Sep 29, 2021 at 11:47 AM Ke Wu <ke.wu...@gmail.com> wrote: >> >> >> Thanks for the advice. >> >> Here are some more background: >> >> We are building a feature called “split deployment” such that, we can >> isolate framework/platform core from user code/dependencies to address >> couple of operational challenges such as dependency conflict, >> alert/exception triaging. >> >> With Beam’s portability framework, runner and sdk worker process naturally >> decouples beam core and user UDFs(DoFn), which is awesome! On top of this, >> we could further distinguish DoFn(s) that end user authors from DoFn(s) that >> platform provides, therefore, we would like these DoFn(s) to be executed in >> different environments, even in the same language, e.g. Java. >> >> Therefore, I am exploring approaches and recommendations what are the proper >> way to do that. >> >> Let me know your thoughts, any feedback/advice is welcome. >> >> Best, >> Ke >> >> On Sep 27, 2021, at 11:56 AM, Luke Cwik <lc...@google.com> wrote: >> >> Resource hints have a limited use case and might fit your need. >> You could also try to use the expansion service XLang route to bring in a >> different Java environment. >> Finally, you could modify the pipeline proto that is generated directly to >> choose which environment is used for which PTransform. >> >> Can you provide additional details as to why you would want to have two >> separate java environments (e.g. incompatible versions of libraries)? >> >> On Wed, Sep 22, 2021 at 3:41 PM Ke Wu <ke.wu...@gmail.com> wrote: >> >> >> Thanks Luke for the reply, do you know what is the preferred way to >> configure a PTransform to be executed in a different environment from >> another PTransform when both are in the same SDK, e.g. Java ? >> >> Best, >> Ke >> >> On Sep 21, 2021, at 9:48 PM, Luke Cwik <lc...@google.com> wrote: >> >> Environments that aren't exactly the same are already in separate >> ExecutableStages. The GreedyPCollectionFuser ensures that today[1]. >> >> Workarounds like getOnlyEnvironmentId would need to be removed. It may also >> be effectively dead-code. >> >> 1: >> https://github.com/apache/beam/blob/ebf2aacf37b97fc85b167271f184f61f5b06ddc3/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyPCollectionFusers.java#L144 >> >> On Tue, Sep 21, 2021 at 1:45 PM Ke Wu <ke.wu...@gmail.com> wrote: >> >> >> Hello All, >> >> We have a use case where in a java portable pipeline, we would like to have >> multiple environments setup in order that some executable stage runs in one >> environment while some other executable stages runs in another environment. >> Couple of questions on this: >> >> 1. Is this current supported? I noticed a TODO in [1] which suggests it is >> feature pending support >> 2. If we did support it, what would the ideal mechanism to distinguish >> ParDo/ExecutableStage to be executed in different environment, is it through >> ResourceHints? >> >> >> Best, >> Ke >> >> >> [1] >> https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java#L344 >> >> >> >> >> >>