mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00
*: move "internal/store" to "etcdserver/v2store"
Signed-off-by: Gyuho Lee <gyuhox@gmail.com>
This commit is contained in:
parent
dd2f3b0de8
commit
0e12e888e0
@ -30,16 +30,17 @@ import (
|
||||
"github.com/coreos/etcd/etcdserver/api"
|
||||
pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
|
||||
"github.com/coreos/etcd/etcdserver/membership"
|
||||
"github.com/coreos/etcd/etcdserver/v2store"
|
||||
"github.com/coreos/etcd/internal/mvcc"
|
||||
"github.com/coreos/etcd/internal/mvcc/backend"
|
||||
"github.com/coreos/etcd/internal/mvcc/mvccpb"
|
||||
"github.com/coreos/etcd/internal/raftsnap"
|
||||
"github.com/coreos/etcd/internal/store"
|
||||
"github.com/coreos/etcd/pkg/pbutil"
|
||||
"github.com/coreos/etcd/pkg/types"
|
||||
"github.com/coreos/etcd/raft/raftpb"
|
||||
"github.com/coreos/etcd/wal"
|
||||
"github.com/coreos/etcd/wal/walpb"
|
||||
|
||||
"github.com/gogo/protobuf/proto"
|
||||
"github.com/spf13/cobra"
|
||||
)
|
||||
@ -124,7 +125,7 @@ func prepareBackend() backend.Backend {
|
||||
return be
|
||||
}
|
||||
|
||||
func rebuildStoreV2() (store.Store, uint64) {
|
||||
func rebuildStoreV2() (v2store.Store, uint64) {
|
||||
var index uint64
|
||||
cl := membership.NewCluster("")
|
||||
|
||||
@ -157,7 +158,7 @@ func rebuildStoreV2() (store.Store, uint64) {
|
||||
ExitWithError(ExitError, err)
|
||||
}
|
||||
|
||||
st := store.New()
|
||||
st := v2store.New()
|
||||
if snapshot != nil {
|
||||
err := st.Recovery(snapshot.Data)
|
||||
if err != nil {
|
||||
@ -237,7 +238,7 @@ func applyRequest(req *pb.Request, applyV2 etcdserver.ApplierV2) {
|
||||
}
|
||||
}
|
||||
|
||||
func writeStore(w io.Writer, st store.Store) uint64 {
|
||||
func writeStore(w io.Writer, st v2store.Store) uint64 {
|
||||
all, err := st.Get("/1", true, true)
|
||||
if err != nil {
|
||||
if eerr, ok := err.(*etcdErr.Error); ok && eerr.ErrorCode == etcdErr.EcodeKeyNotFound {
|
||||
@ -249,7 +250,7 @@ func writeStore(w io.Writer, st store.Store) uint64 {
|
||||
return writeKeys(w, all.Node)
|
||||
}
|
||||
|
||||
func writeKeys(w io.Writer, n *store.NodeExtern) uint64 {
|
||||
func writeKeys(w io.Writer, n *v2store.NodeExtern) uint64 {
|
||||
maxIndex := n.ModifiedIndex
|
||||
|
||||
nodes := n.Nodes
|
||||
|
@ -36,7 +36,7 @@ import (
|
||||
"github.com/coreos/etcd/etcdserver/membership"
|
||||
"github.com/coreos/etcd/etcdserver/stats"
|
||||
"github.com/coreos/etcd/etcdserver/v2auth"
|
||||
"github.com/coreos/etcd/internal/store"
|
||||
"github.com/coreos/etcd/etcdserver/v2store"
|
||||
"github.com/coreos/etcd/pkg/types"
|
||||
|
||||
"github.com/jonboulle/clockwork"
|
||||
@ -518,8 +518,8 @@ func writeKeyEvent(w http.ResponseWriter, resp etcdserver.Response, noValueOnSuc
|
||||
|
||||
ev = trimEventPrefix(ev, etcdserver.StoreKeysPrefix)
|
||||
if noValueOnSuccess &&
|
||||
(ev.Action == store.Set || ev.Action == store.CompareAndSwap ||
|
||||
ev.Action == store.Create || ev.Action == store.Update) {
|
||||
(ev.Action == v2store.Set || ev.Action == v2store.CompareAndSwap ||
|
||||
ev.Action == v2store.Create || ev.Action == v2store.Update) {
|
||||
ev.Node = nil
|
||||
ev.PrevNode = nil
|
||||
}
|
||||
@ -599,7 +599,7 @@ func handleKeyWatch(ctx context.Context, w http.ResponseWriter, resp etcdserver.
|
||||
}
|
||||
}
|
||||
|
||||
func trimEventPrefix(ev *store.Event, prefix string) *store.Event {
|
||||
func trimEventPrefix(ev *v2store.Event, prefix string) *v2store.Event {
|
||||
if ev == nil {
|
||||
return nil
|
||||
}
|
||||
@ -611,7 +611,7 @@ func trimEventPrefix(ev *store.Event, prefix string) *store.Event {
|
||||
return e
|
||||
}
|
||||
|
||||
func trimNodeExternPrefix(n *store.NodeExtern, prefix string) {
|
||||
func trimNodeExternPrefix(n *v2store.NodeExtern, prefix string) {
|
||||
if n == nil {
|
||||
return
|
||||
}
|
||||
|
@ -35,7 +35,7 @@ import (
|
||||
"github.com/coreos/etcd/etcdserver/api/v2http/httptypes"
|
||||
"github.com/coreos/etcd/etcdserver/etcdserverpb"
|
||||
"github.com/coreos/etcd/etcdserver/membership"
|
||||
"github.com/coreos/etcd/internal/store"
|
||||
"github.com/coreos/etcd/etcdserver/v2store"
|
||||
"github.com/coreos/etcd/pkg/testutil"
|
||||
"github.com/coreos/etcd/pkg/types"
|
||||
"github.com/coreos/etcd/raft/raftpb"
|
||||
@ -44,7 +44,7 @@ import (
|
||||
"github.com/jonboulle/clockwork"
|
||||
)
|
||||
|
||||
func mustMarshalEvent(t *testing.T, ev *store.Event) string {
|
||||
func mustMarshalEvent(t *testing.T, ev *v2store.Event) string {
|
||||
b := new(bytes.Buffer)
|
||||
if err := json.NewEncoder(b).Encode(ev); err != nil {
|
||||
t.Fatalf("error marshalling event %#v: %v", ev, err)
|
||||
@ -176,11 +176,11 @@ func (drt dummyRaftTimer) Index() uint64 { return uint64(100) }
|
||||
func (drt dummyRaftTimer) Term() uint64 { return uint64(5) }
|
||||
|
||||
type dummyWatcher struct {
|
||||
echan chan *store.Event
|
||||
echan chan *v2store.Event
|
||||
sidx uint64
|
||||
}
|
||||
|
||||
func (w *dummyWatcher) EventChan() chan *store.Event {
|
||||
func (w *dummyWatcher) EventChan() chan *v2store.Event {
|
||||
return w.echan
|
||||
}
|
||||
func (w *dummyWatcher) StartIndex() uint64 { return w.sidx }
|
||||
@ -1172,7 +1172,7 @@ func TestWriteEvent(t *testing.T) {
|
||||
}
|
||||
|
||||
tests := []struct {
|
||||
ev *store.Event
|
||||
ev *v2store.Event
|
||||
noValue bool
|
||||
idx string
|
||||
// TODO(jonboulle): check body as well as just status code
|
||||
@ -1181,10 +1181,10 @@ func TestWriteEvent(t *testing.T) {
|
||||
}{
|
||||
// standard case, standard 200 response
|
||||
{
|
||||
&store.Event{
|
||||
Action: store.Get,
|
||||
Node: &store.NodeExtern{},
|
||||
PrevNode: &store.NodeExtern{},
|
||||
&v2store.Event{
|
||||
Action: v2store.Get,
|
||||
Node: &v2store.NodeExtern{},
|
||||
PrevNode: &v2store.NodeExtern{},
|
||||
},
|
||||
false,
|
||||
"0",
|
||||
@ -1193,10 +1193,10 @@ func TestWriteEvent(t *testing.T) {
|
||||
},
|
||||
// check new nodes return StatusCreated
|
||||
{
|
||||
&store.Event{
|
||||
Action: store.Create,
|
||||
Node: &store.NodeExtern{},
|
||||
PrevNode: &store.NodeExtern{},
|
||||
&v2store.Event{
|
||||
Action: v2store.Create,
|
||||
Node: &v2store.NodeExtern{},
|
||||
PrevNode: &v2store.NodeExtern{},
|
||||
},
|
||||
false,
|
||||
"0",
|
||||
@ -1539,9 +1539,9 @@ func TestServeKeysGood(t *testing.T) {
|
||||
}
|
||||
server := &resServer{
|
||||
res: etcdserver.Response{
|
||||
Event: &store.Event{
|
||||
Action: store.Get,
|
||||
Node: &store.NodeExtern{},
|
||||
Event: &v2store.Event{
|
||||
Action: v2store.Get,
|
||||
Node: &v2store.NodeExtern{},
|
||||
},
|
||||
},
|
||||
}
|
||||
@ -1564,20 +1564,20 @@ func TestServeKeysEvent(t *testing.T) {
|
||||
req *http.Request
|
||||
rsp etcdserver.Response
|
||||
wcode int
|
||||
event *store.Event
|
||||
event *v2store.Event
|
||||
}{
|
||||
{
|
||||
mustNewRequest(t, "foo"),
|
||||
etcdserver.Response{
|
||||
Event: &store.Event{
|
||||
Action: store.Get,
|
||||
Node: &store.NodeExtern{},
|
||||
Event: &v2store.Event{
|
||||
Action: v2store.Get,
|
||||
Node: &v2store.NodeExtern{},
|
||||
},
|
||||
},
|
||||
http.StatusOK,
|
||||
&store.Event{
|
||||
Action: store.Get,
|
||||
Node: &store.NodeExtern{},
|
||||
&v2store.Event{
|
||||
Action: v2store.Get,
|
||||
Node: &v2store.NodeExtern{},
|
||||
},
|
||||
},
|
||||
{
|
||||
@ -1587,13 +1587,13 @@ func TestServeKeysEvent(t *testing.T) {
|
||||
url.Values{"noValueOnSuccess": []string{"true"}},
|
||||
),
|
||||
etcdserver.Response{
|
||||
Event: &store.Event{
|
||||
Event: &v2store.Event{
|
||||
Action: store.CompareAndSwap,
|
||||
Node: &store.NodeExtern{},
|
||||
Node: &v2store.NodeExtern{},
|
||||
},
|
||||
},
|
||||
http.StatusOK,
|
||||
&store.Event{
|
||||
&v2store.Event{
|
||||
Action: store.CompareAndSwap,
|
||||
Node: nil,
|
||||
},
|
||||
@ -1634,7 +1634,7 @@ func TestServeKeysEvent(t *testing.T) {
|
||||
|
||||
func TestServeKeysWatch(t *testing.T) {
|
||||
req := mustNewRequest(t, "/foo/bar")
|
||||
ec := make(chan *store.Event)
|
||||
ec := make(chan *v2store.Event)
|
||||
dw := &dummyWatcher{
|
||||
echan: ec,
|
||||
}
|
||||
@ -1649,9 +1649,9 @@ func TestServeKeysWatch(t *testing.T) {
|
||||
cluster: &fakeCluster{id: 1},
|
||||
}
|
||||
go func() {
|
||||
ec <- &store.Event{
|
||||
Action: store.Get,
|
||||
Node: &store.NodeExtern{},
|
||||
ec <- &v2store.Event{
|
||||
Action: v2store.Get,
|
||||
Node: &v2store.NodeExtern{},
|
||||
}
|
||||
}()
|
||||
rw := httptest.NewRecorder()
|
||||
@ -1661,9 +1661,9 @@ func TestServeKeysWatch(t *testing.T) {
|
||||
wcode := http.StatusOK
|
||||
wbody := mustMarshalEvent(
|
||||
t,
|
||||
&store.Event{
|
||||
Action: store.Get,
|
||||
Node: &store.NodeExtern{},
|
||||
&v2store.Event{
|
||||
Action: v2store.Get,
|
||||
Node: &v2store.NodeExtern{},
|
||||
},
|
||||
)
|
||||
|
||||
@ -1695,12 +1695,12 @@ func TestHandleWatch(t *testing.T) {
|
||||
r := httptest.NewRecorder()
|
||||
return r, r
|
||||
}
|
||||
noopEv := func(chan *store.Event) {}
|
||||
noopEv := func(chan *v2store.Event) {}
|
||||
|
||||
tests := []struct {
|
||||
getCtx func() context.Context
|
||||
getRwRr func() (http.ResponseWriter, *httptest.ResponseRecorder)
|
||||
doToChan func(chan *store.Event)
|
||||
doToChan func(chan *v2store.Event)
|
||||
|
||||
wbody string
|
||||
}{
|
||||
@ -1708,18 +1708,18 @@ func TestHandleWatch(t *testing.T) {
|
||||
// Normal case: one event
|
||||
context.Background,
|
||||
defaultRwRr,
|
||||
func(ch chan *store.Event) {
|
||||
ch <- &store.Event{
|
||||
Action: store.Get,
|
||||
Node: &store.NodeExtern{},
|
||||
func(ch chan *v2store.Event) {
|
||||
ch <- &v2store.Event{
|
||||
Action: v2store.Get,
|
||||
Node: &v2store.NodeExtern{},
|
||||
}
|
||||
},
|
||||
|
||||
mustMarshalEvent(
|
||||
t,
|
||||
&store.Event{
|
||||
Action: store.Get,
|
||||
Node: &store.NodeExtern{},
|
||||
&v2store.Event{
|
||||
Action: v2store.Get,
|
||||
Node: &v2store.NodeExtern{},
|
||||
},
|
||||
),
|
||||
},
|
||||
@ -1727,7 +1727,7 @@ func TestHandleWatch(t *testing.T) {
|
||||
// Channel is closed, no event
|
||||
context.Background,
|
||||
defaultRwRr,
|
||||
func(ch chan *store.Event) {
|
||||
func(ch chan *v2store.Event) {
|
||||
close(ch)
|
||||
},
|
||||
|
||||
@ -1765,7 +1765,7 @@ func TestHandleWatch(t *testing.T) {
|
||||
for i, tt := range tests {
|
||||
rw, rr := tt.getRwRr()
|
||||
wa := &dummyWatcher{
|
||||
echan: make(chan *store.Event, 1),
|
||||
echan: make(chan *v2store.Event, 1),
|
||||
sidx: 10,
|
||||
}
|
||||
tt.doToChan(wa.echan)
|
||||
@ -1808,7 +1808,7 @@ func TestHandleWatchStreaming(t *testing.T) {
|
||||
make(chan struct{}, 1),
|
||||
}
|
||||
wa := &dummyWatcher{
|
||||
echan: make(chan *store.Event),
|
||||
echan: make(chan *v2store.Event),
|
||||
}
|
||||
|
||||
// Launch the streaming handler in the background with a cancellable context
|
||||
@ -1846,9 +1846,9 @@ func TestHandleWatchStreaming(t *testing.T) {
|
||||
|
||||
// Now send the first event
|
||||
select {
|
||||
case wa.echan <- &store.Event{
|
||||
Action: store.Get,
|
||||
Node: &store.NodeExtern{},
|
||||
case wa.echan <- &v2store.Event{
|
||||
Action: v2store.Get,
|
||||
Node: &v2store.NodeExtern{},
|
||||
}:
|
||||
case <-time.After(time.Second):
|
||||
t.Fatal("timed out waiting for send")
|
||||
@ -1864,9 +1864,9 @@ func TestHandleWatchStreaming(t *testing.T) {
|
||||
// And check the body is as expected
|
||||
wbody = mustMarshalEvent(
|
||||
t,
|
||||
&store.Event{
|
||||
Action: store.Get,
|
||||
Node: &store.NodeExtern{},
|
||||
&v2store.Event{
|
||||
Action: v2store.Get,
|
||||
Node: &v2store.NodeExtern{},
|
||||
},
|
||||
)
|
||||
g = rw.Body.String()
|
||||
@ -1876,9 +1876,9 @@ func TestHandleWatchStreaming(t *testing.T) {
|
||||
|
||||
// Rinse and repeat
|
||||
select {
|
||||
case wa.echan <- &store.Event{
|
||||
Action: store.Get,
|
||||
Node: &store.NodeExtern{},
|
||||
case wa.echan <- &v2store.Event{
|
||||
Action: v2store.Get,
|
||||
Node: &v2store.NodeExtern{},
|
||||
}:
|
||||
case <-time.After(time.Second):
|
||||
t.Fatal("timed out waiting for send")
|
||||
@ -1910,33 +1910,33 @@ func TestHandleWatchStreaming(t *testing.T) {
|
||||
func TestTrimEventPrefix(t *testing.T) {
|
||||
pre := "/abc"
|
||||
tests := []struct {
|
||||
ev *store.Event
|
||||
wev *store.Event
|
||||
ev *v2store.Event
|
||||
wev *v2store.Event
|
||||
}{
|
||||
{
|
||||
nil,
|
||||
nil,
|
||||
},
|
||||
{
|
||||
&store.Event{},
|
||||
&store.Event{},
|
||||
&v2store.Event{},
|
||||
&v2store.Event{},
|
||||
},
|
||||
{
|
||||
&store.Event{Node: &store.NodeExtern{Key: "/abc/def"}},
|
||||
&store.Event{Node: &store.NodeExtern{Key: "/def"}},
|
||||
&v2store.Event{Node: &v2store.NodeExtern{Key: "/abc/def"}},
|
||||
&v2store.Event{Node: &v2store.NodeExtern{Key: "/def"}},
|
||||
},
|
||||
{
|
||||
&store.Event{PrevNode: &store.NodeExtern{Key: "/abc/ghi"}},
|
||||
&store.Event{PrevNode: &store.NodeExtern{Key: "/ghi"}},
|
||||
&v2store.Event{PrevNode: &v2store.NodeExtern{Key: "/abc/ghi"}},
|
||||
&v2store.Event{PrevNode: &v2store.NodeExtern{Key: "/ghi"}},
|
||||
},
|
||||
{
|
||||
&store.Event{
|
||||
Node: &store.NodeExtern{Key: "/abc/def"},
|
||||
PrevNode: &store.NodeExtern{Key: "/abc/ghi"},
|
||||
&v2store.Event{
|
||||
Node: &v2store.NodeExtern{Key: "/abc/def"},
|
||||
PrevNode: &v2store.NodeExtern{Key: "/abc/ghi"},
|
||||
},
|
||||
&store.Event{
|
||||
Node: &store.NodeExtern{Key: "/def"},
|
||||
PrevNode: &store.NodeExtern{Key: "/ghi"},
|
||||
&v2store.Event{
|
||||
Node: &v2store.NodeExtern{Key: "/def"},
|
||||
PrevNode: &v2store.NodeExtern{Key: "/ghi"},
|
||||
},
|
||||
},
|
||||
}
|
||||
@ -1951,28 +1951,28 @@ func TestTrimEventPrefix(t *testing.T) {
|
||||
func TestTrimNodeExternPrefix(t *testing.T) {
|
||||
pre := "/abc"
|
||||
tests := []struct {
|
||||
n *store.NodeExtern
|
||||
wn *store.NodeExtern
|
||||
n *v2store.NodeExtern
|
||||
wn *v2store.NodeExtern
|
||||
}{
|
||||
{
|
||||
nil,
|
||||
nil,
|
||||
},
|
||||
{
|
||||
&store.NodeExtern{Key: "/abc/def"},
|
||||
&store.NodeExtern{Key: "/def"},
|
||||
&v2store.NodeExtern{Key: "/abc/def"},
|
||||
&v2store.NodeExtern{Key: "/def"},
|
||||
},
|
||||
{
|
||||
&store.NodeExtern{
|
||||
&v2store.NodeExtern{
|
||||
Key: "/abc/def",
|
||||
Nodes: []*store.NodeExtern{
|
||||
Nodes: []*v2store.NodeExtern{
|
||||
{Key: "/abc/def/1"},
|
||||
{Key: "/abc/def/2"},
|
||||
},
|
||||
},
|
||||
&store.NodeExtern{
|
||||
&v2store.NodeExtern{
|
||||
Key: "/def",
|
||||
Nodes: []*store.NodeExtern{
|
||||
Nodes: []*v2store.NodeExtern{
|
||||
{Key: "/def/1"},
|
||||
{Key: "/def/2"},
|
||||
},
|
||||
|
@ -24,8 +24,8 @@ import (
|
||||
"github.com/coreos/etcd/clientv3"
|
||||
"github.com/coreos/etcd/clientv3/concurrency"
|
||||
etcdErr "github.com/coreos/etcd/error"
|
||||
"github.com/coreos/etcd/etcdserver/v2store"
|
||||
"github.com/coreos/etcd/internal/mvcc/mvccpb"
|
||||
"github.com/coreos/etcd/internal/store"
|
||||
)
|
||||
|
||||
// store implements the Store interface for V2 using
|
||||
@ -41,13 +41,13 @@ const maxPathDepth = 63
|
||||
|
||||
var errUnsupported = fmt.Errorf("TTLs are unsupported")
|
||||
|
||||
func NewStore(c *clientv3.Client, pfx string) store.Store { return newStore(c, pfx) }
|
||||
func NewStore(c *clientv3.Client, pfx string) v2store.Store { return newStore(c, pfx) }
|
||||
|
||||
func newStore(c *clientv3.Client, pfx string) *v2v3Store { return &v2v3Store{c, pfx, c.Ctx()} }
|
||||
|
||||
func (s *v2v3Store) Index() uint64 { panic("STUB") }
|
||||
|
||||
func (s *v2v3Store) Get(nodePath string, recursive, sorted bool) (*store.Event, error) {
|
||||
func (s *v2v3Store) Get(nodePath string, recursive, sorted bool) (*v2store.Event, error) {
|
||||
key := s.mkPath(nodePath)
|
||||
resp, err := s.c.Txn(s.ctx).Then(
|
||||
clientv3.OpGet(key+"/"),
|
||||
@ -66,9 +66,9 @@ func (s *v2v3Store) Get(nodePath string, recursive, sorted bool) (*store.Event,
|
||||
if len(kvs) > 0 {
|
||||
cidx, midx = mkV2Rev(kvs[0].CreateRevision), mkV2Rev(kvs[0].ModRevision)
|
||||
}
|
||||
return &store.Event{
|
||||
Action: store.Get,
|
||||
Node: &store.NodeExtern{
|
||||
return &v2store.Event{
|
||||
Action: v2store.Get,
|
||||
Node: &v2store.NodeExtern{
|
||||
Key: nodePath,
|
||||
Dir: true,
|
||||
Nodes: nodes,
|
||||
@ -84,20 +84,20 @@ func (s *v2v3Store) Get(nodePath string, recursive, sorted bool) (*store.Event,
|
||||
return nil, etcdErr.NewError(etcdErr.EcodeKeyNotFound, nodePath, mkV2Rev(resp.Header.Revision))
|
||||
}
|
||||
|
||||
return &store.Event{
|
||||
Action: store.Get,
|
||||
return &v2store.Event{
|
||||
Action: v2store.Get,
|
||||
Node: s.mkV2Node(kvs[0]),
|
||||
EtcdIndex: mkV2Rev(resp.Header.Revision),
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (s *v2v3Store) getDir(nodePath string, recursive, sorted bool, rev int64) ([]*store.NodeExtern, error) {
|
||||
func (s *v2v3Store) getDir(nodePath string, recursive, sorted bool, rev int64) ([]*v2store.NodeExtern, error) {
|
||||
rootNodes, err := s.getDirDepth(nodePath, 1, rev)
|
||||
if err != nil || !recursive {
|
||||
return rootNodes, err
|
||||
}
|
||||
nextNodes := rootNodes
|
||||
nodes := make(map[string]*store.NodeExtern)
|
||||
nodes := make(map[string]*v2store.NodeExtern)
|
||||
// Breadth walk the subdirectories
|
||||
for i := 2; len(nextNodes) > 0; i++ {
|
||||
for _, n := range nextNodes {
|
||||
@ -113,14 +113,14 @@ func (s *v2v3Store) getDir(nodePath string, recursive, sorted bool, rev int64) (
|
||||
return rootNodes, nil
|
||||
}
|
||||
|
||||
func (s *v2v3Store) getDirDepth(nodePath string, depth int, rev int64) ([]*store.NodeExtern, error) {
|
||||
func (s *v2v3Store) getDirDepth(nodePath string, depth int, rev int64) ([]*v2store.NodeExtern, error) {
|
||||
pd := s.mkPathDepth(nodePath, depth)
|
||||
resp, err := s.c.Get(s.ctx, pd, clientv3.WithPrefix(), clientv3.WithRev(rev))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
nodes := make([]*store.NodeExtern, len(resp.Kvs))
|
||||
nodes := make([]*v2store.NodeExtern, len(resp.Kvs))
|
||||
for i, kv := range resp.Kvs {
|
||||
nodes[i] = s.mkV2Node(kv)
|
||||
}
|
||||
@ -131,8 +131,8 @@ func (s *v2v3Store) Set(
|
||||
nodePath string,
|
||||
dir bool,
|
||||
value string,
|
||||
expireOpts store.TTLOptionSet,
|
||||
) (*store.Event, error) {
|
||||
expireOpts v2store.TTLOptionSet,
|
||||
) (*v2store.Event, error) {
|
||||
if expireOpts.Refresh || !expireOpts.ExpireTime.IsZero() {
|
||||
return nil, errUnsupported
|
||||
}
|
||||
@ -162,7 +162,7 @@ func (s *v2v3Store) Set(
|
||||
return nil
|
||||
}
|
||||
stm.Put(key, value, clientv3.WithPrevKV())
|
||||
stm.Put(s.mkActionKey(), store.Set)
|
||||
stm.Put(s.mkActionKey(), v2store.Set)
|
||||
return nil
|
||||
}
|
||||
|
||||
@ -175,7 +175,7 @@ func (s *v2v3Store) Set(
|
||||
}
|
||||
|
||||
createRev := resp.Header.Revision
|
||||
var pn *store.NodeExtern
|
||||
var pn *v2store.NodeExtern
|
||||
if pkv := prevKeyFromPuts(resp); pkv != nil {
|
||||
pn = s.mkV2Node(pkv)
|
||||
createRev = pkv.CreateRevision
|
||||
@ -185,9 +185,9 @@ func (s *v2v3Store) Set(
|
||||
if dir {
|
||||
vp = nil
|
||||
}
|
||||
return &store.Event{
|
||||
Action: store.Set,
|
||||
Node: &store.NodeExtern{
|
||||
return &v2store.Event{
|
||||
Action: v2store.Set,
|
||||
Node: &v2store.NodeExtern{
|
||||
Key: nodePath,
|
||||
Value: vp,
|
||||
Dir: dir,
|
||||
@ -199,7 +199,7 @@ func (s *v2v3Store) Set(
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (s *v2v3Store) Update(nodePath, newValue string, expireOpts store.TTLOptionSet) (*store.Event, error) {
|
||||
func (s *v2v3Store) Update(nodePath, newValue string, expireOpts v2store.TTLOptionSet) (*v2store.Event, error) {
|
||||
if isRoot(nodePath) {
|
||||
return nil, etcdErr.NewError(etcdErr.EcodeRootROnly, nodePath, 0)
|
||||
}
|
||||
@ -220,7 +220,7 @@ func (s *v2v3Store) Update(nodePath, newValue string, expireOpts store.TTLOption
|
||||
return nil
|
||||
}
|
||||
stm.Put(key, newValue, clientv3.WithPrevKV())
|
||||
stm.Put(s.mkActionKey(), store.Update)
|
||||
stm.Put(s.mkActionKey(), v2store.Update)
|
||||
return nil
|
||||
}
|
||||
|
||||
@ -233,9 +233,9 @@ func (s *v2v3Store) Update(nodePath, newValue string, expireOpts store.TTLOption
|
||||
}
|
||||
|
||||
pkv := prevKeyFromPuts(resp)
|
||||
return &store.Event{
|
||||
Action: store.Update,
|
||||
Node: &store.NodeExtern{
|
||||
return &v2store.Event{
|
||||
Action: v2store.Update,
|
||||
Node: &v2store.NodeExtern{
|
||||
Key: nodePath,
|
||||
Value: &newValue,
|
||||
ModifiedIndex: mkV2Rev(resp.Header.Revision),
|
||||
@ -251,8 +251,8 @@ func (s *v2v3Store) Create(
|
||||
dir bool,
|
||||
value string,
|
||||
unique bool,
|
||||
expireOpts store.TTLOptionSet,
|
||||
) (*store.Event, error) {
|
||||
expireOpts v2store.TTLOptionSet,
|
||||
) (*v2store.Event, error) {
|
||||
if isRoot(nodePath) {
|
||||
return nil, etcdErr.NewError(etcdErr.EcodeRootROnly, nodePath, 0)
|
||||
}
|
||||
@ -299,7 +299,7 @@ func (s *v2v3Store) Create(
|
||||
key += "/"
|
||||
}
|
||||
stm.Put(key, value)
|
||||
stm.Put(s.mkActionKey(), store.Create)
|
||||
stm.Put(s.mkActionKey(), v2store.Create)
|
||||
return nil
|
||||
}
|
||||
|
||||
@ -316,9 +316,9 @@ func (s *v2v3Store) Create(
|
||||
v = &value
|
||||
}
|
||||
|
||||
return &store.Event{
|
||||
Action: store.Create,
|
||||
Node: &store.NodeExtern{
|
||||
return &v2store.Event{
|
||||
Action: v2store.Create,
|
||||
Node: &v2store.NodeExtern{
|
||||
Key: nodePath,
|
||||
Value: v,
|
||||
Dir: dir,
|
||||
@ -334,8 +334,8 @@ func (s *v2v3Store) CompareAndSwap(
|
||||
prevValue string,
|
||||
prevIndex uint64,
|
||||
value string,
|
||||
expireOpts store.TTLOptionSet,
|
||||
) (*store.Event, error) {
|
||||
expireOpts v2store.TTLOptionSet,
|
||||
) (*v2store.Event, error) {
|
||||
if isRoot(nodePath) {
|
||||
return nil, etcdErr.NewError(etcdErr.EcodeRootROnly, nodePath, 0)
|
||||
}
|
||||
@ -348,7 +348,7 @@ func (s *v2v3Store) CompareAndSwap(
|
||||
s.mkCompare(nodePath, prevValue, prevIndex)...,
|
||||
).Then(
|
||||
clientv3.OpPut(key, value, clientv3.WithPrevKV()),
|
||||
clientv3.OpPut(s.mkActionKey(), store.CompareAndSwap),
|
||||
clientv3.OpPut(s.mkActionKey(), v2store.CompareAndSwap),
|
||||
).Else(
|
||||
clientv3.OpGet(key),
|
||||
clientv3.OpGet(key+"/"),
|
||||
@ -362,9 +362,9 @@ func (s *v2v3Store) CompareAndSwap(
|
||||
}
|
||||
|
||||
pkv := resp.Responses[0].GetResponsePut().PrevKv
|
||||
return &store.Event{
|
||||
Action: store.CompareAndSwap,
|
||||
Node: &store.NodeExtern{
|
||||
return &v2store.Event{
|
||||
Action: v2store.CompareAndSwap,
|
||||
Node: &v2store.NodeExtern{
|
||||
Key: nodePath,
|
||||
Value: &value,
|
||||
CreatedIndex: mkV2Rev(pkv.CreateRevision),
|
||||
@ -375,7 +375,7 @@ func (s *v2v3Store) CompareAndSwap(
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (s *v2v3Store) Delete(nodePath string, dir, recursive bool) (*store.Event, error) {
|
||||
func (s *v2v3Store) Delete(nodePath string, dir, recursive bool) (*v2store.Event, error) {
|
||||
if isRoot(nodePath) {
|
||||
return nil, etcdErr.NewError(etcdErr.EcodeRootROnly, nodePath, 0)
|
||||
}
|
||||
@ -391,7 +391,7 @@ func (s *v2v3Store) Delete(nodePath string, dir, recursive bool) (*store.Event,
|
||||
for i := 1; i < maxPathDepth; i++ {
|
||||
dels[i] = clientv3.OpDelete(s.mkPathDepth(nodePath, i), clientv3.WithPrefix())
|
||||
}
|
||||
dels[maxPathDepth] = clientv3.OpPut(s.mkActionKey(), store.Delete)
|
||||
dels[maxPathDepth] = clientv3.OpPut(s.mkActionKey(), v2store.Delete)
|
||||
|
||||
resp, err := s.c.Txn(s.ctx).If(
|
||||
clientv3.Compare(clientv3.Version(s.mkPath(nodePath)+"/"), ">", 0),
|
||||
@ -406,19 +406,19 @@ func (s *v2v3Store) Delete(nodePath string, dir, recursive bool) (*store.Event,
|
||||
return nil, etcdErr.NewError(etcdErr.EcodeNodeExist, nodePath, mkV2Rev(resp.Header.Revision))
|
||||
}
|
||||
dresp := resp.Responses[0].GetResponseDeleteRange()
|
||||
return &store.Event{
|
||||
Action: store.Delete,
|
||||
return &v2store.Event{
|
||||
Action: v2store.Delete,
|
||||
PrevNode: s.mkV2Node(dresp.PrevKvs[0]),
|
||||
EtcdIndex: mkV2Rev(resp.Header.Revision),
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (s *v2v3Store) deleteEmptyDir(nodePath string) (*store.Event, error) {
|
||||
func (s *v2v3Store) deleteEmptyDir(nodePath string) (*v2store.Event, error) {
|
||||
resp, err := s.c.Txn(s.ctx).If(
|
||||
clientv3.Compare(clientv3.Version(s.mkPathDepth(nodePath, 1)), "=", 0).WithPrefix(),
|
||||
).Then(
|
||||
clientv3.OpDelete(s.mkPath(nodePath)+"/", clientv3.WithPrevKV()),
|
||||
clientv3.OpPut(s.mkActionKey(), store.Delete),
|
||||
clientv3.OpPut(s.mkActionKey(), v2store.Delete),
|
||||
).Commit()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@ -430,19 +430,19 @@ func (s *v2v3Store) deleteEmptyDir(nodePath string) (*store.Event, error) {
|
||||
if len(dresp.PrevKvs) == 0 {
|
||||
return nil, etcdErr.NewError(etcdErr.EcodeNodeExist, nodePath, mkV2Rev(resp.Header.Revision))
|
||||
}
|
||||
return &store.Event{
|
||||
Action: store.Delete,
|
||||
return &v2store.Event{
|
||||
Action: v2store.Delete,
|
||||
PrevNode: s.mkV2Node(dresp.PrevKvs[0]),
|
||||
EtcdIndex: mkV2Rev(resp.Header.Revision),
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (s *v2v3Store) deleteNode(nodePath string) (*store.Event, error) {
|
||||
func (s *v2v3Store) deleteNode(nodePath string) (*v2store.Event, error) {
|
||||
resp, err := s.c.Txn(s.ctx).If(
|
||||
clientv3.Compare(clientv3.Version(s.mkPath(nodePath)+"/"), "=", 0),
|
||||
).Then(
|
||||
clientv3.OpDelete(s.mkPath(nodePath), clientv3.WithPrevKV()),
|
||||
clientv3.OpPut(s.mkActionKey(), store.Delete),
|
||||
clientv3.OpPut(s.mkActionKey(), v2store.Delete),
|
||||
).Commit()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@ -455,9 +455,9 @@ func (s *v2v3Store) deleteNode(nodePath string) (*store.Event, error) {
|
||||
return nil, etcdErr.NewError(etcdErr.EcodeKeyNotFound, nodePath, mkV2Rev(resp.Header.Revision))
|
||||
}
|
||||
pkv := pkvs[0]
|
||||
return &store.Event{
|
||||
Action: store.Delete,
|
||||
Node: &store.NodeExtern{
|
||||
return &v2store.Event{
|
||||
Action: v2store.Delete,
|
||||
Node: &v2store.NodeExtern{
|
||||
Key: nodePath,
|
||||
CreatedIndex: mkV2Rev(pkv.CreateRevision),
|
||||
ModifiedIndex: mkV2Rev(resp.Header.Revision),
|
||||
@ -467,7 +467,7 @@ func (s *v2v3Store) deleteNode(nodePath string) (*store.Event, error) {
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (s *v2v3Store) CompareAndDelete(nodePath, prevValue string, prevIndex uint64) (*store.Event, error) {
|
||||
func (s *v2v3Store) CompareAndDelete(nodePath, prevValue string, prevIndex uint64) (*v2store.Event, error) {
|
||||
if isRoot(nodePath) {
|
||||
return nil, etcdErr.NewError(etcdErr.EcodeRootROnly, nodePath, 0)
|
||||
}
|
||||
@ -477,7 +477,7 @@ func (s *v2v3Store) CompareAndDelete(nodePath, prevValue string, prevIndex uint6
|
||||
s.mkCompare(nodePath, prevValue, prevIndex)...,
|
||||
).Then(
|
||||
clientv3.OpDelete(key, clientv3.WithPrevKV()),
|
||||
clientv3.OpPut(s.mkActionKey(), store.CompareAndDelete),
|
||||
clientv3.OpPut(s.mkActionKey(), v2store.CompareAndDelete),
|
||||
).Else(
|
||||
clientv3.OpGet(key),
|
||||
clientv3.OpGet(key+"/"),
|
||||
@ -492,9 +492,9 @@ func (s *v2v3Store) CompareAndDelete(nodePath, prevValue string, prevIndex uint6
|
||||
|
||||
// len(pkvs) > 1 since txn only succeeds when key exists
|
||||
pkv := resp.Responses[0].GetResponseDeleteRange().PrevKvs[0]
|
||||
return &store.Event{
|
||||
Action: store.CompareAndDelete,
|
||||
Node: &store.NodeExtern{
|
||||
return &v2store.Event{
|
||||
Action: v2store.CompareAndDelete,
|
||||
Node: &v2store.NodeExtern{
|
||||
Key: nodePath,
|
||||
CreatedIndex: mkV2Rev(pkv.CreateRevision),
|
||||
ModifiedIndex: mkV2Rev(resp.Header.Revision),
|
||||
@ -548,7 +548,7 @@ func (s *v2v3Store) Version() int { return 2 }
|
||||
|
||||
func (s *v2v3Store) Save() ([]byte, error) { panic("STUB") }
|
||||
func (s *v2v3Store) Recovery(state []byte) error { panic("STUB") }
|
||||
func (s *v2v3Store) Clone() store.Store { panic("STUB") }
|
||||
func (s *v2v3Store) Clone() v2store.Store { panic("STUB") }
|
||||
func (s *v2v3Store) SaveNoCopy() ([]byte, error) { panic("STUB") }
|
||||
func (s *v2v3Store) HasTTLKeys() bool { panic("STUB") }
|
||||
|
||||
@ -586,11 +586,11 @@ func mkV3Rev(v2Rev uint64) int64 {
|
||||
}
|
||||
|
||||
// mkV2Node creates a V2 NodeExtern from a V3 KeyValue
|
||||
func (s *v2v3Store) mkV2Node(kv *mvccpb.KeyValue) *store.NodeExtern {
|
||||
func (s *v2v3Store) mkV2Node(kv *mvccpb.KeyValue) *v2store.NodeExtern {
|
||||
if kv == nil {
|
||||
return nil
|
||||
}
|
||||
n := &store.NodeExtern{
|
||||
n := &v2store.NodeExtern{
|
||||
Key: string(s.mkNodePath(string(kv.Key))),
|
||||
Dir: kv.Key[len(kv.Key)-1] == '/',
|
||||
CreatedIndex: mkV2Rev(kv.CreateRevision),
|
||||
|
@ -20,10 +20,10 @@ import (
|
||||
|
||||
"github.com/coreos/etcd/clientv3"
|
||||
etcdErr "github.com/coreos/etcd/error"
|
||||
"github.com/coreos/etcd/internal/store"
|
||||
"github.com/coreos/etcd/etcdserver/v2store"
|
||||
)
|
||||
|
||||
func (s *v2v3Store) Watch(prefix string, recursive, stream bool, sinceIndex uint64) (store.Watcher, error) {
|
||||
func (s *v2v3Store) Watch(prefix string, recursive, stream bool, sinceIndex uint64) (v2store.Watcher, error) {
|
||||
ctx, cancel := context.WithCancel(s.ctx)
|
||||
wch := s.c.Watch(
|
||||
ctx,
|
||||
@ -39,7 +39,7 @@ func (s *v2v3Store) Watch(prefix string, recursive, stream bool, sinceIndex uint
|
||||
return nil, etcdErr.NewError(etcdErr.EcodeRaftInternal, prefix, 0)
|
||||
}
|
||||
|
||||
evc, donec := make(chan *store.Event), make(chan struct{})
|
||||
evc, donec := make(chan *v2store.Event), make(chan struct{})
|
||||
go func() {
|
||||
defer func() {
|
||||
close(evc)
|
||||
@ -82,7 +82,7 @@ func (s *v2v3Store) Watch(prefix string, recursive, stream bool, sinceIndex uint
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (s *v2v3Store) mkV2Events(wr clientv3.WatchResponse) (evs []*store.Event) {
|
||||
func (s *v2v3Store) mkV2Events(wr clientv3.WatchResponse) (evs []*v2store.Event) {
|
||||
ak := s.mkActionKey()
|
||||
for _, rev := range mkRevs(wr) {
|
||||
var act, key *clientv3.Event
|
||||
@ -97,7 +97,7 @@ func (s *v2v3Store) mkV2Events(wr clientv3.WatchResponse) (evs []*store.Event) {
|
||||
key = ev
|
||||
}
|
||||
}
|
||||
v2ev := &store.Event{
|
||||
v2ev := &v2store.Event{
|
||||
Action: string(act.Kv.Value),
|
||||
Node: s.mkV2Node(key.Kv),
|
||||
PrevNode: s.mkV2Node(key.PrevKv),
|
||||
@ -125,7 +125,7 @@ func mkRevs(wr clientv3.WatchResponse) (revs [][]*clientv3.Event) {
|
||||
|
||||
type v2v3Watcher struct {
|
||||
startRev int64
|
||||
evc chan *store.Event
|
||||
evc chan *v2store.Event
|
||||
donec chan struct{}
|
||||
cancel context.CancelFunc
|
||||
}
|
||||
@ -137,4 +137,4 @@ func (w *v2v3Watcher) Remove() {
|
||||
<-w.donec
|
||||
}
|
||||
|
||||
func (w *v2v3Watcher) EventChan() chan *store.Event { return w.evc }
|
||||
func (w *v2v3Watcher) EventChan() chan *v2store.Event { return w.evc }
|
||||
|
@ -21,8 +21,9 @@ import (
|
||||
|
||||
"github.com/coreos/etcd/etcdserver/api"
|
||||
"github.com/coreos/etcd/etcdserver/membership"
|
||||
"github.com/coreos/etcd/internal/store"
|
||||
"github.com/coreos/etcd/etcdserver/v2store"
|
||||
"github.com/coreos/etcd/pkg/pbutil"
|
||||
|
||||
"github.com/coreos/go-semver/semver"
|
||||
)
|
||||
|
||||
@ -35,12 +36,12 @@ type ApplierV2 interface {
|
||||
Sync(r *RequestV2) Response
|
||||
}
|
||||
|
||||
func NewApplierV2(s store.Store, c *membership.RaftCluster) ApplierV2 {
|
||||
func NewApplierV2(s v2store.Store, c *membership.RaftCluster) ApplierV2 {
|
||||
return &applierV2store{store: s, cluster: c}
|
||||
}
|
||||
|
||||
type applierV2store struct {
|
||||
store store.Store
|
||||
store v2store.Store
|
||||
cluster *membership.RaftCluster
|
||||
}
|
||||
|
||||
@ -104,8 +105,8 @@ func (a *applierV2store) Sync(r *RequestV2) Response {
|
||||
return Response{}
|
||||
}
|
||||
|
||||
// applyV2Request interprets r as a call to store.X and returns a Response interpreted
|
||||
// from store.Event
|
||||
// applyV2Request interprets r as a call to v2store.X
|
||||
// and returns a Response interpreted from v2store.Event
|
||||
func (s *EtcdServer) applyV2Request(r *RequestV2) Response {
|
||||
defer warnOfExpensiveRequest(time.Now(), r)
|
||||
|
||||
@ -126,15 +127,15 @@ func (s *EtcdServer) applyV2Request(r *RequestV2) Response {
|
||||
}
|
||||
}
|
||||
|
||||
func (r *RequestV2) TTLOptions() store.TTLOptionSet {
|
||||
func (r *RequestV2) TTLOptions() v2store.TTLOptionSet {
|
||||
refresh, _ := pbutil.GetBool(r.Refresh)
|
||||
ttlOptions := store.TTLOptionSet{Refresh: refresh}
|
||||
ttlOptions := v2store.TTLOptionSet{Refresh: refresh}
|
||||
if r.Expiration != 0 {
|
||||
ttlOptions.ExpireTime = time.Unix(0, r.Expiration)
|
||||
}
|
||||
return ttlOptions
|
||||
}
|
||||
|
||||
func toResponse(ev *store.Event, err error) Response {
|
||||
func toResponse(ev *v2store.Event, err error) Response {
|
||||
return Response{Event: ev, Err: err}
|
||||
}
|
||||
|
@ -27,8 +27,8 @@ import (
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/coreos/etcd/etcdserver/v2store"
|
||||
"github.com/coreos/etcd/internal/mvcc/backend"
|
||||
"github.com/coreos/etcd/internal/store"
|
||||
"github.com/coreos/etcd/pkg/netutil"
|
||||
"github.com/coreos/etcd/pkg/types"
|
||||
"github.com/coreos/etcd/raft"
|
||||
@ -43,8 +43,8 @@ type RaftCluster struct {
|
||||
id types.ID
|
||||
token string
|
||||
|
||||
store store.Store
|
||||
be backend.Backend
|
||||
v2store v2store.Store
|
||||
be backend.Backend
|
||||
|
||||
sync.Mutex // guards the fields below
|
||||
version *semver.Version
|
||||
@ -196,7 +196,7 @@ func (c *RaftCluster) genID() {
|
||||
|
||||
func (c *RaftCluster) SetID(id types.ID) { c.id = id }
|
||||
|
||||
func (c *RaftCluster) SetStore(st store.Store) { c.store = st }
|
||||
func (c *RaftCluster) SetStore(st v2store.Store) { c.v2store = st }
|
||||
|
||||
func (c *RaftCluster) SetBackend(be backend.Backend) {
|
||||
c.be = be
|
||||
@ -207,8 +207,8 @@ func (c *RaftCluster) Recover(onSet func(*semver.Version)) {
|
||||
c.Lock()
|
||||
defer c.Unlock()
|
||||
|
||||
c.members, c.removed = membersFromStore(c.store)
|
||||
c.version = clusterVersionFromStore(c.store)
|
||||
c.members, c.removed = membersFromStore(c.v2store)
|
||||
c.version = clusterVersionFromStore(c.v2store)
|
||||
mustDetectDowngrade(c.version)
|
||||
onSet(c.version)
|
||||
|
||||
@ -223,7 +223,7 @@ func (c *RaftCluster) Recover(onSet func(*semver.Version)) {
|
||||
// ValidateConfigurationChange takes a proposed ConfChange and
|
||||
// ensures that it is still valid.
|
||||
func (c *RaftCluster) ValidateConfigurationChange(cc raftpb.ConfChange) error {
|
||||
members, removed := membersFromStore(c.store)
|
||||
members, removed := membersFromStore(c.v2store)
|
||||
id := types.ID(cc.NodeID)
|
||||
if removed[id] {
|
||||
return ErrIDRemoved
|
||||
@ -286,8 +286,8 @@ func (c *RaftCluster) ValidateConfigurationChange(cc raftpb.ConfChange) error {
|
||||
func (c *RaftCluster) AddMember(m *Member) {
|
||||
c.Lock()
|
||||
defer c.Unlock()
|
||||
if c.store != nil {
|
||||
mustSaveMemberToStore(c.store, m)
|
||||
if c.v2store != nil {
|
||||
mustSaveMemberToStore(c.v2store, m)
|
||||
}
|
||||
if c.be != nil {
|
||||
mustSaveMemberToBackend(c.be, m)
|
||||
@ -303,8 +303,8 @@ func (c *RaftCluster) AddMember(m *Member) {
|
||||
func (c *RaftCluster) RemoveMember(id types.ID) {
|
||||
c.Lock()
|
||||
defer c.Unlock()
|
||||
if c.store != nil {
|
||||
mustDeleteMemberFromStore(c.store, id)
|
||||
if c.v2store != nil {
|
||||
mustDeleteMemberFromStore(c.v2store, id)
|
||||
}
|
||||
if c.be != nil {
|
||||
mustDeleteMemberFromBackend(c.be, id)
|
||||
@ -321,8 +321,8 @@ func (c *RaftCluster) UpdateAttributes(id types.ID, attr Attributes) {
|
||||
defer c.Unlock()
|
||||
if m, ok := c.members[id]; ok {
|
||||
m.Attributes = attr
|
||||
if c.store != nil {
|
||||
mustUpdateMemberAttrInStore(c.store, m)
|
||||
if c.v2store != nil {
|
||||
mustUpdateMemberAttrInStore(c.v2store, m)
|
||||
}
|
||||
if c.be != nil {
|
||||
mustSaveMemberToBackend(c.be, m)
|
||||
@ -341,8 +341,8 @@ func (c *RaftCluster) UpdateRaftAttributes(id types.ID, raftAttr RaftAttributes)
|
||||
defer c.Unlock()
|
||||
|
||||
c.members[id].RaftAttributes = raftAttr
|
||||
if c.store != nil {
|
||||
mustUpdateMemberInStore(c.store, c.members[id])
|
||||
if c.v2store != nil {
|
||||
mustUpdateMemberInStore(c.v2store, c.members[id])
|
||||
}
|
||||
if c.be != nil {
|
||||
mustSaveMemberToBackend(c.be, c.members[id])
|
||||
@ -370,8 +370,8 @@ func (c *RaftCluster) SetVersion(ver *semver.Version, onSet func(*semver.Version
|
||||
}
|
||||
c.version = ver
|
||||
mustDetectDowngrade(c.version)
|
||||
if c.store != nil {
|
||||
mustSaveClusterVersionToStore(c.store, ver)
|
||||
if c.v2store != nil {
|
||||
mustSaveClusterVersionToStore(c.v2store, ver)
|
||||
}
|
||||
if c.be != nil {
|
||||
mustSaveClusterVersionToBackend(c.be, ver)
|
||||
@ -431,7 +431,7 @@ func (c *RaftCluster) IsReadyToRemoveMember(id uint64) bool {
|
||||
return true
|
||||
}
|
||||
|
||||
func membersFromStore(st store.Store) (map[types.ID]*Member, map[types.ID]bool) {
|
||||
func membersFromStore(st v2store.Store) (map[types.ID]*Member, map[types.ID]bool) {
|
||||
members := make(map[types.ID]*Member)
|
||||
removed := make(map[types.ID]bool)
|
||||
e, err := st.Get(StoreMembersPrefix, true, true)
|
||||
@ -463,7 +463,7 @@ func membersFromStore(st store.Store) (map[types.ID]*Member, map[types.ID]bool)
|
||||
return members, removed
|
||||
}
|
||||
|
||||
func clusterVersionFromStore(st store.Store) *semver.Version {
|
||||
func clusterVersionFromStore(st v2store.Store) *semver.Version {
|
||||
e, err := st.Get(path.Join(storePrefix, "version"), false, false)
|
||||
if err != nil {
|
||||
if isKeyNotFound(err) {
|
||||
|
@ -21,7 +21,7 @@ import (
|
||||
"reflect"
|
||||
"testing"
|
||||
|
||||
"github.com/coreos/etcd/internal/store"
|
||||
"github.com/coreos/etcd/etcdserver/v2store"
|
||||
"github.com/coreos/etcd/pkg/mock/mockstore"
|
||||
"github.com/coreos/etcd/pkg/testutil"
|
||||
"github.com/coreos/etcd/pkg/types"
|
||||
@ -275,7 +275,7 @@ func TestClusterValidateAndAssignIDs(t *testing.T) {
|
||||
|
||||
func TestClusterValidateConfigurationChange(t *testing.T) {
|
||||
cl := NewCluster("")
|
||||
cl.SetStore(store.New())
|
||||
cl.SetStore(v2store.New())
|
||||
for i := 1; i <= 4; i++ {
|
||||
attr := RaftAttributes{PeerURLs: []string{fmt.Sprintf("http://127.0.0.1:%d", i)}}
|
||||
cl.AddMember(&Member{ID: types.ID(i), RaftAttributes: attr})
|
||||
@ -416,25 +416,25 @@ func TestClusterGenID(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestNodeToMemberBad(t *testing.T) {
|
||||
tests := []*store.NodeExtern{
|
||||
{Key: "/1234", Nodes: []*store.NodeExtern{
|
||||
tests := []*v2store.NodeExtern{
|
||||
{Key: "/1234", Nodes: []*v2store.NodeExtern{
|
||||
{Key: "/1234/strange"},
|
||||
}},
|
||||
{Key: "/1234", Nodes: []*store.NodeExtern{
|
||||
{Key: "/1234", Nodes: []*v2store.NodeExtern{
|
||||
{Key: "/1234/raftAttributes", Value: stringp("garbage")},
|
||||
}},
|
||||
{Key: "/1234", Nodes: []*store.NodeExtern{
|
||||
{Key: "/1234", Nodes: []*v2store.NodeExtern{
|
||||
{Key: "/1234/attributes", Value: stringp(`{"name":"node1","clientURLs":null}`)},
|
||||
}},
|
||||
{Key: "/1234", Nodes: []*store.NodeExtern{
|
||||
{Key: "/1234", Nodes: []*v2store.NodeExtern{
|
||||
{Key: "/1234/raftAttributes", Value: stringp(`{"peerURLs":null}`)},
|
||||
{Key: "/1234/strange"},
|
||||
}},
|
||||
{Key: "/1234", Nodes: []*store.NodeExtern{
|
||||
{Key: "/1234", Nodes: []*v2store.NodeExtern{
|
||||
{Key: "/1234/raftAttributes", Value: stringp(`{"peerURLs":null}`)},
|
||||
{Key: "/1234/attributes", Value: stringp("garbage")},
|
||||
}},
|
||||
{Key: "/1234", Nodes: []*store.NodeExtern{
|
||||
{Key: "/1234", Nodes: []*v2store.NodeExtern{
|
||||
{Key: "/1234/raftAttributes", Value: stringp(`{"peerURLs":null}`)},
|
||||
{Key: "/1234/attributes", Value: stringp(`{"name":"node1","clientURLs":null}`)},
|
||||
{Key: "/1234/strange"},
|
||||
@ -461,7 +461,7 @@ func TestClusterAddMember(t *testing.T) {
|
||||
false,
|
||||
`{"peerURLs":null}`,
|
||||
false,
|
||||
store.TTLOptionSet{ExpireTime: store.Permanent},
|
||||
v2store.TTLOptionSet{ExpireTime: v2store.Permanent},
|
||||
},
|
||||
},
|
||||
}
|
||||
@ -500,7 +500,7 @@ func TestClusterRemoveMember(t *testing.T) {
|
||||
|
||||
wactions := []testutil.Action{
|
||||
{Name: "Delete", Params: []interface{}{MemberStoreKey(1), true, true}},
|
||||
{Name: "Create", Params: []interface{}{RemovedMemberStoreKey(1), false, "", false, store.TTLOptionSet{ExpireTime: store.Permanent}}},
|
||||
{Name: "Create", Params: []interface{}{RemovedMemberStoreKey(1), false, "", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent}}},
|
||||
}
|
||||
if !reflect.DeepEqual(st.Action(), wactions) {
|
||||
t.Errorf("actions = %v, want %v", st.Action(), wactions)
|
||||
@ -544,7 +544,7 @@ func TestClusterUpdateAttributes(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestNodeToMember(t *testing.T) {
|
||||
n := &store.NodeExtern{Key: "/1234", Nodes: []*store.NodeExtern{
|
||||
n := &store.NodeExtern{Key: "/1234", Nodes: []*v2store.NodeExtern{
|
||||
{Key: "/1234/attributes", Value: stringp(`{"name":"node1","clientURLs":null}`)},
|
||||
{Key: "/1234/raftAttributes", Value: stringp(`{"peerURLs":null}`)},
|
||||
}}
|
||||
|
@ -19,8 +19,8 @@ import (
|
||||
"fmt"
|
||||
"path"
|
||||
|
||||
"github.com/coreos/etcd/etcdserver/v2store"
|
||||
"github.com/coreos/etcd/internal/mvcc/backend"
|
||||
"github.com/coreos/etcd/internal/store"
|
||||
"github.com/coreos/etcd/pkg/types"
|
||||
|
||||
"github.com/coreos/go-semver/semver"
|
||||
@ -75,57 +75,57 @@ func mustSaveClusterVersionToBackend(be backend.Backend, ver *semver.Version) {
|
||||
tx.UnsafePut(clusterBucketName, ckey, []byte(ver.String()))
|
||||
}
|
||||
|
||||
func mustSaveMemberToStore(s store.Store, m *Member) {
|
||||
func mustSaveMemberToStore(s v2store.Store, m *Member) {
|
||||
b, err := json.Marshal(m.RaftAttributes)
|
||||
if err != nil {
|
||||
plog.Panicf("marshal raftAttributes should never fail: %v", err)
|
||||
}
|
||||
p := path.Join(MemberStoreKey(m.ID), raftAttributesSuffix)
|
||||
if _, err := s.Create(p, false, string(b), false, store.TTLOptionSet{ExpireTime: store.Permanent}); err != nil {
|
||||
if _, err := s.Create(p, false, string(b), false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent}); err != nil {
|
||||
plog.Panicf("create raftAttributes should never fail: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
func mustDeleteMemberFromStore(s store.Store, id types.ID) {
|
||||
func mustDeleteMemberFromStore(s v2store.Store, id types.ID) {
|
||||
if _, err := s.Delete(MemberStoreKey(id), true, true); err != nil {
|
||||
plog.Panicf("delete member should never fail: %v", err)
|
||||
}
|
||||
if _, err := s.Create(RemovedMemberStoreKey(id), false, "", false, store.TTLOptionSet{ExpireTime: store.Permanent}); err != nil {
|
||||
if _, err := s.Create(RemovedMemberStoreKey(id), false, "", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent}); err != nil {
|
||||
plog.Panicf("create removedMember should never fail: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
func mustUpdateMemberInStore(s store.Store, m *Member) {
|
||||
func mustUpdateMemberInStore(s v2store.Store, m *Member) {
|
||||
b, err := json.Marshal(m.RaftAttributes)
|
||||
if err != nil {
|
||||
plog.Panicf("marshal raftAttributes should never fail: %v", err)
|
||||
}
|
||||
p := path.Join(MemberStoreKey(m.ID), raftAttributesSuffix)
|
||||
if _, err := s.Update(p, string(b), store.TTLOptionSet{ExpireTime: store.Permanent}); err != nil {
|
||||
if _, err := s.Update(p, string(b), v2store.TTLOptionSet{ExpireTime: v2store.Permanent}); err != nil {
|
||||
plog.Panicf("update raftAttributes should never fail: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
func mustUpdateMemberAttrInStore(s store.Store, m *Member) {
|
||||
func mustUpdateMemberAttrInStore(s v2store.Store, m *Member) {
|
||||
b, err := json.Marshal(m.Attributes)
|
||||
if err != nil {
|
||||
plog.Panicf("marshal raftAttributes should never fail: %v", err)
|
||||
}
|
||||
p := path.Join(MemberStoreKey(m.ID), attributesSuffix)
|
||||
if _, err := s.Set(p, false, string(b), store.TTLOptionSet{ExpireTime: store.Permanent}); err != nil {
|
||||
if _, err := s.Set(p, false, string(b), v2store.TTLOptionSet{ExpireTime: v2store.Permanent}); err != nil {
|
||||
plog.Panicf("update raftAttributes should never fail: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
func mustSaveClusterVersionToStore(s store.Store, ver *semver.Version) {
|
||||
if _, err := s.Set(StoreClusterVersionKey(), false, ver.String(), store.TTLOptionSet{ExpireTime: store.Permanent}); err != nil {
|
||||
func mustSaveClusterVersionToStore(s v2store.Store, ver *semver.Version) {
|
||||
if _, err := s.Set(StoreClusterVersionKey(), false, ver.String(), v2store.TTLOptionSet{ExpireTime: v2store.Permanent}); err != nil {
|
||||
plog.Panicf("save cluster version should never fail: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
// nodeToMember builds member from a key value node.
|
||||
// the child nodes of the given node MUST be sorted by key.
|
||||
func nodeToMember(n *store.NodeExtern) (*Member, error) {
|
||||
func nodeToMember(n *v2store.NodeExtern) (*Member, error) {
|
||||
m := &Member{ID: MustParseMemberIDFromKey(n.Key)}
|
||||
attrs := make(map[string][]byte)
|
||||
raftAttrKey := path.Join(n.Key, raftAttributesSuffix)
|
||||
|
@ -38,10 +38,10 @@ import (
|
||||
pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
|
||||
"github.com/coreos/etcd/etcdserver/membership"
|
||||
"github.com/coreos/etcd/etcdserver/stats"
|
||||
"github.com/coreos/etcd/etcdserver/v2store"
|
||||
"github.com/coreos/etcd/internal/mvcc"
|
||||
"github.com/coreos/etcd/internal/mvcc/backend"
|
||||
"github.com/coreos/etcd/internal/raftsnap"
|
||||
"github.com/coreos/etcd/internal/store"
|
||||
"github.com/coreos/etcd/lease"
|
||||
"github.com/coreos/etcd/lease/leasehttp"
|
||||
"github.com/coreos/etcd/pkg/fileutil"
|
||||
@ -112,8 +112,8 @@ func init() {
|
||||
type Response struct {
|
||||
Term uint64
|
||||
Index uint64
|
||||
Event *store.Event
|
||||
Watcher store.Watcher
|
||||
Event *v2store.Event
|
||||
Watcher v2store.Watcher
|
||||
Err error
|
||||
}
|
||||
|
||||
@ -205,7 +205,7 @@ type EtcdServer struct {
|
||||
|
||||
cluster *membership.RaftCluster
|
||||
|
||||
store store.Store
|
||||
v2store v2store.Store
|
||||
snapshotter *raftsnap.Snapshotter
|
||||
|
||||
applyV2 ApplierV2
|
||||
@ -256,7 +256,7 @@ type EtcdServer struct {
|
||||
// NewServer creates a new EtcdServer from the supplied configuration. The
|
||||
// configuration is considered static for the lifetime of the EtcdServer.
|
||||
func NewServer(cfg ServerConfig) (srv *EtcdServer, err error) {
|
||||
st := store.New(StoreClusterPrefix, StoreKeysPrefix)
|
||||
st := v2store.New(StoreClusterPrefix, StoreKeysPrefix)
|
||||
|
||||
var (
|
||||
w *wal.WAL
|
||||
@ -414,7 +414,7 @@ func NewServer(cfg ServerConfig) (srv *EtcdServer, err error) {
|
||||
readych: make(chan struct{}),
|
||||
Cfg: cfg,
|
||||
errorc: make(chan error, 1),
|
||||
store: st,
|
||||
v2store: st,
|
||||
snapshotter: ss,
|
||||
r: *newRaftNode(
|
||||
raftNodeConfig{
|
||||
@ -436,7 +436,7 @@ func NewServer(cfg ServerConfig) (srv *EtcdServer, err error) {
|
||||
forceVersionC: make(chan struct{}),
|
||||
}
|
||||
|
||||
srv.applyV2 = &applierV2store{store: srv.store, cluster: srv.cluster}
|
||||
srv.applyV2 = &applierV2store{store: srv.v2store, cluster: srv.cluster}
|
||||
|
||||
srv.be = be
|
||||
minTTL := time.Duration((3*cfg.ElectionTicks)/2) * heartbeat
|
||||
@ -783,7 +783,7 @@ func (s *EtcdServer) run() {
|
||||
plog.Infof("the data-dir used by this member must be removed.")
|
||||
return
|
||||
case <-getSyncC():
|
||||
if s.store.HasTTLKeys() {
|
||||
if s.v2store.HasTTLKeys() {
|
||||
s.sync(s.Cfg.ReqTimeout())
|
||||
}
|
||||
case <-s.stop:
|
||||
@ -881,7 +881,7 @@ func (s *EtcdServer) applySnapshot(ep *etcdProgress, apply *apply) {
|
||||
}
|
||||
|
||||
plog.Info("recovering store v2...")
|
||||
if err := s.store.Recovery(apply.snapshot.Data); err != nil {
|
||||
if err := s.v2store.Recovery(apply.snapshot.Data); err != nil {
|
||||
plog.Panicf("recovery store error: %v", err)
|
||||
}
|
||||
plog.Info("finished recovering store v2")
|
||||
@ -1047,7 +1047,7 @@ func (s *EtcdServer) LeaderStats() []byte {
|
||||
return s.lstats.JSON()
|
||||
}
|
||||
|
||||
func (s *EtcdServer) StoreStats() []byte { return s.store.JsonStats() }
|
||||
func (s *EtcdServer) StoreStats() []byte { return s.v2store.JsonStats() }
|
||||
|
||||
func (s *EtcdServer) checkMembershipOperationPermission(ctx context.Context) error {
|
||||
if s.authStore == nil {
|
||||
@ -1441,7 +1441,7 @@ func (s *EtcdServer) applyConfChange(cc raftpb.ConfChange, confState *raftpb.Con
|
||||
|
||||
// TODO: non-blocking snapshot
|
||||
func (s *EtcdServer) snapshot(snapi uint64, confState raftpb.ConfState) {
|
||||
clone := s.store.Clone()
|
||||
clone := s.v2store.Clone()
|
||||
// commit kv to write metadata (for example: consistent index) to disk.
|
||||
// KV().commit() updates the consistent index in backend.
|
||||
// All operations that update consistent index must be called sequentially
|
||||
|
@ -28,6 +28,7 @@ import (
|
||||
|
||||
pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
|
||||
"github.com/coreos/etcd/etcdserver/membership"
|
||||
"github.com/coreos/etcd/etcdserver/v2store"
|
||||
"github.com/coreos/etcd/internal/mvcc"
|
||||
"github.com/coreos/etcd/internal/mvcc/backend"
|
||||
"github.com/coreos/etcd/internal/raftsnap"
|
||||
@ -59,11 +60,11 @@ func TestDoLocalAction(t *testing.T) {
|
||||
}{
|
||||
{
|
||||
pb.Request{Method: "GET", ID: 1, Wait: true},
|
||||
Response{Watcher: store.NewNopWatcher()}, nil, []testutil.Action{{Name: "Watch"}},
|
||||
Response{Watcher: v2store.NewNopWatcher()}, nil, []testutil.Action{{Name: "Watch"}},
|
||||
},
|
||||
{
|
||||
pb.Request{Method: "GET", ID: 1},
|
||||
Response{Event: &store.Event{}}, nil,
|
||||
Response{Event: &v2store.Event{}}, nil,
|
||||
[]testutil.Action{
|
||||
{
|
||||
Name: "Get",
|
||||
@ -73,7 +74,7 @@ func TestDoLocalAction(t *testing.T) {
|
||||
},
|
||||
{
|
||||
pb.Request{Method: "HEAD", ID: 1},
|
||||
Response{Event: &store.Event{}}, nil,
|
||||
Response{Event: &v2store.Event{}}, nil,
|
||||
[]testutil.Action{
|
||||
{
|
||||
Name: "Get",
|
||||
@ -167,8 +168,8 @@ func TestApplyRepeat(t *testing.T) {
|
||||
SoftState: &raft.SoftState{RaftState: raft.StateLeader},
|
||||
}
|
||||
cl := newTestCluster(nil)
|
||||
st := store.New()
|
||||
cl.SetStore(store.New())
|
||||
st := v2store.New()
|
||||
cl.SetStore(v2store.New())
|
||||
cl.AddMember(&membership.Member{ID: 1234})
|
||||
r := newRaftNode(raftNodeConfig{
|
||||
Node: n,
|
||||
@ -233,139 +234,139 @@ func TestApplyRequest(t *testing.T) {
|
||||
// POST ==> Create
|
||||
{
|
||||
pb.Request{Method: "POST", ID: 1},
|
||||
Response{Event: &store.Event{}},
|
||||
Response{Event: &v2store.Event{}},
|
||||
[]testutil.Action{
|
||||
{
|
||||
Name: "Create",
|
||||
Params: []interface{}{"", false, "", true, store.TTLOptionSet{ExpireTime: time.Time{}}},
|
||||
Params: []interface{}{"", false, "", true, v2store.TTLOptionSet{ExpireTime: time.Time{}}},
|
||||
},
|
||||
},
|
||||
},
|
||||
// POST ==> Create, with expiration
|
||||
{
|
||||
pb.Request{Method: "POST", ID: 1, Expiration: 1337},
|
||||
Response{Event: &store.Event{}},
|
||||
Response{Event: &v2store.Event{}},
|
||||
[]testutil.Action{
|
||||
{
|
||||
Name: "Create",
|
||||
Params: []interface{}{"", false, "", true, store.TTLOptionSet{ExpireTime: time.Unix(0, 1337)}},
|
||||
Params: []interface{}{"", false, "", true, v2store.TTLOptionSet{ExpireTime: time.Unix(0, 1337)}},
|
||||
},
|
||||
},
|
||||
},
|
||||
// POST ==> Create, with dir
|
||||
{
|
||||
pb.Request{Method: "POST", ID: 1, Dir: true},
|
||||
Response{Event: &store.Event{}},
|
||||
Response{Event: &v2store.Event{}},
|
||||
[]testutil.Action{
|
||||
{
|
||||
Name: "Create",
|
||||
Params: []interface{}{"", true, "", true, store.TTLOptionSet{ExpireTime: time.Time{}}},
|
||||
Params: []interface{}{"", true, "", true, v2store.TTLOptionSet{ExpireTime: time.Time{}}},
|
||||
},
|
||||
},
|
||||
},
|
||||
// PUT ==> Set
|
||||
{
|
||||
pb.Request{Method: "PUT", ID: 1},
|
||||
Response{Event: &store.Event{}},
|
||||
Response{Event: &v2store.Event{}},
|
||||
[]testutil.Action{
|
||||
{
|
||||
Name: "Set",
|
||||
Params: []interface{}{"", false, "", store.TTLOptionSet{ExpireTime: time.Time{}}},
|
||||
Params: []interface{}{"", false, "", v2store.TTLOptionSet{ExpireTime: time.Time{}}},
|
||||
},
|
||||
},
|
||||
},
|
||||
// PUT ==> Set, with dir
|
||||
{
|
||||
pb.Request{Method: "PUT", ID: 1, Dir: true},
|
||||
Response{Event: &store.Event{}},
|
||||
Response{Event: &v2store.Event{}},
|
||||
[]testutil.Action{
|
||||
{
|
||||
Name: "Set",
|
||||
Params: []interface{}{"", true, "", store.TTLOptionSet{ExpireTime: time.Time{}}},
|
||||
Params: []interface{}{"", true, "", v2store.TTLOptionSet{ExpireTime: time.Time{}}},
|
||||
},
|
||||
},
|
||||
},
|
||||
// PUT with PrevExist=true ==> Update
|
||||
{
|
||||
pb.Request{Method: "PUT", ID: 1, PrevExist: pbutil.Boolp(true)},
|
||||
Response{Event: &store.Event{}},
|
||||
Response{Event: &v2store.Event{}},
|
||||
[]testutil.Action{
|
||||
{
|
||||
Name: "Update",
|
||||
Params: []interface{}{"", "", store.TTLOptionSet{ExpireTime: time.Time{}}},
|
||||
Params: []interface{}{"", "", v2store.TTLOptionSet{ExpireTime: time.Time{}}},
|
||||
},
|
||||
},
|
||||
},
|
||||
// PUT with PrevExist=false ==> Create
|
||||
{
|
||||
pb.Request{Method: "PUT", ID: 1, PrevExist: pbutil.Boolp(false)},
|
||||
Response{Event: &store.Event{}},
|
||||
Response{Event: &v2store.Event{}},
|
||||
[]testutil.Action{
|
||||
{
|
||||
Name: "Create",
|
||||
Params: []interface{}{"", false, "", false, store.TTLOptionSet{ExpireTime: time.Time{}}},
|
||||
Params: []interface{}{"", false, "", false, v2store.TTLOptionSet{ExpireTime: time.Time{}}},
|
||||
},
|
||||
},
|
||||
},
|
||||
// PUT with PrevExist=true *and* PrevIndex set ==> CompareAndSwap
|
||||
{
|
||||
pb.Request{Method: "PUT", ID: 1, PrevExist: pbutil.Boolp(true), PrevIndex: 1},
|
||||
Response{Event: &store.Event{}},
|
||||
Response{Event: &v2store.Event{}},
|
||||
[]testutil.Action{
|
||||
{
|
||||
Name: "CompareAndSwap",
|
||||
Params: []interface{}{"", "", uint64(1), "", store.TTLOptionSet{ExpireTime: time.Time{}}},
|
||||
Params: []interface{}{"", "", uint64(1), "", v2store.TTLOptionSet{ExpireTime: time.Time{}}},
|
||||
},
|
||||
},
|
||||
},
|
||||
// PUT with PrevExist=false *and* PrevIndex set ==> Create
|
||||
{
|
||||
pb.Request{Method: "PUT", ID: 1, PrevExist: pbutil.Boolp(false), PrevIndex: 1},
|
||||
Response{Event: &store.Event{}},
|
||||
Response{Event: &v2store.Event{}},
|
||||
[]testutil.Action{
|
||||
{
|
||||
Name: "Create",
|
||||
Params: []interface{}{"", false, "", false, store.TTLOptionSet{ExpireTime: time.Time{}}},
|
||||
Params: []interface{}{"", false, "", false, v2store.TTLOptionSet{ExpireTime: time.Time{}}},
|
||||
},
|
||||
},
|
||||
},
|
||||
// PUT with PrevIndex set ==> CompareAndSwap
|
||||
{
|
||||
pb.Request{Method: "PUT", ID: 1, PrevIndex: 1},
|
||||
Response{Event: &store.Event{}},
|
||||
Response{Event: &v2store.Event{}},
|
||||
[]testutil.Action{
|
||||
{
|
||||
Name: "CompareAndSwap",
|
||||
Params: []interface{}{"", "", uint64(1), "", store.TTLOptionSet{ExpireTime: time.Time{}}},
|
||||
Params: []interface{}{"", "", uint64(1), "", v2store.TTLOptionSet{ExpireTime: time.Time{}}},
|
||||
},
|
||||
},
|
||||
},
|
||||
// PUT with PrevValue set ==> CompareAndSwap
|
||||
{
|
||||
pb.Request{Method: "PUT", ID: 1, PrevValue: "bar"},
|
||||
Response{Event: &store.Event{}},
|
||||
Response{Event: &v2store.Event{}},
|
||||
[]testutil.Action{
|
||||
{
|
||||
Name: "CompareAndSwap",
|
||||
Params: []interface{}{"", "bar", uint64(0), "", store.TTLOptionSet{ExpireTime: time.Time{}}},
|
||||
Params: []interface{}{"", "bar", uint64(0), "", v2store.TTLOptionSet{ExpireTime: time.Time{}}},
|
||||
},
|
||||
},
|
||||
},
|
||||
// PUT with PrevIndex and PrevValue set ==> CompareAndSwap
|
||||
{
|
||||
pb.Request{Method: "PUT", ID: 1, PrevIndex: 1, PrevValue: "bar"},
|
||||
Response{Event: &store.Event{}},
|
||||
Response{Event: &v2store.Event{}},
|
||||
[]testutil.Action{
|
||||
{
|
||||
Name: "CompareAndSwap",
|
||||
Params: []interface{}{"", "bar", uint64(1), "", store.TTLOptionSet{ExpireTime: time.Time{}}},
|
||||
Params: []interface{}{"", "bar", uint64(1), "", v2store.TTLOptionSet{ExpireTime: time.Time{}}},
|
||||
},
|
||||
},
|
||||
},
|
||||
// DELETE ==> Delete
|
||||
{
|
||||
pb.Request{Method: "DELETE", ID: 1},
|
||||
Response{Event: &store.Event{}},
|
||||
Response{Event: &v2store.Event{}},
|
||||
[]testutil.Action{
|
||||
{
|
||||
Name: "Delete",
|
||||
@ -376,7 +377,7 @@ func TestApplyRequest(t *testing.T) {
|
||||
// DELETE with PrevIndex set ==> CompareAndDelete
|
||||
{
|
||||
pb.Request{Method: "DELETE", ID: 1, PrevIndex: 1},
|
||||
Response{Event: &store.Event{}},
|
||||
Response{Event: &v2store.Event{}},
|
||||
[]testutil.Action{
|
||||
{
|
||||
Name: "CompareAndDelete",
|
||||
@ -387,7 +388,7 @@ func TestApplyRequest(t *testing.T) {
|
||||
// DELETE with PrevValue set ==> CompareAndDelete
|
||||
{
|
||||
pb.Request{Method: "DELETE", ID: 1, PrevValue: "bar"},
|
||||
Response{Event: &store.Event{}},
|
||||
Response{Event: &v2store.Event{}},
|
||||
[]testutil.Action{
|
||||
{
|
||||
Name: "CompareAndDelete",
|
||||
@ -398,7 +399,7 @@ func TestApplyRequest(t *testing.T) {
|
||||
// DELETE with PrevIndex *and* PrevValue set ==> CompareAndDelete
|
||||
{
|
||||
pb.Request{Method: "DELETE", ID: 1, PrevIndex: 5, PrevValue: "bar"},
|
||||
Response{Event: &store.Event{}},
|
||||
Response{Event: &v2store.Event{}},
|
||||
[]testutil.Action{
|
||||
{
|
||||
Name: "CompareAndDelete",
|
||||
@ -409,7 +410,7 @@ func TestApplyRequest(t *testing.T) {
|
||||
// QGET ==> Get
|
||||
{
|
||||
pb.Request{Method: "QGET", ID: 1},
|
||||
Response{Event: &store.Event{}},
|
||||
Response{Event: &v2store.Event{}},
|
||||
[]testutil.Action{
|
||||
{
|
||||
Name: "Get",
|
||||
@ -485,7 +486,7 @@ func TestApplyRequestOnAdminMemberAttributes(t *testing.T) {
|
||||
|
||||
func TestApplyConfChangeError(t *testing.T) {
|
||||
cl := membership.NewCluster("")
|
||||
cl.SetStore(store.New())
|
||||
cl.SetStore(v2store.New())
|
||||
for i := 1; i <= 4; i++ {
|
||||
cl.AddMember(&membership.Member{ID: types.ID(i)})
|
||||
}
|
||||
@ -549,7 +550,7 @@ func TestApplyConfChangeError(t *testing.T) {
|
||||
|
||||
func TestApplyConfChangeShouldStop(t *testing.T) {
|
||||
cl := membership.NewCluster("")
|
||||
cl.SetStore(store.New())
|
||||
cl.SetStore(v2store.New())
|
||||
for i := 1; i <= 3; i++ {
|
||||
cl.AddMember(&membership.Member{ID: types.ID(i)})
|
||||
}
|
||||
@ -590,7 +591,7 @@ func TestApplyConfChangeShouldStop(t *testing.T) {
|
||||
// where consistIndex equals to applied index.
|
||||
func TestApplyConfigChangeUpdatesConsistIndex(t *testing.T) {
|
||||
cl := membership.NewCluster("")
|
||||
cl.SetStore(store.New())
|
||||
cl.SetStore(v2store.New())
|
||||
cl.AddMember(&membership.Member{ID: types.ID(1)})
|
||||
r := newRaftNode(raftNodeConfig{
|
||||
Node: newNodeNop(),
|
||||
@ -633,7 +634,7 @@ func TestApplyConfigChangeUpdatesConsistIndex(t *testing.T) {
|
||||
// if the local member is removed along with other conf updates.
|
||||
func TestApplyMultiConfChangeShouldStop(t *testing.T) {
|
||||
cl := membership.NewCluster("")
|
||||
cl.SetStore(store.New())
|
||||
cl.SetStore(v2store.New())
|
||||
for i := 1; i <= 5; i++ {
|
||||
cl.AddMember(&membership.Member{ID: types.ID(i)})
|
||||
}
|
||||
@ -702,7 +703,7 @@ func TestDoProposal(t *testing.T) {
|
||||
t.Fatalf("#%d: err = %v, want nil", i, err)
|
||||
}
|
||||
// resp.Index is set in Do() based on the raft state; may either be 0 or 1
|
||||
wresp := Response{Event: &store.Event{}, Index: resp.Index}
|
||||
wresp := Response{Event: &v2store.Event{}, Index: resp.Index}
|
||||
if !reflect.DeepEqual(resp, wresp) {
|
||||
t.Errorf("#%d: resp = %v, want %v", i, resp, wresp)
|
||||
}
|
||||
@ -1224,7 +1225,7 @@ func TestRemoveMember(t *testing.T) {
|
||||
}
|
||||
cl := newTestCluster(nil)
|
||||
st := store.New()
|
||||
cl.SetStore(store.New())
|
||||
cl.SetStore(v2store.New())
|
||||
cl.AddMember(&membership.Member{ID: 1234})
|
||||
r := newRaftNode(raftNodeConfig{
|
||||
Node: n,
|
||||
|
@ -27,7 +27,7 @@ import (
|
||||
// as ReadCloser.
|
||||
func (s *EtcdServer) createMergedSnapshotMessage(m raftpb.Message, snapt, snapi uint64, confState raftpb.ConfState) raftsnap.Message {
|
||||
// get a snapshot of v2 store as []byte
|
||||
clone := s.store.Clone()
|
||||
clone := s.v2store.Clone()
|
||||
d, err := clone.SaveNoCopy()
|
||||
if err != nil {
|
||||
plog.Panicf("store save should never fail: %v", err)
|
||||
|
@ -19,7 +19,7 @@ import (
|
||||
"time"
|
||||
|
||||
pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
|
||||
"github.com/coreos/etcd/internal/store"
|
||||
"github.com/coreos/etcd/etcdserver/v2store"
|
||||
)
|
||||
|
||||
type RequestV2 pb.Request
|
||||
@ -39,11 +39,11 @@ type reqV2HandlerEtcdServer struct {
|
||||
}
|
||||
|
||||
type reqV2HandlerStore struct {
|
||||
store store.Store
|
||||
store v2store.Store
|
||||
applier ApplierV2
|
||||
}
|
||||
|
||||
func NewStoreRequestV2Handler(s store.Store, applier ApplierV2) RequestV2Handler {
|
||||
func NewStoreRequestV2Handler(s v2store.Store, applier ApplierV2) RequestV2Handler {
|
||||
return &reqV2HandlerStore{s, applier}
|
||||
}
|
||||
|
||||
@ -122,7 +122,7 @@ func (s *EtcdServer) Do(ctx context.Context, r pb.Request) (Response, error) {
|
||||
r.ID = s.reqIDGen.Next()
|
||||
h := &reqV2HandlerEtcdServer{
|
||||
reqV2HandlerStore: reqV2HandlerStore{
|
||||
store: s.store,
|
||||
store: s.v2store,
|
||||
applier: s.applyV2,
|
||||
},
|
||||
s: s,
|
||||
|
@ -23,7 +23,7 @@ import (
|
||||
etcderr "github.com/coreos/etcd/error"
|
||||
"github.com/coreos/etcd/etcdserver"
|
||||
"github.com/coreos/etcd/etcdserver/etcdserverpb"
|
||||
etcdstore "github.com/coreos/etcd/internal/store"
|
||||
"github.com/coreos/etcd/etcdserver/v2store"
|
||||
)
|
||||
|
||||
type fakeDoer struct{}
|
||||
@ -164,9 +164,9 @@ func (td *testDoer) Do(_ context.Context, req etcdserverpb.Request) (etcdserver.
|
||||
if td.explicitlyEnabled && (req.Path == StorePermsPrefix+"/enabled") {
|
||||
t := "true"
|
||||
return etcdserver.Response{
|
||||
Event: &etcdstore.Event{
|
||||
Action: etcdstore.Get,
|
||||
Node: &etcdstore.NodeExtern{
|
||||
Event: &v2store.Event{
|
||||
Action: v2store.Get,
|
||||
Node: &v2store.NodeExtern{
|
||||
Key: StorePermsPrefix + "/users/cat",
|
||||
Value: &t,
|
||||
},
|
||||
@ -202,10 +202,10 @@ func TestAllUsers(t *testing.T) {
|
||||
d := &testDoer{
|
||||
get: []etcdserver.Response{
|
||||
{
|
||||
Event: &etcdstore.Event{
|
||||
Action: etcdstore.Get,
|
||||
Node: &etcdstore.NodeExtern{
|
||||
Nodes: etcdstore.NodeExterns([]*etcdstore.NodeExtern{
|
||||
Event: &v2store.Event{
|
||||
Action: v2store.Get,
|
||||
Node: &v2store.NodeExtern{
|
||||
Nodes: v2store.NodeExterns([]*v2store.NodeExtern{
|
||||
{
|
||||
Key: StorePermsPrefix + "/users/cat",
|
||||
},
|
||||
@ -235,9 +235,9 @@ func TestGetAndDeleteUser(t *testing.T) {
|
||||
d := &testDoer{
|
||||
get: []etcdserver.Response{
|
||||
{
|
||||
Event: &etcdstore.Event{
|
||||
Action: etcdstore.Get,
|
||||
Node: &etcdstore.NodeExtern{
|
||||
Event: &v2store.Event{
|
||||
Action: v2store.Get,
|
||||
Node: &v2store.NodeExtern{
|
||||
Key: StorePermsPrefix + "/users/cat",
|
||||
Value: &data,
|
||||
},
|
||||
@ -266,10 +266,10 @@ func TestAllRoles(t *testing.T) {
|
||||
d := &testDoer{
|
||||
get: []etcdserver.Response{
|
||||
{
|
||||
Event: &etcdstore.Event{
|
||||
Action: etcdstore.Get,
|
||||
Node: &etcdstore.NodeExtern{
|
||||
Nodes: etcdstore.NodeExterns([]*etcdstore.NodeExtern{
|
||||
Event: &v2store.Event{
|
||||
Action: v2store.Get,
|
||||
Node: &v2store.NodeExtern{
|
||||
Nodes: v2store.NodeExterns([]*v2store.NodeExtern{
|
||||
{
|
||||
Key: StorePermsPrefix + "/roles/animal",
|
||||
},
|
||||
@ -300,9 +300,9 @@ func TestGetAndDeleteRole(t *testing.T) {
|
||||
d := &testDoer{
|
||||
get: []etcdserver.Response{
|
||||
{
|
||||
Event: &etcdstore.Event{
|
||||
Action: etcdstore.Get,
|
||||
Node: &etcdstore.NodeExtern{
|
||||
Event: &v2store.Event{
|
||||
Action: v2store.Get,
|
||||
Node: &v2store.NodeExtern{
|
||||
Key: StorePermsPrefix + "/roles/animal",
|
||||
Value: &data,
|
||||
},
|
||||
@ -331,27 +331,27 @@ func TestEnsure(t *testing.T) {
|
||||
d := &testDoer{
|
||||
get: []etcdserver.Response{
|
||||
{
|
||||
Event: &etcdstore.Event{
|
||||
Action: etcdstore.Set,
|
||||
Node: &etcdstore.NodeExtern{
|
||||
Event: &v2store.Event{
|
||||
Action: v2store.Set,
|
||||
Node: &v2store.NodeExtern{
|
||||
Key: StorePermsPrefix,
|
||||
Dir: true,
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
Event: &etcdstore.Event{
|
||||
Action: etcdstore.Set,
|
||||
Node: &etcdstore.NodeExtern{
|
||||
Event: &v2store.Event{
|
||||
Action: v2store.Set,
|
||||
Node: &v2store.NodeExtern{
|
||||
Key: StorePermsPrefix + "/users/",
|
||||
Dir: true,
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
Event: &etcdstore.Event{
|
||||
Action: etcdstore.Set,
|
||||
Node: &etcdstore.NodeExtern{
|
||||
Event: &v2store.Event{
|
||||
Action: v2store.Set,
|
||||
Node: &v2store.NodeExtern{
|
||||
Key: StorePermsPrefix + "/roles/",
|
||||
Dir: true,
|
||||
},
|
||||
@ -385,18 +385,18 @@ func TestCreateAndUpdateUser(t *testing.T) {
|
||||
Event: nil,
|
||||
},
|
||||
{
|
||||
Event: &etcdstore.Event{
|
||||
Action: etcdstore.Get,
|
||||
Node: &etcdstore.NodeExtern{
|
||||
Event: &v2store.Event{
|
||||
Action: v2store.Get,
|
||||
Node: &v2store.NodeExtern{
|
||||
Key: StorePermsPrefix + "/users/cat",
|
||||
Value: &olduser,
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
Event: &etcdstore.Event{
|
||||
Action: etcdstore.Get,
|
||||
Node: &etcdstore.NodeExtern{
|
||||
Event: &v2store.Event{
|
||||
Action: v2store.Get,
|
||||
Node: &v2store.NodeExtern{
|
||||
Key: StorePermsPrefix + "/users/cat",
|
||||
Value: &olduser,
|
||||
},
|
||||
@ -405,18 +405,18 @@ func TestCreateAndUpdateUser(t *testing.T) {
|
||||
},
|
||||
put: []etcdserver.Response{
|
||||
{
|
||||
Event: &etcdstore.Event{
|
||||
Action: etcdstore.Update,
|
||||
Node: &etcdstore.NodeExtern{
|
||||
Event: &v2store.Event{
|
||||
Action: v2store.Update,
|
||||
Node: &v2store.NodeExtern{
|
||||
Key: StorePermsPrefix + "/users/cat",
|
||||
Value: &olduser,
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
Event: &etcdstore.Event{
|
||||
Action: etcdstore.Update,
|
||||
Node: &etcdstore.NodeExtern{
|
||||
Event: &v2store.Event{
|
||||
Action: v2store.Update,
|
||||
Node: &v2store.NodeExtern{
|
||||
Key: StorePermsPrefix + "/users/cat",
|
||||
Value: &newuser,
|
||||
},
|
||||
@ -459,9 +459,9 @@ func TestUpdateRole(t *testing.T) {
|
||||
d := &testDoer{
|
||||
get: []etcdserver.Response{
|
||||
{
|
||||
Event: &etcdstore.Event{
|
||||
Action: etcdstore.Get,
|
||||
Node: &etcdstore.NodeExtern{
|
||||
Event: &v2store.Event{
|
||||
Action: v2store.Get,
|
||||
Node: &v2store.NodeExtern{
|
||||
Key: StorePermsPrefix + "/roles/animal",
|
||||
Value: &oldrole,
|
||||
},
|
||||
@ -470,9 +470,9 @@ func TestUpdateRole(t *testing.T) {
|
||||
},
|
||||
put: []etcdserver.Response{
|
||||
{
|
||||
Event: &etcdstore.Event{
|
||||
Action: etcdstore.Update,
|
||||
Node: &etcdstore.NodeExtern{
|
||||
Event: &v2store.Event{
|
||||
Action: v2store.Update,
|
||||
Node: &v2store.NodeExtern{
|
||||
Key: StorePermsPrefix + "/roles/animal",
|
||||
Value: &newrole,
|
||||
},
|
||||
@ -499,9 +499,9 @@ func TestCreateRole(t *testing.T) {
|
||||
d := &testDoer{
|
||||
put: []etcdserver.Response{
|
||||
{
|
||||
Event: &etcdstore.Event{
|
||||
Action: etcdstore.Create,
|
||||
Node: &etcdstore.NodeExtern{
|
||||
Event: &v2store.Event{
|
||||
Action: v2store.Create,
|
||||
Node: &v2store.NodeExtern{
|
||||
Key: StorePermsPrefix + "/roles/animal",
|
||||
Value: &role,
|
||||
},
|
||||
@ -538,18 +538,18 @@ func TestEnableAuth(t *testing.T) {
|
||||
d := &testDoer{
|
||||
get: []etcdserver.Response{
|
||||
{
|
||||
Event: &etcdstore.Event{
|
||||
Action: etcdstore.Get,
|
||||
Node: &etcdstore.NodeExtern{
|
||||
Event: &v2store.Event{
|
||||
Action: v2store.Get,
|
||||
Node: &v2store.NodeExtern{
|
||||
Key: StorePermsPrefix + "/enabled",
|
||||
Value: &falseval,
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
Event: &etcdstore.Event{
|
||||
Action: etcdstore.Get,
|
||||
Node: &etcdstore.NodeExtern{
|
||||
Event: &v2store.Event{
|
||||
Action: v2store.Get,
|
||||
Node: &v2store.NodeExtern{
|
||||
Key: StorePermsPrefix + "/user/root",
|
||||
Value: &rootUser,
|
||||
},
|
||||
@ -561,18 +561,18 @@ func TestEnableAuth(t *testing.T) {
|
||||
},
|
||||
put: []etcdserver.Response{
|
||||
{
|
||||
Event: &etcdstore.Event{
|
||||
Action: etcdstore.Create,
|
||||
Node: &etcdstore.NodeExtern{
|
||||
Event: &v2store.Event{
|
||||
Action: v2store.Create,
|
||||
Node: &v2store.NodeExtern{
|
||||
Key: StorePermsPrefix + "/roles/guest",
|
||||
Value: &guestRole,
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
Event: &etcdstore.Event{
|
||||
Action: etcdstore.Update,
|
||||
Node: &etcdstore.NodeExtern{
|
||||
Event: &v2store.Event{
|
||||
Action: v2store.Update,
|
||||
Node: &v2store.NodeExtern{
|
||||
Key: StorePermsPrefix + "/enabled",
|
||||
Value: &trueval,
|
||||
},
|
||||
@ -594,18 +594,18 @@ func TestDisableAuth(t *testing.T) {
|
||||
d := &testDoer{
|
||||
get: []etcdserver.Response{
|
||||
{
|
||||
Event: &etcdstore.Event{
|
||||
Action: etcdstore.Get,
|
||||
Node: &etcdstore.NodeExtern{
|
||||
Event: &v2store.Event{
|
||||
Action: v2store.Get,
|
||||
Node: &v2store.NodeExtern{
|
||||
Key: StorePermsPrefix + "/enabled",
|
||||
Value: &falseval,
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
Event: &etcdstore.Event{
|
||||
Action: etcdstore.Get,
|
||||
Node: &etcdstore.NodeExtern{
|
||||
Event: &v2store.Event{
|
||||
Action: v2store.Get,
|
||||
Node: &v2store.NodeExtern{
|
||||
Key: StorePermsPrefix + "/enabled",
|
||||
Value: &trueval,
|
||||
},
|
||||
@ -614,9 +614,9 @@ func TestDisableAuth(t *testing.T) {
|
||||
},
|
||||
put: []etcdserver.Response{
|
||||
{
|
||||
Event: &etcdstore.Event{
|
||||
Action: etcdstore.Update,
|
||||
Node: &etcdstore.NodeExtern{
|
||||
Event: &v2store.Event{
|
||||
Action: v2store.Update,
|
||||
Node: &v2store.NodeExtern{
|
||||
Key: StorePermsPrefix + "/enabled",
|
||||
Value: &falseval,
|
||||
},
|
||||
|
@ -12,5 +12,6 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
// Package store defines etcd's in-memory key/value store.
|
||||
package store
|
||||
// Package v2store defines etcd's in-memory key/value store in v2 API.
|
||||
// To be deprecated in favor of v3 storage.
|
||||
package v2store
|
@ -12,7 +12,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package store
|
||||
package v2store
|
||||
|
||||
const (
|
||||
Get = "get"
|
@ -12,7 +12,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package store
|
||||
package v2store
|
||||
|
||||
import (
|
||||
"fmt"
|
@ -12,7 +12,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package store
|
||||
package v2store
|
||||
|
||||
type eventQueue struct {
|
||||
Events []*Event
|
@ -12,7 +12,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package store
|
||||
package v2store
|
||||
|
||||
import (
|
||||
"testing"
|
@ -12,7 +12,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package store
|
||||
package v2store
|
||||
|
||||
import (
|
||||
"fmt"
|
@ -12,11 +12,9 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package store
|
||||
package v2store
|
||||
|
||||
import (
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
)
|
||||
import "github.com/prometheus/client_golang/prometheus"
|
||||
|
||||
// Set of raw Prometheus metrics.
|
||||
// Labels
|
@ -12,7 +12,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package store
|
||||
package v2store
|
||||
|
||||
import (
|
||||
"path"
|
||||
@ -20,6 +20,7 @@ import (
|
||||
"time"
|
||||
|
||||
etcdErr "github.com/coreos/etcd/error"
|
||||
|
||||
"github.com/jonboulle/clockwork"
|
||||
)
|
||||
|
@ -12,7 +12,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package store
|
||||
package v2store
|
||||
|
||||
import (
|
||||
"sort"
|
@ -12,7 +12,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package store
|
||||
package v2store
|
||||
|
||||
import (
|
||||
"reflect"
|
@ -12,7 +12,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package store
|
||||
package v2store
|
||||
|
||||
import (
|
||||
"testing"
|
@ -12,7 +12,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package store
|
||||
package v2store
|
||||
|
||||
import (
|
||||
"encoding/json"
|
@ -12,7 +12,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package store
|
||||
package v2store
|
||||
|
||||
import (
|
||||
"testing"
|
@ -12,7 +12,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package store
|
||||
package v2store
|
||||
|
||||
import (
|
||||
"encoding/json"
|
||||
@ -25,6 +25,7 @@ import (
|
||||
|
||||
etcdErr "github.com/coreos/etcd/error"
|
||||
"github.com/coreos/etcd/pkg/types"
|
||||
|
||||
"github.com/jonboulle/clockwork"
|
||||
)
|
||||
|
@ -12,7 +12,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package store
|
||||
package v2store
|
||||
|
||||
import (
|
||||
"encoding/json"
|
@ -12,19 +12,19 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package store_test
|
||||
package v2store_test
|
||||
|
||||
import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
etcdErr "github.com/coreos/etcd/error"
|
||||
"github.com/coreos/etcd/internal/store"
|
||||
"github.com/coreos/etcd/etcdserver/v2store"
|
||||
"github.com/coreos/etcd/pkg/testutil"
|
||||
)
|
||||
|
||||
type StoreCloser interface {
|
||||
store.Store
|
||||
v2store.Store
|
||||
Close()
|
||||
}
|
||||
|
||||
@ -43,7 +43,7 @@ func TestStoreGetValue(t *testing.T) {
|
||||
s := newTestStore(t)
|
||||
defer s.Close()
|
||||
|
||||
s.Create("/foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/foo", false, "bar", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
|
||||
var eidx uint64 = 1
|
||||
e, err := s.Get("/foo", false, false)
|
||||
testutil.AssertNil(t, err)
|
||||
@ -58,18 +58,18 @@ func TestStoreGetSorted(t *testing.T) {
|
||||
s := newTestStore(t)
|
||||
defer s.Close()
|
||||
|
||||
s.Create("/foo", true, "", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/foo/x", false, "0", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/foo/z", false, "0", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/foo/y", true, "", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/foo/y/a", false, "0", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/foo/y/b", false, "0", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/foo", true, "", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
|
||||
s.Create("/foo/x", false, "0", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
|
||||
s.Create("/foo/z", false, "0", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
|
||||
s.Create("/foo/y", true, "", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
|
||||
s.Create("/foo/y/a", false, "0", false, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
|
||||
s.Create("/foo/y/b", false, "0", false, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
|
||||
var eidx uint64 = 6
|
||||
e, err := s.Get("/foo", true, true)
|
||||
testutil.AssertNil(t, err)
|
||||
testutil.AssertEqual(t, e.EtcdIndex, eidx)
|
||||
|
||||
var yNodes store.NodeExterns
|
||||
var yNodes v2store.NodeExterns
|
||||
sortedStrings := []string{"/foo/x", "/foo/y", "/foo/z"}
|
||||
for i := range e.Node.Nodes {
|
||||
node := e.Node.Nodes[i]
|
||||
@ -96,7 +96,7 @@ func TestSet(t *testing.T) {
|
||||
|
||||
// Set /foo=""
|
||||
var eidx uint64 = 1
|
||||
e, err := s.Set("/foo", false, "", store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
e, err := s.Set("/foo", false, "", v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
|
||||
testutil.AssertNil(t, err)
|
||||
testutil.AssertEqual(t, e.EtcdIndex, eidx)
|
||||
testutil.AssertEqual(t, e.Action, "set")
|
||||
@ -110,7 +110,7 @@ func TestSet(t *testing.T) {
|
||||
|
||||
// Set /foo="bar"
|
||||
eidx = 2
|
||||
e, err = s.Set("/foo", false, "bar", store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
e, err = s.Set("/foo", false, "bar", v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
|
||||
testutil.AssertNil(t, err)
|
||||
testutil.AssertEqual(t, e.EtcdIndex, eidx)
|
||||
testutil.AssertEqual(t, e.Action, "set")
|
||||
@ -128,7 +128,7 @@ func TestSet(t *testing.T) {
|
||||
testutil.AssertEqual(t, e.PrevNode.ModifiedIndex, uint64(1))
|
||||
// Set /foo="baz" (for testing prevNode)
|
||||
eidx = 3
|
||||
e, err = s.Set("/foo", false, "baz", store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
e, err = s.Set("/foo", false, "baz", v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
|
||||
testutil.AssertNil(t, err)
|
||||
testutil.AssertEqual(t, e.EtcdIndex, eidx)
|
||||
testutil.AssertEqual(t, e.Action, "set")
|
||||
@ -147,7 +147,7 @@ func TestSet(t *testing.T) {
|
||||
|
||||
// Set /dir as a directory
|
||||
eidx = 4
|
||||
e, err = s.Set("/dir", true, "", store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
e, err = s.Set("/dir", true, "", v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
|
||||
testutil.AssertNil(t, err)
|
||||
testutil.AssertEqual(t, e.EtcdIndex, eidx)
|
||||
testutil.AssertEqual(t, e.Action, "set")
|
||||
@ -167,7 +167,7 @@ func TestStoreCreateValue(t *testing.T) {
|
||||
|
||||
// Create /foo=bar
|
||||
var eidx uint64 = 1
|
||||
e, err := s.Create("/foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
e, err := s.Create("/foo", false, "bar", false, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
|
||||
testutil.AssertNil(t, err)
|
||||
testutil.AssertEqual(t, e.EtcdIndex, eidx)
|
||||
testutil.AssertEqual(t, e.Action, "create")
|
||||
@ -181,7 +181,7 @@ func TestStoreCreateValue(t *testing.T) {
|
||||
|
||||
// Create /empty=""
|
||||
eidx = 2
|
||||
e, err = s.Create("/empty", false, "", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
e, err = s.Create("/empty", false, "", false, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
|
||||
testutil.AssertNil(t, err)
|
||||
testutil.AssertEqual(t, e.EtcdIndex, eidx)
|
||||
testutil.AssertEqual(t, e.Action, "create")
|
||||
@ -201,7 +201,7 @@ func TestStoreCreateDirectory(t *testing.T) {
|
||||
defer s.Close()
|
||||
|
||||
var eidx uint64 = 1
|
||||
e, err := s.Create("/foo", true, "", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
e, err := s.Create("/foo", true, "", false, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
|
||||
testutil.AssertNil(t, err)
|
||||
testutil.AssertEqual(t, e.EtcdIndex, eidx)
|
||||
testutil.AssertEqual(t, e.Action, "create")
|
||||
@ -215,10 +215,10 @@ func TestStoreCreateFailsIfExists(t *testing.T) {
|
||||
defer s.Close()
|
||||
|
||||
// create /foo as dir
|
||||
s.Create("/foo", true, "", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/foo", true, "", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
|
||||
|
||||
// create /foo as dir again
|
||||
e, _err := s.Create("/foo", true, "", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
e, _err := s.Create("/foo", true, "", false, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
|
||||
err := _err.(*etcdErr.Error)
|
||||
testutil.AssertEqual(t, err.ErrorCode, etcdErr.EcodeNodeExist)
|
||||
testutil.AssertEqual(t, err.Message, "Key already exists")
|
||||
@ -233,10 +233,10 @@ func TestStoreUpdateValue(t *testing.T) {
|
||||
defer s.Close()
|
||||
|
||||
// create /foo=bar
|
||||
s.Create("/foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/foo", false, "bar", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
|
||||
// update /foo="bzr"
|
||||
var eidx uint64 = 2
|
||||
e, err := s.Update("/foo", "baz", store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
e, err := s.Update("/foo", "baz", v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
|
||||
testutil.AssertNil(t, err)
|
||||
testutil.AssertEqual(t, e.EtcdIndex, eidx)
|
||||
testutil.AssertEqual(t, e.Action, "update")
|
||||
@ -257,7 +257,7 @@ func TestStoreUpdateValue(t *testing.T) {
|
||||
|
||||
// update /foo=""
|
||||
eidx = 3
|
||||
e, err = s.Update("/foo", "", store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
e, err = s.Update("/foo", "", v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
|
||||
testutil.AssertNil(t, err)
|
||||
testutil.AssertEqual(t, e.EtcdIndex, eidx)
|
||||
testutil.AssertEqual(t, e.Action, "update")
|
||||
@ -282,8 +282,8 @@ func TestStoreUpdateFailsIfDirectory(t *testing.T) {
|
||||
s := newTestStore(t)
|
||||
defer s.Close()
|
||||
|
||||
s.Create("/foo", true, "", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
e, _err := s.Update("/foo", "baz", store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/foo", true, "", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
|
||||
e, _err := s.Update("/foo", "baz", v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
|
||||
err := _err.(*etcdErr.Error)
|
||||
testutil.AssertEqual(t, err.ErrorCode, etcdErr.EcodeNotFile)
|
||||
testutil.AssertEqual(t, err.Message, "Not a file")
|
||||
@ -297,7 +297,7 @@ func TestStoreDeleteValue(t *testing.T) {
|
||||
defer s.Close()
|
||||
|
||||
var eidx uint64 = 2
|
||||
s.Create("/foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/foo", false, "bar", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
|
||||
e, err := s.Delete("/foo", false, false)
|
||||
testutil.AssertNil(t, err)
|
||||
testutil.AssertEqual(t, e.EtcdIndex, eidx)
|
||||
@ -315,7 +315,7 @@ func TestStoreDeleteDirectory(t *testing.T) {
|
||||
|
||||
// create directory /foo
|
||||
var eidx uint64 = 2
|
||||
s.Create("/foo", true, "", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/foo", true, "", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
|
||||
// delete /foo with dir = true and recursive = false
|
||||
// this should succeed, since the directory is empty
|
||||
e, err := s.Delete("/foo", true, false)
|
||||
@ -328,7 +328,7 @@ func TestStoreDeleteDirectory(t *testing.T) {
|
||||
testutil.AssertEqual(t, e.PrevNode.Dir, true)
|
||||
|
||||
// create directory /foo and directory /foo/bar
|
||||
_, err = s.Create("/foo/bar", true, "", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
_, err = s.Create("/foo/bar", true, "", false, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
|
||||
testutil.AssertNil(t, err)
|
||||
// delete /foo with dir = true and recursive = false
|
||||
// this should fail, since the directory is not empty
|
||||
@ -351,7 +351,7 @@ func TestStoreDeleteDirectoryFailsIfNonRecursiveAndDir(t *testing.T) {
|
||||
s := newTestStore(t)
|
||||
defer s.Close()
|
||||
|
||||
s.Create("/foo", true, "", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/foo", true, "", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
|
||||
e, _err := s.Delete("/foo", false, false)
|
||||
err := _err.(*etcdErr.Error)
|
||||
testutil.AssertEqual(t, err.ErrorCode, etcdErr.EcodeNotFile)
|
||||
@ -364,19 +364,19 @@ func TestRootRdOnly(t *testing.T) {
|
||||
defer s.Close()
|
||||
|
||||
for _, tt := range []string{"/", "/0"} {
|
||||
_, err := s.Set(tt, true, "", store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
_, err := s.Set(tt, true, "", v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
|
||||
testutil.AssertNotNil(t, err)
|
||||
|
||||
_, err = s.Delete(tt, true, true)
|
||||
testutil.AssertNotNil(t, err)
|
||||
|
||||
_, err = s.Create(tt, true, "", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
_, err = s.Create(tt, true, "", false, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
|
||||
testutil.AssertNotNil(t, err)
|
||||
|
||||
_, err = s.Update(tt, "", store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
_, err = s.Update(tt, "", v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
|
||||
testutil.AssertNotNil(t, err)
|
||||
|
||||
_, err = s.CompareAndSwap(tt, "", 0, "", store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
_, err = s.CompareAndSwap(tt, "", 0, "", v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
|
||||
testutil.AssertNotNil(t, err)
|
||||
}
|
||||
}
|
||||
@ -386,7 +386,7 @@ func TestStoreCompareAndDeletePrevValue(t *testing.T) {
|
||||
defer s.Close()
|
||||
|
||||
var eidx uint64 = 2
|
||||
s.Create("/foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/foo", false, "bar", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
|
||||
e, err := s.CompareAndDelete("/foo", "bar", 0)
|
||||
testutil.AssertNil(t, err)
|
||||
testutil.AssertEqual(t, e.EtcdIndex, eidx)
|
||||
@ -406,7 +406,7 @@ func TestStoreCompareAndDeletePrevValueFailsIfNotMatch(t *testing.T) {
|
||||
defer s.Close()
|
||||
|
||||
var eidx uint64 = 1
|
||||
s.Create("/foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/foo", false, "bar", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
|
||||
e, _err := s.CompareAndDelete("/foo", "baz", 0)
|
||||
err := _err.(*etcdErr.Error)
|
||||
testutil.AssertEqual(t, err.ErrorCode, etcdErr.EcodeTestFailed)
|
||||
@ -422,7 +422,7 @@ func TestStoreCompareAndDeletePrevIndex(t *testing.T) {
|
||||
defer s.Close()
|
||||
|
||||
var eidx uint64 = 2
|
||||
s.Create("/foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/foo", false, "bar", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
|
||||
e, err := s.CompareAndDelete("/foo", "", 1)
|
||||
testutil.AssertNil(t, err)
|
||||
testutil.AssertEqual(t, e.EtcdIndex, eidx)
|
||||
@ -440,7 +440,7 @@ func TestStoreCompareAndDeletePrevIndexFailsIfNotMatch(t *testing.T) {
|
||||
defer s.Close()
|
||||
|
||||
var eidx uint64 = 1
|
||||
s.Create("/foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/foo", false, "bar", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
|
||||
e, _err := s.CompareAndDelete("/foo", "", 100)
|
||||
testutil.AssertNotNil(t, _err)
|
||||
err := _err.(*etcdErr.Error)
|
||||
@ -457,7 +457,7 @@ func TestStoreCompareAndDeleteDirectoryFail(t *testing.T) {
|
||||
s := newTestStore(t)
|
||||
defer s.Close()
|
||||
|
||||
s.Create("/foo", true, "", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/foo", true, "", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
|
||||
_, _err := s.CompareAndDelete("/foo", "", 0)
|
||||
testutil.AssertNotNil(t, _err)
|
||||
err := _err.(*etcdErr.Error)
|
||||
@ -470,8 +470,8 @@ func TestStoreCompareAndSwapPrevValue(t *testing.T) {
|
||||
defer s.Close()
|
||||
|
||||
var eidx uint64 = 2
|
||||
s.Create("/foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
e, err := s.CompareAndSwap("/foo", "bar", 0, "baz", store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/foo", false, "bar", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
|
||||
e, err := s.CompareAndSwap("/foo", "bar", 0, "baz", v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
|
||||
testutil.AssertNil(t, err)
|
||||
testutil.AssertEqual(t, e.EtcdIndex, eidx)
|
||||
testutil.AssertEqual(t, e.Action, "compareAndSwap")
|
||||
@ -492,8 +492,8 @@ func TestStoreCompareAndSwapPrevValueFailsIfNotMatch(t *testing.T) {
|
||||
s := newTestStore(t)
|
||||
defer s.Close()
|
||||
var eidx uint64 = 1
|
||||
s.Create("/foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
e, _err := s.CompareAndSwap("/foo", "wrong_value", 0, "baz", store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/foo", false, "bar", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
|
||||
e, _err := s.CompareAndSwap("/foo", "wrong_value", 0, "baz", v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
|
||||
err := _err.(*etcdErr.Error)
|
||||
testutil.AssertEqual(t, err.ErrorCode, etcdErr.EcodeTestFailed)
|
||||
testutil.AssertEqual(t, err.Message, "Compare failed")
|
||||
@ -508,8 +508,8 @@ func TestStoreCompareAndSwapPrevIndex(t *testing.T) {
|
||||
s := newTestStore(t)
|
||||
defer s.Close()
|
||||
var eidx uint64 = 2
|
||||
s.Create("/foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
e, err := s.CompareAndSwap("/foo", "", 1, "baz", store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/foo", false, "bar", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
|
||||
e, err := s.CompareAndSwap("/foo", "", 1, "baz", v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
|
||||
testutil.AssertNil(t, err)
|
||||
testutil.AssertEqual(t, e.EtcdIndex, eidx)
|
||||
testutil.AssertEqual(t, e.Action, "compareAndSwap")
|
||||
@ -531,8 +531,8 @@ func TestStoreCompareAndSwapPrevIndexFailsIfNotMatch(t *testing.T) {
|
||||
s := newTestStore(t)
|
||||
defer s.Close()
|
||||
var eidx uint64 = 1
|
||||
s.Create("/foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
e, _err := s.CompareAndSwap("/foo", "", 100, "baz", store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/foo", false, "bar", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
|
||||
e, _err := s.CompareAndSwap("/foo", "", 100, "baz", v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
|
||||
err := _err.(*etcdErr.Error)
|
||||
testutil.AssertEqual(t, err.ErrorCode, etcdErr.EcodeTestFailed)
|
||||
testutil.AssertEqual(t, err.Message, "Compare failed")
|
||||
@ -550,7 +550,7 @@ func TestStoreWatchCreate(t *testing.T) {
|
||||
w, _ := s.Watch("/foo", false, false, 0)
|
||||
c := w.EventChan()
|
||||
testutil.AssertEqual(t, w.StartIndex(), eidx)
|
||||
s.Create("/foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/foo", false, "bar", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
|
||||
eidx = 1
|
||||
e := timeoutSelect(t, c)
|
||||
testutil.AssertEqual(t, e.EtcdIndex, eidx)
|
||||
@ -572,7 +572,7 @@ func TestStoreWatchRecursiveCreate(t *testing.T) {
|
||||
testutil.AssertNil(t, err)
|
||||
testutil.AssertEqual(t, w.StartIndex(), eidx)
|
||||
eidx = 1
|
||||
s.Create("/foo/bar", false, "baz", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/foo/bar", false, "baz", false, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
|
||||
e := timeoutSelect(t, w.EventChan())
|
||||
testutil.AssertEqual(t, e.EtcdIndex, eidx)
|
||||
testutil.AssertEqual(t, e.Action, "create")
|
||||
@ -584,11 +584,11 @@ func TestStoreWatchUpdate(t *testing.T) {
|
||||
s := newTestStore(t)
|
||||
defer s.Close()
|
||||
var eidx uint64 = 1
|
||||
s.Create("/foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/foo", false, "bar", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
|
||||
w, _ := s.Watch("/foo", false, false, 0)
|
||||
testutil.AssertEqual(t, w.StartIndex(), eidx)
|
||||
eidx = 2
|
||||
s.Update("/foo", "baz", store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Update("/foo", "baz", v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
|
||||
e := timeoutSelect(t, w.EventChan())
|
||||
testutil.AssertEqual(t, e.EtcdIndex, eidx)
|
||||
testutil.AssertEqual(t, e.Action, "update")
|
||||
@ -600,12 +600,12 @@ func TestStoreWatchRecursiveUpdate(t *testing.T) {
|
||||
s := newTestStore(t)
|
||||
defer s.Close()
|
||||
var eidx uint64 = 1
|
||||
s.Create("/foo/bar", false, "baz", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/foo/bar", false, "baz", false, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
|
||||
w, err := s.Watch("/foo", true, false, 0)
|
||||
testutil.AssertNil(t, err)
|
||||
testutil.AssertEqual(t, w.StartIndex(), eidx)
|
||||
eidx = 2
|
||||
s.Update("/foo/bar", "baz", store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Update("/foo/bar", "baz", v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
|
||||
e := timeoutSelect(t, w.EventChan())
|
||||
testutil.AssertEqual(t, e.EtcdIndex, eidx)
|
||||
testutil.AssertEqual(t, e.Action, "update")
|
||||
@ -617,7 +617,7 @@ func TestStoreWatchDelete(t *testing.T) {
|
||||
s := newTestStore(t)
|
||||
defer s.Close()
|
||||
var eidx uint64 = 1
|
||||
s.Create("/foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/foo", false, "bar", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
|
||||
w, _ := s.Watch("/foo", false, false, 0)
|
||||
testutil.AssertEqual(t, w.StartIndex(), eidx)
|
||||
eidx = 2
|
||||
@ -633,7 +633,7 @@ func TestStoreWatchRecursiveDelete(t *testing.T) {
|
||||
s := newTestStore(t)
|
||||
defer s.Close()
|
||||
var eidx uint64 = 1
|
||||
s.Create("/foo/bar", false, "baz", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/foo/bar", false, "baz", false, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
|
||||
w, err := s.Watch("/foo", true, false, 0)
|
||||
testutil.AssertNil(t, err)
|
||||
testutil.AssertEqual(t, w.StartIndex(), eidx)
|
||||
@ -650,11 +650,11 @@ func TestStoreWatchCompareAndSwap(t *testing.T) {
|
||||
s := newTestStore(t)
|
||||
defer s.Close()
|
||||
var eidx uint64 = 1
|
||||
s.Create("/foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/foo", false, "bar", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
|
||||
w, _ := s.Watch("/foo", false, false, 0)
|
||||
testutil.AssertEqual(t, w.StartIndex(), eidx)
|
||||
eidx = 2
|
||||
s.CompareAndSwap("/foo", "bar", 0, "baz", store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.CompareAndSwap("/foo", "bar", 0, "baz", v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
|
||||
e := timeoutSelect(t, w.EventChan())
|
||||
testutil.AssertEqual(t, e.EtcdIndex, eidx)
|
||||
testutil.AssertEqual(t, e.Action, "compareAndSwap")
|
||||
@ -666,11 +666,11 @@ func TestStoreWatchRecursiveCompareAndSwap(t *testing.T) {
|
||||
s := newTestStore(t)
|
||||
defer s.Close()
|
||||
var eidx uint64 = 1
|
||||
s.Create("/foo/bar", false, "baz", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/foo/bar", false, "baz", false, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
|
||||
w, _ := s.Watch("/foo", true, false, 0)
|
||||
testutil.AssertEqual(t, w.StartIndex(), eidx)
|
||||
eidx = 2
|
||||
s.CompareAndSwap("/foo/bar", "baz", 0, "bat", store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.CompareAndSwap("/foo/bar", "baz", 0, "bat", v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
|
||||
e := timeoutSelect(t, w.EventChan())
|
||||
testutil.AssertEqual(t, e.EtcdIndex, eidx)
|
||||
testutil.AssertEqual(t, e.Action, "compareAndSwap")
|
||||
@ -684,7 +684,7 @@ func TestStoreWatchStream(t *testing.T) {
|
||||
var eidx uint64 = 1
|
||||
w, _ := s.Watch("/foo", false, true, 0)
|
||||
// first modification
|
||||
s.Create("/foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/foo", false, "bar", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
|
||||
e := timeoutSelect(t, w.EventChan())
|
||||
testutil.AssertEqual(t, e.EtcdIndex, eidx)
|
||||
testutil.AssertEqual(t, e.Action, "create")
|
||||
@ -697,7 +697,7 @@ func TestStoreWatchStream(t *testing.T) {
|
||||
}
|
||||
// second modification
|
||||
eidx = 2
|
||||
s.Update("/foo", "baz", store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Update("/foo", "baz", v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
|
||||
e = timeoutSelect(t, w.EventChan())
|
||||
testutil.AssertEqual(t, e.EtcdIndex, eidx)
|
||||
testutil.AssertEqual(t, e.Action, "update")
|
||||
@ -716,7 +716,7 @@ func TestStoreWatchCreateWithHiddenKey(t *testing.T) {
|
||||
defer s.Close()
|
||||
var eidx uint64 = 1
|
||||
w, _ := s.Watch("/_foo", false, false, 0)
|
||||
s.Create("/_foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/_foo", false, "bar", false, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
|
||||
e := timeoutSelect(t, w.EventChan())
|
||||
testutil.AssertEqual(t, e.EtcdIndex, eidx)
|
||||
testutil.AssertEqual(t, e.Action, "create")
|
||||
@ -733,17 +733,17 @@ func TestStoreWatchRecursiveCreateWithHiddenKey(t *testing.T) {
|
||||
s := newTestStore(t)
|
||||
defer s.Close()
|
||||
w, _ := s.Watch("/foo", true, false, 0)
|
||||
s.Create("/foo/_bar", false, "baz", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/foo/_bar", false, "baz", false, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
|
||||
e := nbselect(w.EventChan())
|
||||
testutil.AssertNil(t, e)
|
||||
w, _ = s.Watch("/foo", true, false, 0)
|
||||
s.Create("/foo/_baz", true, "", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/foo/_baz", true, "", false, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
|
||||
select {
|
||||
case e = <-w.EventChan():
|
||||
testutil.AssertNil(t, e)
|
||||
case <-time.After(100 * time.Millisecond):
|
||||
}
|
||||
s.Create("/foo/_baz/quux", false, "quux", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/foo/_baz/quux", false, "quux", false, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
|
||||
select {
|
||||
case e = <-w.EventChan():
|
||||
testutil.AssertNil(t, e)
|
||||
@ -755,9 +755,9 @@ func TestStoreWatchRecursiveCreateWithHiddenKey(t *testing.T) {
|
||||
func TestStoreWatchUpdateWithHiddenKey(t *testing.T) {
|
||||
s := newTestStore(t)
|
||||
defer s.Close()
|
||||
s.Create("/_foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/_foo", false, "bar", false, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
|
||||
w, _ := s.Watch("/_foo", false, false, 0)
|
||||
s.Update("/_foo", "baz", store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Update("/_foo", "baz", v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
|
||||
e := timeoutSelect(t, w.EventChan())
|
||||
testutil.AssertEqual(t, e.Action, "update")
|
||||
testutil.AssertEqual(t, e.Node.Key, "/_foo")
|
||||
@ -769,9 +769,9 @@ func TestStoreWatchUpdateWithHiddenKey(t *testing.T) {
|
||||
func TestStoreWatchRecursiveUpdateWithHiddenKey(t *testing.T) {
|
||||
s := newTestStore(t)
|
||||
defer s.Close()
|
||||
s.Create("/foo/_bar", false, "baz", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/foo/_bar", false, "baz", false, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
|
||||
w, _ := s.Watch("/foo", true, false, 0)
|
||||
s.Update("/foo/_bar", "baz", store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Update("/foo/_bar", "baz", v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
|
||||
e := nbselect(w.EventChan())
|
||||
testutil.AssertNil(t, e)
|
||||
}
|
||||
@ -781,7 +781,7 @@ func TestStoreWatchDeleteWithHiddenKey(t *testing.T) {
|
||||
s := newTestStore(t)
|
||||
defer s.Close()
|
||||
var eidx uint64 = 2
|
||||
s.Create("/_foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/_foo", false, "bar", false, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
|
||||
w, _ := s.Watch("/_foo", false, false, 0)
|
||||
s.Delete("/_foo", false, false)
|
||||
e := timeoutSelect(t, w.EventChan())
|
||||
@ -796,7 +796,7 @@ func TestStoreWatchDeleteWithHiddenKey(t *testing.T) {
|
||||
func TestStoreWatchRecursiveDeleteWithHiddenKey(t *testing.T) {
|
||||
s := newTestStore(t)
|
||||
defer s.Close()
|
||||
s.Create("/foo/_bar", false, "baz", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/foo/_bar", false, "baz", false, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
|
||||
w, _ := s.Watch("/foo", true, false, 0)
|
||||
s.Delete("/foo/_bar", false, false)
|
||||
e := nbselect(w.EventChan())
|
||||
@ -809,7 +809,7 @@ func TestStoreWatchRecursiveCreateDeeperThanHiddenKey(t *testing.T) {
|
||||
defer s.Close()
|
||||
var eidx uint64 = 1
|
||||
w, _ := s.Watch("/_foo/bar", true, false, 0)
|
||||
s.Create("/_foo/bar/baz", false, "baz", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/_foo/bar/baz", false, "baz", false, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
|
||||
|
||||
e := timeoutSelect(t, w.EventChan())
|
||||
testutil.AssertNotNil(t, e)
|
||||
@ -831,13 +831,13 @@ func TestStoreWatchSlowConsumer(t *testing.T) {
|
||||
s.Watch("/foo", true, true, 0) // stream must be true
|
||||
// Fill watch channel with 100 events
|
||||
for i := 1; i <= 100; i++ {
|
||||
s.Set("/foo", false, string(i), store.TTLOptionSet{ExpireTime: store.Permanent}) // ok
|
||||
s.Set("/foo", false, string(i), v2store.TTLOptionSet{ExpireTime: v2store.Permanent}) // ok
|
||||
}
|
||||
// testutil.AssertEqual(t, s.WatcherHub.count, int64(1))
|
||||
s.Set("/foo", false, "101", store.TTLOptionSet{ExpireTime: store.Permanent}) // ok
|
||||
s.Set("/foo", false, "101", v2store.TTLOptionSet{ExpireTime: v2store.Permanent}) // ok
|
||||
// remove watcher
|
||||
// testutil.AssertEqual(t, s.WatcherHub.count, int64(0))
|
||||
s.Set("/foo", false, "102", store.TTLOptionSet{ExpireTime: store.Permanent}) // must not panic
|
||||
s.Set("/foo", false, "102", v2store.TTLOptionSet{ExpireTime: v2store.Permanent}) // must not panic
|
||||
}
|
||||
|
||||
// Performs a non-blocking select on an event channel.
|
@ -12,7 +12,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package store
|
||||
package v2store
|
||||
|
||||
import (
|
||||
"testing"
|
||||
@ -20,6 +20,7 @@ import (
|
||||
|
||||
etcdErr "github.com/coreos/etcd/error"
|
||||
"github.com/coreos/etcd/pkg/testutil"
|
||||
|
||||
"github.com/jonboulle/clockwork"
|
||||
)
|
||||
|
@ -14,23 +14,23 @@
|
||||
|
||||
// +build !v2v3
|
||||
|
||||
package store_test
|
||||
package v2store_test
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/coreos/etcd/internal/store"
|
||||
"github.com/coreos/etcd/etcdserver/v2store"
|
||||
"github.com/coreos/etcd/pkg/testutil"
|
||||
)
|
||||
|
||||
type v2TestStore struct {
|
||||
store.Store
|
||||
v2store.Store
|
||||
}
|
||||
|
||||
func (s *v2TestStore) Close() {}
|
||||
|
||||
func newTestStore(t *testing.T, ns ...string) StoreCloser {
|
||||
return &v2TestStore{store.New(ns...)}
|
||||
return &v2TestStore{v2store.New(ns...)}
|
||||
}
|
||||
|
||||
// Ensure that the store can recover from a previously saved state.
|
||||
@ -38,10 +38,10 @@ func TestStoreRecover(t *testing.T) {
|
||||
s := newTestStore(t)
|
||||
defer s.Close()
|
||||
var eidx uint64 = 4
|
||||
s.Create("/foo", true, "", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/foo/x", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Update("/foo/x", "barbar", store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/foo/y", false, "baz", false, store.TTLOptionSet{ExpireTime: store.Permanent})
|
||||
s.Create("/foo", true, "", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
|
||||
s.Create("/foo/x", false, "bar", false, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
|
||||
s.Update("/foo/x", "barbar", v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
|
||||
s.Create("/foo/y", false, "baz", false, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
|
||||
b, err := s.Save()
|
||||
testutil.AssertNil(t, err)
|
||||
|
@ -14,7 +14,7 @@
|
||||
|
||||
// +build v2v3
|
||||
|
||||
package store_test
|
||||
package v2store_test
|
||||
|
||||
import (
|
||||
"io/ioutil"
|
||||
@ -22,8 +22,8 @@ import (
|
||||
|
||||
"github.com/coreos/etcd/clientv3"
|
||||
"github.com/coreos/etcd/etcdserver/api/v2v3"
|
||||
"github.com/coreos/etcd/etcdserver/v2store"
|
||||
"github.com/coreos/etcd/integration"
|
||||
"github.com/coreos/etcd/internal/store"
|
||||
|
||||
"github.com/coreos/pkg/capnslog"
|
||||
"google.golang.org/grpc/grpclog"
|
||||
@ -35,7 +35,7 @@ func init() {
|
||||
}
|
||||
|
||||
type v2v3TestStore struct {
|
||||
store.Store
|
||||
v2store.Store
|
||||
clus *integration.ClusterV3
|
||||
t *testing.T
|
||||
}
|
@ -12,11 +12,9 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package store
|
||||
package v2store
|
||||
|
||||
import (
|
||||
"container/heap"
|
||||
)
|
||||
import "container/heap"
|
||||
|
||||
// An TTLKeyHeap is a min-heap of TTLKeys order by expiration time
|
||||
type ttlKeyHeap struct {
|
@ -12,7 +12,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package store
|
||||
package v2store
|
||||
|
||||
type Watcher interface {
|
||||
EventChan() chan *Event
|
@ -12,7 +12,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package store
|
||||
package v2store
|
||||
|
||||
import (
|
||||
"container/list"
|
@ -12,11 +12,9 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package store
|
||||
package v2store
|
||||
|
||||
import (
|
||||
"testing"
|
||||
)
|
||||
import "testing"
|
||||
|
||||
// TestIsHidden tests isHidden functions.
|
||||
func TestIsHidden(t *testing.T) {
|
@ -12,11 +12,9 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package store
|
||||
package v2store
|
||||
|
||||
import (
|
||||
"testing"
|
||||
)
|
||||
import "testing"
|
||||
|
||||
func TestWatcher(t *testing.T) {
|
||||
s := newStore()
|
@ -17,25 +17,25 @@ package mockstore
|
||||
import (
|
||||
"time"
|
||||
|
||||
"github.com/coreos/etcd/internal/store"
|
||||
"github.com/coreos/etcd/etcdserver/v2store"
|
||||
"github.com/coreos/etcd/pkg/testutil"
|
||||
)
|
||||
|
||||
// StoreRecorder provides a Store interface with a testutil.Recorder
|
||||
type StoreRecorder struct {
|
||||
store.Store
|
||||
v2store.Store
|
||||
testutil.Recorder
|
||||
}
|
||||
|
||||
// storeRecorder records all the methods it receives.
|
||||
// storeRecorder DOES NOT work as a actual store.
|
||||
// storeRecorder DOES NOT work as a actual v2store.
|
||||
// It always returns invalid empty response and no error.
|
||||
type storeRecorder struct {
|
||||
store.Store
|
||||
v2store.Store
|
||||
testutil.Recorder
|
||||
}
|
||||
|
||||
func NewNop() store.Store { return &storeRecorder{Recorder: &testutil.RecorderBuffered{}} }
|
||||
func NewNop() v2store.Store { return &storeRecorder{Recorder: &testutil.RecorderBuffered{}} }
|
||||
func NewRecorder() *StoreRecorder {
|
||||
sr := &storeRecorder{Recorder: &testutil.RecorderBuffered{}}
|
||||
return &StoreRecorder{Store: sr, Recorder: sr.Recorder}
|
||||
@ -47,54 +47,54 @@ func NewRecorderStream() *StoreRecorder {
|
||||
|
||||
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) {
|
||||
func (s *storeRecorder) Get(path string, recursive, sorted bool) (*v2store.Event, error) {
|
||||
s.Record(testutil.Action{
|
||||
Name: "Get",
|
||||
Params: []interface{}{path, recursive, sorted},
|
||||
})
|
||||
return &store.Event{}, nil
|
||||
return &v2store.Event{}, nil
|
||||
}
|
||||
func (s *storeRecorder) Set(path string, dir bool, val string, expireOpts store.TTLOptionSet) (*store.Event, error) {
|
||||
func (s *storeRecorder) Set(path string, dir bool, val string, expireOpts store.TTLOptionSet) (*v2store.Event, error) {
|
||||
s.Record(testutil.Action{
|
||||
Name: "Set",
|
||||
Params: []interface{}{path, dir, val, expireOpts},
|
||||
})
|
||||
return &store.Event{}, nil
|
||||
return &v2store.Event{}, nil
|
||||
}
|
||||
func (s *storeRecorder) Update(path, val string, expireOpts store.TTLOptionSet) (*store.Event, error) {
|
||||
func (s *storeRecorder) Update(path, val string, expireOpts store.TTLOptionSet) (*v2store.Event, error) {
|
||||
s.Record(testutil.Action{
|
||||
Name: "Update",
|
||||
Params: []interface{}{path, val, expireOpts},
|
||||
})
|
||||
return &store.Event{}, nil
|
||||
return &v2store.Event{}, nil
|
||||
}
|
||||
func (s *storeRecorder) Create(path string, dir bool, val string, uniq bool, expireOpts store.TTLOptionSet) (*store.Event, error) {
|
||||
func (s *storeRecorder) Create(path string, dir bool, val string, uniq bool, expireOpts store.TTLOptionSet) (*v2store.Event, error) {
|
||||
s.Record(testutil.Action{
|
||||
Name: "Create",
|
||||
Params: []interface{}{path, dir, val, uniq, expireOpts},
|
||||
})
|
||||
return &store.Event{}, nil
|
||||
return &v2store.Event{}, nil
|
||||
}
|
||||
func (s *storeRecorder) CompareAndSwap(path, prevVal string, prevIdx uint64, val string, expireOpts store.TTLOptionSet) (*store.Event, error) {
|
||||
func (s *storeRecorder) CompareAndSwap(path, prevVal string, prevIdx uint64, val string, expireOpts store.TTLOptionSet) (*v2store.Event, error) {
|
||||
s.Record(testutil.Action{
|
||||
Name: "CompareAndSwap",
|
||||
Params: []interface{}{path, prevVal, prevIdx, val, expireOpts},
|
||||
})
|
||||
return &store.Event{}, nil
|
||||
return &v2store.Event{}, nil
|
||||
}
|
||||
func (s *storeRecorder) Delete(path string, dir, recursive bool) (*store.Event, error) {
|
||||
func (s *storeRecorder) Delete(path string, dir, recursive bool) (*v2store.Event, error) {
|
||||
s.Record(testutil.Action{
|
||||
Name: "Delete",
|
||||
Params: []interface{}{path, dir, recursive},
|
||||
})
|
||||
return &store.Event{}, nil
|
||||
return &v2store.Event{}, nil
|
||||
}
|
||||
func (s *storeRecorder) CompareAndDelete(path, prevVal string, prevIdx uint64) (*store.Event, error) {
|
||||
func (s *storeRecorder) CompareAndDelete(path, prevVal string, prevIdx uint64) (*v2store.Event, error) {
|
||||
s.Record(testutil.Action{
|
||||
Name: "CompareAndDelete",
|
||||
Params: []interface{}{path, prevVal, prevIdx},
|
||||
})
|
||||
return &store.Event{}, nil
|
||||
return &v2store.Event{}, nil
|
||||
}
|
||||
func (s *storeRecorder) Watch(_ string, _, _ bool, _ uint64) (store.Watcher, error) {
|
||||
s.Record(testutil.Action{Name: "Watch"})
|
||||
@ -147,7 +147,7 @@ func NewErrRecorder(err error) *StoreRecorder {
|
||||
return &StoreRecorder{Store: sr, Recorder: sr.Recorder}
|
||||
}
|
||||
|
||||
func (s *errStoreRecorder) Get(path string, recursive, sorted bool) (*store.Event, error) {
|
||||
func (s *errStoreRecorder) Get(path string, recursive, sorted bool) (*v2store.Event, error) {
|
||||
s.storeRecorder.Get(path, recursive, sorted)
|
||||
return nil, s.err
|
||||
}
|
||||
|
@ -30,10 +30,10 @@ import (
|
||||
"github.com/coreos/etcd/etcdserver"
|
||||
"github.com/coreos/etcd/etcdserver/etcdserverpb"
|
||||
"github.com/coreos/etcd/etcdserver/membership"
|
||||
"github.com/coreos/etcd/etcdserver/v2store"
|
||||
"github.com/coreos/etcd/internal/mvcc"
|
||||
"github.com/coreos/etcd/internal/mvcc/backend"
|
||||
"github.com/coreos/etcd/internal/raftsnap"
|
||||
"github.com/coreos/etcd/internal/store"
|
||||
"github.com/coreos/etcd/lease"
|
||||
"github.com/coreos/etcd/pkg/fileutil"
|
||||
"github.com/coreos/etcd/pkg/logger"
|
||||
@ -354,7 +354,7 @@ func (s *v3Manager) saveWALAndSnap() error {
|
||||
}
|
||||
|
||||
// add members again to persist them to the store we create.
|
||||
st := store.New(etcdserver.StoreClusterPrefix, etcdserver.StoreKeysPrefix)
|
||||
st := v2store.New(etcdserver.StoreClusterPrefix, etcdserver.StoreKeysPrefix)
|
||||
s.cl.SetStore(st)
|
||||
for _, m := range s.cl.Members() {
|
||||
s.cl.AddMember(m)
|
||||
|
Loading…
x
Reference in New Issue
Block a user