Forráskód Böngészése

Merge pull request #1952 from yichengq/262

etcdserver: add id generator
Yicheng Qin 11 éve
szülő
commit
6ccaadc95d

+ 4 - 5
etcdctl/command/backup_command.go

@@ -18,14 +18,13 @@ package command
 
 import (
 	"log"
-	"math/rand"
 	"os"
 	"path"
 	"time"
 
 	"github.com/coreos/etcd/Godeps/_workspace/src/github.com/codegangsta/cli"
-	"github.com/coreos/etcd/etcdserver"
 	"github.com/coreos/etcd/etcdserver/etcdserverpb"
+	"github.com/coreos/etcd/etcdserver/idutil"
 	"github.com/coreos/etcd/pkg/pbutil"
 	"github.com/coreos/etcd/snap"
 	"github.com/coreos/etcd/wal"
@@ -78,9 +77,9 @@ func handleBackup(c *cli.Context) {
 	}
 	var metadata etcdserverpb.Metadata
 	pbutil.MustUnmarshal(&metadata, wmetadata)
-	rand.Seed(time.Now().UnixNano())
-	metadata.NodeID = etcdserver.GenID()
-	metadata.ClusterID = etcdserver.GenID()
+	idgen := idutil.NewGenerator(0, time.Now())
+	metadata.NodeID = idgen.Next()
+	metadata.ClusterID = idgen.Next()
 
 	neww, err := wal.Create(destWAL, pbutil.MustMarshal(&metadata))
 	if err != nil {

+ 2 - 3
etcdserver/etcdhttp/client.go

@@ -102,7 +102,7 @@ func (h *keysHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
 	ctx, cancel := context.WithTimeout(context.Background(), h.timeout)
 	defer cancel()
 
-	rr, err := parseKeyRequest(r, etcdserver.GenID(), clockwork.NewRealClock())
+	rr, err := parseKeyRequest(r, clockwork.NewRealClock())
 	if err != nil {
 		writeError(w, err)
 		return
@@ -279,7 +279,7 @@ func serveVersion(w http.ResponseWriter, r *http.Request) {
 // parseKeyRequest converts a received http.Request on keysPrefix to
 // a server Request, performing validation of supplied fields as appropriate.
 // If any validation fails, an empty Request and non-nil error is returned.
-func parseKeyRequest(r *http.Request, id uint64, clock clockwork.Clock) (etcdserverpb.Request, error) {
+func parseKeyRequest(r *http.Request, clock clockwork.Clock) (etcdserverpb.Request, error) {
 	emptyReq := etcdserverpb.Request{}
 
 	err := r.ParseForm()
@@ -394,7 +394,6 @@ func parseKeyRequest(r *http.Request, id uint64, clock clockwork.Clock) (etcdser
 	}
 
 	rr := etcdserverpb.Request{
-		ID:        id,
 		Method:    r.Method,
 		Path:      p,
 		Val:       r.FormValue("value"),

+ 2 - 19
etcdserver/etcdhttp/client_test.go

@@ -312,7 +312,7 @@ func TestBadParseRequest(t *testing.T) {
 		},
 	}
 	for i, tt := range tests {
-		got, err := parseKeyRequest(tt.in, 1234, clockwork.NewFakeClock())
+		got, err := parseKeyRequest(tt.in, clockwork.NewFakeClock())
 		if err == nil {
 			t.Errorf("#%d: unexpected nil error!", i)
 			continue
@@ -343,7 +343,6 @@ func TestGoodParseRequest(t *testing.T) {
 			// good prefix, all other values default
 			mustNewRequest(t, "foo"),
 			etcdserverpb.Request{
-				ID:     1234,
 				Method: "GET",
 				Path:   path.Join(etcdserver.StoreKeysPrefix, "/foo"),
 			},
@@ -356,7 +355,6 @@ func TestGoodParseRequest(t *testing.T) {
 				url.Values{"value": []string{"some_value"}},
 			),
 			etcdserverpb.Request{
-				ID:     1234,
 				Method: "PUT",
 				Val:    "some_value",
 				Path:   path.Join(etcdserver.StoreKeysPrefix, "/foo"),
@@ -370,7 +368,6 @@ func TestGoodParseRequest(t *testing.T) {
 				url.Values{"prevIndex": []string{"98765"}},
 			),
 			etcdserverpb.Request{
-				ID:        1234,
 				Method:    "PUT",
 				PrevIndex: 98765,
 				Path:      path.Join(etcdserver.StoreKeysPrefix, "/foo"),
@@ -384,7 +381,6 @@ func TestGoodParseRequest(t *testing.T) {
 				url.Values{"recursive": []string{"true"}},
 			),
 			etcdserverpb.Request{
-				ID:        1234,
 				Method:    "PUT",
 				Recursive: true,
 				Path:      path.Join(etcdserver.StoreKeysPrefix, "/foo"),
@@ -398,7 +394,6 @@ func TestGoodParseRequest(t *testing.T) {
 				url.Values{"sorted": []string{"true"}},
 			),
 			etcdserverpb.Request{
-				ID:     1234,
 				Method: "PUT",
 				Sorted: true,
 				Path:   path.Join(etcdserver.StoreKeysPrefix, "/foo"),
@@ -412,7 +407,6 @@ func TestGoodParseRequest(t *testing.T) {
 				url.Values{"quorum": []string{"true"}},
 			),
 			etcdserverpb.Request{
-				ID:     1234,
 				Method: "PUT",
 				Quorum: true,
 				Path:   path.Join(etcdserver.StoreKeysPrefix, "/foo"),
@@ -422,7 +416,6 @@ func TestGoodParseRequest(t *testing.T) {
 			// wait specified
 			mustNewRequest(t, "foo?wait=true"),
 			etcdserverpb.Request{
-				ID:     1234,
 				Method: "GET",
 				Wait:   true,
 				Path:   path.Join(etcdserver.StoreKeysPrefix, "/foo"),
@@ -432,7 +425,6 @@ func TestGoodParseRequest(t *testing.T) {
 			// empty TTL specified
 			mustNewRequest(t, "foo?ttl="),
 			etcdserverpb.Request{
-				ID:         1234,
 				Method:     "GET",
 				Path:       path.Join(etcdserver.StoreKeysPrefix, "/foo"),
 				Expiration: 0,
@@ -442,7 +434,6 @@ func TestGoodParseRequest(t *testing.T) {
 			// non-empty TTL specified
 			mustNewRequest(t, "foo?ttl=5678"),
 			etcdserverpb.Request{
-				ID:         1234,
 				Method:     "GET",
 				Path:       path.Join(etcdserver.StoreKeysPrefix, "/foo"),
 				Expiration: fc.Now().Add(5678 * time.Second).UnixNano(),
@@ -452,7 +443,6 @@ func TestGoodParseRequest(t *testing.T) {
 			// zero TTL specified
 			mustNewRequest(t, "foo?ttl=0"),
 			etcdserverpb.Request{
-				ID:         1234,
 				Method:     "GET",
 				Path:       path.Join(etcdserver.StoreKeysPrefix, "/foo"),
 				Expiration: fc.Now().UnixNano(),
@@ -462,7 +452,6 @@ func TestGoodParseRequest(t *testing.T) {
 			// dir specified
 			mustNewRequest(t, "foo?dir=true"),
 			etcdserverpb.Request{
-				ID:     1234,
 				Method: "GET",
 				Dir:    true,
 				Path:   path.Join(etcdserver.StoreKeysPrefix, "/foo"),
@@ -472,7 +461,6 @@ func TestGoodParseRequest(t *testing.T) {
 			// dir specified negatively
 			mustNewRequest(t, "foo?dir=false"),
 			etcdserverpb.Request{
-				ID:     1234,
 				Method: "GET",
 				Dir:    false,
 				Path:   path.Join(etcdserver.StoreKeysPrefix, "/foo"),
@@ -486,7 +474,6 @@ func TestGoodParseRequest(t *testing.T) {
 				url.Values{"prevExist": []string{"true"}},
 			),
 			etcdserverpb.Request{
-				ID:        1234,
 				Method:    "PUT",
 				PrevExist: boolp(true),
 				Path:      path.Join(etcdserver.StoreKeysPrefix, "/foo"),
@@ -500,7 +487,6 @@ func TestGoodParseRequest(t *testing.T) {
 				url.Values{"prevExist": []string{"false"}},
 			),
 			etcdserverpb.Request{
-				ID:        1234,
 				Method:    "PUT",
 				PrevExist: boolp(false),
 				Path:      path.Join(etcdserver.StoreKeysPrefix, "/foo"),
@@ -518,7 +504,6 @@ func TestGoodParseRequest(t *testing.T) {
 				},
 			),
 			etcdserverpb.Request{
-				ID:        1234,
 				Method:    "PUT",
 				PrevExist: boolp(true),
 				PrevValue: "previous value",
@@ -534,7 +519,6 @@ func TestGoodParseRequest(t *testing.T) {
 				url.Values{},
 			),
 			etcdserverpb.Request{
-				ID:        1234,
 				Method:    "PUT",
 				PrevValue: "woof",
 				Path:      path.Join(etcdserver.StoreKeysPrefix, "/foo"),
@@ -550,7 +534,6 @@ func TestGoodParseRequest(t *testing.T) {
 				},
 			),
 			etcdserverpb.Request{
-				ID:        1234,
 				Method:    "PUT",
 				PrevValue: "miaow",
 				Path:      path.Join(etcdserver.StoreKeysPrefix, "/foo"),
@@ -559,7 +542,7 @@ func TestGoodParseRequest(t *testing.T) {
 	}
 
 	for i, tt := range tests {
-		got, err := parseKeyRequest(tt.in, 1234, fc)
+		got, err := parseKeyRequest(tt.in, fc)
 		if err != nil {
 			t.Errorf("#%d: err = %v, want %v", i, err, nil)
 		}

+ 75 - 0
etcdserver/idutil/id.go

@@ -0,0 +1,75 @@
+/*
+   Copyright 2014 CoreOS, Inc.
+
+   Licensed under the Apache License, Version 2.0 (the "License");
+   you may not use this file except in compliance with the License.
+   You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+*/
+
+package idutil
+
+import (
+	"math"
+	"sync"
+	"time"
+)
+
+const (
+	tsLen     = 5 * 8
+	cntLen    = 2 * 8
+	suffixLen = tsLen + cntLen
+)
+
+// The initial id is in this format:
+// High order byte is memberID, next 5 bytes are from timestamp,
+// and low order 2 bytes are 0s.
+// | prefix   | suffix              |
+// | 1 byte   | 5 bytes   | 2 bytes |
+// | memberID | timestamp | cnt     |
+//
+// The timestamp 5 bytes is different when the machine is restart
+// after 1 ms and before 35 years.
+//
+// It increases suffix to generate the next id.
+// The count field may overflow to timestamp field, which is intentional.
+// It helps to extend the event window to 2^56. This doesn't break that
+// id generated after restart is unique because etcd throughput is <<
+// 65536req/ms.
+type Generator struct {
+	mu sync.Mutex
+	// high order byte
+	prefix uint64
+	// low order 7 bytes
+	suffix uint64
+}
+
+func NewGenerator(memberID uint8, now time.Time) *Generator {
+	prefix := uint64(memberID) << suffixLen
+	unixMilli := uint64(now.UnixNano()) / uint64(time.Millisecond/time.Nanosecond)
+	suffix := lowbit(unixMilli, tsLen) << cntLen
+	return &Generator{
+		prefix: prefix,
+		suffix: suffix,
+	}
+}
+
+// Next generates a id that is unique.
+func (g *Generator) Next() uint64 {
+	g.mu.Lock()
+	defer g.mu.Unlock()
+	g.suffix++
+	id := g.prefix | lowbit(g.suffix, suffixLen)
+	return id
+}
+
+func lowbit(x uint64, n uint) uint64 {
+	return x & (math.MaxUint64 >> (64 - n))
+}

+ 57 - 0
etcdserver/idutil/id_test.go

@@ -0,0 +1,57 @@
+/*
+   Copyright 2014 CoreOS, Inc.
+
+   Licensed under the Apache License, Version 2.0 (the "License");
+   you may not use this file except in compliance with the License.
+   You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+*/
+
+package idutil
+
+import (
+	"testing"
+	"time"
+)
+
+func TestNewGenerator(t *testing.T) {
+	g := NewGenerator(0x12, time.Unix(0, 0).Add(0x3456*time.Millisecond))
+	id := g.Next()
+	wid := uint64(0x1200000034560001)
+	if id != wid {
+		t.Errorf("id = %x, want %x", id, wid)
+	}
+}
+
+func TestNewGeneratorUnique(t *testing.T) {
+	g := NewGenerator(0, time.Time{})
+	id := g.Next()
+	// different server generates different ID
+	g1 := NewGenerator(1, time.Time{})
+	if gid := g1.Next(); id == gid {
+		t.Errorf("generate the same id %x using different server ID", id)
+	}
+	// restarted server generates different ID
+	g2 := NewGenerator(0, time.Now())
+	if gid := g2.Next(); id == gid {
+		t.Errorf("generate the same id %x after restart", id)
+	}
+}
+
+func TestNext(t *testing.T) {
+	g := NewGenerator(0x12, time.Unix(0, 0).Add(0x3456*time.Millisecond))
+	wid := uint64(0x1200000034560001)
+	for i := 0; i < 1000; i++ {
+		id := g.Next()
+		if id != wid+uint64(i) {
+			t.Errorf("id = %x, want %x", id, wid+uint64(i))
+		}
+	}
+}

+ 7 - 17
etcdserver/server.go

@@ -35,6 +35,7 @@ import (
 	"github.com/coreos/etcd/discovery"
 	"github.com/coreos/etcd/etcdserver/etcdhttp/httptypes"
 	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
+	"github.com/coreos/etcd/etcdserver/idutil"
 	"github.com/coreos/etcd/etcdserver/stats"
 	"github.com/coreos/etcd/pkg/fileutil"
 	"github.com/coreos/etcd/pkg/pbutil"
@@ -161,6 +162,8 @@ type EtcdServer struct {
 	raftTerm  uint64
 
 	raftLead uint64
+
+	reqIDGen *idutil.Generator
 }
 
 // NewServer creates a new EtcdServer from the supplied configuration. The
@@ -271,6 +274,7 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) {
 		Ticker:      time.Tick(100 * time.Millisecond),
 		SyncTicker:  time.Tick(500 * time.Millisecond),
 		snapCount:   cfg.SnapCount,
+		reqIDGen:    idutil.NewGenerator(uint8(id), time.Now()),
 	}
 	tr := &rafthttp.Transport{
 		RoundTripper: cfg.Transport,
@@ -475,9 +479,7 @@ func (s *EtcdServer) StopNotify() <-chan struct{} { return s.done }
 // respective operation. Do will block until an action is performed or there is
 // an error.
 func (s *EtcdServer) Do(ctx context.Context, r pb.Request) (Response, error) {
-	if r.ID == 0 {
-		log.Panicf("request ID should never be 0")
-	}
+	r.ID = s.reqIDGen.Next()
 	if r.Method == "GET" && r.Quorum {
 		r.Method = "QGET"
 	}
@@ -544,7 +546,6 @@ func (s *EtcdServer) AddMember(ctx context.Context, memb Member) error {
 		return err
 	}
 	cc := raftpb.ConfChange{
-		ID:      GenID(),
 		Type:    raftpb.ConfChangeAddNode,
 		NodeID:  uint64(memb.ID),
 		Context: b,
@@ -554,7 +555,6 @@ func (s *EtcdServer) AddMember(ctx context.Context, memb Member) error {
 
 func (s *EtcdServer) RemoveMember(ctx context.Context, id uint64) error {
 	cc := raftpb.ConfChange{
-		ID:     GenID(),
 		Type:   raftpb.ConfChangeRemoveNode,
 		NodeID: id,
 	}
@@ -567,7 +567,6 @@ func (s *EtcdServer) UpdateMember(ctx context.Context, memb Member) error {
 		return err
 	}
 	cc := raftpb.ConfChange{
-		ID:      GenID(),
 		Type:    raftpb.ConfChangeUpdateNode,
 		NodeID:  uint64(memb.ID),
 		Context: b,
@@ -589,6 +588,7 @@ func (s *EtcdServer) Lead() uint64 { return atomic.LoadUint64(&s.raftLead) }
 // then waits for it to be applied to the server. It
 // will block until the change is performed or there is an error.
 func (s *EtcdServer) configure(ctx context.Context, cc raftpb.ConfChange) error {
+	cc.ID = s.reqIDGen.Next()
 	ch := s.w.Register(cc.ID)
 	if err := s.node.ProposeConfChange(ctx, cc); err != nil {
 		s.w.Trigger(cc.ID, nil)
@@ -618,7 +618,7 @@ func (s *EtcdServer) sync(timeout time.Duration) {
 	ctx, cancel := context.WithTimeout(context.Background(), timeout)
 	req := pb.Request{
 		Method: "SYNC",
-		ID:     GenID(),
+		ID:     s.reqIDGen.Next(),
 		Time:   time.Now().UnixNano(),
 	}
 	data := pbutil.MustMarshal(&req)
@@ -642,7 +642,6 @@ func (s *EtcdServer) publish(retryInterval time.Duration) {
 		return
 	}
 	req := pb.Request{
-		ID:     GenID(),
 		Method: "PUT",
 		Path:   MemberAttributesStorePath(s.id),
 		Val:    string(b),
@@ -980,15 +979,6 @@ func getOtherPeerURLs(cl ClusterInfo, self string) []string {
 	return us
 }
 
-// TODO: move the function to /id pkg maybe?
-// GenID generates a random id that is not equal to 0.
-func GenID() (n uint64) {
-	for n == 0 {
-		n = uint64(rand.Int63())
-	}
-	return
-}
-
 func parseCtxErr(err error) error {
 	switch err {
 	case context.Canceled:

+ 35 - 27
etcdserver/server_test.go

@@ -32,6 +32,7 @@ import (
 
 	"github.com/coreos/etcd/Godeps/_workspace/src/golang.org/x/net/context"
 	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
+	"github.com/coreos/etcd/etcdserver/idutil"
 	"github.com/coreos/etcd/pkg/pbutil"
 	"github.com/coreos/etcd/pkg/testutil"
 	"github.com/coreos/etcd/pkg/types"
@@ -85,7 +86,10 @@ func TestDoLocalAction(t *testing.T) {
 	}
 	for i, tt := range tests {
 		st := &storeRecorder{}
-		srv := &EtcdServer{store: st}
+		srv := &EtcdServer{
+			store:    st,
+			reqIDGen: idutil.NewGenerator(0, time.Time{}),
+		}
 		resp, err := srv.Do(context.TODO(), tt.req)
 
 		if err != tt.werr {
@@ -125,7 +129,10 @@ func TestDoBadLocalAction(t *testing.T) {
 	}
 	for i, tt := range tests {
 		st := &errStoreRecorder{err: storeErr}
-		srv := &EtcdServer{store: st}
+		srv := &EtcdServer{
+			store:    st,
+			reqIDGen: idutil.NewGenerator(0, time.Time{}),
+		}
 		resp, err := srv.Do(context.Background(), tt.req)
 
 		if err != storeErr {
@@ -550,6 +557,7 @@ func testServer(t *testing.T, ns uint64) {
 			storage:     &storageRecorder{},
 			Ticker:      tk.C,
 			Cluster:     cl,
+			reqIDGen:    idutil.NewGenerator(uint8(i), time.Time{}),
 		}
 		ss[i] = srv
 	}
@@ -562,7 +570,6 @@ func testServer(t *testing.T, ns uint64) {
 	for i := 1; i <= 10; i++ {
 		r := pb.Request{
 			Method: "PUT",
-			ID:     uint64(i),
 			Path:   "/foo",
 			Val:    "bar",
 		}
@@ -625,6 +632,7 @@ func TestDoProposal(t *testing.T) {
 			storage:     &storageRecorder{},
 			Ticker:      tk,
 			Cluster:     cl,
+			reqIDGen:    idutil.NewGenerator(0, time.Time{}),
 		}
 		srv.start()
 		resp, err := srv.Do(ctx, tt)
@@ -657,12 +665,13 @@ func TestDoProposalCancelled(t *testing.T) {
 		raftStorage: s,
 		store:       st,
 		w:           wait,
+		reqIDGen:    idutil.NewGenerator(0, time.Time{}),
 	}
 
 	done := make(chan struct{})
 	var err error
 	go func() {
-		_, err = srv.Do(ctx, pb.Request{Method: "PUT", ID: 1})
+		_, err = srv.Do(ctx, pb.Request{Method: "PUT"})
 		close(done)
 	}()
 	cancel()
@@ -675,7 +684,7 @@ func TestDoProposalCancelled(t *testing.T) {
 	if err != ErrCanceled {
 		t.Fatalf("err = %v, want %v", err, ErrCanceled)
 	}
-	w := []action{action{name: "Register1"}, action{name: "Trigger1"}}
+	w := []action{action{name: "Register"}, action{name: "Trigger"}}
 	if !reflect.DeepEqual(wait.action, w) {
 		t.Errorf("wait.action = %+v, want %+v", wait.action, w)
 	}
@@ -684,8 +693,9 @@ func TestDoProposalCancelled(t *testing.T) {
 func TestDoProposalTimeout(t *testing.T) {
 	ctx, _ := context.WithTimeout(context.Background(), 0)
 	srv := &EtcdServer{
-		node: &nodeRecorder{},
-		w:    &waitRecorder{},
+		node:     &nodeRecorder{},
+		w:        &waitRecorder{},
+		reqIDGen: idutil.NewGenerator(0, time.Time{}),
 	}
 	_, err := srv.Do(ctx, pb.Request{Method: "PUT", ID: 1})
 	if err != ErrTimeout {
@@ -714,6 +724,7 @@ func TestDoProposalStopped(t *testing.T) {
 		storage:     &storageRecorder{},
 		Ticker:      tk,
 		Cluster:     cl,
+		reqIDGen:    idutil.NewGenerator(0, time.Time{}),
 	}
 	srv.start()
 
@@ -739,7 +750,8 @@ func TestDoProposalStopped(t *testing.T) {
 func TestSync(t *testing.T) {
 	n := &nodeProposeDataRecorder{}
 	srv := &EtcdServer{
-		node: n,
+		node:     n,
+		reqIDGen: idutil.NewGenerator(0, time.Time{}),
 	}
 	done := make(chan struct{})
 	go func() {
@@ -773,7 +785,8 @@ func TestSync(t *testing.T) {
 func TestSyncTimeout(t *testing.T) {
 	n := &nodeProposalBlockerRecorder{}
 	srv := &EtcdServer{
-		node: n,
+		node:     n,
+		reqIDGen: idutil.NewGenerator(0, time.Time{}),
 	}
 	done := make(chan struct{})
 	go func() {
@@ -825,6 +838,7 @@ func TestSyncTrigger(t *testing.T) {
 		transport:   &nopTransporter{},
 		storage:     &storageRecorder{},
 		SyncTicker:  st,
+		reqIDGen:    idutil.NewGenerator(0, time.Time{}),
 	}
 	srv.start()
 	// trigger the server to become a leader and accept sync requests
@@ -914,6 +928,7 @@ func TestTriggerSnap(t *testing.T) {
 		raftStorage: s,
 		snapCount:   10,
 		Cluster:     cl,
+		reqIDGen:    idutil.NewGenerator(0, time.Time{}),
 	}
 
 	srv.start()
@@ -1060,6 +1075,7 @@ func TestAddMember(t *testing.T) {
 		transport:   &nopTransporter{},
 		storage:     &storageRecorder{},
 		Cluster:     cl,
+		reqIDGen:    idutil.NewGenerator(0, time.Time{}),
 	}
 	s.start()
 	m := Member{ID: 1234, RaftAttributes: RaftAttributes{PeerURLs: []string{"foo"}}}
@@ -1095,6 +1111,7 @@ func TestRemoveMember(t *testing.T) {
 		transport:   &nopTransporter{},
 		storage:     &storageRecorder{},
 		Cluster:     cl,
+		reqIDGen:    idutil.NewGenerator(0, time.Time{}),
 	}
 	s.start()
 	err := s.RemoveMember(context.TODO(), 1234)
@@ -1129,6 +1146,7 @@ func TestUpdateMember(t *testing.T) {
 		transport:   &nopTransporter{},
 		storage:     &storageRecorder{},
 		Cluster:     cl,
+		reqIDGen:    idutil.NewGenerator(0, time.Time{}),
 	}
 	s.start()
 	wm := Member{ID: 1234, RaftAttributes: RaftAttributes{PeerURLs: []string{"http://127.0.0.1:1"}}}
@@ -1164,6 +1182,7 @@ func TestPublish(t *testing.T) {
 		Cluster:    &Cluster{},
 		node:       n,
 		w:          w,
+		reqIDGen:   idutil.NewGenerator(0, time.Time{}),
 	}
 	srv.publish(time.Hour)
 
@@ -1200,6 +1219,7 @@ func TestPublishStopped(t *testing.T) {
 		w:         &waitRecorder{},
 		done:      make(chan struct{}),
 		stop:      make(chan struct{}),
+		reqIDGen:  idutil.NewGenerator(0, time.Time{}),
 	}
 	close(srv.done)
 	srv.publish(time.Hour)
@@ -1209,9 +1229,10 @@ func TestPublishStopped(t *testing.T) {
 func TestPublishRetry(t *testing.T) {
 	n := &nodeRecorder{}
 	srv := &EtcdServer{
-		node: n,
-		w:    &waitRecorder{},
-		done: make(chan struct{}),
+		node:     n,
+		w:        &waitRecorder{},
+		done:     make(chan struct{}),
+		reqIDGen: idutil.NewGenerator(0, time.Time{}),
 	}
 	time.AfterFunc(500*time.Microsecond, func() { close(srv.done) })
 	srv.publish(10 * time.Nanosecond)
@@ -1309,19 +1330,6 @@ func TestGetBool(t *testing.T) {
 	}
 }
 
-func TestGenID(t *testing.T) {
-	// Sanity check that the GenID function has been seeded appropriately
-	// (math/rand is seeded with 1 by default)
-	r := rand.NewSource(int64(1))
-	var n uint64
-	for n == 0 {
-		n = uint64(r.Int63())
-	}
-	if n == GenID() {
-		t.Fatalf("GenID's rand seeded with 1!")
-	}
-}
-
 type action struct {
 	name   string
 	params []interface{}
@@ -1446,11 +1454,11 @@ type waitRecorder struct {
 }
 
 func (w *waitRecorder) Register(id uint64) <-chan interface{} {
-	w.action = append(w.action, action{name: fmt.Sprint("Register", id)})
+	w.action = append(w.action, action{name: "Register"})
 	return nil
 }
 func (w *waitRecorder) Trigger(id uint64, x interface{}) {
-	w.action = append(w.action, action{name: fmt.Sprint("Trigger", id)})
+	w.action = append(w.action, action{name: "Trigger"})
 }
 
 func boolp(b bool) *bool { return &b }

+ 1 - 1
test

@@ -15,7 +15,7 @@ COVER=${COVER:-"-cover"}
 source ./build
 
 # Hack: gofmt ./ will recursively check the .git directory. So use *.go for gofmt.
-TESTABLE_AND_FORMATTABLE="client discovery error etcdctl/command etcdmain etcdserver etcdserver/etcdhttp etcdserver/etcdhttp/httptypes etcdserver/etcdserverpb integration migrate pkg/fileutil pkg/flags pkg/ioutils pkg/netutil pkg/types pkg/transport pkg/wait proxy raft rafthttp snap store wal"
+TESTABLE_AND_FORMATTABLE="client discovery error etcdctl/command etcdmain etcdserver etcdserver/etcdhttp etcdserver/etcdhttp/httptypes etcdserver/etcdserverpb etcdserver/idutil integration migrate pkg/fileutil pkg/flags pkg/ioutils pkg/netutil pkg/types pkg/transport pkg/wait proxy raft rafthttp snap store wal"
 FORMATTABLE="$TESTABLE_AND_FORMATTABLE *.go etcdctl/"
 
 # user has not provided PKG override