Browse Source

*: fix "v2store" imports

Signed-off-by: Gyuho Lee <gyuhox@gmail.com>
Gyuho Lee 7 years ago
parent
commit
811566f2f2

+ 2 - 2
etcdserver/api/v2http/client_test.go

@@ -1588,13 +1588,13 @@ func TestServeKeysEvent(t *testing.T) {
 			),
 			etcdserver.Response{
 				Event: &v2store.Event{
-					Action: store.CompareAndSwap,
+					Action: v2store.CompareAndSwap,
 					Node:   &v2store.NodeExtern{},
 				},
 			},
 			http.StatusOK,
 			&v2store.Event{
-				Action: store.CompareAndSwap,
+				Action: v2store.CompareAndSwap,
 				Node:   nil,
 			},
 		},

+ 1 - 1
etcdserver/membership/cluster_test.go

@@ -544,7 +544,7 @@ func TestClusterUpdateAttributes(t *testing.T) {
 }
 
 func TestNodeToMember(t *testing.T) {
-	n := &store.NodeExtern{Key: "/1234", Nodes: []*v2store.NodeExtern{
+	n := &v2store.NodeExtern{Key: "/1234", Nodes: []*v2store.NodeExtern{
 		{Key: "/1234/attributes", Value: stringp(`{"name":"node1","clientURLs":null}`)},
 		{Key: "/1234/raftAttributes", Value: stringp(`{"peerURLs":null}`)},
 	}}

+ 32 - 33
etcdserver/server_test.go

@@ -29,7 +29,6 @@ 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/store"
 	"github.com/coreos/etcd/lease"
 	"github.com/coreos/etcd/mvcc"
 	"github.com/coreos/etcd/mvcc/backend"
@@ -90,7 +89,7 @@ func TestDoLocalAction(t *testing.T) {
 	for i, tt := range tests {
 		st := mockstore.NewRecorder()
 		srv := &EtcdServer{
-			store:    st,
+			v2store:  st,
 			reqIDGen: idutil.NewGenerator(0, time.Time{}),
 		}
 		resp, err := srv.Do(context.TODO(), tt.req)
@@ -143,7 +142,7 @@ func TestDoBadLocalAction(t *testing.T) {
 	for i, tt := range tests {
 		st := mockstore.NewErrRecorder(storeErr)
 		srv := &EtcdServer{
-			store:    st,
+			v2store:  st,
 			reqIDGen: idutil.NewGenerator(0, time.Time{}),
 		}
 		resp, err := srv.Do(context.Background(), tt.req)
@@ -179,12 +178,12 @@ func TestApplyRepeat(t *testing.T) {
 	})
 	s := &EtcdServer{
 		r:          *r,
-		store:      st,
+		v2store:    st,
 		cluster:    cl,
 		reqIDGen:   idutil.NewGenerator(0, time.Time{}),
 		SyncTicker: &time.Ticker{},
 	}
-	s.applyV2 = &applierV2store{store: s.store, cluster: s.cluster}
+	s.applyV2 = &applierV2store{store: s.v2store, cluster: s.cluster}
 	s.start()
 	req := &pb.Request{Method: "QGET", ID: uint64(1)}
 	ents := []raftpb.Entry{{Index: 1, Data: pbutil.MustMarshal(req)}}
@@ -449,8 +448,8 @@ func TestApplyRequest(t *testing.T) {
 
 	for i, tt := range tests {
 		st := mockstore.NewRecorder()
-		srv := &EtcdServer{store: st}
-		srv.applyV2 = &applierV2store{store: srv.store, cluster: srv.cluster}
+		srv := &EtcdServer{v2store: st}
+		srv.applyV2 = &applierV2store{store: srv.v2store, cluster: srv.cluster}
 		resp := srv.applyV2Request((*RequestV2)(&tt.req))
 
 		if !reflect.DeepEqual(resp, tt.wresp) {
@@ -466,10 +465,10 @@ func TestApplyRequest(t *testing.T) {
 func TestApplyRequestOnAdminMemberAttributes(t *testing.T) {
 	cl := newTestCluster([]*membership.Member{{ID: 1}})
 	srv := &EtcdServer{
-		store:   mockstore.NewRecorder(),
+		v2store: mockstore.NewRecorder(),
 		cluster: cl,
 	}
-	srv.applyV2 = &applierV2store{store: srv.store, cluster: srv.cluster}
+	srv.applyV2 = &applierV2store{store: srv.v2store, cluster: srv.cluster}
 
 	req := pb.Request{
 		Method: "PUT",
@@ -686,11 +685,11 @@ func TestDoProposal(t *testing.T) {
 		srv := &EtcdServer{
 			Cfg:        ServerConfig{TickMs: 1},
 			r:          *r,
-			store:      st,
+			v2store:    st,
 			reqIDGen:   idutil.NewGenerator(0, time.Time{}),
 			SyncTicker: &time.Ticker{},
 		}
-		srv.applyV2 = &applierV2store{store: srv.store, cluster: srv.cluster}
+		srv.applyV2 = &applierV2store{store: srv.v2store, cluster: srv.cluster}
 		srv.start()
 		resp, err := srv.Do(context.Background(), tt)
 		srv.Stop()
@@ -718,7 +717,7 @@ func TestDoProposalCancelled(t *testing.T) {
 		w:        wt,
 		reqIDGen: idutil.NewGenerator(0, time.Time{}),
 	}
-	srv.applyV2 = &applierV2store{store: srv.store, cluster: srv.cluster}
+	srv.applyV2 = &applierV2store{store: srv.v2store, cluster: srv.cluster}
 
 	ctx, cancel := context.WithCancel(context.Background())
 	cancel()
@@ -740,7 +739,7 @@ func TestDoProposalTimeout(t *testing.T) {
 		w:        mockwait.NewNop(),
 		reqIDGen: idutil.NewGenerator(0, time.Time{}),
 	}
-	srv.applyV2 = &applierV2store{store: srv.store, cluster: srv.cluster}
+	srv.applyV2 = &applierV2store{store: srv.v2store, cluster: srv.cluster}
 
 	ctx, cancel := context.WithTimeout(context.Background(), 0)
 	_, err := srv.Do(ctx, pb.Request{Method: "PUT"})
@@ -757,7 +756,7 @@ func TestDoProposalStopped(t *testing.T) {
 		w:        mockwait.NewNop(),
 		reqIDGen: idutil.NewGenerator(0, time.Time{}),
 	}
-	srv.applyV2 = &applierV2store{store: srv.store, cluster: srv.cluster}
+	srv.applyV2 = &applierV2store{store: srv.v2store, cluster: srv.cluster}
 
 	srv.stopping = make(chan struct{})
 	close(srv.stopping)
@@ -777,7 +776,7 @@ func TestSync(t *testing.T) {
 		ctx:      ctx,
 		cancel:   cancel,
 	}
-	srv.applyV2 = &applierV2store{store: srv.store, cluster: srv.cluster}
+	srv.applyV2 = &applierV2store{store: srv.v2store, cluster: srv.cluster}
 
 	// check that sync is non-blocking
 	done := make(chan struct{})
@@ -820,7 +819,7 @@ func TestSyncTimeout(t *testing.T) {
 		ctx:      ctx,
 		cancel:   cancel,
 	}
-	srv.applyV2 = &applierV2store{store: srv.store, cluster: srv.cluster}
+	srv.applyV2 = &applierV2store{store: srv.v2store, cluster: srv.cluster}
 
 	// check that sync is non-blocking
 	done := make(chan struct{})
@@ -858,7 +857,7 @@ func TestSyncTrigger(t *testing.T) {
 	srv := &EtcdServer{
 		Cfg:        ServerConfig{TickMs: 1},
 		r:          *r,
-		store:      mockstore.NewNop(),
+		v2store:    mockstore.NewNop(),
 		SyncTicker: tk,
 		reqIDGen:   idutil.NewGenerator(0, time.Time{}),
 	}
@@ -914,8 +913,8 @@ func TestSnapshot(t *testing.T) {
 		storage:     p,
 	})
 	srv := &EtcdServer{
-		r:     *r,
-		store: st,
+		r:       *r,
+		v2store: st,
 	}
 	srv.kv = mvcc.New(be, &lease.FakeLessor{}, &srv.consistIndex)
 	srv.be = be
@@ -958,7 +957,7 @@ func TestSnapshot(t *testing.T) {
 // snapshot db is applied.
 func TestSnapshotOrdering(t *testing.T) {
 	n := newNopReadyNode()
-	st := store.New()
+	st := v2store.New()
 	cl := membership.NewCluster("abc")
 	cl.SetStore(st)
 
@@ -986,12 +985,12 @@ func TestSnapshotOrdering(t *testing.T) {
 	s := &EtcdServer{
 		Cfg:         ServerConfig{DataDir: testdir},
 		r:           *r,
-		store:       st,
+		v2store:     st,
 		snapshotter: raftsnap.New(snapdir),
 		cluster:     cl,
 		SyncTicker:  &time.Ticker{},
 	}
-	s.applyV2 = &applierV2store{store: s.store, cluster: s.cluster}
+	s.applyV2 = &applierV2store{store: s.v2store, cluster: s.cluster}
 
 	be, tmpPath := backend.NewDefaultTmpBackend()
 	defer os.RemoveAll(tmpPath)
@@ -1047,11 +1046,11 @@ func TestTriggerSnap(t *testing.T) {
 	srv := &EtcdServer{
 		Cfg:        ServerConfig{TickMs: 1, SnapCount: uint64(snapc)},
 		r:          *r,
-		store:      st,
+		v2store:    st,
 		reqIDGen:   idutil.NewGenerator(0, time.Time{}),
 		SyncTicker: &time.Ticker{},
 	}
-	srv.applyV2 = &applierV2store{store: srv.store, cluster: srv.cluster}
+	srv.applyV2 = &applierV2store{store: srv.v2store, cluster: srv.cluster}
 
 	srv.kv = mvcc.New(be, &lease.FakeLessor{}, &srv.consistIndex)
 	srv.be = be
@@ -1086,7 +1085,7 @@ func TestTriggerSnap(t *testing.T) {
 // proposals.
 func TestConcurrentApplyAndSnapshotV3(t *testing.T) {
 	n := newNopReadyNode()
-	st := store.New()
+	st := v2store.New()
 	cl := membership.NewCluster("abc")
 	cl.SetStore(st)
 
@@ -1111,12 +1110,12 @@ func TestConcurrentApplyAndSnapshotV3(t *testing.T) {
 	s := &EtcdServer{
 		Cfg:         ServerConfig{DataDir: testdir},
 		r:           *r,
-		store:       st,
+		v2store:     st,
 		snapshotter: raftsnap.New(testdir),
 		cluster:     cl,
 		SyncTicker:  &time.Ticker{},
 	}
-	s.applyV2 = &applierV2store{store: s.store, cluster: s.cluster}
+	s.applyV2 = &applierV2store{store: s.v2store, cluster: s.cluster}
 
 	be, tmpPath := backend.NewDefaultTmpBackend()
 	defer func() {
@@ -1184,7 +1183,7 @@ func TestAddMember(t *testing.T) {
 		SoftState: &raft.SoftState{RaftState: raft.StateLeader},
 	}
 	cl := newTestCluster(nil)
-	st := store.New()
+	st := v2store.New()
 	cl.SetStore(st)
 	r := newRaftNode(raftNodeConfig{
 		Node:        n,
@@ -1194,7 +1193,7 @@ func TestAddMember(t *testing.T) {
 	})
 	s := &EtcdServer{
 		r:          *r,
-		store:      st,
+		v2store:    st,
 		cluster:    cl,
 		reqIDGen:   idutil.NewGenerator(0, time.Time{}),
 		SyncTicker: &time.Ticker{},
@@ -1224,7 +1223,7 @@ func TestRemoveMember(t *testing.T) {
 		SoftState: &raft.SoftState{RaftState: raft.StateLeader},
 	}
 	cl := newTestCluster(nil)
-	st := store.New()
+	st := v2store.New()
 	cl.SetStore(v2store.New())
 	cl.AddMember(&membership.Member{ID: 1234})
 	r := newRaftNode(raftNodeConfig{
@@ -1235,7 +1234,7 @@ func TestRemoveMember(t *testing.T) {
 	})
 	s := &EtcdServer{
 		r:          *r,
-		store:      st,
+		v2store:    st,
 		cluster:    cl,
 		reqIDGen:   idutil.NewGenerator(0, time.Time{}),
 		SyncTicker: &time.Ticker{},
@@ -1264,7 +1263,7 @@ func TestUpdateMember(t *testing.T) {
 		SoftState: &raft.SoftState{RaftState: raft.StateLeader},
 	}
 	cl := newTestCluster(nil)
-	st := store.New()
+	st := v2store.New()
 	cl.SetStore(st)
 	cl.AddMember(&membership.Member{ID: 1234})
 	r := newRaftNode(raftNodeConfig{
@@ -1275,7 +1274,7 @@ func TestUpdateMember(t *testing.T) {
 	})
 	s := &EtcdServer{
 		r:          *r,
-		store:      st,
+		v2store:    st,
 		cluster:    cl,
 		reqIDGen:   idutil.NewGenerator(0, time.Time{}),
 		SyncTicker: &time.Ticker{},

+ 29 - 29
etcdserver/v2store/store_test.go

@@ -62,8 +62,8 @@ func TestStoreGetSorted(t *testing.T) {
 	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})
+	s.Create("/foo/y/a", false, "0", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
+	s.Create("/foo/y/b", false, "0", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
 	var eidx uint64 = 6
 	e, err := s.Get("/foo", true, true)
 	testutil.AssertNil(t, err)
@@ -167,7 +167,7 @@ func TestStoreCreateValue(t *testing.T) {
 
 	// Create /foo=bar
 	var eidx uint64 = 1
-	e, err := s.Create("/foo", false, "bar", false, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
+	e, err := s.Create("/foo", false, "bar", false, v2store.TTLOptionSet{ExpireTime: v2store.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, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
+	e, err = s.Create("/empty", false, "", false, v2store.TTLOptionSet{ExpireTime: v2store.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, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
+	e, err := s.Create("/foo", true, "", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
 	testutil.AssertNil(t, err)
 	testutil.AssertEqual(t, e.EtcdIndex, eidx)
 	testutil.AssertEqual(t, e.Action, "create")
@@ -218,7 +218,7 @@ func TestStoreCreateFailsIfExists(t *testing.T) {
 	s.Create("/foo", true, "", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
 
 	// create /foo as dir again
-	e, _err := s.Create("/foo", true, "", false, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
+	e, _err := s.Create("/foo", true, "", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
 	err := _err.(*v2error.Error)
 	testutil.AssertEqual(t, err.ErrorCode, v2error.EcodeNodeExist)
 	testutil.AssertEqual(t, err.Message, "Key already exists")
@@ -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, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
+	_, err = s.Create("/foo/bar", true, "", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
 	testutil.AssertNil(t, err)
 	// delete /foo with dir = true and recursive = false
 	// this should fail, since the directory is not empty
@@ -370,13 +370,13 @@ func TestRootRdOnly(t *testing.T) {
 		_, err = s.Delete(tt, true, true)
 		testutil.AssertNotNil(t, err)
 
-		_, err = s.Create(tt, true, "", false, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
+		_, err = s.Create(tt, true, "", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
 		testutil.AssertNotNil(t, err)
 
 		_, err = s.Update(tt, "", v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
 		testutil.AssertNotNil(t, err)
 
-		_, err = s.CompareAndSwap(tt, "", 0, "", v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
+		_, err = s.CompareAndSwap(tt, "", 0, "", v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
 		testutil.AssertNotNil(t, err)
 	}
 }
@@ -471,7 +471,7 @@ func TestStoreCompareAndSwapPrevValue(t *testing.T) {
 
 	var eidx uint64 = 2
 	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})
+	e, err := s.CompareAndSwap("/foo", "bar", 0, "baz", v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
 	testutil.AssertNil(t, err)
 	testutil.AssertEqual(t, e.EtcdIndex, eidx)
 	testutil.AssertEqual(t, e.Action, "compareAndSwap")
@@ -509,7 +509,7 @@ func TestStoreCompareAndSwapPrevIndex(t *testing.T) {
 	defer s.Close()
 	var eidx uint64 = 2
 	s.Create("/foo", false, "bar", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
-	e, err := s.CompareAndSwap("/foo", "", 1, "baz", v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
+	e, err := s.CompareAndSwap("/foo", "", 1, "baz", v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
 	testutil.AssertNil(t, err)
 	testutil.AssertEqual(t, e.EtcdIndex, eidx)
 	testutil.AssertEqual(t, e.Action, "compareAndSwap")
@@ -532,7 +532,7 @@ func TestStoreCompareAndSwapPrevIndexFailsIfNotMatch(t *testing.T) {
 	defer s.Close()
 	var eidx uint64 = 1
 	s.Create("/foo", false, "bar", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
-	e, _err := s.CompareAndSwap("/foo", "", 100, "baz", v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
+	e, _err := s.CompareAndSwap("/foo", "", 100, "baz", v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
 	err := _err.(*v2error.Error)
 	testutil.AssertEqual(t, err.ErrorCode, v2error.EcodeTestFailed)
 	testutil.AssertEqual(t, err.Message, "Compare failed")
@@ -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, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
+	s.Create("/foo/bar", false, "baz", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
 	e := timeoutSelect(t, w.EventChan())
 	testutil.AssertEqual(t, e.EtcdIndex, eidx)
 	testutil.AssertEqual(t, e.Action, "create")
@@ -600,7 +600,7 @@ func TestStoreWatchRecursiveUpdate(t *testing.T) {
 	s := newTestStore(t)
 	defer s.Close()
 	var eidx uint64 = 1
-	s.Create("/foo/bar", false, "baz", false, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
+	s.Create("/foo/bar", false, "baz", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
 	w, err := s.Watch("/foo", true, false, 0)
 	testutil.AssertNil(t, err)
 	testutil.AssertEqual(t, w.StartIndex(), eidx)
@@ -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, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
+	s.Create("/foo/bar", false, "baz", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
 	w, err := s.Watch("/foo", true, false, 0)
 	testutil.AssertNil(t, err)
 	testutil.AssertEqual(t, w.StartIndex(), eidx)
@@ -654,7 +654,7 @@ func TestStoreWatchCompareAndSwap(t *testing.T) {
 	w, _ := s.Watch("/foo", false, false, 0)
 	testutil.AssertEqual(t, w.StartIndex(), eidx)
 	eidx = 2
-	s.CompareAndSwap("/foo", "bar", 0, "baz", v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
+	s.CompareAndSwap("/foo", "bar", 0, "baz", v2store.TTLOptionSet{ExpireTime: v2store.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, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
+	s.Create("/foo/bar", false, "baz", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
 	w, _ := s.Watch("/foo", true, false, 0)
 	testutil.AssertEqual(t, w.StartIndex(), eidx)
 	eidx = 2
-	s.CompareAndSwap("/foo/bar", "baz", 0, "bat", v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
+	s.CompareAndSwap("/foo/bar", "baz", 0, "bat", v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
 	e := timeoutSelect(t, w.EventChan())
 	testutil.AssertEqual(t, e.EtcdIndex, eidx)
 	testutil.AssertEqual(t, e.Action, "compareAndSwap")
@@ -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, v2store.TTLOptionSet{ExpireTime:v2 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")
@@ -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, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
+	s.Create("/foo/_bar", false, "baz", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
 	e := nbselect(w.EventChan())
 	testutil.AssertNil(t, e)
 	w, _ = s.Watch("/foo", true, false, 0)
-	s.Create("/foo/_baz", true, "", false, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
+	s.Create("/foo/_baz", true, "", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
 	select {
 	case e = <-w.EventChan():
 		testutil.AssertNil(t, e)
 	case <-time.After(100 * time.Millisecond):
 	}
-	s.Create("/foo/_baz/quux", false, "quux", false, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
+	s.Create("/foo/_baz/quux", false, "quux", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
 	select {
 	case e = <-w.EventChan():
 		testutil.AssertNil(t, e)
@@ -755,7 +755,7 @@ func TestStoreWatchRecursiveCreateWithHiddenKey(t *testing.T) {
 func TestStoreWatchUpdateWithHiddenKey(t *testing.T) {
 	s := newTestStore(t)
 	defer s.Close()
-	s.Create("/_foo", false, "bar", false, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
+	s.Create("/_foo", false, "bar", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
 	w, _ := s.Watch("/_foo", false, false, 0)
 	s.Update("/_foo", "baz", v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
 	e := timeoutSelect(t, w.EventChan())
@@ -769,7 +769,7 @@ func TestStoreWatchUpdateWithHiddenKey(t *testing.T) {
 func TestStoreWatchRecursiveUpdateWithHiddenKey(t *testing.T) {
 	s := newTestStore(t)
 	defer s.Close()
-	s.Create("/foo/_bar", false, "baz", false, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
+	s.Create("/foo/_bar", false, "baz", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
 	w, _ := s.Watch("/foo", true, false, 0)
 	s.Update("/foo/_bar", "baz", v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
 	e := nbselect(w.EventChan())
@@ -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, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
+	s.Create("/_foo", false, "bar", false, v2store.TTLOptionSet{ExpireTime: v2store.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, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
+	s.Create("/foo/_bar", false, "baz", false, v2store.TTLOptionSet{ExpireTime: v2store.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, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
+	s.Create("/_foo/bar/baz", false, "baz", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
 
 	e := timeoutSelect(t, w.EventChan())
 	testutil.AssertNotNil(t, e)
@@ -841,7 +841,7 @@ func TestStoreWatchSlowConsumer(t *testing.T) {
 }
 
 // Performs a non-blocking select on an event channel.
-func nbselect(c <-chan *store.Event) *store.Event {
+func nbselect(c <-chan *v2store.Event) *v2store.Event {
 	select {
 	case e := <-c:
 		return e
@@ -851,7 +851,7 @@ func nbselect(c <-chan *store.Event) *store.Event {
 }
 
 // Performs a non-blocking select on an event channel.
-func timeoutSelect(t *testing.T, c <-chan *store.Event) *store.Event {
+func timeoutSelect(t *testing.T, c <-chan *v2store.Event) *v2store.Event {
 	select {
 	case e := <-c:
 		return e

+ 2 - 2
etcdserver/v2store/store_v2_test.go

@@ -39,9 +39,9 @@ func TestStoreRecover(t *testing.T) {
 	defer s.Close()
 	var eidx uint64 = 4
 	s.Create("/foo", true, "", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
-	s.Create("/foo/x", false, "bar", false, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
+	s.Create("/foo/x", false, "bar", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
 	s.Update("/foo/x", "barbar", v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
-	s.Create("/foo/y", false, "baz", false, v2store.TTLOptionSet{ExpireTime:v2 store.Permanent})
+	s.Create("/foo/y", false, "baz", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent})
 	b, err := s.Save()
 	testutil.AssertNil(t, err)
 

+ 8 - 8
pkg/mock/mockstore/store_recorder.go

@@ -54,28 +54,28 @@ func (s *storeRecorder) Get(path string, recursive, sorted bool) (*v2store.Event
 	})
 	return &v2store.Event{}, nil
 }
-func (s *storeRecorder) Set(path string, dir bool, val string, expireOpts store.TTLOptionSet) (*v2store.Event, error) {
+func (s *storeRecorder) Set(path string, dir bool, val string, expireOpts v2store.TTLOptionSet) (*v2store.Event, error) {
 	s.Record(testutil.Action{
 		Name:   "Set",
 		Params: []interface{}{path, dir, val, expireOpts},
 	})
 	return &v2store.Event{}, nil
 }
-func (s *storeRecorder) Update(path, val string, expireOpts store.TTLOptionSet) (*v2store.Event, error) {
+func (s *storeRecorder) Update(path, val string, expireOpts v2store.TTLOptionSet) (*v2store.Event, error) {
 	s.Record(testutil.Action{
 		Name:   "Update",
 		Params: []interface{}{path, val, expireOpts},
 	})
 	return &v2store.Event{}, nil
 }
-func (s *storeRecorder) Create(path string, dir bool, val string, uniq bool, expireOpts store.TTLOptionSet) (*v2store.Event, error) {
+func (s *storeRecorder) Create(path string, dir bool, val string, uniq bool, expireOpts v2store.TTLOptionSet) (*v2store.Event, error) {
 	s.Record(testutil.Action{
 		Name:   "Create",
 		Params: []interface{}{path, dir, val, uniq, expireOpts},
 	})
 	return &v2store.Event{}, nil
 }
-func (s *storeRecorder) CompareAndSwap(path, prevVal string, prevIdx uint64, val string, expireOpts store.TTLOptionSet) (*v2store.Event, error) {
+func (s *storeRecorder) CompareAndSwap(path, prevVal string, prevIdx uint64, val string, expireOpts v2store.TTLOptionSet) (*v2store.Event, error) {
 	s.Record(testutil.Action{
 		Name:   "CompareAndSwap",
 		Params: []interface{}{path, prevVal, prevIdx, val, expireOpts},
@@ -96,9 +96,9 @@ func (s *storeRecorder) CompareAndDelete(path, prevVal string, prevIdx uint64) (
 	})
 	return &v2store.Event{}, nil
 }
-func (s *storeRecorder) Watch(_ string, _, _ bool, _ uint64) (store.Watcher, error) {
+func (s *storeRecorder) Watch(_ string, _, _ bool, _ uint64) (v2store.Watcher, error) {
 	s.Record(testutil.Action{Name: "Watch"})
-	return store.NewNopWatcher(), nil
+	return v2store.NewNopWatcher(), nil
 }
 func (s *storeRecorder) Save() ([]byte, error) {
 	s.Record(testutil.Action{Name: "Save"})
@@ -114,7 +114,7 @@ func (s *storeRecorder) SaveNoCopy() ([]byte, error) {
 	return nil, nil
 }
 
-func (s *storeRecorder) Clone() store.Store {
+func (s *storeRecorder) Clone() v2store.Store {
 	s.Record(testutil.Action{Name: "Clone"})
 	return s
 }
@@ -151,7 +151,7 @@ func (s *errStoreRecorder) Get(path string, recursive, sorted bool) (*v2store.Ev
 	s.storeRecorder.Get(path, recursive, sorted)
 	return nil, s.err
 }
-func (s *errStoreRecorder) Watch(path string, recursive, sorted bool, index uint64) (store.Watcher, error) {
+func (s *errStoreRecorder) Watch(path string, recursive, sorted bool, index uint64) (v2store.Watcher, error) {
 	s.storeRecorder.Watch(path, recursive, sorted, index)
 	return nil, s.err
 }