From 785beb10e8452bd8f14e7aa657b6c6fd1db6d31f Mon Sep 17 00:00:00 2001 From: lostluck <13907733+lostluck@users.noreply.github.com> Date: Mon, 4 Mar 2024 15:45:52 -0800 Subject: [PATCH 01/21] [prism] Add basic processing time queue. --- .../prism/internal/engine/processingtime.go | 122 +++++++++++++++ .../internal/engine/processingtime_test.go | 139 ++++++++++++++++++ 2 files changed, 261 insertions(+) create mode 100644 sdks/go/pkg/beam/runners/prism/internal/engine/processingtime.go create mode 100644 sdks/go/pkg/beam/runners/prism/internal/engine/processingtime_test.go diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/processingtime.go b/sdks/go/pkg/beam/runners/prism/internal/engine/processingtime.go new file mode 100644 index 000000000000..227928054d70 --- /dev/null +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/processingtime.go @@ -0,0 +1,122 @@ +// 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 engine + +import ( + "container/heap" + + "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/mtime" +) + +// ProcessingTime events (processingTime timers, process continuations, triggers) necessarily need to operate on a global queue. +// However, PT timers are per key+family+tag, and may be overwritten by subsequent elements. +// So, similarly to event time timers, we need to manage a "last set" queue, and to manage the holds. +// This implies they should probably be handled by state, instead of globally. +// In reality, it's probably going to be "both", a global PT event queue, and per stage state. +// +// Also, in principle, timers would be how to implement the related features, so getting those right will simplify their handling. +// Test stream is already central, but doesn't set events, it controls their execution. +// But it would be easier to just *start* with ProcessContinuation handling, which may inform handling time. +// +// Currently, the ElementManager doesn't retain any data itself, nor really should it. +// So it should not hold material data about what is being triggered. +// It should really only contain which stage state should be triggered via watermark refresh. +// Watermark refreshes should also receive a current ProcessingTime so they can then trigger +// their own events. +// +// ProcessContinuations count as pending events, and must be drained accordingly before time expires. +// +// +// A stage may trigger on multiple ticks. +// It's up to a stage to schedule additional work on those notices. + +// mtimeHeap is a minHeap to find the earliest processing time event. +type mtimeHeap []mtime.Time + +func (h mtimeHeap) Len() int { return len(h) } +func (h mtimeHeap) Less(i, j int) bool { + return h[i] < h[j] +} +func (h mtimeHeap) Swap(i, j int) { h[i], h[j] = h[j], h[i] } + +func (h *mtimeHeap) Push(x any) { + // Push and Pop use pointer receivers because they modify the slice's length, + // not just its contents. + *h = append(*h, x.(mtime.Time)) +} + +func (h *mtimeHeap) Pop() any { + old := *h + n := len(old) + x := old[n-1] + *h = old[0 : n-1] + return x +} + +// ptQueue manages ProcessingTime events, in particular, which stages need notification +// at which points in processing time they occur. It doesn't handle the interface between +// walltime or any synthetic notions of time. +// +// ptQueue is not goroutine safe and relies on external concurrency for same. +type ptQueue struct { + events map[mtime.Time]set[string] + order mtimeHeap +} + +// newPtQueue creates an initialuzed ptQueue. +func newPtQueue() *ptQueue { + return &ptQueue{ + events: map[mtime.Time]set[string]{}, + } +} + +// Schedule a stage event at the given time. +func (q *ptQueue) Schedule(t mtime.Time, stageID string) { + if s, ok := q.events[t]; ok { + // We already have a trigger at this time, mutate that instead. + if s.present(stageID) { + // We already notify this stage at this time, no action required. + return + } + s.insert(stageID) + return + } + q.events[t] = set[string]{stageID: struct{}{}} + heap.Push(&q.order, t) +} + +// Peek returns the minimum time in the queue and whether it is valid. +// If there are no times left in the queue, the boolean will be false. +func (q *ptQueue) Peek() (mtime.Time, bool) { + if len(q.order) == 0 { + return mtime.MaxTimestamp, false + } + return q.order[0], true +} + +// AdvanceTo takes in the current now time, and returns the set of ids that need a refresh. +func (q *ptQueue) AdvanceTo(now mtime.Time) set[string] { + // If there are no elements, then we're done. + notify := set[string]{} + for { + if len(q.order) == 0 || q.order[0] > now { + return notify + } + // pop elements off the queue until the next time is later than now. + next := heap.Pop(&q.order).(mtime.Time) + notify.merge(q.events[next]) + delete(q.events, next) + } +} diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/processingtime_test.go b/sdks/go/pkg/beam/runners/prism/internal/engine/processingtime_test.go new file mode 100644 index 000000000000..64d29dac9237 --- /dev/null +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/processingtime_test.go @@ -0,0 +1,139 @@ +// 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 engine + +import ( + "testing" + + "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/mtime" + "github.com/google/go-cmp/cmp" +) + +func TestProcessingTimeQueue(t *testing.T) { + t.Run("empty", func(t *testing.T) { + q := newPtQueue() + emptyTime, ok := q.Peek() + if ok != false { + t.Errorf("q.Peek() on empty queue should have returned false") + } + if got, want := emptyTime, mtime.MaxTimestamp; got != want { + t.Errorf("q.Peek() on empty queue returned %v, want %v", got, want) + } + + tests := []mtime.Time{ + mtime.MinTimestamp, + -273, + 0, + 42, + mtime.EndOfGlobalWindowTime, + mtime.MaxTimestamp, + } + for _, test := range tests { + if got, want := q.AdvanceTo(test), (set[string]{}); len(got) > 0 { + t.Errorf("q.AdvanceTo(%v) on empty queue returned %v, want %v", test, got, want) + } + } + }) + t.Run("scheduled", func(t *testing.T) { + type event struct { + t mtime.Time + stage string + } + + s := func(ids ...string) set[string] { + ret := set[string]{} + for _, id := range ids { + ret.insert(id) + } + return ret + } + + tests := []struct { + name string + events []event + + minTime mtime.Time + + advanceTime mtime.Time + want set[string] + }{ + { + "singleBefore", + []event{{1, "test1"}}, + 1, + 0, + s(), + }, { + "singleAt", + []event{{1, "test1"}}, + 1, + 1, + s("test1"), + }, { + "singleAfter", + []event{{1, "test1"}}, + 1, + 2, + s("test1"), + }, { + "trioDistinct", + []event{{1, "test1"}, {2, "test2"}, {3, "test3"}}, + 1, + 2, + s("test1", "test2"), + }, { + "trioDistinctReversed", + []event{{3, "test3"}, {2, "test2"}, {1, "test1"}}, + 1, + 2, + s("test1", "test2"), + }, { + "trioDistinctTimeSameId", + []event{{3, "test"}, {2, "test"}, {1, "test"}}, + 1, + 2, + s("test"), + }, { + "trioOneTime", + []event{{1, "test3"}, {1, "test2"}, {1, "test1"}}, + 1, + 1, + s("test1", "test2", "test3"), + }, { + "trioDuplicates", + []event{{1, "test"}, {1, "test"}, {1, "test"}}, + 1, + 1, + s("test", "test", "test"), + }, + } + + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + q := newPtQueue() + for _, e := range test.events { + q.Schedule(e.t, e.stage) + } + if got, _ := q.Peek(); got != test.minTime { + t.Errorf("q.Peek() = %v, want %v", got, test.minTime) + } + + if got, want := q.AdvanceTo(test.advanceTime), test.want; !cmp.Equal(got, want) { + t.Errorf("q.AdvanceTo(%v) = %v, want %v", test.advanceTime, got, want) + } + }) + } + }) +} From 5d6b29d72ad815c0b5bbd7d419931c7b6ed656e7 Mon Sep 17 00:00:00 2001 From: lostluck <13907733+lostluck@users.noreply.github.com> Date: Mon, 4 Mar 2024 15:46:22 -0800 Subject: [PATCH 02/21] Initial residual handling refactor. --- .../prism/internal/engine/elementmanager.go | 32 +++++++++++++++++++ 1 file changed, 32 insertions(+) diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go b/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go index 5d665edf2862..71b846082544 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go @@ -169,6 +169,7 @@ type ElementManager struct { livePending atomic.Int64 // An accessible live pending count. DEBUG USE ONLY pendingElements sync.WaitGroup // pendingElements counts all unprocessed elements in a job. Jobs with no pending elements terminate successfully. + processTimeEvents *ptQueue // Manages sequence of stage updates when interfacing with processing time. testStreamHandler *testStreamHandler // Optional test stream handler when a test stream is in the pipeline. } @@ -611,6 +612,20 @@ func reElementResiduals(residuals []Residual, inputInfo PColInfo, rb RunBundle) return unprocessedElements } +// Residual represents the unprocessed portion of a single element. +type Residual struct { + Element []byte + Delay time.Duration // The relative time delay. + Bounded bool // Whether this element is finite or not. +} + +// Residuals is used to specify process continuations within a bundle. +type Residuals struct { + Data []Residual + TransformID, InputID string // We only allow one SDF at the root of a bundledescriptor so there should only be one each. + MinOutputWatermarks map[string]mtime.Time // Output watermarks (technically per Residual, but aggregated here until it makes a difference.) +} + // PersistBundle uses the tentative bundle output to update the watermarks for the stage. // Each stage has two monotonically increasing watermarks, the input watermark, and the output // watermark. @@ -1359,3 +1374,20 @@ func (ss *stageState) bundleReady(em *ElementManager) (mtime.Time, bool) { } return upstreamW, ready } + +// ProcessingTimeNow gives the current processing time for the runner. +func (em *ElementManager) ProcessingTimeNow() time.Time { + if em.testStreamHandler != nil { + return em.testStreamHandler.Now() + } + // "Test" mode -> advance to next processing time event if any, to allow execution. + + // "Production" mode, always real time now. + return time.Now() +} + +// rebaseProcessingTime turns an absolute processing time to be relative to the provided local clock now. +// Necessary to reasonably schedule ProcessingTime timers within a TestStream using pipeline. +func (em *ElementManager) rebaseProcessingTime(localNow, scheduled mtime.Time) mtime.Time { + return localNow + (scheduled - mtime.Now()) +} From fbf807d824f8b6f1adeab7994c01abc082f2e35d Mon Sep 17 00:00:00 2001 From: lostluck <13907733+lostluck@users.noreply.github.com> Date: Fri, 29 Mar 2024 15:31:13 -0700 Subject: [PATCH 03/21] Re-work teststream initilization. Remove pending element race. --- .../prism/internal/engine/elementmanager.go | 121 ++++++++++++++-- .../prism/internal/engine/engine_test.go | 42 ++++++ .../prism/internal/engine/processingtime.go | 65 +++++++-- .../internal/engine/processingtime_test.go | 4 +- .../prism/internal/engine/teststream.go | 10 +- .../beam/runners/prism/internal/execute.go | 3 + .../prism/internal/jobservices/management.go | 7 +- .../pkg/beam/runners/prism/internal/stage.go | 25 ++-- sdks/go/test/integration/primitives/timers.go | 136 +++++++++++++++++- .../integration/primitives/timers_test.go | 5 +- 10 files changed, 380 insertions(+), 38 deletions(-) diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go b/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go index 71b846082544..2170afa52a02 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go @@ -169,7 +169,7 @@ type ElementManager struct { livePending atomic.Int64 // An accessible live pending count. DEBUG USE ONLY pendingElements sync.WaitGroup // pendingElements counts all unprocessed elements in a job. Jobs with no pending elements terminate successfully. - processTimeEvents *ptQueue // Manages sequence of stage updates when interfacing with processing time. + processTimeEvents *stageRefreshQueue // Manages sequence of stage updates when interfacing with processing time. testStreamHandler *testStreamHandler // Optional test stream handler when a test stream is in the pipeline. } @@ -193,6 +193,7 @@ func NewElementManager(config Config) *ElementManager { watermarkRefreshes: set[string]{}, inprogressBundles: set[string]{}, refreshCond: sync.Cond{L: &sync.Mutex{}}, + processTimeEvents: newStageRefreshQueue(), } } @@ -228,6 +229,12 @@ func (em *ElementManager) StageStateful(ID string) { em.stages[ID].stateful = true } +// StageProcessingTimeTimers indicates which timers are processingTime domain timers. +func (em *ElementManager) StageProcessingTimeTimers(ID string, ptTimers map[string]bool) { + em.stages[ID].processingTimeTimers = ptTimers + fmt.Println("XXXX stage", ID, "with PT timers", ptTimers, "agg", em.stages[ID].aggregate, "stateful", em.stages[ID].stateful) +} + // AddTestStream provides a builder interface for the execution layer to build the test stream from // the protos. func (em *ElementManager) AddTestStream(id string, tagToPCol map[string]string) TestStreamBuilder { @@ -301,11 +308,15 @@ func (em *ElementManager) Bundles(ctx context.Context, nextBundID func() string) // If we have a test stream, clear out existing refreshes, so the test stream can // insert any elements it needs. if em.testStreamHandler != nil { + fmt.Println("XXXXX CLEARING OUT REFRESHSES", em.watermarkRefreshes) em.watermarkRefreshes = singleSet(em.testStreamHandler.ID) } for { em.refreshCond.L.Lock() + // Check if processing time has advanced before the wait loop. + // emNow := em.ProcessingTimeNow() + // em.watermarkRefreshes.merge(em.processTimeEvents.AdvanceTo(emNow)) // If there are no watermark refreshes available, we wait until there are. for len(em.watermarkRefreshes) == 0 { // Check to see if we must exit @@ -316,23 +327,30 @@ func (em *ElementManager) Bundles(ctx context.Context, nextBundID func() string) default: } em.refreshCond.Wait() // until watermarks may have changed. + + // Check if processing time has advanced while we waited, and add refreshes here. (TODO waking on real time here for prod mode) + // emNow := em.ProcessingTimeNow() + // em.watermarkRefreshes.merge(em.processTimeEvents.AdvanceTo(emNow)) } // We know there is some work we can do that may advance the watermarks, // refresh them, and see which stages have advanced. advanced := em.refreshWatermarks() + fmt.Println("XXXXX refreshedWatermarks", advanced) // Check each advanced stage, to see if it's able to execute based on the watermark. for stageID := range advanced { ss := em.stages[stageID] watermark, ready := ss.bundleReady(em) if ready { + // TODO Move handling ProcessingTime bundle fireing *HERE* to avoid nonsense against normal eventtime handling. bundleID, ok, reschedule := ss.startBundle(watermark, nextBundID) // Handle the reschedule even when there's no bundle. if reschedule { em.watermarkRefreshes.insert(stageID) } if !ok { + fmt.Println("XXXX Bundle Not Fired stage", stageID, "watermark", watermark, ss.pending, "aggregate", ss.aggregate) continue } rb := RunBundle{StageID: stageID, BundleID: bundleID, Watermark: watermark} @@ -346,6 +364,8 @@ func (em *ElementManager) Bundles(ctx context.Context, nextBundID func() string) case runStageCh <- rb: } em.refreshCond.L.Lock() + } else { + fmt.Println("XXXX BundleNotReady stage", stageID, "watermark", watermark, ss.pending, "aggregate", ss.aggregate) } } em.checkForQuiescence(advanced) @@ -385,6 +405,7 @@ func (em *ElementManager) checkForQuiescence(advanced set[string]) { // There are no further incoming watermark changes, see if there are test stream events for this job. nextEvent := em.testStreamHandler.NextEvent() if nextEvent != nil { + fmt.Println("XXXX TestStreamEvent!", nextEvent) nextEvent.Execute(em) // Decrement pending for the event being processed. em.addPending(-1) @@ -705,6 +726,7 @@ func (em *ElementManager) PersistBundle(rb RunBundle, col2Coders map[string]PCol // Add unprocessed back to the pending stack. if len(unprocessedElements) > 0 { + // TODO actually reschedule based on the residuals delay... count := stage.AddPending(unprocessedElements) em.addPending(count) } @@ -768,8 +790,10 @@ func (em *ElementManager) triageTimers(d TentativeData, inputInfo PColInfo, stag tag string win typex.Window } + emNow := em.ProcessingTimeNow() var pendingEventTimers []element + var pendingProcessingTimers map[mtime.Time][]element for tentativeKey, timers := range d.timers { keyToTimers := map[timerKey]element{} for _, t := range timers { @@ -787,7 +811,23 @@ func (em *ElementManager) triageTimers(d TentativeData, inputInfo PColInfo, stag elm.transform = tentativeKey.Transform elm.family = tentativeKey.Family - pendingEventTimers = append(pendingEventTimers, elm) + if stage.processingTimeTimers[elm.family] { + // Conditionally rebase processing time? Could be handled directly in rebaseProcessingTime... + newTimerFire := em.rebaseProcessingTime(emNow, elm.timestamp) + fmt.Println("XXXXXXX Processing Time Timer:", elm.family, "rebased from", elm.timestamp, "to", newTimerFire, "using", emNow, "fireable?", newTimerFire < emNow) + elm.timestamp = newTimerFire + // Add pending rRocessing timers to the stage's processing time store & schedule event in the manager. + em.processTimeEvents.Schedule(newTimerFire, stage.ID) + + if pendingProcessingTimers == nil { + pendingProcessingTimers = map[mtime.Time][]element{} + } + prev := pendingProcessingTimers[newTimerFire] + prev = append(prev, elm) + pendingProcessingTimers[newTimerFire] = prev + } else { + pendingEventTimers = append(pendingEventTimers, elm) + } } } @@ -795,6 +835,14 @@ func (em *ElementManager) triageTimers(d TentativeData, inputInfo PColInfo, stag count := stage.AddPending(pendingEventTimers) em.addPending(count) } + if len(pendingProcessingTimers) > 0 { + count := stage.AddProcessingTimePending(pendingProcessingTimers) + fmt.Println("XXXXXXX Processing Time Timers:", stage.ID, "num pending:", count, "processingTimeFamilies", stage.processingTimeTimers, pendingProcessingTimers) + em.addPending(count) + for t := range pendingProcessingTimers { + em.processTimeEvents.Schedule(t, stage.ID) + } + } } // FailBundle clears the extant data allowing the execution to shut down. @@ -845,6 +893,9 @@ func (em *ElementManager) refreshWatermarks() set[string] { // Need to have at least one refresh signal. nextUpdates := set[string]{} refreshed := set[string]{} + // Use a single consolidated processing time during a given refresh for consistency. + emNow := em.ProcessingTimeNow() + var i int for stageID := range em.watermarkRefreshes { // clear out old one. @@ -854,6 +905,8 @@ func (em *ElementManager) refreshWatermarks() set[string] { refreshes := ss.updateWatermarks(em) nextUpdates.merge(refreshes) + ss.AdvanceProcessingTimeTo(emNow) + // cap refreshes incrementally. if i < 10 { i++ @@ -898,9 +951,10 @@ type stageState struct { sides []LinkID // PCollection IDs of side inputs that can block execution. // Special handling bits - stateful bool // whether this stage uses state or timers, and needs keyed processing. - aggregate bool // whether this stage needs to block for aggregation. - strat winStrat // Windowing Strategy for aggregation fireings. + stateful bool // whether this stage uses state or timers, and needs keyed processing. + aggregate bool // whether this stage needs to block for aggregation. + strat winStrat // Windowing Strategy for aggregation fireings. + processingTimeTimers map[string]bool // Indicates which timer families use the processing time domain. mu sync.Mutex upstreamWatermarks sync.Map // watermark set from inputPCollection's parent. @@ -924,6 +978,8 @@ type stageState struct { // This avoids scanning the heap to remove or access a hold for each element. watermarkHolds *holdTracker inprogressHoldsByBundle map[string]map[mtime.Time]int // bundle to associated holds. + + processTimeEvents *processingTimeElementQueue } // timerKey uniquely identifies a given timer within the space of a user key. @@ -956,6 +1012,8 @@ func makeStageState(ID string, inputIDs, outputIDs []string, sides []LinkID) *st input: mtime.MinTimestamp, output: mtime.MinTimestamp, estimatedOutput: mtime.MinTimestamp, + + processTimeEvents: newPtQueue(), } // Initialize the upstream watermarks to minTime. @@ -992,6 +1050,13 @@ func (ss *stageState) AddPending(newPending []element) int { dnt.elements.Push(e) if e.IsTimer() { + if ss.processingTimeTimers[e.family] { + // override the firing timestamp with the current output watermark + // TODO sort out bypassing event time timer behavior for processing time timers. + e.timestamp = ss.output + // TODO adjust the count + fmt.Println("XXXXXX Pending Adjusted Processing Time", e) + } if lastSet, ok := dnt.timers[timerKey{family: e.family, tag: e.tag, window: e.window}]; ok { // existing timer! // don't increase the count this time, as "this" timer is already pending. @@ -1032,6 +1097,31 @@ func (ss *stageState) AddPendingSide(newPending []element, tID, inputID string) } } +// AddProcessingTimePending is to add elements and timers that rely on processing time to trigger. +func (ss *stageState) AddProcessingTimePending(newPending map[mtime.Time][]element) int { + ss.mu.Lock() + defer ss.mu.Unlock() + var count int + + // TODO sort out processing time watermark holds here. + // TODO sort out per key processing time uniqueness here (only "one" can be set for a given key+fam+window, just like event times.) + // - Only the last set one can work. + for t, event := range newPending { + count += len(event) + ss.processTimeEvents.Schedule(t, event) + } + return count +} + +func (ss *stageState) AdvanceProcessingTimeTo(now mtime.Time) { + ss.mu.Lock() + events := ss.processTimeEvents.AdvanceTo(now) + ss.mu.Unlock() + for _, es := range events { + ss.AddPending(es) + } +} + // GetSideData returns side input data for the provided transform+input pair, valid to the watermark. func (ss *stageState) GetSideData(tID, inputID string, watermark mtime.Time) map[typex.Window][][]byte { ss.mu.Lock() @@ -1059,6 +1149,7 @@ func (em *ElementManager) GetSideData(sID, tID, inputID string, watermark mtime. func (ss *stageState) updateUpstreamWatermark(pcol string, upstream mtime.Time) { // A stage will only have a single upstream watermark, so // we simply set this. + fmt.Println("XXXXXXX updateUpstreamWatermark for", ss.ID, "in", pcol, upstream) ss.upstreamWatermarks.Store(pcol, upstream) } @@ -1112,6 +1203,9 @@ func (ss *stageState) startBundle(watermark mtime.Time, genBundID func() string) var toProcess, notYet []element for _, e := range ss.pending { if !ss.aggregate || ss.aggregate && ss.strat.EarliestCompletion(e.window) < watermark { + if ss.aggregate { + fmt.Println("XXXXX SSSSSSSS aggregation stage", ss.ID, "earliest window", ss.strat.EarliestCompletion(e.window), "watermark", watermark) + } toProcess = append(toProcess, e) } else { notYet = append(notYet, e) @@ -1303,6 +1397,7 @@ func (ss *stageState) updateWatermarks(em *ElementManager) set[string] { } refreshes := set[string]{} // If bigger, advance the output watermark + fmt.Println("XXXX updating watermarks for", ss.ID, "update downstream?", newOut > ss.output, "input", ss.input, "output", ss.output, "newOut", newOut, "hold", minWatermarkHold) if newOut > ss.output { ss.output = newOut for _, outputCol := range ss.outputIDs { @@ -1350,7 +1445,7 @@ func (ss *stageState) bundleReady(em *ElementManager) (mtime.Time, bool) { inputW := ss.input _, upstreamW := ss.UpstreamWatermark() if inputW == upstreamW { - slog.Debug("bundleReady: insufficient upstream watermark", + slog.Info("bundleReady: insufficient upstream watermark", slog.String("stage", ss.ID), slog.Group("watermark", slog.Any("upstream", upstreamW), @@ -1372,18 +1467,28 @@ func (ss *stageState) bundleReady(em *ElementManager) (mtime.Time, bool) { ready = false } } + if ss.aggregate { + fmt.Println("XXXXXX SSSSSS aggregate stage", ss.ID, "bundleReady", "upstream", upstreamW, "input", inputW, ss.pending) + } return upstreamW, ready } // ProcessingTimeNow gives the current processing time for the runner. -func (em *ElementManager) ProcessingTimeNow() time.Time { +func (em *ElementManager) ProcessingTimeNow() (ret mtime.Time) { + // defer func() { + // fmt.Println("XXXX ProcessingTimeNow -> ", ret, em.processTimeEvents.order) + // }() if em.testStreamHandler != nil { return em.testStreamHandler.Now() } // "Test" mode -> advance to next processing time event if any, to allow execution. + // if test mode... + if t, ok := em.processTimeEvents.Peek(); ok { + return t + } // "Production" mode, always real time now. - return time.Now() + return mtime.Now() } // rebaseProcessingTime turns an absolute processing time to be relative to the provided local clock now. diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/engine_test.go b/sdks/go/pkg/beam/runners/prism/internal/engine/engine_test.go index 0c042d731d6a..eb6db41d2587 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/engine_test.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/engine_test.go @@ -117,6 +117,7 @@ func TestStatefulStages(t *testing.T) { {pipeline: primitives.SetStateParDoClear}, {pipeline: primitives.TimersEventTimeBounded}, {pipeline: primitives.TimersEventTimeUnbounded}, + {pipeline: primitives.TimersEventTimeTestStream}, } configs := []struct { @@ -221,3 +222,44 @@ func TestTestStream(t *testing.T) { } } } + +// TestProcessingTime is the suite for validating behaviors around ProcessingTime. +// Separate from the TestStream, Timers, and Triggers tests due to the unique nature +// of the time domain. +func FOOTestProcessingTime(t *testing.T) { + initRunner(t) + + tests := []struct { + pipeline func(s beam.Scope) + }{ + {pipeline: primitives.TimersProcessingTimeTestStream_Infinity}, + } + + configs := []struct { + name string + OneElementPerKey, OneKeyPerBundle bool + }{ + {"Greedy", false, false}, + // {"AllElementsPerKey", false, true}, + // {"OneElementPerKey", true, false}, + // {"OneElementPerBundle", true, true}, + } + for _, config := range configs { + for _, test := range tests { + t.Run(initTestName(test.pipeline)+"_"+config.name, func(t *testing.T) { + t.Cleanup(func() { + engine.OneElementPerKey = false + engine.OneKeyPerBundle = false + }) + engine.OneElementPerKey = config.OneElementPerKey + engine.OneKeyPerBundle = config.OneKeyPerBundle + p, s := beam.NewPipelineWithRoot() + test.pipeline(s) + _, err := executeWithT(context.Background(), t, p) + if err != nil { + t.Fatalf("pipeline failed, but feature should be implemented in Prism: %v", err) + } + }) + } + } +} diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/processingtime.go b/sdks/go/pkg/beam/runners/prism/internal/engine/processingtime.go index 227928054d70..4222c06b5adc 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/processingtime.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/processingtime.go @@ -16,6 +16,7 @@ package engine import ( "container/heap" + "fmt" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/mtime" ) @@ -65,25 +66,26 @@ func (h *mtimeHeap) Pop() any { return x } -// ptQueue manages ProcessingTime events, in particular, which stages need notification +// stageRefreshQueue manages ProcessingTime events, in particular, which stages need notification // at which points in processing time they occur. It doesn't handle the interface between // walltime or any synthetic notions of time. // -// ptQueue is not goroutine safe and relies on external concurrency for same. -type ptQueue struct { +// stageRefreshQueue is not goroutine safe and relies on external synchronization. +type stageRefreshQueue struct { events map[mtime.Time]set[string] order mtimeHeap } -// newPtQueue creates an initialuzed ptQueue. -func newPtQueue() *ptQueue { - return &ptQueue{ +// newStageRefreshQueue creates an initialized stageRefreshQueue. +func newStageRefreshQueue() *stageRefreshQueue { + return &stageRefreshQueue{ events: map[mtime.Time]set[string]{}, } } // Schedule a stage event at the given time. -func (q *ptQueue) Schedule(t mtime.Time, stageID string) { +func (q *stageRefreshQueue) Schedule(t mtime.Time, stageID string) { + fmt.Println("XXXX stageRefreshQueue scheduled! ", t, stageID) if s, ok := q.events[t]; ok { // We already have a trigger at this time, mutate that instead. if s.present(stageID) { @@ -99,7 +101,7 @@ func (q *ptQueue) Schedule(t mtime.Time, stageID string) { // Peek returns the minimum time in the queue and whether it is valid. // If there are no times left in the queue, the boolean will be false. -func (q *ptQueue) Peek() (mtime.Time, bool) { +func (q *stageRefreshQueue) Peek() (mtime.Time, bool) { if len(q.order) == 0 { return mtime.MaxTimestamp, false } @@ -107,11 +109,13 @@ func (q *ptQueue) Peek() (mtime.Time, bool) { } // AdvanceTo takes in the current now time, and returns the set of ids that need a refresh. -func (q *ptQueue) AdvanceTo(now mtime.Time) set[string] { +func (q *stageRefreshQueue) AdvanceTo(now mtime.Time) set[string] { // If there are no elements, then we're done. notify := set[string]{} for { if len(q.order) == 0 || q.order[0] > now { + t, ok := q.Peek() + fmt.Println("XXXX stageRefreshQueue advanced to ", now, "refreshed", notify, "next time?", ok, t, t-now) return notify } // pop elements off the queue until the next time is later than now. @@ -120,3 +124,46 @@ func (q *ptQueue) AdvanceTo(now mtime.Time) set[string] { delete(q.events, next) } } + +// newPtQueue creates an initialized processingTimeElementQueue. +func newPtQueue() *processingTimeElementQueue { + return &processingTimeElementQueue{ + events: map[mtime.Time][]element{}, + } +} + +// processingTimeElementQueue tracks the actual elements associated with a given processing time. +// +// processingTimeElementQueue is not goroutine safe and relies on external synchronization. +type processingTimeElementQueue struct { + events map[mtime.Time][]element + order mtimeHeap +} + +// Schedule elements for the given processing time. Takes ownership of the +// passed in slice of elements. +func (q *processingTimeElementQueue) Schedule(t mtime.Time, event []element) { + if s, ok := q.events[t]; ok { + // We already have a trigger at this time, mutate that instead. + s = append(s, event...) + q.events[t] = s + return + } + q.events[t] = event + heap.Push(&q.order, t) +} + +// AdvanceTo takes in the current now time, and returns the set of ids that need a refresh. +func (q *processingTimeElementQueue) AdvanceTo(now mtime.Time) [][]element { + // If there are no elements, then we're done. + var ret [][]element + for { + if len(q.order) == 0 || q.order[0] > now { + return ret + } + // pop elements off the queue until the next time is later than now. + next := heap.Pop(&q.order).(mtime.Time) + ret = append(ret, q.events[next]) + delete(q.events, next) + } +} diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/processingtime_test.go b/sdks/go/pkg/beam/runners/prism/internal/engine/processingtime_test.go index 64d29dac9237..0f0b67b46135 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/processingtime_test.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/processingtime_test.go @@ -23,7 +23,7 @@ import ( func TestProcessingTimeQueue(t *testing.T) { t.Run("empty", func(t *testing.T) { - q := newPtQueue() + q := newStageRefreshQueue() emptyTime, ok := q.Peek() if ok != false { t.Errorf("q.Peek() on empty queue should have returned false") @@ -122,7 +122,7 @@ func TestProcessingTimeQueue(t *testing.T) { for _, test := range tests { t.Run(test.name, func(t *testing.T) { - q := newPtQueue() + q := newStageRefreshQueue() for _, e := range test.events { q.Schedule(e.t, e.stage) } diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/teststream.go b/sdks/go/pkg/beam/runners/prism/internal/engine/teststream.go index 34b79d455ce1..49a24842ab04 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/teststream.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/teststream.go @@ -66,8 +66,8 @@ type tagState struct { // Now represents the overridden ProcessingTime, which is only advanced when directed by an event. // Overrides the elementManager "clock". -func (ts *testStreamHandler) Now() time.Time { - return ts.processingTime +func (ts *testStreamHandler) Now() mtime.Time { + return mtime.FromTime(ts.processingTime) } // TagsToPCollections recieves the map of local output tags to global pcollection ids. @@ -217,6 +217,8 @@ func (ev tsWatermarkEvent) Execute(em *ElementManager) { em.testStreamHandler.tagState[ev.Tag] = t // Update the upstream watermarks in the consumers. + + fmt.Println("XXXXX updating watermark to", t.watermark, "for stages", em.consumers[t.pcollection]) for _, sID := range em.consumers[t.pcollection] { ss := em.stages[sID] ss.updateUpstreamWatermark(ss.inputID, t.watermark) @@ -234,6 +236,10 @@ type tsProcessingTimeEvent struct { // Execute this ProcessingTime event by advancing the synthetic processing time. func (ev tsProcessingTimeEvent) Execute(em *ElementManager) { em.testStreamHandler.processingTime = em.testStreamHandler.processingTime.Add(ev.AdvanceBy) + + // // Add the refreshes now so our block prevention logic works. + // emNow := em.ProcessingTimeNow() + // em.watermarkRefreshes.merge(em.processTimeEvents.AdvanceTo(emNow)) } // tsFinalEvent is the "last" event we perform after all preceeding events. diff --git a/sdks/go/pkg/beam/runners/prism/internal/execute.go b/sdks/go/pkg/beam/runners/prism/internal/execute.go index 504125a2bd6e..707e61f723a0 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/execute.go +++ b/sdks/go/pkg/beam/runners/prism/internal/execute.go @@ -310,6 +310,9 @@ func executePipeline(ctx context.Context, wks map[string]*worker.W, j *jobservic if stage.stateful { em.StageStateful(stage.ID) } + if len(stage.processingTimeTimers) > 0 { + em.StageProcessingTimeTimers(stage.ID, stage.processingTimeTimers) + } default: err := fmt.Errorf("unknown environment[%v]", t.GetEnvironmentId()) slog.Error("Execute", err) diff --git a/sdks/go/pkg/beam/runners/prism/internal/jobservices/management.go b/sdks/go/pkg/beam/runners/prism/internal/jobservices/management.go index 4cff2ae92e7c..6eae311b08f5 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/jobservices/management.go +++ b/sdks/go/pkg/beam/runners/prism/internal/jobservices/management.go @@ -160,7 +160,7 @@ func (s *Server) Prepare(ctx context.Context, req *jobpb.PrepareJobRequest) (*jo } // Validate all the timer features for _, spec := range pardo.GetTimerFamilySpecs() { - check("TimerFamilySpecs.TimeDomain.Urn", spec.GetTimeDomain(), pipepb.TimeDomain_EVENT_TIME) + check("TimerFamilySpecs.TimeDomain.Urn", spec.GetTimeDomain(), pipepb.TimeDomain_EVENT_TIME, pipepb.TimeDomain_PROCESSING_TIME) } check("OnWindowExpirationTimerFamily", pardo.GetOnWindowExpirationTimerFamilySpec(), "") // Unsupported for now. @@ -176,11 +176,6 @@ func (s *Server) Prepare(ctx context.Context, req *jobpb.PrepareJobRequest) (*jo if err := proto.Unmarshal(t.GetSpec().GetPayload(), &testStream); err != nil { return nil, fmt.Errorf("unable to unmarshal TestStreamPayload for %v - %q: %w", tid, t.GetUniqueName(), err) } - for _, ev := range testStream.GetEvents() { - if ev.GetProcessingTimeEvent() != nil { - check("TestStream.Event - ProcessingTimeEvents unsupported.", ev.GetProcessingTimeEvent()) - } - } t.EnvironmentId = "" // Unset the environment, to ensure it's handled prism side. testStreamIds = append(testStreamIds, tid) diff --git a/sdks/go/pkg/beam/runners/prism/internal/stage.go b/sdks/go/pkg/beam/runners/prism/internal/stage.go index 6762f2a3ff70..a8b8bdd918e4 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/stage.go +++ b/sdks/go/pkg/beam/runners/prism/internal/stage.go @@ -55,15 +55,16 @@ type link struct { // account, but all serialization boundaries remain since the pcollections // would continue to get serialized. type stage struct { - ID string - transforms []string - primaryInput string // PCollection used as the parallel input. - outputs []link // PCollections that must escape this stage. - sideInputs []engine.LinkID // Non-parallel input PCollections and their consumers - internalCols []string // PCollections that escape. Used for precise coder sending. - envID string - stateful bool - hasTimers []string + ID string + transforms []string + primaryInput string // PCollection used as the parallel input. + outputs []link // PCollections that must escape this stage. + sideInputs []engine.LinkID // Non-parallel input PCollections and their consumers + internalCols []string // PCollections that escape. Used for precise coder sending. + envID string + stateful bool + hasTimers []string + processingTimeTimers map[string]bool exe transformExecuter inputTransformID string @@ -367,6 +368,12 @@ func buildDescriptor(stg *stage, comps *pipepb.Components, wk *worker.W, em *eng } for timerID, v := range pardo.GetTimerFamilySpecs() { stg.hasTimers = append(stg.hasTimers, tid) + if v.TimeDomain == pipepb.TimeDomain_PROCESSING_TIME { + if stg.processingTimeTimers == nil { + stg.processingTimeTimers = map[string]bool{} + } + stg.processingTimeTimers[timerID] = true + } rewrite = true newCid, err := lpUnknownCoders(v.GetTimerFamilyCoderId(), coders, comps.GetCoders()) if err != nil { diff --git a/sdks/go/test/integration/primitives/timers.go b/sdks/go/test/integration/primitives/timers.go index 4b7f6e9765f0..d3d0bb42fbff 100644 --- a/sdks/go/test/integration/primitives/timers.go +++ b/sdks/go/test/integration/primitives/timers.go @@ -17,14 +17,17 @@ package primitives import ( "context" + "fmt" "strconv" "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/state" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/timers" "github.com/apache/beam/sdks/v2/go/pkg/beam/register" "github.com/apache/beam/sdks/v2/go/pkg/beam/testing/passert" + "github.com/apache/beam/sdks/v2/go/pkg/beam/testing/teststream" "github.com/apache/beam/sdks/v2/go/pkg/beam/transforms/periodic" ) @@ -50,7 +53,8 @@ type inputFn[K, V any] struct { Inputs []kv[K, V] } -func (fn *inputFn[K, V]) ProcessElement(_ []byte, emit func(K, V)) { +func (fn *inputFn[K, V]) ProcessElement(imp []byte, emit func(K, V)) { + fmt.Println("XXXX inputFn.ProcessElement kv:", imp) for _, in := range fn.Inputs { emit(in.Key, in.Value) } @@ -150,3 +154,133 @@ func TimersEventTimeUnbounded(s beam.Scope) { return periodic.Impulse(s, now, now.Add(10*time.Second), 0, false) })(s) } + +// TimersEventTimeTestStream validates event time timers in a test stream "driven" pipeline. +func TimersEventTimeTestStream(s beam.Scope) { + timersEventTimePipelineBuilder(func(s beam.Scope) beam.PCollection { + c := teststream.NewConfig() + c.AddElements(123456, []byte{42}) + c.AdvanceWatermarkToInfinity() + return teststream.Create(s, c) + })(s) +} + +// Below here are tests for ProcessingTime timers using TestStream. + +func init() { + register.DoFn2x0[[]byte, func(string, int)](&inputFn[string, int]{}) + register.DoFn5x0[state.Provider, timers.Provider, string, int, func(string, int)](&processingTimeFn{}) +} + +type processingTimeFn struct { + Callback timers.ProcessingTime + MyValue state.Value[int] + + Offset int + TimerOutput int +} + +func (fn *processingTimeFn) ProcessElement(sp state.Provider, tp timers.Provider, key string, value int, emit func(string, int)) { + fmt.Println("XXXX processingTimeFn.ProcessElement kv:", key, value) + fn.Callback.Set(tp, time.Now().Add(10*time.Second)) + fn.MyValue.Write(sp, 0) +} + +func (fn *processingTimeFn) OnTimer(ctx context.Context, ts beam.EventTime, sp state.Provider, tp timers.Provider, key string, timer timers.Context, emit func(string, int)) { + fmt.Println("XXXX processingTimeFn.OnTimer k:", key, ts, timer) + switch timer.Family { + case fn.Callback.Family: + switch timer.Tag { + case "": + read, ok, err := fn.MyValue.Read(sp) + if err != nil { + panic(err) + } + if !ok { + panic("State must be set for key: " + key) + } + emit(key, read) + if read < 3 { + fn.MyValue.Write(sp, read+1) + fn.Callback.Set(tp, time.Now().Add(10*time.Second)) + } + default: + panic("unexpected timer tag: " + timer.Family + " tag:" + timer.Tag + " want: \"\", for key:" + key) + } + default: + if fn.Callback.Family != timer.Family || timer.Tag != "" { + panic("unexpected timer family: " + timer.Family + " tag:" + timer.Tag + " want: " + fn.Callback.Family + ", for key:" + key) + } + } +} + +func regroup(key string, vs func(*int) bool, emit func(kv[string, int])) { + fmt.Println("XXXXX GROUPING OCCURED for key ", key) + var v int + for vs(&v) { + emit(kvfn(key, v)) + } +} + +func dumbdumb(key string, v int) { + fmt.Println("XXXXX DUMBDUMB OCCURED for key ", key, v) +} + +func init() { + register.Function3x0(regroup) + register.Function2x0(dumbdumb) +} + +func timersProcessingTimePipelineBuilder(makeImp func(s beam.Scope) beam.PCollection) func(s beam.Scope) { + return func(s beam.Scope) { + var inputs, wantOutputs []kv[string, int] + + offset := 5000 + timerOutput := 4093 + + numKeys := 1 + numDuplicateTimers := 4 + + for key := 0; key < numKeys; key++ { + k := strconv.Itoa(key) + for i := 0; i < numDuplicateTimers; i++ { + inputs = append(inputs, kvfn(k, i)) + wantOutputs = append(wantOutputs, kvfn(k, i)) + } + } + + imp := makeImp(s) + + fmt.Println("XXXX inputs and outputs", inputs, wantOutputs) + + keyed := beam.ParDo(s, &inputFn[string, int]{ + Inputs: inputs, + }, imp) + times := beam.ParDo(s, &processingTimeFn{ + Offset: offset, + TimerOutput: timerOutput, + Callback: timers.InProcessingTime("Callback"), + MyValue: state.MakeValueState[int]("MyKey"), + }, keyed) + beam.ParDo0(s, dumbdumb, times) + // We GroupByKey here so input to passert is blocked until teststream advances time to Infinity. + // gbk := beam.GroupByKey(s, times) + // beam.ParDo(s, regroup, gbk) + //passert.EqualsList(s, regrouped, wantOutputs) + } +} + +// TimersProcessingTimeBounded validates processing time timers in a bounded pipeline. +func TimersProcessingTimeTestStream_Infinity(s beam.Scope) { + timersProcessingTimePipelineBuilder(func(s beam.Scope) beam.PCollection { + c := teststream.NewConfig() + c.AddElements(123456, []byte{42}) + c.AdvanceWatermarkToInfinity() + c.AdvanceProcessingTime(int64(mtime.FromDuration(10 * time.Second))) + c.AdvanceProcessingTime(int64(mtime.FromDuration(10 * time.Second))) + c.AdvanceProcessingTime(int64(mtime.FromDuration(10 * time.Second))) + // We should get one emission per advancement. + c.AdvanceProcessingTimeToInfinity() + return teststream.Create(s, c) + })(s) +} diff --git a/sdks/go/test/integration/primitives/timers_test.go b/sdks/go/test/integration/primitives/timers_test.go index 01f2706c6293..3e84c0edcbe9 100644 --- a/sdks/go/test/integration/primitives/timers_test.go +++ b/sdks/go/test/integration/primitives/timers_test.go @@ -32,4 +32,7 @@ func TestTimers_EventTime_Unbounded(t *testing.T) { ptest.BuildAndRun(t, TimersEventTimeUnbounded) } -// TODO(https://github.com/apache/beam/issues/29772): Add ProcessingTime Timer tests. +func TestTimers_ProcessingTime_Infinity(t *testing.T) { + integration.CheckFilters(t) + ptest.BuildAndRun(t, TimersProcessingTimeTestStream_Infinity) +} From 10a08f12dd57a0cdaf4305227b8b5c138a8130cc Mon Sep 17 00:00:00 2001 From: lostluck <13907733+lostluck@users.noreply.github.com> Date: Thu, 18 Apr 2024 13:23:12 -0700 Subject: [PATCH 04/21] touch up --- .../prism/internal/engine/elementmanager.go | 16 +++++++++------- .../runners/prism/internal/engine/engine_test.go | 2 +- .../runners/prism/internal/engine/teststream.go | 4 ++-- 3 files changed, 12 insertions(+), 10 deletions(-) diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go b/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go index 2170afa52a02..ac3d2cf2617b 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go @@ -315,10 +315,10 @@ func (em *ElementManager) Bundles(ctx context.Context, nextBundID func() string) for { em.refreshCond.L.Lock() // Check if processing time has advanced before the wait loop. - // emNow := em.ProcessingTimeNow() - // em.watermarkRefreshes.merge(em.processTimeEvents.AdvanceTo(emNow)) + emNow := em.ProcessingTimeNow() + em.watermarkRefreshes.merge(em.processTimeEvents.AdvanceTo(emNow)) // If there are no watermark refreshes available, we wait until there are. - for len(em.watermarkRefreshes) == 0 { + for len(em.watermarkRefreshes) == 0 { // TODO Add processing time event condition instead of piggybacking on watermarks? // Check to see if we must exit select { case <-ctx.Done(): @@ -329,8 +329,8 @@ func (em *ElementManager) Bundles(ctx context.Context, nextBundID func() string) em.refreshCond.Wait() // until watermarks may have changed. // Check if processing time has advanced while we waited, and add refreshes here. (TODO waking on real time here for prod mode) - // emNow := em.ProcessingTimeNow() - // em.watermarkRefreshes.merge(em.processTimeEvents.AdvanceTo(emNow)) + emNow := em.ProcessingTimeNow() + em.watermarkRefreshes.merge(em.processTimeEvents.AdvanceTo(emNow)) } // We know there is some work we can do that may advance the watermarks, @@ -1053,7 +1053,7 @@ func (ss *stageState) AddPending(newPending []element) int { if ss.processingTimeTimers[e.family] { // override the firing timestamp with the current output watermark // TODO sort out bypassing event time timer behavior for processing time timers. - e.timestamp = ss.output + e.timestamp = mtime.MinTimestamp //ss.output // TODO adjust the count fmt.Println("XXXXXX Pending Adjusted Processing Time", e) } @@ -1117,9 +1117,11 @@ func (ss *stageState) AdvanceProcessingTimeTo(now mtime.Time) { ss.mu.Lock() events := ss.processTimeEvents.AdvanceTo(now) ss.mu.Unlock() + var count int for _, es := range events { - ss.AddPending(es) + count += ss.AddPending(es) } + fmt.Println("AAAAAAA AdvanceProcessingTimeTo", ss.ID, count, "added") } // GetSideData returns side input data for the provided transform+input pair, valid to the watermark. diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/engine_test.go b/sdks/go/pkg/beam/runners/prism/internal/engine/engine_test.go index eb6db41d2587..48e18f670276 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/engine_test.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/engine_test.go @@ -226,7 +226,7 @@ func TestTestStream(t *testing.T) { // TestProcessingTime is the suite for validating behaviors around ProcessingTime. // Separate from the TestStream, Timers, and Triggers tests due to the unique nature // of the time domain. -func FOOTestProcessingTime(t *testing.T) { +func TestProcessingTime(t *testing.T) { initRunner(t) tests := []struct { diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/teststream.go b/sdks/go/pkg/beam/runners/prism/internal/engine/teststream.go index 49a24842ab04..464a43088f07 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/teststream.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/teststream.go @@ -238,8 +238,8 @@ func (ev tsProcessingTimeEvent) Execute(em *ElementManager) { em.testStreamHandler.processingTime = em.testStreamHandler.processingTime.Add(ev.AdvanceBy) // // Add the refreshes now so our block prevention logic works. - // emNow := em.ProcessingTimeNow() - // em.watermarkRefreshes.merge(em.processTimeEvents.AdvanceTo(emNow)) + emNow := em.ProcessingTimeNow() + em.watermarkRefreshes.merge(em.processTimeEvents.AdvanceTo(emNow)) } // tsFinalEvent is the "last" event we perform after all preceeding events. From 7478c65a8b5323d4097fa1dadedb1d51082a0b43 Mon Sep 17 00:00:00 2001 From: lostluck <13907733+lostluck@users.noreply.github.com> Date: Tue, 23 Apr 2024 15:06:47 -0700 Subject: [PATCH 05/21] rm merge duplicate --- .../prism/internal/engine/elementmanager.go | 22 ++++++------------- 1 file changed, 7 insertions(+), 15 deletions(-) diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go b/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go index ac3d2cf2617b..f62dd7e73a82 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go @@ -343,7 +343,6 @@ func (em *ElementManager) Bundles(ctx context.Context, nextBundID func() string) ss := em.stages[stageID] watermark, ready := ss.bundleReady(em) if ready { - // TODO Move handling ProcessingTime bundle fireing *HERE* to avoid nonsense against normal eventtime handling. bundleID, ok, reschedule := ss.startBundle(watermark, nextBundID) // Handle the reschedule even when there's no bundle. if reschedule { @@ -633,20 +632,6 @@ func reElementResiduals(residuals []Residual, inputInfo PColInfo, rb RunBundle) return unprocessedElements } -// Residual represents the unprocessed portion of a single element. -type Residual struct { - Element []byte - Delay time.Duration // The relative time delay. - Bounded bool // Whether this element is finite or not. -} - -// Residuals is used to specify process continuations within a bundle. -type Residuals struct { - Data []Residual - TransformID, InputID string // We only allow one SDF at the root of a bundledescriptor so there should only be one each. - MinOutputWatermarks map[string]mtime.Time // Output watermarks (technically per Residual, but aggregated here until it makes a difference.) -} - // PersistBundle uses the tentative bundle output to update the watermarks for the stage. // Each stage has two monotonically increasing watermarks, the input watermark, and the output // watermark. @@ -1442,6 +1427,13 @@ func (ss *stageState) updateWatermarks(em *ElementManager) set[string] { func (ss *stageState) bundleReady(em *ElementManager) (mtime.Time, bool) { ss.mu.Lock() defer ss.mu.Unlock() + + // TODO Move handling ProcessingTime bundle fireing *HERE* to avoid nonsense against normal eventtime handling. + // Specifically, have the override here (is data ready at this time?) + // And in startBundle pull out that data. + // Or I add a new ready based on processing time signal that this returns, that is independently checked + // for sending. + // If the upstream watermark and the input watermark are the same, // then we can't yet process this stage. inputW := ss.input From 73d1fb7df415f873bdc5b299a7da26e84f18144e Mon Sep 17 00:00:00 2001 From: lostluck <13907733+lostluck@users.noreply.github.com> Date: Wed, 24 Apr 2024 15:52:57 -0700 Subject: [PATCH 06/21] Simplify watermark hold tracking. --- .../prism/internal/engine/elementmanager.go | 83 +++++++++++++++---- .../prism/internal/engine/teststream.go | 2 - 2 files changed, 66 insertions(+), 19 deletions(-) diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go b/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go index f62dd7e73a82..bf73754a0a97 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go @@ -328,8 +328,8 @@ func (em *ElementManager) Bundles(ctx context.Context, nextBundID func() string) } em.refreshCond.Wait() // until watermarks may have changed. - // Check if processing time has advanced while we waited, and add refreshes here. (TODO waking on real time here for prod mode) - emNow := em.ProcessingTimeNow() + // Update if the processing time has advanced while we waited, and add refreshes here. (TODO waking on real time here for prod mode) + emNow = em.ProcessingTimeNow() em.watermarkRefreshes.merge(em.processTimeEvents.AdvanceTo(emNow)) } @@ -342,7 +342,11 @@ func (em *ElementManager) Bundles(ctx context.Context, nextBundID func() string) for stageID := range advanced { ss := em.stages[stageID] watermark, ready := ss.bundleReady(em) + ptimeEventsReady := ss.processTimeEvents.Peek() <= emNow if ready { + if ptimeEventsReady { + fmt.Println("XXXX Both ProcessingTime ready & Regularevents ready. stage", stageID) // Panic ? + } bundleID, ok, reschedule := ss.startBundle(watermark, nextBundID) // Handle the reschedule even when there's no bundle. if reschedule { @@ -357,6 +361,62 @@ func (em *ElementManager) Bundles(ctx context.Context, nextBundID func() string) em.inprogressBundles.insert(rb.BundleID) em.refreshCond.L.Unlock() + select { + case <-ctx.Done(): + return + case runStageCh <- rb: + } + em.refreshCond.L.Lock() + } else if ss.processTimeEvents.Peek() <= emNow { + // TODO what if both regular events and processing time are ready? + fmt.Println("XXXX ProcessingTime ready stage", stageID, "watermark", watermark, ss.pending, "aggregate", ss.aggregate) + // Queue up elements! + // TODO impose processing strategy limits. + elems := ss.AdvanceProcessingTimeTo(emNow) + + var toProcess []element + // TODO Sort out real holds (improve hold logic?) + minTs := mtime.MaxTimestamp + newKeys := set[string]{} + holdsInBundle := map[mtime.Time]int{} + for _, es := range elems { + for _, e := range es { + toProcess = append(toProcess, e) + if e.holdTimestamp < minTs { + minTs = e.holdTimestamp + } + holdsInBundle[e.holdTimestamp] = holdsInBundle[e.holdTimestamp] + 1 + newKeys.insert((string)(e.keyBytes)) + } + } + + bundID := nextBundID() + + es := elements{ + es: toProcess, + minTimestamp: minTs, + } + if ss.inprogress == nil { + ss.inprogress = make(map[string]elements) + } + if ss.inprogressKeysByBundle == nil { + ss.inprogressKeysByBundle = make(map[string]set[string]) + } + if ss.inprogressHoldsByBundle == nil { + ss.inprogressHoldsByBundle = make(map[string]map[mtime.Time]int) + } + ss.inprogress[bundID] = es + ss.inprogressKeysByBundle[bundID] = newKeys + ss.inprogressKeys.merge(newKeys) + ss.inprogressHoldsByBundle[bundID] = holdsInBundle + + fmt.Println("XXXXX QQQQQ ProcessingTimeBundle Started stage", stageID, "bundID", bundID, "watermark", watermark, ss.pending, "aggregate", ss.aggregate, "numElms", len(es.es)) + + rb := RunBundle{StageID: stageID, BundleID: bundID, Watermark: watermark} + + em.inprogressBundles.insert(rb.BundleID) + em.refreshCond.L.Unlock() + select { case <-ctx.Done(): return @@ -879,7 +939,6 @@ func (em *ElementManager) refreshWatermarks() set[string] { nextUpdates := set[string]{} refreshed := set[string]{} // Use a single consolidated processing time during a given refresh for consistency. - emNow := em.ProcessingTimeNow() var i int for stageID := range em.watermarkRefreshes { @@ -890,7 +949,6 @@ func (em *ElementManager) refreshWatermarks() set[string] { refreshes := ss.updateWatermarks(em) nextUpdates.merge(refreshes) - ss.AdvanceProcessingTimeTo(emNow) // cap refreshes incrementally. if i < 10 { @@ -1035,13 +1093,6 @@ func (ss *stageState) AddPending(newPending []element) int { dnt.elements.Push(e) if e.IsTimer() { - if ss.processingTimeTimers[e.family] { - // override the firing timestamp with the current output watermark - // TODO sort out bypassing event time timer behavior for processing time timers. - e.timestamp = mtime.MinTimestamp //ss.output - // TODO adjust the count - fmt.Println("XXXXXX Pending Adjusted Processing Time", e) - } if lastSet, ok := dnt.timers[timerKey{family: e.family, tag: e.tag, window: e.window}]; ok { // existing timer! // don't increase the count this time, as "this" timer is already pending. @@ -1098,15 +1149,13 @@ func (ss *stageState) AddProcessingTimePending(newPending map[mtime.Time][]eleme return count } -func (ss *stageState) AdvanceProcessingTimeTo(now mtime.Time) { +// AdvanceProcessingTimeTo pops the processing time and returns all ready to execute process +// continuation elements and processing time timers. +func (ss *stageState) AdvanceProcessingTimeTo(now mtime.Time) [][]element { ss.mu.Lock() events := ss.processTimeEvents.AdvanceTo(now) ss.mu.Unlock() - var count int - for _, es := range events { - count += ss.AddPending(es) - } - fmt.Println("AAAAAAA AdvanceProcessingTimeTo", ss.ID, count, "added") + return events } // GetSideData returns side input data for the provided transform+input pair, valid to the watermark. diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/teststream.go b/sdks/go/pkg/beam/runners/prism/internal/engine/teststream.go index 464a43088f07..693209da6d78 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/teststream.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/teststream.go @@ -217,8 +217,6 @@ func (ev tsWatermarkEvent) Execute(em *ElementManager) { em.testStreamHandler.tagState[ev.Tag] = t // Update the upstream watermarks in the consumers. - - fmt.Println("XXXXX updating watermark to", t.watermark, "for stages", em.consumers[t.pcollection]) for _, sID := range em.consumers[t.pcollection] { ss := em.stages[sID] ss.updateUpstreamWatermark(ss.inputID, t.watermark) From c14e51726b2f4a31119a112b2048033c6c5e3c74 Mon Sep 17 00:00:00 2001 From: lostluck <13907733+lostluck@users.noreply.github.com> Date: Fri, 26 Apr 2024 13:10:50 -0700 Subject: [PATCH 07/21] First successful run! --- .../prism/internal/engine/elementmanager.go | 94 ++++++++++++------- .../prism/internal/engine/processingtime.go | 13 ++- .../prism/internal/engine/teststream.go | 16 +++- .../beam/runners/prism/internal/execute.go | 7 +- sdks/go/test/integration/primitives/timers.go | 28 ++---- 5 files changed, 96 insertions(+), 62 deletions(-) diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go b/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go index bf73754a0a97..4cc8f71466a9 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go @@ -342,7 +342,7 @@ func (em *ElementManager) Bundles(ctx context.Context, nextBundID func() string) for stageID := range advanced { ss := em.stages[stageID] watermark, ready := ss.bundleReady(em) - ptimeEventsReady := ss.processTimeEvents.Peek() <= emNow + ptimeEventsReady := ss.processTimeEvents.Peek() <= emNow || emNow == mtime.MaxTimestamp if ready { if ptimeEventsReady { fmt.Println("XXXX Both ProcessingTime ready & Regularevents ready. stage", stageID) // Panic ? @@ -367,7 +367,7 @@ func (em *ElementManager) Bundles(ctx context.Context, nextBundID func() string) case runStageCh <- rb: } em.refreshCond.L.Lock() - } else if ss.processTimeEvents.Peek() <= emNow { + } else if ptimeEventsReady { // TODO what if both regular events and processing time are ready? fmt.Println("XXXX ProcessingTime ready stage", stageID, "watermark", watermark, ss.pending, "aggregate", ss.aggregate) // Queue up elements! @@ -375,9 +375,9 @@ func (em *ElementManager) Bundles(ctx context.Context, nextBundID func() string) elems := ss.AdvanceProcessingTimeTo(emNow) var toProcess []element - // TODO Sort out real holds (improve hold logic?) minTs := mtime.MaxTimestamp newKeys := set[string]{} + holdsInBundle := map[mtime.Time]int{} for _, es := range elems { for _, e := range es { @@ -390,41 +390,46 @@ func (em *ElementManager) Bundles(ctx context.Context, nextBundID func() string) } } - bundID := nextBundID() + // Only do this if we have anything to process. + if len(toProcess) > 0 { + bundID := nextBundID() - es := elements{ - es: toProcess, - minTimestamp: minTs, - } - if ss.inprogress == nil { - ss.inprogress = make(map[string]elements) - } - if ss.inprogressKeysByBundle == nil { - ss.inprogressKeysByBundle = make(map[string]set[string]) - } - if ss.inprogressHoldsByBundle == nil { - ss.inprogressHoldsByBundle = make(map[string]map[mtime.Time]int) - } - ss.inprogress[bundID] = es - ss.inprogressKeysByBundle[bundID] = newKeys - ss.inprogressKeys.merge(newKeys) - ss.inprogressHoldsByBundle[bundID] = holdsInBundle + es := elements{ + es: toProcess, + minTimestamp: minTs, + } + if ss.inprogress == nil { + ss.inprogress = make(map[string]elements) + } + if ss.inprogressKeysByBundle == nil { + ss.inprogressKeysByBundle = make(map[string]set[string]) + } + if ss.inprogressHoldsByBundle == nil { + ss.inprogressHoldsByBundle = make(map[string]map[mtime.Time]int) + } + ss.inprogress[bundID] = es + ss.inprogressKeysByBundle[bundID] = newKeys + ss.inprogressKeys.merge(newKeys) + ss.inprogressHoldsByBundle[bundID] = holdsInBundle - fmt.Println("XXXXX QQQQQ ProcessingTimeBundle Started stage", stageID, "bundID", bundID, "watermark", watermark, ss.pending, "aggregate", ss.aggregate, "numElms", len(es.es)) + fmt.Println("XXXXX QQQQQ ProcessingTimeBundle Started stage", stageID, "bundID", bundID, "watermark", watermark, ss.pending, "aggregate", ss.aggregate, "numElms", len(es.es), "holds in bundle", holdsInBundle, "stage", ss.watermarkHolds.counts, "pendingTime", ss.processTimeEvents.events) - rb := RunBundle{StageID: stageID, BundleID: bundID, Watermark: watermark} + rb := RunBundle{StageID: stageID, BundleID: bundID, Watermark: watermark} - em.inprogressBundles.insert(rb.BundleID) - em.refreshCond.L.Unlock() + em.inprogressBundles.insert(rb.BundleID) + em.refreshCond.L.Unlock() - select { - case <-ctx.Done(): - return - case runStageCh <- rb: + select { + case <-ctx.Done(): + return + case runStageCh <- rb: + } + em.refreshCond.L.Lock() + } else { + fmt.Println("XXXXXXX no processing time events") } - em.refreshCond.L.Lock() } else { - fmt.Println("XXXX BundleNotReady stage", stageID, "watermark", watermark, ss.pending, "aggregate", ss.aggregate) + fmt.Println("XXXX BundleNotReady stage", stageID, "watermark", watermark, ss.pending, "aggregate", ss.aggregate, ss.processTimeEvents.Peek(), emNow, ptimeEventsReady, ss.processTimeEvents.Peek()-emNow) } } em.checkForQuiescence(advanced) @@ -464,7 +469,7 @@ func (em *ElementManager) checkForQuiescence(advanced set[string]) { // There are no further incoming watermark changes, see if there are test stream events for this job. nextEvent := em.testStreamHandler.NextEvent() if nextEvent != nil { - fmt.Println("XXXX TestStreamEvent!", nextEvent) + fmt.Printf("XXXX TestStreamEvent! %T %v\n", nextEvent, nextEvent) nextEvent.Execute(em) // Decrement pending for the event being processed. em.addPending(-1) @@ -475,6 +480,7 @@ func (em *ElementManager) checkForQuiescence(advanced set[string]) { return } // If there are no refreshes, then there's no mechanism to make progress, so it's time to fast fail. + fmt.Println("XXXX TesStream event was a no-op!") } v := em.livePending.Load() @@ -495,7 +501,7 @@ func (em *ElementManager) checkForQuiescence(advanced set[string]) { outW := ss.OutputWatermark() upPCol, upW := ss.UpstreamWatermark() upS := em.pcolParents[upPCol] - stageState = append(stageState, fmt.Sprintln(id, "watermark in", inW, "out", outW, "upstream", upW, "from", upS, "pending", ss.pending, "byKey", ss.pendingByKeys, "inprogressKeys", ss.inprogressKeys, "byBundle", ss.inprogressKeysByBundle, "holds", ss.watermarkHolds.heap, "holdCounts", ss.watermarkHolds.counts)) + stageState = append(stageState, fmt.Sprintln(id, "watermark in", inW, "out", outW, "upstream", upW, "from", upS, "pending", ss.pending, "byKey", ss.pendingByKeys, "inprogressKeys", ss.inprogressKeys, "byBundle", ss.inprogressKeysByBundle, "holds", ss.watermarkHolds.heap, "holdCounts", ss.watermarkHolds.counts, "ptEvents", ss.processTimeEvents.events)) } panic(fmt.Sprintf("nothing in progress and no refreshes with non zero pending elements: %v\n%v", v, strings.Join(stageState, ""))) } @@ -767,6 +773,7 @@ func (em *ElementManager) PersistBundle(rb RunBundle, col2Coders map[string]PCol em.triageTimers(d, inputInfo, stage) // Return unprocessed to this stage's pending + // TODO sort out pending element watermark holds for process continuation residuals. unprocessedElements := reElementResiduals(residuals.Data, inputInfo, rb) // Add unprocessed back to the pending stack. @@ -839,6 +846,7 @@ func (em *ElementManager) triageTimers(d TentativeData, inputInfo PColInfo, stag var pendingEventTimers []element var pendingProcessingTimers map[mtime.Time][]element + holds := map[mtime.Time]int{} for tentativeKey, timers := range d.timers { keyToTimers := map[timerKey]element{} for _, t := range timers { @@ -870,12 +878,22 @@ func (em *ElementManager) triageTimers(d TentativeData, inputInfo PColInfo, stag prev := pendingProcessingTimers[newTimerFire] prev = append(prev, elm) pendingProcessingTimers[newTimerFire] = prev + holds[elm.holdTimestamp] += holds[elm.holdTimestamp] + 1 } else { pendingEventTimers = append(pendingEventTimers, elm) } } } + // Add holds for pending processing time timers. + if len(holds) > 0 { + stage.mu.Lock() + for h, v := range holds { + stage.watermarkHolds.Add(h, v) + } + stage.mu.Unlock() + } + if len(pendingEventTimers) > 0 { count := stage.AddPending(pendingEventTimers) em.addPending(count) @@ -1139,6 +1157,8 @@ func (ss *stageState) AddProcessingTimePending(newPending map[mtime.Time][]eleme defer ss.mu.Unlock() var count int + // TODO add pending watermark for input data for watermark propagation, for non-timers. + // TODO sort out processing time watermark holds here. // TODO sort out per key processing time uniqueness here (only "one" can be set for a given key+fam+window, just like event times.) // - Only the last set one can work. @@ -1521,17 +1541,21 @@ func (em *ElementManager) ProcessingTimeNow() (ret mtime.Time) { // defer func() { // fmt.Println("XXXX ProcessingTimeNow -> ", ret, em.processTimeEvents.order) // }() - if em.testStreamHandler != nil { + if em.testStreamHandler != nil && !em.testStreamHandler.completed { + fmt.Println("XXXX ProcessingTimeNow via-testStream ", em.testStreamHandler.Now()) return em.testStreamHandler.Now() } // "Test" mode -> advance to next processing time event if any, to allow execution. // if test mode... if t, ok := em.processTimeEvents.Peek(); ok { + fmt.Println("XXXX ProcessingTimeNow via-event ", t) return t } // "Production" mode, always real time now. - return mtime.Now() + now := mtime.Now() + fmt.Println("XXXX ProcessingTimeNow via-now ", now) + return now } // rebaseProcessingTime turns an absolute processing time to be relative to the provided local clock now. diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/processingtime.go b/sdks/go/pkg/beam/runners/prism/internal/engine/processingtime.go index 4222c06b5adc..9dfedc7ce912 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/processingtime.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/processingtime.go @@ -16,7 +16,6 @@ package engine import ( "container/heap" - "fmt" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/mtime" ) @@ -85,7 +84,6 @@ func newStageRefreshQueue() *stageRefreshQueue { // Schedule a stage event at the given time. func (q *stageRefreshQueue) Schedule(t mtime.Time, stageID string) { - fmt.Println("XXXX stageRefreshQueue scheduled! ", t, stageID) if s, ok := q.events[t]; ok { // We already have a trigger at this time, mutate that instead. if s.present(stageID) { @@ -114,8 +112,6 @@ func (q *stageRefreshQueue) AdvanceTo(now mtime.Time) set[string] { notify := set[string]{} for { if len(q.order) == 0 || q.order[0] > now { - t, ok := q.Peek() - fmt.Println("XXXX stageRefreshQueue advanced to ", now, "refreshed", notify, "next time?", ok, t, t-now) return notify } // pop elements off the queue until the next time is later than now. @@ -167,3 +163,12 @@ func (q *processingTimeElementQueue) AdvanceTo(now mtime.Time) [][]element { delete(q.events, next) } } + +// Peek returns the next scheduled ProcessingTime event in the queue. +// Returns [mtime.MaxTimestamp] if the queue is empty. +func (q *processingTimeElementQueue) Peek() mtime.Time { + if len(q.order) == 0 { + return mtime.MaxTimestamp + } + return q.order[0] +} diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/teststream.go b/sdks/go/pkg/beam/runners/prism/internal/engine/teststream.go index 693209da6d78..e8565d9c5ec4 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/teststream.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/teststream.go @@ -233,11 +233,18 @@ type tsProcessingTimeEvent struct { // Execute this ProcessingTime event by advancing the synthetic processing time. func (ev tsProcessingTimeEvent) Execute(em *ElementManager) { - em.testStreamHandler.processingTime = em.testStreamHandler.processingTime.Add(ev.AdvanceBy) + if ev.AdvanceBy == time.Duration(mtime.MaxTimestamp) { + em.testStreamHandler.processingTime = mtime.MaxTimestamp.ToTime() + } else { + em.testStreamHandler.processingTime = em.testStreamHandler.processingTime.Add(ev.AdvanceBy) + } // // Add the refreshes now so our block prevention logic works. emNow := em.ProcessingTimeNow() - em.watermarkRefreshes.merge(em.processTimeEvents.AdvanceTo(emNow)) + toRefresh := em.processTimeEvents.AdvanceTo(emNow) + em.watermarkRefreshes.merge(toRefresh) + + fmt.Println("XXXXX processing time advance by", ev.AdvanceBy, "event refreshes", emNow, toRefresh) } // tsFinalEvent is the "last" event we perform after all preceeding events. @@ -251,6 +258,11 @@ func (ev tsFinalEvent) Execute(em *ElementManager) { em.testStreamHandler.UpdateHold(em, mtime.MaxTimestamp) ss := em.stages[ev.stageID] kickSet := ss.updateWatermarks(em) + if ss.OutputWatermark() == mtime.MaxTimestamp { + for k := range em.stages { + kickSet.insert(k) + } + } kickSet.insert(ev.stageID) em.watermarkRefreshes.merge(kickSet) } diff --git a/sdks/go/pkg/beam/runners/prism/internal/execute.go b/sdks/go/pkg/beam/runners/prism/internal/execute.go index 707e61f723a0..372c463f9b6e 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/execute.go +++ b/sdks/go/pkg/beam/runners/prism/internal/execute.go @@ -286,7 +286,12 @@ func executePipeline(ctx context.Context, wks map[string]*worker.W, j *jobservic case *pipepb.TestStreamPayload_Event_WatermarkEvent: tsb.AddWatermarkEvent(ev.WatermarkEvent.GetTag(), mtime.Time(ev.WatermarkEvent.GetNewWatermark())) case *pipepb.TestStreamPayload_Event_ProcessingTimeEvent: - tsb.AddProcessingTimeEvent(time.Duration(ev.ProcessingTimeEvent.GetAdvanceDuration()) * time.Millisecond) + if ev.ProcessingTimeEvent.GetAdvanceDuration() == int64(mtime.MaxTimestamp) { + // TODO: Determine the SDK common formalism for setting processing time to infinity. + tsb.AddProcessingTimeEvent(time.Duration(mtime.MaxTimestamp)) + } else { + tsb.AddProcessingTimeEvent(time.Duration(ev.ProcessingTimeEvent.GetAdvanceDuration()) * time.Millisecond) + } default: return fmt.Errorf("prism error building stage %v - unknown TestStream event type: %T", stage.ID, ev) } diff --git a/sdks/go/test/integration/primitives/timers.go b/sdks/go/test/integration/primitives/timers.go index d3d0bb42fbff..d0a1529ab336 100644 --- a/sdks/go/test/integration/primitives/timers.go +++ b/sdks/go/test/integration/primitives/timers.go @@ -17,7 +17,6 @@ package primitives import ( "context" - "fmt" "strconv" "time" @@ -54,7 +53,6 @@ type inputFn[K, V any] struct { } func (fn *inputFn[K, V]) ProcessElement(imp []byte, emit func(K, V)) { - fmt.Println("XXXX inputFn.ProcessElement kv:", imp) for _, in := range fn.Inputs { emit(in.Key, in.Value) } @@ -178,16 +176,15 @@ type processingTimeFn struct { Offset int TimerOutput int + Cap int } func (fn *processingTimeFn) ProcessElement(sp state.Provider, tp timers.Provider, key string, value int, emit func(string, int)) { - fmt.Println("XXXX processingTimeFn.ProcessElement kv:", key, value) - fn.Callback.Set(tp, time.Now().Add(10*time.Second)) + fn.Callback.Set(tp, time.Now().Add(9*time.Second)) fn.MyValue.Write(sp, 0) } func (fn *processingTimeFn) OnTimer(ctx context.Context, ts beam.EventTime, sp state.Provider, tp timers.Provider, key string, timer timers.Context, emit func(string, int)) { - fmt.Println("XXXX processingTimeFn.OnTimer k:", key, ts, timer) switch timer.Family { case fn.Callback.Family: switch timer.Tag { @@ -200,9 +197,9 @@ func (fn *processingTimeFn) OnTimer(ctx context.Context, ts beam.EventTime, sp s panic("State must be set for key: " + key) } emit(key, read) - if read < 3 { + if read < fn.Cap-1 { fn.MyValue.Write(sp, read+1) - fn.Callback.Set(tp, time.Now().Add(10*time.Second)) + fn.Callback.Set(tp, time.Now().Add(9*time.Second)) } default: panic("unexpected timer tag: " + timer.Family + " tag:" + timer.Tag + " want: \"\", for key:" + key) @@ -215,20 +212,14 @@ func (fn *processingTimeFn) OnTimer(ctx context.Context, ts beam.EventTime, sp s } func regroup(key string, vs func(*int) bool, emit func(kv[string, int])) { - fmt.Println("XXXXX GROUPING OCCURED for key ", key) var v int for vs(&v) { emit(kvfn(key, v)) } } -func dumbdumb(key string, v int) { - fmt.Println("XXXXX DUMBDUMB OCCURED for key ", key, v) -} - func init() { register.Function3x0(regroup) - register.Function2x0(dumbdumb) } func timersProcessingTimePipelineBuilder(makeImp func(s beam.Scope) beam.PCollection) func(s beam.Scope) { @@ -251,8 +242,6 @@ func timersProcessingTimePipelineBuilder(makeImp func(s beam.Scope) beam.PCollec imp := makeImp(s) - fmt.Println("XXXX inputs and outputs", inputs, wantOutputs) - keyed := beam.ParDo(s, &inputFn[string, int]{ Inputs: inputs, }, imp) @@ -261,12 +250,12 @@ func timersProcessingTimePipelineBuilder(makeImp func(s beam.Scope) beam.PCollec TimerOutput: timerOutput, Callback: timers.InProcessingTime("Callback"), MyValue: state.MakeValueState[int]("MyKey"), + Cap: numDuplicateTimers, // Syncs the cycles to the number of duplicate keyed inputs. }, keyed) - beam.ParDo0(s, dumbdumb, times) // We GroupByKey here so input to passert is blocked until teststream advances time to Infinity. - // gbk := beam.GroupByKey(s, times) - // beam.ParDo(s, regroup, gbk) - //passert.EqualsList(s, regrouped, wantOutputs) + gbk := beam.GroupByKey(s, times) + regrouped := beam.ParDo(s, regroup, gbk) + passert.EqualsList(s, regrouped, wantOutputs) } } @@ -279,7 +268,6 @@ func TimersProcessingTimeTestStream_Infinity(s beam.Scope) { c.AdvanceProcessingTime(int64(mtime.FromDuration(10 * time.Second))) c.AdvanceProcessingTime(int64(mtime.FromDuration(10 * time.Second))) c.AdvanceProcessingTime(int64(mtime.FromDuration(10 * time.Second))) - // We should get one emission per advancement. c.AdvanceProcessingTimeToInfinity() return teststream.Create(s, c) })(s) From 0deb0a9089c9592184218a3207dd49fd8eca83a9 Mon Sep 17 00:00:00 2001 From: lostluck <13907733+lostluck@users.noreply.github.com> Date: Fri, 26 Apr 2024 15:18:31 -0700 Subject: [PATCH 08/21] Remove duplicated test run. --- sdks/go/pkg/beam/runners/prism/internal/engine/engine_test.go | 1 - 1 file changed, 1 deletion(-) diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/engine_test.go b/sdks/go/pkg/beam/runners/prism/internal/engine/engine_test.go index 48e18f670276..7b46e96c2c81 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/engine_test.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/engine_test.go @@ -117,7 +117,6 @@ func TestStatefulStages(t *testing.T) { {pipeline: primitives.SetStateParDoClear}, {pipeline: primitives.TimersEventTimeBounded}, {pipeline: primitives.TimersEventTimeUnbounded}, - {pipeline: primitives.TimersEventTimeTestStream}, } configs := []struct { From 1b27632e31f5434fb03341ed693d497f631c5046 Mon Sep 17 00:00:00 2001 From: lostluck <13907733+lostluck@users.noreply.github.com> Date: Fri, 26 Apr 2024 15:27:39 -0700 Subject: [PATCH 09/21] Deduplicate processing time heap. --- .../runners/prism/internal/engine/holds.go | 20 ++++++++-------- .../prism/internal/engine/processingtime.go | 23 ------------------- 2 files changed, 11 insertions(+), 32 deletions(-) diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/holds.go b/sdks/go/pkg/beam/runners/prism/internal/engine/holds.go index 9077b3f439d6..31828fa2f287 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/holds.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/holds.go @@ -22,21 +22,23 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/mtime" ) -// holdHeap orders holds based on their timestamps -// so we can always find the minimum timestamp of pending holds. -type holdHeap []mtime.Time +// mtimeHeap is a minHeap to find the earliest processing time event. +// Used for holds, and general processing time event ordering. +type mtimeHeap []mtime.Time -func (h holdHeap) Len() int { return len(h) } -func (h holdHeap) Less(i, j int) bool { return h[i] < h[j] } -func (h holdHeap) Swap(i, j int) { h[i], h[j] = h[j], h[i] } +func (h mtimeHeap) Len() int { return len(h) } +func (h mtimeHeap) Less(i, j int) bool { + return h[i] < h[j] +} +func (h mtimeHeap) Swap(i, j int) { h[i], h[j] = h[j], h[i] } -func (h *holdHeap) Push(x any) { +func (h *mtimeHeap) Push(x any) { // Push and Pop use pointer receivers because they modify the slice's length, // not just its contents. *h = append(*h, x.(mtime.Time)) } -func (h *holdHeap) Pop() any { +func (h *mtimeHeap) Pop() any { old := *h n := len(old) x := old[n-1] @@ -55,7 +57,7 @@ func (h *holdHeap) Pop() any { // A heap of the hold times is kept so we have quick access to the minimum hold, for calculating // how to advance the watermark. type holdTracker struct { - heap holdHeap + heap mtimeHeap counts map[mtime.Time]int } diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/processingtime.go b/sdks/go/pkg/beam/runners/prism/internal/engine/processingtime.go index 9dfedc7ce912..9c29739598c7 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/processingtime.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/processingtime.go @@ -42,29 +42,6 @@ import ( // A stage may trigger on multiple ticks. // It's up to a stage to schedule additional work on those notices. -// mtimeHeap is a minHeap to find the earliest processing time event. -type mtimeHeap []mtime.Time - -func (h mtimeHeap) Len() int { return len(h) } -func (h mtimeHeap) Less(i, j int) bool { - return h[i] < h[j] -} -func (h mtimeHeap) Swap(i, j int) { h[i], h[j] = h[j], h[i] } - -func (h *mtimeHeap) Push(x any) { - // Push and Pop use pointer receivers because they modify the slice's length, - // not just its contents. - *h = append(*h, x.(mtime.Time)) -} - -func (h *mtimeHeap) Pop() any { - old := *h - n := len(old) - x := old[n-1] - *h = old[0 : n-1] - return x -} - // stageRefreshQueue manages ProcessingTime events, in particular, which stages need notification // at which points in processing time they occur. It doesn't handle the interface between // walltime or any synthetic notions of time. From dce3d47ceb6bb40c742ca8bbd2ad0f800ebd93fa Mon Sep 17 00:00:00 2001 From: lostluck <13907733+lostluck@users.noreply.github.com> Date: Fri, 26 Apr 2024 15:27:51 -0700 Subject: [PATCH 10/21] rm debug text --- sdks/go/pkg/beam/runners/prism/internal/engine/teststream.go | 2 -- 1 file changed, 2 deletions(-) diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/teststream.go b/sdks/go/pkg/beam/runners/prism/internal/engine/teststream.go index e8565d9c5ec4..102901776fe9 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/teststream.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/teststream.go @@ -243,8 +243,6 @@ func (ev tsProcessingTimeEvent) Execute(em *ElementManager) { emNow := em.ProcessingTimeNow() toRefresh := em.processTimeEvents.AdvanceTo(emNow) em.watermarkRefreshes.merge(toRefresh) - - fmt.Println("XXXXX processing time advance by", ev.AdvanceBy, "event refreshes", emNow, toRefresh) } // tsFinalEvent is the "last" event we perform after all preceeding events. From 15010edb28df3424fc5059eb60cdbe0b84602813 Mon Sep 17 00:00:00 2001 From: lostluck <13907733+lostluck@users.noreply.github.com> Date: Fri, 26 Apr 2024 16:05:17 -0700 Subject: [PATCH 11/21] Remove some debug prints, cleanup. --- .../prism/internal/engine/elementmanager.go | 36 ++----------------- .../prism/internal/engine/engine_test.go | 1 + .../prism/internal/engine/processingtime.go | 2 +- .../prism/internal/engine/teststream.go | 12 ++----- sdks/go/test/integration/primitives/timers.go | 22 +++++------- 5 files changed, 16 insertions(+), 57 deletions(-) diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go b/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go index 4cc8f71466a9..4f16235470c1 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go @@ -232,7 +232,6 @@ func (em *ElementManager) StageStateful(ID string) { // StageProcessingTimeTimers indicates which timers are processingTime domain timers. func (em *ElementManager) StageProcessingTimeTimers(ID string, ptTimers map[string]bool) { em.stages[ID].processingTimeTimers = ptTimers - fmt.Println("XXXX stage", ID, "with PT timers", ptTimers, "agg", em.stages[ID].aggregate, "stateful", em.stages[ID].stateful) } // AddTestStream provides a builder interface for the execution layer to build the test stream from @@ -308,7 +307,6 @@ func (em *ElementManager) Bundles(ctx context.Context, nextBundID func() string) // If we have a test stream, clear out existing refreshes, so the test stream can // insert any elements it needs. if em.testStreamHandler != nil { - fmt.Println("XXXXX CLEARING OUT REFRESHSES", em.watermarkRefreshes) em.watermarkRefreshes = singleSet(em.testStreamHandler.ID) } @@ -336,7 +334,6 @@ func (em *ElementManager) Bundles(ctx context.Context, nextBundID func() string) // We know there is some work we can do that may advance the watermarks, // refresh them, and see which stages have advanced. advanced := em.refreshWatermarks() - fmt.Println("XXXXX refreshedWatermarks", advanced) // Check each advanced stage, to see if it's able to execute based on the watermark. for stageID := range advanced { @@ -344,16 +341,12 @@ func (em *ElementManager) Bundles(ctx context.Context, nextBundID func() string) watermark, ready := ss.bundleReady(em) ptimeEventsReady := ss.processTimeEvents.Peek() <= emNow || emNow == mtime.MaxTimestamp if ready { - if ptimeEventsReady { - fmt.Println("XXXX Both ProcessingTime ready & Regularevents ready. stage", stageID) // Panic ? - } bundleID, ok, reschedule := ss.startBundle(watermark, nextBundID) // Handle the reschedule even when there's no bundle. if reschedule { em.watermarkRefreshes.insert(stageID) } if !ok { - fmt.Println("XXXX Bundle Not Fired stage", stageID, "watermark", watermark, ss.pending, "aggregate", ss.aggregate) continue } rb := RunBundle{StageID: stageID, BundleID: bundleID, Watermark: watermark} @@ -368,8 +361,6 @@ func (em *ElementManager) Bundles(ctx context.Context, nextBundID func() string) } em.refreshCond.L.Lock() } else if ptimeEventsReady { - // TODO what if both regular events and processing time are ready? - fmt.Println("XXXX ProcessingTime ready stage", stageID, "watermark", watermark, ss.pending, "aggregate", ss.aggregate) // Queue up elements! // TODO impose processing strategy limits. elems := ss.AdvanceProcessingTimeTo(emNow) @@ -412,8 +403,6 @@ func (em *ElementManager) Bundles(ctx context.Context, nextBundID func() string) ss.inprogressKeys.merge(newKeys) ss.inprogressHoldsByBundle[bundID] = holdsInBundle - fmt.Println("XXXXX QQQQQ ProcessingTimeBundle Started stage", stageID, "bundID", bundID, "watermark", watermark, ss.pending, "aggregate", ss.aggregate, "numElms", len(es.es), "holds in bundle", holdsInBundle, "stage", ss.watermarkHolds.counts, "pendingTime", ss.processTimeEvents.events) - rb := RunBundle{StageID: stageID, BundleID: bundID, Watermark: watermark} em.inprogressBundles.insert(rb.BundleID) @@ -425,11 +414,7 @@ func (em *ElementManager) Bundles(ctx context.Context, nextBundID func() string) case runStageCh <- rb: } em.refreshCond.L.Lock() - } else { - fmt.Println("XXXXXXX no processing time events") } - } else { - fmt.Println("XXXX BundleNotReady stage", stageID, "watermark", watermark, ss.pending, "aggregate", ss.aggregate, ss.processTimeEvents.Peek(), emNow, ptimeEventsReady, ss.processTimeEvents.Peek()-emNow) } } em.checkForQuiescence(advanced) @@ -469,7 +454,6 @@ func (em *ElementManager) checkForQuiescence(advanced set[string]) { // There are no further incoming watermark changes, see if there are test stream events for this job. nextEvent := em.testStreamHandler.NextEvent() if nextEvent != nil { - fmt.Printf("XXXX TestStreamEvent! %T %v\n", nextEvent, nextEvent) nextEvent.Execute(em) // Decrement pending for the event being processed. em.addPending(-1) @@ -480,7 +464,6 @@ func (em *ElementManager) checkForQuiescence(advanced set[string]) { return } // If there are no refreshes, then there's no mechanism to make progress, so it's time to fast fail. - fmt.Println("XXXX TesStream event was a no-op!") } v := em.livePending.Load() @@ -867,7 +850,6 @@ func (em *ElementManager) triageTimers(d TentativeData, inputInfo PColInfo, stag if stage.processingTimeTimers[elm.family] { // Conditionally rebase processing time? Could be handled directly in rebaseProcessingTime... newTimerFire := em.rebaseProcessingTime(emNow, elm.timestamp) - fmt.Println("XXXXXXX Processing Time Timer:", elm.family, "rebased from", elm.timestamp, "to", newTimerFire, "using", emNow, "fireable?", newTimerFire < emNow) elm.timestamp = newTimerFire // Add pending rRocessing timers to the stage's processing time store & schedule event in the manager. em.processTimeEvents.Schedule(newTimerFire, stage.ID) @@ -900,7 +882,6 @@ func (em *ElementManager) triageTimers(d TentativeData, inputInfo PColInfo, stag } if len(pendingProcessingTimers) > 0 { count := stage.AddProcessingTimePending(pendingProcessingTimers) - fmt.Println("XXXXXXX Processing Time Timers:", stage.ID, "num pending:", count, "processingTimeFamilies", stage.processingTimeTimers, pendingProcessingTimers) em.addPending(count) for t := range pendingProcessingTimers { em.processTimeEvents.Schedule(t, stage.ID) @@ -1205,7 +1186,6 @@ func (em *ElementManager) GetSideData(sID, tID, inputID string, watermark mtime. func (ss *stageState) updateUpstreamWatermark(pcol string, upstream mtime.Time) { // A stage will only have a single upstream watermark, so // we simply set this. - fmt.Println("XXXXXXX updateUpstreamWatermark for", ss.ID, "in", pcol, upstream) ss.upstreamWatermarks.Store(pcol, upstream) } @@ -1259,9 +1239,6 @@ func (ss *stageState) startBundle(watermark mtime.Time, genBundID func() string) var toProcess, notYet []element for _, e := range ss.pending { if !ss.aggregate || ss.aggregate && ss.strat.EarliestCompletion(e.window) < watermark { - if ss.aggregate { - fmt.Println("XXXXX SSSSSSSS aggregation stage", ss.ID, "earliest window", ss.strat.EarliestCompletion(e.window), "watermark", watermark) - } toProcess = append(toProcess, e) } else { notYet = append(notYet, e) @@ -1453,7 +1430,6 @@ func (ss *stageState) updateWatermarks(em *ElementManager) set[string] { } refreshes := set[string]{} // If bigger, advance the output watermark - fmt.Println("XXXX updating watermarks for", ss.ID, "update downstream?", newOut > ss.output, "input", ss.input, "output", ss.output, "newOut", newOut, "hold", minWatermarkHold) if newOut > ss.output { ss.output = newOut for _, outputCol := range ss.outputIDs { @@ -1508,7 +1484,7 @@ func (ss *stageState) bundleReady(em *ElementManager) (mtime.Time, bool) { inputW := ss.input _, upstreamW := ss.UpstreamWatermark() if inputW == upstreamW { - slog.Info("bundleReady: insufficient upstream watermark", + slog.Debug("bundleReady: insufficient upstream watermark", slog.String("stage", ss.ID), slog.Group("watermark", slog.Any("upstream", upstreamW), @@ -1530,31 +1506,23 @@ func (ss *stageState) bundleReady(em *ElementManager) (mtime.Time, bool) { ready = false } } - if ss.aggregate { - fmt.Println("XXXXXX SSSSSS aggregate stage", ss.ID, "bundleReady", "upstream", upstreamW, "input", inputW, ss.pending) - } return upstreamW, ready } // ProcessingTimeNow gives the current processing time for the runner. func (em *ElementManager) ProcessingTimeNow() (ret mtime.Time) { - // defer func() { - // fmt.Println("XXXX ProcessingTimeNow -> ", ret, em.processTimeEvents.order) - // }() if em.testStreamHandler != nil && !em.testStreamHandler.completed { - fmt.Println("XXXX ProcessingTimeNow via-testStream ", em.testStreamHandler.Now()) return em.testStreamHandler.Now() } + // TODO toggle between testmode and production mode. // "Test" mode -> advance to next processing time event if any, to allow execution. // if test mode... if t, ok := em.processTimeEvents.Peek(); ok { - fmt.Println("XXXX ProcessingTimeNow via-event ", t) return t } // "Production" mode, always real time now. now := mtime.Now() - fmt.Println("XXXX ProcessingTimeNow via-now ", now) return now } diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/engine_test.go b/sdks/go/pkg/beam/runners/prism/internal/engine/engine_test.go index 7b46e96c2c81..0853be83939c 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/engine_test.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/engine_test.go @@ -232,6 +232,7 @@ func TestProcessingTime(t *testing.T) { pipeline func(s beam.Scope) }{ {pipeline: primitives.TimersProcessingTimeTestStream_Infinity}, + {pipeline: primitives.TimersProcessingTime_Bounded}, } configs := []struct { diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/processingtime.go b/sdks/go/pkg/beam/runners/prism/internal/engine/processingtime.go index 9c29739598c7..a74251849760 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/processingtime.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/processingtime.go @@ -85,9 +85,9 @@ func (q *stageRefreshQueue) Peek() (mtime.Time, bool) { // AdvanceTo takes in the current now time, and returns the set of ids that need a refresh. func (q *stageRefreshQueue) AdvanceTo(now mtime.Time) set[string] { - // If there are no elements, then we're done. notify := set[string]{} for { + // If there are no elements, then we're done. if len(q.order) == 0 || q.order[0] > now { return notify } diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/teststream.go b/sdks/go/pkg/beam/runners/prism/internal/engine/teststream.go index 102901776fe9..5f99c2591752 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/teststream.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/teststream.go @@ -233,13 +233,12 @@ type tsProcessingTimeEvent struct { // Execute this ProcessingTime event by advancing the synthetic processing time. func (ev tsProcessingTimeEvent) Execute(em *ElementManager) { - if ev.AdvanceBy == time.Duration(mtime.MaxTimestamp) { + em.testStreamHandler.processingTime = em.testStreamHandler.processingTime.Add(ev.AdvanceBy) + if em.testStreamHandler.processingTime.After(mtime.MaxTimestamp.ToTime()) || ev.AdvanceBy == time.Duration(mtime.MaxTimestamp) { em.testStreamHandler.processingTime = mtime.MaxTimestamp.ToTime() - } else { - em.testStreamHandler.processingTime = em.testStreamHandler.processingTime.Add(ev.AdvanceBy) } - // // Add the refreshes now so our block prevention logic works. + // Add the refreshes now so our block prevention logic works. emNow := em.ProcessingTimeNow() toRefresh := em.processTimeEvents.AdvanceTo(emNow) em.watermarkRefreshes.merge(toRefresh) @@ -256,11 +255,6 @@ func (ev tsFinalEvent) Execute(em *ElementManager) { em.testStreamHandler.UpdateHold(em, mtime.MaxTimestamp) ss := em.stages[ev.stageID] kickSet := ss.updateWatermarks(em) - if ss.OutputWatermark() == mtime.MaxTimestamp { - for k := range em.stages { - kickSet.insert(k) - } - } kickSet.insert(ev.stageID) em.watermarkRefreshes.merge(kickSet) } diff --git a/sdks/go/test/integration/primitives/timers.go b/sdks/go/test/integration/primitives/timers.go index d0a1529ab336..9041870499a2 100644 --- a/sdks/go/test/integration/primitives/timers.go +++ b/sdks/go/test/integration/primitives/timers.go @@ -52,7 +52,7 @@ type inputFn[K, V any] struct { Inputs []kv[K, V] } -func (fn *inputFn[K, V]) ProcessElement(imp []byte, emit func(K, V)) { +func (fn *inputFn[K, V]) ProcessElement(_ []byte, emit func(K, V)) { for _, in := range fn.Inputs { emit(in.Key, in.Value) } @@ -153,17 +153,7 @@ func TimersEventTimeUnbounded(s beam.Scope) { })(s) } -// TimersEventTimeTestStream validates event time timers in a test stream "driven" pipeline. -func TimersEventTimeTestStream(s beam.Scope) { - timersEventTimePipelineBuilder(func(s beam.Scope) beam.PCollection { - c := teststream.NewConfig() - c.AddElements(123456, []byte{42}) - c.AdvanceWatermarkToInfinity() - return teststream.Create(s, c) - })(s) -} - -// Below here are tests for ProcessingTime timers using TestStream. +// Below here are tests for ProcessingTime timers. func init() { register.DoFn2x0[[]byte, func(string, int)](&inputFn[string, int]{}) @@ -259,7 +249,8 @@ func timersProcessingTimePipelineBuilder(makeImp func(s beam.Scope) beam.PCollec } } -// TimersProcessingTimeBounded validates processing time timers in a bounded pipeline. +// TimersProcessingTimeTestStream_Infinity validates processing time timers in a bounded pipeline +// kicked along by TestStream. func TimersProcessingTimeTestStream_Infinity(s beam.Scope) { timersProcessingTimePipelineBuilder(func(s beam.Scope) beam.PCollection { c := teststream.NewConfig() @@ -272,3 +263,8 @@ func TimersProcessingTimeTestStream_Infinity(s beam.Scope) { return teststream.Create(s, c) })(s) } + +// TimersProcessingTimeBounded validates processing time timers in a bounded pipeline. +func TimersProcessingTime_Bounded(s beam.Scope) { + timersProcessingTimePipelineBuilder(beam.Impulse)(s) +} From c9fbb21fdf27b6a69fa3679b0e60c9b7d0248aba Mon Sep 17 00:00:00 2001 From: lostluck <13907733+lostluck@users.noreply.github.com> Date: Fri, 26 Apr 2024 16:15:42 -0700 Subject: [PATCH 12/21] tiny todo cleanup --- .../beam/runners/prism/internal/engine/elementmanager.go | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go b/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go index 4f16235470c1..843479b6c7c7 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go @@ -1138,9 +1138,7 @@ func (ss *stageState) AddProcessingTimePending(newPending map[mtime.Time][]eleme defer ss.mu.Unlock() var count int - // TODO add pending watermark for input data for watermark propagation, for non-timers. - - // TODO sort out processing time watermark holds here. + // TODO move processing time watermark holds here. // TODO sort out per key processing time uniqueness here (only "one" can be set for a given key+fam+window, just like event times.) // - Only the last set one can work. for t, event := range newPending { @@ -1154,9 +1152,8 @@ func (ss *stageState) AddProcessingTimePending(newPending map[mtime.Time][]eleme // continuation elements and processing time timers. func (ss *stageState) AdvanceProcessingTimeTo(now mtime.Time) [][]element { ss.mu.Lock() - events := ss.processTimeEvents.AdvanceTo(now) - ss.mu.Unlock() - return events + defer ss.mu.Unlock() + return ss.processTimeEvents.AdvanceTo(now) } // GetSideData returns side input data for the provided transform+input pair, valid to the watermark. From 4fb8033e4d2e241531817a41db24ab25d65edbf4 Mon Sep 17 00:00:00 2001 From: lostluck <13907733+lostluck@users.noreply.github.com> Date: Wed, 15 May 2024 08:09:00 -0700 Subject: [PATCH 13/21] ProcessingTime workming most of the time! --- .../pkg/beam/core/runtime/exec/userstate.go | 10 +- sdks/go/pkg/beam/pardo.go | 1 + .../prism/internal/engine/elementmanager.go | 453 +++++++++++------- .../prism/internal/engine/engine_test.go | 8 +- .../runners/prism/internal/engine/holds.go | 12 +- .../prism/internal/engine/processingtime.go | 52 -- .../runners/prism/internal/engine/timers.go | 166 +++++++ .../prism/internal/engine/timers_test.go | 291 +++++++++++ .../runners/prism/internal/worker/worker.go | 4 +- sdks/go/test/integration/primitives/state.go | 85 +++- .../test/integration/primitives/state_test.go | 5 + sdks/go/test/integration/primitives/timers.go | 18 +- 12 files changed, 879 insertions(+), 226 deletions(-) create mode 100644 sdks/go/pkg/beam/runners/prism/internal/engine/timers_test.go diff --git a/sdks/go/pkg/beam/core/runtime/exec/userstate.go b/sdks/go/pkg/beam/core/runtime/exec/userstate.go index f83aee4bf741..e4a0ef169887 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/userstate.go +++ b/sdks/go/pkg/beam/core/runtime/exec/userstate.go @@ -57,7 +57,10 @@ func (s *stateProvider) ReadValueState(userStateID string) (any, []state.Transac return nil, nil, err } dec := MakeElementDecoder(coder.SkipW(s.codersByKey[userStateID])) - resp, err := dec.Decode(rw) + var buf bytes.Buffer + tee := io.TeeReader(rw, &buf) + resp, err := dec.Decode(tee) + // fmt.Println("CCCCCC - read", s.codersByKey[userStateID], userStateID, buf.Bytes()) if err != nil && err != io.EOF { return nil, nil, err } @@ -92,12 +95,15 @@ func (s *stateProvider) WriteValueState(val state.Transaction) error { if err != nil { return err } + var buf bytes.Buffer + mw := io.MultiWriter(ap, &buf) fv := FullValue{Elm: val.Val} enc := MakeElementEncoder(coder.SkipW(s.codersByKey[val.Key])) - err = enc.Encode(&fv, ap) + err = enc.Encode(&fv, mw) if err != nil { return err } + // fmt.Println("CCCCCC - write", s.codersByKey[val.Key], val.Key, buf.Bytes()) // Any transactions before a set don't matter s.transactionsByKey[val.Key] = []state.Transaction{val} diff --git a/sdks/go/pkg/beam/pardo.go b/sdks/go/pkg/beam/pardo.go index 629ce329c9ba..b8a1bc27a1c2 100644 --- a/sdks/go/pkg/beam/pardo.go +++ b/sdks/go/pkg/beam/pardo.go @@ -100,6 +100,7 @@ func TryParDo(s Scope, dofn any, col PCollection, opts ...Option) ([]PCollection if ct := ps.CoderType(); ct != nil { sT := typex.New(ps.CoderType()) c, err := inferCoder(sT) + // fmt.Println("CCCCCC", c, graphx.UserStateCoderID(ps)) if err != nil { return nil, addParDoCtx(err, s) } diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go b/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go index 843479b6c7c7..a4f08cd571da 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go @@ -231,7 +231,7 @@ func (em *ElementManager) StageStateful(ID string) { // StageProcessingTimeTimers indicates which timers are processingTime domain timers. func (em *ElementManager) StageProcessingTimeTimers(ID string, ptTimers map[string]bool) { - em.stages[ID].processingTimeTimers = ptTimers + em.stages[ID].processingTimeTimersFamilies = ptTimers } // AddTestStream provides a builder interface for the execution layer to build the test stream from @@ -314,9 +314,11 @@ func (em *ElementManager) Bundles(ctx context.Context, nextBundID func() string) em.refreshCond.L.Lock() // Check if processing time has advanced before the wait loop. emNow := em.ProcessingTimeNow() - em.watermarkRefreshes.merge(em.processTimeEvents.AdvanceTo(emNow)) + ptRefreshed := em.processTimeEvents.AdvanceTo(emNow) + em.watermarkRefreshes.merge(ptRefreshed) + // If there are no watermark refreshes available, we wait until there are. - for len(em.watermarkRefreshes) == 0 { // TODO Add processing time event condition instead of piggybacking on watermarks? + for len(em.watermarkRefreshes)+len(ptRefreshed) == 0 { // TODO Add processing time event condition instead of piggybacking on watermarks? // Check to see if we must exit select { case <-ctx.Done(): @@ -328,82 +330,51 @@ func (em *ElementManager) Bundles(ctx context.Context, nextBundID func() string) // Update if the processing time has advanced while we waited, and add refreshes here. (TODO waking on real time here for prod mode) emNow = em.ProcessingTimeNow() - em.watermarkRefreshes.merge(em.processTimeEvents.AdvanceTo(emNow)) + ptRefreshed = em.processTimeEvents.AdvanceTo(emNow) + em.watermarkRefreshes.merge(ptRefreshed) } // We know there is some work we can do that may advance the watermarks, // refresh them, and see which stages have advanced. advanced := em.refreshWatermarks() + advanced.merge(ptRefreshed) // Check each advanced stage, to see if it's able to execute based on the watermark. for stageID := range advanced { ss := em.stages[stageID] - watermark, ready := ss.bundleReady(em) - ptimeEventsReady := ss.processTimeEvents.Peek() <= emNow || emNow == mtime.MaxTimestamp + watermark, ready, ptimeEventsReady := ss.bundleReady(em, emNow) + //fmt.Println("ptimeEventsReady ", ss.ID, ptimeEventsReady, "now", emNow, "watermark", watermark, "watermarkReady", ready) if ready { - bundleID, ok, reschedule := ss.startBundle(watermark, nextBundID) + bundleID, ok, reschedule := ss.startEventTimeBundle(watermark, nextBundID) // Handle the reschedule even when there's no bundle. if reschedule { em.watermarkRefreshes.insert(stageID) } - if !ok { - continue - } - rb := RunBundle{StageID: stageID, BundleID: bundleID, Watermark: watermark} + if ok { + rb := RunBundle{StageID: stageID, BundleID: bundleID, Watermark: watermark} - em.inprogressBundles.insert(rb.BundleID) - em.refreshCond.L.Unlock() + em.inprogressBundles.insert(rb.BundleID) + em.refreshCond.L.Unlock() - select { - case <-ctx.Done(): - return - case runStageCh <- rb: + select { + case <-ctx.Done(): + return + case runStageCh <- rb: + } + em.refreshCond.L.Lock() } - em.refreshCond.L.Lock() - } else if ptimeEventsReady { + } + if ptimeEventsReady { + // fmt.Println("what's firing next?", ss.ID, "now", emNow, ss.processingTimeTimers.toFire) // Queue up elements! // TODO impose processing strategy limits. - elems := ss.AdvanceProcessingTimeTo(emNow) - - var toProcess []element - minTs := mtime.MaxTimestamp - newKeys := set[string]{} - - holdsInBundle := map[mtime.Time]int{} - for _, es := range elems { - for _, e := range es { - toProcess = append(toProcess, e) - if e.holdTimestamp < minTs { - minTs = e.holdTimestamp - } - holdsInBundle[e.holdTimestamp] = holdsInBundle[e.holdTimestamp] + 1 - newKeys.insert((string)(e.keyBytes)) - } + bundleID, ok, reschedule := ss.startProcessingTimeBundle(em, emNow, nextBundID) + // Handle the reschedule even when there's no bundle. + if reschedule { + em.watermarkRefreshes.insert(stageID) } - - // Only do this if we have anything to process. - if len(toProcess) > 0 { - bundID := nextBundID() - - es := elements{ - es: toProcess, - minTimestamp: minTs, - } - if ss.inprogress == nil { - ss.inprogress = make(map[string]elements) - } - if ss.inprogressKeysByBundle == nil { - ss.inprogressKeysByBundle = make(map[string]set[string]) - } - if ss.inprogressHoldsByBundle == nil { - ss.inprogressHoldsByBundle = make(map[string]map[mtime.Time]int) - } - ss.inprogress[bundID] = es - ss.inprogressKeysByBundle[bundID] = newKeys - ss.inprogressKeys.merge(newKeys) - ss.inprogressHoldsByBundle[bundID] = holdsInBundle - - rb := RunBundle{StageID: stageID, BundleID: bundID, Watermark: watermark} + if ok { + rb := RunBundle{StageID: stageID, BundleID: bundleID, Watermark: watermark} em.inprogressBundles.insert(rb.BundleID) em.refreshCond.L.Unlock() @@ -462,6 +433,11 @@ func (em *ElementManager) checkForQuiescence(advanced set[string]) { // It's not correct to move to the next event if no refreshes would occur. if len(em.watermarkRefreshes) > 0 { return + } else if _, ok := nextEvent.(tsProcessingTimeEvent); ok { + // It's impossible to fully control processing time SDK side handling for processing time + // Runner side, so we specialize refresh handling here to avoid spuriously getting stuck. + em.watermarkRefreshes.insert(em.testStreamHandler.ID) + return } // If there are no refreshes, then there's no mechanism to make progress, so it's time to fast fail. } @@ -477,6 +453,12 @@ func (em *ElementManager) checkForQuiescence(advanced set[string]) { slog.Debug("Bundles: nothing in progress and no refreshes", slog.Int64("pendingElementCount", v)) var stageState []string ids := maps.Keys(em.stages) + if em.testStreamHandler != nil { + stageState = append(stageState, fmt.Sprintf("TestStreamHandler: completed %v, curIndex %v of %v events: %+v, processingTime %v, %v, ptEvents %v \n", + em.testStreamHandler.completed, em.testStreamHandler.nextEventIndex, len(em.testStreamHandler.events), em.testStreamHandler.events, em.testStreamHandler.processingTime, mtime.FromTime(em.testStreamHandler.processingTime), em.processTimeEvents)) + } else { + stageState = append(stageState, fmt.Sprintf("ElementManager Now: %v processingTimeEvents: %v\n", em.ProcessingTimeNow(), em.processTimeEvents.events)) + } sort.Strings(ids) for _, id := range ids { ss := em.stages[id] @@ -484,7 +466,7 @@ func (em *ElementManager) checkForQuiescence(advanced set[string]) { outW := ss.OutputWatermark() upPCol, upW := ss.UpstreamWatermark() upS := em.pcolParents[upPCol] - stageState = append(stageState, fmt.Sprintln(id, "watermark in", inW, "out", outW, "upstream", upW, "from", upS, "pending", ss.pending, "byKey", ss.pendingByKeys, "inprogressKeys", ss.inprogressKeys, "byBundle", ss.inprogressKeysByBundle, "holds", ss.watermarkHolds.heap, "holdCounts", ss.watermarkHolds.counts, "ptEvents", ss.processTimeEvents.events)) + stageState = append(stageState, fmt.Sprintln(id, "watermark in", inW, "out", outW, "upstream", upW, "from", upS, "pending", ss.pending, "byKey", ss.pendingByKeys, "inprogressKeys", ss.inprogressKeys, "byBundle", ss.inprogressKeysByBundle, "holds", ss.watermarkHolds.heap, "holdCounts", ss.watermarkHolds.counts, "holdsInBundle", ss.inprogressHoldsByBundle, "pttEvents", ss.processingTimeTimers.toFire)) } panic(fmt.Sprintf("nothing in progress and no refreshes with non zero pending elements: %v\n%v", v, strings.Join(stageState, ""))) } @@ -753,7 +735,7 @@ func (em *ElementManager) PersistBundle(rb RunBundle, col2Coders map[string]PCol // Triage timers into their time domains for scheduling. // EventTime timers are handled with normal elements, // ProcessingTime timers need to be scheduled into the processing time based queue. - em.triageTimers(d, inputInfo, stage) + newHolds, ptRefreshes := em.triageTimers(d, inputInfo, stage) // Return unprocessed to this stage's pending // TODO sort out pending element watermark holds for process continuation residuals. @@ -777,10 +759,20 @@ func (em *ElementManager) PersistBundle(rb RunBundle, col2Coders map[string]PCol } delete(stage.inprogressKeysByBundle, rb.BundleID) + // Add new holds. + for h, c := range newHolds { + if c > 0 { + stage.watermarkHolds.Add(h, c) + } else if c < 0 { + stage.watermarkHolds.Drop(h, -c) + } + } + //fmt.Println("QQQQQ persistbundle", rb.StageID, rb.BundleID, "drop holds", stage.inprogressHoldsByBundle[rb.BundleID]) for hold, v := range stage.inprogressHoldsByBundle[rb.BundleID] { stage.watermarkHolds.Drop(hold, v) } delete(stage.inprogressHoldsByBundle, rb.BundleID) + //fmt.Println("QQQQQ persistbundle", rb.StageID, rb.BundleID, "post drop holds", stage.watermarkHolds.counts) // If there are estimated output watermarks, set the estimated // output watermark for the stage. @@ -812,11 +804,11 @@ func (em *ElementManager) PersistBundle(rb RunBundle, col2Coders map[string]PCol } stage.mu.Unlock() - em.addRefreshAndClearBundle(stage.ID, rb.BundleID) + em.addRefreshAndClearBundle(stage.ID, rb.BundleID, ptRefreshes) } // triageTimers prepares received timers for eventual firing, as well as rebasing processing time timers as needed. -func (em *ElementManager) triageTimers(d TentativeData, inputInfo PColInfo, stage *stageState) { +func (em *ElementManager) triageTimers(d TentativeData, inputInfo PColInfo, stage *stageState) (map[mtime.Time]int, set[mtime.Time]) { // Process each timer family in the order we received them, so we can filter to the last one. // Since we're process each timer family individually, use a unique key for each userkey, tag, window. // The last timer set for each combination is the next one we're keeping. @@ -825,11 +817,13 @@ func (em *ElementManager) triageTimers(d TentativeData, inputInfo PColInfo, stag tag string win typex.Window } + em.refreshCond.L.Lock() emNow := em.ProcessingTimeNow() + em.refreshCond.L.Unlock() var pendingEventTimers []element - var pendingProcessingTimers map[mtime.Time][]element - holds := map[mtime.Time]int{} + var pendingProcessingTimers []fireElement + stageRefreshTimes := set[mtime.Time]{} for tentativeKey, timers := range d.timers { keyToTimers := map[timerKey]element{} for _, t := range timers { @@ -838,7 +832,7 @@ func (em *ElementManager) triageTimers(d TentativeData, inputInfo PColInfo, stag keyToTimers[timerKey{key: string(key), tag: tag, win: e.window}] = e } if len(elms) == 0 { - // TODO(lostluck): Determine best way to mark clear a timer cleared. + // TODO(lostluck): Determine best way to mark a timer cleared. continue } } @@ -847,46 +841,39 @@ func (em *ElementManager) triageTimers(d TentativeData, inputInfo PColInfo, stag elm.transform = tentativeKey.Transform elm.family = tentativeKey.Family - if stage.processingTimeTimers[elm.family] { - // Conditionally rebase processing time? Could be handled directly in rebaseProcessingTime... - newTimerFire := em.rebaseProcessingTime(emNow, elm.timestamp) - elm.timestamp = newTimerFire - // Add pending rRocessing timers to the stage's processing time store & schedule event in the manager. - em.processTimeEvents.Schedule(newTimerFire, stage.ID) + if stage.processingTimeTimersFamilies[elm.family] { + // Conditionally rebase processing time or always rebase? + newTimerFire := rebaseProcessingTime(emNow, elm.timestamp) + elm.timestamp = elm.holdTimestamp // Processing Time always uses the hold timestamp as the resulting event time. + pendingProcessingTimers = append(pendingProcessingTimers, fireElement{ + firing: newTimerFire, + timer: elm, + }) - if pendingProcessingTimers == nil { - pendingProcessingTimers = map[mtime.Time][]element{} - } - prev := pendingProcessingTimers[newTimerFire] - prev = append(prev, elm) - pendingProcessingTimers[newTimerFire] = prev - holds[elm.holdTimestamp] += holds[elm.holdTimestamp] + 1 + // Add pending Processing timers to the stage's processing time store & schedule event in the manager. + stageRefreshTimes.insert(newTimerFire) } else { pendingEventTimers = append(pendingEventTimers, elm) } } } - // Add holds for pending processing time timers. - if len(holds) > 0 { - stage.mu.Lock() - for h, v := range holds { - stage.watermarkHolds.Add(h, v) - } - stage.mu.Unlock() - } - if len(pendingEventTimers) > 0 { count := stage.AddPending(pendingEventTimers) em.addPending(count) } + changedHolds := map[mtime.Time]int{} if len(pendingProcessingTimers) > 0 { - count := stage.AddProcessingTimePending(pendingProcessingTimers) - em.addPending(count) - for t := range pendingProcessingTimers { - em.processTimeEvents.Schedule(t, stage.ID) + stage.mu.Lock() + var count int + for _, v := range pendingProcessingTimers { + count += stage.processingTimeTimers.Persist(v.firing, v.timer, changedHolds) } + // fmt.Println("PERSISTED ", pendingProcessingTimers) + em.addPending(count) + stage.mu.Unlock() } + return changedHolds, stageRefreshTimes } // FailBundle clears the extant data allowing the execution to shut down. @@ -897,7 +884,7 @@ func (em *ElementManager) FailBundle(rb RunBundle) { em.addPending(-len(completed.es)) delete(stage.inprogress, rb.BundleID) stage.mu.Unlock() - em.addRefreshAndClearBundle(rb.StageID, rb.BundleID) + em.addRefreshAndClearBundle(rb.StageID, rb.BundleID, nil) } // ReturnResiduals is called after a successful split, so the remaining work @@ -922,11 +909,15 @@ func (em *ElementManager) addRefreshes(stages set[string]) { em.refreshCond.Broadcast() } -func (em *ElementManager) addRefreshAndClearBundle(stageID, bundID string) { +func (em *ElementManager) addRefreshAndClearBundle(stageID, bundID string, ptRefreshes set[mtime.Time]) { em.refreshCond.L.Lock() defer em.refreshCond.L.Unlock() delete(em.inprogressBundles, bundID) em.watermarkRefreshes.insert(stageID) + //fmt.Println("addRefreshAndClearBundle stage", stageID, "due to", bundID, ptRefreshes, "inEmAlready", em.processTimeEvents.events) + for t := range ptRefreshes { + em.processTimeEvents.Schedule(t, stageID) + } em.refreshCond.Broadcast() } @@ -993,10 +984,10 @@ type stageState struct { sides []LinkID // PCollection IDs of side inputs that can block execution. // Special handling bits - stateful bool // whether this stage uses state or timers, and needs keyed processing. - aggregate bool // whether this stage needs to block for aggregation. - strat winStrat // Windowing Strategy for aggregation fireings. - processingTimeTimers map[string]bool // Indicates which timer families use the processing time domain. + stateful bool // whether this stage uses state or timers, and needs keyed processing. + aggregate bool // whether this stage needs to block for aggregation. + strat winStrat // Windowing Strategy for aggregation fireings. + processingTimeTimersFamilies map[string]bool // Indicates which timer families use the processing time domain. mu sync.Mutex upstreamWatermarks sync.Map // watermark set from inputPCollection's parent. @@ -1009,7 +1000,7 @@ type stageState struct { sideInputs map[LinkID]map[typex.Window][][]byte // side input data for this stage, from {tid, inputID} -> window // Fields for stateful stages which need to be per key. - pendingByKeys map[string]*dataAndTimers // pending input elements by Key, if stateful. + pendingByKeys keyedElements // pending input elements by Key, if stateful. inprogressKeys set[string] // all keys that are assigned to bundles. inprogressKeysByBundle map[string]set[string] // bundle to key assignments. state map[LinkID]map[typex.Window]map[string]StateData // state data for this stage, from {tid, stateID} -> window -> userKey @@ -1021,7 +1012,7 @@ type stageState struct { watermarkHolds *holdTracker inprogressHoldsByBundle map[string]map[mtime.Time]int // bundle to associated holds. - processTimeEvents *processingTimeElementQueue + processingTimeTimers *timerHandler } // timerKey uniquely identifies a given timer within the space of a user key. @@ -1055,7 +1046,7 @@ func makeStageState(ID string, inputIDs, outputIDs []string, sides []LinkID) *st output: mtime.MinTimestamp, estimatedOutput: mtime.MinTimestamp, - processTimeEvents: newPtQueue(), + processingTimeTimers: newTimerHandler(), } // Initialize the upstream watermarks to minTime. @@ -1073,38 +1064,20 @@ func (ss *stageState) AddPending(newPending []element) int { ss.mu.Lock() defer ss.mu.Unlock() if ss.stateful { + changedHolds := map[mtime.Time]int{} if ss.pendingByKeys == nil { ss.pendingByKeys = map[string]*dataAndTimers{} } count := 0 for _, e := range newPending { - count++ if len(e.keyBytes) == 0 { panic(fmt.Sprintf("zero length key: %v %v", ss.ID, ss.inputID)) } - dnt, ok := ss.pendingByKeys[string(e.keyBytes)] - if !ok { - dnt = &dataAndTimers{ - timers: map[timerKey]timerTimes{}, - } - ss.pendingByKeys[string(e.keyBytes)] = dnt - } - dnt.elements.Push(e) - - if e.IsTimer() { - if lastSet, ok := dnt.timers[timerKey{family: e.family, tag: e.tag, window: e.window}]; ok { - // existing timer! - // don't increase the count this time, as "this" timer is already pending. - count-- - // clear out the existing hold for accounting purposes. - ss.watermarkHolds.Drop(lastSet.hold, 1) - } - // Update the last set time on the timer. - dnt.timers[timerKey{family: e.family, tag: e.tag, window: e.window}] = timerTimes{firing: e.timestamp, hold: e.holdTimestamp} - - // Mark the hold in the heap. - ss.watermarkHolds.Add(e.holdTimestamp, 1) - } + count += ss.pendingByKeys.AddElement(changedHolds, e) + } + // fmt.Println("QQQQQ AddPending changedHolds", changedHolds) + for hold, v := range changedHolds { + ss.watermarkHolds.Add(hold, v) } return count } @@ -1114,6 +1087,94 @@ func (ss *stageState) AddPending(newPending []element) int { return len(newPending) } +// keyedElements stores elements bucketed by their key. This simplifies ensuring that +// in a stateful transform, a given key is only being processed by a single bundle at a time. +type keyedElements map[string]*dataAndTimers + +func (ke keyedElements) AddElement(holds map[mtime.Time]int, e element) int { + dnt, ok := ke[string(e.keyBytes)] + if !ok { + dnt = &dataAndTimers{ + timers: map[timerKey]timerTimes{}, + } + ke[string(e.keyBytes)] = dnt + } + return dnt.AddElement(holds, e) +} + +func (ke keyedElements) Merge(holds *holdTracker, o keyedElements) int { + var count int + for k, ov := range o { + v, ok := ke[k] + if !ok { + ke[k] = ov + continue + } + count += v.Merge(holds, ov) + ke[k] = v + } + return count +} + +func (ke keyedElements) String() string { + var b strings.Builder + b.WriteString("keyedElems[") + for k, v := range ke { + b.WriteRune('\n') + b.WriteRune('\t') + b.WriteString(k) + b.WriteRune(':') + b.WriteString(fmt.Sprintf("%+v", v)) + b.WriteRune(',') + } + b.WriteString("]") + return b.String() +} + +// AddElement pushes the element onto the queue and handles timer uniqueness for the key, including +// addressing any watermark holds of the timer. +// +// Returns adjustments to the pending count. +func (dnt *dataAndTimers) AddElement(holds map[mtime.Time]int, e element) int { + dnt.elements.Push(e) + count := 1 + if e.IsTimer() { + if lastSet, ok := dnt.timers[timerKey{family: e.family, tag: e.tag, window: e.window}]; ok { + // existing timer! + // don't increase the count this time, as "this" timer is already pending. + count-- + // clear out the existing hold for accounting purposes. + holds[lastSet.hold] -= 1 + // fmt.Println("ZZZZZZ dropping old hold", lastSet.hold, "for", e) + } + // Update the last set time on the timer. + dnt.timers[timerKey{family: e.family, tag: e.tag, window: e.window}] = timerTimes{firing: e.timestamp, hold: e.holdTimestamp} + + // Mark the hold in the heap. + holds[e.holdTimestamp] += 1 + //fmt.Println("ZZZZZZ addHold for ", e) + } + return count +} + +func (dnt *dataAndTimers) Merge(holds *holdTracker, o *dataAndTimers) int { + count := len(o.elements) + dnt.elements = append(dnt.elements, o.elements...) + heap.Init(&dnt.elements) + for tk, ot := range o.timers { + ct, ok := dnt.timers[tk] + if !ok { + dnt.timers[tk] = ot + continue + } + count-- // subtract the timer from the count, since we aren't adding any. + holds.Drop(ct.hold, 1) + // For timers, keep any existing holds, or last sets, since there's only + // one timer per key anyway. Extra timers will be garbage collected in later processing. + } + return count +} + // AddPendingSide adds elements to be consumed as side inputs. func (ss *stageState) AddPendingSide(newPending []element, tID, inputID string) { ss.mu.Lock() @@ -1132,30 +1193,6 @@ func (ss *stageState) AddPendingSide(newPending []element, tID, inputID string) } } -// AddProcessingTimePending is to add elements and timers that rely on processing time to trigger. -func (ss *stageState) AddProcessingTimePending(newPending map[mtime.Time][]element) int { - ss.mu.Lock() - defer ss.mu.Unlock() - var count int - - // TODO move processing time watermark holds here. - // TODO sort out per key processing time uniqueness here (only "one" can be set for a given key+fam+window, just like event times.) - // - Only the last set one can work. - for t, event := range newPending { - count += len(event) - ss.processTimeEvents.Schedule(t, event) - } - return count -} - -// AdvanceProcessingTimeTo pops the processing time and returns all ready to execute process -// continuation elements and processing time timers. -func (ss *stageState) AdvanceProcessingTimeTo(now mtime.Time) [][]element { - ss.mu.Lock() - defer ss.mu.Unlock() - return ss.processTimeEvents.AdvanceTo(now) -} - // GetSideData returns side input data for the provided transform+input pair, valid to the watermark. func (ss *stageState) GetSideData(tID, inputID string, watermark mtime.Time) map[typex.Window][][]byte { ss.mu.Lock() @@ -1224,10 +1261,10 @@ var ( // startBundle initializes a bundle with elements if possible. // A bundle only starts if there are elements at all, and if it's // an aggregation stage, if the windowing stratgy allows it. -func (ss *stageState) startBundle(watermark mtime.Time, genBundID func() string) (string, bool, bool) { +func (ss *stageState) startEventTimeBundle(watermark mtime.Time, genBundID func() string) (string, bool, bool) { defer func() { if e := recover(); e != nil { - panic(fmt.Sprintf("generating bundle for stage %v at %v panicked\n%v", ss.ID, watermark, e)) + panic(fmt.Sprintf("generating bundle for stage %v at watermark %v panicked\n%v", ss.ID, watermark, e)) } }() ss.mu.Lock() @@ -1251,7 +1288,6 @@ func (ss *stageState) startBundle(watermark mtime.Time, genBundID func() string) // TODO: when we do, we need to ensure that the stage remains schedualable for bundle execution, for remaining pending elements and keys. // With the greedy approach, we don't need to since "new data" triggers a refresh, and so should completing processing of a bundle. newKeys := set[string]{} - stillSchedulable := true holdsInBundle := map[mtime.Time]int{} @@ -1286,7 +1322,7 @@ keysPerBundle: timerCleared = true continue } - holdsInBundle[e.holdTimestamp] = holdsInBundle[e.holdTimestamp] + 1 + holdsInBundle[e.holdTimestamp] += 1 // Clear the "fired" timer so subsequent matches can be ignored. delete(dnt.timers, timerKey{family: e.family, tag: e.tag, window: e.window}) } @@ -1302,8 +1338,9 @@ keysPerBundle: break keysPerBundle } } + stillSchedulable := true if len(ss.pendingByKeys) == 0 && !timerCleared { - // If we're out of data, and timers were not cleared then the watermark is are accurate. + // If we're out of data, and timers were not cleared then the watermark is accurate. stillSchedulable = false } @@ -1312,8 +1349,101 @@ keysPerBundle: return "", false, stillSchedulable } + bundID := ss.makeInProgressBundle(genBundID, toProcess, minTs, newKeys, holdsInBundle) + //fmt.Println("QQQQ eventTimeBundle stage", ss.ID, "bundle", bundID, "holds", holdsInBundle, "newKeys", newKeys, "holdsInStage", ss.watermarkHolds.counts, "state", ss.state, "elements", toProcess) + return bundID, true, stillSchedulable +} + +func (ss *stageState) startProcessingTimeBundle(em *ElementManager, emNow mtime.Time, genBundID func() string) (string, bool, bool) { + ss.mu.Lock() + defer ss.mu.Unlock() + + // TODO: Determine if it's possible and a good idea to treat all EventTime processing as a MinTime + // Special Case for ProcessintTime handling. + // Eg. Always queue EventTime elements at minTime. + // Iterate all available processingTime events until we can't anymore. + // + // Potentially puts too much work on the scheduling thread though. + + var toProcess []element + minTs := mtime.MaxTimestamp + holdsInBundle := map[mtime.Time]int{} + + var notYet []fireElement + + nextTime := ss.processingTimeTimers.Peek() + blockedOnKey := set[string]{} + keyCounts := map[string]int{} + newKeys := set[string]{} + + for nextTime <= emNow { + // fmt.Println("top of the loop", nextTime, "now", emNow, ss.processingTimeTimers.order, ss.processingTimeTimers.toFire) + elems := ss.processingTimeTimers.FireAt(nextTime) + for _, e := range elems { + // Check if we're already executing this timer's key. + if ss.inprogressKeys.present(string(e.keyBytes)) { + blockedOnKey.insert(string(e.keyBytes)) + notYet = append(notYet, fireElement{firing: nextTime, timer: e}) + continue + } + + // If we are set to have OneKeyPerBundle, and we already have a key for this bundle, we process it later. + if len(keyCounts) > 0 && OneKeyPerBundle { + notYet = append(notYet, fireElement{firing: nextTime, timer: e}) + continue + } + // If we are set to have OneElementPerKey, and we already have an element for this key we set this to process later. + if v := keyCounts[string(e.keyBytes)]; v > 0 && OneElementPerKey { + notYet = append(notYet, fireElement{firing: nextTime, timer: e}) + continue + } + keyCounts[string(e.keyBytes)]++ + newKeys.insert(string(e.keyBytes)) + if e.timestamp < minTs { + minTs = e.timestamp + } + holdsInBundle[e.holdTimestamp]++ + + // We're going to process this timer! + toProcess = append(toProcess, e) + // fmt.Println("startProcessingTimeBundle: nextTime", nextTime, "emNow", emNow, "timer", e) + } + + nextTime = ss.processingTimeTimers.Peek() + if nextTime == mtime.MaxTimestamp { + // Escape the loop if there are no more events. + break + } + } + // fmt.Println("out of the loop", nextTime, "now", emNow, ss.processingTimeTimers.order, ss.processingTimeTimers.toFire) + + // Reschedule + // fmt.Println("QQQQQ NOT YET; now", emNow, "dropping", notYet, "processing", toProcess, "curCounts", ss.watermarkHolds.counts) + notYetHolds := map[mtime.Time]int{} + for _, v := range notYet { + ss.processingTimeTimers.Persist(v.firing, v.timer, notYetHolds) + em.processTimeEvents.Schedule(v.firing, ss.ID) + } + // fmt.Println("post-reschedule", nextTime, "now", emNow, "order", ss.processingTimeTimers.order, "toFire", ss.processingTimeTimers.toFire, "emProcTime", em.processTimeEvents) + + // Add a refresh if there are still processing time events to process. + stillSchedulable := (nextTime < emNow && nextTime != mtime.MaxTimestamp || len(notYet) > 0) + + if len(toProcess) == 0 { + // If we have nothing + return "", false, stillSchedulable + } + bundID := ss.makeInProgressBundle(genBundID, toProcess, minTs, newKeys, holdsInBundle) + // fmt.Println("QQQQQ processingTimeBundle", ss.ID, bundID, toProcess, holdsInBundle, "rejected holds", notYetHolds, "stillSchedulable", stillSchedulable) + return bundID, true, stillSchedulable +} + +// makeInProgressBundle is common code to store a set of elements as a bundle in progress. +// +// Callers must hold the stage lock. +func (ss *stageState) makeInProgressBundle(genBundID func() string, toProcess []element, minTs mtime.Time, newKeys set[string], holdsInBundle map[mtime.Time]int) string { + // Catch the ordinary case for the minimum timestamp. if toProcess[0].timestamp < minTs { - // Catch the ordinary case. minTs = toProcess[0].timestamp } @@ -1335,7 +1465,8 @@ keysPerBundle: ss.inprogressKeysByBundle[bundID] = newKeys ss.inprogressKeys.merge(newKeys) ss.inprogressHoldsByBundle[bundID] = holdsInBundle - return bundID, true, stillSchedulable + // fmt.Println("QQQQQ makeInProgressBundle", ss.ID, bundID, "add holds", holdsInBundle) + return bundID } func (ss *stageState) splitBundle(rb RunBundle, firstResidual int) { @@ -1466,7 +1597,7 @@ func (ss *stageState) updateWatermarks(em *ElementManager) set[string] { // bundleReady returns the maximum allowed watermark for this stage, and whether // it's permitted to execute by side inputs. -func (ss *stageState) bundleReady(em *ElementManager) (mtime.Time, bool) { +func (ss *stageState) bundleReady(em *ElementManager, emNow mtime.Time) (mtime.Time, bool, bool) { ss.mu.Lock() defer ss.mu.Unlock() @@ -1476,6 +1607,8 @@ func (ss *stageState) bundleReady(em *ElementManager) (mtime.Time, bool) { // Or I add a new ready based on processing time signal that this returns, that is independently checked // for sending. + ptimeEventsReady := ss.processingTimeTimers.Peek() <= emNow || emNow == mtime.MaxTimestamp + // If the upstream watermark and the input watermark are the same, // then we can't yet process this stage. inputW := ss.input @@ -1486,7 +1619,7 @@ func (ss *stageState) bundleReady(em *ElementManager) (mtime.Time, bool) { slog.Group("watermark", slog.Any("upstream", upstreamW), slog.Any("input", inputW))) - return mtime.MinTimestamp, false + return mtime.MinTimestamp, false, ptimeEventsReady } ready := true for _, side := range ss.sides { @@ -1503,7 +1636,7 @@ func (ss *stageState) bundleReady(em *ElementManager) (mtime.Time, bool) { ready = false } } - return upstreamW, ready + return upstreamW, ready, ptimeEventsReady } // ProcessingTimeNow gives the current processing time for the runner. @@ -1525,6 +1658,6 @@ func (em *ElementManager) ProcessingTimeNow() (ret mtime.Time) { // rebaseProcessingTime turns an absolute processing time to be relative to the provided local clock now. // Necessary to reasonably schedule ProcessingTime timers within a TestStream using pipeline. -func (em *ElementManager) rebaseProcessingTime(localNow, scheduled mtime.Time) mtime.Time { +func rebaseProcessingTime(localNow, scheduled mtime.Time) mtime.Time { return localNow + (scheduled - mtime.Now()) } diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/engine_test.go b/sdks/go/pkg/beam/runners/prism/internal/engine/engine_test.go index 0853be83939c..a8d683127f6c 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/engine_test.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/engine_test.go @@ -117,6 +117,8 @@ func TestStatefulStages(t *testing.T) { {pipeline: primitives.SetStateParDoClear}, {pipeline: primitives.TimersEventTimeBounded}, {pipeline: primitives.TimersEventTimeUnbounded}, + {pipeline: primitives.ValueStateParDo_Bytes}, + // {pipeline: primitives.ValueStateParDo_Row}, } configs := []struct { @@ -240,9 +242,9 @@ func TestProcessingTime(t *testing.T) { OneElementPerKey, OneKeyPerBundle bool }{ {"Greedy", false, false}, - // {"AllElementsPerKey", false, true}, - // {"OneElementPerKey", true, false}, - // {"OneElementPerBundle", true, true}, + {"AllElementsPerKey", false, true}, + {"OneElementPerKey", true, false}, + {"OneElementPerBundle", true, true}, } for _, config := range configs { for _, test := range tests { diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/holds.go b/sdks/go/pkg/beam/runners/prism/internal/engine/holds.go index 31828fa2f287..d46b7f69bc4d 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/holds.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/holds.go @@ -46,6 +46,15 @@ func (h *mtimeHeap) Pop() any { return x } +func (h *mtimeHeap) Remove(toRemove mtime.Time) { + for i, v := range *h { + if v == toRemove { + heap.Remove(h, i) + return + } + } +} + // holdTracker track the watermark holds for a stage. // // Timers hold back the watermark until they fire, but multiple @@ -89,8 +98,7 @@ func (ht *holdTracker) Drop(hold mtime.Time, v int) { // Add a hold a number of times to heap. If the hold time isn't already present in the heap, it is added. func (ht *holdTracker) Add(hold mtime.Time, v int) { // Mark the hold in the heap. - ht.counts[hold] = ht.counts[hold] + v - + ht.counts[hold] += v if len(ht.counts) != len(ht.heap) { // Since there's a difference, the hold should not be in the heap, so we add it. heap.Push(&ht.heap, hold) diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/processingtime.go b/sdks/go/pkg/beam/runners/prism/internal/engine/processingtime.go index a74251849760..54f402b3b0c0 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/processingtime.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/processingtime.go @@ -97,55 +97,3 @@ func (q *stageRefreshQueue) AdvanceTo(now mtime.Time) set[string] { delete(q.events, next) } } - -// newPtQueue creates an initialized processingTimeElementQueue. -func newPtQueue() *processingTimeElementQueue { - return &processingTimeElementQueue{ - events: map[mtime.Time][]element{}, - } -} - -// processingTimeElementQueue tracks the actual elements associated with a given processing time. -// -// processingTimeElementQueue is not goroutine safe and relies on external synchronization. -type processingTimeElementQueue struct { - events map[mtime.Time][]element - order mtimeHeap -} - -// Schedule elements for the given processing time. Takes ownership of the -// passed in slice of elements. -func (q *processingTimeElementQueue) Schedule(t mtime.Time, event []element) { - if s, ok := q.events[t]; ok { - // We already have a trigger at this time, mutate that instead. - s = append(s, event...) - q.events[t] = s - return - } - q.events[t] = event - heap.Push(&q.order, t) -} - -// AdvanceTo takes in the current now time, and returns the set of ids that need a refresh. -func (q *processingTimeElementQueue) AdvanceTo(now mtime.Time) [][]element { - // If there are no elements, then we're done. - var ret [][]element - for { - if len(q.order) == 0 || q.order[0] > now { - return ret - } - // pop elements off the queue until the next time is later than now. - next := heap.Pop(&q.order).(mtime.Time) - ret = append(ret, q.events[next]) - delete(q.events, next) - } -} - -// Peek returns the next scheduled ProcessingTime event in the queue. -// Returns [mtime.MaxTimestamp] if the queue is empty. -func (q *processingTimeElementQueue) Peek() mtime.Time { - if len(q.order) == 0 { - return mtime.MaxTimestamp - } - return q.order[0] -} diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/timers.go b/sdks/go/pkg/beam/runners/prism/internal/engine/timers.go index 245b82dd10dd..3f52ebc4510c 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/timers.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/timers.go @@ -17,10 +17,12 @@ package engine import ( "bytes" + "container/heap" "encoding/binary" "fmt" "io" "math" + "sync" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/coder" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/mtime" @@ -172,3 +174,167 @@ func (d *decoder) Pane() typex.PaneInfo { } return pn } + +// timerHandler tracks timers and ensures that the timer invariant is maintained +// and reports changes in watermark holds. +// +// The invariant is that only a single timer exists for a given userKey+timerID+tag+window. +// +// Timers may prevent the event time watermark using a watermark Hold. +// However due to the invariant, the watermark hold must be changed if a given timer +// has it's firing + hold time updated. +// +// A timerHandler may only hold timers of a single domain, either event time timers, or +// processing time timers. They must not be mixed. +type timerHandler struct { + order mtimeHeap // Maintain the next times to fire at. + toFire map[mtime.Time]map[string]set[timerKey] // fireing time -> userkey -> timerID+tag+window: Lookup + nextFiring map[string]map[timerKey]fireElement // userkey -> timerID+tag+window: actual timer + + timerKeySetPool sync.Pool // set[timerKey]{} + userKeysSetPool sync.Pool // map[string]set[timerKey] + firingMapPool sync.Pool // map[timerKey]element +} + +type fireElement struct { + firing mtime.Time + timer element +} + +func newTimerHandler() *timerHandler { + return &timerHandler{ + toFire: map[mtime.Time]map[string]set[timerKey]{}, + nextFiring: map[string]map[timerKey]fireElement{}, + + timerKeySetPool: sync.Pool{New: func() any { + return set[timerKey]{} + }}, + userKeysSetPool: sync.Pool{New: func() any { + return map[string]set[timerKey]{} + }}, + firingMapPool: sync.Pool{New: func() any { + return map[timerKey]fireElement{} + }}, + } +} + +// timers returns the timers for the userkey. +func (th *timerHandler) timers(timer element) map[timerKey]fireElement { + timers, ok := th.nextFiring[string(timer.keyBytes)] + if !ok { + timers = th.firingMapPool.Get().(map[timerKey]fireElement) + th.nextFiring[string(timer.keyBytes)] = timers + } + return timers +} + +func (th *timerHandler) removeTimer(userKey string, key timerKey) element { + timers, ok := th.nextFiring[userKey] + if !ok { + panic(fmt.Sprintf("prism consistency error: trying to remove a timer for a key without timers: %v,%+v", userKey, key)) + } + times, ok := timers[key] + if !ok { + panic(fmt.Sprintf("prism consistency error: trying to remove a non-existent timer for a key: %v,%+v", userKey, key)) + } + delete(timers, key) + if len(timers) == 0 { + delete(th.nextFiring, userKey) + th.firingMapPool.Put(timers) + } + return times.timer +} + +func (th *timerHandler) add(key timerKey, newFire fireElement) { + byKeys, ok := th.toFire[newFire.firing] + if !ok { + byKeys = th.userKeysSetPool.Get().(map[string]set[timerKey]) + th.toFire[newFire.firing] = byKeys + heap.Push(&th.order, newFire.firing) // We only need to add a firing order when inserting. + } + timers, ok := byKeys[string(newFire.timer.keyBytes)] + if !ok { + timers = th.timerKeySetPool.Get().(set[timerKey]) + byKeys[string(newFire.timer.keyBytes)] = timers + + } + timers.insert(key) + +} + +func (th *timerHandler) replace(key timerKey, oldTimer, newTimer fireElement) { + byKeys := th.toFire[oldTimer.firing] + timers := byKeys[string(oldTimer.timer.keyBytes)] + timers.remove(key) + + th.add(key, newTimer) + + // Clean up timers. + if len(timers) == 0 { + th.timerKeySetPool.Put(timers) + delete(byKeys, string(oldTimer.timer.keyBytes)) + } + if len(byKeys) > 0 { + return + } + th.userKeysSetPool.Put(byKeys) + delete(th.toFire, oldTimer.firing) + th.order.Remove(oldTimer.firing) +} + +// Persist the given timer, and updates the provided hold times map with changes to the hold counts. +func (th *timerHandler) Persist(fire mtime.Time, timer element, holdChanges map[mtime.Time]int) int { + timers := th.timers(timer) + key := timerKey{family: timer.family, tag: timer.tag, window: timer.window} + newTimer := fireElement{firing: fire, timer: timer} + if oldTimer, ok := timers[key]; ok { + // Update with the new times + timers[key] = newTimer + th.replace(key, oldTimer, newTimer) + + holdChanges[newTimer.timer.holdTimestamp] += 1 + holdChanges[oldTimer.timer.holdTimestamp] -= 1 + if holdChanges[oldTimer.timer.holdTimestamp] == 0 { + delete(holdChanges, oldTimer.timer.holdTimestamp) + } + return 0 + } + timers[key] = newTimer + th.add(key, newTimer) + holdChanges[newTimer.timer.holdTimestamp] += 1 + return 1 +} + +// FireAt returns all timers for a key able to fire at the given time. +func (th *timerHandler) FireAt(now mtime.Time) []element { + if th.order.Len() == 0 { + return nil + } + var ret []element + for len(th.order) > 0 && th.order[0] <= now { + next := th.order[0] + byKeys, ok := th.toFire[next] + if ok { + for k, vs := range byKeys { + for v := range vs { + timer := th.removeTimer(k, v) + ret = append(ret, timer) + } + delete(byKeys, k) + } + } + delete(th.toFire, next) + th.userKeysSetPool.Put(byKeys) + heap.Pop(&th.order) + } + return ret +} + +// Peek returns the next scheduled event in the queue. +// Returns [mtime.MaxTimestamp] if the queue is empty. +func (th *timerHandler) Peek() mtime.Time { + if th.order.Len() == 0 { + return mtime.MaxTimestamp + } + return th.order[0] +} diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/timers_test.go b/sdks/go/pkg/beam/runners/prism/internal/engine/timers_test.go new file mode 100644 index 000000000000..b1878ca7991b --- /dev/null +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/timers_test.go @@ -0,0 +1,291 @@ +// 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 engine + +import ( + "slices" + "testing" + + "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/google/go-cmp/cmp" +) + +func TestTimerHandler(t *testing.T) { + fireTime1, fireTime2, fireTime3 := mtime.FromMilliseconds(1000), mtime.FromMilliseconds(1100), mtime.FromMilliseconds(1200) + eventTime1, eventTime2, eventTime3 := mtime.FromMilliseconds(200), mtime.FromMilliseconds(210), mtime.FromMilliseconds(220) + holdTime1, holdTime2, holdTime3 := mtime.FromMilliseconds(300), mtime.FromMilliseconds(301), mtime.FromMilliseconds(303) + tid := "testtransform" + family := "testfamily" + userKey1, userKey2, userKey3 := []byte("userKey1"), []byte("userKey2"), []byte("userKey3") + glo := window.SingleGlobalWindow[0] + iw1, iw2, iw3 := window.IntervalWindow{End: fireTime1}, window.IntervalWindow{End: fireTime2}, window.IntervalWindow{End: fireTime3} + + elemTagWin := func(userKey []byte, eventTime, holdTime mtime.Time, tag string, window typex.Window) element { + return element{ + window: window, + timestamp: eventTime, + holdTimestamp: holdTime, + pane: typex.NoFiringPane(), // TODO, do something with pane. + transform: tid, + family: family, + tag: tag, + keyBytes: userKey, + } + } + + elem := func(userKey []byte, eventTime, holdTime mtime.Time) element { + return elemTagWin(userKey, eventTime, holdTime, "", glo) + } + + fireElem := func(fire mtime.Time, userKey []byte, eventTime, holdTime mtime.Time) fireElement { + return fireElement{ + firing: fire, + timer: elem(userKey, eventTime, holdTime), + } + } + + tests := []struct { + name string + insert []fireElement + + wantHolds map[mtime.Time]int + + onFire mtime.Time + wantTimers []element + }{{ + name: "noTimers", + insert: []fireElement{}, + + wantHolds: map[mtime.Time]int{}, + onFire: fireTime1, + wantTimers: nil, + }, { + name: "singleTimer-singleKey", + insert: []fireElement{fireElem(fireTime1, userKey1, eventTime1, holdTime1)}, + + wantHolds: map[mtime.Time]int{ + holdTime1: 1, + }, + onFire: fireTime1, + wantTimers: []element{elem(userKey1, eventTime1, holdTime1)}, + }, { + name: "singleTimer-multipleKeys", + insert: []fireElement{ + fireElem(fireTime1, userKey1, eventTime1, holdTime1), + fireElem(fireTime1, userKey2, eventTime1, holdTime1), + fireElem(fireTime1, userKey3, eventTime1, holdTime1), + }, + wantHolds: map[mtime.Time]int{ + holdTime1: 3, + }, + onFire: fireTime1, + + wantTimers: []element{ + elem(userKey1, eventTime1, holdTime1), + elem(userKey2, eventTime1, holdTime1), + elem(userKey3, eventTime1, holdTime1), + }, + }, { + name: "multipleTimers-holdsChange", + insert: []fireElement{ + fireElem(fireTime1, userKey1, eventTime1, holdTime1), + fireElem(fireTime1, userKey1, eventTime2, holdTime2), + fireElem(fireTime1, userKey1, eventTime3, holdTime3), + }, + wantHolds: map[mtime.Time]int{ + holdTime3: 1, + }, + onFire: fireTime3, + wantTimers: []element{ + elem(userKey1, eventTime3, holdTime3), + }, + }, { + name: "multipleTimers-firesChange-nofire", + insert: []fireElement{ + fireElem(fireTime1, userKey1, eventTime1, holdTime1), + fireElem(fireTime2, userKey1, eventTime2, holdTime2), + fireElem(fireTime3, userKey1, eventTime3, holdTime3), + }, + wantHolds: map[mtime.Time]int{ + holdTime3: 1, + }, + onFire: fireTime1, + wantTimers: nil, // Nothing should fire. + }, { + name: "multipleTimerKeys-firesAll", + insert: []fireElement{ + fireElem(fireTime1, userKey1, eventTime1, holdTime1), + fireElem(fireTime2, userKey2, eventTime2, holdTime2), + fireElem(fireTime3, userKey3, eventTime3, holdTime3), + }, + wantHolds: map[mtime.Time]int{ + holdTime1: 1, + holdTime2: 1, + holdTime3: 1, + }, + onFire: fireTime3, + wantTimers: []element{ + elem(userKey1, eventTime1, holdTime1), + elem(userKey2, eventTime2, holdTime2), + elem(userKey3, eventTime3, holdTime3), + }, + }, { + name: "multipleTimerKeys-firesTwo", + insert: []fireElement{ + fireElem(fireTime3, userKey3, eventTime3, holdTime3), + fireElem(fireTime2, userKey2, eventTime2, holdTime2), + fireElem(fireTime1, userKey1, eventTime1, holdTime1), + }, + wantHolds: map[mtime.Time]int{ + holdTime1: 1, + holdTime2: 1, + holdTime3: 1, + }, + onFire: fireTime2, + wantTimers: []element{ + elem(userKey1, eventTime1, holdTime1), + elem(userKey2, eventTime2, holdTime2), + }, + }, { + name: "multipleTimerKeys-multiple-replacements", + insert: []fireElement{ + fireElem(fireTime3, userKey3, eventTime3, holdTime3), + fireElem(fireTime2, userKey2, eventTime2, holdTime2), + fireElem(fireTime1, userKey1, eventTime1, holdTime1), + + fireElem(fireTime1, userKey3, eventTime3, holdTime1), // last userKey3 - present at fireTime2 + fireElem(fireTime2, userKey2, eventTime1, holdTime1), + fireElem(fireTime3, userKey1, eventTime1, holdTime1), + + fireElem(fireTime3, userKey1, eventTime3, holdTime3), + fireElem(fireTime2, userKey2, eventTime2, holdTime2), + fireElem(fireTime1, userKey1, eventTime2, holdTime2), + + fireElem(fireTime2, userKey2, eventTime1, holdTime3), + fireElem(fireTime3, userKey2, eventTime2, holdTime2), // last userkey2 - not present at fireTime2 + fireElem(fireTime1, userKey1, eventTime1, holdTime3), // last userkey1 - present at fireTime2 + }, + wantHolds: map[mtime.Time]int{ + holdTime1: 1, + holdTime2: 1, + holdTime3: 1, + }, + onFire: fireTime2, + wantTimers: []element{ + elem(userKey3, eventTime3, holdTime1), + elem(userKey1, eventTime1, holdTime3), + }, + }, { + name: "multipleTimerTags-firesAll", + insert: []fireElement{ + {firing: fireTime1, timer: elemTagWin(userKey1, eventTime1, holdTime1, "tag1", glo)}, + {firing: fireTime2, timer: elemTagWin(userKey1, eventTime1, holdTime1, "tag2", glo)}, + {firing: fireTime3, timer: elemTagWin(userKey1, eventTime1, holdTime1, "tag3", glo)}, + + // Validate replacements on tags + {firing: fireTime1, timer: elemTagWin(userKey1, eventTime2, holdTime1, "tag1", glo)}, + {firing: fireTime2, timer: elemTagWin(userKey1, eventTime2, holdTime1, "tag2", glo)}, + {firing: fireTime3, timer: elemTagWin(userKey1, eventTime2, holdTime1, "tag3", glo)}, + }, + wantHolds: map[mtime.Time]int{ + holdTime1: 3, + }, + onFire: fireTime3, + wantTimers: []element{ + elemTagWin(userKey1, eventTime2, holdTime1, "tag1", glo), + elemTagWin(userKey1, eventTime2, holdTime1, "tag2", glo), + elemTagWin(userKey1, eventTime2, holdTime1, "tag3", glo), + }, + }, { + name: "multipleTimerTags-firesAll", + insert: []fireElement{ + {firing: fireTime1, timer: elemTagWin(userKey1, eventTime1, holdTime1, "", iw1)}, + {firing: fireTime2, timer: elemTagWin(userKey1, eventTime1, holdTime1, "", iw2)}, + {firing: fireTime3, timer: elemTagWin(userKey1, eventTime1, holdTime1, "", iw3)}, + + // Validate replacements on windows + {firing: fireTime1, timer: elemTagWin(userKey1, eventTime2, holdTime1, "", iw1)}, + {firing: fireTime2, timer: elemTagWin(userKey1, eventTime2, holdTime1, "", iw2)}, + {firing: fireTime3, timer: elemTagWin(userKey1, eventTime2, holdTime1, "", iw3)}, + }, + wantHolds: map[mtime.Time]int{ + holdTime1: 3, + }, + onFire: fireTime3, + wantTimers: []element{ + elemTagWin(userKey1, eventTime2, holdTime1, "", iw1), + elemTagWin(userKey1, eventTime2, holdTime1, "", iw2), + elemTagWin(userKey1, eventTime2, holdTime1, "", iw3), + }, + }, + } + + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + th := newTimerHandler() + + holdChanges := map[mtime.Time]int{} + + for _, ft := range test.insert { + if ft.timer.IsData() { + t.Fatalf("generated bad timer: %+v", ft) + } + th.Persist(ft.firing, ft.timer, holdChanges) + } + + if d := cmp.Diff(test.wantHolds, holdChanges, cmp.AllowUnexported(element{}, fireElement{})); d != "" { + t.Errorf("Persist(): diff (-want,+got):\n%v", d) + } + fired := th.FireAt(test.onFire) + + lessElement := func(a, b element) int { + if a.timestamp < b.timestamp { + return -1 + } else if a.timestamp > b.timestamp { + return 1 + } + if a.holdTimestamp < b.holdTimestamp { + return -1 + } else if a.holdTimestamp > b.holdTimestamp { + return 1 + } + if string(a.keyBytes) < string(b.keyBytes) { + return -1 + } else if string(a.keyBytes) > string(b.keyBytes) { + return 1 + } + if a.tag < b.tag { + return -1 + } else if a.tag > b.tag { + return 1 + } + if a.window.MaxTimestamp() < b.window.MaxTimestamp() { + return -1 + } else if a.window.MaxTimestamp() > b.window.MaxTimestamp() { + return 1 + } + return 0 + } + slices.SortFunc(fired, lessElement) + slices.SortFunc(test.wantTimers, lessElement) + if d := cmp.Diff(test.wantTimers, fired, cmp.AllowUnexported(element{})); d != "" { + t.Errorf("FireAt(%v): diff (-want,+got):\n%v", test.onFire, d) + } + }) + } +} diff --git a/sdks/go/pkg/beam/runners/prism/internal/worker/worker.go b/sdks/go/pkg/beam/runners/prism/internal/worker/worker.go index 155f59a1487b..47fc2cccfc54 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/worker/worker.go +++ b/sdks/go/pkg/beam/runners/prism/internal/worker/worker.go @@ -354,7 +354,7 @@ func (wk *W) Data(data fnpb.BeamFnData_DataServer) error { for _, d := range resp.GetData() { cr, ok := wk.activeInstructions[d.GetInstructionId()] if !ok { - slog.Info("data.Recv for unknown bundle", "response", resp) + slog.Info("data.Recv data for unknown bundle", "response", resp) continue } // Received data is always for an active ProcessBundle instruction @@ -373,7 +373,7 @@ func (wk *W) Data(data fnpb.BeamFnData_DataServer) error { for _, t := range resp.GetTimers() { cr, ok := wk.activeInstructions[t.GetInstructionId()] if !ok { - slog.Info("data.Recv for unknown bundle", "response", resp) + slog.Info("data.Recv timers for unknown bundle", "response", resp) continue } // Received data is always for an active ProcessBundle instruction diff --git a/sdks/go/test/integration/primitives/state.go b/sdks/go/test/integration/primitives/state.go index acf1bf8fa665..cb3be341ac5e 100644 --- a/sdks/go/test/integration/primitives/state.go +++ b/sdks/go/test/integration/primitives/state.go @@ -79,7 +79,7 @@ func (f *valueStateFn) ProcessElement(s state.Provider, w string, c int) string return fmt.Sprintf("%s: %v, %s", w, i, j) } -func pairWithOne(w string, emit func(string, int)) { +func pairWithOne(w beam.T, emit func(beam.T, int)) { emit(w, 1) } @@ -520,3 +520,86 @@ func SetStateParDoClear(s beam.Scope) { counts := beam.ParDo(s, &setStateClearFn{State1: state.MakeSetState[string]("key1")}, keyed) passert.Equals(s, counts, "apple: [apple]", "pear: [pear]", "peach: [peach]", "apple: [apple1 apple2 apple3]", "apple: []", "pear: [pear1 pear2 pear3]") } + +// genValueStateFn uses Go generics to dictate the type of State2. +type genValueStateFn[T any] struct { + State1 state.Value[int] + State2 state.Value[T] +} + +func (f *genValueStateFn[T]) ProcessElement(s state.Provider, c T, w int, emit func(string)) { + i, ok, err := f.State1.Read(s) + if err != nil { + panic(err) + } + if !ok { + i = 1 + } + err = f.State1.Write(s, i+1) + if err != nil { + panic(err) + } + + j, ok, err := f.State2.Read(s) + if err != nil { + panic(err) + } + + prnt := fmt.Sprintf("%v: %v, %s", c, i, j) + fmt.Println("XXXXX", ok, prnt) + // Only emit if we have written out before, but include the latest count. + if ok { + emit(prnt) + } + err = f.State2.Write(s, c) + if err != nil { + panic(err) + } +} + +func regGenValueState[T any]() { + register.DoFn4x0[state.Provider, T, int, func(string)](&genValueStateFn[T]{}) +} + +// fruit is a vanilla struct that beam will row encode. +type fruit struct { + Name string +} + +func (f *fruit) String() string { + return f.Name +} + +func init() { + regGenValueState[[]byte]() // validate byte slice usage. + regGenValueState[fruit]() // validate row struct default encoding usage. +} + +func ValueStateParDo_Bytes(s beam.Scope) { + apple := []byte("apple") + pear := []byte("pear") + + peach := []byte("peach") + + in := beam.Create(s, apple, pear, peach, apple, apple, pear) + keyed := beam.ParDo(s, pairWithOne, in) + counts := beam.ParDo(s, &genValueStateFn[[]byte]{}, keyed) + passert.Equals(s, counts, + fmt.Sprintf("%v: 2, %s", apple, string(apple)), + fmt.Sprintf("%v: 3, %s", apple, string(apple)), + fmt.Sprintf("%v: 2, %s", pear, string(pear))) +} + +func ValueStateParDo_Row(s beam.Scope) { + apple := fruit{"apple"} + pear := fruit{"pear"} + peach := fruit{"peach"} + + in := beam.Create(s, apple, pear, peach, apple, apple, pear) + keyed := beam.ParDo(s, pairWithOne, in) + counts := beam.ParDo(s, &genValueStateFn[fruit]{}, keyed) + passert.Equals(s, counts, + fmt.Sprintf("%v: 2, %s", apple, apple), + fmt.Sprintf("%v: 3, %s", apple, apple), + fmt.Sprintf("%v: 2, %s", pear, pear)) +} diff --git a/sdks/go/test/integration/primitives/state_test.go b/sdks/go/test/integration/primitives/state_test.go index 79cb8c1839fc..985dfa785506 100644 --- a/sdks/go/test/integration/primitives/state_test.go +++ b/sdks/go/test/integration/primitives/state_test.go @@ -27,6 +27,11 @@ func TestValueState(t *testing.T) { ptest.BuildAndRun(t, ValueStateParDo) } +func TestValueState_Bytes(t *testing.T) { + integration.CheckFilters(t) + ptest.BuildAndRun(t, ValueStateParDo_Bytes) +} + func TestValueStateWindowed(t *testing.T) { integration.CheckFilters(t) ptest.BuildAndRun(t, ValueStateParDoWindowed) diff --git a/sdks/go/test/integration/primitives/timers.go b/sdks/go/test/integration/primitives/timers.go index 9041870499a2..5d647dea6609 100644 --- a/sdks/go/test/integration/primitives/timers.go +++ b/sdks/go/test/integration/primitives/timers.go @@ -170,8 +170,18 @@ type processingTimeFn struct { } func (fn *processingTimeFn) ProcessElement(sp state.Provider, tp timers.Provider, key string, value int, emit func(string, int)) { + // Sets a processing time callback to occur. fn.Callback.Set(tp, time.Now().Add(9*time.Second)) - fn.MyValue.Write(sp, 0) + + // Only write to the state if we haven't done so already. + // Writing blind would reset the state, and cause duplicated outputs. + _, ok, err := fn.MyValue.Read(sp) + if err != nil { + panic(err) + } + if !ok { + fn.MyValue.Write(sp, 0) + } } func (fn *processingTimeFn) OnTimer(ctx context.Context, ts beam.EventTime, sp state.Provider, tp timers.Provider, key string, timer timers.Context, emit func(string, int)) { @@ -219,8 +229,8 @@ func timersProcessingTimePipelineBuilder(makeImp func(s beam.Scope) beam.PCollec offset := 5000 timerOutput := 4093 - numKeys := 1 - numDuplicateTimers := 4 + numKeys := 40 + numDuplicateTimers := 15 for key := 0; key < numKeys; key++ { k := strconv.Itoa(key) @@ -239,7 +249,7 @@ func timersProcessingTimePipelineBuilder(makeImp func(s beam.Scope) beam.PCollec Offset: offset, TimerOutput: timerOutput, Callback: timers.InProcessingTime("Callback"), - MyValue: state.MakeValueState[int]("MyKey"), + MyValue: state.MakeValueState[int]("MyValue"), Cap: numDuplicateTimers, // Syncs the cycles to the number of duplicate keyed inputs. }, keyed) // We GroupByKey here so input to passert is blocked until teststream advances time to Infinity. From ad599c5b0057cd33b0830ca41f504fc93085c5a1 Mon Sep 17 00:00:00 2001 From: lostluck <13907733+lostluck@users.noreply.github.com> Date: Thu, 16 May 2024 10:55:08 -0700 Subject: [PATCH 14/21] Some cleanup --- .../prism/internal/engine/elementmanager.go | 151 ++++-------------- .../prism/internal/engine/engine_test.go | 8 +- sdks/go/test/integration/primitives/state.go | 2 +- sdks/go/test/integration/primitives/timers.go | 25 ++- .../integration/primitives/timers_test.go | 5 + 5 files changed, 61 insertions(+), 130 deletions(-) diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go b/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go index a4f08cd571da..0865a8cf2505 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go @@ -343,7 +343,6 @@ func (em *ElementManager) Bundles(ctx context.Context, nextBundID func() string) for stageID := range advanced { ss := em.stages[stageID] watermark, ready, ptimeEventsReady := ss.bundleReady(em, emNow) - //fmt.Println("ptimeEventsReady ", ss.ID, ptimeEventsReady, "now", emNow, "watermark", watermark, "watermarkReady", ready) if ready { bundleID, ok, reschedule := ss.startEventTimeBundle(watermark, nextBundID) // Handle the reschedule even when there's no bundle. @@ -365,9 +364,6 @@ func (em *ElementManager) Bundles(ctx context.Context, nextBundID func() string) } } if ptimeEventsReady { - // fmt.Println("what's firing next?", ss.ID, "now", emNow, ss.processingTimeTimers.toFire) - // Queue up elements! - // TODO impose processing strategy limits. bundleID, ok, reschedule := ss.startProcessingTimeBundle(em, emNow, nextBundID) // Handle the reschedule even when there's no bundle. if reschedule { @@ -420,6 +416,11 @@ func (em *ElementManager) checkForQuiescence(advanced set[string]) { ) return } + if em.testStreamHandler == nil && len(em.processTimeEvents.events) > 0 { + // If there's no test stream involved, and processing time events exist, then + // it's only a matter of time. + return + } // The job has quiesced! // There are no further incoming watermark changes, see if there are test stream events for this job. @@ -767,12 +768,10 @@ func (em *ElementManager) PersistBundle(rb RunBundle, col2Coders map[string]PCol stage.watermarkHolds.Drop(h, -c) } } - //fmt.Println("QQQQQ persistbundle", rb.StageID, rb.BundleID, "drop holds", stage.inprogressHoldsByBundle[rb.BundleID]) for hold, v := range stage.inprogressHoldsByBundle[rb.BundleID] { stage.watermarkHolds.Drop(hold, v) } delete(stage.inprogressHoldsByBundle, rb.BundleID) - //fmt.Println("QQQQQ persistbundle", rb.StageID, rb.BundleID, "post drop holds", stage.watermarkHolds.counts) // If there are estimated output watermarks, set the estimated // output watermark for the stage. @@ -869,7 +868,6 @@ func (em *ElementManager) triageTimers(d TentativeData, inputInfo PColInfo, stag for _, v := range pendingProcessingTimers { count += stage.processingTimeTimers.Persist(v.firing, v.timer, changedHolds) } - // fmt.Println("PERSISTED ", pendingProcessingTimers) em.addPending(count) stage.mu.Unlock() } @@ -914,7 +912,6 @@ func (em *ElementManager) addRefreshAndClearBundle(stageID, bundID string, ptRef defer em.refreshCond.L.Unlock() delete(em.inprogressBundles, bundID) em.watermarkRefreshes.insert(stageID) - //fmt.Println("addRefreshAndClearBundle stage", stageID, "due to", bundID, ptRefreshes, "inEmAlready", em.processTimeEvents.events) for t := range ptRefreshes { em.processTimeEvents.Schedule(t, stageID) } @@ -1000,7 +997,7 @@ type stageState struct { sideInputs map[LinkID]map[typex.Window][][]byte // side input data for this stage, from {tid, inputID} -> window // Fields for stateful stages which need to be per key. - pendingByKeys keyedElements // pending input elements by Key, if stateful. + pendingByKeys map[string]*dataAndTimers // pending input elements by Key, if stateful. inprogressKeys set[string] // all keys that are assigned to bundles. inprogressKeysByBundle map[string]set[string] // bundle to key assignments. state map[LinkID]map[typex.Window]map[string]StateData // state data for this stage, from {tid, stateID} -> window -> userKey @@ -1064,20 +1061,38 @@ func (ss *stageState) AddPending(newPending []element) int { ss.mu.Lock() defer ss.mu.Unlock() if ss.stateful { - changedHolds := map[mtime.Time]int{} if ss.pendingByKeys == nil { ss.pendingByKeys = map[string]*dataAndTimers{} } count := 0 for _, e := range newPending { + count++ if len(e.keyBytes) == 0 { panic(fmt.Sprintf("zero length key: %v %v", ss.ID, ss.inputID)) } - count += ss.pendingByKeys.AddElement(changedHolds, e) - } - // fmt.Println("QQQQQ AddPending changedHolds", changedHolds) - for hold, v := range changedHolds { - ss.watermarkHolds.Add(hold, v) + dnt, ok := ss.pendingByKeys[string(e.keyBytes)] + if !ok { + dnt = &dataAndTimers{ + timers: map[timerKey]timerTimes{}, + } + ss.pendingByKeys[string(e.keyBytes)] = dnt + } + dnt.elements.Push(e) + + if e.IsTimer() { + if lastSet, ok := dnt.timers[timerKey{family: e.family, tag: e.tag, window: e.window}]; ok { + // existing timer! + // don't increase the count this time, as "this" timer is already pending. + count-- + // clear out the existing hold for accounting purposes. + ss.watermarkHolds.Drop(lastSet.hold, 1) + } + // Update the last set time on the timer. + dnt.timers[timerKey{family: e.family, tag: e.tag, window: e.window}] = timerTimes{firing: e.timestamp, hold: e.holdTimestamp} + + // Mark the hold in the heap. + ss.watermarkHolds.Add(e.holdTimestamp, 1) + } } return count } @@ -1087,94 +1102,6 @@ func (ss *stageState) AddPending(newPending []element) int { return len(newPending) } -// keyedElements stores elements bucketed by their key. This simplifies ensuring that -// in a stateful transform, a given key is only being processed by a single bundle at a time. -type keyedElements map[string]*dataAndTimers - -func (ke keyedElements) AddElement(holds map[mtime.Time]int, e element) int { - dnt, ok := ke[string(e.keyBytes)] - if !ok { - dnt = &dataAndTimers{ - timers: map[timerKey]timerTimes{}, - } - ke[string(e.keyBytes)] = dnt - } - return dnt.AddElement(holds, e) -} - -func (ke keyedElements) Merge(holds *holdTracker, o keyedElements) int { - var count int - for k, ov := range o { - v, ok := ke[k] - if !ok { - ke[k] = ov - continue - } - count += v.Merge(holds, ov) - ke[k] = v - } - return count -} - -func (ke keyedElements) String() string { - var b strings.Builder - b.WriteString("keyedElems[") - for k, v := range ke { - b.WriteRune('\n') - b.WriteRune('\t') - b.WriteString(k) - b.WriteRune(':') - b.WriteString(fmt.Sprintf("%+v", v)) - b.WriteRune(',') - } - b.WriteString("]") - return b.String() -} - -// AddElement pushes the element onto the queue and handles timer uniqueness for the key, including -// addressing any watermark holds of the timer. -// -// Returns adjustments to the pending count. -func (dnt *dataAndTimers) AddElement(holds map[mtime.Time]int, e element) int { - dnt.elements.Push(e) - count := 1 - if e.IsTimer() { - if lastSet, ok := dnt.timers[timerKey{family: e.family, tag: e.tag, window: e.window}]; ok { - // existing timer! - // don't increase the count this time, as "this" timer is already pending. - count-- - // clear out the existing hold for accounting purposes. - holds[lastSet.hold] -= 1 - // fmt.Println("ZZZZZZ dropping old hold", lastSet.hold, "for", e) - } - // Update the last set time on the timer. - dnt.timers[timerKey{family: e.family, tag: e.tag, window: e.window}] = timerTimes{firing: e.timestamp, hold: e.holdTimestamp} - - // Mark the hold in the heap. - holds[e.holdTimestamp] += 1 - //fmt.Println("ZZZZZZ addHold for ", e) - } - return count -} - -func (dnt *dataAndTimers) Merge(holds *holdTracker, o *dataAndTimers) int { - count := len(o.elements) - dnt.elements = append(dnt.elements, o.elements...) - heap.Init(&dnt.elements) - for tk, ot := range o.timers { - ct, ok := dnt.timers[tk] - if !ok { - dnt.timers[tk] = ot - continue - } - count-- // subtract the timer from the count, since we aren't adding any. - holds.Drop(ct.hold, 1) - // For timers, keep any existing holds, or last sets, since there's only - // one timer per key anyway. Extra timers will be garbage collected in later processing. - } - return count -} - // AddPendingSide adds elements to be consumed as side inputs. func (ss *stageState) AddPendingSide(newPending []element, tID, inputID string) { ss.mu.Lock() @@ -1350,7 +1277,6 @@ keysPerBundle: } bundID := ss.makeInProgressBundle(genBundID, toProcess, minTs, newKeys, holdsInBundle) - //fmt.Println("QQQQ eventTimeBundle stage", ss.ID, "bundle", bundID, "holds", holdsInBundle, "newKeys", newKeys, "holdsInStage", ss.watermarkHolds.counts, "state", ss.state, "elements", toProcess) return bundID, true, stillSchedulable } @@ -1372,17 +1298,14 @@ func (ss *stageState) startProcessingTimeBundle(em *ElementManager, emNow mtime. var notYet []fireElement nextTime := ss.processingTimeTimers.Peek() - blockedOnKey := set[string]{} keyCounts := map[string]int{} newKeys := set[string]{} for nextTime <= emNow { - // fmt.Println("top of the loop", nextTime, "now", emNow, ss.processingTimeTimers.order, ss.processingTimeTimers.toFire) elems := ss.processingTimeTimers.FireAt(nextTime) for _, e := range elems { // Check if we're already executing this timer's key. if ss.inprogressKeys.present(string(e.keyBytes)) { - blockedOnKey.insert(string(e.keyBytes)) notYet = append(notYet, fireElement{firing: nextTime, timer: e}) continue } @@ -1406,7 +1329,6 @@ func (ss *stageState) startProcessingTimeBundle(em *ElementManager, emNow mtime. // We're going to process this timer! toProcess = append(toProcess, e) - // fmt.Println("startProcessingTimeBundle: nextTime", nextTime, "emNow", emNow, "timer", e) } nextTime = ss.processingTimeTimers.Peek() @@ -1415,16 +1337,13 @@ func (ss *stageState) startProcessingTimeBundle(em *ElementManager, emNow mtime. break } } - // fmt.Println("out of the loop", nextTime, "now", emNow, ss.processingTimeTimers.order, ss.processingTimeTimers.toFire) - // Reschedule - // fmt.Println("QQQQQ NOT YET; now", emNow, "dropping", notYet, "processing", toProcess, "curCounts", ss.watermarkHolds.counts) + // Reschedule unfired timers. notYetHolds := map[mtime.Time]int{} for _, v := range notYet { ss.processingTimeTimers.Persist(v.firing, v.timer, notYetHolds) em.processTimeEvents.Schedule(v.firing, ss.ID) } - // fmt.Println("post-reschedule", nextTime, "now", emNow, "order", ss.processingTimeTimers.order, "toFire", ss.processingTimeTimers.toFire, "emProcTime", em.processTimeEvents) // Add a refresh if there are still processing time events to process. stillSchedulable := (nextTime < emNow && nextTime != mtime.MaxTimestamp || len(notYet) > 0) @@ -1434,7 +1353,6 @@ func (ss *stageState) startProcessingTimeBundle(em *ElementManager, emNow mtime. return "", false, stillSchedulable } bundID := ss.makeInProgressBundle(genBundID, toProcess, minTs, newKeys, holdsInBundle) - // fmt.Println("QQQQQ processingTimeBundle", ss.ID, bundID, toProcess, holdsInBundle, "rejected holds", notYetHolds, "stillSchedulable", stillSchedulable) return bundID, true, stillSchedulable } @@ -1465,7 +1383,6 @@ func (ss *stageState) makeInProgressBundle(genBundID func() string, toProcess [] ss.inprogressKeysByBundle[bundID] = newKeys ss.inprogressKeys.merge(newKeys) ss.inprogressHoldsByBundle[bundID] = holdsInBundle - // fmt.Println("QQQQQ makeInProgressBundle", ss.ID, bundID, "add holds", holdsInBundle) return bundID } @@ -1601,12 +1518,6 @@ func (ss *stageState) bundleReady(em *ElementManager, emNow mtime.Time) (mtime.T ss.mu.Lock() defer ss.mu.Unlock() - // TODO Move handling ProcessingTime bundle fireing *HERE* to avoid nonsense against normal eventtime handling. - // Specifically, have the override here (is data ready at this time?) - // And in startBundle pull out that data. - // Or I add a new ready based on processing time signal that this returns, that is independently checked - // for sending. - ptimeEventsReady := ss.processingTimeTimers.Peek() <= emNow || emNow == mtime.MaxTimestamp // If the upstream watermark and the input watermark are the same, diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/engine_test.go b/sdks/go/pkg/beam/runners/prism/internal/engine/engine_test.go index a8d683127f6c..42f682c4784d 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/engine_test.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/engine_test.go @@ -233,7 +233,7 @@ func TestProcessingTime(t *testing.T) { tests := []struct { pipeline func(s beam.Scope) }{ - {pipeline: primitives.TimersProcessingTimeTestStream_Infinity}, + //{pipeline: primitives.TimersProcessingTimeTestStream_Infinity}, {pipeline: primitives.TimersProcessingTime_Bounded}, } @@ -241,10 +241,10 @@ func TestProcessingTime(t *testing.T) { name string OneElementPerKey, OneKeyPerBundle bool }{ - {"Greedy", false, false}, - {"AllElementsPerKey", false, true}, + // {"Greedy", false, false}, + // {"AllElementsPerKey", false, true}, {"OneElementPerKey", true, false}, - {"OneElementPerBundle", true, true}, + // {"OneElementPerBundle", true, true}, } for _, config := range configs { for _, test := range tests { diff --git a/sdks/go/test/integration/primitives/state.go b/sdks/go/test/integration/primitives/state.go index cb3be341ac5e..e8fc89ec4c10 100644 --- a/sdks/go/test/integration/primitives/state.go +++ b/sdks/go/test/integration/primitives/state.go @@ -545,7 +545,7 @@ func (f *genValueStateFn[T]) ProcessElement(s state.Provider, c T, w int, emit f panic(err) } - prnt := fmt.Sprintf("%v: %v, %s", c, i, j) + prnt := fmt.Sprintf("%v: %v, %v", c, i, j) fmt.Println("XXXXX", ok, prnt) // Only emit if we have written out before, but include the latest count. if ok { diff --git a/sdks/go/test/integration/primitives/timers.go b/sdks/go/test/integration/primitives/timers.go index 5d647dea6609..80e4dbb530d9 100644 --- a/sdks/go/test/integration/primitives/timers.go +++ b/sdks/go/test/integration/primitives/timers.go @@ -17,6 +17,7 @@ package primitives import ( "context" + "fmt" "strconv" "time" @@ -161,8 +162,9 @@ func init() { } type processingTimeFn struct { - Callback timers.ProcessingTime - MyValue state.Value[int] + Callback timers.ProcessingTime + MyValue state.Value[int] + Emissions state.Value[int] Offset int TimerOutput int @@ -180,7 +182,9 @@ func (fn *processingTimeFn) ProcessElement(sp state.Provider, tp timers.Provider panic(err) } if !ok { - fn.MyValue.Write(sp, 0) + if err := fn.MyValue.Write(sp, 0); err != nil { + panic(err) + } } } @@ -198,9 +202,20 @@ func (fn *processingTimeFn) OnTimer(ctx context.Context, ts beam.EventTime, sp s } emit(key, read) if read < fn.Cap-1 { - fn.MyValue.Write(sp, read+1) + if err := fn.MyValue.Write(sp, read+1); err != nil { + panic(err) + } fn.Callback.Set(tp, time.Now().Add(9*time.Second)) } + if num, _, err := fn.Emissions.Read(sp); err != nil { + panic(err) + } else if num == fn.Cap { + panic(fmt.Sprintf("cap reached! This shouldn't be possible. key %v, num: %v, cap %v read %v", key, num, fn.Cap, read)) + } else { + if err := fn.Emissions.Write(sp, num+1); err != nil { + panic(err) + } + } default: panic("unexpected timer tag: " + timer.Family + " tag:" + timer.Tag + " want: \"\", for key:" + key) } @@ -229,7 +244,7 @@ func timersProcessingTimePipelineBuilder(makeImp func(s beam.Scope) beam.PCollec offset := 5000 timerOutput := 4093 - numKeys := 40 + numKeys := 100 numDuplicateTimers := 15 for key := 0; key < numKeys; key++ { diff --git a/sdks/go/test/integration/primitives/timers_test.go b/sdks/go/test/integration/primitives/timers_test.go index 3e84c0edcbe9..7e62e9da6920 100644 --- a/sdks/go/test/integration/primitives/timers_test.go +++ b/sdks/go/test/integration/primitives/timers_test.go @@ -36,3 +36,8 @@ func TestTimers_ProcessingTime_Infinity(t *testing.T) { integration.CheckFilters(t) ptest.BuildAndRun(t, TimersProcessingTimeTestStream_Infinity) } + +func TestTimers_ProcessingTime_Bounded(t *testing.T) { + integration.CheckFilters(t) + ptest.BuildAndRun(t, TimersProcessingTime_Bounded) +} From ff911c6c0e68e7746774de3f202bf725ff4c56c1 Mon Sep 17 00:00:00 2001 From: lostluck <13907733+lostluck@users.noreply.github.com> Date: Thu, 16 May 2024 11:34:41 -0700 Subject: [PATCH 15/21] try to get github suite to pass #1 --- sdks/go/pkg/beam/runners/prism/internal/engine/holds.go | 7 +------ sdks/go/test/integration/primitives/state.go | 7 +++---- 2 files changed, 4 insertions(+), 10 deletions(-) diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/holds.go b/sdks/go/pkg/beam/runners/prism/internal/engine/holds.go index d46b7f69bc4d..796ce1c2276a 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/holds.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/holds.go @@ -87,12 +87,7 @@ func (ht *holdTracker) Drop(hold mtime.Time, v int) { panic(fmt.Sprintf("prism error: negative watermark hold count %v for time %v", n, hold)) } delete(ht.counts, hold) - for i, h := range ht.heap { - if hold == h { - heap.Remove(&ht.heap, i) - break - } - } + ht.heap.Remove(hold) } // Add a hold a number of times to heap. If the hold time isn't already present in the heap, it is added. diff --git a/sdks/go/test/integration/primitives/state.go b/sdks/go/test/integration/primitives/state.go index e8fc89ec4c10..e8577ac3be4d 100644 --- a/sdks/go/test/integration/primitives/state.go +++ b/sdks/go/test/integration/primitives/state.go @@ -546,7 +546,6 @@ func (f *genValueStateFn[T]) ProcessElement(s state.Provider, c T, w int, emit f } prnt := fmt.Sprintf("%v: %v, %v", c, i, j) - fmt.Println("XXXXX", ok, prnt) // Only emit if we have written out before, but include the latest count. if ok { emit(prnt) @@ -585,9 +584,9 @@ func ValueStateParDo_Bytes(s beam.Scope) { keyed := beam.ParDo(s, pairWithOne, in) counts := beam.ParDo(s, &genValueStateFn[[]byte]{}, keyed) passert.Equals(s, counts, - fmt.Sprintf("%v: 2, %s", apple, string(apple)), - fmt.Sprintf("%v: 3, %s", apple, string(apple)), - fmt.Sprintf("%v: 2, %s", pear, string(pear))) + fmt.Sprintf("%v: 2, %v", apple, apple), + fmt.Sprintf("%v: 3, %v", apple, apple), + fmt.Sprintf("%v: 2, %v", pear, pear)) } func ValueStateParDo_Row(s beam.Scope) { From 4914a379ce5fd881cdd427af8c14832699492f42 Mon Sep 17 00:00:00 2001 From: lostluck <13907733+lostluck@users.noreply.github.com> Date: Thu, 16 May 2024 13:09:01 -0700 Subject: [PATCH 16/21] touch --- .github/trigger_files/beam_PostCommit_Go.json | 0 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 .github/trigger_files/beam_PostCommit_Go.json diff --git a/.github/trigger_files/beam_PostCommit_Go.json b/.github/trigger_files/beam_PostCommit_Go.json new file mode 100644 index 000000000000..e69de29bb2d1 From 1f433f97cbd6a91c6e75d0daca87d83f2ae1ce6b Mon Sep 17 00:00:00 2001 From: lostluck <13907733+lostluck@users.noreply.github.com> Date: Thu, 16 May 2024 13:36:39 -0700 Subject: [PATCH 17/21] reduce counts a bit, filter tests some. --- .../beam/runners/prism/internal/engine/engine_test.go | 9 +++++---- sdks/go/test/integration/integration.go | 6 +++--- sdks/go/test/integration/primitives/timers.go | 10 +++++++++- 3 files changed, 17 insertions(+), 8 deletions(-) diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/engine_test.go b/sdks/go/pkg/beam/runners/prism/internal/engine/engine_test.go index 42f682c4784d..e0bab8fa8e09 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/engine_test.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/engine_test.go @@ -233,18 +233,19 @@ func TestProcessingTime(t *testing.T) { tests := []struct { pipeline func(s beam.Scope) }{ - //{pipeline: primitives.TimersProcessingTimeTestStream_Infinity}, + {pipeline: primitives.TimersProcessingTimeTestStream_Infinity}, {pipeline: primitives.TimersProcessingTime_Bounded}, + {pipeline: primitives.TimersProcessingTime_Unbounded}, } configs := []struct { name string OneElementPerKey, OneKeyPerBundle bool }{ - // {"Greedy", false, false}, - // {"AllElementsPerKey", false, true}, + {"Greedy", false, false}, + {"AllElementsPerKey", false, true}, {"OneElementPerKey", true, false}, - // {"OneElementPerBundle", true, true}, + // {"OneElementPerBundle", true, true}, // Reveals flaky behavior } for _, config := range configs { for _, test := range tests { diff --git a/sdks/go/test/integration/integration.go b/sdks/go/test/integration/integration.go index 310d6b2b1797..94e18d543482 100644 --- a/sdks/go/test/integration/integration.go +++ b/sdks/go/test/integration/integration.go @@ -154,9 +154,6 @@ var prismFilters = []string{ "TestFhirIO.*", // OOMs currently only lead to heap dumps on Dataflow runner "TestOomParDo", - - // The prism runner does not support processing time timers https://github.com/apache/beam/issues/29772. - "TestTimers_ProcessingTime.*", } var flinkFilters = []string{ @@ -188,6 +185,7 @@ var flinkFilters = []string{ "TestTestStreamInt16Sequence", "TestTimers_EventTime_Unbounded", // (failure when comparing on side inputs (NPE on window lookup)) + "TestTimers_ProcessingTime.*", // Flink doesn't support processing time timers. } var samzaFilters = []string{ @@ -292,6 +290,8 @@ var dataflowFilters = []string{ "TestSpannerIO.*", // Dataflow does not drain jobs by itself. "TestDrain", + // Timers + "TestTimers_ProcessingTime_Infinity", // Uses test stream. } // CheckFilters checks if an integration test is filtered to be skipped, either diff --git a/sdks/go/test/integration/primitives/timers.go b/sdks/go/test/integration/primitives/timers.go index 80e4dbb530d9..40afe98234a7 100644 --- a/sdks/go/test/integration/primitives/timers.go +++ b/sdks/go/test/integration/primitives/timers.go @@ -244,7 +244,7 @@ func timersProcessingTimePipelineBuilder(makeImp func(s beam.Scope) beam.PCollec offset := 5000 timerOutput := 4093 - numKeys := 100 + numKeys := 40 numDuplicateTimers := 15 for key := 0; key < numKeys; key++ { @@ -293,3 +293,11 @@ func TimersProcessingTimeTestStream_Infinity(s beam.Scope) { func TimersProcessingTime_Bounded(s beam.Scope) { timersProcessingTimePipelineBuilder(beam.Impulse)(s) } + +// TimersProcessingTimeBounded validates processing time timers in an unbounded pipeline. +func TimersProcessingTime_Unbounded(s beam.Scope) { + timersProcessingTimePipelineBuilder(func(s beam.Scope) beam.PCollection { + now := time.Now() + return periodic.Impulse(s, now, now.Add(10*time.Second), 0, false) + })(s) +} From 47da23bf401aa9ad7b28264b4cfb85d0690271fb Mon Sep 17 00:00:00 2001 From: lostluck <13907733+lostluck@users.noreply.github.com> Date: Tue, 28 May 2024 11:23:44 -0700 Subject: [PATCH 18/21] Clean up unrelated state changes. Clean up comments somewhat. --- .../pkg/beam/core/runtime/exec/userstate.go | 10 +-- sdks/go/pkg/beam/pardo.go | 1 - .../prism/internal/engine/elementmanager.go | 5 +- .../prism/internal/engine/engine_test.go | 2 - .../prism/internal/engine/processingtime.go | 13 ++- sdks/go/test/integration/primitives/state.go | 84 +------------------ .../test/integration/primitives/state_test.go | 5 -- 7 files changed, 9 insertions(+), 111 deletions(-) diff --git a/sdks/go/pkg/beam/core/runtime/exec/userstate.go b/sdks/go/pkg/beam/core/runtime/exec/userstate.go index e4a0ef169887..f83aee4bf741 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/userstate.go +++ b/sdks/go/pkg/beam/core/runtime/exec/userstate.go @@ -57,10 +57,7 @@ func (s *stateProvider) ReadValueState(userStateID string) (any, []state.Transac return nil, nil, err } dec := MakeElementDecoder(coder.SkipW(s.codersByKey[userStateID])) - var buf bytes.Buffer - tee := io.TeeReader(rw, &buf) - resp, err := dec.Decode(tee) - // fmt.Println("CCCCCC - read", s.codersByKey[userStateID], userStateID, buf.Bytes()) + resp, err := dec.Decode(rw) if err != nil && err != io.EOF { return nil, nil, err } @@ -95,15 +92,12 @@ func (s *stateProvider) WriteValueState(val state.Transaction) error { if err != nil { return err } - var buf bytes.Buffer - mw := io.MultiWriter(ap, &buf) fv := FullValue{Elm: val.Val} enc := MakeElementEncoder(coder.SkipW(s.codersByKey[val.Key])) - err = enc.Encode(&fv, mw) + err = enc.Encode(&fv, ap) if err != nil { return err } - // fmt.Println("CCCCCC - write", s.codersByKey[val.Key], val.Key, buf.Bytes()) // Any transactions before a set don't matter s.transactionsByKey[val.Key] = []state.Transaction{val} diff --git a/sdks/go/pkg/beam/pardo.go b/sdks/go/pkg/beam/pardo.go index b8a1bc27a1c2..629ce329c9ba 100644 --- a/sdks/go/pkg/beam/pardo.go +++ b/sdks/go/pkg/beam/pardo.go @@ -100,7 +100,6 @@ func TryParDo(s Scope, dofn any, col PCollection, opts ...Option) ([]PCollection if ct := ps.CoderType(); ct != nil { sT := typex.New(ps.CoderType()) c, err := inferCoder(sT) - // fmt.Println("CCCCCC", c, graphx.UserStateCoderID(ps)) if err != nil { return nil, addParDoCtx(err, s) } diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go b/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go index 0865a8cf2505..8300088a3792 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go @@ -760,7 +760,7 @@ func (em *ElementManager) PersistBundle(rb RunBundle, col2Coders map[string]PCol } delete(stage.inprogressKeysByBundle, rb.BundleID) - // Add new holds. + // Adjust holds as needed. for h, c := range newHolds { if c > 0 { stage.watermarkHolds.Add(h, c) @@ -925,8 +925,6 @@ func (em *ElementManager) refreshWatermarks() set[string] { // Need to have at least one refresh signal. nextUpdates := set[string]{} refreshed := set[string]{} - // Use a single consolidated processing time during a given refresh for consistency. - var i int for stageID := range em.watermarkRefreshes { // clear out old one. @@ -936,7 +934,6 @@ func (em *ElementManager) refreshWatermarks() set[string] { refreshes := ss.updateWatermarks(em) nextUpdates.merge(refreshes) - // cap refreshes incrementally. if i < 10 { i++ diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/engine_test.go b/sdks/go/pkg/beam/runners/prism/internal/engine/engine_test.go index e0bab8fa8e09..b6e4412c3a83 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/engine_test.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/engine_test.go @@ -117,8 +117,6 @@ func TestStatefulStages(t *testing.T) { {pipeline: primitives.SetStateParDoClear}, {pipeline: primitives.TimersEventTimeBounded}, {pipeline: primitives.TimersEventTimeUnbounded}, - {pipeline: primitives.ValueStateParDo_Bytes}, - // {pipeline: primitives.ValueStateParDo_Row}, } configs := []struct { diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/processingtime.go b/sdks/go/pkg/beam/runners/prism/internal/engine/processingtime.go index 54f402b3b0c0..9e55c8c43b62 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/processingtime.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/processingtime.go @@ -20,25 +20,22 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/mtime" ) +// Notes on Processing Time handling: +// // ProcessingTime events (processingTime timers, process continuations, triggers) necessarily need to operate on a global queue. // However, PT timers are per key+family+tag, and may be overwritten by subsequent elements. // So, similarly to event time timers, we need to manage a "last set" queue, and to manage the holds. // This implies they should probably be handled by state, instead of globally. // In reality, it's probably going to be "both", a global PT event queue, and per stage state. // -// Also, in principle, timers would be how to implement the related features, so getting those right will simplify their handling. +// In principle, timers would be how to implement the related features, so getting those right will simplify their handling. // Test stream is already central, but doesn't set events, it controls their execution. -// But it would be easier to just *start* with ProcessContinuation handling, which may inform handling time. // -// Currently, the ElementManager doesn't retain any data itself, nor really should it. -// So it should not hold material data about what is being triggered. -// It should really only contain which stage state should be triggered via watermark refresh. -// Watermark refreshes should also receive a current ProcessingTime so they can then trigger -// their own events. +// The ElementManager doesn't retain any data itself, so it should not hold material data about what is being triggered. +// The ElementManager should only contain which stage state should be triggered when in a time domain. // // ProcessContinuations count as pending events, and must be drained accordingly before time expires. // -// // A stage may trigger on multiple ticks. // It's up to a stage to schedule additional work on those notices. diff --git a/sdks/go/test/integration/primitives/state.go b/sdks/go/test/integration/primitives/state.go index e8577ac3be4d..acf1bf8fa665 100644 --- a/sdks/go/test/integration/primitives/state.go +++ b/sdks/go/test/integration/primitives/state.go @@ -79,7 +79,7 @@ func (f *valueStateFn) ProcessElement(s state.Provider, w string, c int) string return fmt.Sprintf("%s: %v, %s", w, i, j) } -func pairWithOne(w beam.T, emit func(beam.T, int)) { +func pairWithOne(w string, emit func(string, int)) { emit(w, 1) } @@ -520,85 +520,3 @@ func SetStateParDoClear(s beam.Scope) { counts := beam.ParDo(s, &setStateClearFn{State1: state.MakeSetState[string]("key1")}, keyed) passert.Equals(s, counts, "apple: [apple]", "pear: [pear]", "peach: [peach]", "apple: [apple1 apple2 apple3]", "apple: []", "pear: [pear1 pear2 pear3]") } - -// genValueStateFn uses Go generics to dictate the type of State2. -type genValueStateFn[T any] struct { - State1 state.Value[int] - State2 state.Value[T] -} - -func (f *genValueStateFn[T]) ProcessElement(s state.Provider, c T, w int, emit func(string)) { - i, ok, err := f.State1.Read(s) - if err != nil { - panic(err) - } - if !ok { - i = 1 - } - err = f.State1.Write(s, i+1) - if err != nil { - panic(err) - } - - j, ok, err := f.State2.Read(s) - if err != nil { - panic(err) - } - - prnt := fmt.Sprintf("%v: %v, %v", c, i, j) - // Only emit if we have written out before, but include the latest count. - if ok { - emit(prnt) - } - err = f.State2.Write(s, c) - if err != nil { - panic(err) - } -} - -func regGenValueState[T any]() { - register.DoFn4x0[state.Provider, T, int, func(string)](&genValueStateFn[T]{}) -} - -// fruit is a vanilla struct that beam will row encode. -type fruit struct { - Name string -} - -func (f *fruit) String() string { - return f.Name -} - -func init() { - regGenValueState[[]byte]() // validate byte slice usage. - regGenValueState[fruit]() // validate row struct default encoding usage. -} - -func ValueStateParDo_Bytes(s beam.Scope) { - apple := []byte("apple") - pear := []byte("pear") - - peach := []byte("peach") - - in := beam.Create(s, apple, pear, peach, apple, apple, pear) - keyed := beam.ParDo(s, pairWithOne, in) - counts := beam.ParDo(s, &genValueStateFn[[]byte]{}, keyed) - passert.Equals(s, counts, - fmt.Sprintf("%v: 2, %v", apple, apple), - fmt.Sprintf("%v: 3, %v", apple, apple), - fmt.Sprintf("%v: 2, %v", pear, pear)) -} - -func ValueStateParDo_Row(s beam.Scope) { - apple := fruit{"apple"} - pear := fruit{"pear"} - peach := fruit{"peach"} - - in := beam.Create(s, apple, pear, peach, apple, apple, pear) - keyed := beam.ParDo(s, pairWithOne, in) - counts := beam.ParDo(s, &genValueStateFn[fruit]{}, keyed) - passert.Equals(s, counts, - fmt.Sprintf("%v: 2, %s", apple, apple), - fmt.Sprintf("%v: 3, %s", apple, apple), - fmt.Sprintf("%v: 2, %s", pear, pear)) -} diff --git a/sdks/go/test/integration/primitives/state_test.go b/sdks/go/test/integration/primitives/state_test.go index 985dfa785506..79cb8c1839fc 100644 --- a/sdks/go/test/integration/primitives/state_test.go +++ b/sdks/go/test/integration/primitives/state_test.go @@ -27,11 +27,6 @@ func TestValueState(t *testing.T) { ptest.BuildAndRun(t, ValueStateParDo) } -func TestValueState_Bytes(t *testing.T) { - integration.CheckFilters(t) - ptest.BuildAndRun(t, ValueStateParDo_Bytes) -} - func TestValueStateWindowed(t *testing.T) { integration.CheckFilters(t) ptest.BuildAndRun(t, ValueStateParDoWindowed) From e89dfc5dea836ce14ff78c17e418708db8bc60bd Mon Sep 17 00:00:00 2001 From: lostluck <13907733+lostluck@users.noreply.github.com> Date: Tue, 28 May 2024 20:02:45 -0700 Subject: [PATCH 19/21] Filter out dataflow incompatible test. --- sdks/go/test/integration/integration.go | 1 + 1 file changed, 1 insertion(+) diff --git a/sdks/go/test/integration/integration.go b/sdks/go/test/integration/integration.go index 94e18d543482..d8aee63718c1 100644 --- a/sdks/go/test/integration/integration.go +++ b/sdks/go/test/integration/integration.go @@ -292,6 +292,7 @@ var dataflowFilters = []string{ "TestDrain", // Timers "TestTimers_ProcessingTime_Infinity", // Uses test stream. + "TestTimers_ProcessingTime_Bounded", // Dataflow ignores processing time timers in batch. } // CheckFilters checks if an integration test is filtered to be skipped, either From bf0b48cc433b94b810581c31d691554416fc4a4a Mon Sep 17 00:00:00 2001 From: lostluck <13907733+lostluck@users.noreply.github.com> Date: Tue, 28 May 2024 20:08:59 -0700 Subject: [PATCH 20/21] Refine processing time event comment. --- .../pkg/beam/runners/prism/internal/engine/elementmanager.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go b/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go index 8300088a3792..76c60e810d40 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go @@ -317,8 +317,8 @@ func (em *ElementManager) Bundles(ctx context.Context, nextBundID func() string) ptRefreshed := em.processTimeEvents.AdvanceTo(emNow) em.watermarkRefreshes.merge(ptRefreshed) - // If there are no watermark refreshes available, we wait until there are. - for len(em.watermarkRefreshes)+len(ptRefreshed) == 0 { // TODO Add processing time event condition instead of piggybacking on watermarks? + // If there are no watermark refreshes or ready processing time events available, we wait until there are. + for len(em.watermarkRefreshes)+len(ptRefreshed) == 0 { // Check to see if we must exit select { case <-ctx.Done(): From 30c239d19865f34e0a2eb91b2f0c5a333992aac7 Mon Sep 17 00:00:00 2001 From: lostluck <13907733+lostluck@users.noreply.github.com> Date: Tue, 28 May 2024 20:13:26 -0700 Subject: [PATCH 21/21] Remove test touch. --- .github/trigger_files/beam_PostCommit_Go.json | 0 1 file changed, 0 insertions(+), 0 deletions(-) delete mode 100644 .github/trigger_files/beam_PostCommit_Go.json diff --git a/.github/trigger_files/beam_PostCommit_Go.json b/.github/trigger_files/beam_PostCommit_Go.json deleted file mode 100644 index e69de29bb2d1..000000000000