Great -- thanks for your help! PR created: [FLINK-26340][statefun-golang-sdk] Add ability in Golang SDK to create new statefun.Context from existing one, but with a new underlying context.Context by galenwarren · Pull Request #303 · apache/flink-statefun (github.com) <https://github.com/apache/flink-statefun/pull/303>
On Wed, Feb 23, 2022 at 11:42 AM Austin Cawley-Edwards < austin.caw...@gmail.com> wrote: > I agree with you Galen, adding a default implementation would not be so > natural here. We could eventually change this from being an interface to > just a struct (like http.Request) if there is no use case for custom > implementations. > > Thanks for the nice discussion here + contribution :) > > Austin > > > On Wed, Feb 23, 2022 at 11:09 AM Galen Warren <ga...@cvillewarrens.com> > wrote: > >> Thanks, Till. >> >> >>> Is it possible in Go to provide a default implementation for a method? >>> Maybe that way the introduction of the new method would not be breaking. >>> >> >> I can't think of a way to do this that would be seamless to someone who >> had chosen to create a custom struct implementing statefun.Context. We >> could create a struct -- say, statefun.ContextDefaultBehavior -- that an >> implementor could embed in such a custom struct and which would provide an >> implementation of WithContext, but that embedding would have to be added to >> the custom struct (i.e. it wouldn't prevent the initial compile-time >> breakage), and there would be no way to provide an implementation that >> would work in the general case. I'm not sure that would be any better than >> just requiring the implementor to add a WithContext method. >> >> Austin, please chime in if you have a different idea. >> >> I'll proceed for now with the intent of adding the method to >> statefun.Context. If we decide that a top-level function is a better >> approach, that would be an easy switch to make. >> >> JIRA created: [FLINK-26340] Add ability in Golang SDK to create new >> statefun.Context from existing one, but with a new underlying >> context.Context - ASF JIRA (apache.org) >> <https://issues.apache.org/jira/browse/FLINK-26340> >> >> I have the code basically done locally, so, barring objection, I'll add a >> PR soon. >> >> >> >> >> >> >> On Wed, Feb 23, 2022 at 3:41 AM Till Rohrmann <trohrm...@apache.org> >> wrote: >> >>> Thanks a lot for the discussion Austin and Galen. I think it would be >>> fine >>> to break the API at this point in time. Moreover, I don't assume that >>> many >>> people have their own statefun.Context implementations. Is it possible in >>> Go to provide a default implementation for a method? Maybe that way the >>> introduction of the new method would not be breaking. >>> >>> Galen, do you want to open a JIRA issue for this proposal? >>> >>> Cheers, >>> Till >>> >>> On Wed, Feb 23, 2022 at 12:50 AM Galen Warren <ga...@cvillewarrens.com> >>> wrote: >>> >>> > Yeah, good point. I wasn't considering that someone else might be >>> > implementing that interface. Practically, I think that's pretty >>> unlikely, >>> > but good question. >>> > >>> > >>> > >>> > On Tue, Feb 22, 2022 at 6:36 PM Austin Cawley-Edwards < >>> > austin.caw...@gmail.com> wrote: >>> > >>> > > I think the only complication with adding `WithContext` to the >>> interface >>> > > is that it makes it breaking :/ If it were the concrete >>> implementation, >>> > > we'd have more flexibility. I'm not sure what guarantees are on those >>> > > interfaces though – @sjwies...@gmail.com, wdyt? >>> > > >>> > > On Tue, Feb 22, 2022 at 4:19 PM Galen Warren < >>> ga...@cvillewarrens.com> >>> > > wrote: >>> > > >>> > >> I think I would choose the WithContext method at this point, >>> assuming >>> > >> that the implementation keeps any instance of the stateful context >>> > >> immutable (which should be doable). It's the simplest option IMO, >>> > simpler >>> > >> than an adapter approach. Thanks for the suggestion. >>> > >> >>> > >> My slight preference would be to add 'WithContext' as a method to >>> > >> statefun.Context (similar to your Request.WithContext example >>> > >> <https://pkg.go.dev/net/http#Request.WithContext>), as opposed to a >>> > >> top-level function, i.e. statefun.WithContext, but either one could >>> > work. >>> > >> >>> > >> Would that work for you? What do the rest of you think? >>> > >> >>> > >> >>> > >> >>> > >> On Tue, Feb 22, 2022 at 3:52 PM Austin Cawley-Edwards < >>> > >> austin.caw...@gmail.com> wrote: >>> > >> >>> > >>> What does "SomeOtherFunc" need with the statefun context >>> > >>>> >>> > >>>> I think it's hard to answer this question, in a general sense. >>> > >>>> Depending on what is being done, it might need to read a value >>> from >>> > >>>> statefun Storage, write one back, etc. >>> > >>> >>> > >>> >>> > >>> To me, this indicates that the context is responsible for too much >>> and >>> > >>> cannot properly be passed to functions with a distinct purpose. I >>> > think the >>> > >>> `context.Context` shares this design but gets away with it because >>> its >>> > >>> functionality is so constrained and generic (deadlines, >>> cancellation, >>> > >>> values – that's it). >>> > >>> This is getting away from the original question of the thread, but >>> I >>> > >>> bring it up to suggest that we take a more holistic look at the >>> > statefun >>> > >>> Context interface and go with a simpler solution (either the >>> > `WithContext` >>> > >>> method or Till's suggestion) to avoid further muddying the possible >>> > uses. >>> > >>> WDYT? >>> > >>> >>> > >>> Austin >>> > >>> >>> > >>> On Tue, Feb 22, 2022 at 1:14 PM Galen Warren < >>> ga...@cvillewarrens.com> >>> > >>> wrote: >>> > >>> >>> > >>>> One place we could look is the `net/http` Request, which has a >>> > >>>>> `WithContext` method[1] that seems to expose the behavior we're >>> > looking >>> > >>>>> for. >>> > >>>>> >>> > >>>> >>> > >>>> I considered something similar, too, and upon another look, maybe >>> I >>> > >>>> dismissed it too quickly. The concern I had was that an >>> > implementation that >>> > >>>> simply updated the internal context.Context of an existing >>> > statefun.Context >>> > >>>> would violate the assumption that contexts are immutable. >>> However, if >>> > the >>> > >>>> implementation copied all the statefun.Context parts to a new >>> stateful >>> > >>>> context structure and associated them with the passed-in >>> > context.Context, >>> > >>>> that seems like it could work. There's a sync.Mutex in the >>> statefun >>> > context >>> > >>>> structure that we'd have to be careful about. >>> > >>>> >>> > >>>> >>> > >>>>> What does "SomeOtherFunc" need with the statefun context? >>> > >>>>> >>> > >>>> >>> > >>>> I think it's hard to answer this question, in a general sense. >>> > >>>> Depending on what is being done, it might need to read a value >>> from >>> > >>>> statefun Storage, write one back, etc. >>> > >>>> >>> > >>>> The solution that Till proposed seems to fit the example you gave >>> > >>>>> quite well, no? >>> > >>>>> >>> > >>>> >>> > >>>> Yes, this would work, but I agree with Till that this is not a >>> perfect >>> > >>>> solution. In the event that downstream code needs to access both >>> the >>> > >>>> context.Context and the statefun.Context, one would be passing two >>> > contexts >>> > >>>> around, but it would be important not to use the statefun.Context >>> one >>> > for >>> > >>>> any context values. That's workable, but it seems a bit clumsy to >>> me. >>> > >>>> >>> > >>>> On Tue, Feb 22, 2022 at 12:47 PM Austin Cawley-Edwards < >>> > >>>> austin.caw...@gmail.com> wrote: >>> > >>>> >>> > >>>>> Hey, >>> > >>>>> >>> > >>>>> Sorry for the late response – been off the ML for a few days. >>> > >>>>> >>> > >>>>> I am not too familiar with other Go libs that use a custom >>> context. >>> > >>>>> One place we could look is the `net/http` Request, which has a >>> > >>>>> `WithContext` method[1] that seems to expose the behavior we're >>> > looking >>> > >>>>> for. That could be added to the statefun context package as a >>> > standalone >>> > >>>>> method (e.g. statefun.WithContext(sCtx Context, ctx >>> > context.Context)), but >>> > >>>>> would only work for the private implementation. I think the >>> statefun >>> > >>>>> Context type being an interface instead of a concrete type >>> > complicates and >>> > >>>>> restricts us a bit here. >>> > >>>>> >>> > >>>>> I guess I am not fully understanding why the statefun Context >>> needs >>> > to >>> > >>>>> be used so far down the line. The solution that Till proposed >>> seems >>> > to fit >>> > >>>>> the example you gave quite well, no? >>> > >>>>> >>> > >>>>> func (f *MyFunc) Invoke(ctx statefun.Context, message >>> > >>>>> statefun.Message) error { >>> > >>>>> logger := NewLogger() >>> > >>>>> downCtx := context.WithValue(ctx, "logger", logger) >>> > >>>>> return SomeOtherFunc(downCtx) >>> > >>>>> } >>> > >>>>> >>> > >>>>> func SomeOtherFunc(ctx context.Context) error { >>> > >>>>> logger := ctx.Value("logger") >>> > >>>>> return nil >>> > >>>>> } >>> > >>>>> >>> > >>>>> What does "SomeOtherFunc" need with the statefun context? I think >>> > that >>> > >>>>> would help me, at least, understand the role of the statefun >>> context. >>> > >>>>> >>> > >>>>> I'm curious what you would think about an approach that kept >>> > >>>>>> everything as-is, by default, but allowed for a separated >>> context >>> > and >>> > >>>>>> runtime in the Invoke method, on an opt-in basis, via an >>> adapter? >>> > >>>>>> >>> > >>>>> >>> > >>>>> I am not involved in statefun really, but IMO that seems like >>> quite a >>> > >>>>> lot of overhead to just pass values via the context. Perhaps we >>> > should >>> > >>>>> consider decomposing the statefun context itself so pieces of >>> > functionality >>> > >>>>> can be passed around more easily? >>> > >>>>> >>> > >>>>> Best, >>> > >>>>> Austin >>> > >>>>> >>> > >>>>> >>> > >>>>> [1]: https://pkg.go.dev/net/http#Request.WithContext >>> > >>>>> >>> > >>>>> >>> > >>>>> On Tue, Feb 22, 2022 at 10:51 AM Galen Warren < >>> > ga...@cvillewarrens.com> >>> > >>>>> wrote: >>> > >>>>> >>> > >>>>>> Thanks, Seth. >>> > >>>>>> >>> > >>>>>> I'm curious what you would think about an approach that kept >>> > >>>>>> everything as-is, by default, but allowed for a separated >>> context >>> > and >>> > >>>>>> runtime in the Invoke method, on an opt-in basis, via an >>> adapter? >>> > >>>>>> >>> > >>>>>> On Tue, Feb 22, 2022 at 10:28 AM Seth Wiesman < >>> sjwies...@gmail.com> >>> > >>>>>> wrote: >>> > >>>>>> >>> > >>>>>>> Hi all, >>> > >>>>>>> >>> > >>>>>>> I believe the discussion revolved around: >>> > >>>>>>> >>> > >>>>>>> 1. fewer parameters >>> > >>>>>>> 2. better aligned with other language sdks >>> > >>>>>>> 3. we found precedent in other libraries (apologies this was >>> long >>> > >>>>>>> enough >>> > >>>>>>> ago I cannot remember which ones, I'm looking through old >>> > >>>>>>> discussions now) >>> > >>>>>>> >>> > >>>>>>> I would in general champion a solution that keeps the SDKs >>> looking >>> > >>>>>>> similar >>> > >>>>>>> across languages. A big part of statefun's positioning in the >>> > market >>> > >>>>>>> is the >>> > >>>>>>> polyglot nature and making the transition between languages as >>> > >>>>>>> seamless as >>> > >>>>>>> possible is very important. >>> > >>>>>>> >>> > >>>>>>> Seth >>> > >>>>>>> >>> > >>>>>>> >>> > >>>>>>> On Tue, Feb 22, 2022 at 4:33 AM Till Rohrmann < >>> > trohrm...@apache.org> >>> > >>>>>>> wrote: >>> > >>>>>>> >>> > >>>>>>> > Hi Galen, >>> > >>>>>>> > >>> > >>>>>>> > Thanks for explaining the problems with the current design. I >>> > >>>>>>> think I've >>> > >>>>>>> > already learned quite a bit wrt Go thanks to you :-) >>> > >>>>>>> > >>> > >>>>>>> > From what you describe it seems indeed a bit restrictive to >>> let >>> > the >>> > >>>>>>> > statefun.Context contain the context.Context w/o giving >>> access to >>> > >>>>>>> it. Maybe @Seth >>> > >>>>>>> > Wiesman <sjwies...@gmail.com> can elaborate a bit more on >>> the >>> > >>>>>>> design >>> > >>>>>>> > decisions to make sure that we have the full picture. >>> > >>>>>>> > >>> > >>>>>>> > As a cheap workaround you could create a context.Context >>> object >>> > by >>> > >>>>>>> calling >>> > >>>>>>> > >>> > >>>>>>> >>> > https://github.com/knative/pkg/blob/d48172451966/logging/logger.go#L45 >>> > >>>>>>> on >>> > >>>>>>> > the statefun.Context and then pass this Context instance to >>> the >>> > >>>>>>> downstream >>> > >>>>>>> > function. But I agree that this is not the perfect solution. >>> > >>>>>>> > >>> > >>>>>>> > How do other libraries handle this situation if they offer a >>> > custom >>> > >>>>>>> > Context type? Maybe @Austin Cawley-Edwards < >>> > >>>>>>> austin.caw...@gmail.com> you >>> > >>>>>>> > have an opinion on the matter. >>> > >>>>>>> > >>> > >>>>>>> > Cheers, >>> > >>>>>>> > Till >>> > >>>>>>> > >>> > >>>>>>> > On Mon, Feb 21, 2022 at 7:42 PM Galen Warren < >>> > >>>>>>> ga...@cvillewarrens.com> >>> > >>>>>>> > wrote: >>> > >>>>>>> > >>> > >>>>>>> >> So, upon further fiddling, I think it would be possible to >>> keep >>> > >>>>>>> full >>> > >>>>>>> >> backward compatibility and add the option for someone to >>> use an >>> > >>>>>>> Invoke >>> > >>>>>>> >> method with a separate context.Context and >>> statefun.Runtime, via >>> > >>>>>>> an >>> > >>>>>>> >> adapter, if direct manipulation of the context.Context is >>> > needed. >>> > >>>>>>> So, >>> > >>>>>>> >> basically, the idea would be to let the user choose the >>> form of >>> > >>>>>>> the Invoke >>> > >>>>>>> >> method, with the default behavior remaining the same as now. >>> > >>>>>>> >> >>> > >>>>>>> >> This would require: >>> > >>>>>>> >> >>> > >>>>>>> >> - Recreating the Runtime interface (all methods currently >>> > >>>>>>> defined on >>> > >>>>>>> >> Context except not embedding context.Context) and >>> embedding >>> > it >>> > >>>>>>> in the >>> > >>>>>>> >> statefun.Context interface, so that statefun.Context >>> remains >>> > >>>>>>> >> effectively >>> > >>>>>>> >> unchanged >>> > >>>>>>> >> - Add StatefulFunctionV2 and StatefunFunctionV2Pointer to >>> > >>>>>>> support the >>> > >>>>>>> >> new signature with separate context and runtime >>> > >>>>>>> >> - Add StatefulFunctionV2Adapter to wrap a >>> StatefulFunctionV2 >>> > >>>>>>> and expose >>> > >>>>>>> >> it as a StatefulFunction. The statefun.Context would get >>> > split >>> > >>>>>>> into a >>> > >>>>>>> >> context.Context and a statefun.Runtime here in order to >>> call >>> > >>>>>>> the new >>> > >>>>>>> >> signature. >>> > >>>>>>> >> >>> > >>>>>>> >> Thoughts? I'd be happy to take a crack at it. >>> > >>>>>>> >> >>> > >>>>>>> >> >>> > >>>>>>> >> On Mon, Feb 21, 2022 at 12:06 PM Galen Warren < >>> > >>>>>>> ga...@cvillewarrens.com> >>> > >>>>>>> >> wrote: >>> > >>>>>>> >> >>> > >>>>>>> >> > Was the reason to combine them the desire to have two >>> > >>>>>>> parameters vs. >>> > >>>>>>> >> > three, or was there another motivation? >>> > >>>>>>> >> > >>> > >>>>>>> >> > On Mon, Feb 21, 2022 at 12:02 PM Seth Wiesman < >>> > >>>>>>> sjwies...@gmail.com> >>> > >>>>>>> >> wrote: >>> > >>>>>>> >> > >>> > >>>>>>> >> >> FWIW I received a lot of early feedback explicitly >>> asking me >>> > >>>>>>> to couple >>> > >>>>>>> >> the >>> > >>>>>>> >> >> statefun specific operations with the Context (why the >>> > runtime >>> > >>>>>>> type >>> > >>>>>>> >> went >>> > >>>>>>> >> >> away). >>> > >>>>>>> >> >> >>> > >>>>>>> >> >> Seth >>> > >>>>>>> >> >> >>> > >>>>>>> >> >> On Mon, Feb 21, 2022 at 10:32 AM Galen Warren < >>> > >>>>>>> ga...@cvillewarrens.com >>> > >>>>>>> >> > >>> > >>>>>>> >> >> wrote: >>> > >>>>>>> >> >> >>> > >>>>>>> >> >> > Thanks for looking into this! >>> > >>>>>>> >> >> > >>> > >>>>>>> >> >> > The issue I think we'd run into with your proposal is >>> that, >>> > >>>>>>> often, >>> > >>>>>>> >> >> > libraries use non-exported types for context keys. >>> Here is >>> > >>>>>>> an example >>> > >>>>>>> >> >> > < >>> > >>>>>>> >> >>> > >>>>>>> >>> > https://github.com/knative/pkg/blob/d48172451966/logging/logger.go#L45 >>> > >>>>>>> >> >> >; >>> > >>>>>>> >> >> > in this case, the non-exported loggerKey{} is used as >>> the >>> > >>>>>>> key, inside >>> > >>>>>>> >> >> the >>> > >>>>>>> >> >> > exported WithLogger function. The key that would have >>> to be >>> > >>>>>>> supplied >>> > >>>>>>> >> to >>> > >>>>>>> >> >> the >>> > >>>>>>> >> >> > proposed Value and WithValue functions would not be >>> > >>>>>>> accessible in >>> > >>>>>>> >> this >>> > >>>>>>> >> >> > case. >>> > >>>>>>> >> >> > >>> > >>>>>>> >> >> > Honestly, if *everything *were on the table -- and >>> > >>>>>>> understand it very >>> > >>>>>>> >> >> well >>> > >>>>>>> >> >> > might not be -- I'd suggest decoupling the Golang >>> > >>>>>>> context.Context and >>> > >>>>>>> >> >> the >>> > >>>>>>> >> >> > statefun Context, i.e. have two separate parameters to >>> > >>>>>>> >> >> > StatefulFunction.Invoke representing Golang context and >>> > >>>>>>> statefun >>> > >>>>>>> >> >> > operations. This is actually how things were in an >>> earlier >>> > >>>>>>> version of >>> > >>>>>>> >> >> the >>> > >>>>>>> >> >> > Golang SDK; the first parameter to Invoke was the >>> > >>>>>>> plain-vanilla >>> > >>>>>>> >> >> > context.Context and a separate parameter provided the >>> > >>>>>>> statefun >>> > >>>>>>> >> >> "runtime". >>> > >>>>>>> >> >> > So maybe something like this: >>> > >>>>>>> >> >> > >>> > >>>>>>> >> >> > > >>> > >>>>>>> >> >> > > type StatefulFunction interface { >>> > >>>>>>> >> >> > > Invoke(ctx context.Context, runtime Runtime, message >>> > >>>>>>> Message) error >>> > >>>>>>> >> >> > > } >>> > >>>>>>> >> >> > >>> > >>>>>>> >> >> > >>> > >>>>>>> >> >> > ... instead of the current: >>> > >>>>>>> >> >> > >>> > >>>>>>> >> >> > type StatefulFunction interface { >>> > >>>>>>> >> >> > > Invoke(ctx Context, message Message) error >>> > >>>>>>> >> >> > > } >>> > >>>>>>> >> >> > >>> > >>>>>>> >> >> > >>> > >>>>>>> >> >> > ... where Runtime would be everything currently in >>> > >>>>>>> statefun.Context, >>> > >>>>>>> >> >> except >>> > >>>>>>> >> >> > the context.Context part. This would allow >>> context.Context >>> > >>>>>>> to be >>> > >>>>>>> >> >> > manipulated and passed around normally. >>> > >>>>>>> >> >> > >>> > >>>>>>> >> >> > I think this could potentially be done in a >>> > >>>>>>> backward-compatible way, >>> > >>>>>>> >> >> with a >>> > >>>>>>> >> >> > new set of types and methods, e.g. StatefulFunctionV2, >>> > >>>>>>> >> >> > StatefufFunctionSpecV2, StatefulFunctions.WithSpecV2, >>> etc. >>> > >>>>>>> Or it >>> > >>>>>>> >> could >>> > >>>>>>> >> >> be >>> > >>>>>>> >> >> > done in an almost backward-compatible way, by changing >>> the >>> > >>>>>>> existing >>> > >>>>>>> >> >> > StatefulFunction, StatefulFunctionSpec, >>> > >>>>>>> StatefulFunctions.WithSpec >>> > >>>>>>> >> and >>> > >>>>>>> >> >> > providing an adapter for people who want to continue >>> to use >>> > >>>>>>> the >>> > >>>>>>> >> >> > two-parameter version of Invoke. >>> > >>>>>>> >> >> > >>> > >>>>>>> >> >> > If those kinds of changes are a non-starter, then IMO >>> the >>> > >>>>>>> next best >>> > >>>>>>> >> >> option >>> > >>>>>>> >> >> > would be adding something like: >>> > >>>>>>> >> >> > >>> > >>>>>>> >> >> > PrepareContext func(ctx statefun.Context) >>> context.Context >>> > >>>>>>> >> >> > >>> > >>>>>>> >> >> > >>> > >>>>>>> >> >> > ... to StatefulFunctionSpec to allow a one-time >>> > >>>>>>> customization of the >>> > >>>>>>> >> >> > underlying context at the beginning of a stateful >>> function >>> > >>>>>>> >> invocation. >>> > >>>>>>> >> >> That >>> > >>>>>>> >> >> > would cover a lot of use cases. >>> > >>>>>>> >> >> > >>> > >>>>>>> >> >> > >>> > >>>>>>> >> >> > On Mon, Feb 21, 2022 at 3:06 AM Till Rohrmann < >>> > >>>>>>> trohrm...@apache.org> >>> > >>>>>>> >> >> > wrote: >>> > >>>>>>> >> >> > >>> > >>>>>>> >> >> > > Thanks a lot for clarifying the problem. I think I >>> now >>> > >>>>>>> understand >>> > >>>>>>> >> the >>> > >>>>>>> >> >> > > problem. As you've probably figured out, I have no >>> clue >>> > >>>>>>> about Go >>> > >>>>>>> >> and >>> > >>>>>>> >> >> > > its usage of the Context type. >>> > >>>>>>> >> >> > > >>> > >>>>>>> >> >> > > After looking into it a bit I was wondering whether >>> we >>> > >>>>>>> can't >>> > >>>>>>> >> follow a >>> > >>>>>>> >> >> > > similar route as it is done for the Context type. By >>> > adding >>> > >>>>>>> >> something >>> > >>>>>>> >> >> > like >>> > >>>>>>> >> >> > > >>> > >>>>>>> >> >> > > type valueCtx struct { >>> > >>>>>>> >> >> > > Context >>> > >>>>>>> >> >> > > key, val interface{} >>> > >>>>>>> >> >> > > } >>> > >>>>>>> >> >> > > >>> > >>>>>>> >> >> > > func (c *valueCtx) Value(key interface{}) >>> interface{} { >>> > >>>>>>> >> >> > > if c.key == key { >>> > >>>>>>> >> >> > > return c.val >>> > >>>>>>> >> >> > > } >>> > >>>>>>> >> >> > > return c.Context.Value(key) >>> > >>>>>>> >> >> > > } >>> > >>>>>>> >> >> > > >>> > >>>>>>> >> >> > > func WithValue(parent Context, key, val interface{}) >>> > >>>>>>> Context { >>> > >>>>>>> >> >> > > if parent == nil { >>> > >>>>>>> >> >> > > panic("cannot create context from nil parent") >>> > >>>>>>> >> >> > > } >>> > >>>>>>> >> >> > > if key == nil { >>> > >>>>>>> >> >> > > panic("nil key") >>> > >>>>>>> >> >> > > } >>> > >>>>>>> >> >> > > return &valueCtx{parent, key, val} >>> > >>>>>>> >> >> > > } >>> > >>>>>>> >> >> > > >>> > >>>>>>> >> >> > > to the statefun/context.go we would allow to extend a >>> > >>>>>>> Statefun >>> > >>>>>>> >> context >>> > >>>>>>> >> >> > with >>> > >>>>>>> >> >> > > values w/o changing the underlying instance. If >>> > >>>>>>> statefun.Context is >>> > >>>>>>> >> >> not >>> > >>>>>>> >> >> > > needed, then there is already the option to unwrap >>> the >>> > >>>>>>> >> context.Context >>> > >>>>>>> >> >> > and >>> > >>>>>>> >> >> > > to extend it with values and then pass on this >>> instance. >>> > >>>>>>> But maybe >>> > >>>>>>> >> >> this >>> > >>>>>>> >> >> > is >>> > >>>>>>> >> >> > > no idiomatic Go. Let me know what you think. >>> > >>>>>>> >> >> > > >>> > >>>>>>> >> >> > > Cheers, >>> > >>>>>>> >> >> > > Till >>> > >>>>>>> >> >> > > >>> > >>>>>>> >> >> > > On Fri, Feb 18, 2022 at 7:01 PM Galen Warren < >>> > >>>>>>> >> ga...@cvillewarrens.com >>> > >>>>>>> >> >> > >>> > >>>>>>> >> >> > > wrote: >>> > >>>>>>> >> >> > > >>> > >>>>>>> >> >> > > > Hmm ... a downside to my proposal is that Go >>> contexts >>> > are >>> > >>>>>>> >> supposed >>> > >>>>>>> >> >> to >>> > >>>>>>> >> >> > be >>> > >>>>>>> >> >> > > > immutable, i.e. when adding a custom value to a >>> > context, >>> > >>>>>>> a new >>> > >>>>>>> >> >> context >>> > >>>>>>> >> >> > is >>> > >>>>>>> >> >> > > > created with the new value and the old context >>> isn't >>> > >>>>>>> changed. >>> > >>>>>>> >> >> Changing >>> > >>>>>>> >> >> > > the >>> > >>>>>>> >> >> > > > context.Context associated with the >>> statefun.Context >>> > >>>>>>> sort of goes >>> > >>>>>>> >> >> > against >>> > >>>>>>> >> >> > > > the spirit of that, i.e. a consumer of >>> statefun.Context >>> > >>>>>>> could see >>> > >>>>>>> >> >> > custom >>> > >>>>>>> >> >> > > > values change unexpectedly if another consumer of >>> the >>> > >>>>>>> same >>> > >>>>>>> >> >> > > statefun.Context >>> > >>>>>>> >> >> > > > modified the underlying context.Context. >>> > >>>>>>> >> >> > > > >>> > >>>>>>> >> >> > > > To avoid that, I think we'd be back to having some >>> > >>>>>>> mechanism to >>> > >>>>>>> >> >> > customize >>> > >>>>>>> >> >> > > > the underlying context.Context once, when the >>> > >>>>>>> statefun.Context is >>> > >>>>>>> >> >> > created >>> > >>>>>>> >> >> > > > at the beginning of a stateful function invocation. >>> > >>>>>>> Adding a >>> > >>>>>>> >> field >>> > >>>>>>> >> >> > like: >>> > >>>>>>> >> >> > > > >>> > >>>>>>> >> >> > > > PrepareContext func(ctx statefun.Context) >>> > context.Context >>> > >>>>>>> >> >> > > > >>> > >>>>>>> >> >> > > > ... to the StatefulFunctionSpec struct could >>> accomplish >>> > >>>>>>> that, >>> > >>>>>>> >> i.e. >>> > >>>>>>> >> >> if >>> > >>>>>>> >> >> > > > PrepareContext were supplied, the context could be >>> > >>>>>>> customized >>> > >>>>>>> >> once >>> > >>>>>>> >> >> at >>> > >>>>>>> >> >> > the >>> > >>>>>>> >> >> > > > start of a function invocation and then left >>> immutable >>> > >>>>>>> after that >>> > >>>>>>> >> >> > point. >>> > >>>>>>> >> >> > > > >>> > >>>>>>> >> >> > > > (Using statefun.Context as the input is deliberate >>> > here, >>> > >>>>>>> in >>> > >>>>>>> >> order to >>> > >>>>>>> >> >> > > allow >>> > >>>>>>> >> >> > > > the context.Context to be populated using values >>> from >>> > the >>> > >>>>>>> >> >> > > statefun.Context, >>> > >>>>>>> >> >> > > > for example the function id). >>> > >>>>>>> >> >> > > > >>> > >>>>>>> >> >> > > > >>> > >>>>>>> >> >> > > > >>> > >>>>>>> >> >> > > > >>> > >>>>>>> >> >> > > > >>> > >>>>>>> >> >> > > > >>> > >>>>>>> >> >> > > > On Fri, Feb 18, 2022 at 11:34 AM Galen Warren < >>> > >>>>>>> >> >> ga...@cvillewarrens.com >>> > >>>>>>> >> >> > > >>> > >>>>>>> >> >> > > > wrote: >>> > >>>>>>> >> >> > > > >>> > >>>>>>> >> >> > > > > An example of passing it around would be: >>> > >>>>>>> >> >> > > > > >>> > >>>>>>> >> >> > > > > func (f *MyFunc) Invoke(ctx statefun.Context, >>> message >>> > >>>>>>> >> >> > statefun.Message) >>> > >>>>>>> >> >> > > > > error { >>> > >>>>>>> >> >> > > > > >>> > >>>>>>> >> >> > > > > logger := NewLogger() >>> > >>>>>>> >> >> > > > > ctx.SetContext(ctxzap.ToContext(ctx, logger)) >>> > >>>>>>> >> >> > > > > >>> > >>>>>>> >> >> > > > > return SomeOtherFunc(ctx) >>> > >>>>>>> >> >> > > > > } >>> > >>>>>>> >> >> > > > > >>> > >>>>>>> >> >> > > > > func SomeOtherFunc(ctx context.Context) error { >>> > >>>>>>> >> >> > > > > >>> > >>>>>>> >> >> > > > > logger := ctxzap.Extract(ctx) >>> > >>>>>>> >> >> > > > > logger.Info(...) >>> > >>>>>>> >> >> > > > > >>> > >>>>>>> >> >> > > > > return nil >>> > >>>>>>> >> >> > > > > } >>> > >>>>>>> >> >> > > > > >>> > >>>>>>> >> >> > > > > This would also work with further nested calls, >>> so >>> > >>>>>>> long as the >>> > >>>>>>> >> >> > context >>> > >>>>>>> >> >> > > is >>> > >>>>>>> >> >> > > > > passed to them. >>> > >>>>>>> >> >> > > > > >>> > >>>>>>> >> >> > > > > On Fri, Feb 18, 2022 at 11:23 AM Galen Warren < >>> > >>>>>>> >> >> > ga...@cvillewarrens.com >>> > >>>>>>> >> >> > > > >>> > >>>>>>> >> >> > > > > wrote: >>> > >>>>>>> >> >> > > > > >>> > >>>>>>> >> >> > > > >> Ha, our emails keep passing. >>> > >>>>>>> >> >> > > > >> >>> > >>>>>>> >> >> > > > >> I've been playing around with options locally, >>> and >>> > the >>> > >>>>>>> >> SetContext >>> > >>>>>>> >> >> > > option >>> > >>>>>>> >> >> > > > >> seems to be the most flexible (and >>> non-breaking), >>> > imo. >>> > >>>>>>> >> >> > > > >> >>> > >>>>>>> >> >> > > > >> The implementation would be trivial, just add: >>> > >>>>>>> >> >> > > > >> >>> > >>>>>>> >> >> > > > >> SetContext(ctx context.Context) >>> > >>>>>>> >> >> > > > >> >>> > >>>>>>> >> >> > > > >> ... to the statefun.Context interface, which is >>> > >>>>>>> implemented >>> > >>>>>>> >> as: >>> > >>>>>>> >> >> > > > >> >>> > >>>>>>> >> >> > > > >> func (s *statefunContext) SetContext(ctx >>> > >>>>>>> context.Context) { >>> > >>>>>>> >> >> > > > >> s.Context = ctx >>> > >>>>>>> >> >> > > > >> } >>> > >>>>>>> >> >> > > > >> >>> > >>>>>>> >> >> > > > >> >>> > >>>>>>> >> >> > > > >> >>> > >>>>>>> >> >> > > > >> >>> > >>>>>>> >> >> > > > >> On Fri, Feb 18, 2022 at 11:18 AM Austin >>> > >>>>>>> Cawley-Edwards < >>> > >>>>>>> >> >> > > > >> austin.caw...@gmail.com> wrote: >>> > >>>>>>> >> >> > > > >> >>> > >>>>>>> >> >> > > > >>> It would be helpful to have a small example >>> though, >>> > >>>>>>> if you >>> > >>>>>>> >> have >>> > >>>>>>> >> >> on >>> > >>>>>>> >> >> > > > Galen, >>> > >>>>>>> >> >> > > > >>> to see how you're passing it around. >>> > >>>>>>> >> >> > > > >>> >>> > >>>>>>> >> >> > > > >>> On Fri, Feb 18, 2022 at 11:10 AM Austin >>> > >>>>>>> Cawley-Edwards < >>> > >>>>>>> >> >> > > > >>> austin.caw...@gmail.com> wrote: >>> > >>>>>>> >> >> > > > >>> >>> > >>>>>>> >> >> > > > >>> > Looking through the statefun Context >>> interface, >>> > it >>> > >>>>>>> indeed >>> > >>>>>>> >> >> doesn't >>> > >>>>>>> >> >> > > > give >>> > >>>>>>> >> >> > > > >>> > access to the underlying context.Context and >>> the >>> > >>>>>>> only >>> > >>>>>>> >> >> > > implementation >>> > >>>>>>> >> >> > > > is >>> > >>>>>>> >> >> > > > >>> > package-private [1]. I don't think there >>> would be >>> > >>>>>>> a way to >>> > >>>>>>> >> >> update >>> > >>>>>>> >> >> > > the >>> > >>>>>>> >> >> > > > >>> > statfun.Context interface without introducing >>> > >>>>>>> breaking >>> > >>>>>>> >> >> changes, >>> > >>>>>>> >> >> > but >>> > >>>>>>> >> >> > > > if >>> > >>>>>>> >> >> > > > >>> we >>> > >>>>>>> >> >> > > > >>> > were to make that implementation public, that >>> > >>>>>>> might be a >>> > >>>>>>> >> >> stopgap >>> > >>>>>>> >> >> > > > >>> solution. >>> > >>>>>>> >> >> > > > >>> > e.g., >>> > >>>>>>> >> >> > > > >>> > >>> > >>>>>>> >> >> > > > >>> > ``` >>> > >>>>>>> >> >> > > > >>> > type StatefunContext struct { >>> > >>>>>>> >> >> > > > >>> > // expose embedded context >>> > >>>>>>> >> >> > > > >>> > context.Context >>> > >>>>>>> >> >> > > > >>> > >>> > >>>>>>> >> >> > > > >>> > // make the mutext private >>> > >>>>>>> >> >> > > > >>> > mu sync.Mutex >>> > >>>>>>> >> >> > > > >>> > >>> > >>>>>>> >> >> > > > >>> > // keep internals private >>> > >>>>>>> >> >> > > > >>> > self Address >>> > >>>>>>> >> >> > > > >>> > caller *Address >>> > >>>>>>> >> >> > > > >>> > storage *storage >>> > >>>>>>> >> >> > > > >>> > response >>> > *protocol.FromFunction_InvocationResponse >>> > >>>>>>> >> >> > > > >>> > } >>> > >>>>>>> >> >> > > > >>> > ``` >>> > >>>>>>> >> >> > > > >>> > >>> > >>>>>>> >> >> > > > >>> > You could then do a type assertion in the >>> > handlers >>> > >>>>>>> for this >>> > >>>>>>> >> >> type >>> > >>>>>>> >> >> > of >>> > >>>>>>> >> >> > > > >>> > context, and modify the context on it >>> directly. >>> > It >>> > >>>>>>> would >>> > >>>>>>> >> be a >>> > >>>>>>> >> >> bit >>> > >>>>>>> >> >> > > > >>> ugly, but >>> > >>>>>>> >> >> > > > >>> > may work. >>> > >>>>>>> >> >> > > > >>> > >>> > >>>>>>> >> >> > > > >>> > ``` >>> > >>>>>>> >> >> > > > >>> > func (s aFunc) Invoke(ctx Context, message >>> > >>>>>>> Message) error { >>> > >>>>>>> >> >> > > > >>> > if sCtx, ok := >>> ctx.(*statefun.StatefunContext); >>> > >>>>>>> ok { >>> > >>>>>>> >> >> > > > >>> > sCtx.Context = >>> > context.WithValue(sCtx.Context, >>> > >>>>>>> >> "logger", >>> > >>>>>>> >> >> > > aLogger) >>> > >>>>>>> >> >> > > > >>> > } >>> > >>>>>>> >> >> > > > >>> > // ... >>> > >>>>>>> >> >> > > > >>> > } >>> > >>>>>>> >> >> > > > >>> > ``` >>> > >>>>>>> >> >> > > > >>> > >>> > >>>>>>> >> >> > > > >>> > Let me know what you all think, >>> > >>>>>>> >> >> > > > >>> > Austin >>> > >>>>>>> >> >> > > > >>> > >>> > >>>>>>> >> >> > > > >>> > >>> > >>>>>>> >> >> > > > >>> > [1]: >>> > >>>>>>> >> >> > > > >>> > >>> > >>>>>>> >> >> > > > >>> >>> > >>>>>>> >> >> > > > >>> > >>>>>>> >> >> > > >>> > >>>>>>> >> >> > >>> > >>>>>>> >> >> >>> > >>>>>>> >> >>> > >>>>>>> >>> > >>> https://github.com/apache/flink-statefun/blob/1dfe226d85fea05a46c8ffa688175b4c0f2d4900/statefun-sdk-go/v3/pkg/statefun/context.go#L66-L73 >>> > >>>>>>> >> >> > > > >>> > >>> > >>>>>>> >> >> > > > >>> > >>> > >>>>>>> >> >> > > > >>> > On Fri, Feb 18, 2022 at 11:03 AM Galen >>> Warren < >>> > >>>>>>> >> >> > > > ga...@cvillewarrens.com >>> > >>>>>>> >> >> > > > >>> > >>> > >>>>>>> >> >> > > > >>> > wrote: >>> > >>>>>>> >> >> > > > >>> > >>> > >>>>>>> >> >> > > > >>> >> Sorry Austin, I didn't see your response >>> before >>> > I >>> > >>>>>>> replied. >>> > >>>>>>> >> >> Yes, >>> > >>>>>>> >> >> > > > we're >>> > >>>>>>> >> >> > > > >>> >> saying the same thing. >>> > >>>>>>> >> >> > > > >>> >> >>> > >>>>>>> >> >> > > > >>> >> On Fri, Feb 18, 2022 at 10:56 AM Austin >>> > >>>>>>> Cawley-Edwards < >>> > >>>>>>> >> >> > > > >>> >> austin.caw...@gmail.com> wrote: >>> > >>>>>>> >> >> > > > >>> >> >>> > >>>>>>> >> >> > > > >>> >> > Hey all, jumping in. This makes sense to >>> me – >>> > >>>>>>> for >>> > >>>>>>> >> instance >>> > >>>>>>> >> >> to >>> > >>>>>>> >> >> > > > >>> attach a >>> > >>>>>>> >> >> > > > >>> >> > logger with some common metadata, e.g >>> trace ID >>> > >>>>>>> for the >>> > >>>>>>> >> >> > request? >>> > >>>>>>> >> >> > > > >>> This is >>> > >>>>>>> >> >> > > > >>> >> > common in go to add arbitrary items >>> without >>> > >>>>>>> updating the >>> > >>>>>>> >> >> > method >>> > >>>>>>> >> >> > > > >>> >> signatures, >>> > >>>>>>> >> >> > > > >>> >> > similar to thread local storage in Java. >>> > >>>>>>> >> >> > > > >>> >> > >>> > >>>>>>> >> >> > > > >>> >> > On Fri, Feb 18, 2022 at 10:53 AM Till >>> > Rohrmann < >>> > >>>>>>> >> >> > > > >>> trohrm...@apache.org> >>> > >>>>>>> >> >> > > > >>> >> > wrote: >>> > >>>>>>> >> >> > > > >>> >> > >>> > >>>>>>> >> >> > > > >>> >> > > Thanks for the clarification Galen. If >>> you >>> > >>>>>>> call the >>> > >>>>>>> >> >> other Go >>> > >>>>>>> >> >> > > > >>> >> functions, >>> > >>>>>>> >> >> > > > >>> >> > > then you could also pass the other >>> values as >>> > >>>>>>> separate >>> > >>>>>>> >> >> > > arguments >>> > >>>>>>> >> >> > > > to >>> > >>>>>>> >> >> > > > >>> >> these >>> > >>>>>>> >> >> > > > >>> >> > > functions, can't you? >>> > >>>>>>> >> >> > > > >>> >> > > >>> > >>>>>>> >> >> > > > >>> >> > > Cheers, >>> > >>>>>>> >> >> > > > >>> >> > > Till >>> > >>>>>>> >> >> > > > >>> >> > > >>> > >>>>>>> >> >> > > > >>> >> > > On Fri, Feb 18, 2022 at 3:31 PM Galen >>> > Warren < >>> > >>>>>>> >> >> > > > >>> ga...@cvillewarrens.com >>> > >>>>>>> >> >> > > > >>> >> > >>> > >>>>>>> >> >> > > > >>> >> > > wrote: >>> > >>>>>>> >> >> > > > >>> >> > > >>> > >>>>>>> >> >> > > > >>> >> > > > The former. >>> > >>>>>>> >> >> > > > >>> >> > > > >>> > >>>>>>> >> >> > > > >>> >> > > > I think there's potential for >>> confusion >>> > >>>>>>> here because >>> > >>>>>>> >> >> we're >>> > >>>>>>> >> >> > > > >>> using the >>> > >>>>>>> >> >> > > > >>> >> > > > word *function >>> > >>>>>>> >> >> > > > >>> >> > > > *in a couple of senses. One sense is a >>> > >>>>>>> *stateful >>> > >>>>>>> >> >> > function*; >>> > >>>>>>> >> >> > > > >>> another >>> > >>>>>>> >> >> > > > >>> >> > sense >>> > >>>>>>> >> >> > > > >>> >> > > > is a *Go function*. >>> > >>>>>>> >> >> > > > >>> >> > > > >>> > >>>>>>> >> >> > > > >>> >> > > > What I'm looking to do is to put >>> values in >>> > >>>>>>> the >>> > >>>>>>> >> Context >>> > >>>>>>> >> >> so >>> > >>>>>>> >> >> > > that >>> > >>>>>>> >> >> > > > >>> >> > downstream >>> > >>>>>>> >> >> > > > >>> >> > > > Go functions that receive the context >>> can >>> > >>>>>>> access >>> > >>>>>>> >> those >>> > >>>>>>> >> >> > > values. >>> > >>>>>>> >> >> > > > >>> Those >>> > >>>>>>> >> >> > > > >>> >> > > > downstream Go functions would be >>> called >>> > >>>>>>> during one >>> > >>>>>>> >> >> > > invocation >>> > >>>>>>> >> >> > > > >>> of the >>> > >>>>>>> >> >> > > > >>> >> > > > stateful function. >>> > >>>>>>> >> >> > > > >>> >> > > > >>> > >>>>>>> >> >> > > > >>> >> > > > On Fri, Feb 18, 2022 at 6:48 AM Till >>> > >>>>>>> Rohrmann < >>> > >>>>>>> >> >> > > > >>> trohrm...@apache.org >>> > >>>>>>> >> >> > > > >>> >> > >>> > >>>>>>> >> >> > > > >>> >> > > > wrote: >>> > >>>>>>> >> >> > > > >>> >> > > > >>> > >>>>>>> >> >> > > > >>> >> > > > > Hi Galen, >>> > >>>>>>> >> >> > > > >>> >> > > > > >>> > >>>>>>> >> >> > > > >>> >> > > > > Am I understanding it correctly, >>> that >>> > you >>> > >>>>>>> would >>> > >>>>>>> >> like >>> > >>>>>>> >> >> to >>> > >>>>>>> >> >> > > set >>> > >>>>>>> >> >> > > > >>> some >>> > >>>>>>> >> >> > > > >>> >> > values >>> > >>>>>>> >> >> > > > >>> >> > > > in >>> > >>>>>>> >> >> > > > >>> >> > > > > the Context of function A that is >>> then >>> > >>>>>>> accessible >>> > >>>>>>> >> in >>> > >>>>>>> >> >> a >>> > >>>>>>> >> >> > > > >>> downstream >>> > >>>>>>> >> >> > > > >>> >> > call >>> > >>>>>>> >> >> > > > >>> >> > > of >>> > >>>>>>> >> >> > > > >>> >> > > > > function B? Or would you like to >>> set a >>> > >>>>>>> value that >>> > >>>>>>> >> is >>> > >>>>>>> >> >> > > > >>> accessible >>> > >>>>>>> >> >> > > > >>> >> once >>> > >>>>>>> >> >> > > > >>> >> > > > > function A is called again (w/ or >>> w/o >>> > the >>> > >>>>>>> same >>> > >>>>>>> >> id)? >>> > >>>>>>> >> >> > > > >>> >> > > > > >>> > >>>>>>> >> >> > > > >>> >> > > > > Cheers, >>> > >>>>>>> >> >> > > > >>> >> > > > > Till >>> > >>>>>>> >> >> > > > >>> >> > > > > >>> > >>>>>>> >> >> > > > >>> >> > > > > On Thu, Feb 17, 2022 at 10:59 PM >>> Galen >>> > >>>>>>> Warren < >>> > >>>>>>> >> >> > > > >>> >> > ga...@cvillewarrens.com >>> > >>>>>>> >> >> > > > >>> >> > > > >>> > >>>>>>> >> >> > > > >>> >> > > > > wrote: >>> > >>>>>>> >> >> > > > >>> >> > > > > >>> > >>>>>>> >> >> > > > >>> >> > > > > > Also, a potentially simpler way to >>> > >>>>>>> support this >>> > >>>>>>> >> >> would >>> > >>>>>>> >> >> > be >>> > >>>>>>> >> >> > > > to >>> > >>>>>>> >> >> > > > >>> add >>> > >>>>>>> >> >> > > > >>> >> a >>> > >>>>>>> >> >> > > > >>> >> > > > > > SetContext method to the >>> > >>>>>>> statefun.Context >>> > >>>>>>> >> >> interface, >>> > >>>>>>> >> >> > and >>> > >>>>>>> >> >> > > > >>> have it >>> > >>>>>>> >> >> > > > >>> >> > > assign >>> > >>>>>>> >> >> > > > >>> >> > > > > the >>> > >>>>>>> >> >> > > > >>> >> > > > > > wrapped context. This would not >>> > require >>> > >>>>>>> changes >>> > >>>>>>> >> to >>> > >>>>>>> >> >> the >>> > >>>>>>> >> >> > > > >>> function >>> > >>>>>>> >> >> > > > >>> >> > spec, >>> > >>>>>>> >> >> > > > >>> >> > > > or >>> > >>>>>>> >> >> > > > >>> >> > > > > > anything else, and would be more >>> > >>>>>>> flexible. >>> > >>>>>>> >> >> > > > >>> >> > > > > > >>> > >>>>>>> >> >> > > > >>> >> > > > > > On Thu, Feb 17, 2022 at 1:05 PM >>> Galen >>> > >>>>>>> Warren < >>> > >>>>>>> >> >> > > > >>> >> > > ga...@cvillewarrens.com> >>> > >>>>>>> >> >> > > > >>> >> > > > > > wrote: >>> > >>>>>>> >> >> > > > >>> >> > > > > > >>> > >>>>>>> >> >> > > > >>> >> > > > > > > Thanks for the quick reply! >>> > >>>>>>> >> >> > > > >>> >> > > > > > > >>> > >>>>>>> >> >> > > > >>> >> > > > > > > What I'm trying to do is put >>> some >>> > >>>>>>> things into >>> > >>>>>>> >> the >>> > >>>>>>> >> >> > > > context >>> > >>>>>>> >> >> > > > >>> so >>> > >>>>>>> >> >> > > > >>> >> that >>> > >>>>>>> >> >> > > > >>> >> > > > > they're >>> > >>>>>>> >> >> > > > >>> >> > > > > > > available in downstream calls, >>> > >>>>>>> perhaps in >>> > >>>>>>> >> methods >>> > >>>>>>> >> >> > with >>> > >>>>>>> >> >> > > > >>> pointer >>> > >>>>>>> >> >> > > > >>> >> > > > > receivers >>> > >>>>>>> >> >> > > > >>> >> > > > > > to >>> > >>>>>>> >> >> > > > >>> >> > > > > > > the function struct (MyFunc) but >>> > also >>> > >>>>>>> perhaps >>> > >>>>>>> >> in >>> > >>>>>>> >> >> > > methods >>> > >>>>>>> >> >> > > > >>> that >>> > >>>>>>> >> >> > > > >>> >> are >>> > >>>>>>> >> >> > > > >>> >> > > > > further >>> > >>>>>>> >> >> > > > >>> >> > > > > > > downstream that don't have >>> access to >>> > >>>>>>> MyFunc. >>> > >>>>>>> >> If >>> > >>>>>>> >> >> I'm >>> > >>>>>>> >> >> > > > >>> >> understanding >>> > >>>>>>> >> >> > > > >>> >> > > > > > > correctly, your proposal would >>> work >>> > >>>>>>> for the >>> > >>>>>>> >> >> former >>> > >>>>>>> >> >> > but >>> > >>>>>>> >> >> > > > >>> not the >>> > >>>>>>> >> >> > > > >>> >> > > > latter. >>> > >>>>>>> >> >> > > > >>> >> > > > > > > >>> > >>>>>>> >> >> > > > >>> >> > > > > > > An example would be to put a >>> > >>>>>>> configured Logger >>> > >>>>>>> >> >> into >>> > >>>>>>> >> >> > > the >>> > >>>>>>> >> >> > > > >>> >> context >>> > >>>>>>> >> >> > > > >>> >> > > via a >>> > >>>>>>> >> >> > > > >>> >> > > > > > > WithLogger method (logging >>> package - >>> > >>>>>>> >> >> > > > >>> knative.dev/pkg/logging >>> > >>>>>>> >> >> > > > >>> >> - >>> > >>>>>>> >> >> > > > >>> >> > > > > > pkg.go.dev >>> > >>>>>>> >> >> > > > >>> >> > > > > > > < >>> > >>>>>>> >> >> > > >>> https://pkg.go.dev/knative.dev/pkg/logging#WithLogger >>> > >>>>>>> >> >> > > > >) >>> > >>>>>>> >> >> > > > >>> and >>> > >>>>>>> >> >> > > > >>> >> > then >>> > >>>>>>> >> >> > > > >>> >> > > > pull >>> > >>>>>>> >> >> > > > >>> >> > > > > > it >>> > >>>>>>> >> >> > > > >>> >> > > > > > > out downstream via FromContext >>> > >>>>>>> (logging >>> > >>>>>>> >> package - >>> > >>>>>>> >> >> > > > >>> >> > > > > > knative.dev/pkg/logging >>> > >>>>>>> >> >> > > > >>> >> > > > > > > - pkg.go.dev < >>> > >>>>>>> >> >> > > > >>> >> > > >>> > >>>>>>> >> https://pkg.go.dev/knative.dev/pkg/logging#FromContext >>> > >>>>>>> >> >> > > > >>> >> > > > > >). >>> > >>>>>>> >> >> > > > >>> >> > > > > > > >>> > >>>>>>> >> >> > > > >>> >> > > > > > > >>> > >>>>>>> >> >> > > > >>> >> > > > > > > >>> > >>>>>>> >> >> > > > >>> >> > > > > > > >>> > >>>>>>> >> >> > > > >>> >> > > > > > > On Wed, Feb 16, 2022 at 5:50 PM >>> Seth >>> > >>>>>>> Wiesman < >>> > >>>>>>> >> >> > > > >>> >> > sjwies...@gmail.com> >>> > >>>>>>> >> >> > > > >>> >> > > > > > wrote: >>> > >>>>>>> >> >> > > > >>> >> > > > > > > >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> Hi Galen, >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> No, that is not currently >>> > supported, >>> > >>>>>>> the >>> > >>>>>>> >> current >>> > >>>>>>> >> >> > > > >>> idiomatic >>> > >>>>>>> >> >> > > > >>> >> way >>> > >>>>>>> >> >> > > > >>> >> > > would >>> > >>>>>>> >> >> > > > >>> >> > > > > be >>> > >>>>>>> >> >> > > > >>> >> > > > > > to >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> pass those values to the struct >>> > >>>>>>> implementing >>> > >>>>>>> >> the >>> > >>>>>>> >> >> > > > Statefun >>> > >>>>>>> >> >> > > > >>> >> > > interface. >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> type MyFunc struct { >>> > someRuntimeInfo >>> > >>>>>>> string } >>> > >>>>>>> >> >> func >>> > >>>>>>> >> >> > (m >>> > >>>>>>> >> >> > > > >>> >> *MyFunc) >>> > >>>>>>> >> >> > > > >>> >> > > > > > Invoke(ctx >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> statefun.Context, message >>> > >>>>>>> statefun.Message) >>> > >>>>>>> >> >> error >>> > >>>>>>> >> >> > { } >>> > >>>>>>> >> >> > > > >>> func >>> > >>>>>>> >> >> > > > >>> >> > main() >>> > >>>>>>> >> >> > > > >>> >> > > { >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> builder >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> := >>> > >>>>>>> statefun.StatefulFunctionsBuilder() >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> f := MyFunc { someRuntimeInfo: >>> > >>>>>>> >> >> "runtime-provided" } >>> > >>>>>>> >> >> > > > >>> >> > > builder.WithSpec >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> (statefun.StatefulFunctionSpec{ >>> > >>>>>>> FunctionType: >>> > >>>>>>> >> >> > > > >>> >> > > statefun.TypeNameFrom( >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> "example/my-func"), Function: >>> f }) >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> http.Handle("/statefun", >>> > >>>>>>> builder.AsHandler()) >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> _ = >>> http.ListenAndServe(":8000", >>> > >>>>>>> nil) } >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> Would this work for you? Or >>> what is >>> > >>>>>>> the >>> > >>>>>>> >> context >>> > >>>>>>> >> >> > (pun >>> > >>>>>>> >> >> > > > >>> >> intended) >>> > >>>>>>> >> >> > > > >>> >> > you >>> > >>>>>>> >> >> > > > >>> >> > > > are >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> looking for? >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> Seth >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> On Wed, Feb 16, 2022 at 4:35 PM >>> > >>>>>>> Galen Warren >>> > >>>>>>> >> < >>> > >>>>>>> >> >> > > > >>> >> > > > ga...@cvillewarrens.com >>> > >>>>>>> >> >> > > > >>> >> > > > > > >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> wrote: >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> > When stateful functions are >>> > >>>>>>> invoked, they >>> > >>>>>>> >> are >>> > >>>>>>> >> >> > > passed >>> > >>>>>>> >> >> > > > an >>> > >>>>>>> >> >> > > > >>> >> > instance >>> > >>>>>>> >> >> > > > >>> >> > > > of >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> > statefun.Context, which >>> wraps the >>> > >>>>>>> >> >> context.Context >>> > >>>>>>> >> >> > > > >>> received >>> > >>>>>>> >> >> > > > >>> >> by >>> > >>>>>>> >> >> > > > >>> >> > > the >>> > >>>>>>> >> >> > > > >>> >> > > > > HTTP >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> > request. Is there any way to >>> > >>>>>>> customize that >>> > >>>>>>> >> >> > > > >>> context.Context >>> > >>>>>>> >> >> > > > >>> >> > to, >>> > >>>>>>> >> >> > > > >>> >> > > > say, >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> hold >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> > custom values, using >>> > >>>>>>> ctx.WithValue()? I >>> > >>>>>>> >> don't >>> > >>>>>>> >> >> > see a >>> > >>>>>>> >> >> > > > way >>> > >>>>>>> >> >> > > > >>> >> but I >>> > >>>>>>> >> >> > > > >>> >> > > > wanted >>> > >>>>>>> >> >> > > > >>> >> > > > > > to >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> > ask. >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> > >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> > If not, would you be >>> interested >>> > in >>> > >>>>>>> a PR to >>> > >>>>>>> >> add >>> > >>>>>>> >> >> > this >>> > >>>>>>> >> >> > > > >>> >> > > > functionality? A >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> simple >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> > way might be to add a >>> property to >>> > >>>>>>> >> >> > > > StatefulFunctionSpec, >>> > >>>>>>> >> >> > > > >>> >> say: >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> > >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> > TransformContext func(ctx >>> > >>>>>>> context.Context) >>> > >>>>>>> >> >> > > > >>> context.Context >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> > >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> > ... that, if supplied, would >>> be >>> > >>>>>>> called to >>> > >>>>>>> >> >> create >>> > >>>>>>> >> >> > a >>> > >>>>>>> >> >> > > > >>> >> customized >>> > >>>>>>> >> >> > > > >>> >> > > > > context >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> that >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> > would be used downstream? >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> > >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> > Thanks. >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> > >>> > >>>>>>> >> >> > > > >>> >> > > > > > >> >>> > >>>>>>> >> >> > > > >>> >> > > > > > > >>> > >>>>>>> >> >> > > > >>> >> > > > > > >>> > >>>>>>> >> >> > > > >>> >> > > > > >>> > >>>>>>> >> >> > > > >>> >> > > > >>> > >>>>>>> >> >> > > > >>> >> > > >>> > >>>>>>> >> >> > > > >>> >> > >>> > >>>>>>> >> >> > > > >>> >> >>> > >>>>>>> >> >> > > > >>> > >>> > >>>>>>> >> >> > > > >>> >>> > >>>>>>> >> >> > > > >> >>> > >>>>>>> >> >> > > > >>> > >>>>>>> >> >> > > >>> > >>>>>>> >> >> > >>> > >>>>>>> >> >> >>> > >>>>>>> >> > >>> > >>>>>>> >> >>> > >>>>>>> > >>> > >>>>>>> >>> > >>>>>> >>> > >>> >>