Percentile metrics in Beam

2020-08-14 Thread Ke Wu
Hi everyone, I am looking to add percentile metrics (p50, p90 etc) to my beam job but I only find Counter , Gauge

Re: Percentile metrics in Beam

2020-08-14 Thread Ke Wu
iNG2BAR-51pRdBCK4-XFmc0WuIkSuBzeb__Zv8owbU/edit> > > I think it'd be reasonable to add percentiles as its own metric type > as well. The tricky bit (though there are lots of resources on this) > is that one would have to publish more than just the percentiles from > each worke

Re: Percentile metrics in Beam

2020-08-18 Thread Ke Wu
gt;> wrote: > >> > > >> > Noone has any plans currently to work on adding a generic histogram > >> > metric, at the moment. > >> > > >> > But I will be actively working on adding it for a specific set of > >> >

@StateId uniqueness across DoFn(s)

2020-08-21 Thread Ke Wu
Hello everyone, After reading through Stateful processing with Apache Beam and DoFn.StateId , I understand that each state id must be unique and

Re: @StateId uniqueness across DoFn(s)

2020-08-21 Thread Ke Wu
ns > for completely different states. > > On Fri, Aug 21, 2020 at 10:21 AM Ke Wu <mailto:ke.wu...@gmail.com>> wrote: > Hello everyone, > > After reading through Stateful processing with Apache Beam > <https://beam.apache.org/blog/stateful-processing/> and DoFn

Re: @StateId uniqueness across DoFn(s)

2020-08-21 Thread Ke Wu
The DoFn is associated with a PTransform and in the pipeline proto there is a > unique id associated with each PTransform. You can use that to generate a > composite key (ptransformid, stateid) which will be unique within the > pipeline. > > On Fri, Aug 21, 2020 at 11:26

Re: @StateId uniqueness across DoFn(s)

2020-08-25 Thread Ke Wu
ify the transform name, and it is > autogenerated differently, this will be an error. This is why we throw > exceptions in the SDK if a name is re-used rather than just appending > a counter or similar.) > > > On Fri, Aug 21, 2020 at 4:12 PM Ke Wu wrote: >> >> If us

Contributor permissions for Beam Jira

2020-11-04 Thread Ke Wu
Hello, I am working at Samza team at LinkedIn and I would like to contribute to Samza runner in Beam. Could I please have permission to add/assign tickets on the Beam Jira? Best, Ke

Re: Contributor permissions for Beam Jira

2020-11-05 Thread Ke Wu
Absolutely, my jira username is kw2542 Thanks, Ke > On Nov 5, 2020, at 2:47 AM, Alexey Romanenko wrote: > > Hi, > > Sure. Could you provide your Jira username, please? > >> On 5 Nov 2020, at 00:48, Ke Wu wrote: >> >> Hello, >> >> I am worki

Re: Contributor permissions for Beam Jira

2020-11-06 Thread Ke Wu
Thank you, Alexey! > On Nov 6, 2020, at 5:58 AM, Alexey Romanenko wrote: > > Done, I added you to contributors list. > > Welcome! > > Please, take a look on Beam Contribution Guide if not yet =) > https://beam.apache.org/contribute/ > > Alexey > >>

Upgrade instruction from TimerDataCoder to TimerDataCoderV2

2020-11-06 Thread Ke Wu
Hello, I found that TimerDataCoderV2 is created to include timer family id and output timestamps fields in TimerData. In addition, the new fields are encoded between old fields, which I suppose V2 coder cannot decode and data that is encoded by V1 coder and vice versus. My ask here is, how shou

Re: Upgrade instruction from TimerDataCoder to TimerDataCoderV2

2020-11-06 Thread Ke Wu
der to upgrade coders. > > On Fri, Nov 6, 2020 at 2:13 PM Ke Wu <mailto:ke.wu...@gmail.com>> wrote: > Hello, > > I found that TimerDataCoderV2 is created to include timer family id and > output timestamps fields in TimerData. In addition, the new fields are > encode

Cross language pipeline example

2020-11-11 Thread Ke Wu
Hello, Is there an example demonstrating how a cross language pipeline look like? e.g. a pipeline where it is composes of Java and Python code/transforms. Best, Ke

Re: Cross language pipeline example

