Explorar o código

*: fix go tool vet -all -shadow errors

Anthony Romano %!s(int64=8) %!d(string=hai) anos
pai
achega
887db5a3db

+ 1 - 1
clientv3/example_metrics_test.go

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

+ 1 - 4
contrib/raftexample/raft.go

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

+ 1 - 1
e2e/ctl_v3_elect_test.go

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

+ 2 - 2
etcdctl/ctlv3/command/check.go

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

+ 1 - 5
etcdserver/server.go

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

+ 10 - 5
etcdserver/stats/leader.go

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

+ 23 - 31
etcdserver/stats/server.go

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

+ 6 - 6
integration/v3_grpc_test.go

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

+ 1 - 3
rafthttp/functional_test.go

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

+ 2 - 6
rafthttp/transport_test.go

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

+ 2 - 3
tools/benchmark/cmd/util.go

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

+ 0 - 1
tools/functional-tester/etcd-runner/command/election_command.go

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

+ 0 - 8
tools/functional-tester/etcd-runner/command/global.go

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

+ 0 - 1
tools/functional-tester/etcd-runner/command/lease_renewer_command.go

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

+ 0 - 1
tools/functional-tester/etcd-runner/command/lock_racer_command.go

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

+ 0 - 1
tools/functional-tester/etcd-runner/command/watch_command.go

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