Browse Source

Merge pull request #9665 from gyuho/unconvert

test: integrate github.com/mdempsky/unconvert
Gyuho Lee 7 years ago
parent
commit
200401248a

+ 6 - 6
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

+ 3 - 3
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)))
 	}
 }

+ 2 - 2
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
 	}

+ 1 - 1
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),

+ 1 - 1
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,
 		}

+ 2 - 2
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")

+ 1 - 1
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)

+ 1 - 1
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")

+ 1 - 1
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 {

+ 4 - 4
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
 	}

+ 1 - 1
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.

+ 1 - 1
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,

+ 3 - 3
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

+ 2 - 2
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 {

+ 2 - 2
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 {

+ 21 - 2
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 \

+ 1 - 0
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 \

+ 1 - 1
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 {