2020-11-12 Thread Ke Wu
e/master/sdks/python/apache_beam/examples/kafkataxi > > > > <https://github.com/apache/beam/tree/master/sdks/python/apache_beam/examples/kafkataxi> > > [2] > > https://github.com/apache/beam/blob/master/sdks/python/apache_beam/examples/wordcount_xlang_sql.py > >

Issue when FnApiDoFnRunner executes Read.BoundedSourceAsSDFWrapperFn

2021-02-04 Thread Ke Wu
Hello Beamers, I am trying out a simple pipeline to be executed on PortableRunner: PortablePipelineOptions options = PipelineOptionsFactory.fromArgs(args).as(PortablePipelineOptions.class); options.setJobEndpoint(some_url); options.setDefaultEnvironmentType("LOOPBACK"); options.setRunner(Po

Re: Issue when FnApiDoFnRunner executes Read.BoundedSourceAsSDFWrapperFn

2021-02-04 Thread Ke Wu
/apache/beam/runners/samza/SamzaPipelineRunner.java#L42-L47> > > > On Thu, Feb 4, 2021 at 11:31 AM Ke Wu <mailto:ke.wu...@gmail.com>> wrote: > Hello Beamers, > > I am trying out a simple pipeline to be executed on PortableRunner: > > > PortablePipeli

Implication on PTransform name

2021-02-17 Thread Ke Wu
Hello everyone, Is there any documentation on the implication of name param when applying a PTransform on PCollection? Some questions I have on mind is Is the name required to be unique across the pipeline? For stateful PTransforms such as stateful ParDo, Combine and Window, if no name is provi

Re: Implication on PTransform name

2021-02-17 Thread Ke Wu
ote: > > > > On Wed, Feb 17, 2021 at 11:25 AM Ke Wu <mailto:ke.wu...@gmail.com>> wrote: > Hello everyone, > > Is there any documentation on the implication of name param when applying a > PTransform on PCollection? Some questions I have on mind is > > Is the name

Unexpected empty state address window set for ACTIVE window

2021-03-10 Thread Ke Wu
Hello, We recently encountered an weird exception: Caused by: java.lang.IllegalStateException: Unexpected empty state address window set for ACTIVE window [2021-02-10T17:25:10.715Z..2021-02-10T20:25:10.715Z) at org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions

Portable Java Pipeline Support

2021-04-23 Thread Ke Wu
Hi All, I am working on add portability support for Samza Runner and having been playing around on the support in Flink and Spark runner recently. One thing I noticed is the lack of documentation on how to run a java pipeline in a portable mode. Almost all document focuses on how to run a pyth

Re: Portable Java Pipeline Support

2021-04-23 Thread Ke Wu
h > will not be the case unless you modify the job server to depend on the > examples module. > > [1] https://issues.apache.org/jira/browse/BEAM-5440 > <https://issues.apache.org/jira/browse/BEAM-5440> > On Fri, Apr 23, 2021 at 11:24 AM Ke Wu <mailto:ke.wu...@gmail.com>&g

Re: Portable Java Pipeline Support

2021-04-26 Thread Ke Wu
7915c66f6206249c31e6160b8b605a013d9f04/runners/spark/job-server/spark_job_server.gradle#L186 > > <https://github.com/apache/beam/blob/247915c66f6206249c31e6160b8b605a013d9f04/runners/spark/job-server/spark_job_server.gradle#L186> > On Fri, Apr 23, 2021 at 3:25 PM Ke Wu <mai

Re: Portable Java Pipeline Support

2021-04-26 Thread Ke Wu
process. In the case of > LOOPBACK, that external process is started by the Python client process that > submitted the job in the first place. > > On Mon, Apr 26, 2021 at 9:57 AM Ke Wu <mailto:ke.wu...@gmail.com>> wrote: > Thank you Kyle, I have created BEAM-12227 > &l

Customizable Artifacts to Stage in Java Portable Runner

2021-04-28 Thread Ke Wu
Hello All, I am expecting FileStagingOptions#setFilesToStage in PortablePipelineOptions is the way to customize artifacts to be staged and resolved in portable

Re: Customizable Artifacts to Stage in Java Portable Runner

2021-04-28 Thread Ke Wu
that ArtifactRetrievalService can > > retrieve artifacts from a remote address instead of default from JobServer, > > which got artifacts from SDK Client. I am asking because I noticed > > Files can use any of Beam's supported remote file systems (GCS, S3, Azure > Blo

Re: Customizable Artifacts to Stage in Java Portable Runner

2021-04-29 Thread Ke Wu
s.apache.org/jira/browse/BEAM-11275 > <https://issues.apache.org/jira/browse/BEAM-11275> > [3] https://issues.apache.org/jira/browse/BEAM-8137 > <https://issues.apache.org/jira/browse/BEAM-8137> > On Wed, Apr 28, 2021 at 6:36 PM Ke Wu <mailto:ke.wu...@gmail.com>

[DISCUSS] Client SDK/Job Server/Worker Pool Lifecycle Management on Kubernetes

2021-05-14 Thread Ke Wu
Hello All, I came across this question when I am reading Beam on Flink on Kubernetes and flink-on-k8s-operator

Re: [DISCUSS] Client SDK/Job Server/Worker Pool Lifecycle Management on Kubernetes

2021-05-27 Thread Ke Wu
-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/ExternalEnvironmentFactory.java#L115> > On Fri, May 14, 2021 at 10:51 AM Ke Wu <mailto:ke.wu...@gmail.com>> wrote: > Hello All, > > I came across this question when I am reading Beam on Flink on Kuberne

Re: [DISCUSS] Client SDK/Job Server/Worker Pool Lifecycle Management on Kubernetes

2021-05-27 Thread Ke Wu
ot;Still waiting for startup of > environment from {} for worker id {}" > > On Thu, May 27, 2021 at 4:04 PM Ke Wu <mailto:ke.wu...@gmail.com>> wrote: > Hi Kyle, > > Thank you for the prompt response and apologize for the late reply. > > [1] seems to be only

Re: [DISCUSS] Client SDK/Job Server/Worker Pool Lifecycle Management on Kubernetes

2021-06-02 Thread Ke Wu
1] [Public] Beam Flink K8s: https://docs.google.com/document/d/1z3LNrRtr8kkiFHonZ5JJM_L4NWNBBNcqRc_yAf6G0VI/edit#heading=h.bzs6y6ms0898 <https://docs.google.com/document/d/1z3LNrRtr8kkiFHonZ5JJM_L4NWNBBNcqRc_yAf6G0VI/edit#heading=h.bzs6y6ms0898> > On May 27, 2021, at 6:11 PM, Ke Wu wrote

