Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[BEAM-14406] Drain test for SDF truncation in Go SDK #17814

Merged
merged 4 commits into from
Jun 9, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
12 changes: 12 additions & 0 deletions sdks/go/test/integration/integration.go
Original file line number Diff line number Diff line change
Expand Up @@ -79,6 +79,8 @@ var directFilters = []string{
"TestLargeWordcount_Loopback",
// The direct runner does not support self-checkpointing
"TestCheckpointing",
// The direct runner does not support pipeline drain for SDF.
"TestDrain",
}

var portableFilters = []string{
Expand All @@ -91,6 +93,8 @@ var portableFilters = []string{
"TestKafkaIO.*",
// The portable runner does not support self-checkpointing
"TestCheckpointing",
// The portable runner does not support pipeline drain for SDF.
"TestDrain",
}

var flinkFilters = []string{
Expand All @@ -101,6 +105,8 @@ var flinkFilters = []string{
"TestDebeziumIO_BasicRead",
// Triggers are not yet supported
"TestTrigger.*",
// The flink runner does not support pipeline drain for SDF.
"TestDrain",
}

var samzaFilters = []string{
Expand All @@ -116,6 +122,8 @@ var samzaFilters = []string{
"TestWordCount.*",
// The Samza runner does not support self-checkpointing
"TestCheckpointing",
// The samza runner does not support pipeline drain for SDF.
"TestDrain",
}

var sparkFilters = []string{
Expand All @@ -132,6 +140,8 @@ var sparkFilters = []string{
"TestDebeziumIO_BasicRead",
// The spark runner does not support self-checkpointing
"TestCheckpointing",
// The spark runner does not support pipeline drain for SDF.
"TestDrain",
}

var dataflowFilters = []string{
Expand All @@ -154,6 +164,8 @@ var dataflowFilters = []string{
// Dataflow does not automatically terminate the TestCheckpointing pipeline when
// complete.
"TestCheckpointing",
// Dataflow does not drain jobs by itself.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'll note that we can add a programatic bit that could drain the job automatically, in particular for this test. But that's a fair amount of work, since we'd need to provide access to the JobID, and the Dataflow client, via some kind of "running job" wrapper.

"TestDrain",
}

// CheckFilters checks if an integration test is filtered to be skipped, either
Expand Down
129 changes: 129 additions & 0 deletions sdks/go/test/integration/primitives/drain.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,129 @@
// 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 (
"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"
"math"
"reflect"
)

import (
"context"
"time"

"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
)

func init() {
beam.RegisterType(reflect.TypeOf((*TruncateFn)(nil)).Elem())
}

// RangeEstimator implements the offsetrange.RangeEndEstimator interface.
// It provides the estimated end for a restriction.
type RangeEstimator struct {
end int64
}

// Estimate returns the estimated end.
func (r *RangeEstimator) Estimate() int64 {
return r.end
}

// SetEstimate sets the estimated end.
func (r *RangeEstimator) SetEstimate(estimate int64) {
r.end = estimate
}

// TruncateFn is an SDF.
type TruncateFn struct {
Estimator RangeEstimator
}

// CreateInitialRestriction creates an initial restriction
func (fn *TruncateFn) CreateInitialRestriction(_ []byte) offsetrange.Restriction {
return offsetrange.Restriction{
Start: int64(1),
End: int64(math.MaxInt64),
}
}

// CreateTracker wraps the given restriction into a LockRTracker type.
func (fn *TruncateFn) CreateTracker(rest offsetrange.Restriction) *sdf.LockRTracker {
fn.Estimator = RangeEstimator{int64(10)}
tracker, err := offsetrange.NewGrowableTracker(rest, &fn.Estimator)
if err != nil {
panic(err)
}
return sdf.NewLockRTracker(tracker)
}

// RestrictionSize returns the size of the current restriction
func (fn *TruncateFn) RestrictionSize(_ []byte, rest offsetrange.Restriction) float64 {
return rest.Size()
}

// SplitRestriction is similar to the one used in checkpointing.go test.
func (fn *TruncateFn) SplitRestriction(_ []byte, rest offsetrange.Restriction) []offsetrange.Restriction {
return rest.EvenSplits(2)
}

// TruncateRestriction truncates the restriction during drain.
func (fn *TruncateFn) TruncateRestriction(rt *sdf.LockRTracker, _ []byte) offsetrange.Restriction {
start := rt.GetRestriction().(offsetrange.Restriction).Start
newEnd := start + 20
return offsetrange.Restriction{
Start: start,
End: newEnd,
}
}

// ProcessElement continually gets the start position of the restriction and emits the element as it is.
func (fn *TruncateFn) ProcessElement(rt *sdf.LockRTracker, _ []byte, emit func(int64)) sdf.ProcessContinuation {
position := rt.GetRestriction().(offsetrange.Restriction).Start
counter := 0
for {
if rt.TryClaim(position) {
// Successful claim, emit the value and move on.
emit(position)
position++
counter++
} else if rt.GetError() != nil || rt.IsDone() {
// Stop processing on error or completion
if err := rt.GetError(); err != nil {
log.Errorf(context.Background(), "error in restriction tracker, got %v", err)
}
return sdf.StopProcessing()
} else {
// Resume later.
return sdf.ResumeProcessingIn(5 * time.Second)
}

if counter >= 10 {
return sdf.ResumeProcessingIn(1 * time.Second)
}
time.Sleep(1 * time.Second)
}
}

// Drain tests the SDF truncation during drain.
func Drain(s beam.Scope) {
beam.Init()
s.Scope("truncate")
beam.ParDo(s, &TruncateFn{}, beam.Impulse(s))
}
34 changes: 34 additions & 0 deletions sdks/go/test/integration/primitives/drain_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,34 @@
// 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"

"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/test/integration"
)

func TestDrain(t *testing.T) {
integration.CheckFilters(t)
p, s := beam.NewPipelineWithRoot()
Drain(s)
_, err := ptest.RunWithMetrics(p)
if err != nil {
t.Errorf("Drain test failed: %v", err)
}
}