diff --git a/Gopkg.lock b/Gopkg.lock index 2db72dde1d39..52f7070ed4d8 100644 --- a/Gopkg.lock +++ b/Gopkg.lock @@ -525,12 +525,6 @@ revision = "3247c84500bff8d9fb6d579d800f20b3e091582c" version = "v1.0.0" -[[projects]] - branch = "master" - name = "github.com/mdempsky/unconvert" - packages = ["."] - revision = "beb68d938016d2dec1d1b078054f4d3db25f97be" - [[projects]] name = "github.com/mibk/dupl" packages = [".","job","output","suffixtree","syntax","syntax/golang"] @@ -761,7 +755,7 @@ [[projects]] name = "golang.org/x/text" - packages = ["collate","collate/build","internal/colltab","internal/gen","internal/tag","internal/triegen","internal/ucd","language","secure/bidirule","transform","unicode/bidi","unicode/cldr","unicode/norm","unicode/rangetable","width"] + packages = ["collate","collate/build","internal/colltab","internal/gen","internal/tag","internal/triegen","internal/ucd","language","secure/bidirule","transform","unicode/bidi","unicode/cldr","unicode/norm","unicode/rangetable"] revision = "470f45bf29f4147d6fbd7dfd0a02a848e49f5bf4" [[projects]] @@ -808,13 +802,13 @@ [[projects]] name = "honnef.co/go/tools" - packages = ["callgraph","callgraph/static","functions","internal/sharedcheck","lint","lint/lintutil","simple","ssa","ssa/ssautil","staticcheck","staticcheck/vrp","unused","version"] + packages = ["callgraph","callgraph/static","functions","internal/sharedcheck","lint","simple","ssa","ssa/ssautil","staticcheck","staticcheck/vrp","unused"] revision = "e5147431c7c056bd6ef33ad2f23c846f835571dd" version = "2017.1" [solve-meta] analyzer-name = "dep" analyzer-version = 1 - inputs-digest = "16fdf207f528a93b9f6d7977d0ceee76993f9c1b921f6e4a70d9a8794c0deb23" + inputs-digest = "034ab7fa506a52f7a2062b0f3bfe1b8ff8a231ab399ea77fff073a95507e4d41" solver-name = "gps-cdcl" solver-version = 1 diff --git a/Gopkg.toml b/Gopkg.toml index 51e9da30034b..780f158d6b16 100644 --- a/Gopkg.toml +++ b/Gopkg.toml @@ -9,7 +9,6 @@ required = [ "github.com/jteeuwen/go-bindata/go-bindata", "github.com/kisielk/errcheck", "github.com/mattn/goveralls", - "github.com/mdempsky/unconvert", "github.com/mibk/dupl", "github.com/wadey/gocovmerge", "golang.org/x/perf/cmd/benchstat", diff --git a/build/common.mk b/build/common.mk index 7c14f54d2e42..36a56199b6f5 100644 --- a/build/common.mk +++ b/build/common.mk @@ -164,7 +164,7 @@ $(BOOTSTRAP_TARGET): $(GITHOOKS) $(REPO_ROOT)/Gopkg.lock ifneq ($(GIT_DIR),) git submodule update --init endif - @$(GO_INSTALL) -v $(PKG_ROOT)/cmd/{metacheck,returncheck} \ + @$(GO_INSTALL) -v $(PKG_ROOT)/cmd/returncheck \ $(REPO_ROOT)/vendor/github.com/golang/dep/cmd/dep \ $(REPO_ROOT)/vendor/github.com/client9/misspell/cmd/misspell \ $(REPO_ROOT)/vendor/github.com/cockroachdb/crlfmt \ @@ -175,7 +175,6 @@ endif $(REPO_ROOT)/vendor/github.com/jteeuwen/go-bindata/go-bindata \ $(REPO_ROOT)/vendor/github.com/kisielk/errcheck \ $(REPO_ROOT)/vendor/github.com/mattn/goveralls \ - $(REPO_ROOT)/vendor/github.com/mdempsky/unconvert \ $(REPO_ROOT)/vendor/github.com/mibk/dupl \ $(REPO_ROOT)/vendor/github.com/wadey/gocovmerge \ $(REPO_ROOT)/vendor/golang.org/x/perf/cmd/benchstat \ diff --git a/build/style_test.go b/build/style_test.go index 540abac4d9f7..9ce8eb3b14ef 100644 --- a/build/style_test.go +++ b/build/style_test.go @@ -19,7 +19,13 @@ package build_test import ( "bufio" "bytes" + "fmt" + "go/ast" "go/build" + "go/parser" + "go/token" + "go/types" + "log" "os" "os/exec" "path/filepath" @@ -29,8 +35,14 @@ import ( "testing" "github.com/ghemawat/stream" + "github.com/kisielk/gotool" "github.com/pkg/errors" "golang.org/x/tools/go/buildutil" + "golang.org/x/tools/go/loader" + "honnef.co/go/tools/lint" + "honnef.co/go/tools/simple" + "honnef.co/go/tools/staticcheck" + "honnef.co/go/tools/unused" ) const cockroachDB = "github.com/cockroachdb/cockroach/pkg" @@ -900,57 +912,65 @@ func TestStyle(t *testing.T) { } }) - t.Run("TestUnconvert", func(t *testing.T) { + t.Run("TestMegacheck", func(t *testing.T) { if testing.Short() { t.Skip("short flag") } t.Parallel() - cmd, stderr, filter, err := dirCmd(pkg.Dir, "unconvert", pkgScope) + noCopyRe, err := regexp.Compile(`^(field no|type No)Copy is unused \(U1000\)$`) if err != nil { t.Fatal(err) } - if err := cmd.Start(); err != nil { + ctx := gotool.DefaultContext + releaseTags := ctx.BuildContext.ReleaseTags + lastTag := releaseTags[len(releaseTags)-1] + dotIdx := strings.IndexByte(lastTag, '.') + goVersion, err := strconv.Atoi(lastTag[dotIdx+1:]) + if err != nil { t.Fatal(err) } - - if err := stream.ForEach(stream.Sequence( - filter, - stream.GrepNot(`\.pb\.go:`), - ), func(s string) { - t.Error(s) - }); err != nil { - t.Error(err) + // NB: this doesn't use `pkgScope` because `honnef.co/go/unused` + // produces many false positives unless it inspects all our packages. + paths := ctx.ImportPaths([]string{cockroachDB + "/..."}) + conf := loader.Config{ + Build: &ctx.BuildContext, + ParserMode: parser.ParseComments, + ImportPkgs: make(map[string]bool, len(paths)), } - - if err := cmd.Wait(); err != nil { - if out := stderr.String(); len(out) > 0 { - t.Fatalf("err=%s, stderr=%s", err, out) - } + for _, path := range paths { + conf.ImportPkgs[path] = true + } + lprog, err := conf.Load() + if err != nil { + t.Fatal(err) } - }) - t.Run("TestMetacheck", func(t *testing.T) { - if testing.Short() { - t.Skip("short flag") + unusedChecker := unused.NewChecker(unused.CheckAll) + unusedChecker.WholeProgram = true + + checker := megaChecker{ + checkers: []lint.Checker{ + &timerChecker{}, + simple.NewChecker(), + staticcheck.NewChecker(), + unused.NewLintChecker(unusedChecker), + }, } - // metacheck uses 2.5GB of ram (as of 2017-02-18), so don't parallelize it. - cmd, stderr, filter, err := dirCmd( - pkg.Dir, - "metacheck", - "-ignore", - strings.Join([]string{ - "github.com/cockroachdb/cockroach/pkg/security/securitytest/embedded.go:S1013", - "github.com/cockroachdb/cockroach/pkg/ui/embedded.go:S1013", + linter := lint.Linter{ + Checker: &checker, + Ignores: []lint.Ignore{ + {Pattern: "github.com/cockroachdb/cockroach/pkg/security/securitytest/embedded.go", Checks: []string{"S1013"}}, + {Pattern: "github.com/cockroachdb/cockroach/pkg/ui/embedded.go", Checks: []string{"S1013"}}, // Intentionally compare an unsigned integer <= 0 to avoid knowledge // of the type at the caller and for consistency with convention. - "github.com/cockroachdb/cockroach/pkg/storage/replica.go:SA4003", + {Pattern: "github.com/cockroachdb/cockroach/pkg/storage/replica.go", Checks: []string{"SA4003"}}, // Allow a comment to refer to an "unused" argument. // // TODO(bdarnell): remove when/if #8360 is fixed. - "github.com/cockroachdb/cockroach/pkg/storage/intent_resolver.go:SA4009", + {Pattern: "github.com/cockroachdb/cockroach/pkg/storage/intent_resolver.go", Checks: []string{"SA4009"}}, // The generated parser is full of `case` arms such as: // // case 1: @@ -1011,49 +1031,33 @@ func TestStyle(t *testing.T) { // which results in the unused warning: // // sql/parser/yaccpar:362:3: this value of sqlDollar is never used (SA4006) - "github.com/cockroachdb/cockroach/pkg/sql/parser/sql.go:SA4006", + {Pattern: "github.com/cockroachdb/cockroach/pkg/sql/parser/sql.go", Checks: []string{"SA4006"}}, // sql/ir/irgen/parser/yaccpar:362:3: this value of irgenDollar is never used (SA4006) - "github.com/cockroachdb/cockroach/pkg/sql/ir/irgen/parser/irgen.go:SA4006", + {Pattern: "github.com/cockroachdb/cockroach/pkg/sql/ir/irgen/parser/irgen.go", Checks: []string{"SA4006"}}, // sql/parser/yaccpar:14:6: type sqlParser is unused (U1000) // sql/parser/yaccpar:15:2: func sqlParser.Parse is unused (U1000) // sql/parser/yaccpar:16:2: func sqlParser.Lookahead is unused (U1000) // sql/parser/yaccpar:29:6: func sqlNewParser is unused (U1000) // sql/parser/yaccpar:152:6: func sqlParse is unused (U1000) - "github.com/cockroachdb/cockroach/pkg/sql/parser/sql.go:U1000", - "github.com/cockroachdb/cockroach/pkg/sql/irgen/parser/irgen.go:U1000", + {Pattern: "github.com/cockroachdb/cockroach/pkg/sql/parser/sql.go", Checks: []string{"U1000"}}, + {Pattern: "github.com/cockroachdb/cockroach/pkg/sql/irgen/parser/irgen.go", Checks: []string{"U1000"}}, // Generated file containing many unused postgres error codes. - "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror/codes.go:U1000", + {Pattern: "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror/codes.go", Checks: []string{"U1000"}}, // Deprecated database/sql/driver interfaces not compatible with 1.7. - "github.com/cockroachdb/cockroach/pkg/sql/*.go:SA1019", - "github.com/cockroachdb/cockroach/pkg/cli/sql_util.go:SA1019", + {Pattern: "github.com/cockroachdb/cockroach/pkg/sql/*.go", Checks: []string{"SA1019"}}, + {Pattern: "github.com/cockroachdb/cockroach/pkg/cli/sql_util.go", Checks: []string{"SA1019"}}, // IR templates. - "github.com/cockroachdb/cockroach/pkg/sql/ir/base/*.go:U1000", - }, " "), - // NB: this doesn't use `pkgScope` because `honnef.co/go/unused` - // produces many false positives unless it inspects all our packages. - "./...", - ) - if err != nil { - t.Fatal(err) - } - - if err := cmd.Start(); err != nil { - t.Fatal(err) - } - - if err := stream.ForEach(stream.Sequence( - filter, - stream.GrepNot(`: (field no|type No)Copy is unused \(U1000\)$`), - ), func(s string) { - t.Error(s) - }); err != nil { - t.Error(err) + {Pattern: "github.com/cockroachdb/cockroach/pkg/sql/ir/base/*.go", Checks: []string{"U1000"}}, + }, + GoVersion: goVersion, } - if err := cmd.Wait(); err != nil { - if out := stderr.String(); len(out) > 0 { - t.Fatalf("err=%s, stderr=%s", err, out) + ps := linter.Lint(lprog) + for _, p := range ps { + if !noCopyRe.MatchString(p.Text) { + pos := lprog.Fset.Position(p.Position) + t.Errorf("%s: %s", pos, p.Text) } } }) @@ -1098,3 +1102,382 @@ func TestStyle(t *testing.T) { } }) } + +// megaChecker implements lint.Checker by multiplexing its consitituent +// checkers. +type megaChecker struct { + checkers []lint.Checker +} + +func (m *megaChecker) Init(program *lint.Program) { + for _, checker := range m.checkers { + checker.Init(program) + } +} + +func (m *megaChecker) Funcs() map[string]lint.Func { + funcs := map[string]lint.Func{ + "FloatToUnsigned": checkConvertFloatToUnsigned, + "Unconvert": checkUnconvert, + } + for _, checker := range m.checkers { + for k, v := range checker.Funcs() { + if _, ok := funcs[k]; ok { + log.Fatalf("duplicate lint function %s", k) + } else { + funcs[k] = v + } + } + } + return funcs +} + +func forAllFiles(j *lint.Job, fn func(node ast.Node) bool) { + for _, f := range j.Program.Files { + if !lint.IsGenerated(f) { + ast.Inspect(f, fn) + } + } +} + +// @ianlancetaylor via golang-nuts[0]: +// +// For the record, the spec says, in https://golang.org/ref/spec#Conversions: +// "In all non-constant conversions involving floating-point or complex +// values, if the result type cannot represent the value the conversion +// succeeds but the result value is implementation-dependent." That is the +// case that applies here: you are converting a negative floating point number +// to uint64, which can not represent a negative value, so the result is +// implementation-dependent. The conversion to int64 works, of course. And +// the conversion to int64 and then to uint64 succeeds in converting to int64, +// and when converting to uint64 follows a different rule: "When converting +// between integer types, if the value is a signed integer, it is sign +// extended to implicit infinite precision; otherwise it is zero extended. It +// is then truncated to fit in the result type's size." +// +// So, basically, don't convert a negative floating point number to an +// unsigned integer type. +// +// [0] https://groups.google.com/d/msg/golang-nuts/LH2AO1GAIZE/PyygYRwLAwAJ +// +// TODO(tamird): upstream this. +func checkConvertFloatToUnsigned(j *lint.Job) { + forAllFiles(j, func(n ast.Node) bool { + call, ok := n.(*ast.CallExpr) + if !ok { + return true + } + castType, ok := j.Program.Info.TypeOf(call.Fun).(*types.Basic) + if !ok { + return true + } + if castType.Info()&types.IsUnsigned == 0 { + return true + } + for _, arg := range call.Args { + argType, ok := j.Program.Info.TypeOf(arg).(*types.Basic) + if !ok { + continue + } + if argType.Info()&types.IsFloat == 0 { + continue + } + j.Errorf(arg, "do not convert a floating point number to an unsigned integer type") + } + return true + }) +} + +func walkForStmts(n ast.Node, fn func(ast.Stmt) bool) bool { + fr, ok := n.(*ast.ForStmt) + if !ok { + return true + } + return walkStmts(fr.Body.List, fn) +} + +func walkSelectStmts(n ast.Node, fn func(ast.Stmt) bool) bool { + sel, ok := n.(*ast.SelectStmt) + if !ok { + return true + } + return walkStmts(sel.Body.List, fn) +} + +func walkStmts(stmts []ast.Stmt, fn func(ast.Stmt) bool) bool { + for _, stmt := range stmts { + if !fn(stmt) { + return false + } + } + return true +} + +// timerChecker assures that timeutil.Timer objects are used correctly, to +// avoid race conditions and deadlocks. These timers require callers to set +// their Read field to true when their channel has been received on. If this +// field is not set and the timer's Reset method is called, we will deadlock. +// This lint assures that the Read field is set in the most common case where +// Reset is used, within a for-loop where each iteration blocks on a select +// statement. The timers are usually used as timeouts on these select +// statements, and need to be reset after each iteration. +// +// for { +// timer.Reset(...) +// select { +// case <-timer.C: +// timer.Read = true <-- lint verifies that this line is present +// case ...: +// } +// } +// +type timerChecker struct { + timerType types.Type +} + +const timerChanName = "C" + +func (m *timerChecker) Init(program *lint.Program) { + timeutilPkg := program.Prog.Package("github.com/cockroachdb/cockroach/pkg/util/timeutil") + if timeutilPkg == nil { + log.Fatal("timeutil package not found") + } + timerObject := timeutilPkg.Pkg.Scope().Lookup("Timer") + if timerObject == nil { + log.Fatal("timeutil.Timer type not found") + } + m.timerType = timerObject.Type() + + func() { + if typ, ok := m.timerType.Underlying().(*types.Struct); ok { + for i := 0; i < typ.NumFields(); i++ { + if typ.Field(i).Name() == timerChanName { + return + } + } + } + log.Fatalf("no field called %q in type %s", timerChanName, m.timerType) + }() +} + +func (m *timerChecker) Funcs() map[string]lint.Func { + return map[string]lint.Func{ + "TimeutilTimerRead": m.checkSetTimeutilTimerRead, + } +} + +func (m *timerChecker) selectorIsTimer(s *ast.SelectorExpr, info *types.Info) bool { + selTyp := info.TypeOf(s.X) + if ptr, ok := selTyp.(*types.Pointer); ok { + selTyp = ptr.Elem() + } + return selTyp == m.timerType +} + +func (m *timerChecker) checkSetTimeutilTimerRead(j *lint.Job) { + forAllFiles(j, func(n ast.Node) bool { + return walkForStmts(n, func(s ast.Stmt) bool { + return walkSelectStmts(s, func(s ast.Stmt) bool { + comm, ok := s.(*ast.CommClause) + if !ok || comm.Comm == nil /* default: */ { + return true + } + + // if receiving on a timer's C chan. + var unary ast.Expr + switch v := comm.Comm.(type) { + case *ast.AssignStmt: + // case `now := <-timer.C:` + unary = v.Rhs[0] + case *ast.ExprStmt: + // case `<-timer.C:` + unary = v.X + default: + return true + } + chanRead, ok := unary.(*ast.UnaryExpr) + if !ok || chanRead.Op != token.ARROW { + return true + } + selector, ok := chanRead.X.(*ast.SelectorExpr) + if !ok { + return true + } + if !m.selectorIsTimer(selector,j.Program.Info) { + return true + } + selectorName := fmt.Sprint(selector.X) + if selector.Sel.String() != timerChanName { + return true + } + + // Verify that the case body contains `timer.Read = true`. + noRead := walkStmts(comm.Body, func(s ast.Stmt) bool { + assign, ok := s.(*ast.AssignStmt) + if !ok || assign.Tok != token.ASSIGN { + return true + } + for i := range assign.Lhs { + l, r := assign.Lhs[i], assign.Rhs[i] + + // if assignment to correct field in timer. + assignSelector, ok := l.(*ast.SelectorExpr) + if !ok { + return true + } + if !m.selectorIsTimer(assignSelector,j.Program.Info) { + return true + } + if fmt.Sprint(assignSelector.X) != selectorName { + return true + } + if assignSelector.Sel.String() != "Read" { + return true + } + + // if assigning `true`. + val, ok := r.(*ast.Ident) + if !ok { + return true + } + if val.String() == "true" { + // returning false will short-circuit walkStmts and assign + // noRead to false instead of the default value of true. + return false + } + } + return true + }) + if noRead { + j.Errorf(comm, "must set timer.Read = true after reading from timer.C (see timeutil/timer.go)") + } + return true + }) + }) + }) +} + +// Adapted from https://github.com/mdempsky/unconvert/blob/beb68d938016d2dec1d1b078054f4d3db25f97be/unconvert.go#L371-L414. +func checkUnconvert(j *lint.Job) { + forAllFiles(j, func(n ast.Node) bool { + call, ok := n.(*ast.CallExpr) + if !ok { + return true + } + if len(call.Args) != 1 || call.Ellipsis != token.NoPos { + return true + } + ft, ok := j.Program.Info.Types[call.Fun] + if !ok { + j.Errorf(call.Fun, "missing type") + return true + } + if !ft.IsType() { + // Function call; not a conversion. + return true + } + at, ok := j.Program.Info.Types[call.Args[0]] + if !ok { + j.Errorf(call.Args[0], "missing type") + return true + } + if !types.Identical(ft.Type, at.Type) { + // A real conversion. + return true + } + if isUntypedValue(call.Args[0], j.Program.Info) { + // Workaround golang.org/issue/13061. + return true + } + // Adapted from https://github.com/mdempsky/unconvert/blob/beb68d938016d2dec1d1b078054f4d3db25f97be/unconvert.go#L416-L430. + // + // cmd/cgo generates explicit type conversions that + // are often redundant when introducing + // _cgoCheckPointer calls (issue #16). Users can't do + // anything about these, so skip over them. + if ident, ok := call.Fun.(*ast.Ident); ok { + if ident.Name == "_cgoCheckPointer" { + return false + } + } + j.Errorf(n, "unnecessary conversion") + return true + }) +} + +// Cribbed from https://github.com/mdempsky/unconvert/blob/beb68d938016d2dec1d1b078054f4d3db25f97be/unconvert.go#L557-L607. +func isUntypedValue(n ast.Expr, info *types.Info) bool { + switch n := n.(type) { + case *ast.BinaryExpr: + switch n.Op { + case token.SHL, token.SHR: + // Shifts yield an untyped value if their LHS is untyped. + return isUntypedValue(n.X, info) + case token.EQL, token.NEQ, token.LSS, token.GTR, token.LEQ, token.GEQ: + // Comparisons yield an untyped boolean value. + return true + case token.ADD, token.SUB, token.MUL, token.QUO, token.REM, + token.AND, token.OR, token.XOR, token.AND_NOT, + token.LAND, token.LOR: + return isUntypedValue(n.X, info) && isUntypedValue(n.Y, info) + } + case *ast.UnaryExpr: + switch n.Op { + case token.ADD, token.SUB, token.NOT, token.XOR: + return isUntypedValue(n.X, info) + } + case *ast.BasicLit: + // Basic literals are always untyped. + return true + case *ast.ParenExpr: + return isUntypedValue(n.X, info) + case *ast.SelectorExpr: + return isUntypedValue(n.Sel, info) + case *ast.Ident: + if obj, ok := info.Uses[n]; ok { + if obj.Pkg() == nil && obj.Name() == "nil" { + // The universal untyped zero value. + return true + } + if b, ok := obj.Type().(*types.Basic); ok && b.Info()&types.IsUntyped != 0 { + // Reference to an untyped constant. + return true + } + } + case *ast.CallExpr: + if b, ok := asBuiltin(n.Fun, info); ok { + switch b.Name() { + case "real", "imag": + return isUntypedValue(n.Args[0], info) + case "complex": + return isUntypedValue(n.Args[0], info) && isUntypedValue(n.Args[1], info) + } + } + } + + return false +} + +// Cribbed from https://github.com/mdempsky/unconvert/blob/beb68d938016d2dec1d1b078054f4d3db25f97be/unconvert.go#L609-L630. +func asBuiltin(n ast.Expr, info *types.Info) (*types.Builtin, bool) { + for { + paren, ok := n.(*ast.ParenExpr) + if !ok { + break + } + n = paren.X + } + + ident, ok := n.(*ast.Ident) + if !ok { + return nil, false + } + + obj, ok := info.Uses[ident] + if !ok { + return nil, false + } + + b, ok := obj.(*types.Builtin) + return b, ok +} diff --git a/pkg/acceptance/allocator_test.go b/pkg/acceptance/allocator_test.go index cec40b1039d0..deb4002a6ab7 100644 --- a/pkg/acceptance/allocator_test.go +++ b/pkg/acceptance/allocator_test.go @@ -406,7 +406,7 @@ func (at *allocatorTest) stdDev() (float64, error) { var replicaCounts stats.Float64Data for _, node := range nodesResp.Nodes { for _, ss := range node.StoreStatuses { - replicaCounts = append(replicaCounts, float64(ss.Metrics["replicas"])) + replicaCounts = append(replicaCounts, ss.Metrics["replicas"]) } } stdDev, err := stats.StdDevP(replicaCounts) diff --git a/pkg/ccl/storageccl/import_test.go b/pkg/ccl/storageccl/import_test.go index 999d8b4bd2cc..8fdadcac6b6b 100644 --- a/pkg/ccl/storageccl/import_test.go +++ b/pkg/ccl/storageccl/import_test.go @@ -86,7 +86,7 @@ func slurpSSTablesLatestKey( var ok bool kv.Key.Key, ok = kr.rewriteKey(kv.Key.Key) if !ok { - return true, errors.Errorf("could not rewrite key: %s", roachpb.Key(kv.Key.Key)) + return true, errors.Errorf("could not rewrite key: %s", kv.Key.Key) } v := roachpb.Value{RawBytes: kv.Value} v.ClearChecksum() diff --git a/pkg/cli/dump_test.go b/pkg/cli/dump_test.go index 9882e2cb1696..4b49b7cd68b5 100644 --- a/pkg/cli/dump_test.go +++ b/pkg/cli/dump_test.go @@ -131,7 +131,7 @@ INSERT INTO t (i, f, s, b, d, t, n, o, e, u, ip, tz, e1, e2, s1) VALUES (NULL, 'NaN', NULL, NULL, NULL, NULL, NULL, NULL, 'NaN', NULL, NULL, NULL, NULL, NULL, NULL); ` - if string(out) != expect { + if out != expect { t.Fatalf("expected: %s\ngot: %s", expect, out) } } @@ -158,7 +158,7 @@ CREATE TABLE f ( INSERT INTO f (x, y) VALUES (42, 69); ` - if string(out) != expected { + if out != expected { t.Fatalf("expected %s\ngot: %s", expected, out) } @@ -173,7 +173,7 @@ CREATE TABLE f ( FAMILY "primary" (x, y, rowid) ); ` - if string(out) != expected { + if out != expected { t.Fatalf("expected %s\ngot: %s", expected, out) } @@ -186,7 +186,7 @@ CREATE TABLE f ( INSERT INTO f (x, y) VALUES (42, 69); ` - if string(out) != expected { + if out != expected { t.Fatalf("expected %s\ngot: %s", expected, out) } } @@ -224,7 +224,7 @@ INSERT INTO f (x, y) VALUES INSERT INTO g (x, y) VALUES (3, 4); ` - if string(out) != expected { + if out != expected { t.Fatalf("expected %s\ngot: %s", expected, out) } @@ -252,7 +252,7 @@ INSERT INTO f (x, y) VALUES INSERT INTO g (x, y) VALUES (3, 4); ` - if string(out) != expected { + if out != expected { t.Fatalf("expected %s\ngot: %s", expected, out) } } @@ -392,7 +392,7 @@ func TestDumpRandom(t *testing.T) { } n := dur.String() o := rnd.Intn(2) == 1 - e := apd.New(rnd.Int63(), int32(rnd.Int31n(20)-10)).String() + e := apd.New(rnd.Int63(), rnd.Int31n(20)-10).String() sr := make([]byte, rnd.Intn(500)) if _, err := rnd.Read(sr); err != nil { t.Fatal(err) @@ -582,7 +582,7 @@ INSERT INTO t (i, j) VALUES if out, err := c.RunWithCaptureArgs([]string{"dump", "d", "t", "--as-of", "2000-01-01 00:00:00"}); err != nil { t.Fatal(err) - } else if !strings.Contains(string(out), "relation d.t does not exist") { + } else if !strings.Contains(out, "relation d.t does not exist") { t.Fatalf("unexpected output: %s", out) } } @@ -604,14 +604,14 @@ func TestDumpIdentifiers(t *testing.T) { if out, err := c.RunWithCaptureArgs([]string{"sql", "-e", create}); err != nil { t.Fatal(err) } else { - t.Log(string(out)) + t.Log(out) } out, err := c.RunWithCaptureArgs([]string{"dump", "d"}) if err != nil { t.Fatal(err) } else { - t.Log(string(out)) + t.Log(out) } const expect = `dump d @@ -625,7 +625,7 @@ INSERT INTO ";" (";") VALUES (1); ` - if string(out) != expect { + if out != expect { t.Fatalf("expected: %s\ngot: %s", expect, out) } } @@ -667,7 +667,7 @@ INSERT INTO c VALUES (1); if out, err := c.RunWithCaptureArgs([]string{"sql", "-e", create}); err != nil { t.Fatal(err) } else { - t.Log(string(out)) + t.Log(out) } out, err := c.RunWithCapture("dump d1") @@ -751,12 +751,12 @@ INSERT INTO c (i) VALUES (1); ` - if string(out) != expectDump { + if out != expectDump { t.Fatalf("expected: %s\ngot: %s", expectDump, out) } // Remove first line of output ("dump a"). - dump := strings.SplitN(string(out), "\n", 2)[1] + dump := strings.SplitN(out, "\n", 2)[1] out, err = c.RunWithCaptureArgs([]string{"sql", "-d", "d2", "-e", dump}) if err != nil { t.Fatal(err) @@ -786,7 +786,7 @@ i # 1 row ` - if string(out) != expect { + if out != expect { t.Fatalf("expected: %s\ngot: %s", expect, out) } @@ -824,7 +824,7 @@ INSERT INTO d (i, e, f) VALUES (1, 1, 1); ` - if string(out) != expectDump2 { + if out != expectDump2 { t.Fatalf("expected: %s\ngot: %s", expectDump2, out) } } @@ -843,21 +843,21 @@ func TestDumpView(t *testing.T) { if out, err := c.RunWithCaptureArgs([]string{"sql", "-e", create}); err != nil { t.Fatal(err) } else { - t.Log(string(out)) + t.Log(out) } out, err := c.RunWithCaptureArgs([]string{"dump", "d"}) if err != nil { t.Fatal(err) } else { - t.Log(string(out)) + t.Log(out) } const expect = `dump d CREATE VIEW bar ("1") AS SELECT 1; ` - if string(out) != expect { + if out != expect { t.Fatalf("expected: %s\ngot: %s", expect, out) } } @@ -897,7 +897,7 @@ INSERT INTO t (i) VALUES (1); ` - if string(out) != expect { + if out != expect { t.Fatalf("expected: %s\ngot: %s", expect, out) } } diff --git a/pkg/cmd/github-pull-request-make/main.go b/pkg/cmd/github-pull-request-make/main.go index 23743fdc6865..303ea9fad5f5 100644 --- a/pkg/cmd/github-pull-request-make/main.go +++ b/pkg/cmd/github-pull-request-make/main.go @@ -114,9 +114,11 @@ func pkgsFromDiff(r io.Reader) (map[string]pkg, error) { case bytes.HasPrefix(line, []byte{'-'}) && bytes.Contains(line, []byte(".Skip")): switch { case len(curTestName) > 0: - curPkg := pkgs[curPkgName] - curPkg.tests = append(curPkg.tests, curTestName) - pkgs[curPkgName] = curPkg + if !(curPkgName == "build" && curTestName == "TestStyle") { + curPkg := pkgs[curPkgName] + curPkg.tests = append(curPkg.tests, curTestName) + pkgs[curPkgName] = curPkg + } case len(curBenchmarkName) > 0: curPkg := pkgs[curPkgName] curPkg.benchmarks = append(curPkg.benchmarks, curBenchmarkName) @@ -223,8 +225,8 @@ func main() { cmd := exec.Command("git", "diff") cmd.Dir = dir log.Println(cmd.Dir, cmd.Args) - if output, err := cmd.Output(); err != nil { - log.Fatal(err) + if output, err := cmd.CombinedOutput(); err != nil { + log.Fatal(err, output) } else if len(output) > 0 { foundDiff = true log.Printf("unexpected diff:\n%s", output) diff --git a/pkg/cmd/metacheck/main.go b/pkg/cmd/metacheck/main.go deleted file mode 100644 index 0b71a4a0827b..000000000000 --- a/pkg/cmd/metacheck/main.go +++ /dev/null @@ -1,283 +0,0 @@ -// Copyright 2016 The Cockroach Authors. -// -// Licensed 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 main - -import ( - "fmt" - "go/ast" - "go/token" - "go/types" - "log" - "os" - - "honnef.co/go/tools/lint" - "honnef.co/go/tools/lint/lintutil" - "honnef.co/go/tools/simple" - "honnef.co/go/tools/staticcheck" - "honnef.co/go/tools/unused" -) - -type metaChecker struct { - checkers []lint.Checker -} - -func (m *metaChecker) Init(program *lint.Program) { - for _, checker := range m.checkers { - checker.Init(program) - } -} - -func (m *metaChecker) Funcs() map[string]lint.Func { - funcs := map[string]lint.Func{ - "FloatToUnsigned": checkConvertFloatToUnsigned, - "TimeutilTimerRead": checkSetTimeutilTimerRead, - } - for _, checker := range m.checkers { - for k, v := range checker.Funcs() { - if _, ok := funcs[k]; ok { - log.Fatalf("duplicate lint function %s", k) - } else { - funcs[k] = v - } - } - } - return funcs -} - -func forAllFiles(j *lint.Job, fn func(node ast.Node) bool) { - for _, f := range j.Program.Files { - ast.Inspect(f, fn) - } -} - -// @ianlancetaylor via golang-nuts[0]: -// -// For the record, the spec says, in https://golang.org/ref/spec#Conversions: -// "In all non-constant conversions involving floating-point or complex -// values, if the result type cannot represent the value the conversion -// succeeds but the result value is implementation-dependent." That is the -// case that applies here: you are converting a negative floating point number -// to uint64, which can not represent a negative value, so the result is -// implementation-dependent. The conversion to int64 works, of course. And -// the conversion to int64 and then to uint64 succeeds in converting to int64, -// and when converting to uint64 follows a different rule: "When converting -// between integer types, if the value is a signed integer, it is sign -// extended to implicit infinite precision; otherwise it is zero extended. It -// is then truncated to fit in the result type's size." -// -// So, basically, don't convert a negative floating point number to an -// unsigned integer type. -// -// [0] https://groups.google.com/d/msg/golang-nuts/LH2AO1GAIZE/PyygYRwLAwAJ -// -// TODO(tamird): upstream this. -func checkConvertFloatToUnsigned(j *lint.Job) { - forAllFiles(j, func(n ast.Node) bool { - call, ok := n.(*ast.CallExpr) - if !ok { - return true - } - castType, ok := j.Program.Info.TypeOf(call.Fun).(*types.Basic) - if !ok { - return true - } - if castType.Info()&types.IsUnsigned == 0 { - return true - } - for _, arg := range call.Args { - argType, ok := j.Program.Info.TypeOf(arg).(*types.Basic) - if !ok { - continue - } - if argType.Info()&types.IsFloat == 0 { - continue - } - j.Errorf(arg, "do not convert a floating point number to an unsigned integer type") - } - return true - }) -} - -func walkForStmts(n ast.Node, fn func(s ast.Stmt) bool) bool { - fr, ok := n.(*ast.ForStmt) - if !ok { - return true - } - return walkStmts(fr.Body.List, fn) -} - -func walkSelectStmts(n ast.Node, fn func(s ast.Stmt) bool) bool { - sel, ok := n.(*ast.SelectStmt) - if !ok { - return true - } - return walkStmts(sel.Body.List, fn) -} - -func walkStmts(stmts []ast.Stmt, fn func(s ast.Stmt) bool) bool { - for _, stmt := range stmts { - if !fn(stmt) { - return false - } - } - return true -} - -// checkSetTimeutilTimerRead assures that timeutil.Timer objects are used -// correctly, to avoid race conditions and deadlocks. These timers require -// callers to set their Read field to true when their channel has been received -// on. If this field is not set and the timer's Reset method is called, we will -// deadlock. This lint assures that the Read field is set in the most common -// case where Reset is used, within a for-loop where each iteration blocks -// on a select statement. The timers are usually used as timeouts on these -// select statements, and need to be reset after each iteration. -// -// for { -// timer.Reset(...) -// select { -// case <-timer.C: -// timer.Read = true <-- lint verifies that this line is present -// case ...: -// } -// } -// -func checkSetTimeutilTimerRead(j *lint.Job) { - var timerType *types.Named - func() { - for _, typVal := range j.Program.Info.Types { - if typ, ok := typVal.Type.(*types.Named); ok { - if typ.String() == "github.com/cockroachdb/cockroach/pkg/util/timeutil.Timer" { - timerType = typ - return - } - } - } - log.Fatal("no timeutil.Timer type found") - }() - - const chanName = "C" - func() { - if typ, ok := timerType.Underlying().(*types.Struct); ok { - for i := 0; i < typ.NumFields(); i++ { - if typ.Field(i).Name() == chanName { - return - } - } - } - log.Fatalf("no field called %q in type %s", chanName, timerType) - }() - - selectorIsTimer := func(s *ast.SelectorExpr) bool { - selTyp := j.Program.Info.TypeOf(s.X) - if ptr, ok := selTyp.(*types.Pointer); ok { - selTyp = ptr.Elem() - } - return selTyp == timerType - } - - forAllFiles(j, func(n ast.Node) bool { - return walkForStmts(n, func(s ast.Stmt) bool { - return walkSelectStmts(s, func(s ast.Stmt) bool { - comm, ok := s.(*ast.CommClause) - if !ok || comm.Comm == nil /* default: */ { - return true - } - - // if receiving on a timer's C chan. - var unary ast.Expr - switch v := comm.Comm.(type) { - case *ast.AssignStmt: - // case `now := <-timer.C:` - unary = v.Rhs[0] - case *ast.ExprStmt: - // case `<-timer.C:` - unary = v.X - default: - return true - } - chanRead, ok := unary.(*ast.UnaryExpr) - if !ok || chanRead.Op != token.ARROW { - return true - } - selector, ok := chanRead.X.(*ast.SelectorExpr) - if !ok { - return true - } - if !selectorIsTimer(selector) { - return true - } - selectorName := fmt.Sprint(selector.X) - if selector.Sel.String() != chanName { - return true - } - - // Verify that the case body contains `timer.Read = true`. - noRead := walkStmts(comm.Body, func(s ast.Stmt) bool { - assign, ok := s.(*ast.AssignStmt) - if !ok || assign.Tok != token.ASSIGN { - return true - } - for i := range assign.Lhs { - l, r := assign.Lhs[i], assign.Rhs[i] - - // if assignment to correct field in timer. - assignSelector, ok := l.(*ast.SelectorExpr) - if !ok { - return true - } - if !selectorIsTimer(assignSelector) { - return true - } - if fmt.Sprint(assignSelector.X) != selectorName { - return true - } - if assignSelector.Sel.String() != "Read" { - return true - } - - // if assigning `true`. - val, ok := r.(*ast.Ident) - if !ok { - return true - } - if val.String() == "true" { - // returning false will short-circuit walkStmts and assign - // noRead to false instead of the default value of true. - return false - } - } - return true - }) - if noRead { - j.Errorf(comm, "must set timer.Read = true after reading from timer.C (see timeutil/timer.go)") - } - return true - }) - }) - }) -} - -func main() { - unusedChecker := unused.NewChecker(unused.CheckAll) - unusedChecker.WholeProgram = true - meta := metaChecker{ - checkers: []lint.Checker{ - simple.NewChecker(), - staticcheck.NewChecker(), - unused.NewLintChecker(unusedChecker), - }, - } - lintutil.ProcessArgs("metacheck", &meta, os.Args[1:]) -} diff --git a/pkg/keys/keys_test.go b/pkg/keys/keys_test.go index 13655581b718..1baef0a115bc 100644 --- a/pkg/keys/keys_test.go +++ b/pkg/keys/keys_test.go @@ -466,7 +466,7 @@ func TestMakeFamilyKey(t *testing.T) { const maxFamID = math.MaxUint32 key := MakeFamilyKey(nil, maxFamID) if expected, n := 6, len(key); expected != n { - t.Errorf("expected %d bytes, but got %d: [% x]", expected, n, []byte(key)) + t.Errorf("expected %d bytes, but got %d: [% x]", expected, n, key) } } diff --git a/pkg/kv/dist_sender_test.go b/pkg/kv/dist_sender_test.go index 0f0049ad6abd..679615ad2f5e 100644 --- a/pkg/kv/dist_sender_test.go +++ b/pkg/kv/dist_sender_test.go @@ -348,11 +348,11 @@ func TestSendRPCOrder(t *testing.T) { } ds.leaseHolderCache.Update( - context.TODO(), roachpb.RangeID(rangeID), roachpb.StoreID(0), + context.TODO(), rangeID, roachpb.StoreID(0), ) if tc.leaseHolder > 0 { ds.leaseHolderCache.Update( - context.TODO(), roachpb.RangeID(rangeID), descriptor.Replicas[tc.leaseHolder-1].StoreID, + context.TODO(), rangeID, descriptor.Replicas[tc.leaseHolder-1].StoreID, ) } @@ -1322,7 +1322,7 @@ func TestMultiRangeGapReverse(t *testing.T) { TestingKnobs: DistSenderTestingKnobs{ TransportFactory: SenderTransportFactory( tracing.NewTracer(), - client.SenderFunc(sender), + sender, ), }, } @@ -2075,9 +2075,9 @@ func TestMultiRangeSplitEndTransaction(t *testing.T) { var ba roachpb.BatchRequest ba.Txn = &roachpb.Transaction{Name: "test"} val := roachpb.MakeValueFromString("val") - ba.Add(roachpb.NewPut(roachpb.Key(test.put1), val)) + ba.Add(roachpb.NewPut(test.put1, val)) val = roachpb.MakeValueFromString("val") - ba.Add(roachpb.NewPut(roachpb.Key(test.put2), val)) + ba.Add(roachpb.NewPut(test.put2, val)) ba.Add(&roachpb.EndTransactionRequest{Span: roachpb.Span{Key: test.et}}) if _, pErr := ds.Send(context.Background(), ba); pErr != nil { @@ -2140,7 +2140,7 @@ func TestCountRanges(t *testing.T) { // Verify counted ranges. keyIn := func(desc roachpb.RangeDescriptor) roachpb.RKey { - return roachpb.RKey(append(desc.StartKey, 'a')) + return append(desc.StartKey, 'a') } testcases := []struct { key roachpb.RKey diff --git a/pkg/server/debug_logspy_test.go b/pkg/server/debug_logspy_test.go index a68207c95a74..6d53681d5194 100644 --- a/pkg/server/debug_logspy_test.go +++ b/pkg/server/debug_logspy_test.go @@ -72,7 +72,7 @@ func TestDebugLogSpyOptions(t *testing.T) { }, expOpts: logSpyOptions{ Count: 1, - Duration: durationAsString(logSpyMaxDuration), + Duration: logSpyMaxDuration, }, }, { diff --git a/pkg/server/node_test.go b/pkg/server/node_test.go index f63cbb325cc6..f3492018b050 100644 --- a/pkg/server/node_test.go +++ b/pkg/server/node_test.go @@ -555,7 +555,7 @@ func TestStatusSummaries(t *testing.T) { t.Fatal(err) } testutils.SucceedsSoon(t, func() error { - for i := 1; i <= int(initialRanges); i++ { + for i := 1; i <= initialRanges; i++ { if s.RaftStatus(roachpb.RangeID(i)) == nil { return errors.Errorf("Store %d replica %d is not present in raft", s.StoreID(), i) } diff --git a/pkg/server/server_test.go b/pkg/server/server_test.go index 257823011931..536ac536d898 100644 --- a/pkg/server/server_test.go +++ b/pkg/server/server_test.go @@ -273,7 +273,7 @@ func TestMultiRangeScanDeleteRange(t *testing.T) { del := &roachpb.DeleteRangeRequest{ Span: roachpb.Span{ Key: writes[0], - EndKey: roachpb.Key(writes[len(writes)-1]).Next(), + EndKey: writes[len(writes)-1].Next(), }, ReturnKeys: true, } diff --git a/pkg/sql/distsqlrun/disk_row_container_test.go b/pkg/sql/distsqlrun/disk_row_container_test.go index 17f00417b580..dac8a457527a 100644 --- a/pkg/sql/distsqlrun/disk_row_container_test.go +++ b/pkg/sql/distsqlrun/disk_row_container_test.go @@ -100,7 +100,7 @@ func TestDiskRowContainer(t *testing.T) { }, } - rng := rand.New(rand.NewSource(int64(timeutil.Now().UnixNano()))) + rng := rand.New(rand.NewSource(timeutil.Now().UnixNano())) evalCtx := parser.MakeTestingEvalContext() diskMonitor := mon.MakeMonitor( diff --git a/pkg/sql/distsqlrun/sorter_test.go b/pkg/sql/distsqlrun/sorter_test.go index 9b848a3ff564..f32e936cbb08 100644 --- a/pkg/sql/distsqlrun/sorter_test.go +++ b/pkg/sql/distsqlrun/sorter_test.go @@ -273,7 +273,7 @@ func BenchmarkSortAll(b *testing.B) { // One column integer rows. columnTypeInt := sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_INT} types := []sqlbase.ColumnType{columnTypeInt} - rng := rand.New(rand.NewSource(int64(timeutil.Now().UnixNano()))) + rng := rand.New(rand.NewSource(timeutil.Now().UnixNano())) spec := SorterSpec{ OutputOrdering: convertToSpecOrdering(sqlbase.ColumnOrdering{{ColIdx: 0, Direction: encoding.Ascending}}), @@ -316,7 +316,7 @@ func BenchmarkSortLimit(b *testing.B) { // One column integer rows. columnTypeInt := sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_INT} types := []sqlbase.ColumnType{columnTypeInt} - rng := rand.New(rand.NewSource(int64(timeutil.Now().UnixNano()))) + rng := rand.New(rand.NewSource(timeutil.Now().UnixNano())) spec := SorterSpec{ OutputOrdering: convertToSpecOrdering(sqlbase.ColumnOrdering{{ColIdx: 0, Direction: encoding.Ascending}}), diff --git a/pkg/sql/logictest/parallel_test.go b/pkg/sql/logictest/parallel_test.go index 96d047a5d04a..2c3c7738429a 100644 --- a/pkg/sql/logictest/parallel_test.go +++ b/pkg/sql/logictest/parallel_test.go @@ -241,7 +241,7 @@ func (t *parallelTest) setup(spec *parTestSpec) { func TestParallel(t *testing.T) { defer leaktest.AfterTest(t)() - glob := string(*paralleltestdata) + glob := *paralleltestdata paths, err := filepath.Glob(glob) if err != nil { t.Fatal(err) diff --git a/pkg/storage/allocator_scorer_test.go b/pkg/storage/allocator_scorer_test.go index 3eb49eb3b7b5..4a6d671e2fb3 100644 --- a/pkg/storage/allocator_scorer_test.go +++ b/pkg/storage/allocator_scorer_test.go @@ -542,7 +542,7 @@ func TestDiversityScore(t *testing.T) { for _, nodeID := range tc.existing { for _, s := range testStores { if s.Node.NodeID == nodeID { - existingNodeLocalities[roachpb.NodeID(s.Node.NodeID)] = s.Node.Locality + existingNodeLocalities[s.Node.NodeID] = s.Node.Locality } } } @@ -675,7 +675,7 @@ func TestRebalanceToDiversityScore(t *testing.T) { for _, nodeID := range tc.existing { for _, s := range testStores { if s.Node.NodeID == nodeID { - existingNodeLocalities[roachpb.NodeID(s.Node.NodeID)] = s.Node.Locality + existingNodeLocalities[s.Node.NodeID] = s.Node.Locality } } } @@ -757,7 +757,7 @@ func TestDiversityRemovalScore(t *testing.T) { existingNodeLocalities := make(map[roachpb.NodeID]roachpb.Locality) for _, s := range testStores { if _, ok := tc.expected[s.StoreID]; ok { - existingNodeLocalities[roachpb.NodeID(s.Node.NodeID)] = s.Node.Locality + existingNodeLocalities[s.Node.NodeID] = s.Node.Locality } } for _, s := range testStores { diff --git a/pkg/storage/client_split_test.go b/pkg/storage/client_split_test.go index 10c803b545ef..b74e18f14ea9 100644 --- a/pkg/storage/client_split_test.go +++ b/pkg/storage/client_split_test.go @@ -192,19 +192,17 @@ func TestStoreRangeSplitInsideRow(t *testing.T) { // Verify the two columns are still on the same range. if !reflect.DeepEqual(repl1, repl2) { - t.Fatalf("%s: ranges differ: %+v vs %+v", roachpb.Key(col1Key), repl1, repl2) + t.Fatalf("%s: ranges differ: %+v vs %+v", col1Key, repl1, repl2) } // Verify we split on a row key. if startKey := repl1.Desc().StartKey; !startKey.Equal(rowKey) { - t.Fatalf("%s: expected split on %s, but found %s", - roachpb.Key(col1Key), roachpb.Key(rowKey), startKey) + t.Fatalf("%s: expected split on %s, but found %s", col1Key, rowKey, startKey) } // Verify the previous range was split on a row key. repl3 := store.LookupReplica(tableKey, nil) if endKey := repl3.Desc().EndKey; !endKey.Equal(rowKey) { - t.Fatalf("%s: expected split on %s, but found %s", - roachpb.Key(col1Key), roachpb.Key(rowKey), endKey) + t.Fatalf("%s: expected split on %s, but found %s", col1Key, rowKey, endKey) } } @@ -975,7 +973,7 @@ func TestStoreRangeSystemSplits(t *testing.T) { testutils.MakeKey(keys.Meta2Prefix, keys.TableDataMin), } numReservedTables := schema.SystemDescriptorCount() - schema.SystemConfigDescriptorCount() - for i := 1; i <= int(numReservedTables); i++ { + for i := 1; i <= numReservedTables; i++ { expKeys = append(expKeys, testutils.MakeKey(keys.Meta2Prefix, keys.MakeTablePrefix(keys.MaxSystemConfigDescID+uint32(i))), @@ -1422,8 +1420,8 @@ func TestStoreSplitGCThreshold(t *testing.T) { } gcArgs := &roachpb.GCRequest{ Span: roachpb.Span{ - Key: roachpb.Key(leftKey), - EndKey: roachpb.Key(rightKey), + Key: leftKey, + EndKey: rightKey, }, Threshold: specifiedGCThreshold, TxnSpanGCThreshold: specifiedTxnSpanGCThreshold, diff --git a/pkg/storage/engine/bench_test.go b/pkg/storage/engine/bench_test.go index 7702618aa57f..3b162ee06bad 100644 --- a/pkg/storage/engine/bench_test.go +++ b/pkg/storage/engine/bench_test.go @@ -659,7 +659,7 @@ func BenchmarkMVCCPutDelete_RocksDB(b *testing.B) { rocksdb := setupMVCCInMemRocksDB(b, "put_delete") defer rocksdb.Close() - r := rand.New(rand.NewSource(int64(timeutil.Now().UnixNano()))) + r := rand.New(rand.NewSource(timeutil.Now().UnixNano())) value := roachpb.MakeValueFromBytes(randutil.RandBytes(r, 10)) var blockNum int64 diff --git a/pkg/storage/engine/disk_map_test.go b/pkg/storage/engine/disk_map_test.go index 768a46fcee35..e6f4f320ab23 100644 --- a/pkg/storage/engine/disk_map_test.go +++ b/pkg/storage/engine/disk_map_test.go @@ -50,7 +50,7 @@ func TestRocksDBMap(t *testing.T) { } }() - rng := rand.New(rand.NewSource(int64(timeutil.Now().UnixNano()))) + rng := rand.New(rand.NewSource(timeutil.Now().UnixNano())) numKeysToWrite := 1 << 12 keys := make([]string, numKeysToWrite) @@ -319,7 +319,7 @@ func BenchmarkRocksDBMapWrite(b *testing.B) { } defer tempEngine.Close() - rng := rand.New(rand.NewSource(int64(timeutil.Now().UnixNano()))) + rng := rand.New(rand.NewSource(timeutil.Now().UnixNano())) for _, inputSize := range []int{1 << 12, 1 << 14, 1 << 16, 1 << 18, 1 << 20} { b.Run(fmt.Sprintf("InputSize%d", inputSize), func(b *testing.B) { @@ -366,7 +366,7 @@ func BenchmarkRocksDBMapIteration(b *testing.B) { diskMap := NewRocksDBMap(tempEngine) defer diskMap.Close(context.Background()) - rng := rand.New(rand.NewSource(int64(timeutil.Now().UnixNano()))) + rng := rand.New(rand.NewSource(timeutil.Now().UnixNano())) for _, inputSize := range []int{1 << 12, 1 << 14, 1 << 16, 1 << 18, 1 << 20} { for i := 0; i < inputSize; i++ { diff --git a/pkg/storage/gc_queue_test.go b/pkg/storage/gc_queue_test.go index e8126904d462..36cbf3a45359 100644 --- a/pkg/storage/gc_queue_test.go +++ b/pkg/storage/gc_queue_test.go @@ -97,7 +97,7 @@ func TestGCQueueMakeGCScoreInvariantQuick(t *testing.T) { if err := quick.Check(func( seed uint16, uTTLSec uint32, uTimePassed time.Duration, uGCBytes uint32, uGCByteAge uint32, ) bool { - ttlSec, timePassed := int32(uTTLSec), time.Duration(uTimePassed) + ttlSec, timePassed := int32(uTTLSec), uTimePassed gcBytes, gcByteAge := int64(uGCBytes), int64(uGCByteAge) ms := enginepb.MVCCStats{ @@ -765,7 +765,7 @@ func TestGCQueueTransactionTable(t *testing.T) { tc.repl.mu.Unlock() // Verify that the new TxnSpanGCThreshold has reached the Replica. - if expWT := int64(gcTxnAndAC); txnSpanThreshold.WallTime != expWT { + if expWT := gcTxnAndAC; txnSpanThreshold.WallTime != expWT { t.Fatalf("expected TxnSpanGCThreshold.Walltime %d, got timestamp %s", expWT, txnSpanThreshold) } diff --git a/pkg/storage/replica_test.go b/pkg/storage/replica_test.go index 37f4c4b97d4d..84961cc0fe1c 100644 --- a/pkg/storage/replica_test.go +++ b/pkg/storage/replica_test.go @@ -6374,7 +6374,7 @@ func TestReplicaLookupUseReverseScan(t *testing.T) { if err != nil { t.Fatal(err) } - pArgs := putArgs(keys.RangeMetaKey(roachpb.RKey(r.EndKey)).AsRawKey(), data) + pArgs := putArgs(keys.RangeMetaKey(r.EndKey).AsRawKey(), data) txn.Sequence++ if _, pErr := tc.SendWrappedWith(roachpb.Header{Txn: txn}, &pArgs); pErr != nil { @@ -6416,7 +6416,7 @@ func TestReplicaLookupUseReverseScan(t *testing.T) { if err != nil { t.Fatal(err) } - pArgs := putArgs(keys.RangeMetaKey(roachpb.RKey(r.EndKey)).AsRawKey(), data) + pArgs := putArgs(keys.RangeMetaKey(r.EndKey).AsRawKey(), data) txn.Sequence++ if _, pErr := tc.SendWrappedWith(roachpb.Header{Txn: txn}, &pArgs); pErr != nil { diff --git a/pkg/ts/query_test.go b/pkg/ts/query_test.go index 202c9c0ce2f2..ec73ad8b3f54 100644 --- a/pkg/ts/query_test.go +++ b/pkg/ts/query_test.go @@ -205,10 +205,10 @@ func TestDataSpanIterator(t *testing.T) { if !expectedValid { return } - if a, e := iter.offset(), int32(expectedOffset); a != e { + if a, e := iter.offset(), expectedOffset; a != e { t.Fatalf("check %d: expected iterator to have offset %d, was %d", checkNum, e, a) } - if a, e := iter.timestamp(), int64(expectedTimestamp); a != e { + if a, e := iter.timestamp(), expectedTimestamp; a != e { t.Fatalf( "check %d: expected iterator to have timestamp %d, was %d", checkNum, e, a, ) @@ -368,15 +368,15 @@ func TestDownsamplingIterator(t *testing.T) { if !expectedValid { return } - if a, e := iter.offset(), int32(expectedOffset); a != e { + if a, e := iter.offset(), expectedOffset; a != e { t.Fatalf("check %d: expected iterator to have offset %d, was %d", checkNum, e, a) } - if a, e := iter.timestamp(), int64(expectedTimestamp); a != e { + if a, e := iter.timestamp(), expectedTimestamp; a != e { t.Fatalf( "check %d: expected iterator to have timestamp %d, was %d", checkNum, e, a, ) } - if a, e := iter.value(), float64(expectedValue); a != e { + if a, e := iter.value(), expectedValue; a != e { t.Fatalf( "check %d: expected iterator to have value %f, was %f", checkNum, e, a, ) diff --git a/pkg/util/interval/btree_based_interval_test.go b/pkg/util/interval/btree_based_interval_test.go index 969667deb5e2..387f85ff58d6 100644 --- a/pkg/util/interval/btree_based_interval_test.go +++ b/pkg/util/interval/btree_based_interval_test.go @@ -47,8 +47,7 @@ func perm(n uint32) (out items) { // rang returns an ordered list of intervals in the range [m, n]. func rang(m, n uint32) (out items) { - for i := m; i <= n; i++ { - u := uint32(i) + for u := m; u <= n; u++ { iv := makeMultiByteInterval(u, u+1, u) out = append(out, iv) } diff --git a/pkg/util/log/clog_test.go b/pkg/util/log/clog_test.go index 51528f60f6ac..2c1396b2ad8e 100644 --- a/pkg/util/log/clog_test.go +++ b/pkg/util/log/clog_test.go @@ -621,7 +621,7 @@ func TestGC(t *testing.T) { defer func(previous int64) { atomic.StoreInt64(&LogFilesCombinedMaxSize, previous) }(LogFilesCombinedMaxSize) - atomic.StoreInt64(&LogFilesCombinedMaxSize, int64(maxTotalLogFileSize)) + atomic.StoreInt64(&LogFilesCombinedMaxSize, maxTotalLogFileSize) for i := 1; i < newLogFiles; i++ { Infof(context.Background(), "%d", i) diff --git a/pkg/util/syncutil/int_map_bench_test.go b/pkg/util/syncutil/int_map_bench_test.go index 84328b888bcd..85cc8e219fe0 100644 --- a/pkg/util/syncutil/int_map_bench_test.go +++ b/pkg/util/syncutil/int_map_bench_test.go @@ -215,7 +215,7 @@ func BenchmarkAdversarialDelete(b *testing.B) { if i%mapSize == 0 { m.Range(func(k int64, _ unsafe.Pointer) bool { - m.Delete(int64(k)) + m.Delete(k) return false }) m.Store(int64(i), v) diff --git a/vendor b/vendor index 3781d4f4ecc9..bbdd4155885b 160000 --- a/vendor +++ b/vendor @@ -1 +1 @@ -Subproject commit 3781d4f4ecc939eccbd3990d83d3a4bd10437e1f +Subproject commit bbdd4155885b0abf10f093eeac150b7f583af981