Re: [DISCUSS] Client SDK/Job Server/Worker Pool Lifecycle Management on Kubernetes

2021-06-02 Thread Ke Wu
So I'm not sure whether or not the problem you describe > would ever actually happen. > > On Wed, Jun 2, 2021 at 11:01 AM Ke Wu <mailto:ke.wu...@gmail.com>> wrote: > Hi Kyle, > > Thanks for reviewing https://github.com/apache/beam/pull/14923 > <https

Re: [DISCUSS] Client SDK/Job Server/Worker Pool Lifecycle Management on Kubernetes

2021-06-02 Thread Ke Wu
Very good point. We are actually talking about the same high level approach where Task Manager Pod has two containers inside running, one is task manager container while the other is worker pool service container. I believe the disconnect probably lies in how a job is being deployed/started. In

Re: [DISCUSS] Client SDK/Job Server/Worker Pool Lifecycle Management on Kubernetes

2021-06-03 Thread Ke Wu
There is also a talk [1] which introduces dynamic scaling a stream processing job at LinkedIn with Samza runner as well [1] https://www.usenix.org/conference/hotcloud20/presentation/singh <https://www.usenix.org/conference/hotcloud20/presentation/singh> > On Jun 3, 2021, at 1:59

Translates composite transform in portable pipeline

