You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2020/05/07 19:35:04 UTC

[GitHub] [beam] lostluck commented on a change in pull request #11605: [BEAM-9883] Refactor SDF test restrictions.

lostluck commented on a change in pull request #11605:
URL: https://github.com/apache/beam/pull/11605#discussion_r421731826



##########
File path: sdks/go/pkg/beam/core/runtime/exec/sdf_invokers_test.go
##########
@@ -210,86 +229,154 @@ func TestInvokes(t *testing.T) {
 	})
 }
 
-type Restriction struct {
-	Val int
+// VetRestriction is a restriction used for validating that SDF methods get
+// called with it. When using VetRestriction, the SDF methods it's used in
+// should pass it as a pointer so the method can make changes to the restriction
+// even if it doesn't output one directly (such as RestrictionSize).
+//
+type VetRestriction struct {
+	// An identifier to differentiate restrictions on the same elements. When
+	// split, a suffix in the form of ".#" is appended to this ID.
+	ID string
+
+	// Key and Val just copy the last seen input element's key and value to
+	// confirm that the restriction saw the expected element.
+	Key, Val interface{}
+
+	// These booleans should be flipped to true by the corresponding SDF methods
+	// to prove that the methods got called on the restriction.
+	CreateRest, SplitRest, RestSize, CreateTracker, ProcessElm bool
 }
 
-// RTracker's methods can all be no-ops, we just need it to implement sdf.RTracker.
-type RTracker struct {
-	Rest Restriction
-	Val  int
+// VetRTracker's methods can all be no-ops, we just need it to implement
+// sdf.RTracker and allow validating that it was passed to ProcessElement.
+type VetRTracker struct {
+	Rest *VetRestriction
 }
 
-func (rt *RTracker) TryClaim(interface{}) bool       { return false }
-func (rt *RTracker) GetError() error                 { return nil }
-func (rt *RTracker) GetProgress() (float64, float64) { return 0, 0 }
-func (rt *RTracker) IsDone() bool                    { return true }
-func (rt *RTracker) TrySplit(fraction float64) (interface{}, interface{}, error) {
+func (rt *VetRTracker) TryClaim(interface{}) bool       { return false }
+func (rt *VetRTracker) GetError() error                 { return nil }
+func (rt *VetRTracker) GetProgress() (float64, float64) { return 0, 0 }
+func (rt *VetRTracker) IsDone() bool                    { return true }
+func (rt *VetRTracker) TrySplit(fraction float64) (interface{}, interface{}, error) {
 	return nil, nil, nil
 }
 
-// In order to test that these methods get called properly, each one has an
-// implementation that lets us confirm that each argument was passed properly.
-
-type Sdf struct {
+// VetSdf runs an SDF In order to test that these methods get called properly,
+// each method will flip the corresponding flag in the passed in VetRestriction,
+// overwrite the restriction's Key and Val with the last seen input elements,
+// and retain the other fields in the VetRestriction.
+type VetSdf struct {
 }
 
-// CreateInitialRestriction creates a restriction with the given value.
-func (fn *Sdf) CreateInitialRestriction(i int) Restriction {
-	return Restriction{i}
+// CreateInitialRestriction creates a restriction with the given values and
+// with the appropriate flags to track that this was called.
+func (fn *VetSdf) CreateInitialRestriction(i int) *VetRestriction {
+	return &VetRestriction{ID: "Sdf", Val: i, CreateRest: true}
 }
 
-// SplitRestriction outputs two restrictions, the first containing the sum of i
-// and rest.Val, the second containing the same value plus 1.
-func (fn *Sdf) SplitRestriction(i int, rest Restriction) []Restriction {
-	return []Restriction{{rest.Val + i}, {rest.Val + i + 1}}
+// SplitRestriction outputs two identical restrictions, each being a copy of the
+// initial one, but with the appropriate flags to track this was called. The
+// split restrictions add a suffix of the form ".#" to the ID.
+func (fn *VetSdf) SplitRestriction(i int, rest *VetRestriction) []*VetRestriction {
+	rest.SplitRest = true
+	rest1 := &VetRestriction{
+		ID:            rest.ID + ".1",
+		Val:           i,
+		CreateRest:    rest.CreateRest,
+		SplitRest:     true,
+		RestSize:      rest.RestSize,
+		CreateTracker: rest.CreateTracker,
+		ProcessElm:    rest.ProcessElm,
+	}
+	rest2 := &VetRestriction{}
+	*rest2 = *rest1
+	rest2.ID = rest.ID + ".2"
+	return []*VetRestriction{rest1, rest2}
 }
 
-// RestrictionSize returns the sum of i and rest.Val as a float64.
-func (fn *Sdf) RestrictionSize(i int, rest Restriction) float64 {
-	return (float64)(i + rest.Val)
+// RestrictionSize just returns i as the size, as well as flipping appropriate
+// flags on the restriction to track that this was called.
+func (fn *VetSdf) RestrictionSize(i int, rest *VetRestriction) float64 {
+	rest.Key = nil
+	rest.Val = i
+	rest.RestSize = true
+	return (float64)(i)
 }
 
-// CreateTracker creates an RTracker containing the given restriction and a Val
-// of 1.
-func (fn *Sdf) CreateTracker(rest Restriction) *RTracker {
-	return &RTracker{rest, 1}
+// CreateTracker creates an RTracker containing the given restriction and flips
+// the appropriate flags on the restriction to track that this was called.
+func (fn *VetSdf) CreateTracker(rest *VetRestriction) *VetRTracker {
+	rest.CreateTracker = true
+	return &VetRTracker{rest}
 }
 
-// ProcessElement emits a pair of ints. The first is the input +
-// RTracker.Rest.Val. The second is the input + RTracker.Val.
-func (fn *Sdf) ProcessElement(rt *RTracker, i int, emit func(int, int)) {
-	emit(i+rt.Rest.Val, i+rt.Val)
+// ProcessElement emits a copy of the restriction in the restriction tracker it
+// received, with the appropriate flags flipped to track that this was called.
+func (fn *VetSdf) ProcessElement(rt *VetRTracker, i int, emit func(VetRestriction)) {
+	rest := *rt.Rest
+	rest.Key = nil
+	rest.Val = i
+	rest.ProcessElm = true
+	emit(rest)
 }
 
-type KvSdf struct {
+// VetKvSdf runs an SDF In order to test that these methods get called properly,
+// each method will flip the corresponding flag in the passed in VetRestriction,
+// overwrite the restriction's Key and Val with the last seen input elements,
+// and retain the other fields in the VetRestriction.
+type VetKvSdf struct {
 }
 
-// CreateInitialRestriction creates a restriction with the sum of the given
-// values.
-func (fn *KvSdf) CreateInitialRestriction(i int, j int) Restriction {
-	return Restriction{i + j}
+// CreateInitialRestriction creates a restriction with the given values and
+// with the appropriate flags to track that this was called.
+func (fn *VetKvSdf) CreateInitialRestriction(i, j int) *VetRestriction {
+	return &VetRestriction{ID: "KvSdf", Key: i, Val: j, CreateRest: true}
 }
 
-// SplitRestriction outputs two restrictions, the first containing the sum of i
-// and rest.Val, the second containing the sum of j and rest.Val.
-func (fn *KvSdf) SplitRestriction(i int, j int, rest Restriction) []Restriction {
-	return []Restriction{{rest.Val + i}, {rest.Val + j}}
+// SplitRestriction outputs two identical restrictions, each being a copy of the
+// initial one, but with the appropriate flags to track this was called. The
+// split restrictions add a suffix of the form ".#" to the ID.
+func (fn *VetKvSdf) SplitRestriction(i, j int, rest *VetRestriction) []*VetRestriction {
+	rest.SplitRest = true
+	rest1 := &VetRestriction{
+		ID:            rest.ID + ".1",
+		Key:           i,
+		Val:           j,
+		CreateRest:    rest.CreateRest,
+		SplitRest:     true,
+		RestSize:      rest.RestSize,
+		CreateTracker: rest.CreateTracker,
+		ProcessElm:    rest.ProcessElm,
+	}
+	rest2 := &VetRestriction{}
+	*rest2 = *rest1
+	rest2.ID = rest.ID + ".2"
+	return []*VetRestriction{rest1, rest2}
 }
 
-// RestrictionSize returns the sum of i, j, and rest.Val as a float64.
-func (fn *KvSdf) RestrictionSize(i int, j int, rest Restriction) float64 {
-	return (float64)(i + j + rest.Val)
+// RestrictionSize just returns the sum of i and j as the size, as well as
+// flipping appropriate flags on the restriction to track that this was called.
+func (fn *VetKvSdf) RestrictionSize(i, j int, rest *VetRestriction) float64 {
+	rest.Key = i
+	rest.Val = j
+	rest.RestSize = true
+	return (float64)(i + j)
 }
 
-// CreateTracker creates an RTracker containing the given restriction and a Val
-// of 2.
-func (fn *KvSdf) CreateTracker(rest Restriction) *RTracker {
-	return &RTracker{rest, 2}
+// CreateTracker creates an RTracker containing the given restriction and flips
+// the appropriate flags on the restriction to track that this was called.
+func (fn *VetKvSdf) CreateTracker(rest *VetRestriction) *VetRTracker {
+	rest.CreateTracker = true
+	return &VetRTracker{rest}
 }
 
-// ProcessElement emits two ints. The first is the first input (key) +
-// RTracker.Rest.Val. The second is the second input (value) + RTracker.Val.
-func (fn *KvSdf) ProcessElement(rt *RTracker, i1 int, i2 int, emit func(int, int)) {
-	emit(i1+rt.Rest.Val, i2+rt.Val)
+// ProcessElement emits a copy of the restriction in the restriction tracker it
+// received, with the appropriate flags flipped to track that this was called.
+func (fn *VetKvSdf) ProcessElement(rt *VetRTracker, i, j int, emit func(VetRestriction)) {

Review comment:
       While this is a Test SDF, consider documenting explicitly that returning a restriction is an artifact of testing SDFs, and not how a correct SDF is written.  

##########
File path: sdks/go/pkg/beam/core/runtime/exec/sdf_invokers_test.go
##########
@@ -210,86 +229,154 @@ func TestInvokes(t *testing.T) {
 	})
 }
 
-type Restriction struct {
-	Val int
+// VetRestriction is a restriction used for validating that SDF methods get
+// called with it. When using VetRestriction, the SDF methods it's used in
+// should pass it as a pointer so the method can make changes to the restriction
+// even if it doesn't output one directly (such as RestrictionSize).
+//
+type VetRestriction struct {
+	// An identifier to differentiate restrictions on the same elements. When
+	// split, a suffix in the form of ".#" is appended to this ID.
+	ID string
+
+	// Key and Val just copy the last seen input element's key and value to
+	// confirm that the restriction saw the expected element.
+	Key, Val interface{}
+
+	// These booleans should be flipped to true by the corresponding SDF methods
+	// to prove that the methods got called on the restriction.
+	CreateRest, SplitRest, RestSize, CreateTracker, ProcessElm bool
 }
 
-// RTracker's methods can all be no-ops, we just need it to implement sdf.RTracker.
-type RTracker struct {
-	Rest Restriction
-	Val  int
+// VetRTracker's methods can all be no-ops, we just need it to implement
+// sdf.RTracker and allow validating that it was passed to ProcessElement.
+type VetRTracker struct {
+	Rest *VetRestriction
 }
 
-func (rt *RTracker) TryClaim(interface{}) bool       { return false }
-func (rt *RTracker) GetError() error                 { return nil }
-func (rt *RTracker) GetProgress() (float64, float64) { return 0, 0 }
-func (rt *RTracker) IsDone() bool                    { return true }
-func (rt *RTracker) TrySplit(fraction float64) (interface{}, interface{}, error) {
+func (rt *VetRTracker) TryClaim(interface{}) bool       { return false }
+func (rt *VetRTracker) GetError() error                 { return nil }
+func (rt *VetRTracker) GetProgress() (float64, float64) { return 0, 0 }
+func (rt *VetRTracker) IsDone() bool                    { return true }
+func (rt *VetRTracker) TrySplit(fraction float64) (interface{}, interface{}, error) {
 	return nil, nil, nil
 }
 
-// In order to test that these methods get called properly, each one has an
-// implementation that lets us confirm that each argument was passed properly.
-
-type Sdf struct {
+// VetSdf runs an SDF In order to test that these methods get called properly,
+// each method will flip the corresponding flag in the passed in VetRestriction,
+// overwrite the restriction's Key and Val with the last seen input elements,
+// and retain the other fields in the VetRestriction.
+type VetSdf struct {
 }
 
-// CreateInitialRestriction creates a restriction with the given value.
-func (fn *Sdf) CreateInitialRestriction(i int) Restriction {
-	return Restriction{i}
+// CreateInitialRestriction creates a restriction with the given values and
+// with the appropriate flags to track that this was called.
+func (fn *VetSdf) CreateInitialRestriction(i int) *VetRestriction {
+	return &VetRestriction{ID: "Sdf", Val: i, CreateRest: true}
 }
 
-// SplitRestriction outputs two restrictions, the first containing the sum of i
-// and rest.Val, the second containing the same value plus 1.
-func (fn *Sdf) SplitRestriction(i int, rest Restriction) []Restriction {
-	return []Restriction{{rest.Val + i}, {rest.Val + i + 1}}
+// SplitRestriction outputs two identical restrictions, each being a copy of the
+// initial one, but with the appropriate flags to track this was called. The
+// split restrictions add a suffix of the form ".#" to the ID.
+func (fn *VetSdf) SplitRestriction(i int, rest *VetRestriction) []*VetRestriction {
+	rest.SplitRest = true
+	rest1 := &VetRestriction{
+		ID:            rest.ID + ".1",
+		Val:           i,
+		CreateRest:    rest.CreateRest,
+		SplitRest:     true,
+		RestSize:      rest.RestSize,
+		CreateTracker: rest.CreateTracker,
+		ProcessElm:    rest.ProcessElm,
+	}
+	rest2 := &VetRestriction{}
+	*rest2 = *rest1
+	rest2.ID = rest.ID + ".2"
+	return []*VetRestriction{rest1, rest2}
 }
 
-// RestrictionSize returns the sum of i and rest.Val as a float64.
-func (fn *Sdf) RestrictionSize(i int, rest Restriction) float64 {
-	return (float64)(i + rest.Val)
+// RestrictionSize just returns i as the size, as well as flipping appropriate
+// flags on the restriction to track that this was called.
+func (fn *VetSdf) RestrictionSize(i int, rest *VetRestriction) float64 {
+	rest.Key = nil
+	rest.Val = i
+	rest.RestSize = true
+	return (float64)(i)
 }
 
-// CreateTracker creates an RTracker containing the given restriction and a Val
-// of 1.
-func (fn *Sdf) CreateTracker(rest Restriction) *RTracker {
-	return &RTracker{rest, 1}
+// CreateTracker creates an RTracker containing the given restriction and flips
+// the appropriate flags on the restriction to track that this was called.
+func (fn *VetSdf) CreateTracker(rest *VetRestriction) *VetRTracker {
+	rest.CreateTracker = true
+	return &VetRTracker{rest}
 }
 
-// ProcessElement emits a pair of ints. The first is the input +
-// RTracker.Rest.Val. The second is the input + RTracker.Val.
-func (fn *Sdf) ProcessElement(rt *RTracker, i int, emit func(int, int)) {
-	emit(i+rt.Rest.Val, i+rt.Val)
+// ProcessElement emits a copy of the restriction in the restriction tracker it
+// received, with the appropriate flags flipped to track that this was called.
+func (fn *VetSdf) ProcessElement(rt *VetRTracker, i int, emit func(VetRestriction)) {
+	rest := *rt.Rest
+	rest.Key = nil
+	rest.Val = i
+	rest.ProcessElm = true
+	emit(rest)
 }
 
-type KvSdf struct {
+// VetKvSdf runs an SDF In order to test that these methods get called properly,
+// each method will flip the corresponding flag in the passed in VetRestriction,
+// overwrite the restriction's Key and Val with the last seen input elements,
+// and retain the other fields in the VetRestriction.
+type VetKvSdf struct {
 }
 
-// CreateInitialRestriction creates a restriction with the sum of the given
-// values.
-func (fn *KvSdf) CreateInitialRestriction(i int, j int) Restriction {
-	return Restriction{i + j}
+// CreateInitialRestriction creates a restriction with the given values and
+// with the appropriate flags to track that this was called.
+func (fn *VetKvSdf) CreateInitialRestriction(i, j int) *VetRestriction {
+	return &VetRestriction{ID: "KvSdf", Key: i, Val: j, CreateRest: true}
 }
 
-// SplitRestriction outputs two restrictions, the first containing the sum of i
-// and rest.Val, the second containing the sum of j and rest.Val.
-func (fn *KvSdf) SplitRestriction(i int, j int, rest Restriction) []Restriction {
-	return []Restriction{{rest.Val + i}, {rest.Val + j}}
+// SplitRestriction outputs two identical restrictions, each being a copy of the
+// initial one, but with the appropriate flags to track this was called. The
+// split restrictions add a suffix of the form ".#" to the ID.
+func (fn *VetKvSdf) SplitRestriction(i, j int, rest *VetRestriction) []*VetRestriction {
+	rest.SplitRest = true
+	rest1 := &VetRestriction{
+		ID:            rest.ID + ".1",
+		Key:           i,
+		Val:           j,
+		CreateRest:    rest.CreateRest,
+		SplitRest:     true,
+		RestSize:      rest.RestSize,
+		CreateTracker: rest.CreateTracker,
+		ProcessElm:    rest.ProcessElm,
+	}
+	rest2 := &VetRestriction{}

Review comment:
       Consider having a copy() method on VetRestriction (on the value), which will make all these copy instances easier to read.

##########
File path: sdks/go/pkg/beam/core/runtime/exec/sdf_invokers_test.go
##########
@@ -210,86 +229,154 @@ func TestInvokes(t *testing.T) {
 	})
 }
 
-type Restriction struct {
-	Val int
+// VetRestriction is a restriction used for validating that SDF methods get
+// called with it. When using VetRestriction, the SDF methods it's used in
+// should pass it as a pointer so the method can make changes to the restriction
+// even if it doesn't output one directly (such as RestrictionSize).
+//
+type VetRestriction struct {
+	// An identifier to differentiate restrictions on the same elements. When
+	// split, a suffix in the form of ".#" is appended to this ID.
+	ID string
+
+	// Key and Val just copy the last seen input element's key and value to
+	// confirm that the restriction saw the expected element.
+	Key, Val interface{}
+
+	// These booleans should be flipped to true by the corresponding SDF methods
+	// to prove that the methods got called on the restriction.
+	CreateRest, SplitRest, RestSize, CreateTracker, ProcessElm bool
 }
 
-// RTracker's methods can all be no-ops, we just need it to implement sdf.RTracker.
-type RTracker struct {
-	Rest Restriction
-	Val  int
+// VetRTracker's methods can all be no-ops, we just need it to implement
+// sdf.RTracker and allow validating that it was passed to ProcessElement.
+type VetRTracker struct {
+	Rest *VetRestriction
 }
 
-func (rt *RTracker) TryClaim(interface{}) bool       { return false }
-func (rt *RTracker) GetError() error                 { return nil }
-func (rt *RTracker) GetProgress() (float64, float64) { return 0, 0 }
-func (rt *RTracker) IsDone() bool                    { return true }
-func (rt *RTracker) TrySplit(fraction float64) (interface{}, interface{}, error) {
+func (rt *VetRTracker) TryClaim(interface{}) bool       { return false }
+func (rt *VetRTracker) GetError() error                 { return nil }
+func (rt *VetRTracker) GetProgress() (float64, float64) { return 0, 0 }
+func (rt *VetRTracker) IsDone() bool                    { return true }
+func (rt *VetRTracker) TrySplit(fraction float64) (interface{}, interface{}, error) {
 	return nil, nil, nil
 }
 
-// In order to test that these methods get called properly, each one has an
-// implementation that lets us confirm that each argument was passed properly.
-
-type Sdf struct {
+// VetSdf runs an SDF In order to test that these methods get called properly,
+// each method will flip the corresponding flag in the passed in VetRestriction,
+// overwrite the restriction's Key and Val with the last seen input elements,
+// and retain the other fields in the VetRestriction.
+type VetSdf struct {
 }
 
-// CreateInitialRestriction creates a restriction with the given value.
-func (fn *Sdf) CreateInitialRestriction(i int) Restriction {
-	return Restriction{i}
+// CreateInitialRestriction creates a restriction with the given values and
+// with the appropriate flags to track that this was called.
+func (fn *VetSdf) CreateInitialRestriction(i int) *VetRestriction {
+	return &VetRestriction{ID: "Sdf", Val: i, CreateRest: true}
 }
 
-// SplitRestriction outputs two restrictions, the first containing the sum of i
-// and rest.Val, the second containing the same value plus 1.
-func (fn *Sdf) SplitRestriction(i int, rest Restriction) []Restriction {
-	return []Restriction{{rest.Val + i}, {rest.Val + i + 1}}
+// SplitRestriction outputs two identical restrictions, each being a copy of the
+// initial one, but with the appropriate flags to track this was called. The
+// split restrictions add a suffix of the form ".#" to the ID.
+func (fn *VetSdf) SplitRestriction(i int, rest *VetRestriction) []*VetRestriction {
+	rest.SplitRest = true
+	rest1 := &VetRestriction{
+		ID:            rest.ID + ".1",
+		Val:           i,
+		CreateRest:    rest.CreateRest,
+		SplitRest:     true,
+		RestSize:      rest.RestSize,
+		CreateTracker: rest.CreateTracker,
+		ProcessElm:    rest.ProcessElm,
+	}
+	rest2 := &VetRestriction{}
+	*rest2 = *rest1
+	rest2.ID = rest.ID + ".2"
+	return []*VetRestriction{rest1, rest2}
 }
 
-// RestrictionSize returns the sum of i and rest.Val as a float64.
-func (fn *Sdf) RestrictionSize(i int, rest Restriction) float64 {
-	return (float64)(i + rest.Val)
+// RestrictionSize just returns i as the size, as well as flipping appropriate
+// flags on the restriction to track that this was called.
+func (fn *VetSdf) RestrictionSize(i int, rest *VetRestriction) float64 {
+	rest.Key = nil
+	rest.Val = i
+	rest.RestSize = true
+	return (float64)(i)
 }
 
-// CreateTracker creates an RTracker containing the given restriction and a Val
-// of 1.
-func (fn *Sdf) CreateTracker(rest Restriction) *RTracker {
-	return &RTracker{rest, 1}
+// CreateTracker creates an RTracker containing the given restriction and flips
+// the appropriate flags on the restriction to track that this was called.
+func (fn *VetSdf) CreateTracker(rest *VetRestriction) *VetRTracker {
+	rest.CreateTracker = true
+	return &VetRTracker{rest}
 }
 
-// ProcessElement emits a pair of ints. The first is the input +
-// RTracker.Rest.Val. The second is the input + RTracker.Val.
-func (fn *Sdf) ProcessElement(rt *RTracker, i int, emit func(int, int)) {
-	emit(i+rt.Rest.Val, i+rt.Val)
+// ProcessElement emits a copy of the restriction in the restriction tracker it
+// received, with the appropriate flags flipped to track that this was called.
+func (fn *VetSdf) ProcessElement(rt *VetRTracker, i int, emit func(VetRestriction)) {
+	rest := *rt.Rest
+	rest.Key = nil
+	rest.Val = i
+	rest.ProcessElm = true
+	emit(rest)
 }
 
-type KvSdf struct {
+// VetKvSdf runs an SDF In order to test that these methods get called properly,
+// each method will flip the corresponding flag in the passed in VetRestriction,
+// overwrite the restriction's Key and Val with the last seen input elements,
+// and retain the other fields in the VetRestriction.
+type VetKvSdf struct {
 }
 
-// CreateInitialRestriction creates a restriction with the sum of the given
-// values.
-func (fn *KvSdf) CreateInitialRestriction(i int, j int) Restriction {
-	return Restriction{i + j}
+// CreateInitialRestriction creates a restriction with the given values and
+// with the appropriate flags to track that this was called.
+func (fn *VetKvSdf) CreateInitialRestriction(i, j int) *VetRestriction {
+	return &VetRestriction{ID: "KvSdf", Key: i, Val: j, CreateRest: true}
 }
 
-// SplitRestriction outputs two restrictions, the first containing the sum of i
-// and rest.Val, the second containing the sum of j and rest.Val.
-func (fn *KvSdf) SplitRestriction(i int, j int, rest Restriction) []Restriction {
-	return []Restriction{{rest.Val + i}, {rest.Val + j}}
+// SplitRestriction outputs two identical restrictions, each being a copy of the
+// initial one, but with the appropriate flags to track this was called. The
+// split restrictions add a suffix of the form ".#" to the ID.
+func (fn *VetKvSdf) SplitRestriction(i, j int, rest *VetRestriction) []*VetRestriction {
+	rest.SplitRest = true
+	rest1 := &VetRestriction{
+		ID:            rest.ID + ".1",
+		Key:           i,
+		Val:           j,
+		CreateRest:    rest.CreateRest,
+		SplitRest:     true,
+		RestSize:      rest.RestSize,
+		CreateTracker: rest.CreateTracker,
+		ProcessElm:    rest.ProcessElm,
+	}
+	rest2 := &VetRestriction{}
+	*rest2 = *rest1
+	rest2.ID = rest.ID + ".2"
+	return []*VetRestriction{rest1, rest2}
 }
 
-// RestrictionSize returns the sum of i, j, and rest.Val as a float64.
-func (fn *KvSdf) RestrictionSize(i int, j int, rest Restriction) float64 {
-	return (float64)(i + j + rest.Val)
+// RestrictionSize just returns the sum of i and j as the size, as well as
+// flipping appropriate flags on the restriction to track that this was called.
+func (fn *VetKvSdf) RestrictionSize(i, j int, rest *VetRestriction) float64 {
+	rest.Key = i
+	rest.Val = j
+	rest.RestSize = true
+	return (float64)(i + j)
 }
 
-// CreateTracker creates an RTracker containing the given restriction and a Val
-// of 2.
-func (fn *KvSdf) CreateTracker(rest Restriction) *RTracker {
-	return &RTracker{rest, 2}
+// CreateTracker creates an RTracker containing the given restriction and flips
+// the appropriate flags on the restriction to track that this was called.
+func (fn *VetKvSdf) CreateTracker(rest *VetRestriction) *VetRTracker {
+	rest.CreateTracker = true
+	return &VetRTracker{rest}
 }
 
-// ProcessElement emits two ints. The first is the first input (key) +
-// RTracker.Rest.Val. The second is the second input (value) + RTracker.Val.
-func (fn *KvSdf) ProcessElement(rt *RTracker, i1 int, i2 int, emit func(int, int)) {
-	emit(i1+rt.Rest.Val, i2+rt.Val)
+// ProcessElement emits a copy of the restriction in the restriction tracker it
+// received, with the appropriate flags flipped to track that this was called.
+func (fn *VetKvSdf) ProcessElement(rt *VetRTracker, i, j int, emit func(VetRestriction)) {

Review comment:
       It's a little awkward to have both Value instances of a type and Pointer instances of a type. Usually one is using one or the other. This makes it harder in to understand the test expectations, which if things need to change, could cause harder to diagnose errors.
   
   Since you're using pointers to allow for the  "was this specific instance processed?" consider consolidating on the pointer types.




----------------------------------------------------------------
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.

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