SamStentz opened a new issue, #32559:
URL: https://github.com/apache/beam/issues/32559

   ### What happened?
   
   I've been trying to use the new beam state and timers to implement a retry 
policy. This retry policy:
   1) uses a processing time timer to defer execution
   2) places event time and values in state to use in OnTimer
   3) sets timer watermarks to incoming event's watermark
   
   The input is a keyed pcollection with a single entry. It seems like 
windowing causes my pardo to panic, which is surprising to me from having read 
[stateful and timely 
processing](https://beam.apache.org/blog/timely-processing/) and the [tour of 
beam](https://beam.apache.org/documentation/programming-guide/#state-and-timers)
 (particularly [this 
example](https://github.com/apache/beam/blob/master/sdks/go/examples/timer_wordcap/wordcap.go)).
   
   Using a debugger to move through my code, the panic occurs after 
`ProcessElement` and before `OnTimer` is called.
   
   ```
   2024/09/25 16:40:30 INFO ProcessElement <%!s(int=0), foo> 
source=/.../poc/donothing.go:45 time=2024-09-25T16:40:30.856Z 
worker.ID=job-001[go-job-1-1727282430841761429]_go 
worker.endpoint=localhost:46073
   panic: unable to decode bool; expected {0, 1} got 127
   ```
   
   using this package in `go.mod`
   
   github.com/apache/beam/sdks/v2 v2.58.0
   
   donothing.go
   ```golang
   package poc
   
   import (
        "context"
        "reflect"
        "time"
   
        "github.com/apache/beam/sdks/v2/go/pkg/beam"
        "github.com/apache/beam/sdks/v2/go/pkg/beam/core/runtime"
        "github.com/apache/beam/sdks/v2/go/pkg/beam/core/state"
        "github.com/apache/beam/sdks/v2/go/pkg/beam/core/timers"
        "github.com/apache/beam/sdks/v2/go/pkg/beam/log"
   )
   
   func init() {
        runtime.RegisterFunction(NewDoNothingTransform)
        runtime.RegisterFunction(DoNothingTransform)
        
runtime.RegisterType(reflect.TypeOf((*doNothingTransformFn)(nil)).Elem())
        runtime.RegisterType(reflect.TypeOf((*context.Context)(nil)).Elem())
   }
   
   type doNothingTransformFn struct {
        TimerCount      state.Value[int64]
        TimerTimerstamp state.Value[int64]
        Value           state.Value[string]
        OutputState     timers.ProcessingTime
   }
   
   func NewDoNothingTransform() *doNothingTransformFn {
        return &doNothingTransformFn{
                TimerCount:      state.MakeValueState[int64]("timerCount"),
                TimerTimerstamp: state.MakeValueState[int64]("timerTimerstamp"),
                Value:           state.MakeValueState[string]("value"),
                OutputState:     timers.InProcessingTime("processingTime"),
        }
   }
   
   func (fn *doNothingTransformFn) ProcessElement(ctx context.Context,
        et beam.EventTime,
        sp state.Provider, tp timers.Provider,
        k beam.X,
        v string,
        _ func(string)) {
   
        log.Infof(ctx, "ProcessElement <%s, %s>", k, v)
   
        err := fn.TimerCount.Write(sp, 0)
        if err != nil {
                panic("couldn't set TimerCount state")
        }
        err = fn.TimerTimerstamp.Write(sp, et.ToTime().UnixMilli())
        if err != nil {
                panic("couldn't set TimerTimerstamp state")
        }
        err = fn.Value.Write(sp, v)
        if err != nil {
                panic("couldn't set Value state")
        }
        // Set timer for 1 second processing time.
        fn.OutputState.Set(tp, time.Now().Add(time.Second), 
timers.WithOutputTimestamp(et.ToTime()))
   }
   
   func (fn *doNothingTransformFn) OnTimer(ctx context.Context,
        sp state.Provider, tp timers.Provider,
        k beam.X, timer timers.Context,
        emit func(string)) {
        // Read state.
        tc, tcp, err := fn.TimerCount.Read(sp)
        if err != nil || !tcp {
                panic("couldn't read TimerCount state")
        }
        timerTimerstamp, tsp, err := fn.TimerTimerstamp.Read(sp)
        if err != nil || !tsp {
                panic("couldn't read TimerTimerstamp state")
        }
        ts := time.UnixMilli(timerTimerstamp)
        v, _, err := fn.Value.Read(sp)
        if err != nil {
                panic("couldn't read Value state")
        }
        log.Infof(ctx, "OnTimer <%s, %s>: count %d", k, v, tc)
        // terminating condition of 100 timer calls.
        if tc > 100 {
                emit(v)
                return
        }
        // Set timer for 1 second processing time.
        fn.OutputState.Set(tp, time.Now().Add(time.Second), 
timers.WithOutputTimestamp(ts))
        err = fn.TimerCount.Write(sp, tc+1)
        if err != nil {
                log.Errorf(ctx, "error writing timer time: %v", err)
        }
   }
   
   func DoNothingTransform(scope beam.Scope, in beam.PCollection) 
beam.PCollection {
        return beam.ParDo(scope.Scope("DoNothing"), NewDoNothingTransform(), in)
   }
   ```
   
   
   donothing_test.go
   ```golang
   package poc
   
   import (
        "testing"
        "time"
   
        "github.com/apache/beam/sdks/v2/go/pkg/beam"
        "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/window"
        "github.com/apache/beam/sdks/v2/go/pkg/beam/testing/ptest"
   )
   
   func TestDoNothingTransform(t *testing.T) {
        beam.Init()
        pipeline, scope, col := ptest.CreateList([]string{"foo"})
        col = beam.AddFixedKey(scope, col)
   
        // If I add this window we hit panics.
        col = beam.WindowInto(scope, window.NewFixedWindows(time.Second), col)
   
        DoNothingTransform(scope, col)
   
        ptest.RunAndValidate(t, pipeline)
   }
   
   func TestMain(m *testing.M) {
        ptest.MainWithDefault(m, "prism")
   }
   ```
   
   ### Issue Priority
   
   Priority: 2 (default / most bugs should be filed as P2)
   
   ### Issue Components
   
   - [ ] Component: Python SDK
   - [ ] Component: Java SDK
   - [X] Component: Go SDK
   - [ ] Component: Typescript SDK
   - [ ] Component: IO connector
   - [ ] Component: Beam YAML
   - [ ] Component: Beam examples
   - [ ] Component: Beam playground
   - [ ] Component: Beam katas
   - [ ] Component: Website
   - [ ] Component: Infrastructure
   - [ ] Component: Spark Runner
   - [ ] Component: Flink Runner
   - [ ] Component: Samza Runner
   - [ ] Component: Twister2 Runner
   - [ ] Component: Hazelcast Jet Runner
   - [ ] Component: Google Cloud Dataflow Runner


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to