From 6cdbcd0d9da5a43e44655a29a15f7c5dba73c515 Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Wed, 29 Mar 2023 11:14:59 -0400 Subject: [PATCH 01/22] added timer package --- sdks/go.mod | 2 +- sdks/go.sum | 2 + sdks/go/pkg/beam/core/timers/timers.go | 124 +++++++++++++++++++++++++ 3 files changed, 127 insertions(+), 1 deletion(-) create mode 100644 sdks/go/pkg/beam/core/timers/timers.go diff --git a/sdks/go.mod b/sdks/go.mod index a6023bb6695f..92a33aa5988a 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -52,7 +52,7 @@ require ( github.com/xitongsys/parquet-go v1.6.2 github.com/xitongsys/parquet-go-source v0.0.0-20220315005136-aec0fe3e777c go.mongodb.org/mongo-driver v1.11.2 - golang.org/x/exp v0.0.0-20230206171751-46f607a40771 + golang.org/x/exp v0.0.0-20220827204233-334a2380cb91 golang.org/x/net v0.8.0 golang.org/x/oauth2 v0.6.0 golang.org/x/sync v0.1.0 diff --git a/sdks/go.sum b/sdks/go.sum index c1f07d4a34b1..3385e7b74b8f 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -521,6 +521,8 @@ golang.org/x/exp v0.0.0-20191227195350-da58074b4299/go.mod h1:2RIsYlXP63K8oxa1u0 golang.org/x/exp v0.0.0-20200119233911-0405dc783f0a/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= golang.org/x/exp v0.0.0-20200207192155-f17229e696bd/go.mod h1:J/WKrq2StrnmMY6+EHIKF9dgMWnmCNThgcyBT1FY9mM= golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6/go.mod h1:3jZMyOhIsHpP37uCMkUooju7aAi5cS1Q23tOzKc+0MU= +golang.org/x/exp v0.0.0-20220827204233-334a2380cb91 h1:tnebWN09GYg9OLPss1KXj8txwZc6X6uMr6VFdcGNbHw= +golang.org/x/exp v0.0.0-20220827204233-334a2380cb91/go.mod h1:cyybsKvd6eL0RnXn6p/Grxp8F5bW7iYuBgsNCOHpMYE= golang.org/x/exp v0.0.0-20230206171751-46f607a40771 h1:xP7rWLUr1e1n2xkK5YB4LI0hPEy3LJC6Wk+D4pGlOJg= golang.org/x/exp v0.0.0-20230206171751-46f607a40771/go.mod h1:CxIveKay+FTh1D0yPZemJVgC/95VzuuOLq5Qi4xnoYc= golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= diff --git a/sdks/go/pkg/beam/core/timers/timers.go b/sdks/go/pkg/beam/core/timers/timers.go new file mode 100644 index 000000000000..dfc51bfd1234 --- /dev/null +++ b/sdks/go/pkg/beam/core/timers/timers.go @@ -0,0 +1,124 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Package timers contains structs for setting pipeline timers. +package timers + +import ( + "reflect" + + "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/mtime" +) + +var ( + ProviderType = reflect.TypeOf((*Provider)(nil)).Elem() +) + +type TimeDomainEnum int32 + +const ( + TimeDomainUnspecified TimeDomainEnum = 0 + TimeDomainEventTime TimeDomainEnum = 1 + TimeDomainProcessingTime TimeDomainEnum = 2 +) + +type TimerMap struct { + Family string + Tag string + Clear bool + FireTimestamp, HoldTimestamp mtime.Time +} + +type timerConfig struct { + Tag string + HoldTimestamp mtime.Time +} + +type timerOptions func(*timerConfig) + +// WithTag sets the tag for the timer. +func WithTag(tag string) timerOptions { + return func(tm *timerConfig) { + tm.Tag = tag + } +} + +// WithOutputTimestamp sets the output timestamp for the timer. +func WithOutputTimestamp(outputTimestamp mtime.Time) timerOptions { + return func(tm *timerConfig) { + tm.HoldTimestamp = outputTimestamp + } +} + +type Provider interface { + Set(t TimerMap) +} + +// EventTime represents the event time timer. +type EventTime struct { + Family string +} + +// Set sets the timer for a event-time timestamp. Calling this method repeatedly for the same key +// will cause it overwrite previously set timer. +func (et *EventTime) Set(p Provider, FiringTimestamp mtime.Time, opts ...timerOptions) { + tc := timerConfig{} + for _, opt := range opts { + opt(&tc) + } + tm := TimerMap{Family: et.Family, Tag: tc.Tag, FireTimestamp: FiringTimestamp, HoldTimestamp: FiringTimestamp} + if !tc.HoldTimestamp.ToTime().IsZero() { + tm.HoldTimestamp = tc.HoldTimestamp + } + p.Set(tm) +} + +// Clear clears this timer. +func (et *EventTime) Clear(p Provider) { + p.Set(TimerMap{Family: et.Family, Clear: true}) +} + +// ProcessingTime represents the processing time timer. +type ProcessingTime struct { + Family string +} + +// Set sets the timer for processing time domain. Calling this method repeatedly for the same key +// will cause it overwrite previously set timer. +func (pt *ProcessingTime) Set(p Provider, FiringTimestamp mtime.Time, opts ...timerOptions) { + tc := timerConfig{} + for _, opt := range opts { + opt(&tc) + } + tm := TimerMap{Family: pt.Family, Tag: tc.Tag, FireTimestamp: FiringTimestamp, HoldTimestamp: FiringTimestamp} + if !tc.HoldTimestamp.ToTime().IsZero() { + tm.HoldTimestamp = tc.HoldTimestamp + } + + p.Set(tm) +} + +// Clear clears this timer. +func (pt ProcessingTime) Clear(p Provider) { + p.Set(TimerMap{Family: pt.Family, Clear: true}) +} + +func InEventTime(Key string) EventTime { + return EventTime{Family: Key} +} + +func InProcessingTime(Key string) ProcessingTime { + return ProcessingTime{Family: Key} +} From f0e9ba69d59e53f74d45f520a81d2918bb0a305f Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Thu, 30 Mar 2023 21:59:55 -0400 Subject: [PATCH 02/22] add timer changes and merged with rebo's pr --- sdks/go.mod | 2 +- sdks/go.sum | 2 + sdks/go/examples/streaming_wordcap/wordcap.go | 214 ++++++++++++++++-- sdks/go/pkg/beam/core/funcx/fn.go | 45 +++- sdks/go/pkg/beam/core/graph/edge.go | 1 + sdks/go/pkg/beam/core/graph/fn.go | 78 +++++++ sdks/go/pkg/beam/core/runtime/exec/coder.go | 21 +- .../pkg/beam/core/runtime/exec/datasource.go | 5 + sdks/go/pkg/beam/core/runtime/exec/fn.go | 89 ++++++-- sdks/go/pkg/beam/core/runtime/exec/pardo.go | 43 +++- sdks/go/pkg/beam/core/runtime/exec/plan.go | 11 + sdks/go/pkg/beam/core/runtime/exec/timers.go | 125 ++++++++++ .../pkg/beam/core/runtime/exec/timers_test.go | 144 ++++++------ .../pkg/beam/core/runtime/exec/translate.go | 24 ++ sdks/go/pkg/beam/core/runtime/genx/genx.go | 3 + sdks/go/pkg/beam/core/runtime/graphx/coder.go | 2 +- .../pkg/beam/core/runtime/graphx/serialize.go | 5 + .../pkg/beam/core/runtime/graphx/translate.go | 15 ++ .../pkg/beam/core/runtime/harness/harness.go | 8 +- sdks/go/pkg/beam/core/timers/timers.go | 29 ++- sdks/go/pkg/beam/core/typex/class.go | 1 + sdks/go/pkg/beam/core/typex/special.go | 3 +- sdks/go/pkg/beam/pardo.go | 12 + .../regression/coders/fromyaml/fromyaml.go | 2 +- 24 files changed, 751 insertions(+), 133 deletions(-) create mode 100644 sdks/go/pkg/beam/core/runtime/exec/timers.go diff --git a/sdks/go.mod b/sdks/go.mod index 92a33aa5988a..dec4fd5b53a9 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -52,7 +52,7 @@ require ( github.com/xitongsys/parquet-go v1.6.2 github.com/xitongsys/parquet-go-source v0.0.0-20220315005136-aec0fe3e777c go.mongodb.org/mongo-driver v1.11.2 - golang.org/x/exp v0.0.0-20220827204233-334a2380cb91 + golang.org/x/exp v0.0.0-20230321023759-10a507213a29 golang.org/x/net v0.8.0 golang.org/x/oauth2 v0.6.0 golang.org/x/sync v0.1.0 diff --git a/sdks/go.sum b/sdks/go.sum index 3385e7b74b8f..af56ce0ace6e 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -525,6 +525,8 @@ golang.org/x/exp v0.0.0-20220827204233-334a2380cb91 h1:tnebWN09GYg9OLPss1KXj8txw golang.org/x/exp v0.0.0-20220827204233-334a2380cb91/go.mod h1:cyybsKvd6eL0RnXn6p/Grxp8F5bW7iYuBgsNCOHpMYE= golang.org/x/exp v0.0.0-20230206171751-46f607a40771 h1:xP7rWLUr1e1n2xkK5YB4LI0hPEy3LJC6Wk+D4pGlOJg= golang.org/x/exp v0.0.0-20230206171751-46f607a40771/go.mod h1:CxIveKay+FTh1D0yPZemJVgC/95VzuuOLq5Qi4xnoYc= +golang.org/x/exp v0.0.0-20230321023759-10a507213a29 h1:ooxPy7fPvB4kwsA2h+iBNHkAbp/4JxTSwCmvdjEYmug= +golang.org/x/exp v0.0.0-20230321023759-10a507213a29/go.mod h1:CxIveKay+FTh1D0yPZemJVgC/95VzuuOLq5Qi4xnoYc= golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= diff --git a/sdks/go/examples/streaming_wordcap/wordcap.go b/sdks/go/examples/streaming_wordcap/wordcap.go index ddd9eab4e5f8..f708df356319 100644 --- a/sdks/go/examples/streaming_wordcap/wordcap.go +++ b/sdks/go/examples/streaming_wordcap/wordcap.go @@ -26,16 +26,21 @@ package main import ( "context" "flag" + "fmt" "os" - "strings" + "time" "github.com/apache/beam/sdks/v2/go/pkg/beam" - "github.com/apache/beam/sdks/v2/go/pkg/beam/io/pubsubio" + "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/mtime" + "github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf" + "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/io/rtrackers/offsetrange" "github.com/apache/beam/sdks/v2/go/pkg/beam/log" - "github.com/apache/beam/sdks/v2/go/pkg/beam/options/gcpopts" - "github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx" + "github.com/apache/beam/sdks/v2/go/pkg/beam/register" "github.com/apache/beam/sdks/v2/go/pkg/beam/x/beamx" "github.com/apache/beam/sdks/v2/go/pkg/beam/x/debug" + "golang.org/x/exp/slog" ) var ( @@ -50,32 +55,207 @@ var ( } ) +type Stateful struct { + ElementBag state.Bag[string] + TimerTime state.Value[int64] + MinTime state.Combining[int64, int64, int64] + + OutputState timers.ProcessingTime +} + +func NewStateful() *Stateful { + return &Stateful{ + ElementBag: state.MakeBagState[string]("elementBag"), + TimerTime: state.MakeValueState[int64]("timerTime"), + MinTime: state.MakeCombiningState[int64, int64, int64]("minTiInBag", func(a, b int64) int64 { + if a < b { + return a + } + return b + }), + + OutputState: timers.InProcessingTime("outputState"), + } +} + +func (s *Stateful) OnTimer(ctx context.Context, ts beam.EventTime, sp state.Provider, tp timers.Provider) { + log.Infof(ctx, "timer fired on stateful") +} + +func (s *Stateful) ProcessElement(ctx context.Context, ts beam.EventTime, sp state.Provider, tp timers.Provider, key, word string, emit func(string, string)) error { + log.Infof(ctx, "stateful dofn invoked key: %q word: %q", key, word) + + s.ElementBag.Add(sp, word) + s.MinTime.Add(sp, int64(ts)) + + toFire, ok, err := s.TimerTime.Read(sp) + if err != nil { + return err + } + if !ok { + toFire = int64(mtime.Now().Add(1 * time.Minute)) + } + minTime, _, err := s.MinTime.Read(sp) + if err != nil { + return err + } + + s.OutputState.Set(tp, mtime.Time(toFire).ToTime(), timers.WithOutputTimestamp(mtime.Time(minTime).ToTime()), timers.WithTag(word)) + s.TimerTime.Write(sp, toFire) + log.Infof(ctx, "stateful dofn key: %v word: %v, timer: %v, minTime: %v", key, word, toFire, minTime) + + // // Get the Value stored in our state + // val, ok, err := s.Val.Read(p) + // if err != nil { + // return err + // } + // log.Infof(ctx, "stateful dofn state read key: %v word: %v val: %v", key, word, val) + // if !ok { + // s.Val.Write(p, 1) + // } else { + // s.Val.Write(p, val+1) + // } + + // if val > 5 { + // log.Infof(ctx, "stateful dofn clearing key: %v word: %v val: %v", key, word, val) + // // Example of clearing and starting again with an empty bag + // s.Val.Clear(p) + // } + // fire := time.Now().Add(10 * time.Second) + + // log.Infof(ctx, "stateful dofn timer family: %v fire: %v now: %v key: %v word: %v", s.Fire.Family, fire, time.Now(), key, word) + // s.Fire.Set(tp, fire) + + // emit(key, word) + + return nil +} + +type eventtimeSDFStream struct { + RestSize, Mod, Fixed int64 + Sleep time.Duration +} + +func (fn *eventtimeSDFStream) Setup() error { + return nil +} + +func (fn *eventtimeSDFStream) CreateInitialRestriction(v beam.T) offsetrange.Restriction { + return offsetrange.Restriction{Start: 0, End: fn.RestSize} +} + +func (fn *eventtimeSDFStream) SplitRestriction(v beam.T, r offsetrange.Restriction) []offsetrange.Restriction { + // No split + return []offsetrange.Restriction{r} +} + +func (fn *eventtimeSDFStream) RestrictionSize(v beam.T, r offsetrange.Restriction) float64 { + return r.Size() +} + +func (fn *eventtimeSDFStream) CreateTracker(r offsetrange.Restriction) *sdf.LockRTracker { + return sdf.NewLockRTracker(offsetrange.NewTracker(r)) +} + +func (fn *eventtimeSDFStream) ProcessElement(ctx context.Context, _ *CWE, rt *sdf.LockRTracker, v beam.T, emit func(beam.EventTime, int64)) sdf.ProcessContinuation { + r := rt.GetRestriction().(offsetrange.Restriction) + i := r.Start + if r.Size() < 1 { + log.Debugf(ctx, "size 0 restriction, stoping to process sentinel %v", slog.Any("value", v)) + return sdf.StopProcessing() + } + slog.Debug("emitting element to restriction", slog.Any("value", v), slog.Group("restriction", + slog.Any("value", v), + slog.Float64("size", r.Size()), + slog.Int64("pos", i), + )) + if rt.TryClaim(i) { + v := (i % fn.Mod) + fn.Fixed + emit(mtime.Now(), v) + } + return sdf.ResumeProcessingIn(fn.Sleep) +} + +func (fn *eventtimeSDFStream) InitialWatermarkEstimatorState(_ beam.EventTime, _ offsetrange.Restriction, _ beam.T) int64 { + return int64(mtime.MinTimestamp) +} + +func (fn *eventtimeSDFStream) CreateWatermarkEstimator(initialState int64) *CWE { + return &CWE{Watermark: initialState} +} + +func (fn *eventtimeSDFStream) WatermarkEstimatorState(e *CWE) int64 { + return e.Watermark +} + +type CWE struct { + Watermark int64 // uses int64, since the SDK prevent mtime.Time from serialization. +} + +func (e *CWE) CurrentWatermark() time.Time { + return mtime.Time(e.Watermark).ToTime() +} + +func (e *CWE) ObserveTimestamp(ts time.Time) { + // We add 10 milliseconds to allow window boundaries to + // progress after emitting + e.Watermark = int64(mtime.FromTime(ts.Add(-90 * time.Millisecond))) +} + +func init() { + register.DoFn7x1[context.Context, beam.EventTime, state.Provider, timers.Provider, string, string, func(string, string), error](&Stateful{}) + register.Emitter2[string, string]() + register.DoFn5x1[context.Context, *CWE, *sdf.LockRTracker, beam.T, func(beam.EventTime, int64), sdf.ProcessContinuation]((*eventtimeSDFStream)(nil)) + register.Emitter2[beam.EventTime, int64]() +} + func main() { flag.Parse() beam.Init() ctx := context.Background() - project := gcpopts.GetProject(ctx) + //project := gcpopts.GetProject(ctx) log.Infof(ctx, "Publishing %v messages to: %v", len(data), *input) - defer pubsubx.CleanupTopic(ctx, project, *input) - sub, err := pubsubx.Publish(ctx, project, *input, data...) - if err != nil { - log.Fatal(ctx, err) - } + // defer pubsubx.CleanupTopic(ctx, project, *input) + // sub, err := pubsubx.Publish(ctx, project, *input, data...) + // if err != nil { + // log.Fatal(ctx, err) + // } - log.Infof(ctx, "Running streaming wordcap with subscription: %v", sub.ID()) + //log.Infof(ctx, "Running streaming wordcap with subscription: %v", sub.ID()) p := beam.NewPipeline() s := p.Root() - col := pubsubio.Read(s, project, *input, &pubsubio.ReadOptions{Subscription: sub.ID()}) - str := beam.ParDo(s, func(b []byte) string { - return (string)(b) - }, col) - cap := beam.ParDo(s, strings.ToUpper, str) - debug.Print(s, cap) + //col := pubsubio.Read(s, project, *input, &pubsubio.ReadOptions{Subscription: sub.ID()}) + // col = beam.WindowInto(s, window.NewFixedWindows(60*time.Second), col) + + // str := beam.ParDo(s, func(b []byte) string { + // return (string)(b) + // }, col) + + imp := beam.Impulse(s) + elms := 3 + out := beam.ParDo(s, &eventtimeSDFStream{ + Sleep: time.Second, + RestSize: int64(elms), + Mod: int64(elms), + Fixed: 1, + }, imp) + // out = beam.WindowInto(s, window.NewFixedWindows(10*time.Second), out) + str := beam.ParDo(s, func(b int64) string { + return fmt.Sprintf("%03d", b) + }, out) + + keyed := beam.ParDo(s, func(ctx context.Context, ts beam.EventTime, s string) (string, string) { + log.Infof(ctx, "adding key ts: %v now: %v word: %v", ts.ToTime(), time.Now(), s) + return "test", s + }, str) + + timed := beam.ParDo(s, NewStateful(), keyed) + debug.Printf(s, "post stateful: %v", timed) if err := beamx.Run(context.Background(), p); err != nil { log.Exitf(ctx, "Failed to execute job: %v", err) diff --git a/sdks/go/pkg/beam/core/funcx/fn.go b/sdks/go/pkg/beam/core/funcx/fn.go index b579cb56d521..3aef5a1695ac 100644 --- a/sdks/go/pkg/beam/core/funcx/fn.go +++ b/sdks/go/pkg/beam/core/funcx/fn.go @@ -21,6 +21,7 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf" "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/core/typex" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/util/reflectx" "github.com/apache/beam/sdks/v2/go/pkg/beam/internal/errors" @@ -85,6 +86,8 @@ const ( FnWatermarkEstimator FnParamKind = 0x1000 // FnStateProvider indicates a function input parameter that implements state.Provider FnStateProvider FnParamKind = 0x2000 + // FnTimerProvider indicates a function input parameter that implements timer.Provider + FnTimerProvider FnParamKind = 0x4000 ) func (k FnParamKind) String() string { @@ -117,6 +120,8 @@ func (k FnParamKind) String() string { return "WatermarkEstimator" case FnStateProvider: return "StateProvider" + case FnTimerProvider: + return "TimerProvider" default: return fmt.Sprintf("%v", int(k)) } @@ -305,6 +310,15 @@ func (u *Fn) StateProvider() (pos int, exists bool) { return -1, false } +func (u *Fn) TimerProvider() (pos int, exists bool) { + for i, p := range u.Param { + if p.Kind == FnTimerProvider { + return i, true + } + } + return -1, false +} + // WatermarkEstimator returns (index, true) iff the function expects a // parameter that implements sdf.WatermarkEstimator. func (u *Fn) WatermarkEstimator() (pos int, exists bool) { @@ -392,6 +406,8 @@ func New(fn reflectx.Func) (*Fn, error) { kind = FnBundleFinalization case t == state.ProviderType: kind = FnStateProvider + case t == timers.ProviderType: + kind = FnTimerProvider case t == reflectx.Type: kind = FnType case t.Implements(reflect.TypeOf((*sdf.RTracker)(nil)).Elem()): @@ -482,7 +498,7 @@ func SubReturns(list []ReturnParam, indices ...int) []ReturnParam { } // The order of present parameters and return values must be as follows: -// func(FnContext?, FnPane?, FnWindow?, FnEventTime?, FnWatermarkEstimator?, FnType?, FnBundleFinalization?, FnRTracker?, FnStateProvider?, (FnValue, SideInput*)?, FnEmit*) (RetEventTime?, RetOutput?, RetError?) +// func(FnContext?, FnPane?, FnWindow?, FnEventTime?, FnWatermarkEstimator?, FnType?, FnBundleFinalization?, FnRTracker?, FnStateProvider?, FnTimerProvider?, (FnValue, SideInput*)?, FnEmit*) (RetEventTime?, RetOutput?, RetError?) // // where ? indicates 0 or 1, and * indicates any number. // and a SideInput is one of FnValue or FnIter or FnReIter @@ -517,6 +533,7 @@ var ( errRTrackerPrecedence = errors.New("may only have a single sdf.RTracker parameter and it must precede the main input parameter") errBundleFinalizationPrecedence = errors.New("may only have a single BundleFinalization parameter and it must precede the main input parameter") errStateProviderPrecedence = errors.New("may only have a single state.Provider parameter and it must precede the main input parameter") + errTimerProviderPrecedence = errors.New("may only have a single timer.Provider parameter and it must precede the main input parameter") errInputPrecedence = errors.New("inputs parameters must precede emit function parameters") ) @@ -535,6 +552,7 @@ const ( psRTracker psBundleFinalization psStateProvider + psTimerProvider ) func nextParamState(cur paramState, transition FnParamKind) (paramState, error) { @@ -559,6 +577,8 @@ func nextParamState(cur paramState, transition FnParamKind) (paramState, error) return psRTracker, nil case FnStateProvider: return psStateProvider, nil + case FnTimerProvider: + return psTimerProvider, nil } case psContext: switch transition { @@ -578,6 +598,8 @@ func nextParamState(cur paramState, transition FnParamKind) (paramState, error) return psRTracker, nil case FnStateProvider: return psStateProvider, nil + case FnTimerProvider: + return psTimerProvider, nil } case psPane: switch transition { @@ -595,6 +617,8 @@ func nextParamState(cur paramState, transition FnParamKind) (paramState, error) return psRTracker, nil case FnStateProvider: return psStateProvider, nil + case FnTimerProvider: + return psTimerProvider, nil } case psWindow: switch transition { @@ -610,6 +634,8 @@ func nextParamState(cur paramState, transition FnParamKind) (paramState, error) return psRTracker, nil case FnStateProvider: return psStateProvider, nil + case FnTimerProvider: + return psTimerProvider, nil } case psEventTime: switch transition { @@ -623,6 +649,8 @@ func nextParamState(cur paramState, transition FnParamKind) (paramState, error) return psRTracker, nil case FnStateProvider: return psStateProvider, nil + case FnTimerProvider: + return psTimerProvider, nil } case psWatermarkEstimator: switch transition { @@ -634,6 +662,8 @@ func nextParamState(cur paramState, transition FnParamKind) (paramState, error) return psRTracker, nil case FnStateProvider: return psStateProvider, nil + case FnTimerProvider: + return psTimerProvider, nil } case psType: switch transition { @@ -643,6 +673,8 @@ func nextParamState(cur paramState, transition FnParamKind) (paramState, error) return psRTracker, nil case FnStateProvider: return psStateProvider, nil + case FnTimerProvider: + return psTimerProvider, nil } case psBundleFinalization: switch transition { @@ -650,13 +682,22 @@ func nextParamState(cur paramState, transition FnParamKind) (paramState, error) return psRTracker, nil case FnStateProvider: return psStateProvider, nil + case FnTimerProvider: + return psTimerProvider, nil } case psRTracker: switch transition { case FnStateProvider: return psStateProvider, nil + case FnTimerProvider: + return psTimerProvider, nil } case psStateProvider: + switch transition { + case FnTimerProvider: + return psTimerProvider, nil + } + case psTimerProvider: // Completely handled by the default clause case psInput: switch transition { @@ -689,6 +730,8 @@ func nextParamState(cur paramState, transition FnParamKind) (paramState, error) return -1, errRTrackerPrecedence case FnStateProvider: return -1, errStateProviderPrecedence + case FnTimerProvider: + return -1, errTimerProviderPrecedence case FnIter, FnReIter, FnValue, FnMultiMap: return psInput, nil case FnEmit: diff --git a/sdks/go/pkg/beam/core/graph/edge.go b/sdks/go/pkg/beam/core/graph/edge.go index a9f1c8a092b0..86891114dd0e 100644 --- a/sdks/go/pkg/beam/core/graph/edge.go +++ b/sdks/go/pkg/beam/core/graph/edge.go @@ -156,6 +156,7 @@ type MultiEdge struct { DoFn *DoFn // ParDo RestrictionCoder *coder.Coder // SplittableParDo StateCoders map[string]*coder.Coder // Stateful ParDo + TimerCoders map[string]*coder.Coder // Stateful ParDo CombineFn *CombineFn // Combine AccumCoder *coder.Coder // Combine Value []byte // Impulse diff --git a/sdks/go/pkg/beam/core/graph/fn.go b/sdks/go/pkg/beam/core/graph/fn.go index 54cc02e07b3d..c87c4525858f 100644 --- a/sdks/go/pkg/beam/core/graph/fn.go +++ b/sdks/go/pkg/beam/core/graph/fn.go @@ -22,6 +22,7 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/core/funcx" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf" "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/core/typex" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/util/reflectx" "github.com/apache/beam/sdks/v2/go/pkg/beam/internal/errors" @@ -167,6 +168,8 @@ const ( initialWatermarkEstimatorStateName = "InitialWatermarkEstimatorState" watermarkEstimatorStateName = "WatermarkEstimatorState" + onTimerName = "OnTimer" + createAccumulatorName = "CreateAccumulator" addInputName = "AddInput" mergeAccumulatorsName = "MergeAccumulators" @@ -304,6 +307,40 @@ func (f *DoFn) PipelineState() []state.PipelineState { return s } +type PipelineTimer interface { + TimerFamily() string + TimerDomain() timers.TimeDomainEnum +} + +var ( + _ PipelineTimer = timers.EventTime{} + _ PipelineTimer = timers.ProcessingTime{} +) + +func (f *DoFn) OnTimerFn() (*funcx.Fn, bool) { + m, ok := f.methods[onTimerName] + return m, ok +} + +func (f *DoFn) PipelineTimers() []PipelineTimer { + var t []PipelineTimer + if f.Recv == nil { + return t + } + + v := reflect.Indirect(reflect.ValueOf(f.Recv)) + + for i := 0; i < v.NumField(); i++ { + f := v.Field(i) + if f.CanInterface() { + if pt, ok := f.Interface().(PipelineTimer); ok { + t = append(t, pt) + } + } + } + return t +} + // SplittableDoFn represents a DoFn implementing SDF methods. type SplittableDoFn DoFn @@ -607,6 +644,11 @@ func AsDoFn(fn *Fn, numMainIn mainInputs) (*DoFn, error) { return nil, addContext(err, fn) } + err = validateTimer(doFn) + if err != nil { + return nil, addContext(err, fn) + } + return doFn, nil } @@ -1350,6 +1392,42 @@ func validateState(fn *DoFn, numIn mainInputs) error { return nil } +func validateTimer(fn *DoFn) error { + if fn.Fn == nil { + return nil + } + + pt := fn.PipelineTimers() + + if _, ok := fn.Fn.TimerProvider(); ok { + if len(pt) == 0 { + err := errors.Errorf("ProcessElement uses a TimerProvider, but no timer struct-tags are attached to the DoFn") + return errors.SetTopLevelMsgf(err, "ProcessElement uses a TimerProvider, but no timer struct-tags are attached to the DoFn"+ + ", Ensure that you are including the timer structs you're using to set/clear global state as uppercase member variables") + } + timerKeys := make(map[string]PipelineTimer) + for _, t := range pt { + k := t.TimerFamily() + if timer, ok := timerKeys[k]; ok { + err := errors.Errorf("Duplicate timer key %v", k) + return errors.SetTopLevelMsgf(err, "Duplicate timer key %v used by %v and %v. Ensure that keys are unique per DoFn", k, timer, t) + } else { + timerKeys[k] = t + } + } + } else { + if len(pt) > 0 { + err := errors.Errorf("ProcessElement doesn't use a TimerProvider, but Timer Struct is attached to the DoFn: %v", pt) + return errors.SetTopLevelMsgf(err, "ProcessElement doesn't use a TimerProvider, but Timer Struct is attached to the DoFn: %v"+ + ", Ensure that you are using the TimerProvider to set/clear the timers.", pt) + } + } + + // DO NOT SUBMIT: Require an OnTimer method existing + + return nil +} + // CombineFn represents a CombineFn. type CombineFn Fn diff --git a/sdks/go/pkg/beam/core/runtime/exec/coder.go b/sdks/go/pkg/beam/core/runtime/exec/coder.go index 39248f7f5ac3..91ae2a950dac 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/coder.go +++ b/sdks/go/pkg/beam/core/runtime/exec/coder.go @@ -1258,8 +1258,11 @@ func DecodeWindowedValueHeader(dec WindowDecoder, r io.Reader) ([]typex.Window, func encodeTimer(elm ElementEncoder, win WindowEncoder, tm typex.TimerMap, w io.Writer) error { var b bytes.Buffer - elm.Encode(&FullValue{Elm: tm.Key}, &b) - + // elm.Encode(&FullValue{Elm: tm.Key}, &b) + _, err := b.Write(tm.Key) + if err != nil { + return errors.WithContext(err, "error encoding key") + } if err := coder.EncodeStringUTF8(tm.Tag, &b); err != nil { return errors.WithContext(err, "error encoding tag") } @@ -1291,12 +1294,18 @@ func encodeTimer(elm ElementEncoder, win WindowEncoder, tm typex.TimerMap, w io. func decodeTimer(dec ElementDecoder, win WindowDecoder, r io.Reader) (typex.TimerMap, error) { tm := typex.TimerMap{} - fv, err := dec.Decode(r) + // fv, err := dec.Decode(r) + // if err != nil { + // return tm, errors.WithContext(err, "error decoding timer key") + // } + // // TODO Change to not type assert once general timers key fix is done. + // tm.Key = fv.Elm.([]byte) + + key, err := coder.DecodeBytes(r) if err != nil { - return tm, errors.WithContext(err, "error decoding timer key") + return tm, errors.WithContext(err, "error decoding key") } - tm.Key = fv.Elm.(string) - + tm.Key = key s, err := coder.DecodeStringUTF8(r) if err != nil && err != io.EOF { return tm, errors.WithContext(err, "error decoding timer tag") diff --git a/sdks/go/pkg/beam/core/runtime/exec/datasource.go b/sdks/go/pkg/beam/core/runtime/exec/datasource.go index 66c081862f13..c1dbc3faa7e8 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/datasource.go +++ b/sdks/go/pkg/beam/core/runtime/exec/datasource.go @@ -265,6 +265,11 @@ func (n *DataSource) Process(ctx context.Context) ([]*Checkpoint, error) { func(bcr *byteCountReader, ptransformID, timerFamilyID string) error { tmap, err := decodeTimer(cp, wc, bcr) log.Infof(ctx, "DEBUGLOG: timer received for: %v and %v - %+v err: %v", ptransformID, timerFamilyID, tmap, err) + log.Infof(ctx, "OnTimerTransforms = %+v", n.OnTimerTransforms[ptransformID].Fn) + // if fn, ok := n.OnTimerTransforms[ptransformID].Fn.OnTimerFn(); ok { + // log.Infof(ctx, "found ontimer method, invoking callback") + n.OnTimerTransforms[ptransformID].InvokeTimerFn(ctx, n.OnTimerTransforms[ptransformID].Fn.Fn, timerFamilyID, tmap) + // } return nil }) diff --git a/sdks/go/pkg/beam/core/runtime/exec/fn.go b/sdks/go/pkg/beam/core/runtime/exec/fn.go index eaf9df81e4aa..0655b0f08028 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/fn.go +++ b/sdks/go/pkg/beam/core/runtime/exec/fn.go @@ -70,15 +70,46 @@ func (bf *bundleFinalizer) RegisterCallback(t time.Duration, cb func() error) { // Invoke invokes the fn with the given values. The extra values must match the non-main // side input and emitters. It returns the direct output, if any. -func Invoke(ctx context.Context, pn typex.PaneInfo, ws []typex.Window, ts typex.EventTime, fn *funcx.Fn, opt *MainInput, bf *bundleFinalizer, we sdf.WatermarkEstimator, sa UserStateAdapter, reader StateReader, extra ...any) (*FullValue, error) { +// +// Deprecated: prefer InvokeWithOpts +func Invoke(ctx context.Context, pn typex.PaneInfo, ws []typex.Window, ts typex.EventTime, fn *funcx.Fn, opt *MainInput, bf *bundleFinalizer, we sdf.WatermarkEstimator, sa UserStateAdapter, sr StateReader, extra ...any) (*FullValue, error) { + if fn == nil { + return nil, nil + } + inv := newInvoker(fn) + return inv.invokeWithOpts(ctx, pn, ws, ts, InvokeOpts{opt: opt, bf: bf, we: we, sa: sa, sr: sr, extra: extra}) +} + +// InvokeOpts are optional parameters to invoke a Fn. +type InvokeOpts struct { + opt *MainInput + bf *bundleFinalizer + we sdf.WatermarkEstimator + sa UserStateAdapter + sr StateReader + ta UserTimerAdapter + tm DataManager + extra []any +} + +// InvokeWithOpts invokes the fn with the given values. The extra values must match the non-main +// side input and emitters. It returns the direct output, if any. +func InvokeWithOpts(ctx context.Context, fn *funcx.Fn, pn typex.PaneInfo, ws []typex.Window, ts typex.EventTime, opts InvokeOpts) (*FullValue, error) { if fn == nil { return nil, nil // ok: nothing to Invoke } inv := newInvoker(fn) - return inv.Invoke(ctx, pn, ws, ts, opt, bf, we, sa, reader, extra...) + return inv.invokeWithOpts(ctx, pn, ws, ts, opts) +} + +// InvokeWithOptsWithoutEventTime runs the given function at time 0 in the global window. +func InvokeWithOptsWithoutEventTime(ctx context.Context, fn *funcx.Fn, opts InvokeOpts) (*FullValue, error) { + return InvokeWithOpts(ctx, fn, typex.NoFiringPane(), window.SingleGlobalWindow, mtime.ZeroTimestamp, opts) } // InvokeWithoutEventTime runs the given function at time 0 in the global window. +// +// Deprecated: prefer InvokeWithOptsWithoutEventTime func InvokeWithoutEventTime(ctx context.Context, fn *funcx.Fn, opt *MainInput, bf *bundleFinalizer, we sdf.WatermarkEstimator, sa UserStateAdapter, reader StateReader, extra ...any) (*FullValue, error) { if fn == nil { return nil, nil // ok: nothing to Invoke @@ -93,10 +124,12 @@ type invoker struct { fn *funcx.Fn args []any sp *stateProvider + tp *timerProvider + // TODO(lostluck): 2018/07/06 consider replacing with a slice of functions to run over the args slice, as an improvement. - ctxIdx, pnIdx, wndIdx, etIdx, bfIdx, weIdx, spIdx int // specialized input indexes - outEtIdx, outPcIdx, outErrIdx int // specialized output indexes - in, out []int // general indexes + ctxIdx, pnIdx, wndIdx, etIdx, bfIdx, weIdx, spIdx, tpIdx int // specialized input indexes + outEtIdx, outPcIdx, outErrIdx int // specialized output indexes + in, out []int // general indexes ret FullValue // ret is a cached allocation for passing to the next Unit. Units never modify the passed in FullValue. elmConvert, elm2Convert func(any) any // Cached conversion functions, which assums this invoker is always used with the same parameter types. @@ -129,6 +162,9 @@ func newInvoker(fn *funcx.Fn) *invoker { if n.spIdx, ok = fn.StateProvider(); !ok { n.spIdx = -1 } + if n.tpIdx, ok = fn.TimerProvider(); !ok { + n.tpIdx = -1 + } if n.outEtIdx, ok = fn.OutEventTime(); !ok { n.outEtIdx = -1 } @@ -163,7 +199,11 @@ func (n *invoker) InvokeWithoutEventTime(ctx context.Context, opt *MainInput, bf // Invoke invokes the fn with the given values. The extra values must match the non-main // side input and emitters. It returns the direct output, if any. -func (n *invoker) Invoke(ctx context.Context, pn typex.PaneInfo, ws []typex.Window, ts typex.EventTime, opt *MainInput, bf *bundleFinalizer, we sdf.WatermarkEstimator, sa UserStateAdapter, reader StateReader, extra ...any) (*FullValue, error) { +func (n *invoker) Invoke(ctx context.Context, pn typex.PaneInfo, ws []typex.Window, ts typex.EventTime, opt *MainInput, bf *bundleFinalizer, we sdf.WatermarkEstimator, sa UserStateAdapter, sr StateReader, extra ...any) (*FullValue, error) { + return n.invokeWithOpts(ctx, pn, ws, ts, InvokeOpts{opt: opt, bf: bf, we: we, sa: sa, sr: sr, extra: extra}) +} + +func (n *invoker) invokeWithOpts(ctx context.Context, pn typex.PaneInfo, ws []typex.Window, ts typex.EventTime, opts InvokeOpts) (*FullValue, error) { // (1) Populate contexts // extract these to make things easier to read. args := n.args @@ -178,7 +218,7 @@ func (n *invoker) Invoke(ctx context.Context, pn typex.PaneInfo, ws []typex.Wind } if n.wndIdx >= 0 { if len(ws) != 1 { - return nil, errors.Errorf("DoFns that observe windows must be invoked with single window: %v", opt.Key.Windows) + return nil, errors.Errorf("DoFns that observe windows must be invoked with single window: %v", opts.opt.Key.Windows) } args[n.wndIdx] = ws[0] } @@ -186,14 +226,14 @@ func (n *invoker) Invoke(ctx context.Context, pn typex.PaneInfo, ws []typex.Wind args[n.etIdx] = ts } if n.bfIdx >= 0 { - args[n.bfIdx] = bf + args[n.bfIdx] = opts.bf } if n.weIdx >= 0 { - args[n.weIdx] = we + args[n.weIdx] = opts.we } if n.spIdx >= 0 { - sp, err := sa.NewStateProvider(ctx, reader, ws[0], opt) + sp, err := opts.sa.NewStateProvider(ctx, opts.sr, ws[0], opts.opt) if err != nil { return nil, err } @@ -201,29 +241,38 @@ func (n *invoker) Invoke(ctx context.Context, pn typex.PaneInfo, ws []typex.Wind args[n.spIdx] = n.sp } + if n.tpIdx >= 0 { + tp, err := opts.ta.NewTimerProvider(ctx, opts.tm, ts, ws, opts.opt) + if err != nil { + return nil, err + } + n.tp = &tp + args[n.tpIdx] = n.tp + } + // (2) Main input from value, if any. i := 0 - if opt != nil { - if opt.RTracker != nil { - args[in[i]] = opt.RTracker + if opts.opt != nil { + if opts.opt.RTracker != nil { + args[in[i]] = opts.opt.RTracker i++ } if n.elmConvert == nil { - from := reflect.TypeOf(opt.Key.Elm) + from := reflect.TypeOf(opts.opt.Key.Elm) n.elmConvert = ConvertFn(from, fn.Param[in[i]].T) } - args[in[i]] = n.elmConvert(opt.Key.Elm) + args[in[i]] = n.elmConvert(opts.opt.Key.Elm) i++ - if opt.Key.Elm2 != nil { + if opts.opt.Key.Elm2 != nil { if n.elm2Convert == nil { - from := reflect.TypeOf(opt.Key.Elm2) + from := reflect.TypeOf(opts.opt.Key.Elm2) n.elm2Convert = ConvertFn(from, fn.Param[in[i]].T) } - args[in[i]] = n.elm2Convert(opt.Key.Elm2) + args[in[i]] = n.elm2Convert(opts.opt.Key.Elm2) i++ } - for _, iter := range opt.Values { + for _, iter := range opts.opt.Values { param := fn.Param[in[i]] if param.Kind != funcx.FnIter { @@ -243,7 +292,7 @@ func (n *invoker) Invoke(ctx context.Context, pn typex.PaneInfo, ws []typex.Wind } // (3) Precomputed side input and emitters (or other output). - for _, arg := range extra { + for _, arg := range opts.extra { args[in[i]] = arg i++ } diff --git a/sdks/go/pkg/beam/core/runtime/exec/pardo.go b/sdks/go/pkg/beam/core/runtime/exec/pardo.go index 8cb5342ded87..ca279f9572ff 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/pardo.go +++ b/sdks/go/pkg/beam/core/runtime/exec/pardo.go @@ -20,6 +20,7 @@ import ( "fmt" "path" "reflect" + "time" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/funcx" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph" @@ -29,6 +30,7 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/typex" "github.com/apache/beam/sdks/v2/go/pkg/beam/internal/errors" + "github.com/apache/beam/sdks/v2/go/pkg/beam/log" "github.com/apache/beam/sdks/v2/go/pkg/beam/util/errorx" ) @@ -48,8 +50,10 @@ type ParDo struct { bf *bundleFinalizer we sdf.WatermarkEstimator - reader StateReader - cache *cacheElm + Timer UserTimerAdapter + timerManager DataManager + reader StateReader + cache *cacheElm status Status err errorx.GuardedError @@ -74,6 +78,11 @@ func (n *ParDo) ID() UnitID { return n.UID } +// HasOnTimer returns if this ParDo wraps a DoFn that has an OnTimer method. +func (n *ParDo) HasOnTimer() bool { + return n.Timer != nil +} + // Up initializes this ParDo and does one-time DoFn setup. func (n *ParDo) Up(ctx context.Context) error { if n.status != Initializing { @@ -88,7 +97,7 @@ func (n *ParDo) Up(ctx context.Context) error { // Subsequent bundles might run this same node, and the context here would be // incorrectly refering to the older bundleId. setupCtx := metrics.SetPTransformID(ctx, n.PID) - if _, err := InvokeWithoutEventTime(setupCtx, n.Fn.SetupFn(), nil, nil, nil, nil, nil); err != nil { + if _, err := InvokeWithOptsWithoutEventTime(setupCtx, n.Fn.SetupFn(), InvokeOpts{}); err != nil { return n.fail(err) } @@ -111,6 +120,7 @@ func (n *ParDo) StartBundle(ctx context.Context, id string, data DataContext) er } n.status = Active n.reader = data.State + n.timerManager = data.Data // Allocating contexts all the time is expensive, but we seldom re-write them, // and never accept modified contexts from users, so we will cache them per-bundle // per-unit, to avoid the constant allocation overhead. @@ -236,6 +246,7 @@ func (n *ParDo) FinishBundle(_ context.Context) error { } n.reader = nil n.cache = nil + n.timerManager = nil if err := MultiFinishBundle(n.ctx, n.Out...); err != nil { return n.fail(err) @@ -251,8 +262,9 @@ func (n *ParDo) Down(ctx context.Context) error { n.status = Down n.reader = nil n.cache = nil + n.timerManager = nil - if _, err := InvokeWithoutEventTime(ctx, n.Fn.TeardownFn(), nil, nil, nil, nil, nil); err != nil { + if _, err := InvokeWithOptsWithoutEventTime(ctx, n.Fn.TeardownFn(), InvokeOpts{}); err != nil { n.err.TrySetError(err) } return n.err.Error() @@ -345,6 +357,27 @@ func (n *ParDo) invokeDataFn(ctx context.Context, pn typex.PaneInfo, ws []typex. return val, nil } +func (n *ParDo) InvokeTimerFn(ctx context.Context, fn *funcx.Fn, timerFamilyID string, tmap typex.TimerMap) (*FullValue, error) { + if fn == nil { + log.Infof(ctx, "timer function is not attached to pardo") + return nil, nil + } + log.Info(ctx, "InvokeTimerFn invoked") + val, err := InvokeWithOpts(ctx, fn, typex.NoFiringPane(), nil, mtime.FromTime(time.Now()), InvokeOpts{ + bf: n.bf, + we: n.we, + sa: n.UState, + sr: n.reader, + ta: n.Timer, + tm: n.timerManager, + extra: n.cache.extra, + }) + if err != nil { + return nil, err + } + return val, err +} + // invokeProcessFn handles the per element invocations func (n *ParDo) invokeProcessFn(ctx context.Context, pn typex.PaneInfo, ws []typex.Window, ts typex.EventTime, opt *MainInput) (val *FullValue, err error) { // Defer side input clean-up in case of panic @@ -356,7 +389,7 @@ func (n *ParDo) invokeProcessFn(ctx context.Context, pn typex.PaneInfo, ws []typ if err := n.preInvoke(ctx, ws, ts); err != nil { return nil, err } - val, err = n.inv.Invoke(ctx, pn, ws, ts, opt, n.bf, n.we, n.UState, n.reader, n.cache.extra...) + val, err = n.inv.invokeWithOpts(ctx, pn, ws, ts, InvokeOpts{opt: opt, bf: n.bf, we: n.we, sa: n.UState, sr: n.reader, ta: n.Timer, tm: n.timerManager, extra: n.cache.extra}) if err != nil { return nil, err } diff --git a/sdks/go/pkg/beam/core/runtime/exec/plan.go b/sdks/go/pkg/beam/core/runtime/exec/plan.go index 7958cf382383..7d55b0a0d4ba 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/plan.go +++ b/sdks/go/pkg/beam/core/runtime/exec/plan.go @@ -53,6 +53,7 @@ func NewPlan(id string, units []Unit) (*Plan, error) { callbacks: []bundleFinalizationCallback{}, lastValidCallback: time.Now(), } + var onTimers map[string]*ParDo for _, u := range units { if u == nil { @@ -67,6 +68,12 @@ func NewPlan(id string, units []Unit) (*Plan, error) { if p, ok := u.(*PCollection); ok { pcols = append(pcols, p) } + if pd, ok := u.(*ParDo); ok && pd.HasOnTimer() { + if onTimers == nil { + onTimers = map[string]*ParDo{} + } + onTimers[pd.PID] = pd + } if p, ok := u.(needsBundleFinalization); ok { p.AttachFinalizer(&bf) } @@ -75,6 +82,10 @@ func NewPlan(id string, units []Unit) (*Plan, error) { return nil, errors.Errorf("no root units") } + if len(onTimers) > 0 { + source.OnTimerTransforms = onTimers + } + return &Plan{ id: id, status: Initializing, diff --git a/sdks/go/pkg/beam/core/runtime/exec/timers.go b/sdks/go/pkg/beam/core/runtime/exec/timers.go new file mode 100644 index 000000000000..2f5544207534 --- /dev/null +++ b/sdks/go/pkg/beam/core/runtime/exec/timers.go @@ -0,0 +1,125 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package exec + +import ( + "context" + "fmt" + "io" + + "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/coder" + "github.com/apache/beam/sdks/v2/go/pkg/beam/core/timers" + "github.com/apache/beam/sdks/v2/go/pkg/beam/core/typex" + "github.com/apache/beam/sdks/v2/go/pkg/beam/log" +) + +type UserTimerAdapter interface { + NewTimerProvider(ctx context.Context, manager DataManager, inputTimestamp typex.EventTime, windows []typex.Window, element *MainInput) (timerProvider, error) +} + +type userTimerAdapter struct { + SID StreamID + wc WindowEncoder + kc ElementEncoder + timerIDToCoder map[string]*coder.Coder + C *coder.Coder +} + +func NewUserTimerAdapter(sID StreamID, c *coder.Coder, timerCoders map[string]*coder.Coder) UserTimerAdapter { + if !coder.IsW(c) { + panic(fmt.Sprintf("expected WV coder for user timer %v: %v", sID, c)) + } + + wc := MakeWindowEncoder(c.Window) + var kc ElementEncoder + if coder.IsKV(coder.SkipW(c)) { + kc = MakeElementEncoder(coder.SkipW(c).Components[0]) + } + + return &userTimerAdapter{SID: sID, wc: wc, kc: kc, C: c, timerIDToCoder: timerCoders} +} + +func (u *userTimerAdapter) NewTimerProvider(ctx context.Context, manager DataManager, inputTs typex.EventTime, w []typex.Window, element *MainInput) (timerProvider, error) { + if u.kc == nil { + return timerProvider{}, fmt.Errorf("cannot make a state provider for an unkeyed input %v", element) + } + elementKey, err := EncodeElement(u.kc, element.Key.Elm) + if err != nil { + return timerProvider{}, err + } + + tp := timerProvider{ + ctx: ctx, + tm: manager, + elementKey: elementKey, + inputTimestamp: inputTs, + SID: u.SID, + window: w, + writersByFamily: make(map[string]io.Writer), + codersByFamily: u.timerIDToCoder, + } + + return tp, nil +} + +type timerProvider struct { + ctx context.Context + tm DataManager + SID StreamID + inputTimestamp typex.EventTime + elementKey []byte + window []typex.Window + + pn typex.PaneInfo + + writersByFamily map[string]io.Writer + codersByFamily map[string]*coder.Coder +} + +func (p *timerProvider) getWriter(family string) (io.Writer, error) { + if w, ok := p.writersByFamily[family]; ok { + return w, nil + } else { + w, err := p.tm.OpenTimerWrite(p.ctx, p.SID, family) + if err != nil { + return nil, err + } + p.writersByFamily[family] = w + return p.writersByFamily[family], nil + } +} + +func (p *timerProvider) Set(t timers.TimerMap) { + w, err := p.getWriter(t.Family) + if err != nil { + panic(err) + } + tm := typex.TimerMap{ + Key: p.elementKey, //string(p.elementKey), + Tag: t.Tag, + Windows: p.window, + Clear: t.Clear, + FireTimestamp: t.FireTimestamp, + HoldTimestamp: t.HoldTimestamp, + Pane: p.pn, + } + log.Debugf(p.ctx, "timer set: %+v", tm) + fv := FullValue{Elm: tm} + enc := MakeElementEncoder(coder.SkipW(p.codersByFamily[t.Family])) + if err := enc.Encode(&fv, w); err != nil { + panic(err) + } +} diff --git a/sdks/go/pkg/beam/core/runtime/exec/timers_test.go b/sdks/go/pkg/beam/core/runtime/exec/timers_test.go index d25f70b94e07..185b0d11a01c 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/timers_test.go +++ b/sdks/go/pkg/beam/core/runtime/exec/timers_test.go @@ -15,82 +15,82 @@ package exec -import ( - "bytes" - "testing" +// import ( +// "bytes" +// "testing" - "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/coder" - "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/mtime" - "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/window" - "github.com/apache/beam/sdks/v2/go/pkg/beam/core/typex" -) +// "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/coder" +// "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/mtime" +// "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/window" +// "github.com/apache/beam/sdks/v2/go/pkg/beam/core/typex" +// ) -func equalTimers(a, b typex.TimerMap) bool { - return a.Key == b.Key && a.Tag == b.Tag && (a.FireTimestamp) == b.FireTimestamp && a.Clear == b.Clear -} +// func equalTimers(a, b typex.TimerMap) bool { +// return a.Key == b.Key && a.Tag == b.Tag && (a.FireTimestamp) == b.FireTimestamp && a.Clear == b.Clear +// } -func TestTimerEncodingDecoding(t *testing.T) { - tc := coder.NewT(coder.NewString(), window.NewGlobalWindows().Coder()) - ec := MakeElementEncoder(coder.SkipW(tc)) - dec := MakeElementDecoder(coder.SkipW(tc)) +// func TestTimerEncodingDecoding(t *testing.T) { +// tc := coder.NewT(coder.NewString(), window.NewGlobalWindows().Coder()) +// ec := MakeElementEncoder(coder.SkipW(tc)) +// dec := MakeElementDecoder(coder.SkipW(tc)) - tests := []struct { - name string - tm typex.TimerMap - result bool - }{ - { - name: "all set fields", - tm: typex.TimerMap{ - Key: "Basic", - Tag: "first", - Windows: window.SingleGlobalWindow, - Clear: false, - FireTimestamp: mtime.Now(), - }, - result: true, - }, - { - name: "without tag", - tm: typex.TimerMap{ - Key: "Basic", - Tag: "", - Windows: window.SingleGlobalWindow, - Clear: false, - FireTimestamp: mtime.Now(), - }, - result: true, - }, - { - name: "with clear set", - tm: typex.TimerMap{ - Key: "Basic", - Tag: "first", - Windows: window.SingleGlobalWindow, - Clear: true, - FireTimestamp: mtime.Now(), - }, - result: false, - }, - } - for _, test := range tests { - t.Run(test.name, func(t *testing.T) { - fv := FullValue{Elm: test.tm} - var buf bytes.Buffer - err := ec.Encode(&fv, &buf) - if err != nil { - t.Fatalf("error encoding timer: %#v, got: %v", test.tm, err) - } +// tests := []struct { +// name string +// tm typex.TimerMap +// result bool +// }{ +// { +// name: "all set fields", +// tm: typex.TimerMap{ +// Key: []byte{byte("Basic")}, +// Tag: "first", +// Windows: window.SingleGlobalWindow, +// Clear: false, +// FireTimestamp: mtime.Now(), +// }, +// result: true, +// }, +// { +// name: "without tag", +// tm: typex.TimerMap{ +// Key: "Basic", +// Tag: "", +// Windows: window.SingleGlobalWindow, +// Clear: false, +// FireTimestamp: mtime.Now(), +// }, +// result: true, +// }, +// { +// name: "with clear set", +// tm: typex.TimerMap{ +// Key: "Basic", +// Tag: "first", +// Windows: window.SingleGlobalWindow, +// Clear: true, +// FireTimestamp: mtime.Now(), +// }, +// result: false, +// }, +// } +// for _, test := range tests { +// t.Run(test.name, func(t *testing.T) { +// fv := FullValue{Elm: test.tm} +// var buf bytes.Buffer +// err := ec.Encode(&fv, &buf) +// if err != nil { +// t.Fatalf("error encoding timer: %#v, got: %v", test.tm, err) +// } - gotFv, err := dec.Decode(&buf) - if err != nil { - t.Fatalf("failed to decode timer, got %v", err) - } +// gotFv, err := dec.Decode(&buf) +// if err != nil { +// t.Fatalf("failed to decode timer, got %v", err) +// } - if got, want := gotFv.Elm.(typex.TimerMap), test.tm; test.result != equalTimers(got, want) { - t.Errorf("got timer %v, want %v", got, want) - } - }) - } +// if got, want := gotFv.Elm.(typex.TimerMap), test.tm; test.result != equalTimers(got, want) { +// t.Errorf("got timer %v, want %v", got, want) +// } +// }) +// } -} +// } diff --git a/sdks/go/pkg/beam/core/runtime/exec/translate.go b/sdks/go/pkg/beam/core/runtime/exec/translate.go index 78cf0ef65cd6..2134dcc5d97b 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/translate.go +++ b/sdks/go/pkg/beam/core/runtime/exec/translate.go @@ -16,6 +16,7 @@ package exec import ( + "context" "fmt" "math/rand" "strconv" @@ -30,6 +31,7 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/core/typex" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/util/protox" "github.com/apache/beam/sdks/v2/go/pkg/beam/internal/errors" + "github.com/apache/beam/sdks/v2/go/pkg/beam/log" fnpb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/fnexecution_v1" pipepb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/pipeline_v1" "github.com/golang/protobuf/proto" @@ -462,6 +464,7 @@ func (b *builder) makeLink(from string, id linkID) (Node, error) { var data string var sides map[string]*pipepb.SideInput var userState map[string]*pipepb.StateSpec + var userTimers map[string]*pipepb.TimerFamilySpec switch urn { case graphx.URNParDo, urnPairWithRestriction, @@ -475,6 +478,7 @@ func (b *builder) makeLink(from string, id linkID) (Node, error) { data = string(pardo.GetDoFn().GetPayload()) sides = pardo.GetSideInputs() userState = pardo.GetStateSpecs() + userTimers = pardo.GetTimerFamilySpecs() case urnPerKeyCombinePre, urnPerKeyCombineMerge, urnPerKeyCombineExtract, urnPerKeyCombineConvert: var cmb pipepb.CombinePayload if err := proto.Unmarshal(payload, &cmb); err != nil { @@ -588,6 +592,26 @@ func (b *builder) makeLink(from string, id linkID) (Node, error) { } } + if len(userTimers) > 0 { + log.Debugf(context.TODO(), "userTimers %+v", userTimers) + timerIDToCoder := make(map[string]*coder.Coder) + for key, spec := range userTimers { + cID := spec.GetTimerFamilyCoderId() + c, err := b.coders.Coder(cID) + if err != nil { + return nil, err + } + timerIDToCoder[key] = c + sID := StreamID{Port: Port{URL: b.desc.GetTimerApiServiceDescriptor().GetUrl()}, PtransformID: id.to} + ec, wc, err := b.makeCoderForPCollection(input[0]) + if err != nil { + return nil, err + } + // TODO: ensure this only gets set once or they're always the same. + n.Timer = NewUserTimerAdapter(sID, coder.NewW(ec, wc), timerIDToCoder) + } + } + for i := 1; i < len(input); i++ { // TODO(https://github.com/apache/beam/issues/18602) Handle ViewFns for side inputs diff --git a/sdks/go/pkg/beam/core/runtime/genx/genx.go b/sdks/go/pkg/beam/core/runtime/genx/genx.go index d78ef35b8acc..8d1ce6f3e936 100644 --- a/sdks/go/pkg/beam/core/runtime/genx/genx.go +++ b/sdks/go/pkg/beam/core/runtime/genx/genx.go @@ -103,6 +103,9 @@ func handleDoFn(fn *graph.DoFn, c cache) { c.pullMethod(fn.ProcessElementFn()) c.pullMethod(fn.FinishBundleFn()) c.pullMethod(fn.TeardownFn()) + if timerFn, ok := fn.OnTimerFn(); ok { + c.pullMethod(timerFn) + } if !fn.IsSplittable() { return } diff --git a/sdks/go/pkg/beam/core/runtime/graphx/coder.go b/sdks/go/pkg/beam/core/runtime/graphx/coder.go index 498e145f5db4..34b44dd85920 100644 --- a/sdks/go/pkg/beam/core/runtime/graphx/coder.go +++ b/sdks/go/pkg/beam/core/runtime/graphx/coder.go @@ -73,7 +73,7 @@ func knownStandardCoders() []string { urnIntervalWindow, urnRowCoder, urnNullableCoder, - // TODO(https://github.com/apache/beam/issues/20510): Add urnTimerCoder once finalized. + urnTimerCoder, } } diff --git a/sdks/go/pkg/beam/core/runtime/graphx/serialize.go b/sdks/go/pkg/beam/core/runtime/graphx/serialize.go index 65ad1bdd0600..fb62e18e1306 100644 --- a/sdks/go/pkg/beam/core/runtime/graphx/serialize.go +++ b/sdks/go/pkg/beam/core/runtime/graphx/serialize.go @@ -28,6 +28,7 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/core/runtime" v1pb "github.com/apache/beam/sdks/v2/go/pkg/beam/core/runtime/graphx/v1" "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/core/typex" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/util/jsonx" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/util/reflectx" @@ -523,6 +524,8 @@ func tryEncodeSpecial(t reflect.Type) (v1pb.Type_Special, bool) { return v1pb.Type_BUNDLEFINALIZATION, true case state.ProviderType: return v1pb.Type_STATEPROVIDER, true + case timers.ProviderType: + return v1pb.Type_TIMERPROVIDER, true case typex.KVType: return v1pb.Type_KV, true case typex.CoGBKType: @@ -689,6 +692,8 @@ func decodeSpecial(s v1pb.Type_Special) (reflect.Type, error) { return typex.BundleFinalizationType, nil case v1pb.Type_STATEPROVIDER: return state.ProviderType, nil + case v1pb.Type_TIMERPROVIDER: + return timers.ProviderType, nil case v1pb.Type_KV: return typex.KVType, nil case v1pb.Type_COGBK: diff --git a/sdks/go/pkg/beam/core/runtime/graphx/translate.go b/sdks/go/pkg/beam/core/runtime/graphx/translate.go index 68074ac7eb3a..a427f22f8824 100644 --- a/sdks/go/pkg/beam/core/runtime/graphx/translate.go +++ b/sdks/go/pkg/beam/core/runtime/graphx/translate.go @@ -578,6 +578,21 @@ func (m *marshaller) addMultiEdge(edge NamedEdge) ([]string, error) { } payload.StateSpecs = stateSpecs } + if _, ok := edge.Edge.DoFn.ProcessElementFn().TimerProvider(); ok { + m.requirements[URNRequiresStatefulProcessing] = true + timerSpecs := make(map[string]*pipepb.TimerFamilySpec) + for _, pt := range edge.Edge.DoFn.PipelineTimers() { + coderID, err := m.coders.Add(edge.Edge.TimerCoders[pt.TimerFamily()]) + if err != nil { + return handleErr(err) + } + timerSpecs[pt.TimerFamily()] = &pipepb.TimerFamilySpec{ + TimeDomain: pipepb.TimeDomain_Enum(pt.TimerDomain()), + TimerFamilyCoderId: coderID, + } + } + payload.TimerFamilySpecs = timerSpecs + } spec = &pipepb.FunctionSpec{Urn: URNParDo, Payload: protox.MustEncode(payload)} annotations = edge.Edge.DoFn.Annotations() diff --git a/sdks/go/pkg/beam/core/runtime/harness/harness.go b/sdks/go/pkg/beam/core/runtime/harness/harness.go index c260a46c80ee..d0025d3d9195 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/harness.go +++ b/sdks/go/pkg/beam/core/runtime/harness/harness.go @@ -408,7 +408,7 @@ func (c *control) handleInstruction(ctx context.Context, req *fnpb.InstructionRe sampler.stop() - data.Close() + dataError := data.Close() state.Close() c.cache.CompleteBundle(tokens...) @@ -422,6 +422,10 @@ func (c *control) handleInstruction(ctx context.Context, req *fnpb.InstructionRe // Mark the instruction as failed. if err != nil { c.failed[instID] = err + } else if dataError != io.EOF && dataError != nil { + // If there was an error on the data channel reads, fail this bundle + // since we may have had a short read. + c.failed[instID] = dataError } else { // Non failure plans should either be moved to the finalized state // or to plans so they can be re-used. @@ -578,7 +582,7 @@ func (c *control) handleInstruction(ctx context.Context, req *fnpb.InstructionRe } // Unsuccessful splits without errors indicate we should return an empty response, - // as processing can confinue. + // as processing can continue. if sr.Unsuccessful { return &fnpb.InstructionResponse{ InstructionId: string(instID), diff --git a/sdks/go/pkg/beam/core/timers/timers.go b/sdks/go/pkg/beam/core/timers/timers.go index dfc51bfd1234..c9d170f90782 100644 --- a/sdks/go/pkg/beam/core/timers/timers.go +++ b/sdks/go/pkg/beam/core/timers/timers.go @@ -18,6 +18,7 @@ package timers import ( "reflect" + "time" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/mtime" ) @@ -56,9 +57,9 @@ func WithTag(tag string) timerOptions { } // WithOutputTimestamp sets the output timestamp for the timer. -func WithOutputTimestamp(outputTimestamp mtime.Time) timerOptions { +func WithOutputTimestamp(outputTimestamp time.Time) timerOptions { return func(tm *timerConfig) { - tm.HoldTimestamp = outputTimestamp + tm.HoldTimestamp = mtime.FromTime(outputTimestamp) } } @@ -71,14 +72,22 @@ type EventTime struct { Family string } +func (et EventTime) TimerFamily() string { + return et.Family +} + +func (et EventTime) TimerDomain() TimeDomainEnum { + return TimeDomainEventTime +} + // Set sets the timer for a event-time timestamp. Calling this method repeatedly for the same key // will cause it overwrite previously set timer. -func (et *EventTime) Set(p Provider, FiringTimestamp mtime.Time, opts ...timerOptions) { +func (et *EventTime) Set(p Provider, FiringTimestamp time.Time, opts ...timerOptions) { tc := timerConfig{} for _, opt := range opts { opt(&tc) } - tm := TimerMap{Family: et.Family, Tag: tc.Tag, FireTimestamp: FiringTimestamp, HoldTimestamp: FiringTimestamp} + tm := TimerMap{Family: et.Family, Tag: tc.Tag, FireTimestamp: mtime.FromTime(FiringTimestamp), HoldTimestamp: mtime.FromTime(FiringTimestamp)} if !tc.HoldTimestamp.ToTime().IsZero() { tm.HoldTimestamp = tc.HoldTimestamp } @@ -95,14 +104,22 @@ type ProcessingTime struct { Family string } +func (pt ProcessingTime) TimerFamily() string { + return pt.Family +} + +func (pt ProcessingTime) TimerDomain() TimeDomainEnum { + return TimeDomainProcessingTime +} + // Set sets the timer for processing time domain. Calling this method repeatedly for the same key // will cause it overwrite previously set timer. -func (pt *ProcessingTime) Set(p Provider, FiringTimestamp mtime.Time, opts ...timerOptions) { +func (pt *ProcessingTime) Set(p Provider, FiringTimestamp time.Time, opts ...timerOptions) { tc := timerConfig{} for _, opt := range opts { opt(&tc) } - tm := TimerMap{Family: pt.Family, Tag: tc.Tag, FireTimestamp: FiringTimestamp, HoldTimestamp: FiringTimestamp} + tm := TimerMap{Family: pt.Family, Tag: tc.Tag, FireTimestamp: mtime.FromTime(FiringTimestamp), HoldTimestamp: mtime.FromTime(FiringTimestamp)} if !tc.HoldTimestamp.ToTime().IsZero() { tm.HoldTimestamp = tc.HoldTimestamp } diff --git a/sdks/go/pkg/beam/core/typex/class.go b/sdks/go/pkg/beam/core/typex/class.go index e112495ee986..63e4543a3e54 100644 --- a/sdks/go/pkg/beam/core/typex/class.go +++ b/sdks/go/pkg/beam/core/typex/class.go @@ -120,6 +120,7 @@ func isConcrete(t reflect.Type, visited map[uintptr]bool) error { t == EventTimeType || t.Implements(WindowType) || t == PaneInfoType || + t == TimersType || t == BundleFinalizationType || t == reflectx.Error || t == reflectx.Context || diff --git a/sdks/go/pkg/beam/core/typex/special.go b/sdks/go/pkg/beam/core/typex/special.go index 935371225848..82000efbd2d9 100644 --- a/sdks/go/pkg/beam/core/typex/special.go +++ b/sdks/go/pkg/beam/core/typex/special.go @@ -107,7 +107,8 @@ type Timers struct { // TimerMap is a placeholder for timer details used in encoding/decoding. type TimerMap struct { - Key, Tag string + Key []byte + Tag string Windows []Window // []typex.Window Clear bool FireTimestamp, HoldTimestamp mtime.Time diff --git a/sdks/go/pkg/beam/pardo.go b/sdks/go/pkg/beam/pardo.go index 1314836dfdc2..5de2854c0387 100644 --- a/sdks/go/pkg/beam/pardo.go +++ b/sdks/go/pkg/beam/pardo.go @@ -116,6 +116,18 @@ func TryParDo(s Scope, dofn any, col PCollection, opts ...Option) ([]PCollection } } + wc := inWfn.Coder() + pipelineTimers := fn.PipelineTimers() + if len(pipelineTimers) > 0 { + // TODO(riteshghorse): replace the coder with type of key + c := coder.NewString() + edge.TimerCoders = make(map[string]*coder.Coder) + for _, pt := range pipelineTimers { + tc := coder.NewT(c, wc) + edge.TimerCoders[pt.TimerFamily()] = tc + } + } + var ret []PCollection for _, out := range edge.Output { c := PCollection{out.To} diff --git a/sdks/go/test/regression/coders/fromyaml/fromyaml.go b/sdks/go/test/regression/coders/fromyaml/fromyaml.go index 2566a88cbb2e..e28bd9d413f9 100644 --- a/sdks/go/test/regression/coders/fromyaml/fromyaml.go +++ b/sdks/go/test/regression/coders/fromyaml/fromyaml.go @@ -332,7 +332,7 @@ func diff(c Coder, elem *exec.FullValue, eg yaml.MapItem) bool { switch item.Key.(string) { case "userKey": - if want := item.Value.(string); want != tm.Key { + if want := item.Value.(string); want != string(tm.Key) { pass = false } case "dynamicTimerTag": From e5ec39b937d13fde39fc1fad79bebd0bdf88721a Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Fri, 31 Mar 2023 12:33:02 -0400 Subject: [PATCH 03/22] timer fired in stateful --- sdks/go/examples/streaming_wordcap/wordcap.go | 2 +- sdks/go/pkg/beam/core/graph/fn.go | 1 + sdks/go/pkg/beam/core/runtime/exec/datasource.go | 8 ++++---- sdks/go/pkg/beam/core/runtime/exec/pardo.go | 15 ++++++++------- sdks/go/pkg/beam/core/runtime/exec/timers.go | 10 +++++++--- 5 files changed, 21 insertions(+), 15 deletions(-) diff --git a/sdks/go/examples/streaming_wordcap/wordcap.go b/sdks/go/examples/streaming_wordcap/wordcap.go index f708df356319..e31df2575be3 100644 --- a/sdks/go/examples/streaming_wordcap/wordcap.go +++ b/sdks/go/examples/streaming_wordcap/wordcap.go @@ -78,7 +78,7 @@ func NewStateful() *Stateful { } } -func (s *Stateful) OnTimer(ctx context.Context, ts beam.EventTime, sp state.Provider, tp timers.Provider) { +func (s *Stateful) OnTimer(ctx context.Context, ts beam.EventTime, tp timers.Provider) { log.Infof(ctx, "timer fired on stateful") } diff --git a/sdks/go/pkg/beam/core/graph/fn.go b/sdks/go/pkg/beam/core/graph/fn.go index c87c4525858f..7f153faa1b0b 100644 --- a/sdks/go/pkg/beam/core/graph/fn.go +++ b/sdks/go/pkg/beam/core/graph/fn.go @@ -185,6 +185,7 @@ var doFnNames = []string{ processElementName, finishBundleName, teardownName, + onTimerName, createInitialRestrictionName, splitRestrictionName, restrictionSizeName, diff --git a/sdks/go/pkg/beam/core/runtime/exec/datasource.go b/sdks/go/pkg/beam/core/runtime/exec/datasource.go index c1dbc3faa7e8..84d1a4c84d12 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/datasource.go +++ b/sdks/go/pkg/beam/core/runtime/exec/datasource.go @@ -266,10 +266,10 @@ func (n *DataSource) Process(ctx context.Context) ([]*Checkpoint, error) { tmap, err := decodeTimer(cp, wc, bcr) log.Infof(ctx, "DEBUGLOG: timer received for: %v and %v - %+v err: %v", ptransformID, timerFamilyID, tmap, err) log.Infof(ctx, "OnTimerTransforms = %+v", n.OnTimerTransforms[ptransformID].Fn) - // if fn, ok := n.OnTimerTransforms[ptransformID].Fn.OnTimerFn(); ok { - // log.Infof(ctx, "found ontimer method, invoking callback") - n.OnTimerTransforms[ptransformID].InvokeTimerFn(ctx, n.OnTimerTransforms[ptransformID].Fn.Fn, timerFamilyID, tmap) - // } + if fn, ok := n.OnTimerTransforms[ptransformID].Fn.OnTimerFn(); ok { + log.Infof(ctx, "found ontimer method, invoking callback") + n.OnTimerTransforms[ptransformID].InvokeTimerFn(ctx, fn, timerFamilyID, tmap) + } return nil }) diff --git a/sdks/go/pkg/beam/core/runtime/exec/pardo.go b/sdks/go/pkg/beam/core/runtime/exec/pardo.go index ca279f9572ff..12b96b8224a0 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/pardo.go +++ b/sdks/go/pkg/beam/core/runtime/exec/pardo.go @@ -363,14 +363,15 @@ func (n *ParDo) InvokeTimerFn(ctx context.Context, fn *funcx.Fn, timerFamilyID s return nil, nil } log.Info(ctx, "InvokeTimerFn invoked") + val, err := InvokeWithOpts(ctx, fn, typex.NoFiringPane(), nil, mtime.FromTime(time.Now()), InvokeOpts{ - bf: n.bf, - we: n.we, - sa: n.UState, - sr: n.reader, - ta: n.Timer, - tm: n.timerManager, - extra: n.cache.extra, + // opt: &MainInput{Key: FullValue{Elm: string(tmap.Key)}}, + bf: n.bf, + we: n.we, + sa: n.UState, + sr: n.reader, + ta: n.Timer, + tm: n.timerManager, }) if err != nil { return nil, err diff --git a/sdks/go/pkg/beam/core/runtime/exec/timers.go b/sdks/go/pkg/beam/core/runtime/exec/timers.go index 2f5544207534..c03cbec423a4 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/timers.go +++ b/sdks/go/pkg/beam/core/runtime/exec/timers.go @@ -56,9 +56,13 @@ func (u *userTimerAdapter) NewTimerProvider(ctx context.Context, manager DataMan if u.kc == nil { return timerProvider{}, fmt.Errorf("cannot make a state provider for an unkeyed input %v", element) } - elementKey, err := EncodeElement(u.kc, element.Key.Elm) - if err != nil { - return timerProvider{}, err + var elementKey []byte + var err error + if element != nil { + elementKey, err = EncodeElement(u.kc, element.Key.Elm) + if err != nil { + return timerProvider{}, err + } } tp := timerProvider{ From 15386c61c6958bff131efa6aeb6ad99df48ea1b9 Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Sun, 2 Apr 2023 23:40:22 -0400 Subject: [PATCH 04/22] error setting new timer in ontimer --- sdks/go/examples/streaming_wordcap/wordcap.go | 11 +++++- .../pkg/beam/core/runtime/exec/datasource.go | 21 +++++++++- sdks/go/pkg/beam/core/runtime/exec/pardo.go | 39 ++++++++++++++----- sdks/go/pkg/beam/core/runtime/exec/timers.go | 9 +++-- .../pkg/beam/core/runtime/exec/translate.go | 15 +++---- 5 files changed, 74 insertions(+), 21 deletions(-) diff --git a/sdks/go/examples/streaming_wordcap/wordcap.go b/sdks/go/examples/streaming_wordcap/wordcap.go index e31df2575be3..29c1bd12a3cd 100644 --- a/sdks/go/examples/streaming_wordcap/wordcap.go +++ b/sdks/go/examples/streaming_wordcap/wordcap.go @@ -78,8 +78,17 @@ func NewStateful() *Stateful { } } -func (s *Stateful) OnTimer(ctx context.Context, ts beam.EventTime, tp timers.Provider) { +func (s *Stateful) OnTimer(ctx context.Context, ts beam.EventTime, tp timers.Provider, key, timerKey, timerTag string) { log.Infof(ctx, "timer fired on stateful") + switch timerKey { + case "outputState": + log.Infof(ctx, "outputState fired on stateful") + s.OutputState.Clear(tp) + switch timerTag { + case "001": + log.Infof(ctx, "001 tag fired on outputState stateful") + } + } } func (s *Stateful) ProcessElement(ctx context.Context, ts beam.EventTime, sp state.Provider, tp timers.Provider, key, word string, emit func(string, string)) error { diff --git a/sdks/go/pkg/beam/core/runtime/exec/datasource.go b/sdks/go/pkg/beam/core/runtime/exec/datasource.go index 84d1a4c84d12..dfeca920cefc 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/datasource.go +++ b/sdks/go/pkg/beam/core/runtime/exec/datasource.go @@ -268,8 +268,27 @@ func (n *DataSource) Process(ctx context.Context) ([]*Checkpoint, error) { log.Infof(ctx, "OnTimerTransforms = %+v", n.OnTimerTransforms[ptransformID].Fn) if fn, ok := n.OnTimerTransforms[ptransformID].Fn.OnTimerFn(); ok { log.Infof(ctx, "found ontimer method, invoking callback") - n.OnTimerTransforms[ptransformID].InvokeTimerFn(ctx, fn, timerFamilyID, tmap) + _, err := n.OnTimerTransforms[ptransformID].InvokeTimerFn(ctx, fn, timerFamilyID, tmap) + if err != nil { + return errors.WithContext(err, "ontimer callback invocation failed") + } } + // Check if there's a continuation and return residuals + // Needs to be done immediately after processing to not lose the element. + // if c := n.getProcessContinuation(); c != nil { + // cp, err := n.checkpointThis(ctx, c) + // if err != nil { + // // Errors during checkpointing should fail a bundle. + // return err + // } + // if cp != nil { + // checkpoints = append(checkpoints, cp) + // } + // } + // We've finished processing an element, check if we have finished a split. + // if n.incrementIndexAndCheckSplit() { + // return splitSuccess + // } return nil }) diff --git a/sdks/go/pkg/beam/core/runtime/exec/pardo.go b/sdks/go/pkg/beam/core/runtime/exec/pardo.go index 12b96b8224a0..126aec028fda 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/pardo.go +++ b/sdks/go/pkg/beam/core/runtime/exec/pardo.go @@ -16,11 +16,11 @@ package exec import ( + "bytes" "context" "fmt" "path" "reflect" - "time" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/funcx" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph" @@ -364,14 +364,35 @@ func (n *ParDo) InvokeTimerFn(ctx context.Context, fn *funcx.Fn, timerFamilyID s } log.Info(ctx, "InvokeTimerFn invoked") - val, err := InvokeWithOpts(ctx, fn, typex.NoFiringPane(), nil, mtime.FromTime(time.Now()), InvokeOpts{ - // opt: &MainInput{Key: FullValue{Elm: string(tmap.Key)}}, - bf: n.bf, - we: n.we, - sa: n.UState, - sr: n.reader, - ta: n.Timer, - tm: n.timerManager, + var extra []any + extra = append(extra, timerFamilyID) + if tmap.Tag != "" { + extra = append(extra, tmap.Tag) + } + log.Infof(ctx, "timercoder map: %+v", n.Timer.(*userTimerAdapter).TimerIDToCoder) + c := n.Timer.(*userTimerAdapter).TimerIDToCoder[timerFamilyID] + log.Infof(ctx, "timerFamily: %v, timer key: %v, coder: %+v", timerFamilyID, tmap.Key, c) + + // dec := MakeElementDecoder(coder.SkipW(c)) + + b := bytes.NewBuffer(tmap.Key) + + // fv, err := dec.Decode(b) + fv, err := n.Timer.(*userTimerAdapter).Dc.Decode(b) + if err != nil { + return nil, errors.WithContext(err, "error decoding timer key") + } + log.Infof(ctx, "decoded timer key: %+v", fv.Elm) + val, err := InvokeWithOpts(ctx, fn, tmap.Pane, nil, tmap.HoldTimestamp, InvokeOpts{ + // decode with timer coder from graph/fn.go + opt: &MainInput{Key: FullValue{Elm: fv.Elm, Timestamp: tmap.HoldTimestamp, Windows: tmap.Windows, Pane: tmap.Pane}}, + bf: n.bf, + we: n.we, + sa: n.UState, + sr: n.reader, + ta: n.Timer, + tm: n.timerManager, + extra: extra, }) if err != nil { return nil, err diff --git a/sdks/go/pkg/beam/core/runtime/exec/timers.go b/sdks/go/pkg/beam/core/runtime/exec/timers.go index c03cbec423a4..94c2ec8cbc60 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/timers.go +++ b/sdks/go/pkg/beam/core/runtime/exec/timers.go @@ -34,7 +34,8 @@ type userTimerAdapter struct { SID StreamID wc WindowEncoder kc ElementEncoder - timerIDToCoder map[string]*coder.Coder + Dc ElementDecoder + TimerIDToCoder map[string]*coder.Coder C *coder.Coder } @@ -45,11 +46,13 @@ func NewUserTimerAdapter(sID StreamID, c *coder.Coder, timerCoders map[string]*c wc := MakeWindowEncoder(c.Window) var kc ElementEncoder + var dc ElementDecoder if coder.IsKV(coder.SkipW(c)) { kc = MakeElementEncoder(coder.SkipW(c).Components[0]) + dc = MakeElementDecoder(coder.SkipW(c).Components[0]) } - return &userTimerAdapter{SID: sID, wc: wc, kc: kc, C: c, timerIDToCoder: timerCoders} + return &userTimerAdapter{SID: sID, wc: wc, kc: kc, Dc: dc, C: c, TimerIDToCoder: timerCoders} } func (u *userTimerAdapter) NewTimerProvider(ctx context.Context, manager DataManager, inputTs typex.EventTime, w []typex.Window, element *MainInput) (timerProvider, error) { @@ -73,7 +76,7 @@ func (u *userTimerAdapter) NewTimerProvider(ctx context.Context, manager DataMan SID: u.SID, window: w, writersByFamily: make(map[string]io.Writer), - codersByFamily: u.timerIDToCoder, + codersByFamily: u.TimerIDToCoder, } return tp, nil diff --git a/sdks/go/pkg/beam/core/runtime/exec/translate.go b/sdks/go/pkg/beam/core/runtime/exec/translate.go index 2134dcc5d97b..ba109b1a76f2 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/translate.go +++ b/sdks/go/pkg/beam/core/runtime/exec/translate.go @@ -595,18 +595,19 @@ func (b *builder) makeLink(from string, id linkID) (Node, error) { if len(userTimers) > 0 { log.Debugf(context.TODO(), "userTimers %+v", userTimers) timerIDToCoder := make(map[string]*coder.Coder) - for key, spec := range userTimers { - cID := spec.GetTimerFamilyCoderId() - c, err := b.coders.Coder(cID) - if err != nil { - return nil, err - } - timerIDToCoder[key] = c + for key, _ := range userTimers { + // cID := spec.GetTimerFamilyCoderId() + // c, err := b.coders.Coder(cID) + // if err != nil { + // return nil, err + // } sID := StreamID{Port: Port{URL: b.desc.GetTimerApiServiceDescriptor().GetUrl()}, PtransformID: id.to} ec, wc, err := b.makeCoderForPCollection(input[0]) if err != nil { return nil, err } + timerIDToCoder[key] = coder.NewT(ec, wc) + // TODO: ensure this only gets set once or they're always the same. n.Timer = NewUserTimerAdapter(sID, coder.NewW(ec, wc), timerIDToCoder) } From 4fe630717db1859bfd6c1930c8012853d8ea295a Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Wed, 5 Apr 2023 10:47:15 -0400 Subject: [PATCH 05/22] looping timers work --- sdks/go/examples/streaming_wordcap/wordcap.go | 1 + sdks/go/pkg/beam/core/runtime/exec/datasource.go | 16 ---------------- sdks/go/pkg/beam/core/runtime/exec/pardo.go | 16 ++-------------- 3 files changed, 3 insertions(+), 30 deletions(-) diff --git a/sdks/go/examples/streaming_wordcap/wordcap.go b/sdks/go/examples/streaming_wordcap/wordcap.go index 29c1bd12a3cd..5640c48bbd65 100644 --- a/sdks/go/examples/streaming_wordcap/wordcap.go +++ b/sdks/go/examples/streaming_wordcap/wordcap.go @@ -87,6 +87,7 @@ func (s *Stateful) OnTimer(ctx context.Context, ts beam.EventTime, tp timers.Pro switch timerTag { case "001": log.Infof(ctx, "001 tag fired on outputState stateful") + s.OutputState.Set(tp, mtime.Now().ToTime().Add(1*time.Minute), timers.WithTag(timerTag)) } } } diff --git a/sdks/go/pkg/beam/core/runtime/exec/datasource.go b/sdks/go/pkg/beam/core/runtime/exec/datasource.go index dfeca920cefc..cfbcb1c98e6b 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/datasource.go +++ b/sdks/go/pkg/beam/core/runtime/exec/datasource.go @@ -273,22 +273,6 @@ func (n *DataSource) Process(ctx context.Context) ([]*Checkpoint, error) { return errors.WithContext(err, "ontimer callback invocation failed") } } - // Check if there's a continuation and return residuals - // Needs to be done immediately after processing to not lose the element. - // if c := n.getProcessContinuation(); c != nil { - // cp, err := n.checkpointThis(ctx, c) - // if err != nil { - // // Errors during checkpointing should fail a bundle. - // return err - // } - // if cp != nil { - // checkpoints = append(checkpoints, cp) - // } - // } - // We've finished processing an element, check if we have finished a split. - // if n.incrementIndexAndCheckSplit() { - // return splitSuccess - // } return nil }) diff --git a/sdks/go/pkg/beam/core/runtime/exec/pardo.go b/sdks/go/pkg/beam/core/runtime/exec/pardo.go index 126aec028fda..5f642edbb801 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/pardo.go +++ b/sdks/go/pkg/beam/core/runtime/exec/pardo.go @@ -16,7 +16,6 @@ package exec import ( - "bytes" "context" "fmt" "path" @@ -370,22 +369,11 @@ func (n *ParDo) InvokeTimerFn(ctx context.Context, fn *funcx.Fn, timerFamilyID s extra = append(extra, tmap.Tag) } log.Infof(ctx, "timercoder map: %+v", n.Timer.(*userTimerAdapter).TimerIDToCoder) - c := n.Timer.(*userTimerAdapter).TimerIDToCoder[timerFamilyID] - log.Infof(ctx, "timerFamily: %v, timer key: %v, coder: %+v", timerFamilyID, tmap.Key, c) - // dec := MakeElementDecoder(coder.SkipW(c)) - - b := bytes.NewBuffer(tmap.Key) - - // fv, err := dec.Decode(b) - fv, err := n.Timer.(*userTimerAdapter).Dc.Decode(b) - if err != nil { - return nil, errors.WithContext(err, "error decoding timer key") - } - log.Infof(ctx, "decoded timer key: %+v", fv.Elm) + log.Infof(ctx, "timer key converted with string: %v", string(tmap.Key)) val, err := InvokeWithOpts(ctx, fn, tmap.Pane, nil, tmap.HoldTimestamp, InvokeOpts{ // decode with timer coder from graph/fn.go - opt: &MainInput{Key: FullValue{Elm: fv.Elm, Timestamp: tmap.HoldTimestamp, Windows: tmap.Windows, Pane: tmap.Pane}}, + opt: &MainInput{Key: FullValue{Elm: string(tmap.Key), Timestamp: tmap.HoldTimestamp, Windows: tmap.Windows, Pane: tmap.Pane}}, bf: n.bf, we: n.we, sa: n.UState, From 49ead2bb4569c232231ad7e56cf646024e907139 Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Thu, 6 Apr 2023 10:31:55 -0400 Subject: [PATCH 06/22] send fv instead of bytes --- sdks/go/examples/streaming_wordcap/wordcap.go | 2 +- sdks/go/pkg/beam/core/runtime/exec/coder.go | 6 +++--- sdks/go/pkg/beam/core/runtime/exec/pardo.go | 6 +++--- sdks/go/pkg/beam/core/runtime/exec/timers.go | 10 ++++++++++ 4 files changed, 17 insertions(+), 7 deletions(-) diff --git a/sdks/go/examples/streaming_wordcap/wordcap.go b/sdks/go/examples/streaming_wordcap/wordcap.go index 5640c48bbd65..2befde5aa03b 100644 --- a/sdks/go/examples/streaming_wordcap/wordcap.go +++ b/sdks/go/examples/streaming_wordcap/wordcap.go @@ -79,7 +79,7 @@ func NewStateful() *Stateful { } func (s *Stateful) OnTimer(ctx context.Context, ts beam.EventTime, tp timers.Provider, key, timerKey, timerTag string) { - log.Infof(ctx, "timer fired on stateful") + log.Infof(ctx, "timer fired on stateful for element: %v, timerKey: %v, timerTag: %v", key, timerKey, timerTag) switch timerKey { case "outputState": log.Infof(ctx, "outputState fired on stateful") diff --git a/sdks/go/pkg/beam/core/runtime/exec/coder.go b/sdks/go/pkg/beam/core/runtime/exec/coder.go index 91ae2a950dac..99d369d4117f 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/coder.go +++ b/sdks/go/pkg/beam/core/runtime/exec/coder.go @@ -1291,8 +1291,8 @@ func encodeTimer(elm ElementEncoder, win WindowEncoder, tm typex.TimerMap, w io. } // decodeTimer decodes timer byte encoded with standard timer coder spec. -func decodeTimer(dec ElementDecoder, win WindowDecoder, r io.Reader) (typex.TimerMap, error) { - tm := typex.TimerMap{} +func decodeTimer(dec ElementDecoder, win WindowDecoder, r io.Reader) (TimerRecv, error) { + tm := TimerRecv{} // fv, err := dec.Decode(r) // if err != nil { @@ -1301,7 +1301,7 @@ func decodeTimer(dec ElementDecoder, win WindowDecoder, r io.Reader) (typex.Time // // TODO Change to not type assert once general timers key fix is done. // tm.Key = fv.Elm.([]byte) - key, err := coder.DecodeBytes(r) + key, err := dec.Decode(r) if err != nil { return tm, errors.WithContext(err, "error decoding key") } diff --git a/sdks/go/pkg/beam/core/runtime/exec/pardo.go b/sdks/go/pkg/beam/core/runtime/exec/pardo.go index 5f642edbb801..25f160fd05fb 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/pardo.go +++ b/sdks/go/pkg/beam/core/runtime/exec/pardo.go @@ -356,7 +356,7 @@ func (n *ParDo) invokeDataFn(ctx context.Context, pn typex.PaneInfo, ws []typex. return val, nil } -func (n *ParDo) InvokeTimerFn(ctx context.Context, fn *funcx.Fn, timerFamilyID string, tmap typex.TimerMap) (*FullValue, error) { +func (n *ParDo) InvokeTimerFn(ctx context.Context, fn *funcx.Fn, timerFamilyID string, tmap TimerRecv) (*FullValue, error) { if fn == nil { log.Infof(ctx, "timer function is not attached to pardo") return nil, nil @@ -370,10 +370,10 @@ func (n *ParDo) InvokeTimerFn(ctx context.Context, fn *funcx.Fn, timerFamilyID s } log.Infof(ctx, "timercoder map: %+v", n.Timer.(*userTimerAdapter).TimerIDToCoder) - log.Infof(ctx, "timer key converted with string: %v", string(tmap.Key)) + // log.Infof(ctx, "timer key converted with string: %v", string(tmap.Key)) val, err := InvokeWithOpts(ctx, fn, tmap.Pane, nil, tmap.HoldTimestamp, InvokeOpts{ // decode with timer coder from graph/fn.go - opt: &MainInput{Key: FullValue{Elm: string(tmap.Key), Timestamp: tmap.HoldTimestamp, Windows: tmap.Windows, Pane: tmap.Pane}}, + opt: &MainInput{Key: *tmap.Key}, bf: n.bf, we: n.we, sa: n.UState, diff --git a/sdks/go/pkg/beam/core/runtime/exec/timers.go b/sdks/go/pkg/beam/core/runtime/exec/timers.go index 94c2ec8cbc60..512e91bfb85f 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/timers.go +++ b/sdks/go/pkg/beam/core/runtime/exec/timers.go @@ -21,6 +21,7 @@ import ( "io" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/coder" + "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/mtime" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/timers" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/typex" "github.com/apache/beam/sdks/v2/go/pkg/beam/log" @@ -130,3 +131,12 @@ func (p *timerProvider) Set(t timers.TimerMap) { panic(err) } } + +type TimerRecv struct { + Key *FullValue + Tag string + Windows []typex.Window // []typex.Window + Clear bool + FireTimestamp, HoldTimestamp mtime.Time + Pane typex.PaneInfo +} From efe50cfff46918480256f11fcee757b2a37c1134 Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Tue, 11 Apr 2023 09:22:53 -0400 Subject: [PATCH 07/22] changes to coder/pardo --- sdks/go/examples/streaming_wordcap/wordcap.go | 2 +- sdks/go/pkg/beam/core/runtime/exec/fn.go | 4 ++++ sdks/go/pkg/beam/core/runtime/exec/pardo.go | 9 ++++----- 3 files changed, 9 insertions(+), 6 deletions(-) diff --git a/sdks/go/examples/streaming_wordcap/wordcap.go b/sdks/go/examples/streaming_wordcap/wordcap.go index 2befde5aa03b..c6f132cd20b9 100644 --- a/sdks/go/examples/streaming_wordcap/wordcap.go +++ b/sdks/go/examples/streaming_wordcap/wordcap.go @@ -78,7 +78,7 @@ func NewStateful() *Stateful { } } -func (s *Stateful) OnTimer(ctx context.Context, ts beam.EventTime, tp timers.Provider, key, timerKey, timerTag string) { +func (s *Stateful) OnTimer(ctx context.Context, ts beam.EventTime, tp timers.Provider, key, word, timerKey, timerTag string) { log.Infof(ctx, "timer fired on stateful for element: %v, timerKey: %v, timerTag: %v", key, timerKey, timerTag) switch timerKey { case "outputState": diff --git a/sdks/go/pkg/beam/core/runtime/exec/fn.go b/sdks/go/pkg/beam/core/runtime/exec/fn.go index 0655b0f08028..319dda57e255 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/fn.go +++ b/sdks/go/pkg/beam/core/runtime/exec/fn.go @@ -28,6 +28,7 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/typex" "github.com/apache/beam/sdks/v2/go/pkg/beam/internal/errors" + "github.com/apache/beam/sdks/v2/go/pkg/beam/log" ) //go:generate specialize --input=fn_arity.tmpl @@ -293,10 +294,13 @@ func (n *invoker) invokeWithOpts(ctx context.Context, pn typex.PaneInfo, ws []ty // (3) Precomputed side input and emitters (or other output). for _, arg := range opts.extra { + log.Infof(ctx, "TIMER: i=%v, in[i]=%v, arg=%v", i, in[i], arg) args[in[i]] = arg i++ } + log.Infof(ctx, "args array = %v", args) + // (4) Invoke return n.call(pn, ws, ts) } diff --git a/sdks/go/pkg/beam/core/runtime/exec/pardo.go b/sdks/go/pkg/beam/core/runtime/exec/pardo.go index 25f160fd05fb..bdc2445992e7 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/pardo.go +++ b/sdks/go/pkg/beam/core/runtime/exec/pardo.go @@ -365,14 +365,13 @@ func (n *ParDo) InvokeTimerFn(ctx context.Context, fn *funcx.Fn, timerFamilyID s var extra []any extra = append(extra, timerFamilyID) + log.Infof(ctx, "timerRecv: %+v", tmap) if tmap.Tag != "" { + log.Infof(ctx, "TIMER: appending tag: %s", tmap.Tag) extra = append(extra, tmap.Tag) } - log.Infof(ctx, "timercoder map: %+v", n.Timer.(*userTimerAdapter).TimerIDToCoder) - - // log.Infof(ctx, "timer key converted with string: %v", string(tmap.Key)) - val, err := InvokeWithOpts(ctx, fn, tmap.Pane, nil, tmap.HoldTimestamp, InvokeOpts{ - // decode with timer coder from graph/fn.go + log.Infof(ctx, "fv received from decoder: %+v", *tmap.Key) + val, err := InvokeWithOpts(ctx, fn, tmap.Pane, tmap.Windows, tmap.HoldTimestamp, InvokeOpts{ opt: &MainInput{Key: *tmap.Key}, bf: n.bf, we: n.we, From 77b143838e75161f722b56cc571814ce4f3054b7 Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Tue, 18 Apr 2023 16:53:08 -0400 Subject: [PATCH 08/22] works for all cases, only cleanup left --- sdks/go/examples/streaming_wordcap/wordcap.go | 2 +- sdks/go/pkg/beam/core/runtime/exec/coder.go | 3 +++ sdks/go/pkg/beam/core/runtime/exec/datasource.go | 9 +++++---- sdks/go/pkg/beam/core/runtime/exec/pardo.go | 11 +---------- sdks/go/pkg/beam/core/runtime/exec/timers.go | 10 +++------- 5 files changed, 13 insertions(+), 22 deletions(-) diff --git a/sdks/go/examples/streaming_wordcap/wordcap.go b/sdks/go/examples/streaming_wordcap/wordcap.go index c6f132cd20b9..2befde5aa03b 100644 --- a/sdks/go/examples/streaming_wordcap/wordcap.go +++ b/sdks/go/examples/streaming_wordcap/wordcap.go @@ -78,7 +78,7 @@ func NewStateful() *Stateful { } } -func (s *Stateful) OnTimer(ctx context.Context, ts beam.EventTime, tp timers.Provider, key, word, timerKey, timerTag string) { +func (s *Stateful) OnTimer(ctx context.Context, ts beam.EventTime, tp timers.Provider, key, timerKey, timerTag string) { log.Infof(ctx, "timer fired on stateful for element: %v, timerKey: %v, timerTag: %v", key, timerKey, timerTag) switch timerKey { case "outputState": diff --git a/sdks/go/pkg/beam/core/runtime/exec/coder.go b/sdks/go/pkg/beam/core/runtime/exec/coder.go index 99d369d4117f..87c56064d7a1 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/coder.go +++ b/sdks/go/pkg/beam/core/runtime/exec/coder.go @@ -17,6 +17,7 @@ package exec import ( "bytes" + "context" "fmt" "io" "reflect" @@ -28,6 +29,7 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/core/typex" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/util/ioutilx" "github.com/apache/beam/sdks/v2/go/pkg/beam/internal/errors" + "github.com/apache/beam/sdks/v2/go/pkg/beam/log" ) // NOTE(herohde) 4/30/2017: The main complication is CoGBK results, which have @@ -1259,6 +1261,7 @@ func encodeTimer(elm ElementEncoder, win WindowEncoder, tm typex.TimerMap, w io. var b bytes.Buffer // elm.Encode(&FullValue{Elm: tm.Key}, &b) + log.Infof(context.Background(), "encoding timer with timer map: %+v", tm) _, err := b.Write(tm.Key) if err != nil { return errors.WithContext(err, "error encoding key") diff --git a/sdks/go/pkg/beam/core/runtime/exec/datasource.go b/sdks/go/pkg/beam/core/runtime/exec/datasource.go index cfbcb1c98e6b..b8dcb87e1425 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/datasource.go +++ b/sdks/go/pkg/beam/core/runtime/exec/datasource.go @@ -263,11 +263,12 @@ func (n *DataSource) Process(ctx context.Context) ([]*Checkpoint, error) { } }, func(bcr *byteCountReader, ptransformID, timerFamilyID string) error { - tmap, err := decodeTimer(cp, wc, bcr) - log.Infof(ctx, "DEBUGLOG: timer received for: %v and %v - %+v err: %v", ptransformID, timerFamilyID, tmap, err) - log.Infof(ctx, "OnTimerTransforms = %+v", n.OnTimerTransforms[ptransformID].Fn) + dc := MakeElementDecoder(coder.SkipW(c).Components[0]) + tmap, err := decodeTimer(dc, wc, bcr) + if err != nil { + return errors.WithContext(err, "error decoding timer in datasource") + } if fn, ok := n.OnTimerTransforms[ptransformID].Fn.OnTimerFn(); ok { - log.Infof(ctx, "found ontimer method, invoking callback") _, err := n.OnTimerTransforms[ptransformID].InvokeTimerFn(ctx, fn, timerFamilyID, tmap) if err != nil { return errors.WithContext(err, "ontimer callback invocation failed") diff --git a/sdks/go/pkg/beam/core/runtime/exec/pardo.go b/sdks/go/pkg/beam/core/runtime/exec/pardo.go index bdc2445992e7..6b56207cc039 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/pardo.go +++ b/sdks/go/pkg/beam/core/runtime/exec/pardo.go @@ -29,7 +29,6 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/typex" "github.com/apache/beam/sdks/v2/go/pkg/beam/internal/errors" - "github.com/apache/beam/sdks/v2/go/pkg/beam/log" "github.com/apache/beam/sdks/v2/go/pkg/beam/util/errorx" ) @@ -357,20 +356,12 @@ func (n *ParDo) invokeDataFn(ctx context.Context, pn typex.PaneInfo, ws []typex. } func (n *ParDo) InvokeTimerFn(ctx context.Context, fn *funcx.Fn, timerFamilyID string, tmap TimerRecv) (*FullValue, error) { - if fn == nil { - log.Infof(ctx, "timer function is not attached to pardo") - return nil, nil - } - log.Info(ctx, "InvokeTimerFn invoked") - var extra []any extra = append(extra, timerFamilyID) - log.Infof(ctx, "timerRecv: %+v", tmap) + if tmap.Tag != "" { - log.Infof(ctx, "TIMER: appending tag: %s", tmap.Tag) extra = append(extra, tmap.Tag) } - log.Infof(ctx, "fv received from decoder: %+v", *tmap.Key) val, err := InvokeWithOpts(ctx, fn, tmap.Pane, tmap.Windows, tmap.HoldTimestamp, InvokeOpts{ opt: &MainInput{Key: *tmap.Key}, bf: n.bf, diff --git a/sdks/go/pkg/beam/core/runtime/exec/timers.go b/sdks/go/pkg/beam/core/runtime/exec/timers.go index 512e91bfb85f..28b248ae7029 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/timers.go +++ b/sdks/go/pkg/beam/core/runtime/exec/timers.go @@ -60,13 +60,9 @@ func (u *userTimerAdapter) NewTimerProvider(ctx context.Context, manager DataMan if u.kc == nil { return timerProvider{}, fmt.Errorf("cannot make a state provider for an unkeyed input %v", element) } - var elementKey []byte - var err error - if element != nil { - elementKey, err = EncodeElement(u.kc, element.Key.Elm) - if err != nil { - return timerProvider{}, err - } + elementKey, err := EncodeElement(u.kc, element.Key.Elm) + if err != nil { + return timerProvider{}, err } tp := timerProvider{ From 3e0e4de04cdd724a4b9653e437bbfdf80f65a28c Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Tue, 18 Apr 2023 17:00:05 -0400 Subject: [PATCH 09/22] remove comments and validate onTimer --- sdks/go/pkg/beam/core/graph/fn.go | 29 ++++++++++++++----- sdks/go/pkg/beam/core/runtime/exec/coder.go | 18 +++--------- sdks/go/pkg/beam/core/runtime/exec/fn.go | 4 --- sdks/go/pkg/beam/core/runtime/exec/timers.go | 26 +++++++---------- .../pkg/beam/core/runtime/exec/translate.go | 7 ----- sdks/go/pkg/beam/pardo.go | 5 +++- 6 files changed, 41 insertions(+), 48 deletions(-) diff --git a/sdks/go/pkg/beam/core/graph/fn.go b/sdks/go/pkg/beam/core/graph/fn.go index 7f153faa1b0b..ab6fb6c9351a 100644 --- a/sdks/go/pkg/beam/core/graph/fn.go +++ b/sdks/go/pkg/beam/core/graph/fn.go @@ -1393,6 +1393,15 @@ func validateState(fn *DoFn, numIn mainInputs) error { return nil } +func validateOnTimerFn(fn *DoFn) error { + if _, ok := fn.OnTimerFn(); !ok { + err := errors.Errorf("OnTimer function not defined for DoFn: %v", fn.Name()) + return errors.SetTopLevelMsgf(err, "OnTimer function not defined for DoFn: %v. Ensure that OnTimer function is implemented for the DoFn.", fn.Name()) + } + + return nil +} + func validateTimer(fn *DoFn) error { if fn.Fn == nil { return nil @@ -1402,9 +1411,9 @@ func validateTimer(fn *DoFn) error { if _, ok := fn.Fn.TimerProvider(); ok { if len(pt) == 0 { - err := errors.Errorf("ProcessElement uses a TimerProvider, but no timer struct-tags are attached to the DoFn") - return errors.SetTopLevelMsgf(err, "ProcessElement uses a TimerProvider, but no timer struct-tags are attached to the DoFn"+ - ", Ensure that you are including the timer structs you're using to set/clear global state as uppercase member variables") + err := errors.New("ProcessElement uses a TimerProvider, but no Timer fields are defined in the DoFn") + return errors.SetTopLevelMsgf(err, "ProcessElement uses a TimerProvider, but no timer fields are defined in the DoFn"+ + ", Ensure that you are including the exported timer field in the DoFn that you're using to set/clear timers.") } timerKeys := make(map[string]PipelineTimer) for _, t := range pt { @@ -1416,16 +1425,22 @@ func validateTimer(fn *DoFn) error { timerKeys[k] = t } } + if err := validateOnTimerFn(fn); err != nil { + return err + } } else { if len(pt) > 0 { - err := errors.Errorf("ProcessElement doesn't use a TimerProvider, but Timer Struct is attached to the DoFn: %v", pt) - return errors.SetTopLevelMsgf(err, "ProcessElement doesn't use a TimerProvider, but Timer Struct is attached to the DoFn: %v"+ + err := errors.Errorf("ProcessElement doesn't use a TimerProvider, but Timer field is attached to the DoFn: %v", pt) + return errors.SetTopLevelMsgf(err, "ProcessElement doesn't use a TimerProvider, but Timer field is attached to the DoFn: %v"+ ", Ensure that you are using the TimerProvider to set/clear the timers.", pt) } + if err := validateOnTimerFn(fn); err == nil { + actualErr := errors.New("OnTimer function is defined for the DoFn but no TimerProvider defined in ProcessElement.") + return errors.SetTopLevelMsgf(actualErr, "OnTimer function is defined for the DoFn but no TimerProvider defined in ProcessElement."+ + "Ensure that timers.Provider is defined in the ProcessElement and OnTimer methods of DoFn.") + } } - // DO NOT SUBMIT: Require an OnTimer method existing - return nil } diff --git a/sdks/go/pkg/beam/core/runtime/exec/coder.go b/sdks/go/pkg/beam/core/runtime/exec/coder.go index 87c56064d7a1..e9a57e138632 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/coder.go +++ b/sdks/go/pkg/beam/core/runtime/exec/coder.go @@ -17,7 +17,6 @@ package exec import ( "bytes" - "context" "fmt" "io" "reflect" @@ -29,7 +28,6 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/core/typex" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/util/ioutilx" "github.com/apache/beam/sdks/v2/go/pkg/beam/internal/errors" - "github.com/apache/beam/sdks/v2/go/pkg/beam/log" ) // NOTE(herohde) 4/30/2017: The main complication is CoGBK results, which have @@ -1259,13 +1257,11 @@ func DecodeWindowedValueHeader(dec WindowDecoder, r io.Reader) ([]typex.Window, // encodeTimer encodes a typex.TimerMap into a byte stream. func encodeTimer(elm ElementEncoder, win WindowEncoder, tm typex.TimerMap, w io.Writer) error { var b bytes.Buffer - - // elm.Encode(&FullValue{Elm: tm.Key}, &b) - log.Infof(context.Background(), "encoding timer with timer map: %+v", tm) _, err := b.Write(tm.Key) if err != nil { return errors.WithContext(err, "error encoding key") } + if err := coder.EncodeStringUTF8(tm.Tag, &b); err != nil { return errors.WithContext(err, "error encoding tag") } @@ -1273,6 +1269,7 @@ func encodeTimer(elm ElementEncoder, win WindowEncoder, tm typex.TimerMap, w io. if err := win.Encode(tm.Windows, &b); err != nil { return errors.WithContext(err, "error encoding window") } + if err := coder.EncodeBool(tm.Clear, &b); err != nil { return errors.WithContext(err, "error encoding clear bit") } @@ -1288,27 +1285,20 @@ func encodeTimer(elm ElementEncoder, win WindowEncoder, tm typex.TimerMap, w io. return errors.WithContext(err, "error encoding paneinfo") } } - w.Write(b.Bytes()) + return nil } // decodeTimer decodes timer byte encoded with standard timer coder spec. func decodeTimer(dec ElementDecoder, win WindowDecoder, r io.Reader) (TimerRecv, error) { tm := TimerRecv{} - - // fv, err := dec.Decode(r) - // if err != nil { - // return tm, errors.WithContext(err, "error decoding timer key") - // } - // // TODO Change to not type assert once general timers key fix is done. - // tm.Key = fv.Elm.([]byte) - key, err := dec.Decode(r) if err != nil { return tm, errors.WithContext(err, "error decoding key") } tm.Key = key + s, err := coder.DecodeStringUTF8(r) if err != nil && err != io.EOF { return tm, errors.WithContext(err, "error decoding timer tag") diff --git a/sdks/go/pkg/beam/core/runtime/exec/fn.go b/sdks/go/pkg/beam/core/runtime/exec/fn.go index 319dda57e255..0655b0f08028 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/fn.go +++ b/sdks/go/pkg/beam/core/runtime/exec/fn.go @@ -28,7 +28,6 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/typex" "github.com/apache/beam/sdks/v2/go/pkg/beam/internal/errors" - "github.com/apache/beam/sdks/v2/go/pkg/beam/log" ) //go:generate specialize --input=fn_arity.tmpl @@ -294,13 +293,10 @@ func (n *invoker) invokeWithOpts(ctx context.Context, pn typex.PaneInfo, ws []ty // (3) Precomputed side input and emitters (or other output). for _, arg := range opts.extra { - log.Infof(ctx, "TIMER: i=%v, in[i]=%v, arg=%v", i, in[i], arg) args[in[i]] = arg i++ } - log.Infof(ctx, "args array = %v", args) - // (4) Invoke return n.call(pn, ws, ts) } diff --git a/sdks/go/pkg/beam/core/runtime/exec/timers.go b/sdks/go/pkg/beam/core/runtime/exec/timers.go index 28b248ae7029..6b47a8f88940 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/timers.go +++ b/sdks/go/pkg/beam/core/runtime/exec/timers.go @@ -32,12 +32,12 @@ type UserTimerAdapter interface { } type userTimerAdapter struct { - SID StreamID + sID StreamID wc WindowEncoder kc ElementEncoder - Dc ElementDecoder - TimerIDToCoder map[string]*coder.Coder - C *coder.Coder + dc ElementDecoder + timerIDToCoder map[string]*coder.Coder + c *coder.Coder } func NewUserTimerAdapter(sID StreamID, c *coder.Coder, timerCoders map[string]*coder.Coder) UserTimerAdapter { @@ -46,14 +46,10 @@ func NewUserTimerAdapter(sID StreamID, c *coder.Coder, timerCoders map[string]*c } wc := MakeWindowEncoder(c.Window) - var kc ElementEncoder - var dc ElementDecoder - if coder.IsKV(coder.SkipW(c)) { - kc = MakeElementEncoder(coder.SkipW(c).Components[0]) - dc = MakeElementDecoder(coder.SkipW(c).Components[0]) - } + kc := MakeElementEncoder(coder.SkipW(c).Components[0]) + dc := MakeElementDecoder(coder.SkipW(c).Components[0]) - return &userTimerAdapter{SID: sID, wc: wc, kc: kc, Dc: dc, C: c, TimerIDToCoder: timerCoders} + return &userTimerAdapter{sID: sID, wc: wc, kc: kc, dc: dc, c: c, timerIDToCoder: timerCoders} } func (u *userTimerAdapter) NewTimerProvider(ctx context.Context, manager DataManager, inputTs typex.EventTime, w []typex.Window, element *MainInput) (timerProvider, error) { @@ -70,10 +66,10 @@ func (u *userTimerAdapter) NewTimerProvider(ctx context.Context, manager DataMan tm: manager, elementKey: elementKey, inputTimestamp: inputTs, - SID: u.SID, + sID: u.sID, window: w, writersByFamily: make(map[string]io.Writer), - codersByFamily: u.TimerIDToCoder, + codersByFamily: u.timerIDToCoder, } return tp, nil @@ -82,7 +78,7 @@ func (u *userTimerAdapter) NewTimerProvider(ctx context.Context, manager DataMan type timerProvider struct { ctx context.Context tm DataManager - SID StreamID + sID StreamID inputTimestamp typex.EventTime elementKey []byte window []typex.Window @@ -97,7 +93,7 @@ func (p *timerProvider) getWriter(family string) (io.Writer, error) { if w, ok := p.writersByFamily[family]; ok { return w, nil } else { - w, err := p.tm.OpenTimerWrite(p.ctx, p.SID, family) + w, err := p.tm.OpenTimerWrite(p.ctx, p.sID, family) if err != nil { return nil, err } diff --git a/sdks/go/pkg/beam/core/runtime/exec/translate.go b/sdks/go/pkg/beam/core/runtime/exec/translate.go index ba109b1a76f2..ebdcb2ee99bc 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/translate.go +++ b/sdks/go/pkg/beam/core/runtime/exec/translate.go @@ -596,19 +596,12 @@ func (b *builder) makeLink(from string, id linkID) (Node, error) { log.Debugf(context.TODO(), "userTimers %+v", userTimers) timerIDToCoder := make(map[string]*coder.Coder) for key, _ := range userTimers { - // cID := spec.GetTimerFamilyCoderId() - // c, err := b.coders.Coder(cID) - // if err != nil { - // return nil, err - // } sID := StreamID{Port: Port{URL: b.desc.GetTimerApiServiceDescriptor().GetUrl()}, PtransformID: id.to} ec, wc, err := b.makeCoderForPCollection(input[0]) if err != nil { return nil, err } timerIDToCoder[key] = coder.NewT(ec, wc) - - // TODO: ensure this only gets set once or they're always the same. n.Timer = NewUserTimerAdapter(sID, coder.NewW(ec, wc), timerIDToCoder) } } diff --git a/sdks/go/pkg/beam/pardo.go b/sdks/go/pkg/beam/pardo.go index 5de2854c0387..3587e4e2b0dc 100644 --- a/sdks/go/pkg/beam/pardo.go +++ b/sdks/go/pkg/beam/pardo.go @@ -119,7 +119,10 @@ func TryParDo(s Scope, dofn any, col PCollection, opts ...Option) ([]PCollection wc := inWfn.Coder() pipelineTimers := fn.PipelineTimers() if len(pipelineTimers) > 0 { - // TODO(riteshghorse): replace the coder with type of key + // check if input is KV type + if !typex.IsKV(col.Type()) { + return nil, addParDoCtx(errors.New("DoFn input should be keyed to be used with timers."), s) + } c := coder.NewString() edge.TimerCoders = make(map[string]*coder.Coder) for _, pt := range pipelineTimers { From 41918395a6cbaf8ef9b7b4b6c02f6f5ff625f8fb Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Wed, 19 Apr 2023 14:07:08 -0400 Subject: [PATCH 10/22] generic coder for user key --- sdks/go/pkg/beam/core/runtime/exec/translate.go | 12 ++++++------ sdks/go/pkg/beam/pardo.go | 5 ++++- 2 files changed, 10 insertions(+), 7 deletions(-) diff --git a/sdks/go/pkg/beam/core/runtime/exec/translate.go b/sdks/go/pkg/beam/core/runtime/exec/translate.go index ebdcb2ee99bc..758c3c0175e8 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/translate.go +++ b/sdks/go/pkg/beam/core/runtime/exec/translate.go @@ -595,15 +595,15 @@ func (b *builder) makeLink(from string, id linkID) (Node, error) { if len(userTimers) > 0 { log.Debugf(context.TODO(), "userTimers %+v", userTimers) timerIDToCoder := make(map[string]*coder.Coder) + sID := StreamID{Port: Port{URL: b.desc.GetTimerApiServiceDescriptor().GetUrl()}, PtransformID: id.to} + ec, wc, err := b.makeCoderForPCollection(input[0]) + if err != nil { + return nil, err + } for key, _ := range userTimers { - sID := StreamID{Port: Port{URL: b.desc.GetTimerApiServiceDescriptor().GetUrl()}, PtransformID: id.to} - ec, wc, err := b.makeCoderForPCollection(input[0]) - if err != nil { - return nil, err - } timerIDToCoder[key] = coder.NewT(ec, wc) - n.Timer = NewUserTimerAdapter(sID, coder.NewW(ec, wc), timerIDToCoder) } + n.Timer = NewUserTimerAdapter(sID, coder.NewW(ec, wc), timerIDToCoder) } for i := 1; i < len(input); i++ { diff --git a/sdks/go/pkg/beam/pardo.go b/sdks/go/pkg/beam/pardo.go index 3587e4e2b0dc..c8d23da443b5 100644 --- a/sdks/go/pkg/beam/pardo.go +++ b/sdks/go/pkg/beam/pardo.go @@ -123,7 +123,10 @@ func TryParDo(s Scope, dofn any, col PCollection, opts ...Option) ([]PCollection if !typex.IsKV(col.Type()) { return nil, addParDoCtx(errors.New("DoFn input should be keyed to be used with timers."), s) } - c := coder.NewString() + c, err := inferCoder(typex.New(reflect.TypeOf(col.Type()))) + if err != nil { + return nil, addParDoCtx(errors.New("error infering coder from col"), s) + } edge.TimerCoders = make(map[string]*coder.Coder) for _, pt := range pipelineTimers { tc := coder.NewT(c, wc) From dd0c4e6ccc5e0ee4eb6276a161b4f917a31a4567 Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Wed, 19 Apr 2023 21:48:13 -0400 Subject: [PATCH 11/22] fixes coder end to end --- sdks/go/pkg/beam/core/runtime/exec/coder.go | 19 +++++++++++++----- sdks/go/pkg/beam/core/runtime/exec/timers.go | 20 +++++++++++++++---- .../regression/coders/fromyaml/fromyaml.go | 4 ++-- 3 files changed, 32 insertions(+), 11 deletions(-) diff --git a/sdks/go/pkg/beam/core/runtime/exec/coder.go b/sdks/go/pkg/beam/core/runtime/exec/coder.go index e9a57e138632..165abc793908 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/coder.go +++ b/sdks/go/pkg/beam/core/runtime/exec/coder.go @@ -17,6 +17,7 @@ package exec import ( "bytes" + "context" "fmt" "io" "reflect" @@ -28,6 +29,7 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/core/typex" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/util/ioutilx" "github.com/apache/beam/sdks/v2/go/pkg/beam/internal/errors" + "github.com/apache/beam/sdks/v2/go/pkg/beam/log" ) // NOTE(herohde) 4/30/2017: The main complication is CoGBK results, which have @@ -145,8 +147,9 @@ func MakeElementEncoder(c *coder.Coder) ElementEncoder { } case coder.Timer: + log.Infof(context.Background(), "coder in elementencoder: %+v", coder.SkipW(c).Components[0]) return &timerEncoder{ - elm: MakeElementEncoder(c.Components[0]), + elm: MakeElementEncoder(coder.SkipW(c).Components[0].Components[0]), win: MakeWindowEncoder(c.Window), } @@ -267,7 +270,7 @@ func MakeElementDecoder(c *coder.Coder) ElementDecoder { case coder.Timer: return &timerDecoder{ - elm: MakeElementDecoder(c.Components[0]), + elm: MakeElementDecoder(coder.SkipW(c).Components[0]), win: MakeWindowDecoder(c.Window), } @@ -902,7 +905,7 @@ type timerEncoder struct { } func (e *timerEncoder) Encode(val *FullValue, w io.Writer) error { - return encodeTimer(e.elm, e.win, val.Elm.(typex.TimerMap), w) + return encodeTimer(e.elm, e.win, val.Elm.(TimerRecv), w) } type timerDecoder struct { @@ -1255,9 +1258,9 @@ func DecodeWindowedValueHeader(dec WindowDecoder, r io.Reader) ([]typex.Window, } // encodeTimer encodes a typex.TimerMap into a byte stream. -func encodeTimer(elm ElementEncoder, win WindowEncoder, tm typex.TimerMap, w io.Writer) error { +func encodeTimer(elm ElementEncoder, win WindowEncoder, tm TimerRecv, w io.Writer) error { var b bytes.Buffer - _, err := b.Write(tm.Key) + err := elm.Encode(tm.Key, &b) if err != nil { return errors.WithContext(err, "error encoding key") } @@ -1298,6 +1301,7 @@ func decodeTimer(dec ElementDecoder, win WindowDecoder, r io.Reader) (TimerRecv, return tm, errors.WithContext(err, "error decoding key") } tm.Key = key + log.Infof(context.Background(), "decoded key from timer: %v", key) s, err := coder.DecodeStringUTF8(r) if err != nil && err != io.EOF { @@ -1308,17 +1312,22 @@ func decodeTimer(dec ElementDecoder, win WindowDecoder, r io.Reader) (TimerRecv, } tm.Tag = s + log.Infof(context.Background(), "decoded tag from timer: %v", tm.Tag) + w, err := win.Decode(r) if err != nil { return tm, errors.WithContext(err, "error decoding timer window") } tm.Windows = w + log.Infof(context.Background(), "decoded win from timer: %v", w) c, err := coder.DecodeBool(r) if err != nil { return tm, errors.WithContext(err, "error decoding clear") } tm.Clear = c + log.Infof(context.Background(), "decoded bool from timer: %v", c) + if tm.Clear { return tm, nil } diff --git a/sdks/go/pkg/beam/core/runtime/exec/timers.go b/sdks/go/pkg/beam/core/runtime/exec/timers.go index 6b47a8f88940..c31d8b86d84d 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/timers.go +++ b/sdks/go/pkg/beam/core/runtime/exec/timers.go @@ -61,13 +61,19 @@ func (u *userTimerAdapter) NewTimerProvider(ctx context.Context, manager DataMan return timerProvider{}, err } + log.Infof(ctx, "element key FV: %+v", element.Key) + userKey := &FullValue{Elm: element.Key.Elm} + log.Infof(ctx, "newly created user key: %+v", userKey) tp := timerProvider{ ctx: ctx, tm: manager, elementKey: elementKey, + userKey: userKey, inputTimestamp: inputTs, sID: u.sID, window: w, + c: u.c, + win: u.wc, writersByFamily: make(map[string]io.Writer), codersByFamily: u.timerIDToCoder, } @@ -81,9 +87,12 @@ type timerProvider struct { sID StreamID inputTimestamp typex.EventTime elementKey []byte + userKey *FullValue window []typex.Window - pn typex.PaneInfo + pn typex.PaneInfo + c *coder.Coder + win WindowEncoder writersByFamily map[string]io.Writer codersByFamily map[string]*coder.Coder @@ -107,8 +116,8 @@ func (p *timerProvider) Set(t timers.TimerMap) { if err != nil { panic(err) } - tm := typex.TimerMap{ - Key: p.elementKey, //string(p.elementKey), + tm := TimerRecv{ + Key: p.userKey, //string(p.elementKey), Tag: t.Tag, Windows: p.window, Clear: t.Clear, @@ -118,7 +127,10 @@ func (p *timerProvider) Set(t timers.TimerMap) { } log.Debugf(p.ctx, "timer set: %+v", tm) fv := FullValue{Elm: tm} - enc := MakeElementEncoder(coder.SkipW(p.codersByFamily[t.Family])) + + // tc := coder.NewT(p.c) + log.Infof(p.ctx, "timer coder for %v: %+v", t.Family, p.codersByFamily[t.Family]) + enc := MakeElementEncoder(p.codersByFamily[t.Family]) if err := enc.Encode(&fv, w); err != nil { panic(err) } diff --git a/sdks/go/test/regression/coders/fromyaml/fromyaml.go b/sdks/go/test/regression/coders/fromyaml/fromyaml.go index e28bd9d413f9..466f361b0138 100644 --- a/sdks/go/test/regression/coders/fromyaml/fromyaml.go +++ b/sdks/go/test/regression/coders/fromyaml/fromyaml.go @@ -326,13 +326,13 @@ func diff(c Coder, elem *exec.FullValue, eg yaml.MapItem) bool { got, want = elem.Elm, rv.Interface() case "beam:coder:timer:v1": pass := true - tm := elem.Elm.(typex.TimerMap) + tm := elem.Elm.(exec.TimerRecv) fs := eg.Value.(yaml.MapSlice) for _, item := range fs { switch item.Key.(string) { case "userKey": - if want := item.Value.(string); want != string(tm.Key) { + if want := item.Value.(string); want != tm.Key.Elm.(string) { pass = false } case "dynamicTimerTag": From 95d641f872b0b426f85a176cddc82a033be3f69d Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Wed, 19 Apr 2023 22:08:07 -0400 Subject: [PATCH 12/22] remove logs --- sdks/go/examples/streaming_wordcap/wordcap.go | 25 ------------------- sdks/go/pkg/beam/core/runtime/exec/coder.go | 9 ------- sdks/go/pkg/beam/core/runtime/exec/timers.go | 10 +------- .../pkg/beam/core/runtime/harness/datamgr.go | 2 -- 4 files changed, 1 insertion(+), 45 deletions(-) diff --git a/sdks/go/examples/streaming_wordcap/wordcap.go b/sdks/go/examples/streaming_wordcap/wordcap.go index 2befde5aa03b..120e1a295418 100644 --- a/sdks/go/examples/streaming_wordcap/wordcap.go +++ b/sdks/go/examples/streaming_wordcap/wordcap.go @@ -112,31 +112,6 @@ func (s *Stateful) ProcessElement(ctx context.Context, ts beam.EventTime, sp sta s.OutputState.Set(tp, mtime.Time(toFire).ToTime(), timers.WithOutputTimestamp(mtime.Time(minTime).ToTime()), timers.WithTag(word)) s.TimerTime.Write(sp, toFire) - log.Infof(ctx, "stateful dofn key: %v word: %v, timer: %v, minTime: %v", key, word, toFire, minTime) - - // // Get the Value stored in our state - // val, ok, err := s.Val.Read(p) - // if err != nil { - // return err - // } - // log.Infof(ctx, "stateful dofn state read key: %v word: %v val: %v", key, word, val) - // if !ok { - // s.Val.Write(p, 1) - // } else { - // s.Val.Write(p, val+1) - // } - - // if val > 5 { - // log.Infof(ctx, "stateful dofn clearing key: %v word: %v val: %v", key, word, val) - // // Example of clearing and starting again with an empty bag - // s.Val.Clear(p) - // } - // fire := time.Now().Add(10 * time.Second) - - // log.Infof(ctx, "stateful dofn timer family: %v fire: %v now: %v key: %v word: %v", s.Fire.Family, fire, time.Now(), key, word) - // s.Fire.Set(tp, fire) - - // emit(key, word) return nil } diff --git a/sdks/go/pkg/beam/core/runtime/exec/coder.go b/sdks/go/pkg/beam/core/runtime/exec/coder.go index 165abc793908..274165a768ae 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/coder.go +++ b/sdks/go/pkg/beam/core/runtime/exec/coder.go @@ -17,7 +17,6 @@ package exec import ( "bytes" - "context" "fmt" "io" "reflect" @@ -29,7 +28,6 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/core/typex" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/util/ioutilx" "github.com/apache/beam/sdks/v2/go/pkg/beam/internal/errors" - "github.com/apache/beam/sdks/v2/go/pkg/beam/log" ) // NOTE(herohde) 4/30/2017: The main complication is CoGBK results, which have @@ -147,7 +145,6 @@ func MakeElementEncoder(c *coder.Coder) ElementEncoder { } case coder.Timer: - log.Infof(context.Background(), "coder in elementencoder: %+v", coder.SkipW(c).Components[0]) return &timerEncoder{ elm: MakeElementEncoder(coder.SkipW(c).Components[0].Components[0]), win: MakeWindowEncoder(c.Window), @@ -1301,7 +1298,6 @@ func decodeTimer(dec ElementDecoder, win WindowDecoder, r io.Reader) (TimerRecv, return tm, errors.WithContext(err, "error decoding key") } tm.Key = key - log.Infof(context.Background(), "decoded key from timer: %v", key) s, err := coder.DecodeStringUTF8(r) if err != nil && err != io.EOF { @@ -1312,22 +1308,17 @@ func decodeTimer(dec ElementDecoder, win WindowDecoder, r io.Reader) (TimerRecv, } tm.Tag = s - log.Infof(context.Background(), "decoded tag from timer: %v", tm.Tag) - w, err := win.Decode(r) if err != nil { return tm, errors.WithContext(err, "error decoding timer window") } tm.Windows = w - log.Infof(context.Background(), "decoded win from timer: %v", w) c, err := coder.DecodeBool(r) if err != nil { return tm, errors.WithContext(err, "error decoding clear") } tm.Clear = c - log.Infof(context.Background(), "decoded bool from timer: %v", c) - if tm.Clear { return tm, nil } diff --git a/sdks/go/pkg/beam/core/runtime/exec/timers.go b/sdks/go/pkg/beam/core/runtime/exec/timers.go index c31d8b86d84d..ef54c68f0971 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/timers.go +++ b/sdks/go/pkg/beam/core/runtime/exec/timers.go @@ -24,7 +24,6 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/mtime" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/timers" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/typex" - "github.com/apache/beam/sdks/v2/go/pkg/beam/log" ) type UserTimerAdapter interface { @@ -60,10 +59,7 @@ func (u *userTimerAdapter) NewTimerProvider(ctx context.Context, manager DataMan if err != nil { return timerProvider{}, err } - - log.Infof(ctx, "element key FV: %+v", element.Key) userKey := &FullValue{Elm: element.Key.Elm} - log.Infof(ctx, "newly created user key: %+v", userKey) tp := timerProvider{ ctx: ctx, tm: manager, @@ -117,7 +113,7 @@ func (p *timerProvider) Set(t timers.TimerMap) { panic(err) } tm := TimerRecv{ - Key: p.userKey, //string(p.elementKey), + Key: p.userKey, Tag: t.Tag, Windows: p.window, Clear: t.Clear, @@ -125,11 +121,7 @@ func (p *timerProvider) Set(t timers.TimerMap) { HoldTimestamp: t.HoldTimestamp, Pane: p.pn, } - log.Debugf(p.ctx, "timer set: %+v", tm) fv := FullValue{Elm: tm} - - // tc := coder.NewT(p.c) - log.Infof(p.ctx, "timer coder for %v: %+v", t.Family, p.codersByFamily[t.Family]) enc := MakeElementEncoder(p.codersByFamily[t.Family]) if err := enc.Encode(&fv, w); err != nil { panic(err) diff --git a/sdks/go/pkg/beam/core/runtime/harness/datamgr.go b/sdks/go/pkg/beam/core/runtime/harness/datamgr.go index 3850ee3a3667..d8c0f4d1d852 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/datamgr.go +++ b/sdks/go/pkg/beam/core/runtime/harness/datamgr.go @@ -690,8 +690,6 @@ func (w *timerWriter) writeTimers(p []byte) error { w.ch.mu.Lock() defer w.ch.mu.Unlock() - log.Infof(context.TODO(), "DEBUGLOG: timer write for %+v: %v", w.id, p) - msg := &fnpb.Elements{ Timers: []*fnpb.Elements_Timers{ { From 6ce7814f8c494616d2459206049d7e004b1075fa Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Thu, 20 Apr 2023 00:02:00 -0400 Subject: [PATCH 13/22] add unit test and refactor --- sdks/go/pkg/beam/core/graph/fn.go | 5 - sdks/go/pkg/beam/core/runtime/exec/coder.go | 20 ++- .../pkg/beam/core/runtime/exec/timers_test.go | 143 +++++++++--------- sdks/go/pkg/beam/core/typex/special.go | 10 -- 4 files changed, 88 insertions(+), 90 deletions(-) diff --git a/sdks/go/pkg/beam/core/graph/fn.go b/sdks/go/pkg/beam/core/graph/fn.go index ab6fb6c9351a..d759703c5c16 100644 --- a/sdks/go/pkg/beam/core/graph/fn.go +++ b/sdks/go/pkg/beam/core/graph/fn.go @@ -313,11 +313,6 @@ type PipelineTimer interface { TimerDomain() timers.TimeDomainEnum } -var ( - _ PipelineTimer = timers.EventTime{} - _ PipelineTimer = timers.ProcessingTime{} -) - func (f *DoFn) OnTimerFn() (*funcx.Fn, bool) { m, ok := f.methods[onTimerName] return m, ok diff --git a/sdks/go/pkg/beam/core/runtime/exec/coder.go b/sdks/go/pkg/beam/core/runtime/exec/coder.go index 274165a768ae..d9e189514925 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/coder.go +++ b/sdks/go/pkg/beam/core/runtime/exec/coder.go @@ -145,8 +145,15 @@ func MakeElementEncoder(c *coder.Coder) ElementEncoder { } case coder.Timer: + tc := coder.SkipW(c).Components[0] + if coder.IsKV(tc) { + return &timerEncoder{ + elm: MakeElementEncoder(tc.Components[0]), + win: MakeWindowEncoder(c.Window), + } + } return &timerEncoder{ - elm: MakeElementEncoder(coder.SkipW(c).Components[0].Components[0]), + elm: MakeElementEncoder(tc), win: MakeWindowEncoder(c.Window), } @@ -266,8 +273,15 @@ func MakeElementDecoder(c *coder.Coder) ElementDecoder { } case coder.Timer: + tc := coder.SkipW(c).Components[0] + if coder.IsKV(tc) { + return &timerDecoder{ + elm: MakeElementDecoder(tc.Components[0]), + win: MakeWindowDecoder(c.Window), + } + } return &timerDecoder{ - elm: MakeElementDecoder(coder.SkipW(c).Components[0]), + elm: MakeElementDecoder(tc), win: MakeWindowDecoder(c.Window), } @@ -1254,7 +1268,7 @@ func DecodeWindowedValueHeader(dec WindowDecoder, r io.Reader) ([]typex.Window, return ws, t, pn, nil } -// encodeTimer encodes a typex.TimerMap into a byte stream. +// encodeTimer encodes a TimerRecv into a byte stream. func encodeTimer(elm ElementEncoder, win WindowEncoder, tm TimerRecv, w io.Writer) error { var b bytes.Buffer err := elm.Encode(tm.Key, &b) diff --git a/sdks/go/pkg/beam/core/runtime/exec/timers_test.go b/sdks/go/pkg/beam/core/runtime/exec/timers_test.go index 185b0d11a01c..5a24ce11a60d 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/timers_test.go +++ b/sdks/go/pkg/beam/core/runtime/exec/timers_test.go @@ -15,82 +15,81 @@ package exec -// import ( -// "bytes" -// "testing" +import ( + "bytes" + "testing" -// "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/coder" -// "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/mtime" -// "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/window" -// "github.com/apache/beam/sdks/v2/go/pkg/beam/core/typex" -// ) + "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/coder" + "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/mtime" + "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/window" +) -// func equalTimers(a, b typex.TimerMap) bool { -// return a.Key == b.Key && a.Tag == b.Tag && (a.FireTimestamp) == b.FireTimestamp && a.Clear == b.Clear -// } +func equalTimers(a, b TimerRecv) bool { + return a.Key.Elm == b.Key.Elm && a.Tag == b.Tag && (a.FireTimestamp) == b.FireTimestamp && a.Clear == b.Clear +} -// func TestTimerEncodingDecoding(t *testing.T) { -// tc := coder.NewT(coder.NewString(), window.NewGlobalWindows().Coder()) -// ec := MakeElementEncoder(coder.SkipW(tc)) -// dec := MakeElementDecoder(coder.SkipW(tc)) +func TestTimerEncodingDecoding(t *testing.T) { + tc := coder.NewT(coder.NewKV([]*coder.Coder{coder.NewString(), coder.NewString()}), window.NewGlobalWindows().Coder()) + ec := MakeElementEncoder(coder.SkipW(tc)) + dec := MakeElementDecoder(coder.SkipW(tc)) -// tests := []struct { -// name string -// tm typex.TimerMap -// result bool -// }{ -// { -// name: "all set fields", -// tm: typex.TimerMap{ -// Key: []byte{byte("Basic")}, -// Tag: "first", -// Windows: window.SingleGlobalWindow, -// Clear: false, -// FireTimestamp: mtime.Now(), -// }, -// result: true, -// }, -// { -// name: "without tag", -// tm: typex.TimerMap{ -// Key: "Basic", -// Tag: "", -// Windows: window.SingleGlobalWindow, -// Clear: false, -// FireTimestamp: mtime.Now(), -// }, -// result: true, -// }, -// { -// name: "with clear set", -// tm: typex.TimerMap{ -// Key: "Basic", -// Tag: "first", -// Windows: window.SingleGlobalWindow, -// Clear: true, -// FireTimestamp: mtime.Now(), -// }, -// result: false, -// }, -// } -// for _, test := range tests { -// t.Run(test.name, func(t *testing.T) { -// fv := FullValue{Elm: test.tm} -// var buf bytes.Buffer -// err := ec.Encode(&fv, &buf) -// if err != nil { -// t.Fatalf("error encoding timer: %#v, got: %v", test.tm, err) -// } + tests := []struct { + name string + tm TimerRecv + result bool + }{ + { + name: "all set fields", + tm: TimerRecv{ + Key: &FullValue{Elm: "Basic"}, + Tag: "first", + Windows: window.SingleGlobalWindow, + Clear: false, + FireTimestamp: mtime.Now(), + }, + result: true, + }, + { + name: "without tag", + tm: TimerRecv{ + Key: &FullValue{Elm: "Basic"}, + Tag: "", + Windows: window.SingleGlobalWindow, + Clear: false, + FireTimestamp: mtime.Now(), + }, + result: true, + }, + { + name: "with clear set", + tm: TimerRecv{ + Key: &FullValue{Elm: "Basic"}, + Tag: "first", + Windows: window.SingleGlobalWindow, + Clear: true, + FireTimestamp: mtime.Now(), + }, + result: false, + }, + } + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + fv := FullValue{Elm: test.tm} + var buf bytes.Buffer + err := ec.Encode(&fv, &buf) + if err != nil { + t.Fatalf("error encoding timer: %#v, got: %v", test.tm, err) + } -// gotFv, err := dec.Decode(&buf) -// if err != nil { -// t.Fatalf("failed to decode timer, got %v", err) -// } + gotFv, err := dec.Decode(&buf) + if err != nil { + t.Fatalf("failed to decode timer, got %v", err) + } -// if got, want := gotFv.Elm.(typex.TimerMap), test.tm; test.result != equalTimers(got, want) { -// t.Errorf("got timer %v, want %v", got, want) -// } -// }) -// } + if got, want := gotFv.Elm.(TimerRecv), test.tm; test.result != equalTimers(got, want) { + t.Errorf("got timer %v, want %v", got, want) + } + }) + } -// } +} diff --git a/sdks/go/pkg/beam/core/typex/special.go b/sdks/go/pkg/beam/core/typex/special.go index 82000efbd2d9..edc1249fe763 100644 --- a/sdks/go/pkg/beam/core/typex/special.go +++ b/sdks/go/pkg/beam/core/typex/special.go @@ -105,16 +105,6 @@ type Timers struct { Pane PaneInfo } -// TimerMap is a placeholder for timer details used in encoding/decoding. -type TimerMap struct { - Key []byte - Tag string - Windows []Window // []typex.Window - Clear bool - FireTimestamp, HoldTimestamp mtime.Time - Pane PaneInfo -} - // KV, Nullable, CoGBK, WindowedValue represent composite generic types. They are not used // directly in user code signatures, but only in FullTypes. From 97626827d4ec7413c4c4a339597422c8db7cfeaf Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Thu, 20 Apr 2023 00:48:20 -0400 Subject: [PATCH 14/22] add docs --- sdks/go/pkg/beam/core/funcx/fn.go | 2 ++ sdks/go/pkg/beam/core/graph/fn.go | 17 +++++------ sdks/go/pkg/beam/core/runtime/exec/timers.go | 31 +++++--------------- sdks/go/pkg/beam/core/timers/timers.go | 24 +++++++++++++-- 4 files changed, 39 insertions(+), 35 deletions(-) diff --git a/sdks/go/pkg/beam/core/funcx/fn.go b/sdks/go/pkg/beam/core/funcx/fn.go index 3aef5a1695ac..23c2dd0e8b2f 100644 --- a/sdks/go/pkg/beam/core/funcx/fn.go +++ b/sdks/go/pkg/beam/core/funcx/fn.go @@ -310,6 +310,8 @@ func (u *Fn) StateProvider() (pos int, exists bool) { return -1, false } +// TimerProvider returns (index, true) iff the function expects a +// parameter that implements timers.Provider. func (u *Fn) TimerProvider() (pos int, exists bool) { for i, p := range u.Param { if p.Kind == FnTimerProvider { diff --git a/sdks/go/pkg/beam/core/graph/fn.go b/sdks/go/pkg/beam/core/graph/fn.go index d759703c5c16..800068bbdc50 100644 --- a/sdks/go/pkg/beam/core/graph/fn.go +++ b/sdks/go/pkg/beam/core/graph/fn.go @@ -308,28 +308,25 @@ func (f *DoFn) PipelineState() []state.PipelineState { return s } -type PipelineTimer interface { - TimerFamily() string - TimerDomain() timers.TimeDomainEnum -} - +// OnTimerFn return the "OnTimer" function and a bool indicating whether the +// function is defined or not for the DoFn. func (f *DoFn) OnTimerFn() (*funcx.Fn, bool) { m, ok := f.methods[onTimerName] return m, ok } -func (f *DoFn) PipelineTimers() []PipelineTimer { - var t []PipelineTimer +// PipelineTimers returns the list of PipelineTimer objects defined for the DoFn. +func (f *DoFn) PipelineTimers() []timers.PipelineTimer { + var t []timers.PipelineTimer if f.Recv == nil { return t } v := reflect.Indirect(reflect.ValueOf(f.Recv)) - for i := 0; i < v.NumField(); i++ { f := v.Field(i) if f.CanInterface() { - if pt, ok := f.Interface().(PipelineTimer); ok { + if pt, ok := f.Interface().(timers.PipelineTimer); ok { t = append(t, pt) } } @@ -1410,7 +1407,7 @@ func validateTimer(fn *DoFn) error { return errors.SetTopLevelMsgf(err, "ProcessElement uses a TimerProvider, but no timer fields are defined in the DoFn"+ ", Ensure that you are including the exported timer field in the DoFn that you're using to set/clear timers.") } - timerKeys := make(map[string]PipelineTimer) + timerKeys := make(map[string]timers.PipelineTimer) for _, t := range pt { k := t.TimerFamily() if timer, ok := timerKeys[k]; ok { diff --git a/sdks/go/pkg/beam/core/runtime/exec/timers.go b/sdks/go/pkg/beam/core/runtime/exec/timers.go index ef54c68f0971..7c5fc53ff8ea 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/timers.go +++ b/sdks/go/pkg/beam/core/runtime/exec/timers.go @@ -26,50 +26,35 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/core/typex" ) +// UserTimerAdapter provides a timer provider to be used for manipulating timers. type UserTimerAdapter interface { NewTimerProvider(ctx context.Context, manager DataManager, inputTimestamp typex.EventTime, windows []typex.Window, element *MainInput) (timerProvider, error) } type userTimerAdapter struct { sID StreamID - wc WindowEncoder - kc ElementEncoder - dc ElementDecoder timerIDToCoder map[string]*coder.Coder - c *coder.Coder } +// NewUserTimerAdapter returns a user timer adapter for the given StreamID and timer coder. func NewUserTimerAdapter(sID StreamID, c *coder.Coder, timerCoders map[string]*coder.Coder) UserTimerAdapter { if !coder.IsW(c) { panic(fmt.Sprintf("expected WV coder for user timer %v: %v", sID, c)) } - wc := MakeWindowEncoder(c.Window) - kc := MakeElementEncoder(coder.SkipW(c).Components[0]) - dc := MakeElementDecoder(coder.SkipW(c).Components[0]) - - return &userTimerAdapter{sID: sID, wc: wc, kc: kc, dc: dc, c: c, timerIDToCoder: timerCoders} + return &userTimerAdapter{sID: sID, timerIDToCoder: timerCoders} } +// NewTimerProvider creates and returns a timer provider to set/clear timers. func (u *userTimerAdapter) NewTimerProvider(ctx context.Context, manager DataManager, inputTs typex.EventTime, w []typex.Window, element *MainInput) (timerProvider, error) { - if u.kc == nil { - return timerProvider{}, fmt.Errorf("cannot make a state provider for an unkeyed input %v", element) - } - elementKey, err := EncodeElement(u.kc, element.Key.Elm) - if err != nil { - return timerProvider{}, err - } userKey := &FullValue{Elm: element.Key.Elm} tp := timerProvider{ ctx: ctx, tm: manager, - elementKey: elementKey, userKey: userKey, inputTimestamp: inputTs, sID: u.sID, window: w, - c: u.c, - win: u.wc, writersByFamily: make(map[string]io.Writer), codersByFamily: u.timerIDToCoder, } @@ -82,13 +67,10 @@ type timerProvider struct { tm DataManager sID StreamID inputTimestamp typex.EventTime - elementKey []byte userKey *FullValue window []typex.Window - pn typex.PaneInfo - c *coder.Coder - win WindowEncoder + pn typex.PaneInfo writersByFamily map[string]io.Writer codersByFamily map[string]*coder.Coder @@ -107,6 +89,8 @@ func (p *timerProvider) getWriter(family string) (io.Writer, error) { } } +// Set writes a new timer. This can be used to both Set as well as Clear the timer. +// Note: This function is intended for internal use only. func (p *timerProvider) Set(t timers.TimerMap) { w, err := p.getWriter(t.Family) if err != nil { @@ -128,6 +112,7 @@ func (p *timerProvider) Set(t timers.TimerMap) { } } +// TimerRecv holds the timer metadata while encoding and decoding timers in exec unit. type TimerRecv struct { Key *FullValue Tag string diff --git a/sdks/go/pkg/beam/core/timers/timers.go b/sdks/go/pkg/beam/core/timers/timers.go index c9d170f90782..7e377db584b9 100644 --- a/sdks/go/pkg/beam/core/timers/timers.go +++ b/sdks/go/pkg/beam/core/timers/timers.go @@ -24,17 +24,24 @@ import ( ) var ( + // ProviderType represents the type of timer provider. ProviderType = reflect.TypeOf((*Provider)(nil)).Elem() ) +// TimeDomainEnum represents different time domains to set timer. type TimeDomainEnum int32 const ( - TimeDomainUnspecified TimeDomainEnum = 0 - TimeDomainEventTime TimeDomainEnum = 1 + // TimeDomainUnspecified represents unspecified time domain. + TimeDomainUnspecified TimeDomainEnum = 0 + // TimeDomainEventTime is time from the perspective of the data + TimeDomainEventTime TimeDomainEnum = 1 + // TimeDomainProcessingTime is time from the perspective of the + // execution of your pipeline TimeDomainProcessingTime TimeDomainEnum = 2 ) +// TimerMap holds timer information obtained from the pipeline. type TimerMap struct { Family string Tag string @@ -63,19 +70,28 @@ func WithOutputTimestamp(outputTimestamp time.Time) timerOptions { } } +// Provider represents a timer provider interface. type Provider interface { Set(t TimerMap) } +// PipelineTimer interface represents valid timer type. +type PipelineTimer interface { + TimerFamily() string + TimerDomain() TimeDomainEnum +} + // EventTime represents the event time timer. type EventTime struct { Family string } +// TimerFamily returns the name of timer family. func (et EventTime) TimerFamily() string { return et.Family } +// TimerDomain returns the time domain of timer. func (et EventTime) TimerDomain() TimeDomainEnum { return TimeDomainEventTime } @@ -104,10 +120,12 @@ type ProcessingTime struct { Family string } +// TimerFamily returns the name of timer family. func (pt ProcessingTime) TimerFamily() string { return pt.Family } +// TimerDomain returns the time domain of timer. func (pt ProcessingTime) TimerDomain() TimeDomainEnum { return TimeDomainProcessingTime } @@ -132,10 +150,12 @@ func (pt ProcessingTime) Clear(p Provider) { p.Set(TimerMap{Family: pt.Family, Clear: true}) } +// InEventTime creates and returns a new EventTime timer object. func InEventTime(Key string) EventTime { return EventTime{Family: Key} } +// InProcessingTime creates and returns a new ProcessingTime timer object. func InProcessingTime(Key string) ProcessingTime { return ProcessingTime{Family: Key} } From 5b012f52afa8ca0fd1ae94f6b3864f78febd17d6 Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Thu, 20 Apr 2023 00:58:02 -0400 Subject: [PATCH 15/22] new example --- sdks/go/examples/streaming_wordcap/wordcap.go | 199 ++------------- sdks/go/examples/timer_wordcap/wordcap.go | 227 ++++++++++++++++++ 2 files changed, 244 insertions(+), 182 deletions(-) create mode 100644 sdks/go/examples/timer_wordcap/wordcap.go diff --git a/sdks/go/examples/streaming_wordcap/wordcap.go b/sdks/go/examples/streaming_wordcap/wordcap.go index 120e1a295418..ddd9eab4e5f8 100644 --- a/sdks/go/examples/streaming_wordcap/wordcap.go +++ b/sdks/go/examples/streaming_wordcap/wordcap.go @@ -26,21 +26,16 @@ package main import ( "context" "flag" - "fmt" "os" - "time" + "strings" "github.com/apache/beam/sdks/v2/go/pkg/beam" - "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/mtime" - "github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf" - "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/io/rtrackers/offsetrange" + "github.com/apache/beam/sdks/v2/go/pkg/beam/io/pubsubio" "github.com/apache/beam/sdks/v2/go/pkg/beam/log" - "github.com/apache/beam/sdks/v2/go/pkg/beam/register" + "github.com/apache/beam/sdks/v2/go/pkg/beam/options/gcpopts" + "github.com/apache/beam/sdks/v2/go/pkg/beam/util/pubsubx" "github.com/apache/beam/sdks/v2/go/pkg/beam/x/beamx" "github.com/apache/beam/sdks/v2/go/pkg/beam/x/debug" - "golang.org/x/exp/slog" ) var ( @@ -55,192 +50,32 @@ var ( } ) -type Stateful struct { - ElementBag state.Bag[string] - TimerTime state.Value[int64] - MinTime state.Combining[int64, int64, int64] - - OutputState timers.ProcessingTime -} - -func NewStateful() *Stateful { - return &Stateful{ - ElementBag: state.MakeBagState[string]("elementBag"), - TimerTime: state.MakeValueState[int64]("timerTime"), - MinTime: state.MakeCombiningState[int64, int64, int64]("minTiInBag", func(a, b int64) int64 { - if a < b { - return a - } - return b - }), - - OutputState: timers.InProcessingTime("outputState"), - } -} - -func (s *Stateful) OnTimer(ctx context.Context, ts beam.EventTime, tp timers.Provider, key, timerKey, timerTag string) { - log.Infof(ctx, "timer fired on stateful for element: %v, timerKey: %v, timerTag: %v", key, timerKey, timerTag) - switch timerKey { - case "outputState": - log.Infof(ctx, "outputState fired on stateful") - s.OutputState.Clear(tp) - switch timerTag { - case "001": - log.Infof(ctx, "001 tag fired on outputState stateful") - s.OutputState.Set(tp, mtime.Now().ToTime().Add(1*time.Minute), timers.WithTag(timerTag)) - } - } -} - -func (s *Stateful) ProcessElement(ctx context.Context, ts beam.EventTime, sp state.Provider, tp timers.Provider, key, word string, emit func(string, string)) error { - log.Infof(ctx, "stateful dofn invoked key: %q word: %q", key, word) - - s.ElementBag.Add(sp, word) - s.MinTime.Add(sp, int64(ts)) - - toFire, ok, err := s.TimerTime.Read(sp) - if err != nil { - return err - } - if !ok { - toFire = int64(mtime.Now().Add(1 * time.Minute)) - } - minTime, _, err := s.MinTime.Read(sp) - if err != nil { - return err - } - - s.OutputState.Set(tp, mtime.Time(toFire).ToTime(), timers.WithOutputTimestamp(mtime.Time(minTime).ToTime()), timers.WithTag(word)) - s.TimerTime.Write(sp, toFire) - - return nil -} - -type eventtimeSDFStream struct { - RestSize, Mod, Fixed int64 - Sleep time.Duration -} - -func (fn *eventtimeSDFStream) Setup() error { - return nil -} - -func (fn *eventtimeSDFStream) CreateInitialRestriction(v beam.T) offsetrange.Restriction { - return offsetrange.Restriction{Start: 0, End: fn.RestSize} -} - -func (fn *eventtimeSDFStream) SplitRestriction(v beam.T, r offsetrange.Restriction) []offsetrange.Restriction { - // No split - return []offsetrange.Restriction{r} -} - -func (fn *eventtimeSDFStream) RestrictionSize(v beam.T, r offsetrange.Restriction) float64 { - return r.Size() -} - -func (fn *eventtimeSDFStream) CreateTracker(r offsetrange.Restriction) *sdf.LockRTracker { - return sdf.NewLockRTracker(offsetrange.NewTracker(r)) -} - -func (fn *eventtimeSDFStream) ProcessElement(ctx context.Context, _ *CWE, rt *sdf.LockRTracker, v beam.T, emit func(beam.EventTime, int64)) sdf.ProcessContinuation { - r := rt.GetRestriction().(offsetrange.Restriction) - i := r.Start - if r.Size() < 1 { - log.Debugf(ctx, "size 0 restriction, stoping to process sentinel %v", slog.Any("value", v)) - return sdf.StopProcessing() - } - slog.Debug("emitting element to restriction", slog.Any("value", v), slog.Group("restriction", - slog.Any("value", v), - slog.Float64("size", r.Size()), - slog.Int64("pos", i), - )) - if rt.TryClaim(i) { - v := (i % fn.Mod) + fn.Fixed - emit(mtime.Now(), v) - } - return sdf.ResumeProcessingIn(fn.Sleep) -} - -func (fn *eventtimeSDFStream) InitialWatermarkEstimatorState(_ beam.EventTime, _ offsetrange.Restriction, _ beam.T) int64 { - return int64(mtime.MinTimestamp) -} - -func (fn *eventtimeSDFStream) CreateWatermarkEstimator(initialState int64) *CWE { - return &CWE{Watermark: initialState} -} - -func (fn *eventtimeSDFStream) WatermarkEstimatorState(e *CWE) int64 { - return e.Watermark -} - -type CWE struct { - Watermark int64 // uses int64, since the SDK prevent mtime.Time from serialization. -} - -func (e *CWE) CurrentWatermark() time.Time { - return mtime.Time(e.Watermark).ToTime() -} - -func (e *CWE) ObserveTimestamp(ts time.Time) { - // We add 10 milliseconds to allow window boundaries to - // progress after emitting - e.Watermark = int64(mtime.FromTime(ts.Add(-90 * time.Millisecond))) -} - -func init() { - register.DoFn7x1[context.Context, beam.EventTime, state.Provider, timers.Provider, string, string, func(string, string), error](&Stateful{}) - register.Emitter2[string, string]() - register.DoFn5x1[context.Context, *CWE, *sdf.LockRTracker, beam.T, func(beam.EventTime, int64), sdf.ProcessContinuation]((*eventtimeSDFStream)(nil)) - register.Emitter2[beam.EventTime, int64]() -} - func main() { flag.Parse() beam.Init() ctx := context.Background() - //project := gcpopts.GetProject(ctx) + project := gcpopts.GetProject(ctx) log.Infof(ctx, "Publishing %v messages to: %v", len(data), *input) - // defer pubsubx.CleanupTopic(ctx, project, *input) - // sub, err := pubsubx.Publish(ctx, project, *input, data...) - // if err != nil { - // log.Fatal(ctx, err) - // } + defer pubsubx.CleanupTopic(ctx, project, *input) + sub, err := pubsubx.Publish(ctx, project, *input, data...) + if err != nil { + log.Fatal(ctx, err) + } - //log.Infof(ctx, "Running streaming wordcap with subscription: %v", sub.ID()) + log.Infof(ctx, "Running streaming wordcap with subscription: %v", sub.ID()) p := beam.NewPipeline() s := p.Root() - //col := pubsubio.Read(s, project, *input, &pubsubio.ReadOptions{Subscription: sub.ID()}) - // col = beam.WindowInto(s, window.NewFixedWindows(60*time.Second), col) - - // str := beam.ParDo(s, func(b []byte) string { - // return (string)(b) - // }, col) - - imp := beam.Impulse(s) - elms := 3 - out := beam.ParDo(s, &eventtimeSDFStream{ - Sleep: time.Second, - RestSize: int64(elms), - Mod: int64(elms), - Fixed: 1, - }, imp) - // out = beam.WindowInto(s, window.NewFixedWindows(10*time.Second), out) - str := beam.ParDo(s, func(b int64) string { - return fmt.Sprintf("%03d", b) - }, out) - - keyed := beam.ParDo(s, func(ctx context.Context, ts beam.EventTime, s string) (string, string) { - log.Infof(ctx, "adding key ts: %v now: %v word: %v", ts.ToTime(), time.Now(), s) - return "test", s - }, str) - - timed := beam.ParDo(s, NewStateful(), keyed) - debug.Printf(s, "post stateful: %v", timed) + col := pubsubio.Read(s, project, *input, &pubsubio.ReadOptions{Subscription: sub.ID()}) + str := beam.ParDo(s, func(b []byte) string { + return (string)(b) + }, col) + cap := beam.ParDo(s, strings.ToUpper, str) + debug.Print(s, cap) if err := beamx.Run(context.Background(), p); err != nil { log.Exitf(ctx, "Failed to execute job: %v", err) diff --git a/sdks/go/examples/timer_wordcap/wordcap.go b/sdks/go/examples/timer_wordcap/wordcap.go new file mode 100644 index 000000000000..bdf382b1ae16 --- /dev/null +++ b/sdks/go/examples/timer_wordcap/wordcap.go @@ -0,0 +1,227 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// timer_wordcap is a toy streaming pipeline that uses State and Timers with PubSub. It +// does the following: +// +// (1) create a topic and publish a few messages to it +// (2) Set user state and timer +// +// NOTE: it only runs on Dataflow and must be manually cancelled. +package main + +import ( + "context" + "flag" + "fmt" + "os" + "time" + + "github.com/apache/beam/sdks/v2/go/pkg/beam" + "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/mtime" + "github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf" + "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/io/rtrackers/offsetrange" + "github.com/apache/beam/sdks/v2/go/pkg/beam/log" + "github.com/apache/beam/sdks/v2/go/pkg/beam/register" + "github.com/apache/beam/sdks/v2/go/pkg/beam/x/beamx" + "github.com/apache/beam/sdks/v2/go/pkg/beam/x/debug" + "golang.org/x/exp/slog" +) + +var ( + input = flag.String("input", os.ExpandEnv("$USER-wordcap"), "Pubsub input topic.") +) + +var ( + data = []string{ + "foo", + "bar", + "baz", + } +) + +type Stateful struct { + ElementBag state.Bag[string] + TimerTime state.Value[int64] + MinTime state.Combining[int64, int64, int64] + + OutputState timers.ProcessingTime +} + +func NewStateful() *Stateful { + return &Stateful{ + ElementBag: state.MakeBagState[string]("elementBag"), + TimerTime: state.MakeValueState[int64]("timerTime"), + MinTime: state.MakeCombiningState[int64, int64, int64]("minTiInBag", func(a, b int64) int64 { + if a < b { + return a + } + return b + }), + + OutputState: timers.InProcessingTime("outputState"), + } +} + +func (s *Stateful) OnTimer(ctx context.Context, ts beam.EventTime, tp timers.Provider, key, timerKey, timerTag string) { + switch timerKey { + case "outputState": + log.Infof(ctx, "Timer outputState fired on stateful for element: %v.", key) + s.OutputState.Clear(tp) + switch timerTag { + case "001": + log.Infof(ctx, "Timer with tag 001 fired on outputState stateful DoFn.") + s.OutputState.Set(tp, mtime.Now().ToTime().Add(1*time.Minute), timers.WithTag(timerTag)) + } + } +} + +func (s *Stateful) ProcessElement(ctx context.Context, ts beam.EventTime, sp state.Provider, tp timers.Provider, key, word string, emit func(string, string)) error { + s.ElementBag.Add(sp, word) + s.MinTime.Add(sp, int64(ts)) + + toFire, ok, err := s.TimerTime.Read(sp) + if err != nil { + return err + } + if !ok { + toFire = int64(mtime.Now().Add(1 * time.Minute)) + } + minTime, _, err := s.MinTime.Read(sp) + if err != nil { + return err + } + + s.OutputState.Set(tp, mtime.Time(toFire).ToTime(), timers.WithOutputTimestamp(mtime.Time(minTime).ToTime()), timers.WithTag(word)) + s.TimerTime.Write(sp, toFire) + + return nil +} + +type eventtimeSDFStream struct { + RestSize, Mod, Fixed int64 + Sleep time.Duration +} + +func (fn *eventtimeSDFStream) Setup() error { + return nil +} + +func (fn *eventtimeSDFStream) CreateInitialRestriction(v beam.T) offsetrange.Restriction { + return offsetrange.Restriction{Start: 0, End: fn.RestSize} +} + +func (fn *eventtimeSDFStream) SplitRestriction(v beam.T, r offsetrange.Restriction) []offsetrange.Restriction { + // No split + return []offsetrange.Restriction{r} +} + +func (fn *eventtimeSDFStream) RestrictionSize(v beam.T, r offsetrange.Restriction) float64 { + return r.Size() +} + +func (fn *eventtimeSDFStream) CreateTracker(r offsetrange.Restriction) *sdf.LockRTracker { + return sdf.NewLockRTracker(offsetrange.NewTracker(r)) +} + +func (fn *eventtimeSDFStream) ProcessElement(ctx context.Context, _ *CWE, rt *sdf.LockRTracker, v beam.T, emit func(beam.EventTime, int64)) sdf.ProcessContinuation { + r := rt.GetRestriction().(offsetrange.Restriction) + i := r.Start + if r.Size() < 1 { + log.Debugf(ctx, "size 0 restriction, stoping to process sentinel %v", slog.Any("value", v)) + return sdf.StopProcessing() + } + slog.Debug("emitting element to restriction", slog.Any("value", v), slog.Group("restriction", + slog.Any("value", v), + slog.Float64("size", r.Size()), + slog.Int64("pos", i), + )) + if rt.TryClaim(i) { + v := (i % fn.Mod) + fn.Fixed + emit(mtime.Now(), v) + } + return sdf.ResumeProcessingIn(fn.Sleep) +} + +func (fn *eventtimeSDFStream) InitialWatermarkEstimatorState(_ beam.EventTime, _ offsetrange.Restriction, _ beam.T) int64 { + return int64(mtime.MinTimestamp) +} + +func (fn *eventtimeSDFStream) CreateWatermarkEstimator(initialState int64) *CWE { + return &CWE{Watermark: initialState} +} + +func (fn *eventtimeSDFStream) WatermarkEstimatorState(e *CWE) int64 { + return e.Watermark +} + +type CWE struct { + Watermark int64 // uses int64, since the SDK prevent mtime.Time from serialization. +} + +func (e *CWE) CurrentWatermark() time.Time { + return mtime.Time(e.Watermark).ToTime() +} + +func (e *CWE) ObserveTimestamp(ts time.Time) { + // We add 10 milliseconds to allow window boundaries to + // progress after emitting + e.Watermark = int64(mtime.FromTime(ts.Add(-90 * time.Millisecond))) +} + +func init() { + register.DoFn7x1[context.Context, beam.EventTime, state.Provider, timers.Provider, string, string, func(string, string), error](&Stateful{}) + register.Emitter2[string, string]() + register.DoFn5x1[context.Context, *CWE, *sdf.LockRTracker, beam.T, func(beam.EventTime, int64), sdf.ProcessContinuation]((*eventtimeSDFStream)(nil)) + register.Emitter2[beam.EventTime, int64]() +} + +func main() { + flag.Parse() + beam.Init() + + ctx := context.Background() + + log.Infof(ctx, "Publishing %v messages to: %v", len(data), *input) + + p := beam.NewPipeline() + s := p.Root() + + imp := beam.Impulse(s) + elms := 3 + out := beam.ParDo(s, &eventtimeSDFStream{ + Sleep: time.Second, + RestSize: int64(elms), + Mod: int64(elms), + Fixed: 1, + }, imp) + + str := beam.ParDo(s, func(b int64) string { + return fmt.Sprintf("%03d", b) + }, out) + + keyed := beam.ParDo(s, func(ctx context.Context, ts beam.EventTime, s string) (string, string) { + return "test", s + }, str) + + timed := beam.ParDo(s, NewStateful(), keyed) + debug.Printf(s, "post stateful: %v", timed) + + if err := beamx.Run(context.Background(), p); err != nil { + log.Exitf(ctx, "Failed to execute job: %v", err) + } +} From 64d47592b679de9f3745bd3c96a5092d10d58d54 Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Thu, 20 Apr 2023 08:26:55 -0400 Subject: [PATCH 16/22] fix static lint --- sdks/go/pkg/beam/core/runtime/exec/translate.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/go/pkg/beam/core/runtime/exec/translate.go b/sdks/go/pkg/beam/core/runtime/exec/translate.go index 758c3c0175e8..62b5f25528ed 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/translate.go +++ b/sdks/go/pkg/beam/core/runtime/exec/translate.go @@ -600,7 +600,7 @@ func (b *builder) makeLink(from string, id linkID) (Node, error) { if err != nil { return nil, err } - for key, _ := range userTimers { + for key := range userTimers { timerIDToCoder[key] = coder.NewT(ec, wc) } n.Timer = NewUserTimerAdapter(sID, coder.NewW(ec, wc), timerIDToCoder) From afff87ebfa8bd77576f059bace7027fa1e860eab Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Tue, 25 Apr 2023 15:15:16 -0400 Subject: [PATCH 17/22] support emitters --- sdks/go/examples/timer_wordcap/wordcap.go | 3 ++- sdks/go/pkg/beam/core/runtime/exec/pardo.go | 12 ++++++++++++ 2 files changed, 14 insertions(+), 1 deletion(-) diff --git a/sdks/go/examples/timer_wordcap/wordcap.go b/sdks/go/examples/timer_wordcap/wordcap.go index bdf382b1ae16..2a82c4ab4e60 100644 --- a/sdks/go/examples/timer_wordcap/wordcap.go +++ b/sdks/go/examples/timer_wordcap/wordcap.go @@ -77,7 +77,7 @@ func NewStateful() *Stateful { } } -func (s *Stateful) OnTimer(ctx context.Context, ts beam.EventTime, tp timers.Provider, key, timerKey, timerTag string) { +func (s *Stateful) OnTimer(ctx context.Context, ts beam.EventTime, tp timers.Provider, key, timerKey, timerTag string, emit func(string, string)) { switch timerKey { case "outputState": log.Infof(ctx, "Timer outputState fired on stateful for element: %v.", key) @@ -86,6 +86,7 @@ func (s *Stateful) OnTimer(ctx context.Context, ts beam.EventTime, tp timers.Pro case "001": log.Infof(ctx, "Timer with tag 001 fired on outputState stateful DoFn.") s.OutputState.Set(tp, mtime.Now().ToTime().Add(1*time.Minute), timers.WithTag(timerTag)) + emit(timerKey, timerTag) } } } diff --git a/sdks/go/pkg/beam/core/runtime/exec/pardo.go b/sdks/go/pkg/beam/core/runtime/exec/pardo.go index 6b56207cc039..e75a91e8e1af 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/pardo.go +++ b/sdks/go/pkg/beam/core/runtime/exec/pardo.go @@ -356,12 +356,24 @@ func (n *ParDo) invokeDataFn(ctx context.Context, pn typex.PaneInfo, ws []typex. } func (n *ParDo) InvokeTimerFn(ctx context.Context, fn *funcx.Fn, timerFamilyID string, tmap TimerRecv) (*FullValue, error) { + // Defer side input clean-up in case of panic + var err error + defer func() { + if postErr := n.postInvoke(); postErr != nil { + err = postErr + } + }() + if err := n.preInvoke(ctx, tmap.Windows, tmap.HoldTimestamp); err != nil { + return nil, err + } + var extra []any extra = append(extra, timerFamilyID) if tmap.Tag != "" { extra = append(extra, tmap.Tag) } + extra = append(extra, n.cache.extra...) val, err := InvokeWithOpts(ctx, fn, tmap.Pane, tmap.Windows, tmap.HoldTimestamp, InvokeOpts{ opt: &MainInput{Key: *tmap.Key}, bf: n.bf, From 601dd58a5763e75c06c71cf98ae7c2d4c9591f8d Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Tue, 25 Apr 2023 15:19:01 -0400 Subject: [PATCH 18/22] allow input col of CoGBK as well --- sdks/go/pkg/beam/pardo.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/go/pkg/beam/pardo.go b/sdks/go/pkg/beam/pardo.go index c8d23da443b5..e69ce5a08e82 100644 --- a/sdks/go/pkg/beam/pardo.go +++ b/sdks/go/pkg/beam/pardo.go @@ -120,7 +120,7 @@ func TryParDo(s Scope, dofn any, col PCollection, opts ...Option) ([]PCollection pipelineTimers := fn.PipelineTimers() if len(pipelineTimers) > 0 { // check if input is KV type - if !typex.IsKV(col.Type()) { + if !typex.IsKV(col.Type()) || !typex.IsCoGBK(col.Type()) { return nil, addParDoCtx(errors.New("DoFn input should be keyed to be used with timers."), s) } c, err := inferCoder(typex.New(reflect.TypeOf(col.Type()))) From 6ea3969162c8c558141c10e513893a6bdc44074f Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Wed, 26 Apr 2023 12:02:23 -0400 Subject: [PATCH 19/22] unit tests, periodic impulse, minor refactor --- sdks/go/examples/timer_wordcap/wordcap.go | 118 +++--------------- sdks/go/pkg/beam/core/funcx/fn_test.go | 115 ++++++++++++++++- sdks/go/pkg/beam/core/graph/fn.go | 40 ++++-- sdks/go/pkg/beam/core/graph/fn_test.go | 57 ++++++++- .../pkg/beam/core/runtime/exec/datasource.go | 6 +- sdks/go/pkg/beam/core/runtime/exec/pardo.go | 17 ++- sdks/go/pkg/beam/core/timers/timers.go | 16 +-- sdks/go/pkg/beam/pardo.go | 2 +- 8 files changed, 234 insertions(+), 137 deletions(-) diff --git a/sdks/go/examples/timer_wordcap/wordcap.go b/sdks/go/examples/timer_wordcap/wordcap.go index 2a82c4ab4e60..96a6d2439656 100644 --- a/sdks/go/examples/timer_wordcap/wordcap.go +++ b/sdks/go/examples/timer_wordcap/wordcap.go @@ -23,35 +23,22 @@ package main import ( + "bytes" "context" + "encoding/binary" "flag" "fmt" - "os" "time" "github.com/apache/beam/sdks/v2/go/pkg/beam" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/mtime" - "github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf" "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/io/rtrackers/offsetrange" "github.com/apache/beam/sdks/v2/go/pkg/beam/log" "github.com/apache/beam/sdks/v2/go/pkg/beam/register" + "github.com/apache/beam/sdks/v2/go/pkg/beam/transforms/periodic" "github.com/apache/beam/sdks/v2/go/pkg/beam/x/beamx" "github.com/apache/beam/sdks/v2/go/pkg/beam/x/debug" - "golang.org/x/exp/slog" -) - -var ( - input = flag.String("input", os.ExpandEnv("$USER-wordcap"), "Pubsub input topic.") -) - -var ( - data = []string{ - "foo", - "bar", - "baz", - } ) type Stateful struct { @@ -83,9 +70,8 @@ func (s *Stateful) OnTimer(ctx context.Context, ts beam.EventTime, tp timers.Pro log.Infof(ctx, "Timer outputState fired on stateful for element: %v.", key) s.OutputState.Clear(tp) switch timerTag { - case "001": - log.Infof(ctx, "Timer with tag 001 fired on outputState stateful DoFn.") - s.OutputState.Set(tp, mtime.Now().ToTime().Add(1*time.Minute), timers.WithTag(timerTag)) + case "1": + log.Infof(ctx, "Timer with tag 1 fired on outputState stateful DoFn.") emit(timerKey, timerTag) } } @@ -107,87 +93,15 @@ func (s *Stateful) ProcessElement(ctx context.Context, ts beam.EventTime, sp sta return err } - s.OutputState.Set(tp, mtime.Time(toFire).ToTime(), timers.WithOutputTimestamp(mtime.Time(minTime).ToTime()), timers.WithTag(word)) + s.OutputState.Set(tp, time.UnixMilli(toFire), timers.WithOutputTimestamp(time.UnixMilli(minTime)), timers.WithTag(word)) s.TimerTime.Write(sp, toFire) return nil } -type eventtimeSDFStream struct { - RestSize, Mod, Fixed int64 - Sleep time.Duration -} - -func (fn *eventtimeSDFStream) Setup() error { - return nil -} - -func (fn *eventtimeSDFStream) CreateInitialRestriction(v beam.T) offsetrange.Restriction { - return offsetrange.Restriction{Start: 0, End: fn.RestSize} -} - -func (fn *eventtimeSDFStream) SplitRestriction(v beam.T, r offsetrange.Restriction) []offsetrange.Restriction { - // No split - return []offsetrange.Restriction{r} -} - -func (fn *eventtimeSDFStream) RestrictionSize(v beam.T, r offsetrange.Restriction) float64 { - return r.Size() -} - -func (fn *eventtimeSDFStream) CreateTracker(r offsetrange.Restriction) *sdf.LockRTracker { - return sdf.NewLockRTracker(offsetrange.NewTracker(r)) -} - -func (fn *eventtimeSDFStream) ProcessElement(ctx context.Context, _ *CWE, rt *sdf.LockRTracker, v beam.T, emit func(beam.EventTime, int64)) sdf.ProcessContinuation { - r := rt.GetRestriction().(offsetrange.Restriction) - i := r.Start - if r.Size() < 1 { - log.Debugf(ctx, "size 0 restriction, stoping to process sentinel %v", slog.Any("value", v)) - return sdf.StopProcessing() - } - slog.Debug("emitting element to restriction", slog.Any("value", v), slog.Group("restriction", - slog.Any("value", v), - slog.Float64("size", r.Size()), - slog.Int64("pos", i), - )) - if rt.TryClaim(i) { - v := (i % fn.Mod) + fn.Fixed - emit(mtime.Now(), v) - } - return sdf.ResumeProcessingIn(fn.Sleep) -} - -func (fn *eventtimeSDFStream) InitialWatermarkEstimatorState(_ beam.EventTime, _ offsetrange.Restriction, _ beam.T) int64 { - return int64(mtime.MinTimestamp) -} - -func (fn *eventtimeSDFStream) CreateWatermarkEstimator(initialState int64) *CWE { - return &CWE{Watermark: initialState} -} - -func (fn *eventtimeSDFStream) WatermarkEstimatorState(e *CWE) int64 { - return e.Watermark -} - -type CWE struct { - Watermark int64 // uses int64, since the SDK prevent mtime.Time from serialization. -} - -func (e *CWE) CurrentWatermark() time.Time { - return mtime.Time(e.Watermark).ToTime() -} - -func (e *CWE) ObserveTimestamp(ts time.Time) { - // We add 10 milliseconds to allow window boundaries to - // progress after emitting - e.Watermark = int64(mtime.FromTime(ts.Add(-90 * time.Millisecond))) -} - func init() { register.DoFn7x1[context.Context, beam.EventTime, state.Provider, timers.Provider, string, string, func(string, string), error](&Stateful{}) register.Emitter2[string, string]() - register.DoFn5x1[context.Context, *CWE, *sdf.LockRTracker, beam.T, func(beam.EventTime, int64), sdf.ProcessContinuation]((*eventtimeSDFStream)(nil)) register.Emitter2[beam.EventTime, int64]() } @@ -197,23 +111,21 @@ func main() { ctx := context.Background() - log.Infof(ctx, "Publishing %v messages to: %v", len(data), *input) - p := beam.NewPipeline() s := p.Root() - imp := beam.Impulse(s) - elms := 3 - out := beam.ParDo(s, &eventtimeSDFStream{ - Sleep: time.Second, - RestSize: int64(elms), - Mod: int64(elms), - Fixed: 1, - }, imp) + out := periodic.Impulse(s, time.Now(), time.Now().Add(5*time.Minute), 5*time.Second, true) + + intOut := beam.ParDo(s, func(b []byte) int64 { + var val int64 + buf := bytes.NewReader(b) + binary.Read(buf, binary.BigEndian, &val) + return val + }, out) str := beam.ParDo(s, func(b int64) string { return fmt.Sprintf("%03d", b) - }, out) + }, intOut) keyed := beam.ParDo(s, func(ctx context.Context, ts beam.EventTime, s string) (string, string) { return "test", s diff --git a/sdks/go/pkg/beam/core/funcx/fn_test.go b/sdks/go/pkg/beam/core/funcx/fn_test.go index 6890adc8ec39..5e1a2c569647 100644 --- a/sdks/go/pkg/beam/core/funcx/fn_test.go +++ b/sdks/go/pkg/beam/core/funcx/fn_test.go @@ -27,6 +27,7 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/mtime" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf" "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/core/typex" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/util/reflectx" ) @@ -114,6 +115,11 @@ func TestNew(t *testing.T) { Fn: func(sdf.RTracker, state.Provider, []byte) {}, Param: []FnParamKind{FnRTracker, FnStateProvider, FnValue}, }, + { + Name: "good11", + Fn: func(typex.PaneInfo, typex.Window, typex.EventTime, state.Provider, timers.Provider, []byte) {}, + Param: []FnParamKind{FnPane, FnWindow, FnEventTime, FnStateProvider, FnTimerProvider, FnValue}, + }, { Name: "good-method", Fn: foo{1}.Do, @@ -227,6 +233,21 @@ func TestNew(t *testing.T) { Fn: func(int, state.Provider) {}, Err: errStateProviderPrecedence, }, + { + Name: "errInputPrecedence- TimerProvider before RTracker", + Fn: func(timers.Provider, sdf.RTracker, int) {}, + Err: errRTrackerPrecedence, + }, + { + Name: "errInputPrecedence- TimerProvider after output", + Fn: func(int, timers.Provider) {}, + Err: errTimerProviderPrecedence, + }, + { + Name: "errInputPrecedence- TimerProvider before StateProvider", + Fn: func(int, timers.Provider, state.Provider) {}, + Err: errTimerProviderPrecedence, + }, { Name: "errInputPrecedence- input after output", Fn: func(int, func(int), int) {}, @@ -487,7 +508,7 @@ func TestWindow(t *testing.T) { } fn := &Fn{Param: params} - // Validate we get expected results for pane function. + // Validate we get expected results for Window function. pos, exists := fn.Window() if exists != test.Exists { t.Errorf("Window(%v) - exists: got %v, want %v", params, exists, test.Exists) @@ -531,7 +552,7 @@ func TestBundleFinalization(t *testing.T) { } fn := &Fn{Param: params} - // Validate we get expected results for pane function. + // Validate we get expected results for BundleFinalization function. pos, exists := fn.BundleFinalization() if exists != test.Exists { t.Errorf("BundleFinalization(%v) - exists: got %v, want %v", params, exists, test.Exists) @@ -575,7 +596,7 @@ func TestWatermarkEstimator(t *testing.T) { } fn := &Fn{Param: params} - // Validate we get expected results for pane function. + // Validate we get expected results for WatermarkEstimator function. pos, exists := fn.WatermarkEstimator() if exists != test.Exists { t.Errorf("WatermarkEstimator(%v) - exists: got %v, want %v", params, exists, test.Exists) @@ -587,6 +608,94 @@ func TestWatermarkEstimator(t *testing.T) { } } +func TestTimerProvider(t *testing.T) { + tests := []struct { + Name string + Params []FnParamKind + Pos int + Exists bool + }{ + { + Name: "TimerProvider input", + Params: []FnParamKind{FnContext, FnWindow, FnEventTime, FnTimerProvider}, + Pos: 3, + Exists: true, + }, + { + Name: "no TimerProvider input", + Params: []FnParamKind{FnContext, FnWindow, FnEventTime}, + Pos: -1, + Exists: false, + }, + } + + for _, test := range tests { + test := test + t.Run(test.Name, func(t *testing.T) { + // Create a Fn with a filled params list. + params := make([]FnParam, len(test.Params)) + for i, kind := range test.Params { + params[i].Kind = kind + params[i].T = nil + } + fn := &Fn{Param: params} + + // Validate we get expected results for TimerProvider function. + pos, exists := fn.TimerProvider() + if exists != test.Exists { + t.Errorf("TimerProvider(%v) - exists: got %v, want %v", params, exists, test.Exists) + } + if pos != test.Pos { + t.Errorf("TimerProvider(%v) - pos: got %v, want %v", params, pos, test.Pos) + } + }) + } +} + +func TestStateProvider(t *testing.T) { + tests := []struct { + Name string + Params []FnParamKind + Pos int + Exists bool + }{ + { + Name: "StateProvider input", + Params: []FnParamKind{FnContext, FnWindow, FnStateProvider}, + Pos: 2, + Exists: true, + }, + { + Name: "no StateProvider input", + Params: []FnParamKind{FnContext, FnWindow}, + Pos: -1, + Exists: false, + }, + } + + for _, test := range tests { + test := test + t.Run(test.Name, func(t *testing.T) { + // Create a Fn with a filled params list. + params := make([]FnParam, len(test.Params)) + for i, kind := range test.Params { + params[i].Kind = kind + params[i].T = nil + } + fn := &Fn{Param: params} + + // Validate we get expected results for StateProvider function. + pos, exists := fn.StateProvider() + if exists != test.Exists { + t.Errorf("StateProvider(%v) - exists: got %v, want %v", params, exists, test.Exists) + } + if pos != test.Pos { + t.Errorf("StateProvider(%v) - pos: got %v, want %v", params, pos, test.Pos) + } + }) + } +} + func TestInputs(t *testing.T) { tests := []struct { Name string diff --git a/sdks/go/pkg/beam/core/graph/fn.go b/sdks/go/pkg/beam/core/graph/fn.go index 800068bbdc50..63e35e6eec20 100644 --- a/sdks/go/pkg/beam/core/graph/fn.go +++ b/sdks/go/pkg/beam/core/graph/fn.go @@ -637,7 +637,7 @@ func AsDoFn(fn *Fn, numMainIn mainInputs) (*DoFn, error) { return nil, addContext(err, fn) } - err = validateTimer(doFn) + err = validateTimer(doFn, numMainIn) if err != nil { return nil, addContext(err, fn) } @@ -1386,33 +1386,51 @@ func validateState(fn *DoFn, numIn mainInputs) error { } func validateOnTimerFn(fn *DoFn) error { - if _, ok := fn.OnTimerFn(); !ok { + if _, ok := fn.methods[onTimerName]; !ok { err := errors.Errorf("OnTimer function not defined for DoFn: %v", fn.Name()) return errors.SetTopLevelMsgf(err, "OnTimer function not defined for DoFn: %v. Ensure that OnTimer function is implemented for the DoFn.", fn.Name()) } - return nil -} + if _, ok := fn.methods[onTimerName].TimerProvider(); !ok { + err := errors.Errorf("OnTimer function doesn't use a TimerProvider, but Timer field is attached to the DoFn(%v): %v", fn.Name(), fn.PipelineTimers()) + return errors.SetTopLevelMsgf(err, "OnTimer function doesn't use a TimerProvider, but Timer field is attached to the DoFn: %v"+ + ", Ensure that you are using the TimerProvider to set and clear the timers.", fn.Name(), fn.PipelineTimers()) + } -func validateTimer(fn *DoFn) error { - if fn.Fn == nil { - return nil + _, otNum, otExists := fn.methods[onTimerName].Emits() + _, peNum, peExists := fn.methods[processElementName].Emits() + + if otExists && peExists { + if otNum != peNum { + return fmt.Errorf("OnTimer and ProcessElement functions for DoFn should have exactly same emitters, no. of emitters used in OnTimer: %v, no. of emitters used in ProcessElement: %v", otNum, peNum) + } + } else { + return fmt.Errorf("OnTimer and ProcessElement functions for DoFn should have exactly same emitters, emitters used in OnTimer: %v, emitters used in ProcessElement: %v", otExists, peExists) } + return nil +} + +func validateTimer(fn *DoFn, numIn mainInputs) error { pt := fn.PipelineTimers() - if _, ok := fn.Fn.TimerProvider(); ok { + if _, ok := fn.methods[processElementName].TimerProvider(); ok { + if numIn == MainSingle { + err := errors.Errorf("ProcessElement uses a TimerProvider, but is not keyed") + return errors.SetTopLevelMsgf(err, "ProcessElement uses a TimerProvider, but is not keyed. "+ + "All stateful DoFns must take a key/value pair as an input.") + } if len(pt) == 0 { err := errors.New("ProcessElement uses a TimerProvider, but no Timer fields are defined in the DoFn") return errors.SetTopLevelMsgf(err, "ProcessElement uses a TimerProvider, but no timer fields are defined in the DoFn"+ - ", Ensure that you are including the exported timer field in the DoFn that you're using to set/clear timers.") + ", Ensure that your DoFn exports the Timer fields used to set and clear timers.") } timerKeys := make(map[string]timers.PipelineTimer) for _, t := range pt { k := t.TimerFamily() if timer, ok := timerKeys[k]; ok { err := errors.Errorf("Duplicate timer key %v", k) - return errors.SetTopLevelMsgf(err, "Duplicate timer key %v used by %v and %v. Ensure that keys are unique per DoFn", k, timer, t) + return errors.SetTopLevelMsgf(err, "Duplicate timer family ID %v used by %v and %v. Ensure that timer family IDs are unique per DoFn", k, timer, t) } else { timerKeys[k] = t } @@ -1424,7 +1442,7 @@ func validateTimer(fn *DoFn) error { if len(pt) > 0 { err := errors.Errorf("ProcessElement doesn't use a TimerProvider, but Timer field is attached to the DoFn: %v", pt) return errors.SetTopLevelMsgf(err, "ProcessElement doesn't use a TimerProvider, but Timer field is attached to the DoFn: %v"+ - ", Ensure that you are using the TimerProvider to set/clear the timers.", pt) + ", Ensure that you are using the TimerProvider to set and clear the timers.", pt) } if err := validateOnTimerFn(fn); err == nil { actualErr := errors.New("OnTimer function is defined for the DoFn but no TimerProvider defined in ProcessElement.") diff --git a/sdks/go/pkg/beam/core/graph/fn_test.go b/sdks/go/pkg/beam/core/graph/fn_test.go index cf44761d4f3c..727230d83035 100644 --- a/sdks/go/pkg/beam/core/graph/fn_test.go +++ b/sdks/go/pkg/beam/core/graph/fn_test.go @@ -26,6 +26,7 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf" "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/core/typex" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/util/reflectx" ) @@ -53,8 +54,8 @@ func TestNewDoFn(t *testing.T) { {dfn: &GoodDoFnCoGbk2{}, opt: CoGBKMainInput(3)}, {dfn: &GoodDoFnCoGbk7{}, opt: CoGBKMainInput(8)}, {dfn: &GoodDoFnCoGbk1wSide{}, opt: NumMainInputs(MainKv)}, - {dfn: &GoodStatefulDoFn{}, opt: NumMainInputs(MainKv)}, - {dfn: &GoodStatefulDoFn2{}, opt: NumMainInputs(MainKv)}, + {dfn: &GoodStatefulDoFn{Timer1: timers.InProcessingTime("processingTimeTimer")}, opt: NumMainInputs(MainKv)}, + {dfn: &GoodStatefulDoFn2{Timer1: timers.InEventTime("eventTimeTimer")}, opt: NumMainInputs(MainKv)}, {dfn: &GoodStatefulDoFn3{State1: state.MakeCombiningState[int, int, int]("state1", func(a, b int) int { return a * b })}, opt: NumMainInputs(MainKv)}, @@ -103,9 +104,13 @@ func TestNewDoFn(t *testing.T) { {dfn: &BadDoFnReturnValuesInFinishBundle{}}, {dfn: &BadDoFnReturnValuesInSetup{}}, {dfn: &BadDoFnReturnValuesInTeardown{}}, + // Validate stateful DoFn {dfn: &BadStatefulDoFnNoStateProvider{State1: state.Value[int](state.MakeValueState[int]("state1"))}}, {dfn: &BadStatefulDoFnNoStateFields{}}, - {dfn: &BadStatefulDoFnNoKV{State1: state.Value[int](state.MakeValueState[int]("state1"))}, numInputs: 1}, + {dfn: &BadStatefulDoFnNoKV{State1: state.Value[int](state.MakeValueState[int]("state1")), Timer1: timers.InEventTime("timer1")}, numInputs: 1}, + {dfn: &BadStatefulDoFnNoTimerProvider{Timer1: timers.InEventTime("timer1")}, numInputs: 2}, + {dfn: &BadStatefulDoFnNoTimerFields{}, numInputs: 2}, + {dfn: &BadStatefulDoFnNoOnTimer{Timer1: timers.InEventTime("timer1")}, numInputs: 2}, } for _, test := range tests { t.Run(reflect.TypeOf(test.dfn).String(), func(t *testing.T) { @@ -1174,17 +1179,27 @@ func (fn *GoodStatefulWatermarkEstimatingKv) WatermarkEstimatorState(estimator * type GoodStatefulDoFn struct { State1 state.Value[int] + Timer1 timers.ProcessingTime } -func (fn *GoodStatefulDoFn) ProcessElement(state.Provider, int, int) int { +func (fn *GoodStatefulDoFn) ProcessElement(state.Provider, timers.Provider, int, int) int { + return 0 +} + +func (fn *GoodStatefulDoFn) OnTimer(state.Provider, timers.Provider, int) int { return 0 } type GoodStatefulDoFn2 struct { State1 state.Bag[int] + Timer1 timers.EventTime } -func (fn *GoodStatefulDoFn2) ProcessElement(state.Provider, int, int) int { +func (fn *GoodStatefulDoFn2) ProcessElement(state.Provider, timers.Provider, int, int) int { + return 0 +} + +func (fn *GoodStatefulDoFn2) OnTimer(state.Provider, timers.Provider, int) int { return 0 } @@ -1593,12 +1608,44 @@ func (fn *BadStatefulDoFnNoStateFields) ProcessElement(state.Provider, int) int type BadStatefulDoFnNoKV struct { State1 state.Value[int] + Timer1 timers.EventTime } func (fn *BadStatefulDoFnNoKV) ProcessElement(state.Provider, int, int) int { return 0 } +type BadStatefulDoFnNoTimerProvider struct { + Timer1 timers.EventTime +} + +func (fn *BadStatefulDoFnNoTimerProvider) ProcessElement(int, int) int { + return 0 +} + +func (fn *BadStatefulDoFnNoTimerProvider) OnTimer(timers.Provider, int) int { + return 0 +} + +type BadStatefulDoFnNoTimerFields struct { +} + +func (fn *BadStatefulDoFnNoTimerFields) ProcessElement(timers.Provider, int, int) int { + return 0 +} + +func (fn *BadStatefulDoFnNoTimerFields) OnTimer(timers.Provider, int) int { + return 0 +} + +type BadStatefulDoFnNoOnTimer struct { + Timer1 timers.EventTime +} + +func (fn *BadStatefulDoFnNoOnTimer) ProcessElement(timers.Provider, int, int) int { + return 0 +} + // Examples of correct CombineFn signatures type MyAccum struct{} diff --git a/sdks/go/pkg/beam/core/runtime/exec/datasource.go b/sdks/go/pkg/beam/core/runtime/exec/datasource.go index 1e6f9532fbe1..7c1796008ee5 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/datasource.go +++ b/sdks/go/pkg/beam/core/runtime/exec/datasource.go @@ -199,6 +199,7 @@ func (n *DataSource) Process(ctx context.Context) ([]*Checkpoint, error) { var cp ElementDecoder // Decoder for the primary element or the key in CoGBKs. var cvs []ElementDecoder // Decoders for each value stream in CoGBKs. + var dc ElementDecoder // Decoder for timer data switch { case coder.IsCoGBK(c): @@ -211,6 +212,10 @@ func (n *DataSource) Process(ctx context.Context) ([]*Checkpoint, error) { cp = MakeElementDecoder(c) } + if len(c.Components) > 0 { + dc = MakeElementDecoder(c.Components[0]) + } + var checkpoints []*Checkpoint err := n.process(ctx, func(bcr *byteCountReader, ptransformID string) error { for { @@ -263,7 +268,6 @@ func (n *DataSource) Process(ctx context.Context) ([]*Checkpoint, error) { } }, func(bcr *byteCountReader, ptransformID, timerFamilyID string) error { - dc := MakeElementDecoder(coder.SkipW(c).Components[0]) tmap, err := decodeTimer(dc, wc, bcr) if err != nil { return errors.WithContext(err, "error decoding timer in datasource") diff --git a/sdks/go/pkg/beam/core/runtime/exec/pardo.go b/sdks/go/pkg/beam/core/runtime/exec/pardo.go index e75a91e8e1af..c76dcccbee6e 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/pardo.go +++ b/sdks/go/pkg/beam/core/runtime/exec/pardo.go @@ -48,10 +48,11 @@ type ParDo struct { bf *bundleFinalizer we sdf.WatermarkEstimator - Timer UserTimerAdapter - timerManager DataManager - reader StateReader - cache *cacheElm + onTimerInvoker *invoker + Timer UserTimerAdapter + timerManager DataManager + reader StateReader + cache *cacheElm status Status err errorx.GuardedError @@ -88,6 +89,9 @@ func (n *ParDo) Up(ctx context.Context) error { } n.status = Up n.inv = newInvoker(n.Fn.ProcessElementFn()) + if fn, ok := n.Fn.OnTimerFn(); ok { + n.onTimerInvoker = newInvoker(fn) + } n.states = metrics.NewPTransformState(n.PID) @@ -236,6 +240,9 @@ func (n *ParDo) FinishBundle(_ context.Context) error { } n.status = Up n.inv.Reset() + if n.onTimerInvoker != nil { + n.onTimerInvoker.Reset() + } n.states.Set(n.ctx, metrics.FinishBundle) @@ -374,7 +381,7 @@ func (n *ParDo) InvokeTimerFn(ctx context.Context, fn *funcx.Fn, timerFamilyID s extra = append(extra, tmap.Tag) } extra = append(extra, n.cache.extra...) - val, err := InvokeWithOpts(ctx, fn, tmap.Pane, tmap.Windows, tmap.HoldTimestamp, InvokeOpts{ + val, err := n.onTimerInvoker.invokeWithOpts(ctx, tmap.Pane, tmap.Windows, tmap.HoldTimestamp, InvokeOpts{ opt: &MainInput{Key: *tmap.Key}, bf: n.bf, we: n.we, diff --git a/sdks/go/pkg/beam/core/timers/timers.go b/sdks/go/pkg/beam/core/timers/timers.go index 7e377db584b9..860ea768db3b 100644 --- a/sdks/go/pkg/beam/core/timers/timers.go +++ b/sdks/go/pkg/beam/core/timers/timers.go @@ -28,17 +28,17 @@ var ( ProviderType = reflect.TypeOf((*Provider)(nil)).Elem() ) -// TimeDomainEnum represents different time domains to set timer. -type TimeDomainEnum int32 +// TimeDomain represents different time domains to set timer. +type TimeDomain int32 const ( // TimeDomainUnspecified represents unspecified time domain. - TimeDomainUnspecified TimeDomainEnum = 0 + TimeDomainUnspecified TimeDomain = 0 // TimeDomainEventTime is time from the perspective of the data - TimeDomainEventTime TimeDomainEnum = 1 + TimeDomainEventTime TimeDomain = 1 // TimeDomainProcessingTime is time from the perspective of the // execution of your pipeline - TimeDomainProcessingTime TimeDomainEnum = 2 + TimeDomainProcessingTime TimeDomain = 2 ) // TimerMap holds timer information obtained from the pipeline. @@ -78,7 +78,7 @@ type Provider interface { // PipelineTimer interface represents valid timer type. type PipelineTimer interface { TimerFamily() string - TimerDomain() TimeDomainEnum + TimerDomain() TimeDomain } // EventTime represents the event time timer. @@ -92,7 +92,7 @@ func (et EventTime) TimerFamily() string { } // TimerDomain returns the time domain of timer. -func (et EventTime) TimerDomain() TimeDomainEnum { +func (et EventTime) TimerDomain() TimeDomain { return TimeDomainEventTime } @@ -126,7 +126,7 @@ func (pt ProcessingTime) TimerFamily() string { } // TimerDomain returns the time domain of timer. -func (pt ProcessingTime) TimerDomain() TimeDomainEnum { +func (pt ProcessingTime) TimerDomain() TimeDomain { return TimeDomainProcessingTime } diff --git a/sdks/go/pkg/beam/pardo.go b/sdks/go/pkg/beam/pardo.go index e69ce5a08e82..5cc9026bcb2c 100644 --- a/sdks/go/pkg/beam/pardo.go +++ b/sdks/go/pkg/beam/pardo.go @@ -120,7 +120,7 @@ func TryParDo(s Scope, dofn any, col PCollection, opts ...Option) ([]PCollection pipelineTimers := fn.PipelineTimers() if len(pipelineTimers) > 0 { // check if input is KV type - if !typex.IsKV(col.Type()) || !typex.IsCoGBK(col.Type()) { + if !typex.IsKV(col.Type()) && !typex.IsCoGBK(col.Type()) { return nil, addParDoCtx(errors.New("DoFn input should be keyed to be used with timers."), s) } c, err := inferCoder(typex.New(reflect.TypeOf(col.Type()))) From 06bad7516c97f96510cea956dd9ecb6a46bc4eeb Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Wed, 26 Apr 2023 13:14:10 -0400 Subject: [PATCH 20/22] update PipelineTimer interface, minor refactor, doc comment for example --- sdks/go/examples/timer_wordcap/wordcap.go | 12 +++--- sdks/go/pkg/beam/core/graph/fn.go | 17 +++++---- .../pkg/beam/core/runtime/graphx/translate.go | 16 ++++---- sdks/go/pkg/beam/core/timers/timers.go | 37 +++++++------------ sdks/go/pkg/beam/pardo.go | 10 ++--- 5 files changed, 40 insertions(+), 52 deletions(-) diff --git a/sdks/go/examples/timer_wordcap/wordcap.go b/sdks/go/examples/timer_wordcap/wordcap.go index 96a6d2439656..ac2eb877e170 100644 --- a/sdks/go/examples/timer_wordcap/wordcap.go +++ b/sdks/go/examples/timer_wordcap/wordcap.go @@ -13,13 +13,11 @@ // See the License for the specific language governing permissions and // limitations under the License. -// timer_wordcap is a toy streaming pipeline that uses State and Timers with PubSub. It -// does the following: -// -// (1) create a topic and publish a few messages to it -// (2) Set user state and timer -// -// NOTE: it only runs on Dataflow and must be manually cancelled. +// timer_wordcap is a toy streaming pipeline that demonstrates the use of State and Timers. +// Periodic Impulse is used as a streaming source that produces sequence of elements upto 5 minutes +// from the start of the pipeline every 5 seconds. These elements are keyed and fed to the Stateful DoFn +// where state and timers are set and cleared. Since this pipeline uses a Periodic Impulse, +// the pipeline is terminated automatically after it is done producing elements for 5 minutes. package main import ( diff --git a/sdks/go/pkg/beam/core/graph/fn.go b/sdks/go/pkg/beam/core/graph/fn.go index 63e35e6eec20..5c2a96474708 100644 --- a/sdks/go/pkg/beam/core/graph/fn.go +++ b/sdks/go/pkg/beam/core/graph/fn.go @@ -1393,14 +1393,14 @@ func validateOnTimerFn(fn *DoFn) error { if _, ok := fn.methods[onTimerName].TimerProvider(); !ok { err := errors.Errorf("OnTimer function doesn't use a TimerProvider, but Timer field is attached to the DoFn(%v): %v", fn.Name(), fn.PipelineTimers()) - return errors.SetTopLevelMsgf(err, "OnTimer function doesn't use a TimerProvider, but Timer field is attached to the DoFn: %v"+ + return errors.SetTopLevelMsgf(err, "OnTimer function doesn't use a TimerProvider, but Timer field is attached to the DoFn(%v): %v"+ ", Ensure that you are using the TimerProvider to set and clear the timers.", fn.Name(), fn.PipelineTimers()) } _, otNum, otExists := fn.methods[onTimerName].Emits() _, peNum, peExists := fn.methods[processElementName].Emits() - if otExists && peExists { + if otExists == peExists { if otNum != peNum { return fmt.Errorf("OnTimer and ProcessElement functions for DoFn should have exactly same emitters, no. of emitters used in OnTimer: %v, no. of emitters used in ProcessElement: %v", otNum, peNum) } @@ -1427,12 +1427,13 @@ func validateTimer(fn *DoFn, numIn mainInputs) error { } timerKeys := make(map[string]timers.PipelineTimer) for _, t := range pt { - k := t.TimerFamily() - if timer, ok := timerKeys[k]; ok { - err := errors.Errorf("Duplicate timer key %v", k) - return errors.SetTopLevelMsgf(err, "Duplicate timer family ID %v used by %v and %v. Ensure that timer family IDs are unique per DoFn", k, timer, t) - } else { - timerKeys[k] = t + for timerFamilyID := range t.Timers() { + if timer, ok := timerKeys[timerFamilyID]; ok { + err := errors.Errorf("Duplicate timer key %v", timerFamilyID) + return errors.SetTopLevelMsgf(err, "Duplicate timer family ID %v used by %v and %v. Ensure that timer family IDs are unique per DoFn", timerFamilyID, timer, t) + } else { + timerKeys[timerFamilyID] = t + } } } if err := validateOnTimerFn(fn); err != nil { diff --git a/sdks/go/pkg/beam/core/runtime/graphx/translate.go b/sdks/go/pkg/beam/core/runtime/graphx/translate.go index a427f22f8824..b77dbb82609e 100644 --- a/sdks/go/pkg/beam/core/runtime/graphx/translate.go +++ b/sdks/go/pkg/beam/core/runtime/graphx/translate.go @@ -582,13 +582,15 @@ func (m *marshaller) addMultiEdge(edge NamedEdge) ([]string, error) { m.requirements[URNRequiresStatefulProcessing] = true timerSpecs := make(map[string]*pipepb.TimerFamilySpec) for _, pt := range edge.Edge.DoFn.PipelineTimers() { - coderID, err := m.coders.Add(edge.Edge.TimerCoders[pt.TimerFamily()]) - if err != nil { - return handleErr(err) - } - timerSpecs[pt.TimerFamily()] = &pipepb.TimerFamilySpec{ - TimeDomain: pipepb.TimeDomain_Enum(pt.TimerDomain()), - TimerFamilyCoderId: coderID, + for timerFamilyID, timeDomain := range pt.Timers() { + coderID, err := m.coders.Add(edge.Edge.TimerCoders[timerFamilyID]) + if err != nil { + return handleErr(err) + } + timerSpecs[timerFamilyID] = &pipepb.TimerFamilySpec{ + TimeDomain: pipepb.TimeDomain_Enum(timeDomain), + TimerFamilyCoderId: coderID, + } } } payload.TimerFamilySpecs = timerSpecs diff --git a/sdks/go/pkg/beam/core/timers/timers.go b/sdks/go/pkg/beam/core/timers/timers.go index 860ea768db3b..ebffa215efa4 100644 --- a/sdks/go/pkg/beam/core/timers/timers.go +++ b/sdks/go/pkg/beam/core/timers/timers.go @@ -32,13 +32,13 @@ var ( type TimeDomain int32 const ( - // TimeDomainUnspecified represents unspecified time domain. - TimeDomainUnspecified TimeDomain = 0 - // TimeDomainEventTime is time from the perspective of the data - TimeDomainEventTime TimeDomain = 1 - // TimeDomainProcessingTime is time from the perspective of the + // UnspecifiedTimeDomain represents unspecified time domain. + UnspecifiedTimeDomain TimeDomain = 0 + // EventTimeDomain is time from the perspective of the data + EventTimeDomain TimeDomain = 1 + // ProcessingTimeDomain is time from the perspective of the // execution of your pipeline - TimeDomainProcessingTime TimeDomain = 2 + ProcessingTimeDomain TimeDomain = 2 ) // TimerMap holds timer information obtained from the pipeline. @@ -77,8 +77,7 @@ type Provider interface { // PipelineTimer interface represents valid timer type. type PipelineTimer interface { - TimerFamily() string - TimerDomain() TimeDomain + Timers() map[string]TimeDomain } // EventTime represents the event time timer. @@ -86,14 +85,9 @@ type EventTime struct { Family string } -// TimerFamily returns the name of timer family. -func (et EventTime) TimerFamily() string { - return et.Family -} - -// TimerDomain returns the time domain of timer. -func (et EventTime) TimerDomain() TimeDomain { - return TimeDomainEventTime +// Timers returns mapping of timer family ID and its time domain. +func (et EventTime) Timers() map[string]TimeDomain { + return map[string]TimeDomain{et.Family: EventTimeDomain} } // Set sets the timer for a event-time timestamp. Calling this method repeatedly for the same key @@ -120,14 +114,9 @@ type ProcessingTime struct { Family string } -// TimerFamily returns the name of timer family. -func (pt ProcessingTime) TimerFamily() string { - return pt.Family -} - -// TimerDomain returns the time domain of timer. -func (pt ProcessingTime) TimerDomain() TimeDomain { - return TimeDomainProcessingTime +// Timers returns mapping of timer family ID and its time domain. +func (pt ProcessingTime) Timers() map[string]TimeDomain { + return map[string]TimeDomain{pt.Family: ProcessingTimeDomain} } // Set sets the timer for processing time domain. Calling this method repeatedly for the same key diff --git a/sdks/go/pkg/beam/pardo.go b/sdks/go/pkg/beam/pardo.go index 5cc9026bcb2c..617de1a0bdd5 100644 --- a/sdks/go/pkg/beam/pardo.go +++ b/sdks/go/pkg/beam/pardo.go @@ -119,18 +119,16 @@ func TryParDo(s Scope, dofn any, col PCollection, opts ...Option) ([]PCollection wc := inWfn.Coder() pipelineTimers := fn.PipelineTimers() if len(pipelineTimers) > 0 { - // check if input is KV type - if !typex.IsKV(col.Type()) && !typex.IsCoGBK(col.Type()) { - return nil, addParDoCtx(errors.New("DoFn input should be keyed to be used with timers."), s) - } c, err := inferCoder(typex.New(reflect.TypeOf(col.Type()))) if err != nil { return nil, addParDoCtx(errors.New("error infering coder from col"), s) } edge.TimerCoders = make(map[string]*coder.Coder) + tc := coder.NewT(c, wc) for _, pt := range pipelineTimers { - tc := coder.NewT(c, wc) - edge.TimerCoders[pt.TimerFamily()] = tc + for timerFamilyID := range pt.Timers() { + edge.TimerCoders[timerFamilyID] = tc + } } } From 1c9e57b978ae1e323e45a3774d4cfae48f28244e Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Wed, 26 Apr 2023 13:44:05 -0400 Subject: [PATCH 21/22] add warn message --- sdks/go/pkg/beam/core/runtime/exec/datasource.go | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/sdks/go/pkg/beam/core/runtime/exec/datasource.go b/sdks/go/pkg/beam/core/runtime/exec/datasource.go index 7c1796008ee5..2f9e12a0fc82 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/datasource.go +++ b/sdks/go/pkg/beam/core/runtime/exec/datasource.go @@ -199,7 +199,7 @@ func (n *DataSource) Process(ctx context.Context) ([]*Checkpoint, error) { var cp ElementDecoder // Decoder for the primary element or the key in CoGBKs. var cvs []ElementDecoder // Decoders for each value stream in CoGBKs. - var dc ElementDecoder // Decoder for timer data + var dc ElementDecoder // Decoder for timer data. switch { case coder.IsCoGBK(c): @@ -212,6 +212,7 @@ func (n *DataSource) Process(ctx context.Context) ([]*Checkpoint, error) { cp = MakeElementDecoder(c) } + // for timer decoder if len(c.Components) > 0 { dc = MakeElementDecoder(c.Components[0]) } @@ -275,6 +276,9 @@ func (n *DataSource) Process(ctx context.Context) ([]*Checkpoint, error) { if fn, ok := n.OnTimerTransforms[ptransformID].Fn.OnTimerFn(); ok { _, err := n.OnTimerTransforms[ptransformID].InvokeTimerFn(ctx, fn, timerFamilyID, tmap) if err != nil { + log.Warnf(ctx, "expected transform %v to have an OnTimer method attached to handle"+ + "Timer Family ID: %v callback, but it did not. Please file an issue with Apache Beam"+ + "if you have defined OnTimer method with reproducible code at https://github.com/apache/beam/issues", ptransformID, timerFamilyID) return errors.WithContext(err, "ontimer callback invocation failed") } } From 161f5a9971e40268d8a5ab014fd90d2c2c871905 Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Thu, 27 Apr 2023 09:14:11 -0400 Subject: [PATCH 22/22] single edge timer coder, rm kv coder check, cache encoder,decoder --- sdks/go/examples/timer_wordcap/wordcap.go | 3 +- sdks/go/pkg/beam/core/graph/edge.go | 2 +- sdks/go/pkg/beam/core/graph/fn.go | 25 +++++++----- sdks/go/pkg/beam/core/runtime/exec/coder.go | 12 ------ .../pkg/beam/core/runtime/exec/datasource.go | 13 +------ sdks/go/pkg/beam/core/runtime/exec/pardo.go | 12 +++++- sdks/go/pkg/beam/core/runtime/exec/timers.go | 39 +++++++++++-------- .../pkg/beam/core/runtime/exec/timers_test.go | 2 +- .../pkg/beam/core/runtime/exec/translate.go | 10 +---- .../pkg/beam/core/runtime/graphx/translate.go | 5 ++- sdks/go/pkg/beam/pardo.go | 9 +---- 11 files changed, 60 insertions(+), 72 deletions(-) diff --git a/sdks/go/examples/timer_wordcap/wordcap.go b/sdks/go/examples/timer_wordcap/wordcap.go index ac2eb877e170..4d1faac50396 100644 --- a/sdks/go/examples/timer_wordcap/wordcap.go +++ b/sdks/go/examples/timer_wordcap/wordcap.go @@ -66,9 +66,10 @@ func (s *Stateful) OnTimer(ctx context.Context, ts beam.EventTime, tp timers.Pro switch timerKey { case "outputState": log.Infof(ctx, "Timer outputState fired on stateful for element: %v.", key) - s.OutputState.Clear(tp) + s.OutputState.Set(tp, ts.ToTime().Add(5*time.Second), timers.WithTag("1")) switch timerTag { case "1": + s.OutputState.Clear(tp) log.Infof(ctx, "Timer with tag 1 fired on outputState stateful DoFn.") emit(timerKey, timerTag) } diff --git a/sdks/go/pkg/beam/core/graph/edge.go b/sdks/go/pkg/beam/core/graph/edge.go index 86891114dd0e..a656cc9fe61e 100644 --- a/sdks/go/pkg/beam/core/graph/edge.go +++ b/sdks/go/pkg/beam/core/graph/edge.go @@ -156,7 +156,7 @@ type MultiEdge struct { DoFn *DoFn // ParDo RestrictionCoder *coder.Coder // SplittableParDo StateCoders map[string]*coder.Coder // Stateful ParDo - TimerCoders map[string]*coder.Coder // Stateful ParDo + TimerCoders *coder.Coder // Stateful ParDo CombineFn *CombineFn // Combine AccumCoder *coder.Coder // Combine Value []byte // Impulse diff --git a/sdks/go/pkg/beam/core/graph/fn.go b/sdks/go/pkg/beam/core/graph/fn.go index 5c2a96474708..630984272382 100644 --- a/sdks/go/pkg/beam/core/graph/fn.go +++ b/sdks/go/pkg/beam/core/graph/fn.go @@ -316,10 +316,12 @@ func (f *DoFn) OnTimerFn() (*funcx.Fn, bool) { } // PipelineTimers returns the list of PipelineTimer objects defined for the DoFn. -func (f *DoFn) PipelineTimers() []timers.PipelineTimer { +func (f *DoFn) PipelineTimers() ([]timers.PipelineTimer, []string) { var t []timers.PipelineTimer + var fieldNames []string + if f.Recv == nil { - return t + return t, fieldNames } v := reflect.Indirect(reflect.ValueOf(f.Recv)) @@ -328,10 +330,11 @@ func (f *DoFn) PipelineTimers() []timers.PipelineTimer { if f.CanInterface() { if pt, ok := f.Interface().(timers.PipelineTimer); ok { t = append(t, pt) + fieldNames = append(fieldNames, v.Type().Field(i).Name) } } } - return t + return t, fieldNames } // SplittableDoFn represents a DoFn implementing SDF methods. @@ -1391,10 +1394,12 @@ func validateOnTimerFn(fn *DoFn) error { return errors.SetTopLevelMsgf(err, "OnTimer function not defined for DoFn: %v. Ensure that OnTimer function is implemented for the DoFn.", fn.Name()) } + pipelineTimers, _ := fn.PipelineTimers() + if _, ok := fn.methods[onTimerName].TimerProvider(); !ok { - err := errors.Errorf("OnTimer function doesn't use a TimerProvider, but Timer field is attached to the DoFn(%v): %v", fn.Name(), fn.PipelineTimers()) + err := errors.Errorf("OnTimer function doesn't use a TimerProvider, but Timer field is attached to the DoFn(%v): %v", fn.Name(), pipelineTimers) return errors.SetTopLevelMsgf(err, "OnTimer function doesn't use a TimerProvider, but Timer field is attached to the DoFn(%v): %v"+ - ", Ensure that you are using the TimerProvider to set and clear the timers.", fn.Name(), fn.PipelineTimers()) + ", Ensure that you are using the TimerProvider to set and clear the timers.", fn.Name(), pipelineTimers) } _, otNum, otExists := fn.methods[onTimerName].Emits() @@ -1412,7 +1417,7 @@ func validateOnTimerFn(fn *DoFn) error { } func validateTimer(fn *DoFn, numIn mainInputs) error { - pt := fn.PipelineTimers() + pt, fieldNames := fn.PipelineTimers() if _, ok := fn.methods[processElementName].TimerProvider(); ok { if numIn == MainSingle { @@ -1425,14 +1430,14 @@ func validateTimer(fn *DoFn, numIn mainInputs) error { return errors.SetTopLevelMsgf(err, "ProcessElement uses a TimerProvider, but no timer fields are defined in the DoFn"+ ", Ensure that your DoFn exports the Timer fields used to set and clear timers.") } - timerKeys := make(map[string]timers.PipelineTimer) - for _, t := range pt { + timerKeys := make(map[string]string) + for i, t := range pt { for timerFamilyID := range t.Timers() { if timer, ok := timerKeys[timerFamilyID]; ok { err := errors.Errorf("Duplicate timer key %v", timerFamilyID) - return errors.SetTopLevelMsgf(err, "Duplicate timer family ID %v used by %v and %v. Ensure that timer family IDs are unique per DoFn", timerFamilyID, timer, t) + return errors.SetTopLevelMsgf(err, "Duplicate timer family ID %v used by struct fields %v and %v. Ensure that timer family IDs are unique per DoFn", timerFamilyID, timer, fieldNames[i]) } else { - timerKeys[timerFamilyID] = t + timerKeys[timerFamilyID] = fieldNames[i] } } } diff --git a/sdks/go/pkg/beam/core/runtime/exec/coder.go b/sdks/go/pkg/beam/core/runtime/exec/coder.go index d9e189514925..f2a9ff728e38 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/coder.go +++ b/sdks/go/pkg/beam/core/runtime/exec/coder.go @@ -146,12 +146,6 @@ func MakeElementEncoder(c *coder.Coder) ElementEncoder { case coder.Timer: tc := coder.SkipW(c).Components[0] - if coder.IsKV(tc) { - return &timerEncoder{ - elm: MakeElementEncoder(tc.Components[0]), - win: MakeWindowEncoder(c.Window), - } - } return &timerEncoder{ elm: MakeElementEncoder(tc), win: MakeWindowEncoder(c.Window), @@ -274,12 +268,6 @@ func MakeElementDecoder(c *coder.Coder) ElementDecoder { case coder.Timer: tc := coder.SkipW(c).Components[0] - if coder.IsKV(tc) { - return &timerDecoder{ - elm: MakeElementDecoder(tc.Components[0]), - win: MakeWindowDecoder(c.Window), - } - } return &timerDecoder{ elm: MakeElementDecoder(tc), win: MakeWindowDecoder(c.Window), diff --git a/sdks/go/pkg/beam/core/runtime/exec/datasource.go b/sdks/go/pkg/beam/core/runtime/exec/datasource.go index 2f9e12a0fc82..40c88134a754 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/datasource.go +++ b/sdks/go/pkg/beam/core/runtime/exec/datasource.go @@ -199,7 +199,6 @@ func (n *DataSource) Process(ctx context.Context) ([]*Checkpoint, error) { var cp ElementDecoder // Decoder for the primary element or the key in CoGBKs. var cvs []ElementDecoder // Decoders for each value stream in CoGBKs. - var dc ElementDecoder // Decoder for timer data. switch { case coder.IsCoGBK(c): @@ -212,11 +211,6 @@ func (n *DataSource) Process(ctx context.Context) ([]*Checkpoint, error) { cp = MakeElementDecoder(c) } - // for timer decoder - if len(c.Components) > 0 { - dc = MakeElementDecoder(c.Components[0]) - } - var checkpoints []*Checkpoint err := n.process(ctx, func(bcr *byteCountReader, ptransformID string) error { for { @@ -269,12 +263,9 @@ func (n *DataSource) Process(ctx context.Context) ([]*Checkpoint, error) { } }, func(bcr *byteCountReader, ptransformID, timerFamilyID string) error { - tmap, err := decodeTimer(dc, wc, bcr) - if err != nil { - return errors.WithContext(err, "error decoding timer in datasource") - } + if fn, ok := n.OnTimerTransforms[ptransformID].Fn.OnTimerFn(); ok { - _, err := n.OnTimerTransforms[ptransformID].InvokeTimerFn(ctx, fn, timerFamilyID, tmap) + _, err := n.OnTimerTransforms[ptransformID].InvokeTimerFn(ctx, fn, timerFamilyID, bcr) if err != nil { log.Warnf(ctx, "expected transform %v to have an OnTimer method attached to handle"+ "Timer Family ID: %v callback, but it did not. Please file an issue with Apache Beam"+ diff --git a/sdks/go/pkg/beam/core/runtime/exec/pardo.go b/sdks/go/pkg/beam/core/runtime/exec/pardo.go index c76dcccbee6e..652a904d16f5 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/pardo.go +++ b/sdks/go/pkg/beam/core/runtime/exec/pardo.go @@ -362,9 +362,17 @@ func (n *ParDo) invokeDataFn(ctx context.Context, pn typex.PaneInfo, ws []typex. return val, nil } -func (n *ParDo) InvokeTimerFn(ctx context.Context, fn *funcx.Fn, timerFamilyID string, tmap TimerRecv) (*FullValue, error) { +func (n *ParDo) InvokeTimerFn(ctx context.Context, fn *funcx.Fn, timerFamilyID string, bcr *byteCountReader) (*FullValue, error) { + timerAdapter, ok := n.Timer.(*userTimerAdapter) + if !ok { + return nil, fmt.Errorf("userTimerAdapter empty for ParDo: %v", n.GetPID()) + } + tmap, err := decodeTimer(timerAdapter.dc, timerAdapter.wc, bcr) + if err != nil { + return nil, errors.WithContext(err, "error decoding received timer callback") + } + // Defer side input clean-up in case of panic - var err error defer func() { if postErr := n.postInvoke(); postErr != nil { err = postErr diff --git a/sdks/go/pkg/beam/core/runtime/exec/timers.go b/sdks/go/pkg/beam/core/runtime/exec/timers.go index 7c5fc53ff8ea..0bc3e3dec35e 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/timers.go +++ b/sdks/go/pkg/beam/core/runtime/exec/timers.go @@ -32,31 +32,36 @@ type UserTimerAdapter interface { } type userTimerAdapter struct { - sID StreamID - timerIDToCoder map[string]*coder.Coder + sID StreamID + ec ElementEncoder + dc ElementDecoder + wc WindowDecoder } // NewUserTimerAdapter returns a user timer adapter for the given StreamID and timer coder. -func NewUserTimerAdapter(sID StreamID, c *coder.Coder, timerCoders map[string]*coder.Coder) UserTimerAdapter { +func NewUserTimerAdapter(sID StreamID, c *coder.Coder, timerCoder *coder.Coder) UserTimerAdapter { if !coder.IsW(c) { panic(fmt.Sprintf("expected WV coder for user timer %v: %v", sID, c)) } - - return &userTimerAdapter{sID: sID, timerIDToCoder: timerCoders} + ec := MakeElementEncoder(timerCoder) + dc := MakeElementDecoder(coder.SkipW(c).Components[0]) + wc := MakeWindowDecoder(c.Window) + return &userTimerAdapter{sID: sID, ec: ec, wc: wc, dc: dc} } // NewTimerProvider creates and returns a timer provider to set/clear timers. func (u *userTimerAdapter) NewTimerProvider(ctx context.Context, manager DataManager, inputTs typex.EventTime, w []typex.Window, element *MainInput) (timerProvider, error) { userKey := &FullValue{Elm: element.Key.Elm} tp := timerProvider{ - ctx: ctx, - tm: manager, - userKey: userKey, - inputTimestamp: inputTs, - sID: u.sID, - window: w, - writersByFamily: make(map[string]io.Writer), - codersByFamily: u.timerIDToCoder, + ctx: ctx, + tm: manager, + userKey: userKey, + inputTimestamp: inputTs, + sID: u.sID, + window: w, + writersByFamily: make(map[string]io.Writer), + timerElementEncoder: u.ec, + keyElementDecoder: u.dc, } return tp, nil @@ -72,8 +77,9 @@ type timerProvider struct { pn typex.PaneInfo - writersByFamily map[string]io.Writer - codersByFamily map[string]*coder.Coder + writersByFamily map[string]io.Writer + timerElementEncoder ElementEncoder + keyElementDecoder ElementDecoder } func (p *timerProvider) getWriter(family string) (io.Writer, error) { @@ -106,8 +112,7 @@ func (p *timerProvider) Set(t timers.TimerMap) { Pane: p.pn, } fv := FullValue{Elm: tm} - enc := MakeElementEncoder(p.codersByFamily[t.Family]) - if err := enc.Encode(&fv, w); err != nil { + if err := p.timerElementEncoder.Encode(&fv, w); err != nil { panic(err) } } diff --git a/sdks/go/pkg/beam/core/runtime/exec/timers_test.go b/sdks/go/pkg/beam/core/runtime/exec/timers_test.go index 5a24ce11a60d..2b3a33de792b 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/timers_test.go +++ b/sdks/go/pkg/beam/core/runtime/exec/timers_test.go @@ -29,7 +29,7 @@ func equalTimers(a, b TimerRecv) bool { } func TestTimerEncodingDecoding(t *testing.T) { - tc := coder.NewT(coder.NewKV([]*coder.Coder{coder.NewString(), coder.NewString()}), window.NewGlobalWindows().Coder()) + tc := coder.NewT(coder.NewString(), window.NewGlobalWindows().Coder()) ec := MakeElementEncoder(coder.SkipW(tc)) dec := MakeElementDecoder(coder.SkipW(tc)) diff --git a/sdks/go/pkg/beam/core/runtime/exec/translate.go b/sdks/go/pkg/beam/core/runtime/exec/translate.go index 62b5f25528ed..a8d1fdb7deee 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/translate.go +++ b/sdks/go/pkg/beam/core/runtime/exec/translate.go @@ -16,7 +16,6 @@ package exec import ( - "context" "fmt" "math/rand" "strconv" @@ -31,7 +30,6 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/core/typex" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/util/protox" "github.com/apache/beam/sdks/v2/go/pkg/beam/internal/errors" - "github.com/apache/beam/sdks/v2/go/pkg/beam/log" fnpb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/fnexecution_v1" pipepb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/pipeline_v1" "github.com/golang/protobuf/proto" @@ -593,17 +591,13 @@ func (b *builder) makeLink(from string, id linkID) (Node, error) { } if len(userTimers) > 0 { - log.Debugf(context.TODO(), "userTimers %+v", userTimers) - timerIDToCoder := make(map[string]*coder.Coder) sID := StreamID{Port: Port{URL: b.desc.GetTimerApiServiceDescriptor().GetUrl()}, PtransformID: id.to} ec, wc, err := b.makeCoderForPCollection(input[0]) if err != nil { return nil, err } - for key := range userTimers { - timerIDToCoder[key] = coder.NewT(ec, wc) - } - n.Timer = NewUserTimerAdapter(sID, coder.NewW(ec, wc), timerIDToCoder) + timerCoder := coder.NewT(ec.Components[0], wc) + n.Timer = NewUserTimerAdapter(sID, coder.NewW(ec, wc), timerCoder) } for i := 1; i < len(input); i++ { diff --git a/sdks/go/pkg/beam/core/runtime/graphx/translate.go b/sdks/go/pkg/beam/core/runtime/graphx/translate.go index b77dbb82609e..6a5cb313d17e 100644 --- a/sdks/go/pkg/beam/core/runtime/graphx/translate.go +++ b/sdks/go/pkg/beam/core/runtime/graphx/translate.go @@ -581,9 +581,10 @@ func (m *marshaller) addMultiEdge(edge NamedEdge) ([]string, error) { if _, ok := edge.Edge.DoFn.ProcessElementFn().TimerProvider(); ok { m.requirements[URNRequiresStatefulProcessing] = true timerSpecs := make(map[string]*pipepb.TimerFamilySpec) - for _, pt := range edge.Edge.DoFn.PipelineTimers() { + pipelineTimers, _ := edge.Edge.DoFn.PipelineTimers() + for _, pt := range pipelineTimers { for timerFamilyID, timeDomain := range pt.Timers() { - coderID, err := m.coders.Add(edge.Edge.TimerCoders[timerFamilyID]) + coderID, err := m.coders.Add(edge.Edge.TimerCoders) if err != nil { return handleErr(err) } diff --git a/sdks/go/pkg/beam/pardo.go b/sdks/go/pkg/beam/pardo.go index 617de1a0bdd5..61091a67a29c 100644 --- a/sdks/go/pkg/beam/pardo.go +++ b/sdks/go/pkg/beam/pardo.go @@ -117,19 +117,14 @@ func TryParDo(s Scope, dofn any, col PCollection, opts ...Option) ([]PCollection } wc := inWfn.Coder() - pipelineTimers := fn.PipelineTimers() + pipelineTimers, _ := fn.PipelineTimers() if len(pipelineTimers) > 0 { c, err := inferCoder(typex.New(reflect.TypeOf(col.Type()))) if err != nil { return nil, addParDoCtx(errors.New("error infering coder from col"), s) } - edge.TimerCoders = make(map[string]*coder.Coder) tc := coder.NewT(c, wc) - for _, pt := range pipelineTimers { - for timerFamilyID := range pt.Timers() { - edge.TimerCoders[timerFamilyID] = tc - } - } + edge.TimerCoders = tc } var ret []PCollection