mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00
move store recorder to 'mock/mockstore'
This commit is contained in:
parent
bae053f57f
commit
dcaf5ef586
@ -21,6 +21,7 @@ import (
|
||||
"reflect"
|
||||
"testing"
|
||||
|
||||
"github.com/coreos/etcd/pkg/mock/mockstore"
|
||||
"github.com/coreos/etcd/pkg/testutil"
|
||||
"github.com/coreos/etcd/pkg/types"
|
||||
"github.com/coreos/etcd/raft/raftpb"
|
||||
@ -406,7 +407,7 @@ func TestClusterGenID(t *testing.T) {
|
||||
}
|
||||
previd := cs.ID()
|
||||
|
||||
cs.SetStore(store.NewNop())
|
||||
cs.SetStore(mockstore.NewNop())
|
||||
cs.AddMember(newTestMember(3, nil, "", nil))
|
||||
cs.genID()
|
||||
if cs.ID() == previd {
|
||||
@ -447,7 +448,7 @@ func TestNodeToMemberBad(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestClusterAddMember(t *testing.T) {
|
||||
st := store.NewRecorder()
|
||||
st := mockstore.NewRecorder()
|
||||
c := newTestCluster(nil)
|
||||
c.SetStore(st)
|
||||
c.AddMember(newTestMember(1, nil, "node1", nil))
|
||||
@ -492,7 +493,7 @@ func TestClusterMembers(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestClusterRemoveMember(t *testing.T) {
|
||||
st := store.NewRecorder()
|
||||
st := mockstore.NewRecorder()
|
||||
c := newTestCluster(nil)
|
||||
c.SetStore(st)
|
||||
c.RemoveMember(1)
|
||||
|
@ -20,6 +20,7 @@ import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/coreos/etcd/pkg/mock/mockstorage"
|
||||
"github.com/coreos/etcd/pkg/pbutil"
|
||||
"github.com/coreos/etcd/pkg/types"
|
||||
"github.com/coreos/etcd/raft"
|
||||
@ -153,7 +154,7 @@ func TestStopRaftWhenWaitingForApplyDone(t *testing.T) {
|
||||
n := newNopReadyNode()
|
||||
r := raftNode{
|
||||
Node: n,
|
||||
storage: newStorageRecorder(""),
|
||||
storage: mockstorage.NewStorageRecorder(""),
|
||||
raftStorage: raft.NewMemoryStorage(),
|
||||
transport: rafthttp.NewNopTransporter(),
|
||||
}
|
||||
|
@ -29,6 +29,9 @@ import (
|
||||
pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
|
||||
"github.com/coreos/etcd/lease"
|
||||
"github.com/coreos/etcd/pkg/idutil"
|
||||
"github.com/coreos/etcd/pkg/mock/mockstorage"
|
||||
"github.com/coreos/etcd/pkg/mock/mockstore"
|
||||
"github.com/coreos/etcd/pkg/mock/mockwait"
|
||||
"github.com/coreos/etcd/pkg/pbutil"
|
||||
"github.com/coreos/etcd/pkg/testutil"
|
||||
"github.com/coreos/etcd/pkg/types"
|
||||
@ -81,7 +84,7 @@ func TestDoLocalAction(t *testing.T) {
|
||||
},
|
||||
}
|
||||
for i, tt := range tests {
|
||||
st := store.NewRecorder()
|
||||
st := mockstore.NewRecorder()
|
||||
srv := &EtcdServer{
|
||||
store: st,
|
||||
reqIDGen: idutil.NewGenerator(0, time.Time{}),
|
||||
@ -134,7 +137,7 @@ func TestDoBadLocalAction(t *testing.T) {
|
||||
},
|
||||
}
|
||||
for i, tt := range tests {
|
||||
st := store.NewErrRecorder(storeErr)
|
||||
st := mockstore.NewErrRecorder(storeErr)
|
||||
srv := &EtcdServer{
|
||||
store: st,
|
||||
reqIDGen: idutil.NewGenerator(0, time.Time{}),
|
||||
@ -168,7 +171,7 @@ func TestApplyRepeat(t *testing.T) {
|
||||
r: raftNode{
|
||||
Node: n,
|
||||
raftStorage: raft.NewMemoryStorage(),
|
||||
storage: newStorageRecorder(""),
|
||||
storage: mockstorage.NewStorageRecorder(""),
|
||||
transport: rafthttp.NewNopTransporter(),
|
||||
},
|
||||
cfg: &ServerConfig{},
|
||||
@ -439,7 +442,7 @@ func TestApplyRequest(t *testing.T) {
|
||||
}
|
||||
|
||||
for i, tt := range tests {
|
||||
st := store.NewRecorder()
|
||||
st := mockstore.NewRecorder()
|
||||
srv := &EtcdServer{store: st}
|
||||
resp := srv.applyRequest(tt.req)
|
||||
|
||||
@ -456,7 +459,7 @@ func TestApplyRequest(t *testing.T) {
|
||||
func TestApplyRequestOnAdminMemberAttributes(t *testing.T) {
|
||||
cl := newTestCluster([]*Member{{ID: 1}})
|
||||
srv := &EtcdServer{
|
||||
store: store.NewRecorder(),
|
||||
store: mockstore.NewRecorder(),
|
||||
cluster: cl,
|
||||
}
|
||||
req := pb.Request{
|
||||
@ -620,12 +623,12 @@ func TestDoProposal(t *testing.T) {
|
||||
{Method: "GET", ID: 1, Quorum: true},
|
||||
}
|
||||
for i, tt := range tests {
|
||||
st := store.NewRecorder()
|
||||
st := mockstore.NewRecorder()
|
||||
srv := &EtcdServer{
|
||||
cfg: &ServerConfig{TickMs: 1},
|
||||
r: raftNode{
|
||||
Node: newNodeCommitter(),
|
||||
storage: newStorageRecorder(""),
|
||||
storage: mockstorage.NewStorageRecorder(""),
|
||||
raftStorage: raft.NewMemoryStorage(),
|
||||
transport: rafthttp.NewNopTransporter(),
|
||||
},
|
||||
@ -651,7 +654,7 @@ func TestDoProposal(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestDoProposalCancelled(t *testing.T) {
|
||||
wait := wait.NewRecorder()
|
||||
wait := mockwait.NewRecorder()
|
||||
srv := &EtcdServer{
|
||||
cfg: &ServerConfig{TickMs: 1},
|
||||
r: raftNode{Node: newNodeNop()},
|
||||
@ -675,7 +678,7 @@ func TestDoProposalTimeout(t *testing.T) {
|
||||
srv := &EtcdServer{
|
||||
cfg: &ServerConfig{TickMs: 1},
|
||||
r: raftNode{Node: newNodeNop()},
|
||||
w: wait.NewNop(),
|
||||
w: mockwait.NewNop(),
|
||||
reqIDGen: idutil.NewGenerator(0, time.Time{}),
|
||||
}
|
||||
ctx, _ := context.WithTimeout(context.Background(), 0)
|
||||
@ -689,7 +692,7 @@ func TestDoProposalStopped(t *testing.T) {
|
||||
srv := &EtcdServer{
|
||||
cfg: &ServerConfig{TickMs: 1},
|
||||
r: raftNode{Node: newNodeNop()},
|
||||
w: wait.NewNop(),
|
||||
w: mockwait.NewNop(),
|
||||
reqIDGen: idutil.NewGenerator(0, time.Time{}),
|
||||
}
|
||||
srv.done = make(chan struct{})
|
||||
@ -776,9 +779,9 @@ func TestSyncTrigger(t *testing.T) {
|
||||
Node: n,
|
||||
raftStorage: raft.NewMemoryStorage(),
|
||||
transport: rafthttp.NewNopTransporter(),
|
||||
storage: newStorageRecorder(""),
|
||||
storage: mockstorage.NewStorageRecorder(""),
|
||||
},
|
||||
store: store.NewNop(),
|
||||
store: mockstore.NewNop(),
|
||||
SyncTicker: st,
|
||||
reqIDGen: idutil.NewGenerator(0, time.Time{}),
|
||||
}
|
||||
@ -821,8 +824,8 @@ func TestSyncTrigger(t *testing.T) {
|
||||
func TestSnapshot(t *testing.T) {
|
||||
s := raft.NewMemoryStorage()
|
||||
s.Append([]raftpb.Entry{{Index: 1}})
|
||||
st := store.NewRecorder()
|
||||
p := newStorageRecorder("")
|
||||
st := mockstore.NewRecorder()
|
||||
p := mockstorage.NewStorageRecorder("")
|
||||
srv := &EtcdServer{
|
||||
cfg: &ServerConfig{},
|
||||
r: raftNode{
|
||||
@ -855,8 +858,8 @@ func TestSnapshot(t *testing.T) {
|
||||
// Applied > SnapCount should trigger a SaveSnap event
|
||||
func TestTriggerSnap(t *testing.T) {
|
||||
snapc := 10
|
||||
st := store.NewRecorder()
|
||||
p := newStorageRecorderStream("")
|
||||
st := mockstore.NewRecorder()
|
||||
p := mockstorage.NewStorageRecorderStream("")
|
||||
srv := &EtcdServer{
|
||||
cfg: &ServerConfig{TickMs: 1},
|
||||
snapCount: uint64(snapc),
|
||||
@ -925,7 +928,7 @@ func TestConcurrentApplyAndSnapshotV3(t *testing.T) {
|
||||
r: raftNode{
|
||||
Node: n,
|
||||
transport: tr,
|
||||
storage: newStorageRecorder(testdir),
|
||||
storage: mockstorage.NewStorageRecorder(testdir),
|
||||
raftStorage: rs,
|
||||
},
|
||||
store: cl.store,
|
||||
@ -996,8 +999,8 @@ func TestConcurrentApplyAndSnapshotV3(t *testing.T) {
|
||||
// it should trigger storage.SaveSnap and also store.Recover.
|
||||
func TestRecvSnapshot(t *testing.T) {
|
||||
n := newNopReadyNode()
|
||||
st := store.NewRecorder()
|
||||
p := newStorageRecorder("")
|
||||
st := mockstore.NewRecorder()
|
||||
p := mockstorage.NewStorageRecorder("")
|
||||
cl := newCluster("abc")
|
||||
cl.SetStore(store.New())
|
||||
s := &EtcdServer{
|
||||
@ -1036,7 +1039,7 @@ func TestRecvSnapshot(t *testing.T) {
|
||||
// first and then committed entries.
|
||||
func TestApplySnapshotAndCommittedEntries(t *testing.T) {
|
||||
n := newNopReadyNode()
|
||||
st := store.NewRecorderStream()
|
||||
st := mockstore.NewRecorderStream()
|
||||
cl := newCluster("abc")
|
||||
cl.SetStore(store.New())
|
||||
storage := raft.NewMemoryStorage()
|
||||
@ -1044,7 +1047,7 @@ func TestApplySnapshotAndCommittedEntries(t *testing.T) {
|
||||
cfg: &ServerConfig{},
|
||||
r: raftNode{
|
||||
Node: n,
|
||||
storage: newStorageRecorder(""),
|
||||
storage: mockstorage.NewStorageRecorder(""),
|
||||
raftStorage: storage,
|
||||
transport: rafthttp.NewNopTransporter(),
|
||||
},
|
||||
@ -1088,7 +1091,7 @@ func TestAddMember(t *testing.T) {
|
||||
r: raftNode{
|
||||
Node: n,
|
||||
raftStorage: raft.NewMemoryStorage(),
|
||||
storage: newStorageRecorder(""),
|
||||
storage: mockstorage.NewStorageRecorder(""),
|
||||
transport: rafthttp.NewNopTransporter(),
|
||||
},
|
||||
cfg: &ServerConfig{},
|
||||
@ -1128,7 +1131,7 @@ func TestRemoveMember(t *testing.T) {
|
||||
r: raftNode{
|
||||
Node: n,
|
||||
raftStorage: raft.NewMemoryStorage(),
|
||||
storage: newStorageRecorder(""),
|
||||
storage: mockstorage.NewStorageRecorder(""),
|
||||
transport: rafthttp.NewNopTransporter(),
|
||||
},
|
||||
cfg: &ServerConfig{},
|
||||
@ -1167,7 +1170,7 @@ func TestUpdateMember(t *testing.T) {
|
||||
r: raftNode{
|
||||
Node: n,
|
||||
raftStorage: raft.NewMemoryStorage(),
|
||||
storage: newStorageRecorder(""),
|
||||
storage: mockstorage.NewStorageRecorder(""),
|
||||
transport: rafthttp.NewNopTransporter(),
|
||||
},
|
||||
store: st,
|
||||
@ -1248,7 +1251,7 @@ func TestPublishStopped(t *testing.T) {
|
||||
transport: rafthttp.NewNopTransporter(),
|
||||
},
|
||||
cluster: &cluster{},
|
||||
w: wait.NewNop(),
|
||||
w: mockwait.NewNop(),
|
||||
done: make(chan struct{}),
|
||||
stop: make(chan struct{}),
|
||||
reqIDGen: idutil.NewGenerator(0, time.Time{}),
|
||||
@ -1263,7 +1266,7 @@ func TestPublishRetry(t *testing.T) {
|
||||
srv := &EtcdServer{
|
||||
cfg: &ServerConfig{TickMs: 1},
|
||||
r: raftNode{Node: n},
|
||||
w: wait.NewNop(),
|
||||
w: mockwait.NewNop(),
|
||||
done: make(chan struct{}),
|
||||
reqIDGen: idutil.NewGenerator(0, time.Time{}),
|
||||
}
|
||||
|
@ -15,16 +15,13 @@
|
||||
package etcdserver
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"io"
|
||||
"os"
|
||||
"path"
|
||||
|
||||
pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
|
||||
"github.com/coreos/etcd/pkg/pbutil"
|
||||
"github.com/coreos/etcd/pkg/testutil"
|
||||
"github.com/coreos/etcd/pkg/types"
|
||||
"github.com/coreos/etcd/raft"
|
||||
"github.com/coreos/etcd/raft/raftpb"
|
||||
"github.com/coreos/etcd/snap"
|
||||
"github.com/coreos/etcd/version"
|
||||
@ -147,39 +144,3 @@ func makeMemberDir(dir string) error {
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
type storageRecorder struct {
|
||||
testutil.Recorder
|
||||
dbPath string // must have '/' suffix if set
|
||||
}
|
||||
|
||||
func newStorageRecorder(db string) *storageRecorder {
|
||||
return &storageRecorder{&testutil.RecorderBuffered{}, db}
|
||||
}
|
||||
|
||||
func newStorageRecorderStream(db string) *storageRecorder {
|
||||
return &storageRecorder{testutil.NewRecorderStream(), db}
|
||||
}
|
||||
|
||||
func (p *storageRecorder) Save(st raftpb.HardState, ents []raftpb.Entry) error {
|
||||
p.Record(testutil.Action{Name: "Save"})
|
||||
return nil
|
||||
}
|
||||
|
||||
func (p *storageRecorder) SaveSnap(st raftpb.Snapshot) error {
|
||||
if !raft.IsEmptySnap(st) {
|
||||
p.Record(testutil.Action{Name: "SaveSnap"})
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (p *storageRecorder) DBFilePath(id uint64) (string, error) {
|
||||
p.Record(testutil.Action{Name: "DBFilePath"})
|
||||
path := p.dbPath
|
||||
if path != "" {
|
||||
path = path + "/"
|
||||
}
|
||||
return fmt.Sprintf("%s%016x.snap.db", path, id), nil
|
||||
}
|
||||
|
||||
func (p *storageRecorder) Close() error { return nil }
|
||||
|
59
pkg/mock/mockstorage/storage_recorder.go
Normal file
59
pkg/mock/mockstorage/storage_recorder.go
Normal file
@ -0,0 +1,59 @@
|
||||
// Copyright 2015 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 mockstorage
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
"github.com/coreos/etcd/pkg/testutil"
|
||||
"github.com/coreos/etcd/raft"
|
||||
"github.com/coreos/etcd/raft/raftpb"
|
||||
)
|
||||
|
||||
type storageRecorder struct {
|
||||
testutil.Recorder
|
||||
dbPath string // must have '/' suffix if set
|
||||
}
|
||||
|
||||
func NewStorageRecorder(db string) *storageRecorder {
|
||||
return &storageRecorder{&testutil.RecorderBuffered{}, db}
|
||||
}
|
||||
|
||||
func NewStorageRecorderStream(db string) *storageRecorder {
|
||||
return &storageRecorder{testutil.NewRecorderStream(), db}
|
||||
}
|
||||
|
||||
func (p *storageRecorder) Save(st raftpb.HardState, ents []raftpb.Entry) error {
|
||||
p.Record(testutil.Action{Name: "Save"})
|
||||
return nil
|
||||
}
|
||||
|
||||
func (p *storageRecorder) SaveSnap(st raftpb.Snapshot) error {
|
||||
if !raft.IsEmptySnap(st) {
|
||||
p.Record(testutil.Action{Name: "SaveSnap"})
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (p *storageRecorder) DBFilePath(id uint64) (string, error) {
|
||||
p.Record(testutil.Action{Name: "DBFilePath"})
|
||||
path := p.dbPath
|
||||
if path != "" {
|
||||
path = path + "/"
|
||||
}
|
||||
return fmt.Sprintf("%s%016x.snap.db", path, id), nil
|
||||
}
|
||||
|
||||
func (p *storageRecorder) Close() error { return nil }
|
150
pkg/mock/mockstore/store_recorder.go
Normal file
150
pkg/mock/mockstore/store_recorder.go
Normal file
@ -0,0 +1,150 @@
|
||||
// Copyright 2015 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 mockstore
|
||||
|
||||
import (
|
||||
"time"
|
||||
|
||||
"github.com/coreos/etcd/pkg/testutil"
|
||||
"github.com/coreos/etcd/store"
|
||||
)
|
||||
|
||||
// StoreRecorder provides a Store interface with a testutil.Recorder
|
||||
type StoreRecorder struct {
|
||||
store.Store
|
||||
testutil.Recorder
|
||||
}
|
||||
|
||||
// storeRecorder records all the methods it receives.
|
||||
// storeRecorder DOES NOT work as a actual store.
|
||||
// It always returns invalid empty response and no error.
|
||||
type storeRecorder struct {
|
||||
store.Store
|
||||
testutil.Recorder
|
||||
}
|
||||
|
||||
func NewNop() store.Store { return &storeRecorder{Recorder: &testutil.RecorderBuffered{}} }
|
||||
func NewRecorder() *StoreRecorder {
|
||||
sr := &storeRecorder{Recorder: &testutil.RecorderBuffered{}}
|
||||
return &StoreRecorder{Store: sr, Recorder: sr.Recorder}
|
||||
}
|
||||
func NewRecorderStream() *StoreRecorder {
|
||||
sr := &storeRecorder{Recorder: testutil.NewRecorderStream()}
|
||||
return &StoreRecorder{Store: sr, Recorder: sr.Recorder}
|
||||
}
|
||||
|
||||
func (s *storeRecorder) Version() int { return 0 }
|
||||
func (s *storeRecorder) Index() uint64 { return 0 }
|
||||
func (s *storeRecorder) Get(path string, recursive, sorted bool) (*store.Event, error) {
|
||||
s.Record(testutil.Action{
|
||||
Name: "Get",
|
||||
Params: []interface{}{path, recursive, sorted},
|
||||
})
|
||||
return &store.Event{}, nil
|
||||
}
|
||||
func (s *storeRecorder) Set(path string, dir bool, val string, expireOpts store.TTLOptionSet) (*store.Event, error) {
|
||||
s.Record(testutil.Action{
|
||||
Name: "Set",
|
||||
Params: []interface{}{path, dir, val, expireOpts},
|
||||
})
|
||||
return &store.Event{}, nil
|
||||
}
|
||||
func (s *storeRecorder) Update(path, val string, expireOpts store.TTLOptionSet) (*store.Event, error) {
|
||||
s.Record(testutil.Action{
|
||||
Name: "Update",
|
||||
Params: []interface{}{path, val, expireOpts},
|
||||
})
|
||||
return &store.Event{}, nil
|
||||
}
|
||||
func (s *storeRecorder) Create(path string, dir bool, val string, uniq bool, expireOpts store.TTLOptionSet) (*store.Event, error) {
|
||||
s.Record(testutil.Action{
|
||||
Name: "Create",
|
||||
Params: []interface{}{path, dir, val, uniq, expireOpts},
|
||||
})
|
||||
return &store.Event{}, nil
|
||||
}
|
||||
func (s *storeRecorder) CompareAndSwap(path, prevVal string, prevIdx uint64, val string, expireOpts store.TTLOptionSet) (*store.Event, error) {
|
||||
s.Record(testutil.Action{
|
||||
Name: "CompareAndSwap",
|
||||
Params: []interface{}{path, prevVal, prevIdx, val, expireOpts},
|
||||
})
|
||||
return &store.Event{}, nil
|
||||
}
|
||||
func (s *storeRecorder) Delete(path string, dir, recursive bool) (*store.Event, error) {
|
||||
s.Record(testutil.Action{
|
||||
Name: "Delete",
|
||||
Params: []interface{}{path, dir, recursive},
|
||||
})
|
||||
return &store.Event{}, nil
|
||||
}
|
||||
func (s *storeRecorder) CompareAndDelete(path, prevVal string, prevIdx uint64) (*store.Event, error) {
|
||||
s.Record(testutil.Action{
|
||||
Name: "CompareAndDelete",
|
||||
Params: []interface{}{path, prevVal, prevIdx},
|
||||
})
|
||||
return &store.Event{}, nil
|
||||
}
|
||||
func (s *storeRecorder) Watch(_ string, _, _ bool, _ uint64) (store.Watcher, error) {
|
||||
s.Record(testutil.Action{Name: "Watch"})
|
||||
return store.NewNopWatcher(), nil
|
||||
}
|
||||
func (s *storeRecorder) Save() ([]byte, error) {
|
||||
s.Record(testutil.Action{Name: "Save"})
|
||||
return nil, nil
|
||||
}
|
||||
func (s *storeRecorder) Recovery(b []byte) error {
|
||||
s.Record(testutil.Action{Name: "Recovery"})
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *storeRecorder) SaveNoCopy() ([]byte, error) {
|
||||
s.Record(testutil.Action{Name: "SaveNoCopy"})
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
func (s *storeRecorder) Clone() store.Store {
|
||||
s.Record(testutil.Action{Name: "Clone"})
|
||||
return s
|
||||
}
|
||||
|
||||
func (s *storeRecorder) JsonStats() []byte { return nil }
|
||||
func (s *storeRecorder) DeleteExpiredKeys(cutoff time.Time) {
|
||||
s.Record(testutil.Action{
|
||||
Name: "DeleteExpiredKeys",
|
||||
Params: []interface{}{cutoff},
|
||||
})
|
||||
}
|
||||
|
||||
// errStoreRecorder is a storeRecorder, but returns the given error on
|
||||
// Get, Watch methods.
|
||||
type errStoreRecorder struct {
|
||||
storeRecorder
|
||||
err error
|
||||
}
|
||||
|
||||
func NewErrRecorder(err error) *StoreRecorder {
|
||||
sr := &errStoreRecorder{err: err}
|
||||
sr.Recorder = &testutil.RecorderBuffered{}
|
||||
return &StoreRecorder{Store: sr, Recorder: sr.Recorder}
|
||||
}
|
||||
|
||||
func (s *errStoreRecorder) Get(path string, recursive, sorted bool) (*store.Event, error) {
|
||||
s.storeRecorder.Get(path, recursive, sorted)
|
||||
return nil, s.err
|
||||
}
|
||||
func (s *errStoreRecorder) Watch(path string, recursive, sorted bool, index uint64) (store.Watcher, error) {
|
||||
s.storeRecorder.Watch(path, recursive, sorted, index)
|
||||
return nil, s.err
|
||||
}
|
43
pkg/mock/mockwait/wait_recorder.go
Normal file
43
pkg/mock/mockwait/wait_recorder.go
Normal file
@ -0,0 +1,43 @@
|
||||
// Copyright 2015 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 mockwait
|
||||
|
||||
import (
|
||||
"github.com/coreos/etcd/pkg/testutil"
|
||||
"github.com/coreos/etcd/pkg/wait"
|
||||
)
|
||||
|
||||
type WaitRecorder struct {
|
||||
wait.Wait
|
||||
testutil.Recorder
|
||||
}
|
||||
|
||||
type waitRecorder struct {
|
||||
testutil.RecorderBuffered
|
||||
}
|
||||
|
||||
func NewRecorder() *WaitRecorder {
|
||||
wr := &waitRecorder{}
|
||||
return &WaitRecorder{Wait: wr, Recorder: wr}
|
||||
}
|
||||
func NewNop() wait.Wait { return NewRecorder() }
|
||||
|
||||
func (w *waitRecorder) Register(id uint64) <-chan interface{} {
|
||||
w.Record(testutil.Action{Name: "Register"})
|
||||
return nil
|
||||
}
|
||||
func (w *waitRecorder) Trigger(id uint64, x interface{}) {
|
||||
w.Record(testutil.Action{Name: "Trigger"})
|
||||
}
|
@ -19,8 +19,6 @@ package wait
|
||||
import (
|
||||
"log"
|
||||
"sync"
|
||||
|
||||
"github.com/coreos/etcd/pkg/testutil"
|
||||
)
|
||||
|
||||
type Wait interface {
|
||||
@ -61,29 +59,6 @@ func (w *List) Trigger(id uint64, x interface{}) {
|
||||
}
|
||||
}
|
||||
|
||||
type WaitRecorder struct {
|
||||
Wait
|
||||
testutil.Recorder
|
||||
}
|
||||
|
||||
type waitRecorder struct {
|
||||
testutil.RecorderBuffered
|
||||
}
|
||||
|
||||
func NewRecorder() *WaitRecorder {
|
||||
wr := &waitRecorder{}
|
||||
return &WaitRecorder{Wait: wr, Recorder: wr}
|
||||
}
|
||||
func NewNop() Wait { return NewRecorder() }
|
||||
|
||||
func (w *waitRecorder) Register(id uint64) <-chan interface{} {
|
||||
w.Record(testutil.Action{Name: "Register"})
|
||||
return nil
|
||||
}
|
||||
func (w *waitRecorder) Trigger(id uint64, x interface{}) {
|
||||
w.Record(testutil.Action{Name: "Trigger"})
|
||||
}
|
||||
|
||||
type waitWithResponse struct {
|
||||
ch <-chan interface{}
|
||||
}
|
||||
|
129
store/store.go
129
store/store.go
@ -25,7 +25,6 @@ import (
|
||||
|
||||
"github.com/coreos/etcd/Godeps/_workspace/src/github.com/jonboulle/clockwork"
|
||||
etcdErr "github.com/coreos/etcd/error"
|
||||
"github.com/coreos/etcd/pkg/testutil"
|
||||
"github.com/coreos/etcd/pkg/types"
|
||||
)
|
||||
|
||||
@ -768,131 +767,3 @@ func (s *store) JsonStats() []byte {
|
||||
s.Stats.Watchers = uint64(s.WatcherHub.count)
|
||||
return s.Stats.toJson()
|
||||
}
|
||||
|
||||
// StoreRecorder provides a Store interface with a testutil.Recorder
|
||||
type StoreRecorder struct {
|
||||
Store
|
||||
testutil.Recorder
|
||||
}
|
||||
|
||||
// storeRecorder records all the methods it receives.
|
||||
// storeRecorder DOES NOT work as a actual store.
|
||||
// It always returns invalid empty response and no error.
|
||||
type storeRecorder struct {
|
||||
Store
|
||||
testutil.Recorder
|
||||
}
|
||||
|
||||
func NewNop() Store { return &storeRecorder{Recorder: &testutil.RecorderBuffered{}} }
|
||||
func NewRecorder() *StoreRecorder {
|
||||
sr := &storeRecorder{Recorder: &testutil.RecorderBuffered{}}
|
||||
return &StoreRecorder{Store: sr, Recorder: sr.Recorder}
|
||||
}
|
||||
func NewRecorderStream() *StoreRecorder {
|
||||
sr := &storeRecorder{Recorder: testutil.NewRecorderStream()}
|
||||
return &StoreRecorder{Store: sr, Recorder: sr.Recorder}
|
||||
}
|
||||
|
||||
func (s *storeRecorder) Version() int { return 0 }
|
||||
func (s *storeRecorder) Index() uint64 { return 0 }
|
||||
func (s *storeRecorder) Get(path string, recursive, sorted bool) (*Event, error) {
|
||||
s.Record(testutil.Action{
|
||||
Name: "Get",
|
||||
Params: []interface{}{path, recursive, sorted},
|
||||
})
|
||||
return &Event{}, nil
|
||||
}
|
||||
func (s *storeRecorder) Set(path string, dir bool, val string, expireOpts TTLOptionSet) (*Event, error) {
|
||||
s.Record(testutil.Action{
|
||||
Name: "Set",
|
||||
Params: []interface{}{path, dir, val, expireOpts},
|
||||
})
|
||||
return &Event{}, nil
|
||||
}
|
||||
func (s *storeRecorder) Update(path, val string, expireOpts TTLOptionSet) (*Event, error) {
|
||||
s.Record(testutil.Action{
|
||||
Name: "Update",
|
||||
Params: []interface{}{path, val, expireOpts},
|
||||
})
|
||||
return &Event{}, nil
|
||||
}
|
||||
func (s *storeRecorder) Create(path string, dir bool, val string, uniq bool, expireOpts TTLOptionSet) (*Event, error) {
|
||||
s.Record(testutil.Action{
|
||||
Name: "Create",
|
||||
Params: []interface{}{path, dir, val, uniq, expireOpts},
|
||||
})
|
||||
return &Event{}, nil
|
||||
}
|
||||
func (s *storeRecorder) CompareAndSwap(path, prevVal string, prevIdx uint64, val string, expireOpts TTLOptionSet) (*Event, error) {
|
||||
s.Record(testutil.Action{
|
||||
Name: "CompareAndSwap",
|
||||
Params: []interface{}{path, prevVal, prevIdx, val, expireOpts},
|
||||
})
|
||||
return &Event{}, nil
|
||||
}
|
||||
func (s *storeRecorder) Delete(path string, dir, recursive bool) (*Event, error) {
|
||||
s.Record(testutil.Action{
|
||||
Name: "Delete",
|
||||
Params: []interface{}{path, dir, recursive},
|
||||
})
|
||||
return &Event{}, nil
|
||||
}
|
||||
func (s *storeRecorder) CompareAndDelete(path, prevVal string, prevIdx uint64) (*Event, error) {
|
||||
s.Record(testutil.Action{
|
||||
Name: "CompareAndDelete",
|
||||
Params: []interface{}{path, prevVal, prevIdx},
|
||||
})
|
||||
return &Event{}, nil
|
||||
}
|
||||
func (s *storeRecorder) Watch(_ string, _, _ bool, _ uint64) (Watcher, error) {
|
||||
s.Record(testutil.Action{Name: "Watch"})
|
||||
return NewNopWatcher(), nil
|
||||
}
|
||||
func (s *storeRecorder) Save() ([]byte, error) {
|
||||
s.Record(testutil.Action{Name: "Save"})
|
||||
return nil, nil
|
||||
}
|
||||
func (s *storeRecorder) Recovery(b []byte) error {
|
||||
s.Record(testutil.Action{Name: "Recovery"})
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *storeRecorder) SaveNoCopy() ([]byte, error) {
|
||||
s.Record(testutil.Action{Name: "SaveNoCopy"})
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
func (s *storeRecorder) Clone() Store {
|
||||
s.Record(testutil.Action{Name: "Clone"})
|
||||
return s
|
||||
}
|
||||
|
||||
func (s *storeRecorder) JsonStats() []byte { return nil }
|
||||
func (s *storeRecorder) DeleteExpiredKeys(cutoff time.Time) {
|
||||
s.Record(testutil.Action{
|
||||
Name: "DeleteExpiredKeys",
|
||||
Params: []interface{}{cutoff},
|
||||
})
|
||||
}
|
||||
|
||||
// errStoreRecorder is a storeRecorder, but returns the given error on
|
||||
// Get, Watch methods.
|
||||
type errStoreRecorder struct {
|
||||
storeRecorder
|
||||
err error
|
||||
}
|
||||
|
||||
func NewErrRecorder(err error) *StoreRecorder {
|
||||
sr := &errStoreRecorder{err: err}
|
||||
sr.Recorder = &testutil.RecorderBuffered{}
|
||||
return &StoreRecorder{Store: sr, Recorder: sr.Recorder}
|
||||
}
|
||||
|
||||
func (s *errStoreRecorder) Get(path string, recursive, sorted bool) (*Event, error) {
|
||||
s.storeRecorder.Get(path, recursive, sorted)
|
||||
return nil, s.err
|
||||
}
|
||||
func (s *errStoreRecorder) Watch(path string, recursive, sorted bool, index uint64) (Watcher, error) {
|
||||
s.storeRecorder.Watch(path, recursive, sorted, index)
|
||||
return nil, s.err
|
||||
}
|
||||
|
Loading…
x
Reference in New Issue
Block a user