mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00
etcdserver: add id generator
This commit is contained in:
parent
e1ee335c3a
commit
05c921229e
@ -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 {
|
||||
|
@ -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"),
|
||||
|
@ -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
etcdserver/idutil/id.go
Normal file
75
etcdserver/idutil/id.go
Normal file
@ -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
etcdserver/idutil/id_test.go
Normal file
57
etcdserver/idutil/id_test.go
Normal file
@ -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))
|
||||
}
|
||||
}
|
||||
}
|
@ -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"
|
||||
@ -160,6 +161,8 @@ type EtcdServer struct {
|
||||
raftTerm uint64
|
||||
|
||||
raftLead uint64
|
||||
|
||||
reqIDGen *idutil.Generator
|
||||
}
|
||||
|
||||
// NewServer creates a new EtcdServer from the supplied configuration. The
|
||||
@ -270,6 +273,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,
|
||||
@ -474,9 +478,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"
|
||||
}
|
||||
@ -543,7 +545,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,
|
||||
@ -553,7 +554,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,
|
||||
}
|
||||
@ -566,7 +566,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,
|
||||
@ -588,6 +587,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)
|
||||
@ -617,7 +617,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)
|
||||
@ -641,7 +641,6 @@ func (s *EtcdServer) publish(retryInterval time.Duration) {
|
||||
return
|
||||
}
|
||||
req := pb.Request{
|
||||
ID: GenID(),
|
||||
Method: "PUT",
|
||||
Path: MemberAttributesStorePath(s.id),
|
||||
Val: string(b),
|
||||
@ -987,15 +986,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:
|
||||
|
@ -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"
|
||||
@ -113,7 +114,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 {
|
||||
@ -153,7 +157,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 {
|
||||
@ -579,6 +586,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
|
||||
}
|
||||
@ -591,7 +599,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",
|
||||
}
|
||||
@ -654,6 +661,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)
|
||||
@ -686,12 +694,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()
|
||||
@ -704,7 +713,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)
|
||||
}
|
||||
@ -713,8 +722,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 {
|
||||
@ -743,6 +753,7 @@ func TestDoProposalStopped(t *testing.T) {
|
||||
storage: &storageRecorder{},
|
||||
Ticker: tk,
|
||||
Cluster: cl,
|
||||
reqIDGen: idutil.NewGenerator(0, time.Time{}),
|
||||
}
|
||||
srv.start()
|
||||
|
||||
@ -768,7 +779,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() {
|
||||
@ -802,7 +814,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() {
|
||||
@ -854,6 +867,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
|
||||
@ -943,6 +957,7 @@ func TestTriggerSnap(t *testing.T) {
|
||||
raftStorage: s,
|
||||
snapCount: 10,
|
||||
Cluster: cl,
|
||||
reqIDGen: idutil.NewGenerator(0, time.Time{}),
|
||||
}
|
||||
|
||||
srv.start()
|
||||
@ -1089,6 +1104,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"}}}
|
||||
@ -1124,6 +1140,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)
|
||||
@ -1158,6 +1175,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"}}}
|
||||
@ -1193,6 +1211,7 @@ func TestPublish(t *testing.T) {
|
||||
Cluster: &Cluster{},
|
||||
node: n,
|
||||
w: w,
|
||||
reqIDGen: idutil.NewGenerator(0, time.Time{}),
|
||||
}
|
||||
srv.publish(time.Hour)
|
||||
|
||||
@ -1229,6 +1248,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)
|
||||
@ -1238,9 +1258,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)
|
||||
@ -1338,19 +1359,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{}
|
||||
@ -1475,11 +1483,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 }
|
||||
|
2
test
2
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
|
||||
|
Loading…
x
Reference in New Issue
Block a user