2021-07-26 Thread Ke Wu
Hello All, I noticed that Flink/Spark/Samza runners are translating portable pipeline in the similar manner: > QueryablePipeline p = > QueryablePipeline.forTransforms( > pipeline.getRootTransformIdsList(), pipeline.getComponents()); > > for (PipelineNode.PTransformNode transform : >

Re: Translates composite transform in portable pipeline

2021-07-27 Thread Ke Wu
ticated ways of handling associative, > commutative CombineFn aggregations (See > https://docs.google.com/presentation/d/1Cso0XP9dmj77OD9Bd53C1M3W1sPJF0ZnA20gzb2BPhE/edit#slide=id.g42e4c9aad6_0_260 > ) > > - Robert > > On Mon, Jul 26, 2021 at 5:27 PM Ke Wu wrote: >>

Identify primitive/native transforms in portable pipeline

2021-07-27 Thread Ke Wu
Hello All, When I am looking at translating composite transforms in potable pipeline, I realized that TrivialNativeTransformExpander[1] is being used to identify native transforms by transform urn, and it removes sub-transform and environment id in the corresponding transform node. However, Que

Re: Command line to run DatstoreIO integration tests for java

2021-07-28 Thread Ke Wu
Comment the PR with "Run SQL PostCommit” would trigger the post commit integration tests for SQL, which I suppose includes DataStoreReadWriteIT Let me know if whether or not this is sufficient. Best, Ke > On Jul 28, 2021, at 12:20 PM, Alex Amato wrote: > > Is it possible to run a Datastore IO

Re: Identify primitive/native transforms in portable pipeline

2021-07-28 Thread Ke Wu
transform directly, > (2) If you don't know the URN, look to see if its outputs are a subset > of the inputs (in which case there's nothing to do) or it has > subtransforms (in which case you can just recurse to compute the > outputs). > (3) If neither (1) nor (2) hold, then yo

Re: Command line to run DatstoreIO integration tests for java

2021-07-29 Thread Ke Wu
ul 29, 2021, at 9:58 AM, Alex Amato wrote: > > I was hoping for the command line to run it. So that the test could be > tweaked to inject an error, and ensure the error handling code works as > expected > > On Wed, Jul 28, 2021 at 8:34 PM Ke Wu <mailto:ke.wu...@gmail.co

Re: Identify primitive/native transforms in portable pipeline

2021-08-02 Thread Ke Wu
e question and infrastructure around "is > this primitive" could simply go away. > > On Wed, Jul 28, 2021 at 8:47 PM Ke Wu wrote: >> >> Hi Robert, >> >> Thanks for the reply, the motivation for this is, I noticed when we need to >> translate a comp

Unexpected in TestStream in Portable Mode

2021-08-30 Thread Ke Wu
Hello everyone, This is Ke. I am working on enable TestStream support for Samza Runner in portable mode and discovers something unexpected. In my implementation for Samza Runner, couple of tests are failing with errors like java.lang.ClassCastException: java.lang.Integer cannot be cast to [B

Re: Unexpected in TestStream in Portable Mode

2021-08-31 Thread Ke Wu
Translator.java#L657> > On 8/31/21 7:27 PM, Luke Cwik wrote: >> I originally wasn't for making it a composite because it changes the "graph" >> structure but the more I thought about it the more I like it. >> >> On Tue, Aug 31, 2021 at 10:06 AM Robert

Re: Unexpected in TestStream in Portable Mode

2021-08-31 Thread Ke Wu
ote: >> >> I originally wasn't for making it a composite because it changes the "graph" >> structure but the more I thought about it the more I like it. >> >> On Tue, Aug 31, 2021 at 10:06 AM Robert Bradshaw wrote: >>> >>> On Tue,

Re: Unexpected in TestStream in Portable Mode

2021-08-31 Thread Ke Wu
he/beam/blob/ff70e740a2155592dfcb302ff6303cc19660a268/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java#L48> > On 8/31/21 11:29 PM, Ke Wu wrote: >> Awesome! Thank you Luke and Robert. >> >> Also created https:/

Re: Unexpected in TestStream in Portable Mode

2021-08-31 Thread Ke Wu
ter/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java#L149 <https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java#L149> > On Aug 31, 2021, at 3:21 PM, Jan Lukavský wrote: > > On 9/1/21 12:13 AM, Ke Wu wrote: >>

Tests failure on master branch

2021-08-31 Thread Ke Wu
Hello, I noticed there are some tests that are failing in master [1] Portable_Python ("Run Portable_Python PreCommit”) Python_PVR_Flink ("Run Python_PVR_Flink PreCommit") Are they known issues? If Yes, is there a ticket for them? Best, Ke [1] https://github.com/apache/beam/pull/15433

Re: MapState API

2021-09-10 Thread Ke Wu
Another question on this topic, is any work planned to add the map state support in portable mode [1], same for set state, list state? [1] https://github.com/apache/beam/blob/master/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/FnApiStateAccessor.java#L337

Re: MapState API

2021-09-13 Thread Ke Wu
Awesome, looking forward to it. > On Sep 10, 2021, at 10:30 AM, Luke Cwik wrote: > > Yes, https://github.com/apache/beam/pull/15238 > <https://github.com/apache/beam/pull/15238> is in progress for MapState and > SetState. > > On Fri, Sep 10, 2021 at 9:22 AM Ke Wu

Re: Percentile metrics in Beam

2021-09-14 Thread Ke Wu
I prefer adding a new metrics type instead of enhancing the existing Distribution [1] to support percentiles etc in order to ensure better compatibility. @Luke @Kyle what are your thoughts on this? Best, Ke [1] https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apac

Re: Percentile metrics in Beam

2021-09-17 Thread Ke Wu
e a design doc with >> the community with possible solutions very soon. >> >> Thanks >> Ajo >> >>> On Wed, Sep 15, 2021 at 9:26 AM Alexey Romanenko >>> wrote: >>> I agree with Ke Wu in the way that we need to keep compatibility across

Multi Environment Support

2021-09-21 Thread Ke Wu
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 support

Re: Multi Environment Support

2021-09-22 Thread Ke Wu
a/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyPCollectionFusers.java#L144> > On Tue, Sep 21, 2021 at 1:45 PM Ke Wu <mailto:ke.wu...@gmail.com>> wrote: > Hello All, > > We have a use case where in a java portable pipeline, we would like to have > mult

Re: Multi Environment Support

2021-09-29 Thread Ke Wu
ansform. > > 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 <mailto:ke.wu...@gmail.com>> wrote: > Thanks Luke for the reply, do

Re: Multi Environment Support

2021-09-30 Thread Ke Wu
ssues are a common problem. > > > On Wed, Sep 29, 2021 at 11:47 AM Ke Wu <mailto: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 > fra

Re: Multi Environment Support

2021-09-30 Thread Ke Wu
sdk/transforms/PTransform.java#L218> > On Sep 30, 2021, at 10:34 AM, Robert Bradshaw wrote: > > On Thu, Sep 30, 2021 at 9:25 AM Ke Wu <mailto:ke.wu...@gmail.com>> wrote: >> >> Ideally, we do not want to expose anything directly to users and we, as the >> framew

Reshuffle Discrepancy in Classic vs Portable Pipeline

2021-10-04 Thread Ke Wu
Hello All, Recent Samza Runner tests failure in python/xlang [1][2] reveals an interesting fact that Reshuffle Transform in classic pipeline requires the input to be KV while portable pipeline does not, where Reshuffle in portable mode it has an extra step to append a random key [3]. This sug

Re: Reshuffle Discrepancy in Classic vs Portable Pipeline

2021-10-04 Thread Ke Wu
> > Reshuffle is not keyed, there is a separate reshuffle-per-key for > that. This is true for both Java and Python. This shouldn't depend on > classic vs. portable mode. It sounds like there's an issue in > translation. > > On Mon, Oct 4, 2021 at 11:18 AM Ke Wu

Re: Reshuffle Discrepancy in Classic vs Portable Pipeline

2021-10-04 Thread Ke Wu
bert Bradshaw wrote: > > Oh, yes. > > Java Reshuffle.of() = Python ReshufflePerKey() > Java Reshuffle.viaRandomKey() == Python Reshuffle() > > We generally try to avoid this kind of discrepancy. It could make > sense to rename Reshuffle.of() to Reshuffle.viaKey(). >

Re: Reshuffle Discrepancy in Classic vs Portable Pipeline

2021-10-04 Thread Ke Wu
believe it was deprecated for wrong reasons - yes, it >> has undocumented and non-portable side-effects, but is still makes sense >> for various use-cases (e.g. fan-out, or SDF). > > +1 > >>> >>> On Mon, Oct 4, 2021 at 2:33 PM Ke Wu wrote: >>>> I

Re: Reshuffle Discrepancy in Classic vs Portable Pipeline

2021-10-04 Thread Ke Wu
Created https://issues.apache.org/jira/browse/BEAM-12999 <https://issues.apache.org/jira/browse/BEAM-12999> > On Oct 4, 2021, at 3:37 PM, Robert Bradshaw wrote: > > Thanks. Happy to help with Python/Go. Do you want to create a JIRA? > > On Mon, Oct 4, 2021 at 3:33 PM Ke

Re: Multi Environment Support

2021-10-04 Thread Ke Wu
This is great, let me try it out. Best, Ke > On Sep 30, 2021, at 6:06 PM, Robert Bradshaw wrote: > > On Thu, Sep 30, 2021 at 6:00 PM Ke Wu wrote: >> >> I am able to annotate/mark a java transform by setting its resource hints >> [1] as well, which resulted in a di

Help needed in Go validates runner tests

2021-10-04 Thread Ke Wu
Hello All, Samza Go Validates Runner started fail around 10/01/2021[1], the three commits that contributes to the first failure [2] looks innocent, I also tried to revert them but still got the same failure. What is confusing is the console log: 11:15:43 --- PASS: TestEmitParDoAfterGBK (5.70s

Re: Help needed in Go validates runner tests

2021-10-04 Thread Ke Wu
d disabled the failing tests > https://github.com/apache/beam/pull/15659 > <https://github.com/apache/beam/pull/15659> > > But Jenkins had not started execution after 16 minutes so i called it a day. > > On Mon, Oct 4, 2021, 5:30 PM Ke Wu <mailto:ke.wu...@gmail.c

Re: Reshuffle Discrepancy in Classic vs Portable Pipeline

2021-10-06 Thread Ke Wu
instead. > > > On Mon, Oct 4, 2021 at 6:31 PM Robert Burke <mailto:rob...@frantil.com>> wrote: > I can handle the Go SDK change once the urn is decided. I'm cleaning up a > change to add the combine_global urn in the Go SDK so this can slip in along > side

Re: Multi Environment Support

2021-10-06 Thread Ke Wu
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. Best, Ke > On Oct 4, 2021, at 4:12 PM, Ke Wu wr

Re: Reshuffle Discrepancy in Classic vs Portable Pipeline

2021-10-07 Thread Ke Wu
is the same as though it were unkeyed. > > Doing anything with the user key specifically would seem to me to defeat the > point of a reshuffle, vs just using a GBK which would align keys to bundles > in it's output. > > > On Wed, Oct 6, 2021, 10:54 AM Ke Wu <mailto:ke

Re: Multi Environment Support

2021-11-17 Thread Ke Wu
service address mapping? Best, Ke > On Oct 6, 2021, at 2:09 PM, Robert Bradshaw wrote: > > On Wed, Oct 6, 2021 at 1:12 PM Ke Wu wrote: >> >> I have a quick follow up questions. >> >> When using multiple external environments, is there a way to configure

Re: Multi Environment Support

2021-12-09 Thread Ke Wu
te example of what such a specification would look > like? > > On Wed, Nov 17, 2021 at 11:05 AM Ke Wu wrote: >> >> Hi Robert, >> >> Thanks for the pointer, using expansion service hack seems to work! >> >> On the other hand, since PipelineOptions is the place t

Re: Multi Environment Support

2021-12-13 Thread Ke Wu
> > On Thu, Dec 9, 2021 at 4:27 PM Ke Wu 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

Re: Multi Environment Support

2021-12-13 Thread Ke Wu
Dec 13, 2021, at 10:40 AM, Robert Bradshaw wrote: > > I think bringing up the expansion service as part of the job server is > just a convenience for the simple case. > > On Mon, Dec 13, 2021 at 9:54 AM Ke Wu wrote: >> >> Awesome, I am happy to make the update. &

Re: Multi Environment Support

2021-12-15 Thread Ke Wu
service but the option of the pipeline that is created with default, i.e. not configurable. > On Dec 14, 2021, at 8:49 PM, Chamikara Jayalath wrote: > > > > On Mon, Dec 13, 2021 at 11:12 AM Ke Wu <mailto:ke.wu...@gmail.com>> wrote: > Agreed, that is why I

Re: Multi Environment Support

2021-12-15 Thread Ke Wu
gt; On Wed, Dec 15, 2021 at 10:38 AM Ke Wu <mailto:ke.wu...@gmail.com>> wrote: > Thanks for Cham to chime in, having each expansion service instance to be > able to serve a distinct environment which is configurable is what I am > looking for, however, I don’t think it is practi