diff --git a/auth/store.go b/auth/store.go index b2a9ae013..a18d5fa52 100644 --- a/auth/store.go +++ b/auth/store.go @@ -683,7 +683,7 @@ func (as *authStore) RoleRevokePermission(r *pb.AuthRoleRevokePermissionRequest) } for _, perm := range role.KeyPermission { - if !bytes.Equal(perm.Key, []byte(r.Key)) || !bytes.Equal(perm.RangeEnd, []byte(r.RangeEnd)) { + if !bytes.Equal(perm.Key, r.Key) || !bytes.Equal(perm.RangeEnd, r.RangeEnd) { updatedRole.KeyPermission = append(updatedRole.KeyPermission, perm) } } @@ -821,7 +821,7 @@ func (as *authStore) RoleGrantPermission(r *pb.AuthRoleGrantPermissionRequest) ( } idx := sort.Search(len(role.KeyPermission), func(i int) bool { - return bytes.Compare(role.KeyPermission[i].Key, []byte(r.Perm.Key)) >= 0 + return bytes.Compare(role.KeyPermission[i].Key, r.Perm.Key) >= 0 }) if idx < len(role.KeyPermission) && bytes.Equal(role.KeyPermission[idx].Key, r.Perm.Key) && bytes.Equal(role.KeyPermission[idx].RangeEnd, r.Perm.RangeEnd) { @@ -830,8 +830,8 @@ func (as *authStore) RoleGrantPermission(r *pb.AuthRoleGrantPermissionRequest) ( } else { // append new permission to the role newPerm := &authpb.Permission{ - Key: []byte(r.Perm.Key), - RangeEnd: []byte(r.Perm.RangeEnd), + Key: r.Perm.Key, + RangeEnd: r.Perm.RangeEnd, PermType: r.Perm.PermType, } @@ -1046,7 +1046,7 @@ func putRole(lg *zap.Logger, tx backend.BatchTx, role *authpb.Role) { } } - tx.UnsafePut(authRolesBucketName, []byte(role.Name), b) + tx.UnsafePut(authRolesBucketName, role.Name, b) } func delRole(tx backend.BatchTx, rolename string) { @@ -1113,7 +1113,7 @@ func (as *authStore) commitRevision(tx backend.BatchTx) { } func getRevision(tx backend.BatchTx) uint64 { - _, vs := tx.UnsafeRange(authBucketName, []byte(revisionKey), nil, 0) + _, vs := tx.UnsafeRange(authBucketName, revisionKey, nil, 0) if len(vs) != 1 { // this can happen in the initialization phase return 0 diff --git a/etcdctl/ctlv3/command/check.go b/etcdctl/ctlv3/command/check.go index e347af295..f9b9cd65c 100644 --- a/etcdctl/ctlv3/command/check.go +++ b/etcdctl/ctlv3/command/check.go @@ -192,7 +192,7 @@ func newCheckPerfCommand(cmd *cobra.Command, args []string) { cctx, ccancel := context.WithTimeout(context.Background(), time.Duration(cfg.duration)*time.Second) defer ccancel() for limit.Wait(cctx) == nil { - binary.PutVarint(k, int64(rand.Int63n(math.MaxInt64))) + binary.PutVarint(k, rand.Int63n(math.MaxInt64)) requests <- v3.OpPut(checkPerfPrefix+string(k), v) } close(requests) @@ -349,7 +349,7 @@ func newCheckDatascaleCommand(cmd *cobra.Command, args []string) { go func() { for i := 0; i < cfg.limit; i++ { - binary.PutVarint(k, int64(rand.Int63n(math.MaxInt64))) + binary.PutVarint(k, rand.Int63n(math.MaxInt64)) requests <- v3.OpPut(checkDatascalePrefix+string(k), v) } close(requests) @@ -400,6 +400,6 @@ func newCheckDatascaleCommand(cmd *cobra.Command, args []string) { } os.Exit(ExitError) } else { - fmt.Println(fmt.Sprintf("PASS: Approximate system memory used : %v MB.", strconv.FormatFloat(float64(mbUsed), 'f', 2, 64))) + fmt.Println(fmt.Sprintf("PASS: Approximate system memory used : %v MB.", strconv.FormatFloat(mbUsed, 'f', 2, 64))) } } diff --git a/etcdserver/api/v2http/metrics.go b/etcdserver/api/v2http/metrics.go index 9f93d8702..2e8998444 100644 --- a/etcdserver/api/v2http/metrics.go +++ b/etcdserver/api/v2http/metrics.go @@ -87,9 +87,9 @@ func codeFromError(err error) int { } switch e := err.(type) { case *v2error.Error: - return (*v2error.Error)(e).StatusCode() + return e.StatusCode() case *httptypes.HTTPError: - return (*httptypes.HTTPError)(e).Code + return e.Code default: return http.StatusInternalServerError } diff --git a/etcdserver/api/v2v3/store.go b/etcdserver/api/v2v3/store.go index fd1dc78b3..b3da8091e 100644 --- a/etcdserver/api/v2v3/store.go +++ b/etcdserver/api/v2v3/store.go @@ -601,7 +601,7 @@ func (s *v2v3Store) mkV2Node(kv *mvccpb.KeyValue) *v2store.NodeExtern { return nil } n := &v2store.NodeExtern{ - Key: string(s.mkNodePath(string(kv.Key))), + Key: s.mkNodePath(string(kv.Key)), Dir: kv.Key[len(kv.Key)-1] == '/', CreatedIndex: mkV2Rev(kv.CreateRevision), ModifiedIndex: mkV2Rev(kv.ModRevision), diff --git a/etcdserver/corrupt.go b/etcdserver/corrupt.go index 2170121a0..5c88d355d 100644 --- a/etcdserver/corrupt.go +++ b/etcdserver/corrupt.go @@ -204,7 +204,7 @@ func (s *EtcdServer) checkHashKV() error { } alarmed = true a := &pb.AlarmRequest{ - MemberID: uint64(id), + MemberID: id, Action: pb.AlarmRequest_ACTIVATE, Alarm: pb.AlarmType_CORRUPT, } diff --git a/etcdserver/server.go b/etcdserver/server.go index 8511383af..eeb8582a1 100644 --- a/etcdserver/server.go +++ b/etcdserver/server.go @@ -2002,7 +2002,7 @@ func (s *EtcdServer) applyConfChange(cc raftpb.ConfChange, confState *raftpb.Con lg.Panic( "got different member ID", zap.String("member-id-from-config-change-entry", types.ID(cc.NodeID).String()), - zap.String("member-id-from-message", types.ID(m.ID).String()), + zap.String("member-id-from-message", m.ID.String()), ) } else { plog.Panicf("nodeID should always be equal to member ID") @@ -2035,7 +2035,7 @@ func (s *EtcdServer) applyConfChange(cc raftpb.ConfChange, confState *raftpb.Con lg.Panic( "got different member ID", zap.String("member-id-from-config-change-entry", types.ID(cc.NodeID).String()), - zap.String("member-id-from-message", types.ID(m.ID).String()), + zap.String("member-id-from-message", m.ID.String()), ) } else { plog.Panicf("nodeID should always be equal to member ID") diff --git a/functional/runner/global.go b/functional/runner/global.go index 94a3a2aae..13f1906c3 100644 --- a/functional/runner/global.go +++ b/functional/runner/global.go @@ -47,7 +47,7 @@ type roundClient struct { func newClient(eps []string, timeout time.Duration) *clientv3.Client { c, err := clientv3.New(clientv3.Config{ Endpoints: eps, - DialTimeout: time.Duration(timeout) * time.Second, + DialTimeout: timeout * time.Second, }) if err != nil { log.Fatal(err) diff --git a/lease/lessor.go b/lease/lessor.go index e8208c0cb..db5b34cc2 100644 --- a/lease/lessor.go +++ b/lease/lessor.go @@ -628,7 +628,7 @@ func (l *Lease) expired() bool { func (l *Lease) persistTo(b backend.Backend) { key := int64ToBytes(int64(l.ID)) - lpb := leasepb.Lease{ID: int64(l.ID), TTL: int64(l.ttl)} + lpb := leasepb.Lease{ID: int64(l.ID), TTL: l.ttl} val, err := lpb.Marshal() if err != nil { panic("failed to marshal lease proto item") diff --git a/mvcc/backend/backend.go b/mvcc/backend/backend.go index 2c16b5d4a..3438e6780 100644 --- a/mvcc/backend/backend.go +++ b/mvcc/backend/backend.go @@ -45,7 +45,7 @@ var ( plog = capnslog.NewPackageLogger("github.com/coreos/etcd", "mvcc/backend") // minSnapshotWarningTimeout is the minimum threshold to trigger a long running snapshot warning. - minSnapshotWarningTimeout = time.Duration(30 * time.Second) + minSnapshotWarningTimeout = 30 * time.Second ) type Backend interface { diff --git a/mvcc/kvstore_txn.go b/mvcc/kvstore_txn.go index 8896fb86d..d568d8a2e 100644 --- a/mvcc/kvstore_txn.go +++ b/mvcc/kvstore_txn.go @@ -83,14 +83,14 @@ func (tw *storeTxnWrite) Range(key, end []byte, ro RangeOptions) (r *RangeResult func (tw *storeTxnWrite) DeleteRange(key, end []byte) (int64, int64) { if n := tw.deleteRange(key, end); n != 0 || len(tw.changes) > 0 { - return n, int64(tw.beginRev + 1) + return n, tw.beginRev + 1 } - return 0, int64(tw.beginRev) + return 0, tw.beginRev } func (tw *storeTxnWrite) Put(key, value []byte, lease lease.LeaseID) int64 { tw.put(key, value, lease) - return int64(tw.beginRev + 1) + return tw.beginRev + 1 } func (tw *storeTxnWrite) End() { @@ -120,7 +120,7 @@ func (tr *storeTxnRead) rangeKeys(key, end []byte, curRev int64, ro RangeOptions return &RangeResult{KVs: nil, Count: -1, Rev: 0}, ErrCompacted } - revpairs := tr.s.kvindex.Revisions(key, end, int64(rev)) + revpairs := tr.s.kvindex.Revisions(key, end, rev) if len(revpairs) == 0 { return &RangeResult{KVs: nil, Count: 0, Rev: curRev}, nil } diff --git a/pkg/flags/unique_strings.go b/pkg/flags/unique_strings.go index 909cc0012..e220ee07a 100644 --- a/pkg/flags/unique_strings.go +++ b/pkg/flags/unique_strings.go @@ -67,7 +67,7 @@ func NewUniqueStringsValue(s string) (us *UniqueStringsValue) { // UniqueStringsFromFlag returns a string slice from the flag. func UniqueStringsFromFlag(fs *flag.FlagSet, flagName string) []string { - return []string((*fs.Lookup(flagName).Value.(*UniqueStringsValue)).stringSlice()) + return (*fs.Lookup(flagName).Value.(*UniqueStringsValue)).stringSlice() } // UniqueStringsMapFromFlag returns a map of strings from the flag. diff --git a/pkg/report/timeseries.go b/pkg/report/timeseries.go index bc1493b2c..a999c2dcc 100644 --- a/pkg/report/timeseries.go +++ b/pkg/report/timeseries.go @@ -102,7 +102,7 @@ func (sp *secondPoints) getTimeSeries() TimeSeries { for k, v := range sp.tm { var lat time.Duration if v.count > 0 { - lat = time.Duration(v.totalLatency) / time.Duration(v.count) + lat = v.totalLatency / time.Duration(v.count) } tslice[i] = DataPoint{ Timestamp: k, diff --git a/rafthttp/msgappv2_codec.go b/rafthttp/msgappv2_codec.go index 013ffe7c7..7f78b35b0 100644 --- a/rafthttp/msgappv2_codec.go +++ b/rafthttp/msgappv2_codec.go @@ -86,12 +86,12 @@ func (enc *msgAppV2Encoder) encode(m *raftpb.Message) error { start := time.Now() switch { case isLinkHeartbeatMessage(m): - enc.uint8buf[0] = byte(msgTypeLinkHeartbeat) + enc.uint8buf[0] = msgTypeLinkHeartbeat if _, err := enc.w.Write(enc.uint8buf); err != nil { return err } case enc.index == m.Index && enc.term == m.LogTerm && m.LogTerm == m.Term: - enc.uint8buf[0] = byte(msgTypeAppEntries) + enc.uint8buf[0] = msgTypeAppEntries if _, err := enc.w.Write(enc.uint8buf); err != nil { return err } @@ -179,7 +179,7 @@ func (dec *msgAppV2Decoder) decode() (raftpb.Message, error) { if _, err := io.ReadFull(dec.r, dec.uint8buf); err != nil { return m, err } - typ = uint8(dec.uint8buf[0]) + typ = dec.uint8buf[0] switch typ { case msgTypeLinkHeartbeat: return linkHeartbeatMessage, nil diff --git a/rafthttp/peer.go b/rafthttp/peer.go index 6ea60ddb5..258e6f03a 100644 --- a/rafthttp/peer.go +++ b/rafthttp/peer.go @@ -256,7 +256,7 @@ func (p *peer) send(m raftpb.Message) { zap.String("message-type", m.Type.String()), zap.String("local-member-id", p.localID.String()), zap.String("from", types.ID(m.From).String()), - zap.String("remote-peer-id", types.ID(p.id).String()), + zap.String("remote-peer-id", p.id.String()), zap.Bool("remote-peer-active", p.status.isActive()), ) } else { @@ -269,7 +269,7 @@ func (p *peer) send(m raftpb.Message) { zap.String("message-type", m.Type.String()), zap.String("local-member-id", p.localID.String()), zap.String("from", types.ID(m.From).String()), - zap.String("remote-peer-id", types.ID(p.id).String()), + zap.String("remote-peer-id", p.id.String()), zap.Bool("remote-peer-active", p.status.isActive()), ) } else { diff --git a/rafthttp/remote.go b/rafthttp/remote.go index 4813843a0..de3e1cc4a 100644 --- a/rafthttp/remote.go +++ b/rafthttp/remote.go @@ -62,7 +62,7 @@ func (g *remote) send(m raftpb.Message) { zap.String("message-type", m.Type.String()), zap.String("local-member-id", g.localID.String()), zap.String("from", types.ID(m.From).String()), - zap.String("remote-peer-id", types.ID(g.id).String()), + zap.String("remote-peer-id", g.id.String()), zap.Bool("remote-peer-active", g.status.isActive()), ) } else { @@ -75,7 +75,7 @@ func (g *remote) send(m raftpb.Message) { zap.String("message-type", m.Type.String()), zap.String("local-member-id", g.localID.String()), zap.String("from", types.ID(m.From).String()), - zap.String("remote-peer-id", types.ID(g.id).String()), + zap.String("remote-peer-id", g.id.String()), zap.Bool("remote-peer-active", g.status.isActive()), ) } else { diff --git a/test b/test index d1e8b0e02..a5276276a 100755 --- a/test +++ b/test @@ -82,6 +82,8 @@ fi # shellcheck disable=SC2206 FMT=($FMT) +# shellcheck disable=SC2128 +echo "Running with FMT:" "${FMT}" # prepend REPO_PATH to each local package split=$TEST @@ -90,12 +92,16 @@ for a in $split; do TEST="$TEST ${REPO_PATH}/${a}"; done # shellcheck disable=SC2206 TEST=($TEST) +# shellcheck disable=SC2128 +echo "Running with TEST:" "${TEST}" # TODO: 'client' pkg fails with gosimple from generated files # TODO: 'rafttest' is failing with unused -STATIC_ANALYSIS_PATHS=$(find . -name \*.go | while read -r a; do dirname "$a"; done | sort | uniq | grep -vE "$IGNORE_PKGS" | grep -v 'client') +STATIC_ANALYSIS_PATHS=$(find . -name \*.go ! -path './vendor/*' ! -path './gopath.proto/*' ! -path '*pb/*' | while read -r a; do dirname "$a"; done | sort | uniq | grep -vE "$IGNORE_PKGS" | grep -v 'client') # shellcheck disable=SC2206 STATIC_ANALYSIS_PATHS=($STATIC_ANALYSIS_PATHS) +# shellcheck disable=SC2128 +echo "Running with STATIC_ANALYSIS_PATHS:" "${STATIC_ANALYSIS_PATHS}" if [ -z "$GOARCH" ]; then GOARCH=$(go env GOARCH); @@ -106,7 +112,7 @@ TEST_CPUS="1,2,4" if [ ! -z "${CPU}" ]; then TEST_CPUS="${CPU}" fi -echo "Running with" "${TEST_CPUS}" +echo "Running with TEST_CPUS:" "${TEST_CPUS}" # determine whether target supports race detection if [ "$GOARCH" == "amd64" ]; then @@ -487,6 +493,18 @@ function staticcheck_pass { fi } +function unconvert_pass { + if which unconvert >/dev/null; then + unconvertResult=$(unconvert -v "${STATIC_ANALYSIS_PATHS[@]}" 2>&1 || true) + if [ -n "${unconvertResult}" ]; then + echo -e "unconvert checking failed:\\n${unconvertResult}" + exit 255 + fi + else + echo "Skipping unconvert..." + fi +} + function ineffassign_pass { if which ineffassign >/dev/null; then ineffassignResult=$(ineffassign "${STATIC_ANALYSIS_PATHS[@]}" 2>&1 || true) @@ -578,6 +596,7 @@ function fmt_pass { gosimple \ unused \ staticcheck \ + unconvert \ ineffassign \ nakedret \ license_header \ diff --git a/tests/Dockerfile b/tests/Dockerfile index 44f768da3..8693f3237 100644 --- a/tests/Dockerfile +++ b/tests/Dockerfile @@ -33,6 +33,7 @@ RUN ln -s /lib64/libhunspell-1.6.so /lib64/libhunspell.so RUN go get -v -u -tags spell github.com/chzchzchz/goword \ && go get -v -u github.com/coreos/license-bill-of-materials \ + && go get -v -u github.com/mdempsky/unconvert \ && go get -v -u honnef.co/go/tools/cmd/gosimple \ && go get -v -u honnef.co/go/tools/cmd/unused \ && go get -v -u honnef.co/go/tools/cmd/staticcheck \ diff --git a/wal/repair.go b/wal/repair.go index 2a7a39fe9..a61db014e 100644 --- a/wal/repair.go +++ b/wal/repair.go @@ -92,7 +92,7 @@ func Repair(lg *zap.Logger, dirpath string) bool { return false } - if err = f.Truncate(int64(lastOffset)); err != nil { + if err = f.Truncate(lastOffset); err != nil { if lg != nil { lg.Warn("failed to truncate", zap.String("path", f.Name())) } else {