Skip to content

Commit

Permalink
Merge #60474
Browse files Browse the repository at this point in the history
60474: forbiddenmethod: lint against calls to (grpc.ClientConn).Close r=ajwerner a=tbg

Our main use of ClientConn is through rpc.Context, where they are
cached. This means that most of the time, in production code, it
would be fairly disruptive to call `Close` when done with the
connection (as I found out, luckily in testing code). Essentially
we are forced into a pattern that is not idiomatic go, so our
muscle memory works against us.

This commit adds linting against calls to that method.

This is all relatively unfortunate. Ideally, we'd return wrapped objects
from `rpc.Context` that don't even have a Close method. But the common
use for these ClientConns is to pass them to an proto-generated wrapper
that makes an RPC client, and those take a naked `*ClientConn`. We could
build more tooling around this, but as a first pass the lint is helpful.

See #60473 for a WIP towards a change that "avoids" handing out the
`*ClientConn`. It seems workable, just needs a lot of (simple) code changes.
I won't be pushing that over the finish line anytime soon, so the lint
should merge first.

Release note: None


Co-authored-by: Tobias Grieger <[email protected]>
  • Loading branch information
craig[bot] and tbg committed Feb 16, 2021
2 parents a296e70 + 6a0525e commit 1e1f915
Show file tree
Hide file tree
Showing 29 changed files with 354 additions and 137 deletions.
2 changes: 1 addition & 1 deletion pkg/cli/start.go
Original file line number Diff line number Diff line change
Expand Up @@ -1085,7 +1085,7 @@ func getClientGRPCConn(
return nil, nil, nil, err
}
stopper.AddCloser(stop.CloserFn(func() {
_ = conn.Close()
_ = conn.Close() // nolint:grpcconnclose
}))

