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

Reply via email to