damondouglas commented on code in PR #29829: URL: https://github.com/apache/beam/pull/29829#discussion_r1437809849
########## sdks/go/test/integration/primitives/timers.go: ########## @@ -0,0 +1,133 @@ +// 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 primitives + +import ( + "context" + "strconv" + + "github.com/apache/beam/sdks/v2/go/pkg/beam" + "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" +) + +// Based on https://github.com/apache/beam/blob/master/runners/flink/src/test/java/org/apache/beam/runners/flink/PortableTimersExecutionTest.java + +func init() { + register.DoFn2x0[[]byte, func(string, int)](&inputFn[string, int]{}) + register.DoFn6x0[beam.Window, state.Provider, timers.Provider, string, int, func(kv[string, int])](&eventTimeFn{}) + register.Emitter2[string, int]() + register.Emitter1[kv[string, int]]() +} + +type kv[K, V any] struct { + Key K + Value V +} + +func kvfn[K, V any](k K, v V) kv[K, V] { + return kv[K, V]{k, v} +} + +type inputFn[K, V any] struct { + Inputs []kv[K, V] +} + +func (fn *inputFn[K, V]) ProcessElement(_ []byte, emit func(K, V)) { + for _, in := range fn.Inputs { + emit(in.Key, in.Value) + } +} + +type eventTimeFn struct { + Callback timers.EventTime + MyKey state.Value[string] + + Offset int + TimerOutput int +} + +func (fn *eventTimeFn) ProcessElement(w beam.Window, sp state.Provider, tp timers.Provider, key string, value int, emit func(kv[string, int])) { + fn.Callback.Set(tp, w.MaxTimestamp().ToTime()) + fn.MyKey.Write(sp, key) + emit(kvfn(key, value+fn.Offset)) +} + +func (fn *eventTimeFn) OnTimer(ctx context.Context, ts beam.EventTime, sp state.Provider, tp timers.Provider, key string, timer timers.Context, emit func(kv[string, int])) { + switch timer.Family { + case fn.Callback.Family: + switch timer.Tag { + case "": + read, ok, err := fn.MyKey.Read(sp) + if err != nil { + panic(err) + } + if !ok { + panic("State must be set.") + } + emit(kvfn(read, fn.TimerOutput)) + } + default: + if fn.Callback.Family != timer.Family || timer.Tag != "" { + panic("unexpected timer family: " + timer.Family + " tag:" + timer.Tag + " want: " + fn.Callback.Family) + } + } +} + +// TimersEventTime takes in an impulse transform and then validates +// event time timer execution. +// +// The impulse is provided outside to swap between a bounded impulse, and +// an unbounded one, because the Go SDK uses that to determine if a pipeline +// is "streaming" or not. This matters at least for executions on Dataflow. +// +// Regardless,the pipelines should pass. Review Comment: May we consider the following? (Blame the comment lines alignment on the GitHub comment editor). ``` // Validator is a func that validates a Pipeline. type Validator func(s beam.Scope) // TimersEventTimeValidator produces a Validator that validates whether an OnTimer callback is invoked. // It passert.EqualsList expected timestamps (as millisecond int) plus an expected offset. // // makeImp is a func that produces an impulse (either beam.Impulse or beam.PeriodicImpulse). The // purpose of makeImp is to provide TimersEventTimeValidator either a bounded impulse or an // unbounded one. Beam, in general, uses this impulse to determine if a pipeline is "streaming" or // "batch". Test coverage for streaming or batch matters for executions on certain runners such as // Dataflow. func TimersEventTimeValidator(makeImp func(s beam.Scope) beam.PCollection) Validator { } ``` ########## sdks/go/test/integration/primitives/timers_test.go: ########## @@ -0,0 +1,41 @@ +// 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 primitives + +import ( + "testing" + "time" + + "github.com/apache/beam/sdks/v2/go/pkg/beam" + "github.com/apache/beam/sdks/v2/go/pkg/beam/testing/ptest" + "github.com/apache/beam/sdks/v2/go/pkg/beam/transforms/periodic" + "github.com/apache/beam/sdks/v2/go/test/integration" +) + +func TestTimers_EventTime_Bounded(t *testing.T) { + integration.CheckFilters(t) + ptest.BuildAndRun(t, TimersEventTime(beam.Impulse)) +} + +func TestTimers_EventTime_Unbounded(t *testing.T) { + integration.CheckFilters(t) + ptest.BuildAndRun(t, TimersEventTime(func(s beam.Scope) beam.PCollection { + now := time.Now() + return periodic.Impulse(s, now, now.Add(10*time.Second), 0, false) + })) +} Review Comment: In the future might we anticipate, `TestTimers_ProcessTime_Bounded` and `TestTimers_ProcessTime_Unbounded` tests? As of this writing, for Java at least, `TestTimers_ProcessTime_Bounded` equivalent tests would fail. ########## sdks/go/test/integration/primitives/timers.go: ########## @@ -0,0 +1,131 @@ +// 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 primitives + +import ( + "context" + "strconv" + + "github.com/apache/beam/sdks/v2/go/pkg/beam" + "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" +) + +// Based on https://github.com/apache/beam/blob/master/runners/flink/src/test/java/org/apache/beam/runners/flink/PortableTimersExecutionTest.java + +func init() { + register.DoFn2x0[[]byte, func(string, int)](&inputFn[string, int]{}) + register.DoFn6x0[beam.Window, state.Provider, timers.Provider, string, int, func(kv[string, int])](&eventTimeFn{}) + register.Emitter2[string, int]() + register.Emitter1[kv[string, int]]() +} + +type kv[K, V any] struct { + Key K + Value V +} + +func kvfn[K, V any](k K, v V) kv[K, V] { + return kv[K, V]{k, v} +} + +type inputFn[K, V any] struct { + Inputs []kv[K, V] +} + +func (fn *inputFn[K, V]) ProcessElement(_ []byte, emit func(K, V)) { + for _, in := range fn.Inputs { + emit(in.Key, in.Value) + } +} + +type eventTimeFn struct { + Foo timers.EventTime + Sizzle state.Value[string] Review Comment: I vote for `StateValue` ########## sdks/go/test/integration/primitives/timers.go: ########## @@ -0,0 +1,131 @@ +// 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 primitives + +import ( + "context" + "strconv" + + "github.com/apache/beam/sdks/v2/go/pkg/beam" + "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" +) + +// Based on https://github.com/apache/beam/blob/master/runners/flink/src/test/java/org/apache/beam/runners/flink/PortableTimersExecutionTest.java + +func init() { + register.DoFn2x0[[]byte, func(string, int)](&inputFn[string, int]{}) + register.DoFn6x0[beam.Window, state.Provider, timers.Provider, string, int, func(kv[string, int])](&eventTimeFn{}) + register.Emitter2[string, int]() + register.Emitter1[kv[string, int]]() +} + +type kv[K, V any] struct { + Key K + Value V +} + +func kvfn[K, V any](k K, v V) kv[K, V] { + return kv[K, V]{k, v} +} + +type inputFn[K, V any] struct { + Inputs []kv[K, V] +} + +func (fn *inputFn[K, V]) ProcessElement(_ []byte, emit func(K, V)) { + for _, in := range fn.Inputs { + emit(in.Key, in.Value) + } +} + +type eventTimeFn struct { + Foo timers.EventTime Review Comment: I vote for `EventTime` ########## sdks/go/test/integration/primitives/timers.go: ########## @@ -0,0 +1,133 @@ +// 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 primitives + +import ( + "context" + "strconv" + + "github.com/apache/beam/sdks/v2/go/pkg/beam" + "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" +) + +// Based on https://github.com/apache/beam/blob/master/runners/flink/src/test/java/org/apache/beam/runners/flink/PortableTimersExecutionTest.java + +func init() { + register.DoFn2x0[[]byte, func(string, int)](&inputFn[string, int]{}) + register.DoFn6x0[beam.Window, state.Provider, timers.Provider, string, int, func(kv[string, int])](&eventTimeFn{}) + register.Emitter2[string, int]() + register.Emitter1[kv[string, int]]() +} + +type kv[K, V any] struct { + Key K + Value V +} + +func kvfn[K, V any](k K, v V) kv[K, V] { + return kv[K, V]{k, v} +} + +type inputFn[K, V any] struct { + Inputs []kv[K, V] +} + +func (fn *inputFn[K, V]) ProcessElement(_ []byte, emit func(K, V)) { + for _, in := range fn.Inputs { + emit(in.Key, in.Value) + } +} + +type eventTimeFn struct { + Callback timers.EventTime + MyKey state.Value[string] + + Offset int + TimerOutput int +} + +func (fn *eventTimeFn) ProcessElement(w beam.Window, sp state.Provider, tp timers.Provider, key string, value int, emit func(kv[string, int])) { + fn.Callback.Set(tp, w.MaxTimestamp().ToTime()) + fn.MyKey.Write(sp, key) + emit(kvfn(key, value+fn.Offset)) +} + +func (fn *eventTimeFn) OnTimer(ctx context.Context, ts beam.EventTime, sp state.Provider, tp timers.Provider, key string, timer timers.Context, emit func(kv[string, int])) { + switch timer.Family { + case fn.Callback.Family: + switch timer.Tag { + case "": Review Comment: Could this just be `if time.Tag == "`? ########## sdks/go/test/integration/primitives/timers.go: ########## @@ -0,0 +1,133 @@ +// 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 primitives + +import ( + "context" + "strconv" + + "github.com/apache/beam/sdks/v2/go/pkg/beam" + "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" +) + +// Based on https://github.com/apache/beam/blob/master/runners/flink/src/test/java/org/apache/beam/runners/flink/PortableTimersExecutionTest.java + +func init() { + register.DoFn2x0[[]byte, func(string, int)](&inputFn[string, int]{}) + register.DoFn6x0[beam.Window, state.Provider, timers.Provider, string, int, func(kv[string, int])](&eventTimeFn{}) + register.Emitter2[string, int]() + register.Emitter1[kv[string, int]]() +} + +type kv[K, V any] struct { + Key K + Value V +} + +func kvfn[K, V any](k K, v V) kv[K, V] { + return kv[K, V]{k, v} +} + +type inputFn[K, V any] struct { + Inputs []kv[K, V] +} + +func (fn *inputFn[K, V]) ProcessElement(_ []byte, emit func(K, V)) { + for _, in := range fn.Inputs { + emit(in.Key, in.Value) + } +} + +type eventTimeFn struct { + Callback timers.EventTime + MyKey state.Value[string] + + Offset int + TimerOutput int +} + +func (fn *eventTimeFn) ProcessElement(w beam.Window, sp state.Provider, tp timers.Provider, key string, value int, emit func(kv[string, int])) { + fn.Callback.Set(tp, w.MaxTimestamp().ToTime()) + fn.MyKey.Write(sp, key) + emit(kvfn(key, value+fn.Offset)) +} + +func (fn *eventTimeFn) OnTimer(ctx context.Context, ts beam.EventTime, sp state.Provider, tp timers.Provider, key string, timer timers.Context, emit func(kv[string, int])) { + switch timer.Family { + case fn.Callback.Family: + switch timer.Tag { + case "": + read, ok, err := fn.MyKey.Read(sp) + if err != nil { + panic(err) + } + if !ok { + panic("State must be set.") + } + emit(kvfn(read, fn.TimerOutput)) + } + default: + if fn.Callback.Family != timer.Family || timer.Tag != "" { + panic("unexpected timer family: " + timer.Family + " tag:" + timer.Tag + " want: " + fn.Callback.Family) + } + } +} + +// TimersEventTime takes in an impulse transform and then validates +// event time timer execution. +// +// The impulse is provided outside to swap between a bounded impulse, and +// an unbounded one, because the Go SDK uses that to determine if a pipeline +// is "streaming" or not. This matters at least for executions on Dataflow. +// +// Regardless,the pipelines should pass. +func TimersEventTime(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 := 50 + numDuplicateTimers := 15 Review Comment: Would it be valuable to hoist these as `const`s? ########## sdks/go/test/integration/primitives/timers.go: ########## @@ -0,0 +1,133 @@ +// 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 primitives + +import ( + "context" + "strconv" + + "github.com/apache/beam/sdks/v2/go/pkg/beam" + "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" +) + +// Based on https://github.com/apache/beam/blob/master/runners/flink/src/test/java/org/apache/beam/runners/flink/PortableTimersExecutionTest.java + +func init() { + register.DoFn2x0[[]byte, func(string, int)](&inputFn[string, int]{}) + register.DoFn6x0[beam.Window, state.Provider, timers.Provider, string, int, func(kv[string, int])](&eventTimeFn{}) + register.Emitter2[string, int]() + register.Emitter1[kv[string, int]]() +} + +type kv[K, V any] struct { + Key K + Value V +} + +func kvfn[K, V any](k K, v V) kv[K, V] { + return kv[K, V]{k, v} +} + +type inputFn[K, V any] struct { + Inputs []kv[K, V] +} + +func (fn *inputFn[K, V]) ProcessElement(_ []byte, emit func(K, V)) { + for _, in := range fn.Inputs { + emit(in.Key, in.Value) + } +} + +type eventTimeFn struct { + Callback timers.EventTime + MyKey state.Value[string] + + Offset int + TimerOutput int +} + +func (fn *eventTimeFn) ProcessElement(w beam.Window, sp state.Provider, tp timers.Provider, key string, value int, emit func(kv[string, int])) { + fn.Callback.Set(tp, w.MaxTimestamp().ToTime()) + fn.MyKey.Write(sp, key) + emit(kvfn(key, value+fn.Offset)) +} + +func (fn *eventTimeFn) OnTimer(ctx context.Context, ts beam.EventTime, sp state.Provider, tp timers.Provider, key string, timer timers.Context, emit func(kv[string, int])) { + switch timer.Family { Review Comment: Do we anticipate additional timer.Family in the future? I only see two cases `fn.Callback.Family` and `default`. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