// Tie the lifetime of the stopper to that of the context.
Expand Down
3 changes: 2 additions & 1 deletion pkg/cmd/roachvet/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -6,14 +6,15 @@ go_library(
importpath = "github.com/cockroachdb/cockroach/pkg/cmd/roachvet",
visibility = ["//visibility:private"],
deps = [
"//pkg/testutils/lint/passes/descriptormarshal",
"//pkg/testutils/lint/passes/errcmp",
"//pkg/testutils/lint/passes/fmtsafe",
"//pkg/testutils/lint/passes/forbiddenmethod:descriptormarshal",
"//pkg/testutils/lint/passes/hash",
"//pkg/testutils/lint/passes/nocopy",
"//pkg/testutils/lint/passes/returnerrcheck",
"//pkg/testutils/lint/passes/timer",
"//pkg/testutils/lint/passes/unconvert",
"@org_golang_x_tools//go/analysis",
"@org_golang_x_tools//go/analysis/passes/asmdecl",
"@org_golang_x_tools//go/analysis/passes/assign",
"@org_golang_x_tools//go/analysis/passes/atomic",
Expand Down
20 changes: 14 additions & 6 deletions pkg/cmd/roachvet/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,14 +14,15 @@
package main

import (
"github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/descriptormarshal"
"github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/errcmp"
"github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/fmtsafe"
"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/nocopy"
"github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/returnerrcheck"
"github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/timer"
"github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/unconvert"
"golang.org/x/tools/go/analysis"
"golang.org/x/tools/go/analysis/passes/asmdecl"
"golang.org/x/tools/go/analysis/passes/assign"
"golang.org/x/tools/go/analysis/passes/atomic"
Expand Down Expand Up @@ -49,18 +50,21 @@ import (
)

func main() {
unitchecker.Main(
// First-party analyzers:
descriptormarshal.Analyzer,
var as []*analysis.Analyzer
// First-party analyzers:
as = append(as, forbiddenmethod.Analyzers...)
as = append(as,
hash.Analyzer,
nocopy.Analyzer,
returnerrcheck.Analyzer,
timer.Analyzer,
unconvert.Analyzer,
fmtsafe.Analyzer,
errcmp.Analyzer,
)

// Standard go vet analyzers:
// Standard go vet analyzers:
as = append(as,
asmdecl.Analyzer,
assign.Analyzer,
atomic.Analyzer,
Expand All @@ -83,8 +87,12 @@ func main() {
unreachable.Analyzer,
unsafeptr.Analyzer,
unusedresult.Analyzer,
)

// Additional analyzers:
// Additional analyzers:
as = append(as,
shadow.Analyzer,
)

unitchecker.Main(as...)
}
9 changes: 5 additions & 4 deletions pkg/kv/kvserver/client_raft_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -3332,13 +3332,14 @@ func TestReplicateRogueRemovedNode(t *testing.T) {
}, incArgs,
)
if _, ok := pErr.GetDetail().(*roachpb.RangeNotFoundError); !ok {
// We're on a goroutine and passing the error out is awkward since
// it would only surface at shutdown time. A panic ought to be good
// enough to get visibility.
panic(fmt.Sprintf("unexpected error: %v", pErr))
// We're on a goroutine, so cannot fatal.
t.Errorf("unexpected error: %v", pErr)
}
}()
startWG.Wait()
if t.Failed() {
t.FailNow()
}

// Sleep a bit to let the command proposed on node 2 proceed if it's
// going to. Prior to the introduction of replica tombstones, this
Expand Down
3 changes: 2 additions & 1 deletion pkg/rpc/context.go
Original file line number Diff line number Diff line change
Expand Up @@ -648,7 +648,8 @@ func (ctx *Context) removeConn(conn *Connection, keys ...connKey) {
log.Health.Infof(ctx.masterCtx, "closing %+v", keys)
}
if grpcConn := conn.grpcConn; grpcConn != nil {
if err := grpcConn.Close(); err != nil && !grpcutil.IsClosedConnection(err) {
err := grpcConn.Close() // nolint:grpcconnclose
if err != nil && !grpcutil.IsClosedConnection(err) {
if log.V(1) {
log.Health.Errorf(ctx.masterCtx, "failed to close client connection: %v", err)
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/rpc/context_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1178,7 +1178,6 @@ func grpcRunKeepaliveTestCase(testCtx context.Context, c grpcKeepaliveTestCase)
if err != nil {
return err
}
defer func() { _ = conn.Close() }()

// Create the heartbeat client.
log.Infof(ctx, "starting heartbeat client")
Expand Down Expand Up @@ -1792,7 +1791,8 @@ func TestRunHeartbeatSetsHeartbeatStateWhenExitingBeforeFirstHeartbeat(t *testin
if _, err = c.Connect(ctx); err != nil {
require.Regexp(t, "not yet heartbeated", err)
}
require.NoError(t, c.grpcConn.Close())
err = c.grpcConn.Close() // nolint:grpcconnclose
require.NoError(t, err)
}

func BenchmarkGRPCDial(b *testing.B) {
Expand Down
8 changes: 6 additions & 2 deletions pkg/server/authentication_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -791,7 +791,9 @@ func TestGRPCAuthentication(t *testing.T) {
if err != nil {
t.Fatal(err)
}
defer func(conn *grpc.ClientConn) { _ = conn.Close() }(conn)
defer func(conn *grpc.ClientConn) {
_ = conn.Close() // nolint:grpcconnclose
}(conn)
for _, subsystem := range subsystems {
t.Run(fmt.Sprintf("no-cert/%s", subsystem.name), func(t *testing.T) {
err := subsystem.sendRPC(ctx, conn)
Expand All @@ -814,7 +816,9 @@ func TestGRPCAuthentication(t *testing.T) {
if err != nil {
t.Fatal(err)
}
defer func(conn *grpc.ClientConn) { _ = conn.Close() }(conn)
defer func(conn *grpc.ClientConn) {
_ = conn.Close() // nolint:grpcconnclose
}(conn)
for _, subsystem := range subsystems {
t.Run(fmt.Sprintf("bad-user/%s", subsystem.name), func(t *testing.T) {
err := subsystem.sendRPC(ctx, conn)
Expand Down
2 changes: 1 addition & 1 deletion pkg/server/connectivity_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -206,7 +206,7 @@ func TestClusterConnectivity(t *testing.T) {
return err
}
defer func() {
_ = conn.Close()
_ = conn.Close() // nolint:grpcconnclose
}()

client := serverpb.NewInitClient(conn)
Expand Down
4 changes: 3 additions & 1 deletion pkg/server/drain_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -202,5 +202,7 @@ func getAdminClientForServer(
return nil, nil, err
}
client := serverpb.NewAdminClient(conn)
return client, func() { _ = conn.Close() }, nil
return client, func() {
_ = conn.Close() // nolint:grpcconnclose
}, nil
}
2 changes: 1 addition & 1 deletion pkg/server/init.go
Original file line number Diff line number Diff line change
Expand Up @@ -455,7 +455,7 @@ func (s *initServer) attemptJoinTo(
}

defer func() {
_ = conn.Close()
_ = conn.Close() // nolint:grpcconnclose
}()

binaryVersion := s.config.binaryVersion
Expand Down
3 changes: 2 additions & 1 deletion pkg/server/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -1321,7 +1321,8 @@ func (s *Server) PreStart(ctx context.Context) error {
// running in a worker and usually sits on accept() which unblocks
// only when the listener closes. In other words, the listener needs
// to close when quiescing starts to allow that worker to shut down.
if err := conn.Close(); err != nil {
err := conn.Close() // nolint:grpcconnclose
if err != nil {
log.Ops.Fatalf(workersCtx, "%v", err)
}
}
Expand Down
16 changes: 12 additions & 4 deletions pkg/sql/colflow/colrpc/colrpc_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -173,7 +173,8 @@ func TestOutboxInbox(t *testing.T) {
require.NoError(t, err)
if cancellationScenario != transportBreaks {
defer func() {
require.NoError(t, conn.Close())
err := conn.Close() // nolint:grpcconnclose
require.NoError(t, err)
}()
}

Expand Down Expand Up @@ -257,7 +258,8 @@ func TestOutboxInbox(t *testing.T) {
case readerCtxCancel:
readerCancelFn()
case transportBreaks:
_ = conn.Close()
err := conn.Close() // nolint:grpcconnclose
require.NoError(t, err)
}
wg.Done()
}()
Expand Down Expand Up @@ -388,7 +390,10 @@ func TestOutboxInboxMetadataPropagation(t *testing.T) {

conn, err := grpc.Dial(addr.String(), grpc.WithInsecure())
require.NoError(t, err)
defer func() { require.NoError(t, conn.Close()) }()
defer func() {
err := conn.Close() // nolint:grpcconnclose
require.NoError(t, err)
}()

rng, _ := randutil.NewPseudoRand()
// numNextsBeforeDrain is used in ExplicitDrainRequest. This number is
Expand Down Expand Up @@ -551,7 +556,10 @@ func BenchmarkOutboxInbox(b *testing.B) {

conn, err := grpc.Dial(addr.String(), grpc.WithInsecure())
require.NoError(b, err)
defer func() { require.NoError(b, conn.Close()) }()
defer func() {
err := conn.Close() // nolint:grpcconnclose
require.NoError(b, err)
}()

client := execinfrapb.NewDistSQLClient(conn)
clientStream, err := client.FlowStream(ctx)
Expand Down
3 changes: 2 additions & 1 deletion pkg/sql/execinfrapb/testutils.go
Original file line number Diff line number Diff line change
Expand Up @@ -150,7 +150,8 @@ func (d *MockDialer) DialNoBreaker(

// Close must be called after the test is done.
func (d *MockDialer) Close() {
if err := d.mu.conn.Close(); err != nil {
err := d.mu.conn.Close() // nolint:grpcconnclose
if err != nil {
panic(err)
}
}
95 changes: 0 additions & 95 deletions pkg/testutils/lint/passes/descriptormarshal/descriptor_marshal.go

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -2,8 +2,11 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")

go_library(
name = "descriptormarshal",
srcs = ["descriptor_marshal.go"],
importpath = "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/descriptormarshal",
srcs = [
"analyzers.go",
"forbiddenmethod.go",
],
importpath = "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/forbiddenmethod",
visibility = ["//visibility:public"],
deps = [
"//pkg/testutils/lint/passes/passesutil",
Expand All @@ -24,3 +27,13 @@ go_test(
"@org_golang_x_tools//go/analysis/analysistest",
],
)

go_test(
name = "forbiddenmethod_test",
srcs = ["descriptormarshal_test.go"],
deps = [
":descriptormarshal",
"//pkg/testutils/skip",
"@org_golang_x_tools//go/analysis/analysistest",
],
)
Loading

0 comments on commit 1e1f915

Please sign in to comment.