Merge pull request #9665 from gyuho/unconvert

test: integrate github.com/mdempsky/unconvert
release-3.4
Gyuho Lee 2018-05-01 09:52:44 -07:00 committed by GitHub
commit 200401248a
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
18 changed files with 54 additions and 34 deletions

View File

@ -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

View File

@ -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)))
}
}

View File

@ -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
}

View File

@ -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),

View File

@ -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,
}

View File

@ -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")

View File

@ -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)

View File

@ -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")

View File

@ -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 {

View File

@ -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
}

View File

@ -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.

View File

@ -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,

View File

@ -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

View File

@ -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 {

View File

@ -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 {

23
test
View File

@ -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 \

View File

@ -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 \

View File

@ -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 {