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. > >> >> > > > >>> >> > > > > > >> > > >> >> > > > >>> >> > > > > > >> > >> >> > > > >>> >> > > > > > > > >> >> > > > >>> >> > > > > > > >> >> > > > >>> >> > > > > > >> >> > > > >>> >> > > > > >> >> > > > >>> >> > > > >> >> > > > >>> >> > > >> >> > > > >>> >> > >> >> > > > >>> > > >> >> > > > >>> > >> >> > > > >> > >> >> > > > > >> >> > > > >> >> > > >> >> > >> > > >> > > >