You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by "lostluck (via GitHub)" <gi...@apache.org> on 2023/02/20 01:41:51 UTC

[GitHub] [beam] lostluck opened a new pull request, #25557: [prism] Add testdofns for later

lostluck opened a new pull request, #25557:
URL: https://github.com/apache/beam/pull/25557

   Add the test DoFns used for bootstrap testing of prism. Unused in this PR.
   
   These are used in the main pipeline tests for the runner, but aren't very interesting themselves.
   
   A separate suit covers complex DoFns WRT "separation", like SDFs, process continuation and watermark estimating.
   
   See https://github.com/apache/beam/issues/24789 for more information.
   
   ------------------------
   
   Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:
   
    - [ ] Mention the appropriate issue in your description (for example: `addresses #123`), if applicable. This will automatically add a link to the pull request in the issue. If you would like the issue to automatically close on merging the pull request, comment `fixes #<ISSUE NUMBER>` instead.
    - [ ] Update `CHANGES.md` with noteworthy changes.
    - [ ] If this contribution is large, please file an Apache [Individual Contributor License Agreement](https://www.apache.org/licenses/icla.pdf).
   
   See the [Contributor Guide](https://beam.apache.org/contribute) for more tips on [how to make review process smoother](https://beam.apache.org/contribute/get-started-contributing/#make-the-reviewers-job-easier).
   
   To check the build health, please visit [https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md](https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md)
   
   GitHub Actions Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   [![Build python source distribution and wheels](https://github.com/apache/beam/workflows/Build%20python%20source%20distribution%20and%20wheels/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Build+python+source+distribution+and+wheels%22+branch%3Amaster+event%3Aschedule)
   [![Python tests](https://github.com/apache/beam/workflows/Python%20tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Python+Tests%22+branch%3Amaster+event%3Aschedule)
   [![Java tests](https://github.com/apache/beam/workflows/Java%20Tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Java+Tests%22+branch%3Amaster+event%3Aschedule)
   [![Go tests](https://github.com/apache/beam/workflows/Go%20tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Go+tests%22+branch%3Amaster+event%3Aschedule)
   
   See [CI.md](https://github.com/apache/beam/blob/master/CI.md) for more information about GitHub Actions CI.
   


-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] lostluck merged pull request #25557: [#24789][prism] Add testdofns for later

Posted by "lostluck (via GitHub)" <gi...@apache.org>.
lostluck merged PR #25557:
URL: https://github.com/apache/beam/pull/25557


-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] lostluck commented on a diff in pull request #25557: [#24789][prism] Add testdofns for later

Posted by "lostluck (via GitHub)" <gi...@apache.org>.
lostluck commented on code in PR #25557:
URL: https://github.com/apache/beam/pull/25557#discussion_r1112165652


##########
sdks/go/pkg/beam/runners/prism/internal/testdofns.go:
##########
@@ -0,0 +1,350 @@
+// 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 internal
+
+import (
+	"context"
+	"fmt"
+	"sort"
+	"time"
+
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/io/rtrackers/offsetrange"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/google/go-cmp/cmp"
+)
+
+// The Test DoFns live outside of the test files to get coverage information on DoFn
+// Lifecycle method execution. This inflates binary size, but ensures the runner is
+// exercising the expected feature set.
+//
+// Once there's enough confidence in the runner, we can move these into a dedicated testing
+// package along with the pipelines that use them.
+
+// Registerations should happen in the test files, so the compiler can prune these
+// when they are not in use.
+
+func dofn1(imp []byte, emit func(int64)) {
+	emit(1)
+	emit(2)
+	emit(3)
+}
+
+func dofn1kv(imp []byte, emit func(int64, int64)) {
+	emit(0, 1)
+	emit(0, 2)
+	emit(0, 3)
+}
+
+func dofn1x2(imp []byte, emitA func(int64), emitB func(int64)) {
+	emitA(1)
+	emitA(2)
+	emitA(3)
+	emitB(4)
+	emitB(5)
+	emitB(6)
+}
+
+func dofn1x5(imp []byte, emitA, emitB, emitC, emitD, emitE func(int64)) {
+	emitA(1)
+	emitB(2)
+	emitC(3)
+	emitD(4)
+	emitE(5)
+	emitA(6)
+	emitB(7)
+	emitC(8)
+	emitD(9)
+	emitE(10)
+}
+
+func dofn2x1(imp []byte, iter func(*int64) bool, emit func(int64)) {
+	var v, sum, c int64
+	for iter(&v) {
+		fmt.Println("dofn2x1 v", v, " c ", c)
+		sum += v
+		c++
+	}
+	fmt.Println("dofn2x1 sum", sum, "count", c)
+	emit(sum)
+}
+
+func dofn2x2KV(imp []byte, iter func(*string, *int64) bool, emitK func(string), emitV func(int64)) {
+	var k string
+	var v, sum int64
+	for iter(&k, &v) {
+		sum += v
+		emitK(k)
+	}
+	emitV(sum)
+}
+
+func dofnMultiMap(key string, lookup func(string) func(*int64) bool, emitK func(string), emitV func(int64)) {
+	var v, sum int64
+	iter := lookup(key)
+	for iter(&v) {
+		sum += v
+	}
+	emitK(key)
+	emitV(sum)
+}
+
+func dofn3x1(sum int64, iter1, iter2 func(*int64) bool, emit func(int64)) {
+	var v int64
+	for iter1(&v) {
+		sum += v
+	}
+	for iter2(&v) {
+		sum += v
+	}
+	emit(sum)
+}
+
+// int64Check validates that within a single bundle,
+// we received the expected int64 values & sends them downstream.
+//
+// Invalid pattern for general testing, as it will fail
+// on other valid execution patterns, like single element bundles.
+type int64Check struct {
+	Name string
+	Want []int
+	got  []int
+}
+
+func (fn *int64Check) ProcessElement(v int64, _ func(int64)) {
+	fn.got = append(fn.got, int(v))
+}
+
+func (fn *int64Check) FinishBundle(_ func(int64)) error {
+	sort.Ints(fn.got)
+	sort.Ints(fn.Want)
+	if d := cmp.Diff(fn.Want, fn.got); d != "" {
+		return fmt.Errorf("int64Check[%v] (-want, +got): %v", fn.Name, d)
+	}
+	// Clear for subsequent calls.
+	fn.got = nil
+	return nil
+}
+
+// stringCheck validates that within a single bundle,
+// we received the expected string values.
+// Re-emits them downstream.
+//
+// Invalid pattern for general testing, as it will fail
+// on other valid execution patterns, like single element bundles.
+type stringCheck struct {
+	Name string
+	Want []string
+	got  []string
+}
+
+func (fn *stringCheck) ProcessElement(v string, _ func(string)) {
+	fn.got = append(fn.got, v)
+}
+
+func (fn *stringCheck) FinishBundle(_ func(string)) error {
+	sort.Strings(fn.got)
+	sort.Strings(fn.Want)
+	if d := cmp.Diff(fn.Want, fn.got); d != "" {
+		return fmt.Errorf("stringCheck[%v] (-want, +got): %v", fn.Name, d)
+	}
+	return nil
+}
+
+func dofn2(v int64, emit func(int64)) {
+	emit(v + 1)
+}
+
+func dofnKV(imp []byte, emit func(string, int64)) {
+	emit("a", 1)
+	emit("b", 2)
+	emit("a", 3)
+	emit("b", 4)
+	emit("a", 5)
+	emit("b", 6)
+}
+
+func dofnKV2(imp []byte, emit func(int64, string)) {
+	emit(1, "a")
+	emit(2, "b")
+	emit(1, "a")
+	emit(2, "b")
+	emit(1, "a")
+	emit(2, "b")
+}
+
+func dofnGBK(k string, vs func(*int64) bool, emit func(int64)) {
+	var v, sum int64
+	for vs(&v) {
+		sum += v
+	}
+	emit(sum)
+}
+
+func dofnGBK2(k int64, vs func(*string) bool, emit func(string)) {
+	var v, sum string
+	for vs(&v) {
+		sum += v
+	}
+	emit(sum)
+}
+
+type testRow struct {
+	A string
+	B int64
+}
+
+func dofnKV3(imp []byte, emit func(testRow, testRow)) {
+	emit(testRow{"a", 1}, testRow{"a", 1})
+}
+
+func dofnGBK3(k testRow, vs func(*testRow) bool, emit func(string)) {
+	var v testRow
+	vs(&v)
+	emit(fmt.Sprintf("%v: %v", k, v))
+}
+
+const (
+	ns = "localtest"
+)
+
+func dofnSink(ctx context.Context, _ []byte) {
+	beam.NewCounter(ns, "sunk").Inc(ctx, 73)
+}
+
+func dofn1Counter(ctx context.Context, _ []byte, emit func(int64)) {
+	beam.NewCounter(ns, "count").Inc(ctx, 1)
+}
+
+func combineIntSum(a, b int64) int64 {
+	return a + b
+}
+
+// SourceConfig is a struct containing all the configuration options for a
+// synthetic source. It should be created via a SourceConfigBuilder, not by
+// directly initializing it (the fields are public to allow encoding).
+type SourceConfig struct {
+	NumElements   int64 `json:"num_records" beam:"num_records"`
+	InitialSplits int64 `json:"initial_splits" beam:"initial_splits"`
+}
+
+// intRangeFn is a splittable DoFn for counting from 1 to N.
+type intRangeFn struct{}
+
+// CreateInitialRestriction creates an offset range restriction representing
+// the number of elements to emit.
+func (fn *intRangeFn) CreateInitialRestriction(config SourceConfig) offsetrange.Restriction {
+	return offsetrange.Restriction{
+		Start: 0,
+		End:   int64(config.NumElements),
+	}
+}
+
+// SplitRestriction splits restrictions equally according to the number of
+// initial splits specified in SourceConfig. Each restriction output by this
+// method will contain at least one element, so the number of splits will not
+// exceed the number of elements.
+func (fn *intRangeFn) SplitRestriction(config SourceConfig, rest offsetrange.Restriction) (splits []offsetrange.Restriction) {
+	return rest.EvenSplits(int64(config.InitialSplits))
+}
+
+// RestrictionSize outputs the size of the restriction as the number of elements
+// that restriction will output.
+func (fn *intRangeFn) RestrictionSize(_ SourceConfig, rest offsetrange.Restriction) float64 {
+	return rest.Size()
+}
+
+// CreateTracker just creates an offset range restriction tracker for the
+// restriction.
+func (fn *intRangeFn) CreateTracker(rest offsetrange.Restriction) *sdf.LockRTracker {
+	return sdf.NewLockRTracker(offsetrange.NewTracker(rest))
+}
+
+// ProcessElement creates a number of random elements based on the restriction
+// tracker received. Each element is a random byte slice key and value, in the
+// form of KV<[]byte, []byte>.
+func (fn *intRangeFn) ProcessElement(rt *sdf.LockRTracker, config SourceConfig, emit func(int64)) error {
+	for i := rt.GetRestriction().(offsetrange.Restriction).Start; rt.TryClaim(i); i++ {
+		// Add 1 since the restrictions are from [0 ,N), but we want [1, N]
+		emit(i + 1)
+	}
+	return nil
+}
+
+func init() {
+	register.DoFn3x1[*sdf.LockRTracker, []byte, func(int64), sdf.ProcessContinuation](&selfCheckpointingDoFn{})
+	register.Emitter1[int64]()
+}
+
+type selfCheckpointingDoFn struct{}
+
+// CreateInitialRestriction creates the restriction being used by the SDF. In this case, the range
+// of values produced by the restriction is [Start, End).
+func (fn *selfCheckpointingDoFn) CreateInitialRestriction(_ []byte) offsetrange.Restriction {
+	return offsetrange.Restriction{
+		Start: int64(0),
+		End:   int64(10),
+	}
+}
+
+// CreateTracker wraps the fiven restriction into a LockRTracker type.
+func (fn *selfCheckpointingDoFn) CreateTracker(rest offsetrange.Restriction) *sdf.LockRTracker {
+	return sdf.NewLockRTracker(offsetrange.NewTracker(rest))
+}
+
+// RestrictionSize returns the size of the current restriction
+func (fn *selfCheckpointingDoFn) RestrictionSize(_ []byte, rest offsetrange.Restriction) float64 {
+	return rest.Size()
+}
+
+// SplitRestriction modifies the offsetrange.Restriction's sized restriction function to produce a size-zero restriction
+// at the end of execution.
+func (fn *selfCheckpointingDoFn) SplitRestriction(_ []byte, rest offsetrange.Restriction) []offsetrange.Restriction {
+	size := int64(3)
+	s := rest.Start
+	var splits []offsetrange.Restriction
+	for e := s + size; e <= rest.End; s, e = e, e+size {
+		splits = append(splits, offsetrange.Restriction{Start: s, End: e})
+	}
+	splits = append(splits, offsetrange.Restriction{Start: s, End: rest.End})
+	return splits
+}
+
+// ProcessElement continually gets the start position of the restriction and emits it as an int64 value before checkpointing.
+// This causes the restriction to be split after the claimed work and produce no primary roots.
+func (fn *selfCheckpointingDoFn) ProcessElement(rt *sdf.LockRTracker, _ []byte, emit func(int64)) sdf.ProcessContinuation {
+	position := rt.GetRestriction().(offsetrange.Restriction).Start
+
+	for {
+		if rt.TryClaim(position) {
+			// Successful claim, emit the value and move on.
+			emit(position)
+			position++
+			continue

Review Comment:
   Not strictly no. Removed!



-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] lostluck commented on pull request #25557: [prism] Add testdofns for later

Posted by "lostluck (via GitHub)" <gi...@apache.org>.
lostluck commented on PR #25557:
URL: https://github.com/apache/beam/pull/25557#issuecomment-1436193423

   R: @johannaojeling 
   
   This one is what it says on the tin. A bunch of Test DoFns that will eventually migrate into just the test file itself, instead of living in the package. 
   
   Having them in the "production" part of the package simplifies getting coverage numbers while iterating, and ensuring the different methods on the DoFns are activated as expected. Once metrics support in the runner is sufficiently robust, that can be changed to using counters and similar.


-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] github-actions[bot] commented on pull request #25557: [prism] Add testdofns for later

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #25557:
URL: https://github.com/apache/beam/pull/25557#issuecomment-1436194388

   Stopping reviewer notifications for this pull request: review requested by someone other than the bot, ceding control


-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] johannaojeling commented on a diff in pull request #25557: [#24789][prism] Add testdofns for later

Posted by "johannaojeling (via GitHub)" <gi...@apache.org>.
johannaojeling commented on code in PR #25557:
URL: https://github.com/apache/beam/pull/25557#discussion_r1112064526


##########
sdks/go/pkg/beam/runners/prism/internal/testdofns.go:
##########
@@ -0,0 +1,350 @@
+// 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 internal
+
+import (
+	"context"
+	"fmt"
+	"sort"
+	"time"
+
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/io/rtrackers/offsetrange"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/google/go-cmp/cmp"
+)
+
+// The Test DoFns live outside of the test files to get coverage information on DoFn
+// Lifecycle method execution. This inflates binary size, but ensures the runner is
+// exercising the expected feature set.
+//
+// Once there's enough confidence in the runner, we can move these into a dedicated testing
+// package along with the pipelines that use them.
+
+// Registerations should happen in the test files, so the compiler can prune these
+// when they are not in use.
+
+func dofn1(imp []byte, emit func(int64)) {
+	emit(1)
+	emit(2)
+	emit(3)
+}
+
+func dofn1kv(imp []byte, emit func(int64, int64)) {
+	emit(0, 1)
+	emit(0, 2)
+	emit(0, 3)
+}
+
+func dofn1x2(imp []byte, emitA func(int64), emitB func(int64)) {
+	emitA(1)
+	emitA(2)
+	emitA(3)
+	emitB(4)
+	emitB(5)
+	emitB(6)
+}
+
+func dofn1x5(imp []byte, emitA, emitB, emitC, emitD, emitE func(int64)) {
+	emitA(1)
+	emitB(2)
+	emitC(3)
+	emitD(4)
+	emitE(5)
+	emitA(6)
+	emitB(7)
+	emitC(8)
+	emitD(9)
+	emitE(10)
+}
+
+func dofn2x1(imp []byte, iter func(*int64) bool, emit func(int64)) {
+	var v, sum, c int64
+	for iter(&v) {
+		fmt.Println("dofn2x1 v", v, " c ", c)
+		sum += v
+		c++
+	}
+	fmt.Println("dofn2x1 sum", sum, "count", c)
+	emit(sum)
+}
+
+func dofn2x2KV(imp []byte, iter func(*string, *int64) bool, emitK func(string), emitV func(int64)) {
+	var k string
+	var v, sum int64
+	for iter(&k, &v) {
+		sum += v
+		emitK(k)
+	}
+	emitV(sum)
+}
+
+func dofnMultiMap(key string, lookup func(string) func(*int64) bool, emitK func(string), emitV func(int64)) {
+	var v, sum int64
+	iter := lookup(key)
+	for iter(&v) {
+		sum += v
+	}
+	emitK(key)
+	emitV(sum)
+}
+
+func dofn3x1(sum int64, iter1, iter2 func(*int64) bool, emit func(int64)) {
+	var v int64
+	for iter1(&v) {
+		sum += v
+	}
+	for iter2(&v) {
+		sum += v
+	}
+	emit(sum)
+}
+
+// int64Check validates that within a single bundle,
+// we received the expected int64 values & sends them downstream.
+//
+// Invalid pattern for general testing, as it will fail
+// on other valid execution patterns, like single element bundles.
+type int64Check struct {
+	Name string
+	Want []int
+	got  []int
+}
+
+func (fn *int64Check) ProcessElement(v int64, _ func(int64)) {
+	fn.got = append(fn.got, int(v))
+}
+
+func (fn *int64Check) FinishBundle(_ func(int64)) error {
+	sort.Ints(fn.got)
+	sort.Ints(fn.Want)
+	if d := cmp.Diff(fn.Want, fn.got); d != "" {
+		return fmt.Errorf("int64Check[%v] (-want, +got): %v", fn.Name, d)
+	}
+	// Clear for subsequent calls.
+	fn.got = nil
+	return nil
+}
+
+// stringCheck validates that within a single bundle,
+// we received the expected string values.
+// Re-emits them downstream.
+//
+// Invalid pattern for general testing, as it will fail
+// on other valid execution patterns, like single element bundles.
+type stringCheck struct {
+	Name string
+	Want []string
+	got  []string
+}
+
+func (fn *stringCheck) ProcessElement(v string, _ func(string)) {
+	fn.got = append(fn.got, v)
+}
+
+func (fn *stringCheck) FinishBundle(_ func(string)) error {
+	sort.Strings(fn.got)
+	sort.Strings(fn.Want)
+	if d := cmp.Diff(fn.Want, fn.got); d != "" {
+		return fmt.Errorf("stringCheck[%v] (-want, +got): %v", fn.Name, d)
+	}
+	return nil
+}
+
+func dofn2(v int64, emit func(int64)) {
+	emit(v + 1)
+}
+
+func dofnKV(imp []byte, emit func(string, int64)) {
+	emit("a", 1)
+	emit("b", 2)
+	emit("a", 3)
+	emit("b", 4)
+	emit("a", 5)
+	emit("b", 6)
+}
+
+func dofnKV2(imp []byte, emit func(int64, string)) {
+	emit(1, "a")
+	emit(2, "b")
+	emit(1, "a")
+	emit(2, "b")
+	emit(1, "a")
+	emit(2, "b")
+}
+
+func dofnGBK(k string, vs func(*int64) bool, emit func(int64)) {
+	var v, sum int64
+	for vs(&v) {
+		sum += v
+	}
+	emit(sum)
+}
+
+func dofnGBK2(k int64, vs func(*string) bool, emit func(string)) {
+	var v, sum string
+	for vs(&v) {
+		sum += v
+	}
+	emit(sum)
+}
+
+type testRow struct {
+	A string
+	B int64
+}
+
+func dofnKV3(imp []byte, emit func(testRow, testRow)) {
+	emit(testRow{"a", 1}, testRow{"a", 1})
+}
+
+func dofnGBK3(k testRow, vs func(*testRow) bool, emit func(string)) {
+	var v testRow
+	vs(&v)
+	emit(fmt.Sprintf("%v: %v", k, v))
+}
+
+const (
+	ns = "localtest"
+)
+
+func dofnSink(ctx context.Context, _ []byte) {
+	beam.NewCounter(ns, "sunk").Inc(ctx, 73)
+}
+
+func dofn1Counter(ctx context.Context, _ []byte, emit func(int64)) {
+	beam.NewCounter(ns, "count").Inc(ctx, 1)
+}
+
+func combineIntSum(a, b int64) int64 {
+	return a + b
+}
+
+// SourceConfig is a struct containing all the configuration options for a
+// synthetic source. It should be created via a SourceConfigBuilder, not by
+// directly initializing it (the fields are public to allow encoding).
+type SourceConfig struct {
+	NumElements   int64 `json:"num_records" beam:"num_records"`
+	InitialSplits int64 `json:"initial_splits" beam:"initial_splits"`
+}
+
+// intRangeFn is a splittable DoFn for counting from 1 to N.
+type intRangeFn struct{}
+
+// CreateInitialRestriction creates an offset range restriction representing
+// the number of elements to emit.
+func (fn *intRangeFn) CreateInitialRestriction(config SourceConfig) offsetrange.Restriction {
+	return offsetrange.Restriction{
+		Start: 0,
+		End:   int64(config.NumElements),
+	}
+}
+
+// SplitRestriction splits restrictions equally according to the number of
+// initial splits specified in SourceConfig. Each restriction output by this
+// method will contain at least one element, so the number of splits will not
+// exceed the number of elements.
+func (fn *intRangeFn) SplitRestriction(config SourceConfig, rest offsetrange.Restriction) (splits []offsetrange.Restriction) {
+	return rest.EvenSplits(int64(config.InitialSplits))
+}
+
+// RestrictionSize outputs the size of the restriction as the number of elements
+// that restriction will output.
+func (fn *intRangeFn) RestrictionSize(_ SourceConfig, rest offsetrange.Restriction) float64 {
+	return rest.Size()
+}
+
+// CreateTracker just creates an offset range restriction tracker for the
+// restriction.
+func (fn *intRangeFn) CreateTracker(rest offsetrange.Restriction) *sdf.LockRTracker {
+	return sdf.NewLockRTracker(offsetrange.NewTracker(rest))
+}
+
+// ProcessElement creates a number of random elements based on the restriction
+// tracker received. Each element is a random byte slice key and value, in the
+// form of KV<[]byte, []byte>.
+func (fn *intRangeFn) ProcessElement(rt *sdf.LockRTracker, config SourceConfig, emit func(int64)) error {
+	for i := rt.GetRestriction().(offsetrange.Restriction).Start; rt.TryClaim(i); i++ {
+		// Add 1 since the restrictions are from [0 ,N), but we want [1, N]
+		emit(i + 1)
+	}
+	return nil
+}
+
+func init() {
+	register.DoFn3x1[*sdf.LockRTracker, []byte, func(int64), sdf.ProcessContinuation](&selfCheckpointingDoFn{})
+	register.Emitter1[int64]()
+}
+
+type selfCheckpointingDoFn struct{}
+
+// CreateInitialRestriction creates the restriction being used by the SDF. In this case, the range
+// of values produced by the restriction is [Start, End).
+func (fn *selfCheckpointingDoFn) CreateInitialRestriction(_ []byte) offsetrange.Restriction {
+	return offsetrange.Restriction{
+		Start: int64(0),
+		End:   int64(10),
+	}
+}
+
+// CreateTracker wraps the fiven restriction into a LockRTracker type.

Review Comment:
   ```suggestion
   // CreateTracker wraps the given restriction into a LockRTracker type.
   ```



##########
sdks/go/pkg/beam/runners/prism/internal/testdofns.go:
##########
@@ -0,0 +1,350 @@
+// 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 internal
+
+import (
+	"context"
+	"fmt"
+	"sort"
+	"time"
+
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/io/rtrackers/offsetrange"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/google/go-cmp/cmp"
+)
+
+// The Test DoFns live outside of the test files to get coverage information on DoFn
+// Lifecycle method execution. This inflates binary size, but ensures the runner is
+// exercising the expected feature set.
+//
+// Once there's enough confidence in the runner, we can move these into a dedicated testing
+// package along with the pipelines that use them.
+
+// Registerations should happen in the test files, so the compiler can prune these

Review Comment:
   ```suggestion
   // Registrations should happen in the test files, so the compiler can prune these
   ```



##########
sdks/go/pkg/beam/runners/prism/internal/testdofns.go:
##########
@@ -0,0 +1,350 @@
+// 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 internal
+
+import (
+	"context"
+	"fmt"
+	"sort"
+	"time"
+
+	"github.com/apache/beam/sdks/v2/go/pkg/beam"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/io/rtrackers/offsetrange"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+	"github.com/google/go-cmp/cmp"
+)
+
+// The Test DoFns live outside of the test files to get coverage information on DoFn
+// Lifecycle method execution. This inflates binary size, but ensures the runner is
+// exercising the expected feature set.
+//
+// Once there's enough confidence in the runner, we can move these into a dedicated testing
+// package along with the pipelines that use them.
+
+// Registerations should happen in the test files, so the compiler can prune these
+// when they are not in use.
+
+func dofn1(imp []byte, emit func(int64)) {
+	emit(1)
+	emit(2)
+	emit(3)
+}
+
+func dofn1kv(imp []byte, emit func(int64, int64)) {
+	emit(0, 1)
+	emit(0, 2)
+	emit(0, 3)
+}
+
+func dofn1x2(imp []byte, emitA func(int64), emitB func(int64)) {
+	emitA(1)
+	emitA(2)
+	emitA(3)
+	emitB(4)
+	emitB(5)
+	emitB(6)
+}
+
+func dofn1x5(imp []byte, emitA, emitB, emitC, emitD, emitE func(int64)) {
+	emitA(1)
+	emitB(2)
+	emitC(3)
+	emitD(4)
+	emitE(5)
+	emitA(6)
+	emitB(7)
+	emitC(8)
+	emitD(9)
+	emitE(10)
+}
+
+func dofn2x1(imp []byte, iter func(*int64) bool, emit func(int64)) {
+	var v, sum, c int64
+	for iter(&v) {
+		fmt.Println("dofn2x1 v", v, " c ", c)
+		sum += v
+		c++
+	}
+	fmt.Println("dofn2x1 sum", sum, "count", c)
+	emit(sum)
+}
+
+func dofn2x2KV(imp []byte, iter func(*string, *int64) bool, emitK func(string), emitV func(int64)) {
+	var k string
+	var v, sum int64
+	for iter(&k, &v) {
+		sum += v
+		emitK(k)
+	}
+	emitV(sum)
+}
+
+func dofnMultiMap(key string, lookup func(string) func(*int64) bool, emitK func(string), emitV func(int64)) {
+	var v, sum int64
+	iter := lookup(key)
+	for iter(&v) {
+		sum += v
+	}
+	emitK(key)
+	emitV(sum)
+}
+
+func dofn3x1(sum int64, iter1, iter2 func(*int64) bool, emit func(int64)) {
+	var v int64
+	for iter1(&v) {
+		sum += v
+	}
+	for iter2(&v) {
+		sum += v
+	}
+	emit(sum)
+}
+
+// int64Check validates that within a single bundle,
+// we received the expected int64 values & sends them downstream.
+//
+// Invalid pattern for general testing, as it will fail
+// on other valid execution patterns, like single element bundles.
+type int64Check struct {
+	Name string
+	Want []int
+	got  []int
+}
+
+func (fn *int64Check) ProcessElement(v int64, _ func(int64)) {
+	fn.got = append(fn.got, int(v))
+}
+
+func (fn *int64Check) FinishBundle(_ func(int64)) error {
+	sort.Ints(fn.got)
+	sort.Ints(fn.Want)
+	if d := cmp.Diff(fn.Want, fn.got); d != "" {
+		return fmt.Errorf("int64Check[%v] (-want, +got): %v", fn.Name, d)
+	}
+	// Clear for subsequent calls.
+	fn.got = nil
+	return nil
+}
+
+// stringCheck validates that within a single bundle,
+// we received the expected string values.
+// Re-emits them downstream.
+//
+// Invalid pattern for general testing, as it will fail
+// on other valid execution patterns, like single element bundles.
+type stringCheck struct {
+	Name string
+	Want []string
+	got  []string
+}
+
+func (fn *stringCheck) ProcessElement(v string, _ func(string)) {
+	fn.got = append(fn.got, v)
+}
+
+func (fn *stringCheck) FinishBundle(_ func(string)) error {
+	sort.Strings(fn.got)
+	sort.Strings(fn.Want)
+	if d := cmp.Diff(fn.Want, fn.got); d != "" {
+		return fmt.Errorf("stringCheck[%v] (-want, +got): %v", fn.Name, d)
+	}
+	return nil
+}
+
+func dofn2(v int64, emit func(int64)) {
+	emit(v + 1)
+}
+
+func dofnKV(imp []byte, emit func(string, int64)) {
+	emit("a", 1)
+	emit("b", 2)
+	emit("a", 3)
+	emit("b", 4)
+	emit("a", 5)
+	emit("b", 6)
+}
+
+func dofnKV2(imp []byte, emit func(int64, string)) {
+	emit(1, "a")
+	emit(2, "b")
+	emit(1, "a")
+	emit(2, "b")
+	emit(1, "a")
+	emit(2, "b")
+}
+
+func dofnGBK(k string, vs func(*int64) bool, emit func(int64)) {
+	var v, sum int64
+	for vs(&v) {
+		sum += v
+	}
+	emit(sum)
+}
+
+func dofnGBK2(k int64, vs func(*string) bool, emit func(string)) {
+	var v, sum string
+	for vs(&v) {
+		sum += v
+	}
+	emit(sum)
+}
+
+type testRow struct {
+	A string
+	B int64
+}
+
+func dofnKV3(imp []byte, emit func(testRow, testRow)) {
+	emit(testRow{"a", 1}, testRow{"a", 1})
+}
+
+func dofnGBK3(k testRow, vs func(*testRow) bool, emit func(string)) {
+	var v testRow
+	vs(&v)
+	emit(fmt.Sprintf("%v: %v", k, v))
+}
+
+const (
+	ns = "localtest"
+)
+
+func dofnSink(ctx context.Context, _ []byte) {
+	beam.NewCounter(ns, "sunk").Inc(ctx, 73)
+}
+
+func dofn1Counter(ctx context.Context, _ []byte, emit func(int64)) {
+	beam.NewCounter(ns, "count").Inc(ctx, 1)
+}
+
+func combineIntSum(a, b int64) int64 {
+	return a + b
+}
+
+// SourceConfig is a struct containing all the configuration options for a
+// synthetic source. It should be created via a SourceConfigBuilder, not by
+// directly initializing it (the fields are public to allow encoding).
+type SourceConfig struct {
+	NumElements   int64 `json:"num_records" beam:"num_records"`
+	InitialSplits int64 `json:"initial_splits" beam:"initial_splits"`
+}
+
+// intRangeFn is a splittable DoFn for counting from 1 to N.
+type intRangeFn struct{}
+
+// CreateInitialRestriction creates an offset range restriction representing
+// the number of elements to emit.
+func (fn *intRangeFn) CreateInitialRestriction(config SourceConfig) offsetrange.Restriction {
+	return offsetrange.Restriction{
+		Start: 0,
+		End:   int64(config.NumElements),
+	}
+}
+
+// SplitRestriction splits restrictions equally according to the number of
+// initial splits specified in SourceConfig. Each restriction output by this
+// method will contain at least one element, so the number of splits will not
+// exceed the number of elements.
+func (fn *intRangeFn) SplitRestriction(config SourceConfig, rest offsetrange.Restriction) (splits []offsetrange.Restriction) {
+	return rest.EvenSplits(int64(config.InitialSplits))
+}
+
+// RestrictionSize outputs the size of the restriction as the number of elements
+// that restriction will output.
+func (fn *intRangeFn) RestrictionSize(_ SourceConfig, rest offsetrange.Restriction) float64 {
+	return rest.Size()
+}
+
+// CreateTracker just creates an offset range restriction tracker for the
+// restriction.
+func (fn *intRangeFn) CreateTracker(rest offsetrange.Restriction) *sdf.LockRTracker {
+	return sdf.NewLockRTracker(offsetrange.NewTracker(rest))
+}
+
+// ProcessElement creates a number of random elements based on the restriction
+// tracker received. Each element is a random byte slice key and value, in the
+// form of KV<[]byte, []byte>.
+func (fn *intRangeFn) ProcessElement(rt *sdf.LockRTracker, config SourceConfig, emit func(int64)) error {
+	for i := rt.GetRestriction().(offsetrange.Restriction).Start; rt.TryClaim(i); i++ {
+		// Add 1 since the restrictions are from [0 ,N), but we want [1, N]
+		emit(i + 1)
+	}
+	return nil
+}
+
+func init() {
+	register.DoFn3x1[*sdf.LockRTracker, []byte, func(int64), sdf.ProcessContinuation](&selfCheckpointingDoFn{})
+	register.Emitter1[int64]()
+}
+
+type selfCheckpointingDoFn struct{}
+
+// CreateInitialRestriction creates the restriction being used by the SDF. In this case, the range
+// of values produced by the restriction is [Start, End).
+func (fn *selfCheckpointingDoFn) CreateInitialRestriction(_ []byte) offsetrange.Restriction {
+	return offsetrange.Restriction{
+		Start: int64(0),
+		End:   int64(10),
+	}
+}
+
+// CreateTracker wraps the fiven restriction into a LockRTracker type.
+func (fn *selfCheckpointingDoFn) CreateTracker(rest offsetrange.Restriction) *sdf.LockRTracker {
+	return sdf.NewLockRTracker(offsetrange.NewTracker(rest))
+}
+
+// RestrictionSize returns the size of the current restriction
+func (fn *selfCheckpointingDoFn) RestrictionSize(_ []byte, rest offsetrange.Restriction) float64 {
+	return rest.Size()
+}
+
+// SplitRestriction modifies the offsetrange.Restriction's sized restriction function to produce a size-zero restriction
+// at the end of execution.
+func (fn *selfCheckpointingDoFn) SplitRestriction(_ []byte, rest offsetrange.Restriction) []offsetrange.Restriction {
+	size := int64(3)
+	s := rest.Start
+	var splits []offsetrange.Restriction
+	for e := s + size; e <= rest.End; s, e = e, e+size {
+		splits = append(splits, offsetrange.Restriction{Start: s, End: e})
+	}
+	splits = append(splits, offsetrange.Restriction{Start: s, End: rest.End})
+	return splits
+}
+
+// ProcessElement continually gets the start position of the restriction and emits it as an int64 value before checkpointing.
+// This causes the restriction to be split after the claimed work and produce no primary roots.
+func (fn *selfCheckpointingDoFn) ProcessElement(rt *sdf.LockRTracker, _ []byte, emit func(int64)) sdf.ProcessContinuation {
+	position := rt.GetRestriction().(offsetrange.Restriction).Start
+
+	for {
+		if rt.TryClaim(position) {
+			// Successful claim, emit the value and move on.
+			emit(position)
+			position++
+			continue

Review Comment:
   Is this `continue` needed?



-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] codecov[bot] commented on pull request #25557: [prism] Add testdofns for later

Posted by "codecov[bot] (via GitHub)" <gi...@apache.org>.
codecov[bot] commented on PR #25557:
URL: https://github.com/apache/beam/pull/25557#issuecomment-1436184132

   # [Codecov](https://codecov.io/gh/apache/beam/pull/25557?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#25557](https://codecov.io/gh/apache/beam/pull/25557?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (ee80586) into [master](https://codecov.io/gh/apache/beam/commit/6667eb4741bad1bad199b61012476d62ea3d7e27?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (6667eb4) will **decrease** coverage by `0.13%`.
   > The diff coverage is `1.70%`.
   
   ```diff
   @@            Coverage Diff             @@
   ##           master   #25557      +/-   ##
   ==========================================
   - Coverage   72.64%   72.51%   -0.13%     
   ==========================================
     Files         763      764       +1     
     Lines      101060   101236     +176     
   ==========================================
   + Hits        73410    73413       +3     
   - Misses      26228    26401     +173     
     Partials     1422     1422              
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | go | `52.03% <1.70%> (-0.28%)` | :arrow_down: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/beam/pull/25557?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...ks/go/pkg/beam/runners/prism/internal/testdofns.go](https://codecov.io/gh/apache/beam/pull/25557?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9nby9wa2cvYmVhbS9ydW5uZXJzL3ByaXNtL2ludGVybmFsL3Rlc3Rkb2Zucy5nbw==) | `1.70% <1.70%> (ø)` | |
   
   :mega: We’re building smart automated test selection to slash your CI/CD build times. [Learn more](https://about.codecov.io/iterative-testing/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   


-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org