From b3d21b26d0c7368b49552b24a884d2212b78db35 Mon Sep 17 00:00:00 2001 From: Renato Costa Date: Fri, 27 May 2022 10:29:52 -0400 Subject: [PATCH] lint: add loopvarcapture linter This commit introduces a new linter: `loopvarcapture`. It reports uses of loop variables captured by reference in Go routines or defer statements, a common source of data races [1]. `govet` currently has a similar linter [2]; however, that project prioritizes having no false positives at the expense of allowing false negatives. This linter, on the other hand, represents the opinion that loop variables should not be captured by reference in Go routines even when it's safe to do so. That behavior is confusing and concurrency added to related code over time could lead to the introduction of data races, potentially manifesting as bugs in the product or flakiness in the tests. These issues are hard to debug and take a lot of developer time. Developers are still able to use their own judgement and disable this linter in specific instances by using a `nolint` comment. [1] A Study of Real-World Data Races in Golang: https://arxiv.org/pdf/2204.00764.pdf [2] https://github.com/golangci/govet/blob/44ddbe260190d79165f4150b828650780405d801/rangeloop.go#L36 Resolves: #80803. Release note: None. --- BUILD.bazel | 1 + pkg/BUILD.bazel | 1 + pkg/cmd/roachvet/BUILD.bazel | 2 +- pkg/cmd/roachvet/main.go | 6 +- .../lint/passes/loopvarcapture/BUILD.bazel | 33 ++ .../lint/passes/loopvarcapture/loop.go | 74 ++++ .../passes/loopvarcapture/loopvarcapture.go | 401 ++++++++++++++++++ .../loopvarcapture/loopvarcapture_test.go | 45 ++ .../example.org/concurrency/concurrency.go | 39 ++ .../golang.org/x/sync/errgroup/errgroup.go | 17 + .../passes/loopvarcapture/testdata/src/p/p.go | 350 +++++++++++++++ vendor | 2 +- 12 files changed, 967 insertions(+), 4 deletions(-) create mode 100644 pkg/testutils/lint/passes/loopvarcapture/BUILD.bazel create mode 100644 pkg/testutils/lint/passes/loopvarcapture/loop.go create mode 100644 pkg/testutils/lint/passes/loopvarcapture/loopvarcapture.go create mode 100644 pkg/testutils/lint/passes/loopvarcapture/loopvarcapture_test.go create mode 100644 pkg/testutils/lint/passes/loopvarcapture/testdata/src/example.org/concurrency/concurrency.go create mode 100644 pkg/testutils/lint/passes/loopvarcapture/testdata/src/golang.org/x/sync/errgroup/errgroup.go create mode 100644 pkg/testutils/lint/passes/loopvarcapture/testdata/src/p/p.go diff --git a/BUILD.bazel b/BUILD.bazel index 2b362408bd9a..55728cace5cc 100644 --- a/BUILD.bazel +++ b/BUILD.bazel @@ -189,6 +189,7 @@ nogo( "//pkg/testutils/lint/passes/grpcstatuswithdetails", "//pkg/testutils/lint/passes/hash", "//pkg/testutils/lint/passes/leaktestcall", + "//pkg/testutils/lint/passes/loopvarcapture", "//pkg/testutils/lint/passes/nilness", "//pkg/testutils/lint/passes/nocopy", "//pkg/testutils/lint/passes/returncheck", diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index bcb4007fd292..714ee1a92633 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -437,6 +437,7 @@ ALL_TESTS = [ "//pkg/testutils/lint/passes/forbiddenmethod:forbiddenmethod_test", "//pkg/testutils/lint/passes/hash:hash_test", "//pkg/testutils/lint/passes/leaktestcall:leaktestcall_test", + "//pkg/testutils/lint/passes/loopvarcapture:loopvarcapture_test", "//pkg/testutils/lint/passes/nilness:nilness_test", "//pkg/testutils/lint/passes/nocopy:nocopy_test", "//pkg/testutils/lint/passes/passesutil:passesutil_test", diff --git a/pkg/cmd/roachvet/BUILD.bazel b/pkg/cmd/roachvet/BUILD.bazel index f9b2ca205862..15f623cdb0cb 100644 --- a/pkg/cmd/roachvet/BUILD.bazel +++ b/pkg/cmd/roachvet/BUILD.bazel @@ -12,6 +12,7 @@ go_library( "//pkg/testutils/lint/passes/forbiddenmethod", "//pkg/testutils/lint/passes/hash", "//pkg/testutils/lint/passes/leaktestcall", + "//pkg/testutils/lint/passes/loopvarcapture", "//pkg/testutils/lint/passes/nilness", "//pkg/testutils/lint/passes/nocopy", "//pkg/testutils/lint/passes/returnerrcheck", @@ -28,7 +29,6 @@ go_library( "@org_golang_x_tools//go/analysis/passes/copylock", "@org_golang_x_tools//go/analysis/passes/errorsas", "@org_golang_x_tools//go/analysis/passes/httpresponse", - "@org_golang_x_tools//go/analysis/passes/loopclosure", "@org_golang_x_tools//go/analysis/passes/lostcancel", "@org_golang_x_tools//go/analysis/passes/nilfunc", "@org_golang_x_tools//go/analysis/passes/printf", diff --git a/pkg/cmd/roachvet/main.go b/pkg/cmd/roachvet/main.go index 40fb3e5010d9..e0f6a6b08052 100644 --- a/pkg/cmd/roachvet/main.go +++ b/pkg/cmd/roachvet/main.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/forbiddenmethod" "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/hash" "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/leaktestcall" + "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/loopvarcapture" "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/nilness" "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/nocopy" "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/returnerrcheck" @@ -36,7 +37,6 @@ import ( "golang.org/x/tools/go/analysis/passes/copylock" "golang.org/x/tools/go/analysis/passes/errorsas" "golang.org/x/tools/go/analysis/passes/httpresponse" - "golang.org/x/tools/go/analysis/passes/loopclosure" "golang.org/x/tools/go/analysis/passes/lostcancel" "golang.org/x/tools/go/analysis/passes/nilfunc" "golang.org/x/tools/go/analysis/passes/printf" @@ -67,6 +67,7 @@ func main() { errcmp.Analyzer, nilness.Analyzer, errwrap.Analyzer, + loopvarcapture.Analyzer, ) // Standard go vet analyzers: @@ -81,7 +82,8 @@ func main() { copylock.Analyzer, errorsas.Analyzer, httpresponse.Analyzer, - loopclosure.Analyzer, + // loopclosure.Analyzer, + // loopclosure is superseded by 'loopvarcapture' lostcancel.Analyzer, nilfunc.Analyzer, printf.Analyzer, diff --git a/pkg/testutils/lint/passes/loopvarcapture/BUILD.bazel b/pkg/testutils/lint/passes/loopvarcapture/BUILD.bazel new file mode 100644 index 000000000000..1509a78fd2ca --- /dev/null +++ b/pkg/testutils/lint/passes/loopvarcapture/BUILD.bazel @@ -0,0 +1,33 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "loopvarcapture", + srcs = [ + "loop.go", + "loopvarcapture.go", + ], + importpath = "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/loopvarcapture", + visibility = ["//visibility:public"], + deps = [ + "//pkg/testutils/lint/passes/passesutil", + "@org_golang_x_tools//go/analysis", + "@org_golang_x_tools//go/analysis/passes/inspect", + "@org_golang_x_tools//go/ast/inspector", + "@org_golang_x_tools//go/types/typeutil", + ], +) + +go_test( + name = "loopvarcapture_test", + srcs = ["loopvarcapture_test.go"], + data = glob(["testdata/**"]) + [ + "@go_sdk//:files", + ], + deps = [ + ":loopvarcapture", + "//pkg/build/bazel", + "//pkg/testutils", + "//pkg/testutils/skip", + "@org_golang_x_tools//go/analysis/analysistest", + ], +) diff --git a/pkg/testutils/lint/passes/loopvarcapture/loop.go b/pkg/testutils/lint/passes/loopvarcapture/loop.go new file mode 100644 index 000000000000..ab6c2944ce82 --- /dev/null +++ b/pkg/testutils/lint/passes/loopvarcapture/loop.go @@ -0,0 +1,74 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package loopvarcapture + +import ( + "fmt" + "go/ast" +) + +// Loop abstracts away the type of loop (`for` loop with index +// variable vs `range` loops) +type Loop struct { + Vars []*ast.Ident + Body *ast.BlockStmt +} + +// NewLoop creates a new Loop struct according to the node passed. If +// the node does not represent either a `for` loop or a `range` loop, +// this function will panic. +func NewLoop(n ast.Node) *Loop { + switch node := n.(type) { + case *ast.ForStmt: + return newForLoop(node) + case *ast.RangeStmt: + return newRange(node) + default: + panic(fmt.Errorf("unexpected loop node: %#v", n)) + } +} + +// IsEmpty returns whether the loop is empty for the purposes of this +// linter; in other words, whether there no loop variables, or whether +// the loop has zero statements. +func (l *Loop) IsEmpty() bool { + return len(l.Vars) == 0 || len(l.Body.List) == 0 +} + +func newForLoop(stmt *ast.ForStmt) *Loop { + loop := Loop{Body: stmt.Body} + + switch post := stmt.Post.(type) { + case *ast.AssignStmt: + for _, lhs := range post.Lhs { + loop.addVar(lhs) + } + + case *ast.IncDecStmt: + loop.addVar(post.X) + } + + return &loop +} + +func newRange(stmt *ast.RangeStmt) *Loop { + loop := Loop{Body: stmt.Body} + loop.addVar(stmt.Key) + loop.addVar(stmt.Value) + + return &loop +} + +func (l *Loop) addVar(e ast.Expr) { + if ident, ok := e.(*ast.Ident); ok && ident.Obj != nil { + l.Vars = append(l.Vars, ident) + } +} diff --git a/pkg/testutils/lint/passes/loopvarcapture/loopvarcapture.go b/pkg/testutils/lint/passes/loopvarcapture/loopvarcapture.go new file mode 100644 index 000000000000..a6d465f16fe6 --- /dev/null +++ b/pkg/testutils/lint/passes/loopvarcapture/loopvarcapture.go @@ -0,0 +1,401 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package loopvarcapture + +import ( + "fmt" + "go/ast" + "go/types" + "strings" + + "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/passesutil" + "golang.org/x/tools/go/analysis" + "golang.org/x/tools/go/analysis/passes/inspect" + astinspector "golang.org/x/tools/go/ast/inspector" + "golang.org/x/tools/go/types/typeutil" +) + +type ( + // statementType indicates which type of statement (`go` or `defer`) + // incorrectly captures a loop variable. + statementType int + + // Function defines the location of a function (package-level or + // method on a type). + Function struct { + Pkg string + Type string // empty for package-level functions + Name string + } +) + +const ( + name = "loopvarcapture" + + doc = `check for loop variables captured by reference in Go routines +or defer calls.` + + goCall = statementType(iota) + deferCall +) + +var ( + // Analyzer implements this linter, looking for loop variables + // captured by reference in closures called in Go routines + Analyzer = &analysis.Analyzer{ + Name: name, + Doc: doc, + Requires: []*analysis.Analyzer{inspect.Analyzer}, + Run: run, + } + + // GoRoutineFunctions is a collection of functions that are known to + // take closures as parameters and invoke them asynchronously (in a + // Go routine). Calling these functions should be equivalent to + // using the `go` keyword in this linter. + GoRoutineFunctions = []Function{ + {Pkg: "golang.org/x/sync/errgroup", Type: "Group", Name: "Go"}, + {Pkg: "github.com/cockroachdb/cockroach/pkg/util/ctxgroup", Type: "Group", Name: "Go"}, + {Pkg: "github.com/cockroachdb/cockroach/pkg/util/ctxgroup", Type: "Group", Name: "GoCtx"}, + {Pkg: "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/cluster", Type: "Monitor", Name: "Go"}, + } +) + +// run is the linter entrypoint +func run(pass *analysis.Pass) (interface{}, error) { + inspector := pass.ResultOf[inspect.Analyzer].(*astinspector.Inspector) + loops := []ast.Node{ + (*ast.RangeStmt)(nil), + (*ast.ForStmt)(nil), + } + + // visit every `for` and `range` loops; when a loop is found, + // instantiate a new `Visitor` that is reponsible for finding + // references to loop variables captured by reference in Go + // routines. + inspector.Preorder(loops, func(n ast.Node) { + loop := NewLoop(n) + if loop.IsEmpty() { + return + } + + v := NewVisitor(pass, loop) + for _, issue := range v.FindCaptures() { + pass.Report(issue) + } + }) + + return nil, nil +} + +// Visitor implements the logic of checking for use of loop variables +// in Go routines either directly (referencing a loop variable in the +// function literal passed to `go`) or indirectly (calling a local +// function that captures loop variables by reference). +type Visitor struct { + loop *Loop + pass *analysis.Pass + + // closures maps a closure assigned to a variable to the + // captured-by-reference loop variable. + closures map[*ast.Object]*ast.Ident + // issues accumulates issues found in a loop + issues []analysis.Diagnostic +} + +// NewVisitor creates a new Visitor instance for the given loop. +func NewVisitor(pass *analysis.Pass, loop *Loop) *Visitor { + return &Visitor{ + loop: loop, + pass: pass, + closures: map[*ast.Object]*ast.Ident{}, + } +} + +// FindCaptures returns a list of Diagnostic instances to be reported +// to the user +func (v *Visitor) FindCaptures() []analysis.Diagnostic { + ast.Inspect(v.loop.Body, v.visitLoopBody) + return v.issues +} + +// visitLoopBody ignores everything but `go` (and GoRoutineFunctions), +// `defer`, and assignment statements. +// +// When an assignment to a closure (function literal) is found, we +// check if the closure captures any of the loop variables; in case it +// does, the `closures` map is updated. +// +// When a `go`, a call to a GoRoutineFunction, or `defer` statement is +// found, we look for closures in either the function being called +// itself, or in parameters in the function call. +// +// In other words, both of the following scenarios are problematic and +// reported by this linter: +// +// 1: +// for k, v := range myMap { +// // same for `defer`, errgroup.Group.Go(), etc +// go func() { +// fmt.Printf("k = %v, v = %v\n", k, v) +// }() +// } +// +// 2: +// for k, v := range myMap { +// // same for `defer`, errgroup.Group.Go(), etc +// go doWork(func() { +// doMoreWork(k, v) +// }) +// } +// +// If a `go` routine (or `defer`) calls a previously-defined closure +// that captures a loop variable, that is also reported. +func (v *Visitor) visitLoopBody(n ast.Node) bool { + switch node := n.(type) { + case *ast.GoStmt: + v.visitCallExpr(goCall, node.Call) + // no need to keep traversing the AST, the function above is + // already doing that. + return false + + case *ast.CallExpr: + if v.isGoRoutineFunction(node) { + v.visitCallExpr(goCall, node) + } + + // keep traversing the AST, as there could be problematic + // references in the parameters passed to the function + return true + + case *ast.DeferStmt: + v.visitCallExpr(deferCall, node.Call) + // no need to keep traversing the AST, the function above is + // already doing that. + return false + + case *ast.AssignStmt: + for i, rhs := range node.Rhs { + lhs, ok := node.Lhs[i].(*ast.Ident) + if !ok || lhs.Obj == nil { + continue + } + + // inspect closure's body, looking for captured variables; if + // found, store the mapping below. + ast.Inspect(rhs, v.funcLitInspector(func(id *ast.Ident) { + v.closures[lhs.Obj] = id + })) + + // keep traversing the AST, as there could be invalid function + // calls that should be detected (one of GoRoutineFunctions) + return true + } + } + + // if the node is none of the above, keep traversing the AST + return true +} + +// visitCallExpr inspects function calls passed to `go` or `defer` +// staments, looking for closures that capture loop variables by +// reference in the body of the closure or in any of the arguments +// passed to it. +func (v *Visitor) visitCallExpr(stmtType statementType, call *ast.CallExpr) { + ast.Inspect(call, v.funcLitInspector(func(ident *ast.Ident) { + v.addIssue(stmtType, ident) + })) + + if funcName, ok := call.Fun.(*ast.Ident); ok { + if _, ok := v.closures[funcName.Obj]; ok { + v.addIssue(stmtType, funcName) + } + } +} + +// funcLitInspector returns a function that can be passed to +// `ast.Inspect`. When a closure (function literal) that references a +// loop variable is found, the `onLoopVarCapture` function is called. +func (v *Visitor) funcLitInspector(onLoopVarCapture func(*ast.Ident)) func(ast.Node) bool { + return func(n ast.Node) bool { + funcLit, ok := n.(*ast.FuncLit) + if !ok { + // not a function literal -- keep traversing the AST + return true + } + + // inspect the closure's body, calling the `onLoopVarCapture` + // function when a reference to a loop variable is found + ast.Inspect(funcLit.Body, v.findLoopVariableReferences(onLoopVarCapture)) + return false + } +} + +// findLoopVariableReferences inspects a closure's body. When a +// reference to a loop variable is found, or when a function that is +// known to capture a loop variable by reference is called, the +// `onLoopVarCapture` function passed is called (whether the capture +// is valid or not is determined by the caller). The return value of +// this function can be passed to `ast.Inspect`. +func (v *Visitor) findLoopVariableReferences( + onLoopVarCapture func(*ast.Ident), +) func(ast.Node) bool { + return func(n ast.Node) bool { + switch expr := n.(type) { + case *ast.Ident: + if expr.Obj == nil { + return true + } + + for _, loopVar := range v.loop.Vars { + // Comparing the *ast.Object associated with the identifiers + // frees us from having to keep tracking of shadowing. If the + // comparison below returns true, it means that the closure + // directly references a loop variable. + if expr.Obj == loopVar.Obj { + onLoopVarCapture(expr) + break + } + } + // `Ident` is a child node; stopping the traversal here + // shouldn't matter + return false + + case *ast.CallExpr: + funcName, ok := expr.Fun.(*ast.Ident) + if ok && funcName.Obj != nil { + if _, ok := v.closures[funcName.Obj]; ok { + onLoopVarCapture(funcName) + return false + } + } + + // if the function call is not to a closure that captures a loop + // variable, keep traversing the AST, as there could be invalid + // references down the subtree + return true + } + + // when the node being visited is not an identifier or a function + // call, keep traversing the AST + return true + } +} + +// addIssue adds a new issue in the `issues` field of the visitor +// associated with the identifier passed. The message is slightly +// different depending on whether the identifier is a loop variable +// directly, or invoking a closure that captures a loop variable by +// reference. In the latter case, the chain of calls that lead to the +// capture is included in the diagnostic. If a `//nolint` comment is +// associated with the use of this identifier, no issue is reported. +func (v *Visitor) addIssue(stmtType statementType, id *ast.Ident) { + if passesutil.HasNolintComment(v.pass, id, name) { + return + } + + var ( + chain = []*ast.Ident{id} + currentIdent = id + ok bool + ) + + for currentIdent, ok = v.closures[currentIdent.Obj]; ok; currentIdent, ok = v.closures[currentIdent.Obj] { + chain = append(chain, currentIdent) + } + + v.issues = append(v.issues, analysis.Diagnostic{ + Pos: id.Pos(), + Message: reportMessage(stmtType, chain), + }) +} + +// reportMessage constructs the message to be reported to the user +// based on the chain of identifiers that lead to the loop variable +// being captured. The last identifier in the chain is always the loop +// variable being captured; everything else is the chain of closure +// calls that lead to the capture. +func reportMessage(stmtType statementType, chain []*ast.Ident) string { + var suffixMsg string + if stmtType == goCall { + suffixMsg = "often leading to data races" + } else { + suffixMsg = "and may hold an undesirable value by the time the deferred function is called" + } + + if len(chain) == 1 { + return fmt.Sprintf("loop variable '%s' captured by reference, %s", chain[0].String(), suffixMsg) + } + + functionName := chain[0] + loopVar := chain[len(chain)-1] + + var path []string + for i := 1; i < len(chain)-1; i++ { + path = append(path, fmt.Sprintf("'%s'", chain[i].String())) + } + + var pathMsg string + if len(path) > 0 { + pathMsg = fmt.Sprintf(" (via %s)", strings.Join(path, " -> ")) + } + + return fmt.Sprintf( + "'%s' function captures loop variable '%s'%s by reference, %s", + functionName.String(), + loopVar.String(), + pathMsg, + suffixMsg, + ) +} + +// isGoRoutineFunction takes a call expression node and returns +// whether that call is being made to one of the functions in the +// GoRoutineFunctions slice. +func (v *Visitor) isGoRoutineFunction(call *ast.CallExpr) bool { + callee := typeutil.StaticCallee(v.pass.TypesInfo, call) + // call to a builtin + if callee == nil { + return false + } + pkg := callee.Pkg() + if pkg == nil { + return false + } + + calleePkg := pkg.Path() + calleeFunc := callee.Name() + calleeObj := "" + + recv := callee.Type().(*types.Signature).Recv() + if recv != nil { + // if there is a receiver (i.e., this is a method call), get the + // name of the type of the receiver + recvType := recv.Type() + if pointerType, ok := recvType.(*types.Pointer); ok { + recvType = pointerType.Elem() + } + named, ok := recvType.(*types.Named) + if !ok { + return false + } + + calleeObj = named.Obj().Name() + } + + for _, goFunc := range GoRoutineFunctions { + if goFunc.Pkg == calleePkg && goFunc.Type == calleeObj && goFunc.Name == calleeFunc { + return true + } + } + + return false +} diff --git a/pkg/testutils/lint/passes/loopvarcapture/loopvarcapture_test.go b/pkg/testutils/lint/passes/loopvarcapture/loopvarcapture_test.go new file mode 100644 index 000000000000..cbee7ddfcc22 --- /dev/null +++ b/pkg/testutils/lint/passes/loopvarcapture/loopvarcapture_test.go @@ -0,0 +1,45 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package loopvarcapture_test + +import ( + "testing" + + "github.com/cockroachdb/cockroach/pkg/build/bazel" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/loopvarcapture" + "github.com/cockroachdb/cockroach/pkg/testutils/skip" + "golang.org/x/tools/go/analysis/analysistest" +) + +var extraGoRoutineFunctions = []loopvarcapture.Function{ + {Pkg: "example.org/concurrency", Type: "Group", Name: "Go"}, // test non-pointer receiver + {Pkg: "example.org/concurrency", Name: "Go"}, // test a package-level function + {Pkg: "example.org/concurrency", Name: "GoWithError"}, // test a function with a return value +} + +func init() { + if bazel.BuiltWithBazel() { + bazel.SetGoEnv() + } +} + +func TestAnalyzer(t *testing.T) { + skip.UnderStress(t) + + originalGoRoutineFunctions := loopvarcapture.GoRoutineFunctions + loopvarcapture.GoRoutineFunctions = append(originalGoRoutineFunctions, extraGoRoutineFunctions...) + defer func() { loopvarcapture.GoRoutineFunctions = originalGoRoutineFunctions }() + + testdata := testutils.TestDataPath(t) + analysistest.TestData = func() string { return testdata } + analysistest.Run(t, testdata, loopvarcapture.Analyzer, "p") +} diff --git a/pkg/testutils/lint/passes/loopvarcapture/testdata/src/example.org/concurrency/concurrency.go b/pkg/testutils/lint/passes/loopvarcapture/testdata/src/example.org/concurrency/concurrency.go new file mode 100644 index 000000000000..5246fce93114 --- /dev/null +++ b/pkg/testutils/lint/passes/loopvarcapture/testdata/src/example.org/concurrency/concurrency.go @@ -0,0 +1,39 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package concurrency + +import ( + "fmt" + "math/rand" +) + +type Group struct{} + +func (g Group) Go(f func()) { + go f() +} + +func Go(f func()) { + go f() +} + +func GoWithError(f func()) error { + if rand.Float64() < 0.5 { + return fmt.Errorf("random error") + } + + go f() + return nil +} + +func SafeFunction(f func()) { + f() +} diff --git a/pkg/testutils/lint/passes/loopvarcapture/testdata/src/golang.org/x/sync/errgroup/errgroup.go b/pkg/testutils/lint/passes/loopvarcapture/testdata/src/golang.org/x/sync/errgroup/errgroup.go new file mode 100644 index 000000000000..28049379c1dd --- /dev/null +++ b/pkg/testutils/lint/passes/loopvarcapture/testdata/src/golang.org/x/sync/errgroup/errgroup.go @@ -0,0 +1,17 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package errgroup + +type Group struct{} + +func (g *Group) Go(f func() error) { + go f() +} diff --git a/pkg/testutils/lint/passes/loopvarcapture/testdata/src/p/p.go b/pkg/testutils/lint/passes/loopvarcapture/testdata/src/p/p.go new file mode 100644 index 000000000000..44b8c7b66e27 --- /dev/null +++ b/pkg/testutils/lint/passes/loopvarcapture/testdata/src/p/p.go @@ -0,0 +1,350 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package p + +import ( + "fmt" + "math/rand" + "net" + "sync" + "testing" + + "example.org/concurrency" + "golang.org/x/sync/errgroup" +) + +var ( + intID = func(n int) int { return n } + doWork = func() {} + runFunc = func(f func()) { f() } + + collection = []int{1, 2, 3} +) + +type MyStruct struct { + closure func() +} + +func OutOfScope() { + var i int + var s MyStruct + for j := range collection { + s.closure = func() { + fmt.Printf("captured: %d\n", j) + } + + i++ + go func() { + intID(i) // valid data race, but out of scope for this linter + + // valid data race, but we don't track assignments to struct + // fields right now: it would add complexity to the linter and + // it's a much less common pattern. + s.closure() + }() + } +} + +// TableDriven ensures that we are able to flag a common pattern in +// table-driven tests. If a Go routine is spawned while iterating over +// the test cases, it's easy to accidentally reference the test case +// variable, leading to flaky tests if the test cases run in parallel. +func TableDriven(t *testing.T) { + values := [][]byte{{0x08, 0x00, 0x00, 0xff, 0xff}} + + for _, tc := range values { + t.Run("", func(t *testing.T) { + w, _ := net.Pipe() + errChan := make(chan error, 1) + + go func() { + if _, err := w.Write(tc); err != nil { // want `loop variable 'tc' captured by reference` + errChan <- err + return + } + }() + }) + } +} + +// NestedLoops makes sure nested loops are supported, and all +// references to loop variables in inner or outer loops are detected. +func NestedLoops() { + for i, n := range collection { + go func() { + defer func() { + fmt.Printf("iter = %d\n", n) // want `loop variable 'n' captured by reference` + }() + doWork() + }() + + for j := range collection { + go func() { + doWork() + intID(j) // want `loop variable 'j' captured by reference` + doWork() + intID(n) // want `loop variable 'n' captured by reference` + doWork() + }() + + for k := j; k < len(collection); k++ { + go func(idx int) { + intID(k) // want `loop variable 'k' captured by reference` + intID(idx) // this is OK + intID(n) // want `loop variable 'n' captured by reference` + + if k > 0 { // want `loop variable 'k' captured by reference` + intID(j) // want `loop variable 'j' captured by reference` + } + }(i) + } + } + } +} + +// Conditional ensures that even when a Go routine is created in more +// syntactically complex subtrees, it's still flagged if it captures a +// loop variable. In this case, the code is technically safe since the +// Go routine is only created in the last iteration of the loop, but +// it is believed that the variable should not be captured either way +// to avoid the chance of introducing bugs when this code is changed +// (it's also not possible to statically determine when using a loop +// variable inside a Go routine is safe, so we err on the side of +// caution). +func Conditional() { + for i, n := range collection { + intID(n) + if i == len(collection)-1 { + go func() { + fmt.Printf("i = %d\n", i) // want `loop variable 'i' captured by reference` + }() + } + } + + for j := 0; j < 10; j++ { + go func() { + doWork() + fmt.Printf("done: %d\n", j) // want `loop variable 'j' captured by reference` + }() + } +} + +// FuncLitArg ensures that function literals (closures) passed as +// argument to a function call in a 'go' statement should also be +// flagged if they capture a loop variable. +func FuncLitArg() { + for _, n := range collection { + doWork() + go runFunc(func() { + intID(n) // want `loop variable 'n' captured by reference` + }) + + go intID(n) // this is OK + doWork() + } + + for j := 0; j < len(collection); j++ { + doWork() + go runFunc(func() { + intID(collection[j]) // want `loop variable 'j' captured by reference` + }) + + go intID(collection[j]) // this is OK + } +} + +// Synchronization is another example of a technically safe use of a +// loop variable in a Go routine that we decide to flag anyway. +func Synchronization() { + for _, n := range collection { + var wg sync.WaitGroup + go func() { + defer wg.Done() + intID(n) // want `loop variable 'n' captured by reference` + }() + + wg.Wait() + } +} + +// IndirectClosure makes sure that closures that capture loop +// variables cannot be called in a Go routine. +func IndirectClosure() { + for i := range collection { + badClosure := func() { fmt.Printf("finished iteration %d\n", i+1) } + goodClosure := func(i int) { fmt.Printf("finished iteration %d\n", i+1) } + + wrapper1 := func() { badClosure() } + wrapper2 := func() { wrapper1() } + wrapper3 := func() { goodClosure(i) } + + iCopy := i + go func() { + defer badClosure() // want `'badClosure' function captures loop variable 'i' by reference` + doWork() + + // referencing a closure without invoking it is fine + if badClosure != nil { + wrapper1() // want `'wrapper1' function captures loop variable 'i' \(via 'badClosure'\)` + doWork() + wrapper2() // want `'wrapper2' function captures loop variable 'i' \(via 'wrapper1' -> 'badClosure'\)` + + wrapper3() // want `'wrapper3' function captures loop variable 'i' by reference` + + // copying here does not solve the problem + k := i // want `loop variable 'i' captured by reference` + goodClosure(k) // still problematic + + goodClosure(iCopy) // this is OK + } + }() + + go badClosure() // want `'badClosure' function captures loop variable 'i' by reference` + go wrapper2() // want `'wrapper2' function captures loop variable 'i' \(via 'wrapper1' -> 'badClosure'\)` + } + + for j := 0; j < len(collection); j++ { + showProgress := func() { + fmt.Printf("finished iteration %d\n", j+1) + } + + go func() { + doWork() + showProgress() // want `'showProgress' function captures loop variable 'j' by reference` + }() + } +} + +// FixedFunction tests that common patterns to fix loop variable +// capture by reference in Go routines work: namely, passing the loop +// variable as an argument to the function called asynchronously; or +// creating a scoped copy of the loop variable within the loop. +func FixedFunction() { + for _, n := range collection { + doWork() + go func(n int) { + intID(n) // this is OK + }(n) + + defer func(n int) { + intID(n) // this i OK + }(n) + } + + for j := 0; j < len(collection); j++ { + j := j + go func() { + intID(j) // this is OK + }() + + defer func() { + intID(j) // this is OK + }() + } +} + +// CapturingDefers makes sure that `defer` statements that are passed +// closures that capture loop variables by reference are also detected. +func CapturingDefers() { + for i, n := range collection { + showProgress := func() { + fmt.Printf("finished iteration: %d\n", i) + } + + if n > 0 { + defer func() { + fmt.Printf("cleaning up: %d\n", n) // want `loop variable 'n' captured by reference` + }() + + defer showProgress() // want `'showProgress' function captures loop variable 'i' by reference` + + defer func(callback func()) { + fmt.Printf("finished loop, nothing to see here") + callback() + }(func() { intID(n) }) // want `loop variable 'n' captured by reference` + } + + for j := 0; i < len(collection); j++ { + defer func(idx int) { + intID(n) // want `loop variable 'n' captured by reference` + fmt.Printf("%d\n", j) // want `loop variable 'j' captured by reference` + intID(idx) // this is OK + }(i) + } + } +} + +// CapturingGoRoutineFunctions tests that captures of loop variables +// in functions that are known to create Go routines are also detected +// and reported. +func CapturingGoRoutineFunctions() { + var eg errgroup.Group + var cg concurrency.Group + + for _, n := range collection { + eg.Go(func() error { + fmt.Printf("working on n = %d\n", n) // want `loop variable 'n' captured by reference` + + if rand.Float64() < 0.5 { + return fmt.Errorf("random error: %d", n) // want `loop variable 'n' captured by reference` + } + + return nil + }) + + cg.Go(func() { intID(n) }) // want `loop variable 'n' captured by reference` + + concurrency.Go(func() { intID(n) }) // want `loop variable 'n' captured by reference` + concurrency.SafeFunction(func() { intID(n) }) // this is OK + + err := concurrency.GoWithError(func() { intID(n) }) // want `loop variable 'n' captured by reference` + if err != nil { + panic(err) + } + } +} + +// RespectsNolintComments makes sure that developers are able to +// silence the linter using their own judgement. +func RespectsNolintComments() { + for _, n := range collection { + var eg errgroup.Group + var wg sync.WaitGroup + wg.Add(1) + + badClosure := func() { fmt.Printf("n = %d\n", n) } + + go func() { + defer wg.Done() + //nolint:loopvarcapture + intID(n) + + //nolint:loopvarcapture + badClosure() + }() + + //nolint:loopvarcapture + go badClosure() + + eg.Go(func() error { + //nolint:loopvarcapture + intID(n) + return nil + }) + + go func() { + //nolint:loopvarcapture + intID(n) + }() + + wg.Wait() + } +} diff --git a/vendor b/vendor index 5e07ac6e2f75..403f5a9f04f6 160000 --- a/vendor +++ b/vendor @@ -1 +1 @@ -Subproject commit 5e07ac6e2f75be22ee7f7aeaf2aa5be2823c9b68 +Subproject commit 403f5a9f04f6d094499aec2108b0f09acd818f7d