*: fix go tool vet -all -shadow errors

release-3.3
Anthony Romano 2017-06-03 21:31:20 -07:00
parent 9b33aa1967
commit 887db5a3db
16 changed files with 50 additions and 79 deletions

View File

@ -30,7 +30,7 @@ import (
"google.golang.org/grpc"
)
func ExampleMetrics_range() {
func ExampleClient_metrics() {
cli, err := clientv3.New(clientv3.Config{
Endpoints: endpoints,
DialOptions: []grpc.DialOption{

View File

@ -288,14 +288,11 @@ func (rc *raftNode) startRaft() {
rc.node = raft.StartNode(c, startPeers)
}
ss := &stats.ServerStats{}
ss.Initialize()
rc.transport = &rafthttp.Transport{
ID: types.ID(rc.id),
ClusterID: 0x1000,
Raft: rc,
ServerStats: ss,
ServerStats: stats.NewServerStats("", ""),
LeaderStats: stats.NewLeaderStats(strconv.Itoa(rc.id)),
ErrorC: make(chan error),
}

View File

@ -80,7 +80,7 @@ func testElect(cx ctlCtx) {
if err = blocked.Signal(os.Interrupt); err != nil {
cx.t.Fatal(err)
}
if err := closeWithTimeout(blocked, time.Second); err != nil {
if err = closeWithTimeout(blocked, time.Second); err != nil {
cx.t.Fatal(err)
}

View File

@ -150,8 +150,8 @@ func newCheckPerfCommand(cmd *cobra.Command, args []string) {
}
go func() {
cctx, _ := context.WithTimeout(context.Background(), time.Duration(cfg.duration)*time.Second)
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)))
requests <- v3.OpPut(checkPerfPrefix+string(k), v)

View File

@ -401,11 +401,7 @@ func NewServer(cfg *ServerConfig) (srv *EtcdServer, err error) {
return nil, fmt.Errorf("cannot access member directory: %v", terr)
}
sstats := &stats.ServerStats{
Name: cfg.Name,
ID: id.String(),
}
sstats.Initialize()
sstats := stats.NewServerStats(cfg.Name, id.String())
lstats := stats.NewLeaderStats(id.String())
heartbeat := time.Duration(cfg.TickMs) * time.Millisecond

View File

@ -24,25 +24,30 @@ import (
// LeaderStats is used by the leader in an etcd cluster, and encapsulates
// statistics about communication with its followers
type LeaderStats struct {
leaderStats
sync.Mutex
}
type leaderStats struct {
// Leader is the ID of the leader in the etcd cluster.
// TODO(jonboulle): clarify that these are IDs, not names
Leader string `json:"leader"`
Followers map[string]*FollowerStats `json:"followers"`
sync.Mutex
}
// NewLeaderStats generates a new LeaderStats with the given id as leader
func NewLeaderStats(id string) *LeaderStats {
return &LeaderStats{
Leader: id,
Followers: make(map[string]*FollowerStats),
leaderStats: leaderStats{
Leader: id,
Followers: make(map[string]*FollowerStats),
},
}
}
func (ls *LeaderStats) JSON() []byte {
ls.Lock()
stats := *ls
stats := ls.leaderStats
ls.Unlock()
b, err := json.Marshal(stats)
// TODO(jonboulle): appropriate error handling?

View File

@ -26,6 +26,26 @@ import (
// ServerStats encapsulates various statistics about an EtcdServer and its
// communication with other members of the cluster
type ServerStats struct {
serverStats
sync.Mutex
}
func NewServerStats(name, id string) *ServerStats {
ss := &ServerStats{
serverStats: serverStats{
Name: name,
ID: id,
},
}
now := time.Now()
ss.StartTime = now
ss.LeaderInfo.StartTime = now
ss.sendRateQueue = &statsQueue{back: -1}
ss.recvRateQueue = &statsQueue{back: -1}
return ss
}
type serverStats struct {
Name string `json:"name"`
// ID is the raft ID of the node.
// TODO(jonboulle): use ID instead of name?
@ -49,17 +69,15 @@ type ServerStats struct {
sendRateQueue *statsQueue
recvRateQueue *statsQueue
sync.Mutex
}
func (ss *ServerStats) JSON() []byte {
ss.Lock()
stats := *ss
stats := ss.serverStats
ss.Unlock()
stats.LeaderInfo.Uptime = time.Since(stats.LeaderInfo.StartTime).String()
stats.SendingPkgRate, stats.SendingBandwidthRate = stats.SendRates()
stats.RecvingPkgRate, stats.RecvingBandwidthRate = stats.RecvRates()
stats.SendingPkgRate, stats.SendingBandwidthRate = stats.sendRateQueue.Rate()
stats.RecvingPkgRate, stats.RecvingBandwidthRate = stats.recvRateQueue.Rate()
b, err := json.Marshal(stats)
// TODO(jonboulle): appropriate error handling?
if err != nil {
@ -68,32 +86,6 @@ func (ss *ServerStats) JSON() []byte {
return b
}
// Initialize clears the statistics of ServerStats and resets its start time
func (ss *ServerStats) Initialize() {
if ss == nil {
return
}
now := time.Now()
ss.StartTime = now
ss.LeaderInfo.StartTime = now
ss.sendRateQueue = &statsQueue{
back: -1,
}
ss.recvRateQueue = &statsQueue{
back: -1,
}
}
// RecvRates calculates and returns the rate of received append requests
func (ss *ServerStats) RecvRates() (float64, float64) {
return ss.recvRateQueue.Rate()
}
// SendRates calculates and returns the rate of sent append requests
func (ss *ServerStats) SendRates() (float64, float64) {
return ss.sendRateQueue.Rate()
}
// RecvAppendReq updates the ServerStats in response to an AppendRequest
// from the given leader being received
func (ss *ServerStats) RecvAppendReq(leader string, reqSize int) {

View File

@ -1402,9 +1402,9 @@ func TestTLSReloadAtomicReplace(t *testing.T) {
defer os.RemoveAll(certsDirExp)
cloneFunc := func() transport.TLSInfo {
tlsInfo, err := copyTLSFiles(testTLSInfo, certsDir)
if err != nil {
t.Fatal(err)
tlsInfo, terr := copyTLSFiles(testTLSInfo, certsDir)
if terr != nil {
t.Fatal(terr)
}
if _, err = copyTLSFiles(testTLSInfoExpired, certsDirExp); err != nil {
t.Fatal(err)
@ -1448,9 +1448,9 @@ func TestTLSReloadCopy(t *testing.T) {
defer os.RemoveAll(certsDir)
cloneFunc := func() transport.TLSInfo {
tlsInfo, err := copyTLSFiles(testTLSInfo, certsDir)
if err != nil {
t.Fatal(err)
tlsInfo, terr := copyTLSFiles(testTLSInfo, certsDir)
if terr != nil {
t.Fatal(terr)
}
return tlsInfo
}

View File

@ -140,9 +140,7 @@ func TestSendMessageWhenStreamIsBroken(t *testing.T) {
}
func newServerStats() *stats.ServerStats {
ss := &stats.ServerStats{}
ss.Initialize()
return ss
return stats.NewServerStats("", "")
}
func waitStreamWorking(p *peer) bool {

View File

@ -30,12 +30,10 @@ import (
// TestTransportSend tests that transport can send messages using correct
// underlying peer, and drop local or unknown-target messages.
func TestTransportSend(t *testing.T) {
ss := &stats.ServerStats{}
ss.Initialize()
peer1 := newFakePeer()
peer2 := newFakePeer()
tr := &Transport{
ServerStats: ss,
ServerStats: stats.NewServerStats("", ""),
peers: map[types.ID]Peer{types.ID(1): peer1, types.ID(2): peer2},
}
wmsgsIgnored := []raftpb.Message{
@ -67,12 +65,10 @@ func TestTransportSend(t *testing.T) {
}
func TestTransportCutMend(t *testing.T) {
ss := &stats.ServerStats{}
ss.Initialize()
peer1 := newFakePeer()
peer2 := newFakePeer()
tr := &Transport{
ServerStats: ss,
ServerStats: stats.NewServerStats("", ""),
peers: map[types.ID]Peer{types.ID(1): peer1, types.ID(2): peer2},
}

View File

@ -44,9 +44,8 @@ func mustFindLeaderEndpoints(c *clientv3.Client) {
leaderId := uint64(0)
for _, ep := range c.Endpoints() {
resp, serr := c.Status(context.TODO(), ep)
if serr == nil {
leaderId = resp.Leader
if sresp, serr := c.Status(context.TODO(), ep); serr == nil {
leaderId = sresp.Leader
break
}
}

View File

@ -49,7 +49,6 @@ func runElectionFunc(cmd *cobra.Command, args []string) {
// nextc closes when election is ready for next round.
nextc := make(chan struct{})
eps := endpointsFromFlag(cmd)
dialTimeout := dialTimeoutFromCmd(cmd)
for i := range rcs {
v := fmt.Sprintf("%d", i)

View File

@ -112,11 +112,3 @@ func endpointsFromFlag(cmd *cobra.Command) []string {
}
return endpoints
}
func dialTimeoutFromCmd(cmd *cobra.Command) time.Duration {
dialTimeout, err := cmd.Flags().GetDuration("dial-timeout")
if err != nil {
ExitWithError(ExitError, err)
}
return dialTimeout
}

View File

@ -49,7 +49,6 @@ func runLeaseRenewerFunc(cmd *cobra.Command, args []string) {
}
eps := endpointsFromFlag(cmd)
dialTimeout := dialTimeoutFromCmd(cmd)
c := newClient(eps, dialTimeout)
ctx := context.Background()

View File

@ -53,7 +53,6 @@ func runRacerFunc(cmd *cobra.Command, args []string) {
cnt := 0
eps := endpointsFromFlag(cmd)
dialTimeout := dialTimeoutFromCmd(cmd)
for i := range rcs {
var (

View File

@ -73,7 +73,6 @@ func performWatchOnPrefixes(ctx context.Context, cmd *cobra.Command, round int)
roundPrefix := fmt.Sprintf("%16x", round)
eps := endpointsFromFlag(cmd)
dialTimeout := dialTimeoutFromCmd(cmd)
var (
revision int64