This is an automated email from the ASF dual-hosted git repository.

lostluck pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/beam.git


The following commit(s) were added to refs/heads/master by this push:
     new 2a8a1ac0f2d [BEAM-14259] Clean up staticcheck warnings in the exec 
package (#17285)
2a8a1ac0f2d is described below

commit 2a8a1ac0f2db04c64b9e655a73b673e61e02de2c
Author: Jack McCluskey <34928439+jrmcclus...@users.noreply.github.com>
AuthorDate: Wed Apr 6 13:34:26 2022 -0400

    [BEAM-14259] Clean up staticcheck warnings in the exec package (#17285)
---
 sdks/go/pkg/beam/core/runtime/exec/coder.go          |  4 ++--
 sdks/go/pkg/beam/core/runtime/exec/datasource.go     |  2 +-
 sdks/go/pkg/beam/core/runtime/exec/dynsplit_test.go  | 16 ++++++++--------
 sdks/go/pkg/beam/core/runtime/exec/fn_test.go        |  6 ++----
 sdks/go/pkg/beam/core/runtime/exec/fullvalue_test.go |  6 ------
 sdks/go/pkg/beam/core/runtime/exec/pardo.go          |  2 +-
 sdks/go/pkg/beam/core/runtime/exec/pardo_test.go     |  4 ----
 7 files changed, 14 insertions(+), 26 deletions(-)

diff --git a/sdks/go/pkg/beam/core/runtime/exec/coder.go 
b/sdks/go/pkg/beam/core/runtime/exec/coder.go
index 145209a492c..75cb1cca231 100644
--- a/sdks/go/pkg/beam/core/runtime/exec/coder.go
+++ b/sdks/go/pkg/beam/core/runtime/exec/coder.go
@@ -336,7 +336,7 @@ type boolDecoder struct{}
 
 func (*boolDecoder) DecodeTo(r io.Reader, fv *FullValue) error {
        // Encoding: false = 0, true = 1
-       b := make([]byte, 1, 1)
+       b := make([]byte, 1)
        if err := ioutilx.ReadNBufUnsafe(r, b); err != nil {
                if err == io.EOF {
                        return err
@@ -1148,7 +1148,7 @@ func (d *intervalWindowDecoder) Decode(r io.Reader) 
([]typex.Window, error) {
 
        n, err := coder.DecodeInt32(r) // #windows
 
-       ret := make([]typex.Window, n, n)
+       ret := make([]typex.Window, n)
        for i := int32(0); i < n; i++ {
                w, err := d.DecodeSingle(r)
                if err != nil {
diff --git a/sdks/go/pkg/beam/core/runtime/exec/datasource.go 
b/sdks/go/pkg/beam/core/runtime/exec/datasource.go
index 2e8f25d41d3..4b7ada9995a 100644
--- a/sdks/go/pkg/beam/core/runtime/exec/datasource.go
+++ b/sdks/go/pkg/beam/core/runtime/exec/datasource.go
@@ -62,7 +62,7 @@ func (n *DataSource) InitSplittable() {
        if n.Out == nil {
                return
        }
-       if u, ok := n.Out.(*ProcessSizedElementsAndRestrictions); ok == true {
+       if u, ok := n.Out.(*ProcessSizedElementsAndRestrictions); ok {
                n.su = u.SU
        }
 }
diff --git a/sdks/go/pkg/beam/core/runtime/exec/dynsplit_test.go 
b/sdks/go/pkg/beam/core/runtime/exec/dynsplit_test.go
index 2c87a3be29c..64355a84c1f 100644
--- a/sdks/go/pkg/beam/core/runtime/exec/dynsplit_test.go
+++ b/sdks/go/pkg/beam/core/runtime/exec/dynsplit_test.go
@@ -44,7 +44,7 @@ func TestDynamicSplit(t *testing.T) {
                name string
                // driver is a function determining how the processing and 
splitting
                // threads are created and coordinated.
-               driver func(*Plan, DataContext, *splitTestSdf) (error, 
splitResult)
+               driver func(*Plan, DataContext, *splitTestSdf) (splitResult, 
error)
        }{
                {
                        // Complete a split before beginning processing.
@@ -81,7 +81,7 @@ func TestDynamicSplit(t *testing.T) {
                        dc := DataContext{Data: &TestDataManager{R: pr}}
 
                        // Call driver to coordinate processing & splitting 
threads.
-                       procRes, splitRes := test.driver(plan, dc, sdf)
+                       splitRes, procRes := test.driver(plan, dc, sdf)
 
                        // Validate we get a valid split result, aside from 
split elements.
                        if splitRes.err != nil {
@@ -141,7 +141,7 @@ func TestDynamicSplit(t *testing.T) {
 
 // nonBlockingDriver performs a split before starting processing, so no thread
 // is forced to wait on a mutex.
-func nonBlockingDriver(plan *Plan, dc DataContext, sdf *splitTestSdf) (procRes 
error, splitRes splitResult) {
+func nonBlockingDriver(plan *Plan, dc DataContext, sdf *splitTestSdf) 
(splitRes splitResult, procRes error) {
        // Begin processing pipeline.
        procResCh := make(chan error)
        go processPlan(plan, dc, procResCh)
@@ -161,12 +161,12 @@ func nonBlockingDriver(plan *Plan, dc DataContext, sdf 
*splitTestSdf) (procRes e
        <-rt.endClaim
        procRes = <-procResCh
 
-       return procRes, splitRes
+       return splitRes, procRes
 }
 
 // splitBlockingDriver blocks on a split request so that the SDF attempts to
 // claim while the split is occurring.
-func splitBlockingDriver(plan *Plan, dc DataContext, sdf *splitTestSdf) 
(procRes error, splitRes splitResult) {
+func splitBlockingDriver(plan *Plan, dc DataContext, sdf *splitTestSdf) 
(splitRes splitResult, procRes error) {
        // Begin processing pipeline.
        procResCh := make(chan error)
        go processPlan(plan, dc, procResCh)
@@ -190,12 +190,12 @@ func splitBlockingDriver(plan *Plan, dc DataContext, sdf 
*splitTestSdf) (procRes
        <-rt.endClaim
        procRes = <-procResCh
 
-       return procRes, splitRes
+       return splitRes, procRes
 }
 
 // claimBlockingDriver blocks on a claim request so that the SDF attempts to
 // split while the claim is occurring.
-func claimBlockingDriver(plan *Plan, dc DataContext, sdf *splitTestSdf) 
(procRes error, splitRes splitResult) {
+func claimBlockingDriver(plan *Plan, dc DataContext, sdf *splitTestSdf) 
(splitRes splitResult, procRes error) {
        // Begin processing pipeline.
        procResCh := make(chan error)
        go processPlan(plan, dc, procResCh)
@@ -219,7 +219,7 @@ func claimBlockingDriver(plan *Plan, dc DataContext, sdf 
*splitTestSdf) (procRes
        <-rt.endClaim // Delay the claim end so we don't process too much 
before splitting.
        procRes = <-procResCh
 
-       return procRes, splitRes
+       return splitRes, procRes
 }
 
 // createElm creates the element for our test pipeline.
diff --git a/sdks/go/pkg/beam/core/runtime/exec/fn_test.go 
b/sdks/go/pkg/beam/core/runtime/exec/fn_test.go
index b4db872395c..a2223c1ae02 100644
--- a/sdks/go/pkg/beam/core/runtime/exec/fn_test.go
+++ b/sdks/go/pkg/beam/core/runtime/exec/fn_test.go
@@ -30,8 +30,6 @@ import (
        "github.com/apache/beam/sdks/v2/go/pkg/beam/core/util/reflectx"
 )
 
-type testInt int32
-
 // TestInvoke verifies the the various forms of input to Invoke are handled 
correctly.
 func TestInvoke(t *testing.T) {
        tests := []struct {
@@ -441,7 +439,7 @@ func BenchmarkReflectCallNewArgs(b *testing.B) {
        fn := reflect.ValueOf(inc)
        n := reflect.ValueOf(0)
        for i := 0; i < b.N; i++ {
-               args := make([]reflect.Value, 1, 1)
+               args := make([]reflect.Value, 1)
                args[0] = n
                n = fn.Call(args)[0]
        }
@@ -451,7 +449,7 @@ func BenchmarkReflectCallNewArgs(b *testing.B) {
 func BenchmarkReflectCallReuseArgs(b *testing.B) {
        fn := reflect.ValueOf(inc)
        n := reflect.ValueOf(0)
-       args := make([]reflect.Value, 1, 1)
+       args := make([]reflect.Value, 1)
        for i := 0; i < b.N; i++ {
                args[0] = n
                n = fn.Call(args)[0]
diff --git a/sdks/go/pkg/beam/core/runtime/exec/fullvalue_test.go 
b/sdks/go/pkg/beam/core/runtime/exec/fullvalue_test.go
index a3216fc194e..d1e62dff1d7 100644
--- a/sdks/go/pkg/beam/core/runtime/exec/fullvalue_test.go
+++ b/sdks/go/pkg/beam/core/runtime/exec/fullvalue_test.go
@@ -183,12 +183,6 @@ func equal(a, b FullValue) bool {
        return true
 }
 
-type testStruct struct {
-       a int
-}
-
-func (*testStruct) M() {}
-
 // Conversion tests.
 func TestConvert(t *testing.T) {
        tests := []struct {
diff --git a/sdks/go/pkg/beam/core/runtime/exec/pardo.go 
b/sdks/go/pkg/beam/core/runtime/exec/pardo.go
index aefcd57e0ed..1c6aaebd2ae 100644
--- a/sdks/go/pkg/beam/core/runtime/exec/pardo.go
+++ b/sdks/go/pkg/beam/core/runtime/exec/pardo.go
@@ -246,7 +246,7 @@ func (n *ParDo) initSideInput(ctx context.Context, w 
typex.Window) error {
 
                n.cache = &cacheElm{
                        key:   w,
-                       extra: make([]interface{}, sideCount+emitCount, 
sideCount+emitCount),
+                       extra: make([]interface{}, sideCount+emitCount),
                }
                for i, emit := range n.emitters {
                        n.cache.extra[i+sideCount] = emit.Value()
diff --git a/sdks/go/pkg/beam/core/runtime/exec/pardo_test.go 
b/sdks/go/pkg/beam/core/runtime/exec/pardo_test.go
index f277aefd511..d7a9bb4b100 100644
--- a/sdks/go/pkg/beam/core/runtime/exec/pardo_test.go
+++ b/sdks/go/pkg/beam/core/runtime/exec/pardo_test.go
@@ -187,10 +187,6 @@ func BenchmarkParDo_EmitSumFn(b *testing.B) {
        if err != nil {
                b.Fatalf("invalid pardo: %v", err)
        }
-       var in []int
-       for i := 0; i < b.N; i++ {
-               in = append(in)
-       }
 
        process := make(chan MainInput, 1)
        errchan := make(chan string, 1)

Reply via email to