From 1d3afd4bb5d3dda36267b4e2fb2cd1aa10ae42d7 Mon Sep 17 00:00:00 2001 From: Anthony Romano Date: Mon, 7 Aug 2017 00:48:05 -0700 Subject: [PATCH 01/11] etcdhttp, v2http, etcdserver: use etcdserver.{Server,ServerV2} interfaces --- etcdserver/api/cluster.go | 3 -- etcdserver/api/etcdhttp/base.go | 2 +- etcdserver/api/etcdhttp/metrics.go | 6 +-- etcdserver/api/etcdhttp/peer.go | 9 +--- etcdserver/api/etcdhttp/peer_test.go | 1 - etcdserver/api/v2http/client.go | 34 ++++++------- etcdserver/api/v2http/client_test.go | 75 +++++++++++++++------------- etcdserver/api/v2http/http_test.go | 8 +-- etcdserver/api/v3rpc/maintenance.go | 3 +- etcdserver/api/v3rpc/member.go | 14 +++--- etcdserver/server.go | 68 ++++++++++++++++--------- etcdserver/server_test.go | 3 +- etcdserver/v2_server.go | 10 +++- 13 files changed, 127 insertions(+), 109 deletions(-) diff --git a/etcdserver/api/cluster.go b/etcdserver/api/cluster.go index 87face4a139..654c25804ce 100644 --- a/etcdserver/api/cluster.go +++ b/etcdserver/api/cluster.go @@ -33,9 +33,6 @@ type Cluster interface { // Member retrieves a particular member based on ID, or nil if the // member does not exist in the cluster Member(id types.ID) *membership.Member - // IsIDRemoved checks whether the given ID has been removed from this - // cluster at some point in the past - IsIDRemoved(id types.ID) bool // Version is the cluster-wide minimum major.minor version. Version() *semver.Version } diff --git a/etcdserver/api/etcdhttp/base.go b/etcdserver/api/etcdhttp/base.go index e7dc144f696..f0d3b0bd3d2 100644 --- a/etcdserver/api/etcdhttp/base.go +++ b/etcdserver/api/etcdhttp/base.go @@ -43,7 +43,7 @@ const ( // HandleBasic adds handlers to a mux for serving JSON etcd client requests // that do not access the v2 store. -func HandleBasic(mux *http.ServeMux, server *etcdserver.EtcdServer) { +func HandleBasic(mux *http.ServeMux, server etcdserver.ServerPeer) { mux.HandleFunc(varsPath, serveVars) mux.HandleFunc(configPath+"/local/log", logHandleFunc) HandleMetricsHealth(mux, server) diff --git a/etcdserver/api/etcdhttp/metrics.go b/etcdserver/api/etcdhttp/metrics.go index 81b63605618..23e7aff86b5 100644 --- a/etcdserver/api/etcdhttp/metrics.go +++ b/etcdserver/api/etcdhttp/metrics.go @@ -33,7 +33,7 @@ const ( ) // HandleMetricsHealth registers metrics and health handlers. -func HandleMetricsHealth(mux *http.ServeMux, srv *etcdserver.EtcdServer) { +func HandleMetricsHealth(mux *http.ServeMux, srv etcdserver.ServerV2) { mux.Handle(pathMetrics, prometheus.Handler()) mux.Handle(PathHealth, NewHealthHandler(func() Health { return checkHealth(srv) })) } @@ -44,7 +44,7 @@ func HandlePrometheus(mux *http.ServeMux) { } // HandleHealth registers health handler on '/health'. -func HandleHealth(mux *http.ServeMux, srv *etcdserver.EtcdServer) { +func HandleHealth(mux *http.ServeMux, srv etcdserver.ServerV2) { mux.Handle(PathHealth, NewHealthHandler(func() Health { return checkHealth(srv) })) } @@ -74,7 +74,7 @@ type Health struct { Errors []string `json:"errors,omitempty"` } -func checkHealth(srv *etcdserver.EtcdServer) Health { +func checkHealth(srv etcdserver.ServerV2) Health { h := Health{Health: false} as := srv.Alarms() diff --git a/etcdserver/api/etcdhttp/peer.go b/etcdserver/api/etcdhttp/peer.go index 721bae3c600..0a9213b01a9 100644 --- a/etcdserver/api/etcdhttp/peer.go +++ b/etcdserver/api/etcdhttp/peer.go @@ -29,13 +29,8 @@ const ( ) // NewPeerHandler generates an http.Handler to handle etcd peer requests. -func NewPeerHandler(s *etcdserver.EtcdServer) http.Handler { - var lh http.Handler - l := s.Lessor() - if l != nil { - lh = leasehttp.NewHandler(l, func() <-chan struct{} { return s.ApplyWait() }) - } - return newPeerHandler(s.Cluster(), s.RaftHandler(), lh) +func NewPeerHandler(s etcdserver.ServerPeer) http.Handler { + return newPeerHandler(s.Cluster(), s.RaftHandler(), s.LeaseHandler()) } func newPeerHandler(cluster api.Cluster, raftHandler http.Handler, leaseHandler http.Handler) http.Handler { diff --git a/etcdserver/api/etcdhttp/peer_test.go b/etcdserver/api/etcdhttp/peer_test.go index c2b14f19541..40dbef8ae34 100644 --- a/etcdserver/api/etcdhttp/peer_test.go +++ b/etcdserver/api/etcdhttp/peer_test.go @@ -47,7 +47,6 @@ func (c *fakeCluster) Members() []*membership.Member { return []*membership.Member(ms) } func (c *fakeCluster) Member(id types.ID) *membership.Member { return c.members[uint64(id)] } -func (c *fakeCluster) IsIDRemoved(id types.ID) bool { return false } func (c *fakeCluster) Version() *semver.Version { return nil } // TestNewPeerHandlerOnRaftPrefix tests that NewPeerHandler returns a handler that diff --git a/etcdserver/api/v2http/client.go b/etcdserver/api/v2http/client.go index aa1e71ec329..c9d86508e01 100644 --- a/etcdserver/api/v2http/client.go +++ b/etcdserver/api/v2http/client.go @@ -50,22 +50,21 @@ const ( ) // NewClientHandler generates a muxed http.Handler with the given parameters to serve etcd client requests. -func NewClientHandler(server *etcdserver.EtcdServer, timeout time.Duration) http.Handler { +func NewClientHandler(server etcdserver.ServerPeer, timeout time.Duration) http.Handler { mux := http.NewServeMux() etcdhttp.HandleBasic(mux, server) handleV2(mux, server, timeout) return requestLogger(mux) } -func handleV2(mux *http.ServeMux, server *etcdserver.EtcdServer, timeout time.Duration) { +func handleV2(mux *http.ServeMux, server etcdserver.ServerV2, timeout time.Duration) { sec := auth.NewStore(server, timeout) kh := &keysHandler{ sec: sec, server: server, cluster: server.Cluster(), - timer: server, timeout: timeout, - clientCertAuthEnabled: server.Cfg.ClientCertAuthEnabled, + clientCertAuthEnabled: server.ClientCertAuthEnabled(), } sh := &statsHandler{ @@ -78,7 +77,7 @@ func handleV2(mux *http.ServeMux, server *etcdserver.EtcdServer, timeout time.Du cluster: server.Cluster(), timeout: timeout, clock: clockwork.NewRealClock(), - clientCertAuthEnabled: server.Cfg.ClientCertAuthEnabled, + clientCertAuthEnabled: server.ClientCertAuthEnabled(), } mah := &machinesHandler{cluster: server.Cluster()} @@ -86,7 +85,7 @@ func handleV2(mux *http.ServeMux, server *etcdserver.EtcdServer, timeout time.Du sech := &authHandler{ sec: sec, cluster: server.Cluster(), - clientCertAuthEnabled: server.Cfg.ClientCertAuthEnabled, + clientCertAuthEnabled: server.ClientCertAuthEnabled(), } mux.HandleFunc("/", http.NotFound) mux.Handle(keysPrefix, kh) @@ -102,9 +101,8 @@ func handleV2(mux *http.ServeMux, server *etcdserver.EtcdServer, timeout time.Du type keysHandler struct { sec auth.Store - server etcdserver.Server + server etcdserver.ServerV2 cluster api.Cluster - timer etcdserver.RaftTimer timeout time.Duration clientCertAuthEnabled bool } @@ -142,7 +140,7 @@ func (h *keysHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { } switch { case resp.Event != nil: - if err := writeKeyEvent(w, resp.Event, noValueOnSuccess, h.timer); err != nil { + if err := writeKeyEvent(w, resp, noValueOnSuccess); err != nil { // Should never be reached plog.Errorf("error writing event (%v)", err) } @@ -150,7 +148,7 @@ func (h *keysHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { case resp.Watcher != nil: ctx, cancel := context.WithTimeout(context.Background(), defaultWatchTimeout) defer cancel() - handleKeyWatch(ctx, w, resp.Watcher, rr.Stream, h.timer) + handleKeyWatch(ctx, w, resp, rr.Stream) default: writeKeyError(w, errors.New("received response with no Event/Watcher!")) } @@ -170,7 +168,7 @@ func (h *machinesHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { type membersHandler struct { sec auth.Store - server etcdserver.Server + server etcdserver.ServerV2 cluster api.Cluster timeout time.Duration clock clockwork.Clock @@ -503,14 +501,15 @@ func parseKeyRequest(r *http.Request, clock clockwork.Clock) (etcdserverpb.Reque // writeKeyEvent trims the prefix of key path in a single Event under // StoreKeysPrefix, serializes it and writes the resulting JSON to the given // ResponseWriter, along with the appropriate headers. -func writeKeyEvent(w http.ResponseWriter, ev *store.Event, noValueOnSuccess bool, rt etcdserver.RaftTimer) error { +func writeKeyEvent(w http.ResponseWriter, resp etcdserver.Response, noValueOnSuccess bool) error { + ev := resp.Event if ev == nil { return errors.New("cannot write empty Event!") } w.Header().Set("Content-Type", "application/json") w.Header().Set("X-Etcd-Index", fmt.Sprint(ev.EtcdIndex)) - w.Header().Set("X-Raft-Index", fmt.Sprint(rt.Index())) - w.Header().Set("X-Raft-Term", fmt.Sprint(rt.Term())) + w.Header().Set("X-Raft-Index", fmt.Sprint(resp.Index)) + w.Header().Set("X-Raft-Term", fmt.Sprint(resp.Term)) if ev.IsCreated() { w.WriteHeader(http.StatusCreated) @@ -552,7 +551,8 @@ func writeKeyError(w http.ResponseWriter, err error) { } } -func handleKeyWatch(ctx context.Context, w http.ResponseWriter, wa store.Watcher, stream bool, rt etcdserver.RaftTimer) { +func handleKeyWatch(ctx context.Context, w http.ResponseWriter, resp etcdserver.Response, stream bool) { + wa := resp.Watcher defer wa.Remove() ech := wa.EventChan() var nch <-chan bool @@ -562,8 +562,8 @@ func handleKeyWatch(ctx context.Context, w http.ResponseWriter, wa store.Watcher w.Header().Set("Content-Type", "application/json") w.Header().Set("X-Etcd-Index", fmt.Sprint(wa.StartIndex())) - w.Header().Set("X-Raft-Index", fmt.Sprint(rt.Index())) - w.Header().Set("X-Raft-Term", fmt.Sprint(rt.Term())) + w.Header().Set("X-Raft-Index", fmt.Sprint(resp.Index)) + w.Header().Set("X-Raft-Term", fmt.Sprint(resp.Term)) w.WriteHeader(http.StatusOK) // Ensure headers are flushed early, in case of long polling diff --git a/etcdserver/api/v2http/client_test.go b/etcdserver/api/v2http/client_test.go index 896021b22f5..0a58082a41b 100644 --- a/etcdserver/api/v2http/client_test.go +++ b/etcdserver/api/v2http/client_test.go @@ -30,6 +30,7 @@ import ( etcdErr "github.com/coreos/etcd/error" "github.com/coreos/etcd/etcdserver" + "github.com/coreos/etcd/etcdserver/api" "github.com/coreos/etcd/etcdserver/api/v2http/httptypes" "github.com/coreos/etcd/etcdserver/etcdserverpb" "github.com/coreos/etcd/etcdserver/membership" @@ -87,14 +88,26 @@ func mustNewMethodRequest(t *testing.T, m, p string) *http.Request { } } +type fakeServer struct { + dummyRaftTimer + dummyStats +} + +func (s *fakeServer) Leader() types.ID { return types.ID(1) } +func (s *fakeServer) Alarms() []*etcdserverpb.AlarmMember { return nil } +func (s *fakeServer) Cluster() api.Cluster { return nil } +func (s *fakeServer) ClusterVersion() *semver.Version { return nil } +func (s *fakeServer) RaftHandler() http.Handler { return nil } +func (s *fakeServer) Do(ctx context.Context, r etcdserverpb.Request) (rr etcdserver.Response, err error) { + return +} +func (s *fakeServer) ClientCertAuthEnabled() bool { return false } + type serverRecorder struct { + fakeServer actions []action } -func (s *serverRecorder) Start() {} -func (s *serverRecorder) Stop() {} -func (s *serverRecorder) Leader() types.ID { return types.ID(1) } -func (s *serverRecorder) ID() types.ID { return types.ID(1) } func (s *serverRecorder) Do(_ context.Context, r etcdserverpb.Request) (etcdserver.Response, error) { s.actions = append(s.actions, action{name: "Do", params: []interface{}{r}}) return etcdserver.Response{}, nil @@ -117,8 +130,6 @@ func (s *serverRecorder) UpdateMember(_ context.Context, m membership.Member) ([ return nil, nil } -func (s *serverRecorder) ClusterVersion() *semver.Version { return nil } - type action struct { name string params []interface{} @@ -138,13 +149,10 @@ func (fr *flushingRecorder) Flush() { // resServer implements the etcd.Server interface for testing. // It returns the given response from any Do calls, and nil error type resServer struct { + fakeServer res etcdserver.Response } -func (rs *resServer) Start() {} -func (rs *resServer) Stop() {} -func (rs *resServer) ID() types.ID { return types.ID(1) } -func (rs *resServer) Leader() types.ID { return types.ID(1) } func (rs *resServer) Do(_ context.Context, _ etcdserverpb.Request) (etcdserver.Response, error) { return rs.res, nil } @@ -158,7 +166,6 @@ func (rs *resServer) RemoveMember(_ context.Context, _ uint64) ([]*membership.Me func (rs *resServer) UpdateMember(_ context.Context, _ membership.Member) ([]*membership.Member, error) { return nil, nil } -func (rs *resServer) ClusterVersion() *semver.Version { return nil } func boolp(b bool) *bool { return &b } @@ -874,7 +881,7 @@ func TestServeMembersUpdate(t *testing.T) { func TestServeMembersFail(t *testing.T) { tests := []struct { req *http.Request - server etcdserver.Server + server etcdserver.ServerV2 wcode int }{ @@ -941,7 +948,7 @@ func TestServeMembersFail(t *testing.T) { Header: map[string][]string{"Content-Type": {"application/json"}}, }, &errServer{ - errors.New("Error while adding a member"), + err: errors.New("Error while adding a member"), }, http.StatusInternalServerError, @@ -955,7 +962,7 @@ func TestServeMembersFail(t *testing.T) { Header: map[string][]string{"Content-Type": {"application/json"}}, }, &errServer{ - membership.ErrIDExists, + err: membership.ErrIDExists, }, http.StatusConflict, @@ -969,7 +976,7 @@ func TestServeMembersFail(t *testing.T) { Header: map[string][]string{"Content-Type": {"application/json"}}, }, &errServer{ - membership.ErrPeerURLexists, + err: membership.ErrPeerURLexists, }, http.StatusConflict, @@ -981,7 +988,7 @@ func TestServeMembersFail(t *testing.T) { Method: "DELETE", }, &errServer{ - errors.New("Error while removing member"), + err: errors.New("Error while removing member"), }, http.StatusInternalServerError, @@ -993,7 +1000,7 @@ func TestServeMembersFail(t *testing.T) { Method: "DELETE", }, &errServer{ - membership.ErrIDRemoved, + err: membership.ErrIDRemoved, }, http.StatusGone, @@ -1005,7 +1012,7 @@ func TestServeMembersFail(t *testing.T) { Method: "DELETE", }, &errServer{ - membership.ErrIDNotFound, + err: membership.ErrIDNotFound, }, http.StatusNotFound, @@ -1075,7 +1082,7 @@ func TestServeMembersFail(t *testing.T) { Header: map[string][]string{"Content-Type": {"application/json"}}, }, &errServer{ - errors.New("blah"), + err: errors.New("blah"), }, http.StatusInternalServerError, @@ -1089,7 +1096,7 @@ func TestServeMembersFail(t *testing.T) { Header: map[string][]string{"Content-Type": {"application/json"}}, }, &errServer{ - membership.ErrPeerURLexists, + err: membership.ErrPeerURLexists, }, http.StatusConflict, @@ -1103,7 +1110,7 @@ func TestServeMembersFail(t *testing.T) { Header: map[string][]string{"Content-Type": {"application/json"}}, }, &errServer{ - membership.ErrIDNotFound, + err: membership.ErrIDNotFound, }, http.StatusNotFound, @@ -1153,7 +1160,7 @@ func TestServeMembersFail(t *testing.T) { func TestWriteEvent(t *testing.T) { // nil event should not panic rec := httptest.NewRecorder() - writeKeyEvent(rec, nil, false, dummyRaftTimer{}) + writeKeyEvent(rec, etcdserver.Response{}, false) h := rec.Header() if len(h) > 0 { t.Fatalf("unexpected non-empty headers: %#v", h) @@ -1199,7 +1206,8 @@ func TestWriteEvent(t *testing.T) { for i, tt := range tests { rw := httptest.NewRecorder() - writeKeyEvent(rw, tt.ev, tt.noValue, dummyRaftTimer{}) + resp := etcdserver.Response{Event: tt.ev, Term: 5, Index: 100} + writeKeyEvent(rw, resp, tt.noValue) if gct := rw.Header().Get("Content-Type"); gct != "application/json" { t.Errorf("case %d: bad Content-Type: got %q, want application/json", i, gct) } @@ -1411,7 +1419,7 @@ func TestServeStoreStats(t *testing.T) { func TestBadServeKeys(t *testing.T) { testBadCases := []struct { req *http.Request - server etcdserver.Server + server etcdserver.ServerV2 wcode int wbody string @@ -1451,7 +1459,7 @@ func TestBadServeKeys(t *testing.T) { // etcdserver.Server error mustNewRequest(t, "foo"), &errServer{ - errors.New("Internal Server Error"), + err: errors.New("Internal Server Error"), }, http.StatusInternalServerError, @@ -1461,7 +1469,7 @@ func TestBadServeKeys(t *testing.T) { // etcdserver.Server etcd error mustNewRequest(t, "foo"), &errServer{ - etcdErr.NewError(etcdErr.EcodeKeyNotFound, "/1/pant", 0), + err: etcdErr.NewError(etcdErr.EcodeKeyNotFound, "/1/pant", 0), }, http.StatusNotFound, @@ -1471,7 +1479,7 @@ func TestBadServeKeys(t *testing.T) { // non-event/watcher response from etcdserver.Server mustNewRequest(t, "foo"), &resServer{ - etcdserver.Response{}, + res: etcdserver.Response{}, }, http.StatusInternalServerError, @@ -1529,7 +1537,7 @@ func TestServeKeysGood(t *testing.T) { }, } server := &resServer{ - etcdserver.Response{ + res: etcdserver.Response{ Event: &store.Event{ Action: store.Get, Node: &store.NodeExtern{}, @@ -1540,7 +1548,6 @@ func TestServeKeysGood(t *testing.T) { h := &keysHandler{ timeout: time.Hour, server: server, - timer: &dummyRaftTimer{}, cluster: &fakeCluster{id: 1}, } rw := httptest.NewRecorder() @@ -1597,7 +1604,6 @@ func TestServeKeysEvent(t *testing.T) { timeout: time.Hour, server: server, cluster: &fakeCluster{id: 1}, - timer: &dummyRaftTimer{}, } for _, tt := range tests { @@ -1632,7 +1638,7 @@ func TestServeKeysWatch(t *testing.T) { echan: ec, } server := &resServer{ - etcdserver.Response{ + res: etcdserver.Response{ Watcher: dw, }, } @@ -1640,7 +1646,6 @@ func TestServeKeysWatch(t *testing.T) { timeout: time.Hour, server: server, cluster: &fakeCluster{id: 1}, - timer: &dummyRaftTimer{}, } go func() { ec <- &store.Event{ @@ -1764,7 +1769,8 @@ func TestHandleWatch(t *testing.T) { } tt.doToChan(wa.echan) - handleKeyWatch(tt.getCtx(), rw, wa, false, dummyRaftTimer{}) + resp := etcdserver.Response{Term: 5, Index: 100, Watcher: wa} + handleKeyWatch(tt.getCtx(), rw, resp, false) wcode := http.StatusOK wct := "application/json" @@ -1808,7 +1814,8 @@ func TestHandleWatchStreaming(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) done := make(chan struct{}) go func() { - handleKeyWatch(ctx, rw, wa, true, dummyRaftTimer{}) + resp := etcdserver.Response{Watcher: wa} + handleKeyWatch(ctx, rw, resp, true) close(done) }() diff --git a/etcdserver/api/v2http/http_test.go b/etcdserver/api/v2http/http_test.go index 61225409e6e..5b68a49a540 100644 --- a/etcdserver/api/v2http/http_test.go +++ b/etcdserver/api/v2http/http_test.go @@ -48,19 +48,15 @@ func (c *fakeCluster) Members() []*membership.Member { return []*membership.Member(ms) } func (c *fakeCluster) Member(id types.ID) *membership.Member { return c.members[uint64(id)] } -func (c *fakeCluster) IsIDRemoved(id types.ID) bool { return false } func (c *fakeCluster) Version() *semver.Version { return nil } // errServer implements the etcd.Server interface for testing. // It returns the given error from any Do/Process/AddMember/RemoveMember calls. type errServer struct { err error + fakeServer } -func (fs *errServer) Start() {} -func (fs *errServer) Stop() {} -func (fs *errServer) ID() types.ID { return types.ID(1) } -func (fs *errServer) Leader() types.ID { return types.ID(1) } func (fs *errServer) Do(ctx context.Context, r etcdserverpb.Request) (etcdserver.Response, error) { return etcdserver.Response{}, fs.err } @@ -77,8 +73,6 @@ func (fs *errServer) UpdateMember(ctx context.Context, m membership.Member) ([]* return nil, fs.err } -func (fs *errServer) ClusterVersion() *semver.Version { return nil } - func TestWriteError(t *testing.T) { // nil error should not panic rec := httptest.NewRecorder() diff --git a/etcdserver/api/v3rpc/maintenance.go b/etcdserver/api/v3rpc/maintenance.go index fdbb118ce01..a88aec9952d 100644 --- a/etcdserver/api/v3rpc/maintenance.go +++ b/etcdserver/api/v3rpc/maintenance.go @@ -46,8 +46,7 @@ type LeaderTransferrer interface { } type RaftStatusGetter interface { - Index() uint64 - Term() uint64 + etcdserver.RaftTimer ID() types.ID Leader() types.ID } diff --git a/etcdserver/api/v3rpc/member.go b/etcdserver/api/v3rpc/member.go index 91a59389b87..ff271e8ddba 100644 --- a/etcdserver/api/v3rpc/member.go +++ b/etcdserver/api/v3rpc/member.go @@ -27,16 +27,14 @@ import ( ) type ClusterServer struct { - cluster api.Cluster - server etcdserver.Server - raftTimer etcdserver.RaftTimer + cluster api.Cluster + server etcdserver.ServerV3 } -func NewClusterServer(s *etcdserver.EtcdServer) *ClusterServer { +func NewClusterServer(s etcdserver.ServerV3) *ClusterServer { return &ClusterServer{ - cluster: s.Cluster(), - server: s, - raftTimer: s, + cluster: s.Cluster(), + server: s, } } @@ -86,7 +84,7 @@ func (cs *ClusterServer) MemberList(ctx context.Context, r *pb.MemberListRequest } func (cs *ClusterServer) header() *pb.ResponseHeader { - return &pb.ResponseHeader{ClusterId: uint64(cs.cluster.ID()), MemberId: uint64(cs.server.ID()), RaftTerm: cs.raftTimer.Term()} + return &pb.ResponseHeader{ClusterId: uint64(cs.cluster.ID()), MemberId: uint64(cs.server.ID()), RaftTerm: cs.server.Term()} } func membersToProtoMembers(membs []*membership.Member) []*pb.Member { diff --git a/etcdserver/server.go b/etcdserver/server.go index ac7601ecb0e..358f25a76c6 100644 --- a/etcdserver/server.go +++ b/etcdserver/server.go @@ -38,6 +38,7 @@ import ( "github.com/coreos/etcd/etcdserver/membership" "github.com/coreos/etcd/etcdserver/stats" "github.com/coreos/etcd/lease" + "github.com/coreos/etcd/lease/leasehttp" "github.com/coreos/etcd/mvcc" "github.com/coreos/etcd/mvcc/backend" "github.com/coreos/etcd/pkg/fileutil" @@ -108,29 +109,33 @@ func init() { } type Response struct { + Term uint64 + Index uint64 Event *store.Event Watcher store.Watcher err error } -type Server interface { - // Start performs any initialization of the Server necessary for it to - // begin serving requests. It must be called before Do or Process. - // Start must be non-blocking; any long-running server functionality - // should be implemented in goroutines. - Start() - // Stop terminates the Server and performs any necessary finalization. - // Do and Process cannot be called after Stop has been invoked. - Stop() - // ID returns the ID of the Server. +type ServerV2 interface { + Server + // Do takes a V2 request and attempts to fulfill it, returning a Response. + Do(ctx context.Context, r pb.Request) (Response, error) + stats.Stats + ClientCertAuthEnabled() bool +} + +type ServerV3 interface { + Server ID() types.ID + RaftTimer +} + +func (s *EtcdServer) ClientCertAuthEnabled() bool { return s.Cfg.ClientCertAuthEnabled } + +type Server interface { // Leader returns the ID of the leader Server. Leader() types.ID - // Do takes a request and attempts to fulfill it, returning a Response. - Do(ctx context.Context, r pb.Request) (Response, error) - // Process takes a raft message and applies it to the server's raft state - // machine, respecting any timeout of the given context. - Process(ctx context.Context, m raftpb.Message) error + // AddMember attempts to add a member into the cluster. It will return // ErrIDRemoved if member ID is removed from the cluster, or return // ErrIDExists if member ID exists in the cluster. @@ -139,7 +144,6 @@ type Server interface { // return ErrIDRemoved if member ID is removed from the cluster, or return // ErrIDNotFound if member ID is not in the cluster. RemoveMember(ctx context.Context, id uint64) ([]*membership.Member, error) - // UpdateMember attempts to update an existing member in the cluster. It will // return ErrIDNotFound if the member ID does not exist. UpdateMember(ctx context.Context, updateMemb membership.Member) ([]*membership.Member, error) @@ -159,6 +163,8 @@ type Server interface { // the leader is etcd 2.0. etcd 2.0 leader will not update clusterVersion since // this feature is introduced post 2.0. ClusterVersion() *semver.Version + Cluster() api.Cluster + Alarms() []*pb.AlarmMember } // EtcdServer is the production implementation of the Server interface @@ -514,9 +520,10 @@ func NewServer(cfg ServerConfig) (srv *EtcdServer, err error) { return srv, nil } -// Start prepares and starts server in a new goroutine. It is no longer safe to -// modify a server's fields after it has been sent to Start. -// It also starts a goroutine to publish its server information. +// Start performs any initialization of the Server necessary for it to +// begin serving requests. It must be called before Do or Process. +// Start must be non-blocking; any long-running server functionality +// should be implemented in goroutines. func (s *EtcdServer) Start() { s.start() s.goAttach(func() { s.publish(s.Cfg.ReqTimeout()) }) @@ -576,14 +583,27 @@ func (s *EtcdServer) purgeFile() { func (s *EtcdServer) ID() types.ID { return s.id } -func (s *EtcdServer) Cluster() *membership.RaftCluster { return s.cluster } +func (s *EtcdServer) Cluster() api.Cluster { return s.cluster } -func (s *EtcdServer) RaftHandler() http.Handler { return s.r.transport.Handler() } +func (s *EtcdServer) ApplyWait() <-chan struct{} { return s.applyWait.Wait(s.getCommittedIndex()) } -func (s *EtcdServer) Lessor() lease.Lessor { return s.lessor } +type ServerPeer interface { + ServerV2 + RaftHandler() http.Handler + LeaseHandler() http.Handler +} -func (s *EtcdServer) ApplyWait() <-chan struct{} { return s.applyWait.Wait(s.getCommittedIndex()) } +func (s *EtcdServer) LeaseHandler() http.Handler { + if s.lessor == nil { + return nil + } + return leasehttp.NewHandler(s.lessor, s.ApplyWait) +} + +func (s *EtcdServer) RaftHandler() http.Handler { return s.r.transport.Handler() } +// Process takes a raft message and applies it to the server's raft state +// machine, respecting any timeout of the given context. func (s *EtcdServer) Process(ctx context.Context, m raftpb.Message) error { if s.cluster.IsIDRemoved(types.ID(m.From)) { plog.Warningf("reject message from removed member %s", types.ID(m.From).String()) @@ -992,6 +1012,8 @@ func (s *EtcdServer) HardStop() { // Stop should be called after a Start(s), otherwise it will block forever. // When stopping leader, Stop transfers its leadership to one of its peers // before stopping the server. +// Stop terminates the Server and performs any necessary finalization. +// Do and Process cannot be called after Stop has been invoked. func (s *EtcdServer) Stop() { if err := s.TransferLeadership(); err != nil { plog.Warningf("%s failed to transfer leadership (%v)", s.ID(), err) diff --git a/etcdserver/server_test.go b/etcdserver/server_test.go index 66c99247bb5..d17fab9c877 100644 --- a/etcdserver/server_test.go +++ b/etcdserver/server_test.go @@ -701,7 +701,8 @@ func TestDoProposal(t *testing.T) { if err != nil { t.Fatalf("#%d: err = %v, want nil", i, err) } - wresp := Response{Event: &store.Event{}} + // 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} if !reflect.DeepEqual(resp, wresp) { t.Errorf("#%d: resp = %v, want %v", i, resp, wresp) } diff --git a/etcdserver/v2_server.go b/etcdserver/v2_server.go index 72c4eb7c5cc..b0a64ad62fa 100644 --- a/etcdserver/v2_server.go +++ b/etcdserver/v2_server.go @@ -96,12 +96,18 @@ func (a *v2apiStore) Head(ctx context.Context, r *pb.Request) (Response, error) return Response{Event: ev}, nil } -// Do interprets r and performs an operation on s.store according to r.Method +func (s *EtcdServer) Do(ctx context.Context, r pb.Request) (Response, error) { + resp, err := s.do(ctx, r) + resp.Term, resp.Index = s.Term(), s.Index() + return resp, err +} + +// do interprets r and performs an operation on s.store according to r.Method // and other fields. If r.Method is "POST", "PUT", "DELETE", or a "GET" with // Quorum == true, r will be sent through consensus before performing its // respective operation. Do will block until an action is performed or there is // an error. -func (s *EtcdServer) Do(ctx context.Context, r pb.Request) (Response, error) { +func (s *EtcdServer) do(ctx context.Context, r pb.Request) (Response, error) { r.ID = s.reqIDGen.Next() if r.Method == "GET" && r.Quorum { r.Method = "QGET" From 758c3c09fd2321c2fd7287db2ed597258e806744 Mon Sep 17 00:00:00 2001 From: Anthony Romano Date: Mon, 7 Aug 2017 03:55:18 -0700 Subject: [PATCH 02/11] etcdserver: refactor v2 request processing Makes interfaces more reusable. --- etcdserver/apply_v2.go | 34 +++++----- etcdserver/server.go | 9 +-- etcdserver/server_test.go | 6 +- etcdserver/v2_server.go | 127 +++++++++++++++++++++++--------------- 4 files changed, 102 insertions(+), 74 deletions(-) diff --git a/etcdserver/apply_v2.go b/etcdserver/apply_v2.go index f278efca88e..aefe19d26bd 100644 --- a/etcdserver/apply_v2.go +++ b/etcdserver/apply_v2.go @@ -20,7 +20,6 @@ import ( "time" "github.com/coreos/etcd/etcdserver/api" - pb "github.com/coreos/etcd/etcdserver/etcdserverpb" "github.com/coreos/etcd/etcdserver/membership" "github.com/coreos/etcd/pkg/pbutil" "github.com/coreos/etcd/store" @@ -29,11 +28,11 @@ import ( // ApplierV2 is the interface for processing V2 raft messages type ApplierV2 interface { - Delete(r *pb.Request) Response - Post(r *pb.Request) Response - Put(r *pb.Request) Response - QGet(r *pb.Request) Response - Sync(r *pb.Request) Response + Delete(r *RequestV2) Response + Post(r *RequestV2) Response + Put(r *RequestV2) Response + QGet(r *RequestV2) Response + Sync(r *RequestV2) Response } func NewApplierV2(s store.Store, c *membership.RaftCluster) ApplierV2 { @@ -45,7 +44,7 @@ type applierV2store struct { cluster *membership.RaftCluster } -func (a *applierV2store) Delete(r *pb.Request) Response { +func (a *applierV2store) Delete(r *RequestV2) Response { switch { case r.PrevIndex > 0 || r.PrevValue != "": return toResponse(a.store.CompareAndDelete(r.Path, r.PrevValue, r.PrevIndex)) @@ -54,12 +53,12 @@ func (a *applierV2store) Delete(r *pb.Request) Response { } } -func (a *applierV2store) Post(r *pb.Request) Response { - return toResponse(a.store.Create(r.Path, r.Dir, r.Val, true, toTTLOptions(r))) +func (a *applierV2store) Post(r *RequestV2) Response { + return toResponse(a.store.Create(r.Path, r.Dir, r.Val, true, r.TTLOptions())) } -func (a *applierV2store) Put(r *pb.Request) Response { - ttlOptions := toTTLOptions(r) +func (a *applierV2store) Put(r *RequestV2) Response { + ttlOptions := r.TTLOptions() exists, existsSet := pbutil.GetBool(r.PrevExist) switch { case existsSet: @@ -96,19 +95,18 @@ func (a *applierV2store) Put(r *pb.Request) Response { } } -func (a *applierV2store) QGet(r *pb.Request) Response { +func (a *applierV2store) QGet(r *RequestV2) Response { return toResponse(a.store.Get(r.Path, r.Recursive, r.Sorted)) } -func (a *applierV2store) Sync(r *pb.Request) Response { +func (a *applierV2store) Sync(r *RequestV2) Response { a.store.DeleteExpiredKeys(time.Unix(0, r.Time)) return Response{} } // applyV2Request interprets r as a call to store.X and returns a Response interpreted // from store.Event -func (s *EtcdServer) applyV2Request(r *pb.Request) Response { - toTTLOptions(r) +func (s *EtcdServer) applyV2Request(r *RequestV2) Response { switch r.Method { case "POST": return s.applyV2.Post(r) @@ -122,11 +120,11 @@ func (s *EtcdServer) applyV2Request(r *pb.Request) Response { return s.applyV2.Sync(r) default: // This should never be reached, but just in case: - return Response{err: ErrUnknownMethod} + return Response{Err: ErrUnknownMethod} } } -func toTTLOptions(r *pb.Request) store.TTLOptionSet { +func (r *RequestV2) TTLOptions() store.TTLOptionSet { refresh, _ := pbutil.GetBool(r.Refresh) ttlOptions := store.TTLOptionSet{Refresh: refresh} if r.Expiration != 0 { @@ -136,5 +134,5 @@ func toTTLOptions(r *pb.Request) store.TTLOptionSet { } func toResponse(ev *store.Event, err error) Response { - return Response{Event: ev, err: err} + return Response{Event: ev, Err: err} } diff --git a/etcdserver/server.go b/etcdserver/server.go index 358f25a76c6..ffeaf88cc42 100644 --- a/etcdserver/server.go +++ b/etcdserver/server.go @@ -113,7 +113,7 @@ type Response struct { Index uint64 Event *store.Event Watcher store.Watcher - err error + Err error } type ServerV2 interface { @@ -1344,12 +1344,13 @@ func (s *EtcdServer) applyEntryNormal(e *raftpb.Entry) { var raftReq pb.InternalRaftRequest if !pbutil.MaybeUnmarshal(&raftReq, e.Data) { // backward compatible var r pb.Request - pbutil.MustUnmarshal(&r, e.Data) - s.w.Trigger(r.ID, s.applyV2Request(&r)) + rp := &r + pbutil.MustUnmarshal(rp, e.Data) + s.w.Trigger(r.ID, s.applyV2Request((*RequestV2)(rp))) return } if raftReq.V2 != nil { - req := raftReq.V2 + req := (*RequestV2)(raftReq.V2) s.w.Trigger(req.ID, s.applyV2Request(req)) return } diff --git a/etcdserver/server_test.go b/etcdserver/server_test.go index d17fab9c877..c2bd0045f3e 100644 --- a/etcdserver/server_test.go +++ b/etcdserver/server_test.go @@ -441,7 +441,7 @@ func TestApplyRequest(t *testing.T) { // Unknown method - error { pb.Request{Method: "BADMETHOD", ID: 1}, - Response{err: ErrUnknownMethod}, + Response{Err: ErrUnknownMethod}, []testutil.Action{}, }, } @@ -450,7 +450,7 @@ func TestApplyRequest(t *testing.T) { st := mockstore.NewRecorder() srv := &EtcdServer{store: st} srv.applyV2 = &applierV2store{store: srv.store, cluster: srv.cluster} - resp := srv.applyV2Request(&tt.req) + resp := srv.applyV2Request((*RequestV2)(&tt.req)) if !reflect.DeepEqual(resp, tt.wresp) { t.Errorf("#%d: resp = %+v, want %+v", i, resp, tt.wresp) @@ -476,7 +476,7 @@ func TestApplyRequestOnAdminMemberAttributes(t *testing.T) { Path: membership.MemberAttributesStorePath(1), Val: `{"Name":"abc","ClientURLs":["http://127.0.0.1:2379"]}`, } - srv.applyV2Request(&req) + srv.applyV2Request((*RequestV2)(&req)) w := membership.Attributes{Name: "abc", ClientURLs: []string{"http://127.0.0.1:2379"}} if g := cl.Member(1).Attributes; !reflect.DeepEqual(g, w) { t.Errorf("attributes = %v, want %v", g, w) diff --git a/etcdserver/v2_server.go b/etcdserver/v2_server.go index b0a64ad62fa..6c4aa8baf19 100644 --- a/etcdserver/v2_server.go +++ b/etcdserver/v2_server.go @@ -18,38 +18,83 @@ import ( "time" pb "github.com/coreos/etcd/etcdserver/etcdserverpb" + "github.com/coreos/etcd/store" "golang.org/x/net/context" ) -type v2API interface { - Post(ctx context.Context, r *pb.Request) (Response, error) - Put(ctx context.Context, r *pb.Request) (Response, error) - Delete(ctx context.Context, r *pb.Request) (Response, error) - QGet(ctx context.Context, r *pb.Request) (Response, error) - Get(ctx context.Context, r *pb.Request) (Response, error) - Head(ctx context.Context, r *pb.Request) (Response, error) +type RequestV2 pb.Request + +type RequestV2Handler interface { + Post(ctx context.Context, r *RequestV2) (Response, error) + Put(ctx context.Context, r *RequestV2) (Response, error) + Delete(ctx context.Context, r *RequestV2) (Response, error) + QGet(ctx context.Context, r *RequestV2) (Response, error) + Get(ctx context.Context, r *RequestV2) (Response, error) + Head(ctx context.Context, r *RequestV2) (Response, error) +} + +type reqV2HandlerEtcdServer struct { + reqV2HandlerStore + s *EtcdServer +} + +type reqV2HandlerStore struct { + store store.Store + applier ApplierV2 +} + +func NewStoreRequestV2Handler(s store.Store, applier ApplierV2) RequestV2Handler { + return &reqV2HandlerStore{s, applier} +} + +func (a *reqV2HandlerStore) Post(ctx context.Context, r *RequestV2) (Response, error) { + return a.applier.Post(r), nil } -type v2apiStore struct{ s *EtcdServer } +func (a *reqV2HandlerStore) Put(ctx context.Context, r *RequestV2) (Response, error) { + return a.applier.Put(r), nil +} + +func (a *reqV2HandlerStore) Delete(ctx context.Context, r *RequestV2) (Response, error) { + return a.applier.Delete(r), nil +} + +func (a *reqV2HandlerStore) QGet(ctx context.Context, r *RequestV2) (Response, error) { + return a.applier.QGet(r), nil +} + +func (a *reqV2HandlerStore) Get(ctx context.Context, r *RequestV2) (Response, error) { + if r.Wait { + wc, err := a.store.Watch(r.Path, r.Recursive, r.Stream, r.Since) + return Response{Watcher: wc}, err + } + ev, err := a.store.Get(r.Path, r.Recursive, r.Sorted) + return Response{Event: ev}, err +} -func (a *v2apiStore) Post(ctx context.Context, r *pb.Request) (Response, error) { +func (a *reqV2HandlerStore) Head(ctx context.Context, r *RequestV2) (Response, error) { + ev, err := a.store.Get(r.Path, r.Recursive, r.Sorted) + return Response{Event: ev}, err +} + +func (a *reqV2HandlerEtcdServer) Post(ctx context.Context, r *RequestV2) (Response, error) { return a.processRaftRequest(ctx, r) } -func (a *v2apiStore) Put(ctx context.Context, r *pb.Request) (Response, error) { +func (a *reqV2HandlerEtcdServer) Put(ctx context.Context, r *RequestV2) (Response, error) { return a.processRaftRequest(ctx, r) } -func (a *v2apiStore) Delete(ctx context.Context, r *pb.Request) (Response, error) { +func (a *reqV2HandlerEtcdServer) Delete(ctx context.Context, r *RequestV2) (Response, error) { return a.processRaftRequest(ctx, r) } -func (a *v2apiStore) QGet(ctx context.Context, r *pb.Request) (Response, error) { +func (a *reqV2HandlerEtcdServer) QGet(ctx context.Context, r *RequestV2) (Response, error) { return a.processRaftRequest(ctx, r) } -func (a *v2apiStore) processRaftRequest(ctx context.Context, r *pb.Request) (Response, error) { - data, err := r.Marshal() +func (a *reqV2HandlerEtcdServer) processRaftRequest(ctx context.Context, r *RequestV2) (Response, error) { + data, err := ((*pb.Request)(r)).Marshal() if err != nil { return Response{}, err } @@ -63,7 +108,7 @@ func (a *v2apiStore) processRaftRequest(ctx context.Context, r *pb.Request) (Res select { case x := <-ch: resp := x.(Response) - return resp, resp.err + return resp, resp.Err case <-ctx.Done(): proposalsFailed.Inc() a.s.w.Trigger(r.ID, nil) // GC wait @@ -73,59 +118,43 @@ func (a *v2apiStore) processRaftRequest(ctx context.Context, r *pb.Request) (Res return Response{}, ErrStopped } -func (a *v2apiStore) Get(ctx context.Context, r *pb.Request) (Response, error) { - if r.Wait { - wc, err := a.s.store.Watch(r.Path, r.Recursive, r.Stream, r.Since) - if err != nil { - return Response{}, err - } - return Response{Watcher: wc}, nil - } - ev, err := a.s.store.Get(r.Path, r.Recursive, r.Sorted) - if err != nil { - return Response{}, err - } - return Response{Event: ev}, nil -} - -func (a *v2apiStore) Head(ctx context.Context, r *pb.Request) (Response, error) { - ev, err := a.s.store.Get(r.Path, r.Recursive, r.Sorted) - if err != nil { - return Response{}, err - } - return Response{Event: ev}, nil -} - func (s *EtcdServer) Do(ctx context.Context, r pb.Request) (Response, error) { - resp, err := s.do(ctx, r) + r.ID = s.reqIDGen.Next() + h := &reqV2HandlerEtcdServer{ + reqV2HandlerStore: reqV2HandlerStore{ + store: s.store, + applier: s.applyV2, + }, + s: s, + } + rp := &r + resp, err := ((*RequestV2)(rp)).Handle(ctx, h) resp.Term, resp.Index = s.Term(), s.Index() return resp, err } -// do interprets r and performs an operation on s.store according to r.Method +// Handle interprets r and performs an operation on s.store according to r.Method // and other fields. If r.Method is "POST", "PUT", "DELETE", or a "GET" with // Quorum == true, r will be sent through consensus before performing its // respective operation. Do will block until an action is performed or there is // an error. -func (s *EtcdServer) do(ctx context.Context, r pb.Request) (Response, error) { - r.ID = s.reqIDGen.Next() +func (r *RequestV2) Handle(ctx context.Context, v2api RequestV2Handler) (Response, error) { if r.Method == "GET" && r.Quorum { r.Method = "QGET" } - v2api := (v2API)(&v2apiStore{s}) switch r.Method { case "POST": - return v2api.Post(ctx, &r) + return v2api.Post(ctx, r) case "PUT": - return v2api.Put(ctx, &r) + return v2api.Put(ctx, r) case "DELETE": - return v2api.Delete(ctx, &r) + return v2api.Delete(ctx, r) case "QGET": - return v2api.QGet(ctx, &r) + return v2api.QGet(ctx, r) case "GET": - return v2api.Get(ctx, &r) + return v2api.Get(ctx, r) case "HEAD": - return v2api.Head(ctx, &r) + return v2api.Head(ctx, r) } return Response{}, ErrUnknownMethod } From 525fbba1bda3e9a123cf231d43fcc417fd98482a Mon Sep 17 00:00:00 2001 From: Anthony Romano Date: Sun, 13 Aug 2017 17:38:12 -0700 Subject: [PATCH 03/11] etcdctl3: update to use RequestV2 instead of Request --- etcdctl/ctlv3/command/migrate_command.go | 14 +++----------- 1 file changed, 3 insertions(+), 11 deletions(-) diff --git a/etcdctl/ctlv3/command/migrate_command.go b/etcdctl/ctlv3/command/migrate_command.go index 634ebb94488..48c17c8ab4a 100644 --- a/etcdctl/ctlv3/command/migrate_command.go +++ b/etcdctl/ctlv3/command/migrate_command.go @@ -218,8 +218,9 @@ func applyConf(cc raftpb.ConfChange, cl *membership.RaftCluster) { } } -func applyRequest(r *pb.Request, applyV2 etcdserver.ApplierV2) { - toTTLOptions(r) +func applyRequest(req *pb.Request, applyV2 etcdserver.ApplierV2) { + r := (*etcdserver.RequestV2)(req) + r.TTLOptions() switch r.Method { case "POST": applyV2.Post(r) @@ -236,15 +237,6 @@ func applyRequest(r *pb.Request, applyV2 etcdserver.ApplierV2) { } } -func toTTLOptions(r *pb.Request) store.TTLOptionSet { - refresh, _ := pbutil.GetBool(r.Refresh) - ttlOptions := store.TTLOptionSet{Refresh: refresh} - if r.Expiration != 0 { - ttlOptions.ExpireTime = time.Unix(0, r.Expiration) - } - return ttlOptions -} - func writeStore(w io.Writer, st store.Store) uint64 { all, err := st.Get("/1", true, true) if err != nil { From 8091be6e970ad26a966e14626bf5382c8675497e Mon Sep 17 00:00:00 2001 From: Anthony Romano Date: Mon, 7 Aug 2017 01:33:24 -0700 Subject: [PATCH 04/11] v2v3: ServerV2 backed by clientv3 --- etcdserver/api/v2v3/cluster.go | 31 ++ etcdserver/api/v2v3/doc.go | 16 + etcdserver/api/v2v3/server.go | 117 +++++++ etcdserver/api/v2v3/store.go | 621 +++++++++++++++++++++++++++++++++ etcdserver/api/v2v3/watcher.go | 140 ++++++++ 5 files changed, 925 insertions(+) create mode 100644 etcdserver/api/v2v3/cluster.go create mode 100644 etcdserver/api/v2v3/doc.go create mode 100644 etcdserver/api/v2v3/server.go create mode 100644 etcdserver/api/v2v3/store.go create mode 100644 etcdserver/api/v2v3/watcher.go diff --git a/etcdserver/api/v2v3/cluster.go b/etcdserver/api/v2v3/cluster.go new file mode 100644 index 00000000000..b53e6d7c8b5 --- /dev/null +++ b/etcdserver/api/v2v3/cluster.go @@ -0,0 +1,31 @@ +// Copyright 2017 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package v2v3 + +import ( + "github.com/coreos/etcd/etcdserver/membership" + "github.com/coreos/etcd/pkg/types" + + "github.com/coreos/go-semver/semver" +) + +func (s *v2v3Server) ID() types.ID { + // TODO: use an actual member ID + return types.ID(0xe7cd2f00d) +} +func (s *v2v3Server) ClientURLs() []string { panic("STUB") } +func (s *v2v3Server) Members() []*membership.Member { panic("STUB") } +func (s *v2v3Server) Member(id types.ID) *membership.Member { panic("STUB") } +func (s *v2v3Server) Version() *semver.Version { panic("STUB") } diff --git a/etcdserver/api/v2v3/doc.go b/etcdserver/api/v2v3/doc.go new file mode 100644 index 00000000000..2ff372f1876 --- /dev/null +++ b/etcdserver/api/v2v3/doc.go @@ -0,0 +1,16 @@ +// Copyright 2017 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Package v2v3 provides a ServerV2 implementation backed by clientv3.Client. +package v2v3 diff --git a/etcdserver/api/v2v3/server.go b/etcdserver/api/v2v3/server.go new file mode 100644 index 00000000000..dae457ccf9d --- /dev/null +++ b/etcdserver/api/v2v3/server.go @@ -0,0 +1,117 @@ +// Copyright 2017 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package v2v3 + +import ( + "net/http" + "time" + + "github.com/coreos/etcd/clientv3" + "github.com/coreos/etcd/etcdserver" + "github.com/coreos/etcd/etcdserver/api" + pb "github.com/coreos/etcd/etcdserver/etcdserverpb" + "github.com/coreos/etcd/etcdserver/membership" + "github.com/coreos/etcd/pkg/types" + + "github.com/coreos/go-semver/semver" + "golang.org/x/net/context" // TODO: replace with context in go1.9 +) + +type fakeStats struct{} + +func (s *fakeStats) SelfStats() []byte { return nil } +func (s *fakeStats) LeaderStats() []byte { return nil } +func (s *fakeStats) StoreStats() []byte { return nil } + +type v2v3Server struct { + c *clientv3.Client + store *v2v3Store + fakeStats +} + +func NewServer(c *clientv3.Client, pfx string) etcdserver.ServerPeer { + return &v2v3Server{c: c, store: newStore(c, pfx)} +} + +func (s *v2v3Server) ClientCertAuthEnabled() bool { return false } + +func (s *v2v3Server) LeaseHandler() http.Handler { panic("STUB: lease handler") } +func (s *v2v3Server) RaftHandler() http.Handler { panic("STUB: raft handler") } + +func (s *v2v3Server) Leader() types.ID { + ctx, cancel := context.WithTimeout(context.TODO(), 5*time.Second) + defer cancel() + resp, err := s.c.Status(ctx, s.c.Endpoints()[0]) + if err != nil { + return 0 + } + return types.ID(resp.Leader) +} + +func (s *v2v3Server) AddMember(ctx context.Context, memb membership.Member) ([]*membership.Member, error) { + resp, err := s.c.MemberAdd(ctx, memb.PeerURLs) + if err != nil { + return nil, err + } + return v3MembersToMembership(resp.Members), nil +} + +func (s *v2v3Server) RemoveMember(ctx context.Context, id uint64) ([]*membership.Member, error) { + resp, err := s.c.MemberRemove(ctx, id) + if err != nil { + return nil, err + } + return v3MembersToMembership(resp.Members), nil +} + +func (s *v2v3Server) UpdateMember(ctx context.Context, m membership.Member) ([]*membership.Member, error) { + resp, err := s.c.MemberUpdate(ctx, uint64(m.ID), m.PeerURLs) + if err != nil { + return nil, err + } + return v3MembersToMembership(resp.Members), nil +} + +func v3MembersToMembership(v3membs []*pb.Member) []*membership.Member { + membs := make([]*membership.Member, len(v3membs)) + for i, m := range v3membs { + membs[i] = &membership.Member{ + ID: types.ID(m.ID), + RaftAttributes: membership.RaftAttributes{ + PeerURLs: m.PeerURLs, + }, + Attributes: membership.Attributes{ + Name: m.Name, + ClientURLs: m.ClientURLs, + }, + } + } + return membs +} + +func (s *v2v3Server) ClusterVersion() *semver.Version { return s.Version() } +func (s *v2v3Server) Cluster() api.Cluster { return s } +func (s *v2v3Server) Alarms() []*pb.AlarmMember { return nil } + +func (s *v2v3Server) Do(ctx context.Context, r pb.Request) (etcdserver.Response, error) { + applier := etcdserver.NewApplierV2(s.store, nil) + reqHandler := etcdserver.NewStoreRequestV2Handler(s.store, applier) + req := (*etcdserver.RequestV2)(&r) + resp, err := req.Handle(ctx, reqHandler) + if resp.Err != nil { + return resp, resp.Err + } + return resp, err +} diff --git a/etcdserver/api/v2v3/store.go b/etcdserver/api/v2v3/store.go new file mode 100644 index 00000000000..22227f8aab5 --- /dev/null +++ b/etcdserver/api/v2v3/store.go @@ -0,0 +1,621 @@ +// Copyright 2017 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package v2v3 + +import ( + "context" + "fmt" + "path" + "strings" + "time" + + "github.com/coreos/etcd/clientv3" + "github.com/coreos/etcd/clientv3/concurrency" + etcdErr "github.com/coreos/etcd/error" + "github.com/coreos/etcd/mvcc/mvccpb" + "github.com/coreos/etcd/store" +) + +// store implements the Store interface for V2 using +// a v3 client. +type v2v3Store struct { + c *clientv3.Client + // pfx is the v3 prefix where keys should be stored. + pfx string + ctx context.Context +} + +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) *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) { + key := s.mkPath(nodePath) + resp, err := s.c.Txn(s.ctx).Then( + clientv3.OpGet(key+"/"), + clientv3.OpGet(key), + ).Commit() + if err != nil { + return nil, err + } + + if kvs := resp.Responses[0].GetResponseRange().Kvs; len(kvs) != 0 || isRoot(nodePath) { + nodes, err := s.getDir(nodePath, recursive, sorted, resp.Header.Revision) + if err != nil { + return nil, err + } + cidx, midx := uint64(0), uint64(0) + if len(kvs) > 0 { + cidx, midx = mkV2Rev(kvs[0].CreateRevision), mkV2Rev(kvs[0].ModRevision) + } + return &store.Event{ + Action: store.Get, + Node: &store.NodeExtern{ + Key: nodePath, + Dir: true, + Nodes: nodes, + CreatedIndex: cidx, + ModifiedIndex: midx, + }, + EtcdIndex: mkV2Rev(resp.Header.Revision), + }, nil + } + + kvs := resp.Responses[1].GetResponseRange().Kvs + if len(kvs) == 0 { + return nil, etcdErr.NewError(etcdErr.EcodeKeyNotFound, nodePath, mkV2Rev(resp.Header.Revision)) + } + + return &store.Event{ + Action: store.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) { + rootNodes, err := s.getDirDepth(nodePath, 1, rev) + if err != nil || !recursive { + return rootNodes, err + } + nextNodes := rootNodes + nodes := make(map[string]*store.NodeExtern) + // Breadth walk the subdirectories + for i := 2; len(nextNodes) > 0; i++ { + for _, n := range nextNodes { + nodes[n.Key] = n + if parent := nodes[path.Dir(n.Key)]; parent != nil { + parent.Nodes = append(parent.Nodes, n) + } + } + if nextNodes, err = s.getDirDepth(nodePath, i, rev); err != nil { + return nil, err + } + } + return rootNodes, nil +} + +func (s *v2v3Store) getDirDepth(nodePath string, depth int, rev int64) ([]*store.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)) + for i, kv := range resp.Kvs { + nodes[i] = s.mkV2Node(kv) + } + return nodes, nil +} + +func (s *v2v3Store) Set( + nodePath string, + dir bool, + value string, + expireOpts store.TTLOptionSet, +) (*store.Event, error) { + if expireOpts.Refresh || !expireOpts.ExpireTime.IsZero() { + return nil, errUnsupported + } + + if isRoot(nodePath) { + return nil, etcdErr.NewError(etcdErr.EcodeRootROnly, nodePath, 0) + } + + ecode := 0 + applyf := func(stm concurrency.STM) error { + parent := path.Dir(nodePath) + if !isRoot(parent) && stm.Rev(s.mkPath(parent)+"/") == 0 { + ecode = etcdErr.EcodeKeyNotFound + return nil + } + + key := s.mkPath(nodePath) + if dir { + if stm.Rev(key) != 0 { + // exists as non-dir + ecode = etcdErr.EcodeNotDir + return nil + } + key = key + "/" + } else if stm.Rev(key+"/") != 0 { + ecode = etcdErr.EcodeNotFile + return nil + } + stm.Put(key, value, clientv3.WithPrevKV()) + stm.Put(s.mkActionKey(), store.Set) + return nil + } + + resp, err := s.newSTM(applyf) + if err != nil { + return nil, err + } + if ecode != 0 { + return nil, etcdErr.NewError(ecode, nodePath, mkV2Rev(resp.Header.Revision)) + } + + createRev := resp.Header.Revision + var pn *store.NodeExtern + if pkv := prevKeyFromPuts(resp); pkv != nil { + pn = s.mkV2Node(pkv) + createRev = pkv.CreateRevision + } + + vp := &value + if dir { + vp = nil + } + return &store.Event{ + Action: store.Set, + Node: &store.NodeExtern{ + Key: nodePath, + Value: vp, + Dir: dir, + ModifiedIndex: mkV2Rev(resp.Header.Revision), + CreatedIndex: mkV2Rev(createRev), + }, + PrevNode: pn, + EtcdIndex: mkV2Rev(resp.Header.Revision), + }, nil +} + +func (s *v2v3Store) Update(nodePath, newValue string, expireOpts store.TTLOptionSet) (*store.Event, error) { + if isRoot(nodePath) { + return nil, etcdErr.NewError(etcdErr.EcodeRootROnly, nodePath, 0) + } + + if expireOpts.Refresh || !expireOpts.ExpireTime.IsZero() { + return nil, errUnsupported + } + + key := s.mkPath(nodePath) + ecode := 0 + applyf := func(stm concurrency.STM) error { + if rev := stm.Rev(key + "/"); rev != 0 { + ecode = etcdErr.EcodeNotFile + return nil + } + if rev := stm.Rev(key); rev == 0 { + ecode = etcdErr.EcodeKeyNotFound + return nil + } + stm.Put(key, newValue, clientv3.WithPrevKV()) + stm.Put(s.mkActionKey(), store.Update) + return nil + } + + resp, err := s.newSTM(applyf) + if err != nil { + return nil, err + } + if ecode != 0 { + return nil, etcdErr.NewError(etcdErr.EcodeNotFile, nodePath, mkV2Rev(resp.Header.Revision)) + } + + pkv := prevKeyFromPuts(resp) + return &store.Event{ + Action: store.Update, + Node: &store.NodeExtern{ + Key: nodePath, + Value: &newValue, + ModifiedIndex: mkV2Rev(resp.Header.Revision), + CreatedIndex: mkV2Rev(pkv.CreateRevision), + }, + PrevNode: s.mkV2Node(pkv), + EtcdIndex: mkV2Rev(resp.Header.Revision), + }, nil +} + +func (s *v2v3Store) Create( + nodePath string, + dir bool, + value string, + unique bool, + expireOpts store.TTLOptionSet, +) (*store.Event, error) { + if isRoot(nodePath) { + return nil, etcdErr.NewError(etcdErr.EcodeRootROnly, nodePath, 0) + } + if expireOpts.Refresh || !expireOpts.ExpireTime.IsZero() { + return nil, errUnsupported + } + ecode := 0 + applyf := func(stm concurrency.STM) error { + ecode = 0 + key := s.mkPath(nodePath) + if unique { + // append unique item under the node path + for { + key = nodePath + "/" + fmt.Sprintf("%020s", time.Now()) + key = path.Clean(path.Join("/", key)) + key = s.mkPath(key) + if stm.Rev(key) == 0 { + break + } + } + } + if stm.Rev(key) > 0 || stm.Rev(key+"/") > 0 { + ecode = etcdErr.EcodeNodeExist + return nil + } + // build path if any directories in path do not exist + dirs := []string{} + for p := path.Dir(nodePath); !isRoot(p); p = path.Dir(p) { + pp := s.mkPath(p) + if stm.Rev(pp) > 0 { + ecode = etcdErr.EcodeNotDir + return nil + } + if stm.Rev(pp+"/") == 0 { + dirs = append(dirs, pp+"/") + } + } + for _, d := range dirs { + stm.Put(d, "") + } + + if dir { + // directories marked with extra slash in key name + key += "/" + } + stm.Put(key, value) + stm.Put(s.mkActionKey(), store.Create) + return nil + } + + resp, err := s.newSTM(applyf) + if err != nil { + return nil, err + } + if ecode != 0 { + return nil, etcdErr.NewError(ecode, nodePath, mkV2Rev(resp.Header.Revision)) + } + + var v *string + if !dir { + v = &value + } + + return &store.Event{ + Action: store.Create, + Node: &store.NodeExtern{ + Key: nodePath, + Value: v, + Dir: dir, + ModifiedIndex: mkV2Rev(resp.Header.Revision), + CreatedIndex: mkV2Rev(resp.Header.Revision), + }, + EtcdIndex: mkV2Rev(resp.Header.Revision), + }, nil +} + +func (s *v2v3Store) CompareAndSwap( + nodePath string, + prevValue string, + prevIndex uint64, + value string, + expireOpts store.TTLOptionSet, +) (*store.Event, error) { + if isRoot(nodePath) { + return nil, etcdErr.NewError(etcdErr.EcodeRootROnly, nodePath, 0) + } + if expireOpts.Refresh || !expireOpts.ExpireTime.IsZero() { + return nil, errUnsupported + } + + key := s.mkPath(nodePath) + resp, err := s.c.Txn(s.ctx).If( + s.mkCompare(nodePath, prevValue, prevIndex)..., + ).Then( + clientv3.OpPut(key, value, clientv3.WithPrevKV()), + clientv3.OpPut(s.mkActionKey(), store.CompareAndSwap), + ).Else( + clientv3.OpGet(key), + clientv3.OpGet(key+"/"), + ).Commit() + + if err != nil { + return nil, err + } + if !resp.Succeeded { + return nil, compareFail(nodePath, prevValue, prevIndex, resp) + } + + pkv := resp.Responses[0].GetResponsePut().PrevKv + return &store.Event{ + Action: store.CompareAndSwap, + Node: &store.NodeExtern{ + Key: nodePath, + Value: &value, + CreatedIndex: mkV2Rev(pkv.CreateRevision), + ModifiedIndex: mkV2Rev(resp.Header.Revision), + }, + PrevNode: s.mkV2Node(pkv), + EtcdIndex: mkV2Rev(resp.Header.Revision), + }, nil +} + +func (s *v2v3Store) Delete(nodePath string, dir, recursive bool) (*store.Event, error) { + if isRoot(nodePath) { + return nil, etcdErr.NewError(etcdErr.EcodeRootROnly, nodePath, 0) + } + if !dir && !recursive { + return s.deleteNode(nodePath) + } + dir = true + if !recursive { + return s.deleteEmptyDir(nodePath) + } + + dels := make([]clientv3.Op, maxPathDepth+1) + dels[0] = clientv3.OpDelete(s.mkPath(nodePath)+"/", clientv3.WithPrevKV()) + for i := 1; i < maxPathDepth; i++ { + dels[i] = clientv3.OpDelete(s.mkPathDepth(nodePath, i), clientv3.WithPrefix()) + } + dels[maxPathDepth] = clientv3.OpPut(s.mkActionKey(), store.Delete) + + resp, err := s.c.Txn(s.ctx).If( + clientv3.Compare(clientv3.Version(s.mkPath(nodePath)+"/"), ">", 0), + clientv3.Compare(clientv3.Version(s.mkPathDepth(nodePath, maxPathDepth)+"/"), "=", 0), + ).Then( + dels..., + ).Commit() + if err != nil { + return nil, err + } + if !resp.Succeeded { + return nil, etcdErr.NewError(etcdErr.EcodeNodeExist, nodePath, mkV2Rev(resp.Header.Revision)) + } + dresp := resp.Responses[0].GetResponseDeleteRange() + return &store.Event{ + Action: store.Delete, + PrevNode: s.mkV2Node(dresp.PrevKvs[0]), + EtcdIndex: mkV2Rev(resp.Header.Revision), + }, nil +} + +func (s *v2v3Store) deleteEmptyDir(nodePath string) (*store.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), + ).Commit() + if err != nil { + return nil, err + } + if !resp.Succeeded { + return nil, etcdErr.NewError(etcdErr.EcodeDirNotEmpty, nodePath, mkV2Rev(resp.Header.Revision)) + } + dresp := resp.Responses[0].GetResponseDeleteRange() + if len(dresp.PrevKvs) == 0 { + return nil, etcdErr.NewError(etcdErr.EcodeNodeExist, nodePath, mkV2Rev(resp.Header.Revision)) + } + return &store.Event{ + Action: store.Delete, + PrevNode: s.mkV2Node(dresp.PrevKvs[0]), + EtcdIndex: mkV2Rev(resp.Header.Revision), + }, nil +} + +func (s *v2v3Store) deleteNode(nodePath string) (*store.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), + ).Commit() + if err != nil { + return nil, err + } + if !resp.Succeeded { + return nil, etcdErr.NewError(etcdErr.EcodeNotFile, nodePath, mkV2Rev(resp.Header.Revision)) + } + pkvs := resp.Responses[0].GetResponseDeleteRange().PrevKvs + if len(pkvs) == 0 { + return nil, etcdErr.NewError(etcdErr.EcodeKeyNotFound, nodePath, mkV2Rev(resp.Header.Revision)) + } + pkv := pkvs[0] + return &store.Event{ + Action: store.Delete, + Node: &store.NodeExtern{ + Key: nodePath, + CreatedIndex: mkV2Rev(pkv.CreateRevision), + ModifiedIndex: mkV2Rev(resp.Header.Revision), + }, + PrevNode: s.mkV2Node(pkv), + EtcdIndex: mkV2Rev(resp.Header.Revision), + }, nil +} + +func (s *v2v3Store) CompareAndDelete(nodePath, prevValue string, prevIndex uint64) (*store.Event, error) { + if isRoot(nodePath) { + return nil, etcdErr.NewError(etcdErr.EcodeRootROnly, nodePath, 0) + } + + key := s.mkPath(nodePath) + resp, err := s.c.Txn(s.ctx).If( + s.mkCompare(nodePath, prevValue, prevIndex)..., + ).Then( + clientv3.OpDelete(key, clientv3.WithPrevKV()), + clientv3.OpPut(s.mkActionKey(), store.CompareAndDelete), + ).Else( + clientv3.OpGet(key), + clientv3.OpGet(key+"/"), + ).Commit() + + if err != nil { + return nil, err + } + if !resp.Succeeded { + return nil, compareFail(nodePath, prevValue, prevIndex, resp) + } + + // 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{ + Key: nodePath, + CreatedIndex: mkV2Rev(pkv.CreateRevision), + ModifiedIndex: mkV2Rev(resp.Header.Revision), + }, + PrevNode: s.mkV2Node(pkv), + EtcdIndex: mkV2Rev(resp.Header.Revision), + }, nil +} + +func compareFail(nodePath, prevValue string, prevIndex uint64, resp *clientv3.TxnResponse) error { + if dkvs := resp.Responses[1].GetResponseRange().Kvs; len(dkvs) > 0 { + return etcdErr.NewError(etcdErr.EcodeNotFile, nodePath, mkV2Rev(resp.Header.Revision)) + } + kvs := resp.Responses[0].GetResponseRange().Kvs + if len(kvs) == 0 { + return etcdErr.NewError(etcdErr.EcodeKeyNotFound, nodePath, mkV2Rev(resp.Header.Revision)) + } + kv := kvs[0] + indexMatch := (prevIndex == 0 || kv.ModRevision == int64(prevIndex)) + valueMatch := (prevValue == "" || string(kv.Value) == prevValue) + cause := "" + switch { + case indexMatch && !valueMatch: + cause = fmt.Sprintf("[%v != %v]", prevValue, string(kv.Value)) + case valueMatch && !indexMatch: + cause = fmt.Sprintf("[%v != %v]", prevIndex, kv.ModRevision) + default: + cause = fmt.Sprintf("[%v != %v] [%v != %v]", prevValue, string(kv.Value), prevIndex, kv.ModRevision) + } + return etcdErr.NewError(etcdErr.EcodeTestFailed, cause, mkV2Rev(resp.Header.Revision)) +} + +func (s *v2v3Store) mkCompare(nodePath, prevValue string, prevIndex uint64) []clientv3.Cmp { + key := s.mkPath(nodePath) + cmps := []clientv3.Cmp{clientv3.Compare(clientv3.Version(key), ">", 0)} + if prevIndex != 0 { + cmps = append(cmps, clientv3.Compare(clientv3.ModRevision(key), "=", mkV3Rev(prevIndex))) + } + if prevValue != "" { + cmps = append(cmps, clientv3.Compare(clientv3.Value(key), "=", prevValue)) + } + return cmps +} + +func (s *v2v3Store) JsonStats() []byte { panic("STUB") } +func (s *v2v3Store) DeleteExpiredKeys(cutoff time.Time) { panic("STUB") } + +func (s *v2v3Store) Version() int { return 2 } + +// TODO: move this out of the Store interface? + +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) SaveNoCopy() ([]byte, error) { panic("STUB") } +func (s *v2v3Store) HasTTLKeys() bool { panic("STUB") } + +func (s *v2v3Store) mkPath(nodePath string) string { return s.mkPathDepth(nodePath, 0) } + +func (s *v2v3Store) mkNodePath(p string) string { + return path.Clean(p[len(s.pfx)+len("/k/000/"):]) +} + +// mkPathDepth makes a path to a key that encodes its directory depth +// for fast directory listing. If a depth is provided, it is added +// to the computed depth. +func (s *v2v3Store) mkPathDepth(nodePath string, depth int) string { + normalForm := path.Clean(path.Join("/", nodePath)) + n := strings.Count(normalForm, "/") + depth + return fmt.Sprintf("%s/%03d/k/%s", s.pfx, n, normalForm) +} + +func (s *v2v3Store) mkActionKey() string { return s.pfx + "/act" } + +func isRoot(s string) bool { return len(s) == 0 || s == "/" || s == "/0" || s == "/1" } + +func mkV2Rev(v3Rev int64) uint64 { + if v3Rev == 0 { + return 0 + } + return uint64(v3Rev - 1) +} + +func mkV3Rev(v2Rev uint64) int64 { + if v2Rev == 0 { + return 0 + } + return int64(v2Rev + 1) +} + +// mkV2Node creates a V2 NodeExtern from a V3 KeyValue +func (s *v2v3Store) mkV2Node(kv *mvccpb.KeyValue) *store.NodeExtern { + if kv == nil { + return nil + } + n := &store.NodeExtern{ + Key: string(s.mkNodePath(string(kv.Key))), + Dir: kv.Key[len(kv.Key)-1] == '/', + CreatedIndex: mkV2Rev(kv.CreateRevision), + ModifiedIndex: mkV2Rev(kv.ModRevision), + } + if !n.Dir { + v := string(kv.Value) + n.Value = &v + } + return n +} + +// prevKeyFromPuts gets the prev key that is being put; ignores +// the put action response. +func prevKeyFromPuts(resp *clientv3.TxnResponse) *mvccpb.KeyValue { + for _, r := range resp.Responses { + pkv := r.GetResponsePut().PrevKv + if pkv != nil && pkv.CreateRevision > 0 { + return pkv + } + } + return nil +} + +func (s *v2v3Store) newSTM(applyf func(concurrency.STM) error) (*clientv3.TxnResponse, error) { + return concurrency.NewSTM(s.c, applyf, concurrency.WithIsolation(concurrency.Serializable)) +} diff --git a/etcdserver/api/v2v3/watcher.go b/etcdserver/api/v2v3/watcher.go new file mode 100644 index 00000000000..1c2680e744a --- /dev/null +++ b/etcdserver/api/v2v3/watcher.go @@ -0,0 +1,140 @@ +// Copyright 2017 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package v2v3 + +import ( + "context" + "strings" + + "github.com/coreos/etcd/clientv3" + etcdErr "github.com/coreos/etcd/error" + "github.com/coreos/etcd/store" +) + +func (s *v2v3Store) Watch(prefix string, recursive, stream bool, sinceIndex uint64) (store.Watcher, error) { + ctx, cancel := context.WithCancel(s.ctx) + wch := s.c.Watch( + ctx, + // TODO: very pricey; use a single store-wide watch in future + s.pfx, + clientv3.WithPrefix(), + clientv3.WithRev(int64(sinceIndex)), + clientv3.WithCreatedNotify(), + clientv3.WithPrevKV()) + resp, ok := <-wch + if err := resp.Err(); err != nil || !ok { + cancel() + return nil, etcdErr.NewError(etcdErr.EcodeRaftInternal, prefix, 0) + } + + evc, donec := make(chan *store.Event), make(chan struct{}) + go func() { + defer func() { + close(evc) + close(donec) + }() + for resp := range wch { + for _, ev := range s.mkV2Events(resp) { + k := ev.Node.Key + if recursive { + if !strings.HasPrefix(k, prefix) { + continue + } + // accept events on hidden keys given in prefix + k = strings.Replace(k, prefix, "/", 1) + // ignore hidden keys deeper than prefix + if strings.Contains(k, "/_") { + continue + } + } + if !recursive && k != prefix { + continue + } + select { + case evc <- ev: + case <-ctx.Done(): + return + } + if !stream { + return + } + } + } + }() + + return &v2v3Watcher{ + startRev: resp.Header.Revision, + evc: evc, + donec: donec, + cancel: cancel, + }, nil +} + +func (s *v2v3Store) mkV2Events(wr clientv3.WatchResponse) (evs []*store.Event) { + ak := s.mkActionKey() + for _, rev := range mkRevs(wr) { + var act, key *clientv3.Event + for _, ev := range rev { + if string(ev.Kv.Key) == ak { + act = ev + } else if key != nil && len(key.Kv.Key) < len(ev.Kv.Key) { + // use longest key to ignore intermediate new + // directories from Create. + key = ev + } else if key == nil { + key = ev + } + } + v2ev := &store.Event{ + Action: string(act.Kv.Value), + Node: s.mkV2Node(key.Kv), + PrevNode: s.mkV2Node(key.PrevKv), + EtcdIndex: mkV2Rev(wr.Header.Revision), + } + evs = append(evs, v2ev) + } + return evs +} + +func mkRevs(wr clientv3.WatchResponse) (revs [][]*clientv3.Event) { + var curRev []*clientv3.Event + for _, ev := range wr.Events { + if curRev != nil && ev.Kv.ModRevision != curRev[0].Kv.ModRevision { + revs = append(revs, curRev) + curRev = nil + } + curRev = append(curRev, ev) + } + if curRev != nil { + revs = append(revs, curRev) + } + return revs +} + +type v2v3Watcher struct { + startRev int64 + evc chan *store.Event + donec chan struct{} + cancel context.CancelFunc +} + +func (w *v2v3Watcher) StartIndex() uint64 { return mkV2Rev(w.startRev) } + +func (w *v2v3Watcher) Remove() { + w.cancel() + <-w.donec +} + +func (w *v2v3Watcher) EventChan() chan *store.Event { return w.evc } From cab7572b003a1774470153ff536eebe13f85cf7c Mon Sep 17 00:00:00 2001 From: Anthony Romano Date: Tue, 15 Aug 2017 10:54:02 -0700 Subject: [PATCH 05/11] store: separate tests that need Store from those needing *store --- store/metrics.go | 6 +- store/store_test.go | 651 +++++++++++----------------------------- store/store_ttl_test.go | 360 ++++++++++++++++++++++ store/store_v2_test.go | 62 ++++ 4 files changed, 606 insertions(+), 473 deletions(-) create mode 100644 store/store_ttl_test.go create mode 100644 store/store_v2_test.go diff --git a/store/metrics.go b/store/metrics.go index 26404ba72e3..08cd4f030ce 100644 --- a/store/metrics.go +++ b/store/metrics.go @@ -86,7 +86,11 @@ const ( ) func init() { - prometheus.MustRegister(readCounter) + if prometheus.Register(readCounter) != nil { + // Tests will try to double register sicne the tests use both + // store and store_test packages; ignore second attempts. + return + } prometheus.MustRegister(writeCounter) prometheus.MustRegister(expireCounter) prometheus.MustRegister(watchRequests) diff --git a/store/store_test.go b/store/store_test.go index 49f6fc620e9..00e837e8117 100644 --- a/store/store_test.go +++ b/store/store_test.go @@ -12,19 +12,24 @@ // See the License for the specific language governing permissions and // limitations under the License. -package store +package store_test import ( "testing" - "time" etcdErr "github.com/coreos/etcd/error" "github.com/coreos/etcd/pkg/testutil" - "github.com/jonboulle/clockwork" + "github.com/coreos/etcd/store" ) +type StoreCloser interface { + store.Store + Close() +} + func TestNewStoreWithNamespaces(t *testing.T) { - s := newStore("/0", "/1") + s := newTestStore(t, "/0", "/1") + defer s.Close() _, err := s.Get("/0", false, false) testutil.AssertNil(t, err) @@ -34,8 +39,10 @@ func TestNewStoreWithNamespaces(t *testing.T) { // Ensure that the store can retrieve an existing value. func TestStoreGetValue(t *testing.T) { - s := newStore() - s.Create("/foo", false, "bar", false, TTLOptionSet{ExpireTime: Permanent}) + s := newTestStore(t) + defer s.Close() + + s.Create("/foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent}) var eidx uint64 = 1 e, err := s.Get("/foo", false, false) testutil.AssertNil(t, err) @@ -45,90 +52,23 @@ func TestStoreGetValue(t *testing.T) { testutil.AssertEqual(t, *e.Node.Value, "bar") } -// Ensure that any TTL <= minExpireTime becomes Permanent -func TestMinExpireTime(t *testing.T) { - s := newStore() - fc := clockwork.NewFakeClock() - s.clock = fc - // FakeClock starts at 0, so minExpireTime should be far in the future.. but just in case - testutil.AssertTrue(t, minExpireTime.After(fc.Now()), "minExpireTime should be ahead of FakeClock!") - s.Create("/foo", false, "Y", false, TTLOptionSet{ExpireTime: fc.Now().Add(3 * time.Second)}) - fc.Advance(5 * time.Second) - // Ensure it hasn't expired - s.DeleteExpiredKeys(fc.Now()) - var eidx uint64 = 1 - e, err := s.Get("/foo", true, false) - testutil.AssertNil(t, err) - testutil.AssertEqual(t, e.EtcdIndex, eidx) - testutil.AssertEqual(t, e.Action, "get") - testutil.AssertEqual(t, e.Node.Key, "/foo") - testutil.AssertEqual(t, e.Node.TTL, int64(0)) -} - -// Ensure that the store can recursively retrieve a directory listing. -// Note that hidden files should not be returned. -func TestStoreGetDirectory(t *testing.T) { - s := newStore() - fc := newFakeClock() - s.clock = fc - s.Create("/foo", true, "", false, TTLOptionSet{ExpireTime: Permanent}) - s.Create("/foo/bar", false, "X", false, TTLOptionSet{ExpireTime: Permanent}) - s.Create("/foo/_hidden", false, "*", false, TTLOptionSet{ExpireTime: Permanent}) - s.Create("/foo/baz", true, "", false, TTLOptionSet{ExpireTime: Permanent}) - s.Create("/foo/baz/bat", false, "Y", false, TTLOptionSet{ExpireTime: Permanent}) - s.Create("/foo/baz/_hidden", false, "*", false, TTLOptionSet{ExpireTime: Permanent}) - s.Create("/foo/baz/ttl", false, "Y", false, TTLOptionSet{ExpireTime: fc.Now().Add(time.Second * 3)}) - var eidx uint64 = 7 - e, err := s.Get("/foo", true, false) - testutil.AssertNil(t, err) - testutil.AssertEqual(t, e.EtcdIndex, eidx) - testutil.AssertEqual(t, e.Action, "get") - testutil.AssertEqual(t, e.Node.Key, "/foo") - testutil.AssertEqual(t, len(e.Node.Nodes), 2) - var bazNodes NodeExterns - for _, node := range e.Node.Nodes { - switch node.Key { - case "/foo/bar": - testutil.AssertEqual(t, *node.Value, "X") - testutil.AssertEqual(t, node.Dir, false) - case "/foo/baz": - testutil.AssertEqual(t, node.Dir, true) - testutil.AssertEqual(t, len(node.Nodes), 2) - bazNodes = node.Nodes - default: - t.Errorf("key = %s, not matched", node.Key) - } - } - for _, node := range bazNodes { - switch node.Key { - case "/foo/baz/bat": - testutil.AssertEqual(t, *node.Value, "Y") - testutil.AssertEqual(t, node.Dir, false) - case "/foo/baz/ttl": - testutil.AssertEqual(t, *node.Value, "Y") - testutil.AssertEqual(t, node.Dir, false) - testutil.AssertEqual(t, node.TTL, int64(3)) - default: - t.Errorf("key = %s, not matched", node.Key) - } - } -} - // Ensure that the store can retrieve a directory in sorted order. func TestStoreGetSorted(t *testing.T) { - s := newStore() - s.Create("/foo", true, "", false, TTLOptionSet{ExpireTime: Permanent}) - s.Create("/foo/x", false, "0", false, TTLOptionSet{ExpireTime: Permanent}) - s.Create("/foo/z", false, "0", false, TTLOptionSet{ExpireTime: Permanent}) - s.Create("/foo/y", true, "", false, TTLOptionSet{ExpireTime: Permanent}) - s.Create("/foo/y/a", false, "0", false, TTLOptionSet{ExpireTime: Permanent}) - s.Create("/foo/y/b", false, "0", false, TTLOptionSet{ExpireTime: Permanent}) + 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}) var eidx uint64 = 6 e, err := s.Get("/foo", true, true) testutil.AssertNil(t, err) testutil.AssertEqual(t, e.EtcdIndex, eidx) - var yNodes NodeExterns + var yNodes store.NodeExterns sortedStrings := []string{"/foo/x", "/foo/y", "/foo/z"} for i := range e.Node.Nodes { node := e.Node.Nodes[i] @@ -150,11 +90,12 @@ func TestStoreGetSorted(t *testing.T) { } func TestSet(t *testing.T) { - s := newStore() + s := newTestStore(t) + defer s.Close() // Set /foo="" var eidx uint64 = 1 - e, err := s.Set("/foo", false, "", TTLOptionSet{ExpireTime: Permanent}) + e, err := s.Set("/foo", false, "", store.TTLOptionSet{ExpireTime: store.Permanent}) testutil.AssertNil(t, err) testutil.AssertEqual(t, e.EtcdIndex, eidx) testutil.AssertEqual(t, e.Action, "set") @@ -168,7 +109,7 @@ func TestSet(t *testing.T) { // Set /foo="bar" eidx = 2 - e, err = s.Set("/foo", false, "bar", TTLOptionSet{ExpireTime: Permanent}) + e, err = s.Set("/foo", false, "bar", store.TTLOptionSet{ExpireTime: store.Permanent}) testutil.AssertNil(t, err) testutil.AssertEqual(t, e.EtcdIndex, eidx) testutil.AssertEqual(t, e.Action, "set") @@ -186,7 +127,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", TTLOptionSet{ExpireTime: Permanent}) + e, err = s.Set("/foo", false, "baz", store.TTLOptionSet{ExpireTime: store.Permanent}) testutil.AssertNil(t, err) testutil.AssertEqual(t, e.EtcdIndex, eidx) testutil.AssertEqual(t, e.Action, "set") @@ -205,7 +146,7 @@ func TestSet(t *testing.T) { // Set /dir as a directory eidx = 4 - e, err = s.Set("/dir", true, "", TTLOptionSet{ExpireTime: Permanent}) + e, err = s.Set("/dir", true, "", store.TTLOptionSet{ExpireTime: store.Permanent}) testutil.AssertNil(t, err) testutil.AssertEqual(t, e.EtcdIndex, eidx) testutil.AssertEqual(t, e.Action, "set") @@ -220,10 +161,12 @@ func TestSet(t *testing.T) { // Ensure that the store can create a new key if it doesn't already exist. func TestStoreCreateValue(t *testing.T) { - s := newStore() + s := newTestStore(t) + defer s.Close() + // Create /foo=bar var eidx uint64 = 1 - e, err := s.Create("/foo", false, "bar", false, TTLOptionSet{ExpireTime: Permanent}) + e, err := s.Create("/foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent}) testutil.AssertNil(t, err) testutil.AssertEqual(t, e.EtcdIndex, eidx) testutil.AssertEqual(t, e.Action, "create") @@ -237,7 +180,7 @@ func TestStoreCreateValue(t *testing.T) { // Create /empty="" eidx = 2 - e, err = s.Create("/empty", false, "", false, TTLOptionSet{ExpireTime: Permanent}) + e, err = s.Create("/empty", false, "", false, store.TTLOptionSet{ExpireTime: store.Permanent}) testutil.AssertNil(t, err) testutil.AssertEqual(t, e.EtcdIndex, eidx) testutil.AssertEqual(t, e.Action, "create") @@ -253,9 +196,11 @@ func TestStoreCreateValue(t *testing.T) { // Ensure that the store can create a new directory if it doesn't already exist. func TestStoreCreateDirectory(t *testing.T) { - s := newStore() + s := newTestStore(t) + defer s.Close() + var eidx uint64 = 1 - e, err := s.Create("/foo", true, "", false, TTLOptionSet{ExpireTime: Permanent}) + e, err := s.Create("/foo", true, "", false, store.TTLOptionSet{ExpireTime: store.Permanent}) testutil.AssertNil(t, err) testutil.AssertEqual(t, e.EtcdIndex, eidx) testutil.AssertEqual(t, e.Action, "create") @@ -265,12 +210,14 @@ func TestStoreCreateDirectory(t *testing.T) { // Ensure that the store fails to create a key if it already exists. func TestStoreCreateFailsIfExists(t *testing.T) { - s := newStore() + s := newTestStore(t) + defer s.Close() + // create /foo as dir - s.Create("/foo", true, "", false, TTLOptionSet{ExpireTime: Permanent}) + s.Create("/foo", true, "", false, store.TTLOptionSet{ExpireTime: store.Permanent}) // create /foo as dir again - e, _err := s.Create("/foo", true, "", false, TTLOptionSet{ExpireTime: Permanent}) + e, _err := s.Create("/foo", true, "", false, store.TTLOptionSet{ExpireTime: store.Permanent}) err := _err.(*etcdErr.Error) testutil.AssertEqual(t, err.ErrorCode, etcdErr.EcodeNodeExist) testutil.AssertEqual(t, err.Message, "Key already exists") @@ -281,12 +228,14 @@ func TestStoreCreateFailsIfExists(t *testing.T) { // Ensure that the store can update a key if it already exists. func TestStoreUpdateValue(t *testing.T) { - s := newStore() + s := newTestStore(t) + defer s.Close() + // create /foo=bar - s.Create("/foo", false, "bar", false, TTLOptionSet{ExpireTime: Permanent}) + s.Create("/foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent}) // update /foo="bzr" var eidx uint64 = 2 - e, err := s.Update("/foo", "baz", TTLOptionSet{ExpireTime: Permanent}) + e, err := s.Update("/foo", "baz", store.TTLOptionSet{ExpireTime: store.Permanent}) testutil.AssertNil(t, err) testutil.AssertEqual(t, e.EtcdIndex, eidx) testutil.AssertEqual(t, e.Action, "update") @@ -307,7 +256,7 @@ func TestStoreUpdateValue(t *testing.T) { // update /foo="" eidx = 3 - e, err = s.Update("/foo", "", TTLOptionSet{ExpireTime: Permanent}) + e, err = s.Update("/foo", "", store.TTLOptionSet{ExpireTime: store.Permanent}) testutil.AssertNil(t, err) testutil.AssertEqual(t, e.EtcdIndex, eidx) testutil.AssertEqual(t, e.Action, "update") @@ -329,9 +278,11 @@ func TestStoreUpdateValue(t *testing.T) { // Ensure that the store cannot update a directory. func TestStoreUpdateFailsIfDirectory(t *testing.T) { - s := newStore() - s.Create("/foo", true, "", false, TTLOptionSet{ExpireTime: Permanent}) - e, _err := s.Update("/foo", "baz", TTLOptionSet{ExpireTime: Permanent}) + 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}) err := _err.(*etcdErr.Error) testutil.AssertEqual(t, err.ErrorCode, etcdErr.EcodeNotFile) testutil.AssertEqual(t, err.Message, "Not a file") @@ -339,55 +290,13 @@ func TestStoreUpdateFailsIfDirectory(t *testing.T) { testutil.AssertNil(t, e) } -// Ensure that the store can update the TTL on a value. -func TestStoreUpdateValueTTL(t *testing.T) { - s := newStore() - fc := newFakeClock() - s.clock = fc - - var eidx uint64 = 2 - s.Create("/foo", false, "bar", false, TTLOptionSet{ExpireTime: Permanent}) - _, err := s.Update("/foo", "baz", TTLOptionSet{ExpireTime: fc.Now().Add(500 * time.Millisecond)}) - testutil.AssertNil(t, err) - e, _ := s.Get("/foo", false, false) - testutil.AssertEqual(t, *e.Node.Value, "baz") - testutil.AssertEqual(t, e.EtcdIndex, eidx) - fc.Advance(600 * time.Millisecond) - s.DeleteExpiredKeys(fc.Now()) - e, err = s.Get("/foo", false, false) - testutil.AssertNil(t, e) - testutil.AssertEqual(t, err.(*etcdErr.Error).ErrorCode, etcdErr.EcodeKeyNotFound) -} - -// Ensure that the store can update the TTL on a directory. -func TestStoreUpdateDirTTL(t *testing.T) { - s := newStore() - fc := newFakeClock() - s.clock = fc - - var eidx uint64 = 3 - s.Create("/foo", true, "", false, TTLOptionSet{ExpireTime: Permanent}) - s.Create("/foo/bar", false, "baz", false, TTLOptionSet{ExpireTime: Permanent}) - e, err := s.Update("/foo", "", TTLOptionSet{ExpireTime: fc.Now().Add(500 * time.Millisecond)}) - testutil.AssertNil(t, err) - testutil.AssertEqual(t, e.Node.Dir, true) - testutil.AssertEqual(t, e.EtcdIndex, eidx) - e, _ = s.Get("/foo/bar", false, false) - testutil.AssertEqual(t, *e.Node.Value, "baz") - testutil.AssertEqual(t, e.EtcdIndex, eidx) - - fc.Advance(600 * time.Millisecond) - s.DeleteExpiredKeys(fc.Now()) - e, err = s.Get("/foo/bar", false, false) - testutil.AssertNil(t, e) - testutil.AssertEqual(t, err.(*etcdErr.Error).ErrorCode, etcdErr.EcodeKeyNotFound) -} - // Ensure that the store can delete a value. func TestStoreDeleteValue(t *testing.T) { - s := newStore() + s := newTestStore(t) + defer s.Close() + var eidx uint64 = 2 - s.Create("/foo", false, "bar", false, TTLOptionSet{ExpireTime: Permanent}) + s.Create("/foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent}) e, err := s.Delete("/foo", false, false) testutil.AssertNil(t, err) testutil.AssertEqual(t, e.EtcdIndex, eidx) @@ -399,11 +308,13 @@ func TestStoreDeleteValue(t *testing.T) { } // Ensure that the store can delete a directory if recursive is specified. -func TestStoreDeleteDiretory(t *testing.T) { - s := newStore() +func TestStoreDeleteDirectory(t *testing.T) { + s := newTestStore(t) + defer s.Close() + // create directory /foo var eidx uint64 = 2 - s.Create("/foo", true, "", false, TTLOptionSet{ExpireTime: Permanent}) + s.Create("/foo", true, "", false, store.TTLOptionSet{ExpireTime: store.Permanent}) // delete /foo with dir = true and recursive = false // this should succeed, since the directory is empty e, err := s.Delete("/foo", true, false) @@ -416,7 +327,8 @@ func TestStoreDeleteDiretory(t *testing.T) { testutil.AssertEqual(t, e.PrevNode.Dir, true) // create directory /foo and directory /foo/bar - s.Create("/foo/bar", true, "", false, TTLOptionSet{ExpireTime: Permanent}) + _, err = s.Create("/foo/bar", true, "", false, store.TTLOptionSet{ExpireTime: store.Permanent}) + testutil.AssertNil(t, err) // delete /foo with dir = true and recursive = false // this should fail, since the directory is not empty _, err = s.Delete("/foo", true, false) @@ -434,9 +346,11 @@ func TestStoreDeleteDiretory(t *testing.T) { // Ensure that the store cannot delete a directory if both of recursive // and dir are not specified. -func TestStoreDeleteDiretoryFailsIfNonRecursiveAndDir(t *testing.T) { - s := newStore() - s.Create("/foo", true, "", false, TTLOptionSet{ExpireTime: Permanent}) +func TestStoreDeleteDirectoryFailsIfNonRecursiveAndDir(t *testing.T) { + s := newTestStore(t) + defer s.Close() + + s.Create("/foo", true, "", false, store.TTLOptionSet{ExpireTime: store.Permanent}) e, _err := s.Delete("/foo", false, false) err := _err.(*etcdErr.Error) testutil.AssertEqual(t, err.ErrorCode, etcdErr.EcodeNotFile) @@ -445,30 +359,33 @@ func TestStoreDeleteDiretoryFailsIfNonRecursiveAndDir(t *testing.T) { } func TestRootRdOnly(t *testing.T) { - s := newStore("/0") + s := newTestStore(t, "/0") + defer s.Close() for _, tt := range []string{"/", "/0"} { - _, err := s.Set(tt, true, "", TTLOptionSet{ExpireTime: Permanent}) + _, err := s.Set(tt, true, "", store.TTLOptionSet{ExpireTime: store.Permanent}) testutil.AssertNotNil(t, err) _, err = s.Delete(tt, true, true) testutil.AssertNotNil(t, err) - _, err = s.Create(tt, true, "", false, TTLOptionSet{ExpireTime: Permanent}) + _, err = s.Create(tt, true, "", false, store.TTLOptionSet{ExpireTime: store.Permanent}) testutil.AssertNotNil(t, err) - _, err = s.Update(tt, "", TTLOptionSet{ExpireTime: Permanent}) + _, err = s.Update(tt, "", store.TTLOptionSet{ExpireTime: store.Permanent}) testutil.AssertNotNil(t, err) - _, err = s.CompareAndSwap(tt, "", 0, "", TTLOptionSet{ExpireTime: Permanent}) + _, err = s.CompareAndSwap(tt, "", 0, "", store.TTLOptionSet{ExpireTime: store.Permanent}) testutil.AssertNotNil(t, err) } } func TestStoreCompareAndDeletePrevValue(t *testing.T) { - s := newStore() + s := newTestStore(t) + defer s.Close() + var eidx uint64 = 2 - s.Create("/foo", false, "bar", false, TTLOptionSet{ExpireTime: Permanent}) + s.Create("/foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent}) e, err := s.CompareAndDelete("/foo", "bar", 0) testutil.AssertNil(t, err) testutil.AssertEqual(t, e.EtcdIndex, eidx) @@ -484,9 +401,11 @@ func TestStoreCompareAndDeletePrevValue(t *testing.T) { } func TestStoreCompareAndDeletePrevValueFailsIfNotMatch(t *testing.T) { - s := newStore() + s := newTestStore(t) + defer s.Close() + var eidx uint64 = 1 - s.Create("/foo", false, "bar", false, TTLOptionSet{ExpireTime: Permanent}) + s.Create("/foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent}) e, _err := s.CompareAndDelete("/foo", "baz", 0) err := _err.(*etcdErr.Error) testutil.AssertEqual(t, err.ErrorCode, etcdErr.EcodeTestFailed) @@ -498,9 +417,11 @@ func TestStoreCompareAndDeletePrevValueFailsIfNotMatch(t *testing.T) { } func TestStoreCompareAndDeletePrevIndex(t *testing.T) { - s := newStore() + s := newTestStore(t) + defer s.Close() + var eidx uint64 = 2 - s.Create("/foo", false, "bar", false, TTLOptionSet{ExpireTime: Permanent}) + s.Create("/foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent}) e, err := s.CompareAndDelete("/foo", "", 1) testutil.AssertNil(t, err) testutil.AssertEqual(t, e.EtcdIndex, eidx) @@ -514,9 +435,11 @@ func TestStoreCompareAndDeletePrevIndex(t *testing.T) { } func TestStoreCompareAndDeletePrevIndexFailsIfNotMatch(t *testing.T) { - s := newStore() + s := newTestStore(t) + defer s.Close() + var eidx uint64 = 1 - s.Create("/foo", false, "bar", false, TTLOptionSet{ExpireTime: Permanent}) + s.Create("/foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent}) e, _err := s.CompareAndDelete("/foo", "", 100) testutil.AssertNotNil(t, _err) err := _err.(*etcdErr.Error) @@ -529,9 +452,11 @@ func TestStoreCompareAndDeletePrevIndexFailsIfNotMatch(t *testing.T) { } // Ensure that the store cannot delete a directory. -func TestStoreCompareAndDeleteDiretoryFail(t *testing.T) { - s := newStore() - s.Create("/foo", true, "", false, TTLOptionSet{ExpireTime: Permanent}) +func TestStoreCompareAndDeleteDirectoryFail(t *testing.T) { + s := newTestStore(t) + defer s.Close() + + s.Create("/foo", true, "", false, store.TTLOptionSet{ExpireTime: store.Permanent}) _, _err := s.CompareAndDelete("/foo", "", 0) testutil.AssertNotNil(t, _err) err := _err.(*etcdErr.Error) @@ -540,10 +465,12 @@ func TestStoreCompareAndDeleteDiretoryFail(t *testing.T) { // Ensure that the store can conditionally update a key if it has a previous value. func TestStoreCompareAndSwapPrevValue(t *testing.T) { - s := newStore() + s := newTestStore(t) + defer s.Close() + var eidx uint64 = 2 - s.Create("/foo", false, "bar", false, TTLOptionSet{ExpireTime: Permanent}) - e, err := s.CompareAndSwap("/foo", "bar", 0, "baz", TTLOptionSet{ExpireTime: Permanent}) + s.Create("/foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent}) + e, err := s.CompareAndSwap("/foo", "bar", 0, "baz", store.TTLOptionSet{ExpireTime: store.Permanent}) testutil.AssertNil(t, err) testutil.AssertEqual(t, e.EtcdIndex, eidx) testutil.AssertEqual(t, e.Action, "compareAndSwap") @@ -561,10 +488,11 @@ func TestStoreCompareAndSwapPrevValue(t *testing.T) { // Ensure that the store cannot conditionally update a key if it has the wrong previous value. func TestStoreCompareAndSwapPrevValueFailsIfNotMatch(t *testing.T) { - s := newStore() + s := newTestStore(t) + defer s.Close() var eidx uint64 = 1 - s.Create("/foo", false, "bar", false, TTLOptionSet{ExpireTime: Permanent}) - e, _err := s.CompareAndSwap("/foo", "wrong_value", 0, "baz", TTLOptionSet{ExpireTime: Permanent}) + 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}) err := _err.(*etcdErr.Error) testutil.AssertEqual(t, err.ErrorCode, etcdErr.EcodeTestFailed) testutil.AssertEqual(t, err.Message, "Compare failed") @@ -576,10 +504,11 @@ func TestStoreCompareAndSwapPrevValueFailsIfNotMatch(t *testing.T) { // Ensure that the store can conditionally update a key if it has a previous index. func TestStoreCompareAndSwapPrevIndex(t *testing.T) { - s := newStore() + s := newTestStore(t) + defer s.Close() var eidx uint64 = 2 - s.Create("/foo", false, "bar", false, TTLOptionSet{ExpireTime: Permanent}) - e, err := s.CompareAndSwap("/foo", "", 1, "baz", TTLOptionSet{ExpireTime: Permanent}) + s.Create("/foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent}) + e, err := s.CompareAndSwap("/foo", "", 1, "baz", store.TTLOptionSet{ExpireTime: store.Permanent}) testutil.AssertNil(t, err) testutil.AssertEqual(t, e.EtcdIndex, eidx) testutil.AssertEqual(t, e.Action, "compareAndSwap") @@ -598,10 +527,11 @@ func TestStoreCompareAndSwapPrevIndex(t *testing.T) { // Ensure that the store cannot conditionally update a key if it has the wrong previous index. func TestStoreCompareAndSwapPrevIndexFailsIfNotMatch(t *testing.T) { - s := newStore() + s := newTestStore(t) + defer s.Close() var eidx uint64 = 1 - s.Create("/foo", false, "bar", false, TTLOptionSet{ExpireTime: Permanent}) - e, _err := s.CompareAndSwap("/foo", "", 100, "baz", TTLOptionSet{ExpireTime: Permanent}) + s.Create("/foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent}) + e, _err := s.CompareAndSwap("/foo", "", 100, "baz", store.TTLOptionSet{ExpireTime: store.Permanent}) err := _err.(*etcdErr.Error) testutil.AssertEqual(t, err.ErrorCode, etcdErr.EcodeTestFailed) testutil.AssertEqual(t, err.Message, "Compare failed") @@ -613,12 +543,13 @@ func TestStoreCompareAndSwapPrevIndexFailsIfNotMatch(t *testing.T) { // Ensure that the store can watch for key creation. func TestStoreWatchCreate(t *testing.T) { - s := newStore() + s := newTestStore(t) + defer s.Close() var eidx uint64 = 0 w, _ := s.Watch("/foo", false, false, 0) c := w.EventChan() testutil.AssertEqual(t, w.StartIndex(), eidx) - s.Create("/foo", false, "bar", false, TTLOptionSet{ExpireTime: Permanent}) + s.Create("/foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent}) eidx = 1 e := nbselect(c) testutil.AssertEqual(t, e.EtcdIndex, eidx) @@ -630,12 +561,13 @@ func TestStoreWatchCreate(t *testing.T) { // Ensure that the store can watch for recursive key creation. func TestStoreWatchRecursiveCreate(t *testing.T) { - s := newStore() + s := newTestStore(t) + defer s.Close() var eidx uint64 = 0 w, _ := s.Watch("/foo", true, false, 0) testutil.AssertEqual(t, w.StartIndex(), eidx) eidx = 1 - s.Create("/foo/bar", false, "baz", false, TTLOptionSet{ExpireTime: Permanent}) + s.Create("/foo/bar", false, "baz", false, store.TTLOptionSet{ExpireTime: store.Permanent}) e := nbselect(w.EventChan()) testutil.AssertEqual(t, e.EtcdIndex, eidx) testutil.AssertEqual(t, e.Action, "create") @@ -644,13 +576,14 @@ func TestStoreWatchRecursiveCreate(t *testing.T) { // Ensure that the store can watch for key updates. func TestStoreWatchUpdate(t *testing.T) { - s := newStore() + s := newTestStore(t) + defer s.Close() var eidx uint64 = 1 - s.Create("/foo", false, "bar", false, TTLOptionSet{ExpireTime: Permanent}) + s.Create("/foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent}) w, _ := s.Watch("/foo", false, false, 0) testutil.AssertEqual(t, w.StartIndex(), eidx) eidx = 2 - s.Update("/foo", "baz", TTLOptionSet{ExpireTime: Permanent}) + s.Update("/foo", "baz", store.TTLOptionSet{ExpireTime: store.Permanent}) e := nbselect(w.EventChan()) testutil.AssertEqual(t, e.EtcdIndex, eidx) testutil.AssertEqual(t, e.Action, "update") @@ -659,13 +592,14 @@ func TestStoreWatchUpdate(t *testing.T) { // Ensure that the store can watch for recursive key updates. func TestStoreWatchRecursiveUpdate(t *testing.T) { - s := newStore() + s := newTestStore(t) + defer s.Close() var eidx uint64 = 1 - s.Create("/foo/bar", false, "baz", false, TTLOptionSet{ExpireTime: Permanent}) + s.Create("/foo/bar", false, "baz", false, store.TTLOptionSet{ExpireTime: store.Permanent}) w, _ := s.Watch("/foo", true, false, 0) testutil.AssertEqual(t, w.StartIndex(), eidx) eidx = 2 - s.Update("/foo/bar", "baz", TTLOptionSet{ExpireTime: Permanent}) + s.Update("/foo/bar", "baz", store.TTLOptionSet{ExpireTime: store.Permanent}) e := nbselect(w.EventChan()) testutil.AssertEqual(t, e.EtcdIndex, eidx) testutil.AssertEqual(t, e.Action, "update") @@ -674,9 +608,10 @@ func TestStoreWatchRecursiveUpdate(t *testing.T) { // Ensure that the store can watch for key deletions. func TestStoreWatchDelete(t *testing.T) { - s := newStore() + s := newTestStore(t) + defer s.Close() var eidx uint64 = 1 - s.Create("/foo", false, "bar", false, TTLOptionSet{ExpireTime: Permanent}) + s.Create("/foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent}) w, _ := s.Watch("/foo", false, false, 0) testutil.AssertEqual(t, w.StartIndex(), eidx) eidx = 2 @@ -689,9 +624,10 @@ func TestStoreWatchDelete(t *testing.T) { // Ensure that the store can watch for recursive key deletions. func TestStoreWatchRecursiveDelete(t *testing.T) { - s := newStore() + s := newTestStore(t) + defer s.Close() var eidx uint64 = 1 - s.Create("/foo/bar", false, "baz", false, TTLOptionSet{ExpireTime: Permanent}) + s.Create("/foo/bar", false, "baz", false, store.TTLOptionSet{ExpireTime: store.Permanent}) w, _ := s.Watch("/foo", true, false, 0) testutil.AssertEqual(t, w.StartIndex(), eidx) eidx = 2 @@ -704,13 +640,14 @@ func TestStoreWatchRecursiveDelete(t *testing.T) { // Ensure that the store can watch for CAS updates. func TestStoreWatchCompareAndSwap(t *testing.T) { - s := newStore() + s := newTestStore(t) + defer s.Close() var eidx uint64 = 1 - s.Create("/foo", false, "bar", false, TTLOptionSet{ExpireTime: Permanent}) + s.Create("/foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent}) w, _ := s.Watch("/foo", false, false, 0) testutil.AssertEqual(t, w.StartIndex(), eidx) eidx = 2 - s.CompareAndSwap("/foo", "bar", 0, "baz", TTLOptionSet{ExpireTime: Permanent}) + s.CompareAndSwap("/foo", "bar", 0, "baz", store.TTLOptionSet{ExpireTime: store.Permanent}) e := nbselect(w.EventChan()) testutil.AssertEqual(t, e.EtcdIndex, eidx) testutil.AssertEqual(t, e.Action, "compareAndSwap") @@ -719,172 +656,28 @@ func TestStoreWatchCompareAndSwap(t *testing.T) { // Ensure that the store can watch for recursive CAS updates. func TestStoreWatchRecursiveCompareAndSwap(t *testing.T) { - s := newStore() + s := newTestStore(t) + defer s.Close() var eidx uint64 = 1 - s.Create("/foo/bar", false, "baz", false, TTLOptionSet{ExpireTime: Permanent}) + s.Create("/foo/bar", false, "baz", false, store.TTLOptionSet{ExpireTime: store.Permanent}) w, _ := s.Watch("/foo", true, false, 0) testutil.AssertEqual(t, w.StartIndex(), eidx) eidx = 2 - s.CompareAndSwap("/foo/bar", "baz", 0, "bat", TTLOptionSet{ExpireTime: Permanent}) + s.CompareAndSwap("/foo/bar", "baz", 0, "bat", store.TTLOptionSet{ExpireTime: store.Permanent}) e := nbselect(w.EventChan()) testutil.AssertEqual(t, e.EtcdIndex, eidx) testutil.AssertEqual(t, e.Action, "compareAndSwap") testutil.AssertEqual(t, e.Node.Key, "/foo/bar") } -// Ensure that the store can watch for key expiration. -func TestStoreWatchExpire(t *testing.T) { - s := newStore() - fc := newFakeClock() - s.clock = fc - - var eidx uint64 = 3 - s.Create("/foo", false, "bar", false, TTLOptionSet{ExpireTime: fc.Now().Add(400 * time.Millisecond)}) - s.Create("/foofoo", false, "barbarbar", false, TTLOptionSet{ExpireTime: fc.Now().Add(450 * time.Millisecond)}) - s.Create("/foodir", true, "", false, TTLOptionSet{ExpireTime: fc.Now().Add(500 * time.Millisecond)}) - - w, _ := s.Watch("/", true, false, 0) - testutil.AssertEqual(t, w.StartIndex(), eidx) - c := w.EventChan() - e := nbselect(c) - testutil.AssertNil(t, e) - fc.Advance(600 * time.Millisecond) - s.DeleteExpiredKeys(fc.Now()) - eidx = 4 - e = nbselect(c) - testutil.AssertEqual(t, e.EtcdIndex, eidx) - testutil.AssertEqual(t, e.Action, "expire") - testutil.AssertEqual(t, e.Node.Key, "/foo") - w, _ = s.Watch("/", true, false, 5) - eidx = 6 - testutil.AssertEqual(t, w.StartIndex(), eidx) - e = nbselect(w.EventChan()) - testutil.AssertEqual(t, e.EtcdIndex, eidx) - testutil.AssertEqual(t, e.Action, "expire") - testutil.AssertEqual(t, e.Node.Key, "/foofoo") - w, _ = s.Watch("/", true, false, 6) - e = nbselect(w.EventChan()) - testutil.AssertEqual(t, e.EtcdIndex, eidx) - testutil.AssertEqual(t, e.Action, "expire") - testutil.AssertEqual(t, e.Node.Key, "/foodir") - testutil.AssertEqual(t, e.Node.Dir, true) -} - -// Ensure that the store can watch for key expiration when refreshing. -func TestStoreWatchExpireRefresh(t *testing.T) { - s := newStore() - fc := newFakeClock() - s.clock = fc - - var eidx uint64 = 2 - s.Create("/foo", false, "bar", false, TTLOptionSet{ExpireTime: fc.Now().Add(500 * time.Millisecond), Refresh: true}) - s.Create("/foofoo", false, "barbarbar", false, TTLOptionSet{ExpireTime: fc.Now().Add(1200 * time.Millisecond), Refresh: true}) - - // Make sure we set watch updates when Refresh is true for newly created keys - w, _ := s.Watch("/", true, false, 0) - testutil.AssertEqual(t, w.StartIndex(), eidx) - c := w.EventChan() - e := nbselect(c) - testutil.AssertNil(t, e) - fc.Advance(600 * time.Millisecond) - s.DeleteExpiredKeys(fc.Now()) - eidx = 3 - e = nbselect(c) - testutil.AssertEqual(t, e.EtcdIndex, eidx) - testutil.AssertEqual(t, e.Action, "expire") - testutil.AssertEqual(t, e.Node.Key, "/foo") - - s.Update("/foofoo", "", TTLOptionSet{ExpireTime: fc.Now().Add(500 * time.Millisecond), Refresh: true}) - w, _ = s.Watch("/", true, false, 4) - fc.Advance(700 * time.Millisecond) - s.DeleteExpiredKeys(fc.Now()) - eidx = 5 // We should skip 4 because a TTL update should occur with no watch notification if set `TTLOptionSet.Refresh` to true - testutil.AssertEqual(t, w.StartIndex(), eidx-1) - e = nbselect(w.EventChan()) - testutil.AssertEqual(t, e.EtcdIndex, eidx) - testutil.AssertEqual(t, e.Action, "expire") - testutil.AssertEqual(t, e.Node.Key, "/foofoo") -} - -// Ensure that the store can watch for key expiration when refreshing with an empty value. -func TestStoreWatchExpireEmptyRefresh(t *testing.T) { - s := newStore() - fc := newFakeClock() - s.clock = fc - - var eidx uint64 = 1 - s.Create("/foo", false, "bar", false, TTLOptionSet{ExpireTime: fc.Now().Add(500 * time.Millisecond), Refresh: true}) - // Should be no-op - fc.Advance(200 * time.Millisecond) - s.DeleteExpiredKeys(fc.Now()) - - s.Update("/foo", "", TTLOptionSet{ExpireTime: fc.Now().Add(500 * time.Millisecond), Refresh: true}) - w, _ := s.Watch("/", true, false, 2) - fc.Advance(700 * time.Millisecond) - s.DeleteExpiredKeys(fc.Now()) - eidx = 3 // We should skip 2 because a TTL update should occur with no watch notification if set `TTLOptionSet.Refresh` to true - testutil.AssertEqual(t, w.StartIndex(), eidx-1) - e := nbselect(w.EventChan()) - testutil.AssertEqual(t, e.EtcdIndex, eidx) - testutil.AssertEqual(t, e.Action, "expire") - testutil.AssertEqual(t, e.Node.Key, "/foo") - testutil.AssertEqual(t, *e.PrevNode.Value, "bar") -} - -// Update TTL of a key (set TTLOptionSet.Refresh to false) and send notification -func TestStoreWatchNoRefresh(t *testing.T) { - s := newStore() - fc := newFakeClock() - s.clock = fc - - var eidx uint64 = 1 - s.Create("/foo", false, "bar", false, TTLOptionSet{ExpireTime: fc.Now().Add(500 * time.Millisecond), Refresh: true}) - // Should be no-op - fc.Advance(200 * time.Millisecond) - s.DeleteExpiredKeys(fc.Now()) - - // Update key's TTL with setting `TTLOptionSet.Refresh` to false will cause an update event - s.Update("/foo", "", TTLOptionSet{ExpireTime: fc.Now().Add(500 * time.Millisecond), Refresh: false}) - w, _ := s.Watch("/", true, false, 2) - fc.Advance(700 * time.Millisecond) - s.DeleteExpiredKeys(fc.Now()) - eidx = 2 - testutil.AssertEqual(t, w.StartIndex(), eidx) - e := nbselect(w.EventChan()) - testutil.AssertEqual(t, e.EtcdIndex, eidx) - testutil.AssertEqual(t, e.Action, "update") - testutil.AssertEqual(t, e.Node.Key, "/foo") - testutil.AssertEqual(t, *e.PrevNode.Value, "bar") -} - -// Ensure that the store can update the TTL on a value with refresh. -func TestStoreRefresh(t *testing.T) { - s := newStore() - fc := newFakeClock() - s.clock = fc - - s.Create("/foo", false, "bar", false, TTLOptionSet{ExpireTime: fc.Now().Add(500 * time.Millisecond)}) - s.Create("/bar", true, "bar", false, TTLOptionSet{ExpireTime: fc.Now().Add(500 * time.Millisecond)}) - _, err := s.Update("/foo", "", TTLOptionSet{ExpireTime: fc.Now().Add(500 * time.Millisecond), Refresh: true}) - testutil.AssertNil(t, err) - - _, err = s.Set("/foo", false, "", TTLOptionSet{ExpireTime: fc.Now().Add(500 * time.Millisecond), Refresh: true}) - testutil.AssertNil(t, err) - - _, err = s.Update("/bar", "", TTLOptionSet{ExpireTime: fc.Now().Add(500 * time.Millisecond), Refresh: true}) - testutil.AssertNil(t, err) - - _, err = s.CompareAndSwap("/foo", "bar", 0, "", TTLOptionSet{ExpireTime: fc.Now().Add(500 * time.Millisecond), Refresh: true}) - testutil.AssertNil(t, err) -} - // Ensure that the store can watch in streaming mode. func TestStoreWatchStream(t *testing.T) { - s := newStore() + s := newTestStore(t) + defer s.Close() var eidx uint64 = 1 w, _ := s.Watch("/foo", false, true, 0) // first modification - s.Create("/foo", false, "bar", false, TTLOptionSet{ExpireTime: Permanent}) + s.Create("/foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent}) e := nbselect(w.EventChan()) testutil.AssertEqual(t, e.EtcdIndex, eidx) testutil.AssertEqual(t, e.Action, "create") @@ -894,7 +687,7 @@ func TestStoreWatchStream(t *testing.T) { testutil.AssertNil(t, e) // second modification eidx = 2 - s.Update("/foo", "baz", TTLOptionSet{ExpireTime: Permanent}) + s.Update("/foo", "baz", store.TTLOptionSet{ExpireTime: store.Permanent}) e = nbselect(w.EventChan()) testutil.AssertEqual(t, e.EtcdIndex, eidx) testutil.AssertEqual(t, e.Action, "update") @@ -904,73 +697,13 @@ func TestStoreWatchStream(t *testing.T) { testutil.AssertNil(t, e) } -// Ensure that the store can recover from a previously saved state. -func TestStoreRecover(t *testing.T) { - s := newStore() - var eidx uint64 = 4 - s.Create("/foo", true, "", false, TTLOptionSet{ExpireTime: Permanent}) - s.Create("/foo/x", false, "bar", false, TTLOptionSet{ExpireTime: Permanent}) - s.Update("/foo/x", "barbar", TTLOptionSet{ExpireTime: Permanent}) - s.Create("/foo/y", false, "baz", false, TTLOptionSet{ExpireTime: Permanent}) - b, err := s.Save() - testutil.AssertNil(t, err) - - s2 := newStore() - s2.Recovery(b) - - e, err := s.Get("/foo/x", false, false) - testutil.AssertEqual(t, e.Node.CreatedIndex, uint64(2)) - testutil.AssertEqual(t, e.Node.ModifiedIndex, uint64(3)) - testutil.AssertEqual(t, e.EtcdIndex, eidx) - testutil.AssertNil(t, err) - testutil.AssertEqual(t, *e.Node.Value, "barbar") - - e, err = s.Get("/foo/y", false, false) - testutil.AssertEqual(t, e.EtcdIndex, eidx) - testutil.AssertNil(t, err) - testutil.AssertEqual(t, *e.Node.Value, "baz") -} - -// Ensure that the store can recover from a previously saved state that includes an expiring key. -func TestStoreRecoverWithExpiration(t *testing.T) { - s := newStore() - s.clock = newFakeClock() - - fc := newFakeClock() - - var eidx uint64 = 4 - s.Create("/foo", true, "", false, TTLOptionSet{ExpireTime: Permanent}) - s.Create("/foo/x", false, "bar", false, TTLOptionSet{ExpireTime: Permanent}) - s.Create("/foo/y", false, "baz", false, TTLOptionSet{ExpireTime: fc.Now().Add(5 * time.Millisecond)}) - b, err := s.Save() - testutil.AssertNil(t, err) - - time.Sleep(10 * time.Millisecond) - - s2 := newStore() - s2.clock = fc - - s2.Recovery(b) - - fc.Advance(600 * time.Millisecond) - s.DeleteExpiredKeys(fc.Now()) - - e, err := s.Get("/foo/x", false, false) - testutil.AssertNil(t, err) - testutil.AssertEqual(t, e.EtcdIndex, eidx) - testutil.AssertEqual(t, *e.Node.Value, "bar") - - e, err = s.Get("/foo/y", false, false) - testutil.AssertNotNil(t, err) - testutil.AssertNil(t, e) -} - // Ensure that the store can watch for hidden keys as long as it's an exact path match. func TestStoreWatchCreateWithHiddenKey(t *testing.T) { - s := newStore() + s := newTestStore(t) + defer s.Close() var eidx uint64 = 1 w, _ := s.Watch("/_foo", false, false, 0) - s.Create("/_foo", false, "bar", false, TTLOptionSet{ExpireTime: Permanent}) + s.Create("/_foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent}) e := nbselect(w.EventChan()) testutil.AssertEqual(t, e.EtcdIndex, eidx) testutil.AssertEqual(t, e.Action, "create") @@ -981,26 +714,28 @@ func TestStoreWatchCreateWithHiddenKey(t *testing.T) { // Ensure that the store doesn't see hidden key creates without an exact path match in recursive mode. func TestStoreWatchRecursiveCreateWithHiddenKey(t *testing.T) { - s := newStore() + s := newTestStore(t) + defer s.Close() w, _ := s.Watch("/foo", true, false, 0) - s.Create("/foo/_bar", false, "baz", false, TTLOptionSet{ExpireTime: Permanent}) + s.Create("/foo/_bar", false, "baz", false, store.TTLOptionSet{ExpireTime: store.Permanent}) e := nbselect(w.EventChan()) testutil.AssertNil(t, e) w, _ = s.Watch("/foo", true, false, 0) - s.Create("/foo/_baz", true, "", false, TTLOptionSet{ExpireTime: Permanent}) + s.Create("/foo/_baz", true, "", false, store.TTLOptionSet{ExpireTime: store.Permanent}) e = nbselect(w.EventChan()) testutil.AssertNil(t, e) - s.Create("/foo/_baz/quux", false, "quux", false, TTLOptionSet{ExpireTime: Permanent}) + s.Create("/foo/_baz/quux", false, "quux", false, store.TTLOptionSet{ExpireTime: store.Permanent}) e = nbselect(w.EventChan()) testutil.AssertNil(t, e) } // Ensure that the store doesn't see hidden key updates. func TestStoreWatchUpdateWithHiddenKey(t *testing.T) { - s := newStore() - s.Create("/_foo", false, "bar", false, TTLOptionSet{ExpireTime: Permanent}) + s := newTestStore(t) + defer s.Close() + s.Create("/_foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent}) w, _ := s.Watch("/_foo", false, false, 0) - s.Update("/_foo", "baz", TTLOptionSet{ExpireTime: Permanent}) + s.Update("/_foo", "baz", store.TTLOptionSet{ExpireTime: store.Permanent}) e := nbselect(w.EventChan()) testutil.AssertEqual(t, e.Action, "update") testutil.AssertEqual(t, e.Node.Key, "/_foo") @@ -1010,19 +745,21 @@ func TestStoreWatchUpdateWithHiddenKey(t *testing.T) { // Ensure that the store doesn't see hidden key updates without an exact path match in recursive mode. func TestStoreWatchRecursiveUpdateWithHiddenKey(t *testing.T) { - s := newStore() - s.Create("/foo/_bar", false, "baz", false, TTLOptionSet{ExpireTime: Permanent}) + s := newTestStore(t) + defer s.Close() + s.Create("/foo/_bar", false, "baz", false, store.TTLOptionSet{ExpireTime: store.Permanent}) w, _ := s.Watch("/foo", true, false, 0) - s.Update("/foo/_bar", "baz", TTLOptionSet{ExpireTime: Permanent}) + s.Update("/foo/_bar", "baz", store.TTLOptionSet{ExpireTime: store.Permanent}) e := nbselect(w.EventChan()) testutil.AssertNil(t, e) } // Ensure that the store can watch for key deletions. func TestStoreWatchDeleteWithHiddenKey(t *testing.T) { - s := newStore() + s := newTestStore(t) + defer s.Close() var eidx uint64 = 2 - s.Create("/_foo", false, "bar", false, TTLOptionSet{ExpireTime: Permanent}) + s.Create("/_foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent}) w, _ := s.Watch("/_foo", false, false, 0) s.Delete("/_foo", false, false) e := nbselect(w.EventChan()) @@ -1035,44 +772,22 @@ func TestStoreWatchDeleteWithHiddenKey(t *testing.T) { // Ensure that the store doesn't see hidden key deletes without an exact path match in recursive mode. func TestStoreWatchRecursiveDeleteWithHiddenKey(t *testing.T) { - s := newStore() - s.Create("/foo/_bar", false, "baz", false, TTLOptionSet{ExpireTime: Permanent}) + s := newTestStore(t) + defer s.Close() + s.Create("/foo/_bar", false, "baz", false, store.TTLOptionSet{ExpireTime: store.Permanent}) w, _ := s.Watch("/foo", true, false, 0) s.Delete("/foo/_bar", false, false) e := nbselect(w.EventChan()) testutil.AssertNil(t, e) } -// Ensure that the store doesn't see expirations of hidden keys. -func TestStoreWatchExpireWithHiddenKey(t *testing.T) { - s := newStore() - fc := newFakeClock() - s.clock = fc - - s.Create("/_foo", false, "bar", false, TTLOptionSet{ExpireTime: fc.Now().Add(500 * time.Millisecond)}) - s.Create("/foofoo", false, "barbarbar", false, TTLOptionSet{ExpireTime: fc.Now().Add(1000 * time.Millisecond)}) - - w, _ := s.Watch("/", true, false, 0) - c := w.EventChan() - e := nbselect(c) - testutil.AssertNil(t, e) - fc.Advance(600 * time.Millisecond) - s.DeleteExpiredKeys(fc.Now()) - e = nbselect(c) - testutil.AssertNil(t, e) - fc.Advance(600 * time.Millisecond) - s.DeleteExpiredKeys(fc.Now()) - e = nbselect(c) - testutil.AssertEqual(t, e.Action, "expire") - testutil.AssertEqual(t, e.Node.Key, "/foofoo") -} - // Ensure that the store does see hidden key creates if watching deeper than a hidden key in recursive mode. func TestStoreWatchRecursiveCreateDeeperThanHiddenKey(t *testing.T) { - s := newStore() + s := newTestStore(t) + defer s.Close() var eidx uint64 = 1 w, _ := s.Watch("/_foo/bar", true, false, 0) - s.Create("/_foo/bar/baz", false, "baz", false, TTLOptionSet{ExpireTime: Permanent}) + s.Create("/_foo/bar/baz", false, "baz", false, store.TTLOptionSet{ExpireTime: store.Permanent}) e := nbselect(w.EventChan()) testutil.AssertNotNil(t, e) @@ -1089,21 +804,22 @@ func TestStoreWatchRecursiveCreateDeeperThanHiddenKey(t *testing.T) { // This test ensures that after closing the channel, the store can continue // to operate correctly. func TestStoreWatchSlowConsumer(t *testing.T) { - s := newStore() + s := newTestStore(t) + defer s.Close() 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), TTLOptionSet{ExpireTime: Permanent}) // ok + s.Set("/foo", false, string(i), store.TTLOptionSet{ExpireTime: store.Permanent}) // ok } - testutil.AssertEqual(t, s.WatcherHub.count, int64(1)) - s.Set("/foo", false, "101", TTLOptionSet{ExpireTime: Permanent}) // ok + // testutil.AssertEqual(t, s.WatcherHub.count, int64(1)) + s.Set("/foo", false, "101", store.TTLOptionSet{ExpireTime: store.Permanent}) // ok // remove watcher - testutil.AssertEqual(t, s.WatcherHub.count, int64(0)) - s.Set("/foo", false, "102", TTLOptionSet{ExpireTime: Permanent}) // must not panic + // testutil.AssertEqual(t, s.WatcherHub.count, int64(0)) + s.Set("/foo", false, "102", store.TTLOptionSet{ExpireTime: store.Permanent}) // must not panic } // Performs a non-blocking select on an event channel. -func nbselect(c <-chan *Event) *Event { +func nbselect(c <-chan *store.Event) *store.Event { select { case e := <-c: return e @@ -1111,12 +827,3 @@ func nbselect(c <-chan *Event) *Event { return nil } } - -// newFakeClock creates a new FakeClock that has been advanced to at least minExpireTime -func newFakeClock() clockwork.FakeClock { - fc := clockwork.NewFakeClock() - for minExpireTime.After(fc.Now()) { - fc.Advance((0x1 << 62) * time.Nanosecond) - } - return fc -} diff --git a/store/store_ttl_test.go b/store/store_ttl_test.go new file mode 100644 index 00000000000..62c6a95ee7a --- /dev/null +++ b/store/store_ttl_test.go @@ -0,0 +1,360 @@ +// Copyright 2017 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package store + +import ( + "testing" + "time" + + etcdErr "github.com/coreos/etcd/error" + "github.com/coreos/etcd/pkg/testutil" + "github.com/jonboulle/clockwork" +) + +// Ensure that any TTL <= minExpireTime becomes Permanent +func TestMinExpireTime(t *testing.T) { + s := newStore() + fc := clockwork.NewFakeClock() + s.clock = fc + // FakeClock starts at 0, so minExpireTime should be far in the future.. but just in case + testutil.AssertTrue(t, minExpireTime.After(fc.Now()), "minExpireTime should be ahead of FakeClock!") + s.Create("/foo", false, "Y", false, TTLOptionSet{ExpireTime: fc.Now().Add(3 * time.Second)}) + fc.Advance(5 * time.Second) + // Ensure it hasn't expired + s.DeleteExpiredKeys(fc.Now()) + var eidx uint64 = 1 + e, err := s.Get("/foo", true, false) + testutil.AssertNil(t, err) + testutil.AssertEqual(t, e.EtcdIndex, eidx) + testutil.AssertEqual(t, e.Action, "get") + testutil.AssertEqual(t, e.Node.Key, "/foo") + testutil.AssertEqual(t, e.Node.TTL, int64(0)) +} + +// Ensure that the store can recursively retrieve a directory listing. +// Note that hidden files should not be returned. +func TestStoreGetDirectory(t *testing.T) { + s := newStore() + fc := newFakeClock() + s.clock = fc + s.Create("/foo", true, "", false, TTLOptionSet{ExpireTime: Permanent}) + s.Create("/foo/bar", false, "X", false, TTLOptionSet{ExpireTime: Permanent}) + s.Create("/foo/_hidden", false, "*", false, TTLOptionSet{ExpireTime: Permanent}) + s.Create("/foo/baz", true, "", false, TTLOptionSet{ExpireTime: Permanent}) + s.Create("/foo/baz/bat", false, "Y", false, TTLOptionSet{ExpireTime: Permanent}) + s.Create("/foo/baz/_hidden", false, "*", false, TTLOptionSet{ExpireTime: Permanent}) + s.Create("/foo/baz/ttl", false, "Y", false, TTLOptionSet{ExpireTime: fc.Now().Add(time.Second * 3)}) + var eidx uint64 = 7 + e, err := s.Get("/foo", true, false) + testutil.AssertNil(t, err) + testutil.AssertEqual(t, e.EtcdIndex, eidx) + testutil.AssertEqual(t, e.Action, "get") + testutil.AssertEqual(t, e.Node.Key, "/foo") + testutil.AssertEqual(t, len(e.Node.Nodes), 2) + var bazNodes NodeExterns + for _, node := range e.Node.Nodes { + switch node.Key { + case "/foo/bar": + testutil.AssertEqual(t, *node.Value, "X") + testutil.AssertEqual(t, node.Dir, false) + case "/foo/baz": + testutil.AssertEqual(t, node.Dir, true) + testutil.AssertEqual(t, len(node.Nodes), 2) + bazNodes = node.Nodes + default: + t.Errorf("key = %s, not matched", node.Key) + } + } + for _, node := range bazNodes { + switch node.Key { + case "/foo/baz/bat": + testutil.AssertEqual(t, *node.Value, "Y") + testutil.AssertEqual(t, node.Dir, false) + case "/foo/baz/ttl": + testutil.AssertEqual(t, *node.Value, "Y") + testutil.AssertEqual(t, node.Dir, false) + testutil.AssertEqual(t, node.TTL, int64(3)) + default: + t.Errorf("key = %s, not matched", node.Key) + } + } +} + +// Ensure that the store can update the TTL on a value. +func TestStoreUpdateValueTTL(t *testing.T) { + s := newStore() + fc := newFakeClock() + s.clock = fc + + var eidx uint64 = 2 + s.Create("/foo", false, "bar", false, TTLOptionSet{ExpireTime: Permanent}) + _, err := s.Update("/foo", "baz", TTLOptionSet{ExpireTime: fc.Now().Add(500 * time.Millisecond)}) + testutil.AssertNil(t, err) + e, _ := s.Get("/foo", false, false) + testutil.AssertEqual(t, *e.Node.Value, "baz") + testutil.AssertEqual(t, e.EtcdIndex, eidx) + fc.Advance(600 * time.Millisecond) + s.DeleteExpiredKeys(fc.Now()) + e, err = s.Get("/foo", false, false) + testutil.AssertNil(t, e) + testutil.AssertEqual(t, err.(*etcdErr.Error).ErrorCode, etcdErr.EcodeKeyNotFound) +} + +// Ensure that the store can update the TTL on a directory. +func TestStoreUpdateDirTTL(t *testing.T) { + s := newStore() + fc := newFakeClock() + s.clock = fc + + var eidx uint64 = 3 + s.Create("/foo", true, "", false, TTLOptionSet{ExpireTime: Permanent}) + s.Create("/foo/bar", false, "baz", false, TTLOptionSet{ExpireTime: Permanent}) + e, err := s.Update("/foo", "", TTLOptionSet{ExpireTime: fc.Now().Add(500 * time.Millisecond)}) + testutil.AssertNil(t, err) + testutil.AssertEqual(t, e.Node.Dir, true) + testutil.AssertEqual(t, e.EtcdIndex, eidx) + e, _ = s.Get("/foo/bar", false, false) + testutil.AssertEqual(t, *e.Node.Value, "baz") + testutil.AssertEqual(t, e.EtcdIndex, eidx) + + fc.Advance(600 * time.Millisecond) + s.DeleteExpiredKeys(fc.Now()) + e, err = s.Get("/foo/bar", false, false) + testutil.AssertNil(t, e) + testutil.AssertEqual(t, err.(*etcdErr.Error).ErrorCode, etcdErr.EcodeKeyNotFound) +} + +// Ensure that the store can watch for key expiration. +func TestStoreWatchExpire(t *testing.T) { + s := newStore() + fc := newFakeClock() + s.clock = fc + + var eidx uint64 = 3 + s.Create("/foo", false, "bar", false, TTLOptionSet{ExpireTime: fc.Now().Add(400 * time.Millisecond)}) + s.Create("/foofoo", false, "barbarbar", false, TTLOptionSet{ExpireTime: fc.Now().Add(450 * time.Millisecond)}) + s.Create("/foodir", true, "", false, TTLOptionSet{ExpireTime: fc.Now().Add(500 * time.Millisecond)}) + + w, _ := s.Watch("/", true, false, 0) + testutil.AssertEqual(t, w.StartIndex(), eidx) + c := w.EventChan() + e := nbselect(c) + testutil.AssertNil(t, e) + fc.Advance(600 * time.Millisecond) + s.DeleteExpiredKeys(fc.Now()) + eidx = 4 + e = nbselect(c) + testutil.AssertEqual(t, e.EtcdIndex, eidx) + testutil.AssertEqual(t, e.Action, "expire") + testutil.AssertEqual(t, e.Node.Key, "/foo") + w, _ = s.Watch("/", true, false, 5) + eidx = 6 + testutil.AssertEqual(t, w.StartIndex(), eidx) + e = nbselect(w.EventChan()) + testutil.AssertEqual(t, e.EtcdIndex, eidx) + testutil.AssertEqual(t, e.Action, "expire") + testutil.AssertEqual(t, e.Node.Key, "/foofoo") + w, _ = s.Watch("/", true, false, 6) + e = nbselect(w.EventChan()) + testutil.AssertEqual(t, e.EtcdIndex, eidx) + testutil.AssertEqual(t, e.Action, "expire") + testutil.AssertEqual(t, e.Node.Key, "/foodir") + testutil.AssertEqual(t, e.Node.Dir, true) +} + +// Ensure that the store can watch for key expiration when refreshing. +func TestStoreWatchExpireRefresh(t *testing.T) { + s := newStore() + fc := newFakeClock() + s.clock = fc + + var eidx uint64 = 2 + s.Create("/foo", false, "bar", false, TTLOptionSet{ExpireTime: fc.Now().Add(500 * time.Millisecond), Refresh: true}) + s.Create("/foofoo", false, "barbarbar", false, TTLOptionSet{ExpireTime: fc.Now().Add(1200 * time.Millisecond), Refresh: true}) + + // Make sure we set watch updates when Refresh is true for newly created keys + w, _ := s.Watch("/", true, false, 0) + testutil.AssertEqual(t, w.StartIndex(), eidx) + c := w.EventChan() + e := nbselect(c) + testutil.AssertNil(t, e) + fc.Advance(600 * time.Millisecond) + s.DeleteExpiredKeys(fc.Now()) + eidx = 3 + e = nbselect(c) + testutil.AssertEqual(t, e.EtcdIndex, eidx) + testutil.AssertEqual(t, e.Action, "expire") + testutil.AssertEqual(t, e.Node.Key, "/foo") + + s.Update("/foofoo", "", TTLOptionSet{ExpireTime: fc.Now().Add(500 * time.Millisecond), Refresh: true}) + w, _ = s.Watch("/", true, false, 4) + fc.Advance(700 * time.Millisecond) + s.DeleteExpiredKeys(fc.Now()) + eidx = 5 // We should skip 4 because a TTL update should occur with no watch notification if set `TTLOptionSet.Refresh` to true + testutil.AssertEqual(t, w.StartIndex(), eidx-1) + e = nbselect(w.EventChan()) + testutil.AssertEqual(t, e.EtcdIndex, eidx) + testutil.AssertEqual(t, e.Action, "expire") + testutil.AssertEqual(t, e.Node.Key, "/foofoo") +} + +// Ensure that the store can watch for key expiration when refreshing with an empty value. +func TestStoreWatchExpireEmptyRefresh(t *testing.T) { + s := newStore() + fc := newFakeClock() + s.clock = fc + + var eidx uint64 = 1 + s.Create("/foo", false, "bar", false, TTLOptionSet{ExpireTime: fc.Now().Add(500 * time.Millisecond), Refresh: true}) + // Should be no-op + fc.Advance(200 * time.Millisecond) + s.DeleteExpiredKeys(fc.Now()) + + s.Update("/foo", "", TTLOptionSet{ExpireTime: fc.Now().Add(500 * time.Millisecond), Refresh: true}) + w, _ := s.Watch("/", true, false, 2) + fc.Advance(700 * time.Millisecond) + s.DeleteExpiredKeys(fc.Now()) + eidx = 3 // We should skip 2 because a TTL update should occur with no watch notification if set `TTLOptionSet.Refresh` to true + testutil.AssertEqual(t, w.StartIndex(), eidx-1) + e := nbselect(w.EventChan()) + testutil.AssertEqual(t, e.EtcdIndex, eidx) + testutil.AssertEqual(t, e.Action, "expire") + testutil.AssertEqual(t, e.Node.Key, "/foo") + testutil.AssertEqual(t, *e.PrevNode.Value, "bar") +} + +// Update TTL of a key (set TTLOptionSet.Refresh to false) and send notification +func TestStoreWatchNoRefresh(t *testing.T) { + s := newStore() + fc := newFakeClock() + s.clock = fc + + var eidx uint64 = 1 + s.Create("/foo", false, "bar", false, TTLOptionSet{ExpireTime: fc.Now().Add(500 * time.Millisecond), Refresh: true}) + // Should be no-op + fc.Advance(200 * time.Millisecond) + s.DeleteExpiredKeys(fc.Now()) + + // Update key's TTL with setting `TTLOptionSet.Refresh` to false will cause an update event + s.Update("/foo", "", TTLOptionSet{ExpireTime: fc.Now().Add(500 * time.Millisecond), Refresh: false}) + w, _ := s.Watch("/", true, false, 2) + fc.Advance(700 * time.Millisecond) + s.DeleteExpiredKeys(fc.Now()) + eidx = 2 + testutil.AssertEqual(t, w.StartIndex(), eidx) + e := nbselect(w.EventChan()) + testutil.AssertEqual(t, e.EtcdIndex, eidx) + testutil.AssertEqual(t, e.Action, "update") + testutil.AssertEqual(t, e.Node.Key, "/foo") + testutil.AssertEqual(t, *e.PrevNode.Value, "bar") +} + +// Ensure that the store can update the TTL on a value with refresh. +func TestStoreRefresh(t *testing.T) { + s := newStore() + fc := newFakeClock() + s.clock = fc + + s.Create("/foo", false, "bar", false, TTLOptionSet{ExpireTime: fc.Now().Add(500 * time.Millisecond)}) + s.Create("/bar", true, "bar", false, TTLOptionSet{ExpireTime: fc.Now().Add(500 * time.Millisecond)}) + _, err := s.Update("/foo", "", TTLOptionSet{ExpireTime: fc.Now().Add(500 * time.Millisecond), Refresh: true}) + testutil.AssertNil(t, err) + + _, err = s.Set("/foo", false, "", TTLOptionSet{ExpireTime: fc.Now().Add(500 * time.Millisecond), Refresh: true}) + testutil.AssertNil(t, err) + + _, err = s.Update("/bar", "", TTLOptionSet{ExpireTime: fc.Now().Add(500 * time.Millisecond), Refresh: true}) + testutil.AssertNil(t, err) + + _, err = s.CompareAndSwap("/foo", "bar", 0, "", TTLOptionSet{ExpireTime: fc.Now().Add(500 * time.Millisecond), Refresh: true}) + testutil.AssertNil(t, err) +} + +// Ensure that the store can recover from a previously saved state that includes an expiring key. +func TestStoreRecoverWithExpiration(t *testing.T) { + s := newStore() + s.clock = newFakeClock() + + fc := newFakeClock() + + var eidx uint64 = 4 + s.Create("/foo", true, "", false, TTLOptionSet{ExpireTime: Permanent}) + s.Create("/foo/x", false, "bar", false, TTLOptionSet{ExpireTime: Permanent}) + s.Create("/foo/y", false, "baz", false, TTLOptionSet{ExpireTime: fc.Now().Add(5 * time.Millisecond)}) + b, err := s.Save() + testutil.AssertNil(t, err) + + time.Sleep(10 * time.Millisecond) + + s2 := newStore() + s2.clock = fc + + s2.Recovery(b) + + fc.Advance(600 * time.Millisecond) + s.DeleteExpiredKeys(fc.Now()) + + e, err := s.Get("/foo/x", false, false) + testutil.AssertNil(t, err) + testutil.AssertEqual(t, e.EtcdIndex, eidx) + testutil.AssertEqual(t, *e.Node.Value, "bar") + + e, err = s.Get("/foo/y", false, false) + testutil.AssertNotNil(t, err) + testutil.AssertNil(t, e) +} + +// Ensure that the store doesn't see expirations of hidden keys. +func TestStoreWatchExpireWithHiddenKey(t *testing.T) { + s := newStore() + fc := newFakeClock() + s.clock = fc + + s.Create("/_foo", false, "bar", false, TTLOptionSet{ExpireTime: fc.Now().Add(500 * time.Millisecond)}) + s.Create("/foofoo", false, "barbarbar", false, TTLOptionSet{ExpireTime: fc.Now().Add(1000 * time.Millisecond)}) + + w, _ := s.Watch("/", true, false, 0) + c := w.EventChan() + e := nbselect(c) + testutil.AssertNil(t, e) + fc.Advance(600 * time.Millisecond) + s.DeleteExpiredKeys(fc.Now()) + e = nbselect(c) + testutil.AssertNil(t, e) + fc.Advance(600 * time.Millisecond) + s.DeleteExpiredKeys(fc.Now()) + e = nbselect(c) + testutil.AssertEqual(t, e.Action, "expire") + testutil.AssertEqual(t, e.Node.Key, "/foofoo") +} + +// newFakeClock creates a new FakeClock that has been advanced to at least minExpireTime +func newFakeClock() clockwork.FakeClock { + fc := clockwork.NewFakeClock() + for minExpireTime.After(fc.Now()) { + fc.Advance((0x1 << 62) * time.Nanosecond) + } + return fc +} + +// Performs a non-blocking select on an event channel. +func nbselect(c <-chan *Event) *Event { + select { + case e := <-c: + return e + default: + return nil + } +} diff --git a/store/store_v2_test.go b/store/store_v2_test.go new file mode 100644 index 00000000000..29569817731 --- /dev/null +++ b/store/store_v2_test.go @@ -0,0 +1,62 @@ +// Copyright 2017 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// +build !v2v3 + +package store_test + +import ( + "testing" + + "github.com/coreos/etcd/pkg/testutil" + "github.com/coreos/etcd/store" +) + +type v2TestStore struct { + store.Store +} + +func (s *v2TestStore) Close() {} + +func newTestStore(t *testing.T, ns ...string) StoreCloser { + return &v2TestStore{store.New(ns...)} +} + +// Ensure that the store can recover from a previously saved state. +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}) + b, err := s.Save() + testutil.AssertNil(t, err) + + s2 := newTestStore(t) + s2.Recovery(b) + + e, err := s.Get("/foo/x", false, false) + testutil.AssertEqual(t, e.Node.CreatedIndex, uint64(2)) + testutil.AssertEqual(t, e.Node.ModifiedIndex, uint64(3)) + testutil.AssertEqual(t, e.EtcdIndex, eidx) + testutil.AssertNil(t, err) + testutil.AssertEqual(t, *e.Node.Value, "barbar") + + e, err = s.Get("/foo/y", false, false) + testutil.AssertEqual(t, e.EtcdIndex, eidx) + testutil.AssertNil(t, err) + testutil.AssertEqual(t, *e.Node.Value, "baz") +} From 75eb05a272a12c3b7f400a366ce4563e1894b1e4 Mon Sep 17 00:00:00 2001 From: Anthony Romano Date: Tue, 15 Aug 2017 10:55:03 -0700 Subject: [PATCH 06/11] store: test v2v3 store Changes main store tests to use a timeout select instead of expecting events to be immediately posted before returning. --- store/store_test.go | 91 +++++++++++++++++++++++++++------------- store/store_v2v3_test.go | 42 +++++++++++++++++++ 2 files changed, 104 insertions(+), 29 deletions(-) create mode 100644 store/store_v2v3_test.go diff --git a/store/store_test.go b/store/store_test.go index 00e837e8117..44f72efe4fd 100644 --- a/store/store_test.go +++ b/store/store_test.go @@ -16,6 +16,7 @@ package store_test import ( "testing" + "time" etcdErr "github.com/coreos/etcd/error" "github.com/coreos/etcd/pkg/testutil" @@ -551,12 +552,15 @@ func TestStoreWatchCreate(t *testing.T) { testutil.AssertEqual(t, w.StartIndex(), eidx) s.Create("/foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent}) eidx = 1 - e := nbselect(c) + e := timeoutSelect(t, c) testutil.AssertEqual(t, e.EtcdIndex, eidx) testutil.AssertEqual(t, e.Action, "create") testutil.AssertEqual(t, e.Node.Key, "/foo") - e = nbselect(c) - testutil.AssertNil(t, e) + select { + case e = <-w.EventChan(): + testutil.AssertNil(t, e) + case <-time.After(100 * time.Millisecond): + } } // Ensure that the store can watch for recursive key creation. @@ -564,11 +568,12 @@ func TestStoreWatchRecursiveCreate(t *testing.T) { s := newTestStore(t) defer s.Close() var eidx uint64 = 0 - w, _ := s.Watch("/foo", true, false, 0) + w, err := s.Watch("/foo", true, false, 0) + testutil.AssertNil(t, err) testutil.AssertEqual(t, w.StartIndex(), eidx) eidx = 1 s.Create("/foo/bar", false, "baz", false, store.TTLOptionSet{ExpireTime: store.Permanent}) - e := nbselect(w.EventChan()) + e := timeoutSelect(t, w.EventChan()) testutil.AssertEqual(t, e.EtcdIndex, eidx) testutil.AssertEqual(t, e.Action, "create") testutil.AssertEqual(t, e.Node.Key, "/foo/bar") @@ -584,7 +589,7 @@ func TestStoreWatchUpdate(t *testing.T) { testutil.AssertEqual(t, w.StartIndex(), eidx) eidx = 2 s.Update("/foo", "baz", store.TTLOptionSet{ExpireTime: store.Permanent}) - e := nbselect(w.EventChan()) + e := timeoutSelect(t, w.EventChan()) testutil.AssertEqual(t, e.EtcdIndex, eidx) testutil.AssertEqual(t, e.Action, "update") testutil.AssertEqual(t, e.Node.Key, "/foo") @@ -596,11 +601,12 @@ func TestStoreWatchRecursiveUpdate(t *testing.T) { defer s.Close() var eidx uint64 = 1 s.Create("/foo/bar", false, "baz", false, store.TTLOptionSet{ExpireTime: store.Permanent}) - w, _ := s.Watch("/foo", true, false, 0) + 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}) - e := nbselect(w.EventChan()) + e := timeoutSelect(t, w.EventChan()) testutil.AssertEqual(t, e.EtcdIndex, eidx) testutil.AssertEqual(t, e.Action, "update") testutil.AssertEqual(t, e.Node.Key, "/foo/bar") @@ -616,7 +622,7 @@ func TestStoreWatchDelete(t *testing.T) { testutil.AssertEqual(t, w.StartIndex(), eidx) eidx = 2 s.Delete("/foo", false, false) - e := nbselect(w.EventChan()) + e := timeoutSelect(t, w.EventChan()) testutil.AssertEqual(t, e.EtcdIndex, eidx) testutil.AssertEqual(t, e.Action, "delete") testutil.AssertEqual(t, e.Node.Key, "/foo") @@ -628,11 +634,12 @@ func TestStoreWatchRecursiveDelete(t *testing.T) { defer s.Close() var eidx uint64 = 1 s.Create("/foo/bar", false, "baz", false, store.TTLOptionSet{ExpireTime: store.Permanent}) - w, _ := s.Watch("/foo", true, false, 0) + w, err := s.Watch("/foo", true, false, 0) + testutil.AssertNil(t, err) testutil.AssertEqual(t, w.StartIndex(), eidx) eidx = 2 s.Delete("/foo/bar", false, false) - e := nbselect(w.EventChan()) + e := timeoutSelect(t, w.EventChan()) testutil.AssertEqual(t, e.EtcdIndex, eidx) testutil.AssertEqual(t, e.Action, "delete") testutil.AssertEqual(t, e.Node.Key, "/foo/bar") @@ -648,7 +655,7 @@ func TestStoreWatchCompareAndSwap(t *testing.T) { testutil.AssertEqual(t, w.StartIndex(), eidx) eidx = 2 s.CompareAndSwap("/foo", "bar", 0, "baz", store.TTLOptionSet{ExpireTime: store.Permanent}) - e := nbselect(w.EventChan()) + e := timeoutSelect(t, w.EventChan()) testutil.AssertEqual(t, e.EtcdIndex, eidx) testutil.AssertEqual(t, e.Action, "compareAndSwap") testutil.AssertEqual(t, e.Node.Key, "/foo") @@ -664,7 +671,7 @@ func TestStoreWatchRecursiveCompareAndSwap(t *testing.T) { testutil.AssertEqual(t, w.StartIndex(), eidx) eidx = 2 s.CompareAndSwap("/foo/bar", "baz", 0, "bat", store.TTLOptionSet{ExpireTime: store.Permanent}) - e := nbselect(w.EventChan()) + e := timeoutSelect(t, w.EventChan()) testutil.AssertEqual(t, e.EtcdIndex, eidx) testutil.AssertEqual(t, e.Action, "compareAndSwap") testutil.AssertEqual(t, e.Node.Key, "/foo/bar") @@ -678,23 +685,29 @@ func TestStoreWatchStream(t *testing.T) { w, _ := s.Watch("/foo", false, true, 0) // first modification s.Create("/foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent}) - e := nbselect(w.EventChan()) + e := timeoutSelect(t, w.EventChan()) testutil.AssertEqual(t, e.EtcdIndex, eidx) testutil.AssertEqual(t, e.Action, "create") testutil.AssertEqual(t, e.Node.Key, "/foo") testutil.AssertEqual(t, *e.Node.Value, "bar") - e = nbselect(w.EventChan()) - testutil.AssertNil(t, e) + select { + case e = <-w.EventChan(): + testutil.AssertNil(t, e) + case <-time.After(100 * time.Millisecond): + } // second modification eidx = 2 s.Update("/foo", "baz", store.TTLOptionSet{ExpireTime: store.Permanent}) - e = nbselect(w.EventChan()) + e = timeoutSelect(t, w.EventChan()) testutil.AssertEqual(t, e.EtcdIndex, eidx) testutil.AssertEqual(t, e.Action, "update") testutil.AssertEqual(t, e.Node.Key, "/foo") testutil.AssertEqual(t, *e.Node.Value, "baz") - e = nbselect(w.EventChan()) - testutil.AssertNil(t, e) + select { + case e = <-w.EventChan(): + testutil.AssertNil(t, e) + case <-time.After(100 * time.Millisecond): + } } // Ensure that the store can watch for hidden keys as long as it's an exact path match. @@ -704,12 +717,15 @@ func TestStoreWatchCreateWithHiddenKey(t *testing.T) { var eidx uint64 = 1 w, _ := s.Watch("/_foo", false, false, 0) s.Create("/_foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent}) - e := nbselect(w.EventChan()) + e := timeoutSelect(t, w.EventChan()) testutil.AssertEqual(t, e.EtcdIndex, eidx) testutil.AssertEqual(t, e.Action, "create") testutil.AssertEqual(t, e.Node.Key, "/_foo") - e = nbselect(w.EventChan()) - testutil.AssertNil(t, e) + select { + case e = <-w.EventChan(): + testutil.AssertNil(t, e) + case <-time.After(100 * time.Millisecond): + } } // Ensure that the store doesn't see hidden key creates without an exact path match in recursive mode. @@ -722,11 +738,17 @@ func TestStoreWatchRecursiveCreateWithHiddenKey(t *testing.T) { testutil.AssertNil(t, e) w, _ = s.Watch("/foo", true, false, 0) s.Create("/foo/_baz", true, "", false, store.TTLOptionSet{ExpireTime: store.Permanent}) - e = nbselect(w.EventChan()) - testutil.AssertNil(t, e) + 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}) - e = nbselect(w.EventChan()) - testutil.AssertNil(t, e) + select { + case e = <-w.EventChan(): + testutil.AssertNil(t, e) + case <-time.After(100 * time.Millisecond): + } } // Ensure that the store doesn't see hidden key updates. @@ -736,7 +758,7 @@ func TestStoreWatchUpdateWithHiddenKey(t *testing.T) { s.Create("/_foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent}) w, _ := s.Watch("/_foo", false, false, 0) s.Update("/_foo", "baz", store.TTLOptionSet{ExpireTime: store.Permanent}) - e := nbselect(w.EventChan()) + e := timeoutSelect(t, w.EventChan()) testutil.AssertEqual(t, e.Action, "update") testutil.AssertEqual(t, e.Node.Key, "/_foo") e = nbselect(w.EventChan()) @@ -762,7 +784,7 @@ func TestStoreWatchDeleteWithHiddenKey(t *testing.T) { s.Create("/_foo", false, "bar", false, store.TTLOptionSet{ExpireTime: store.Permanent}) w, _ := s.Watch("/_foo", false, false, 0) s.Delete("/_foo", false, false) - e := nbselect(w.EventChan()) + e := timeoutSelect(t, w.EventChan()) testutil.AssertEqual(t, e.EtcdIndex, eidx) testutil.AssertEqual(t, e.Action, "delete") testutil.AssertEqual(t, e.Node.Key, "/_foo") @@ -789,7 +811,7 @@ func TestStoreWatchRecursiveCreateDeeperThanHiddenKey(t *testing.T) { w, _ := s.Watch("/_foo/bar", true, false, 0) s.Create("/_foo/bar/baz", false, "baz", false, store.TTLOptionSet{ExpireTime: store.Permanent}) - e := nbselect(w.EventChan()) + e := timeoutSelect(t, w.EventChan()) testutil.AssertNotNil(t, e) testutil.AssertEqual(t, e.EtcdIndex, eidx) testutil.AssertEqual(t, e.Action, "create") @@ -827,3 +849,14 @@ func nbselect(c <-chan *store.Event) *store.Event { return nil } } + +// Performs a non-blocking select on an event channel. +func timeoutSelect(t *testing.T, c <-chan *store.Event) *store.Event { + select { + case e := <-c: + return e + case <-time.After(time.Second): + t.Errorf("timed out waiting on event") + return nil + } +} diff --git a/store/store_v2v3_test.go b/store/store_v2v3_test.go new file mode 100644 index 00000000000..2778b7c958f --- /dev/null +++ b/store/store_v2v3_test.go @@ -0,0 +1,42 @@ +// Copyright 2017 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// +build v2v3 + +package store_test + +import ( + "testing" + + "github.com/coreos/etcd/etcdserver/api/v2v3" + "github.com/coreos/etcd/integration" + "github.com/coreos/etcd/store" +) + +type v2v3TestStore struct { + store.Store + clus *integration.ClusterV3 + t *testing.T +} + +func (s *v2v3TestStore) Close() { s.clus.Terminate(s.t) } + +func newTestStore(t *testing.T, ns ...string) StoreCloser { + clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) + return &v2v3TestStore{ + v2v3.NewStore(clus.Client(0), "/v2/"), + clus, + t, + } +} From 5d669290e360f76505eb027e8f379d0371262fe2 Mon Sep 17 00:00:00 2001 From: Anthony Romano Date: Thu, 10 Aug 2017 22:26:46 -0700 Subject: [PATCH 07/11] embed: support experimental v2v3 proxy option --- embed/config.go | 1 + embed/etcd.go | 9 ++++++++- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/embed/config.go b/embed/config.go index 9f1516b6535..8d429cb0a5c 100644 --- a/embed/config.go +++ b/embed/config.go @@ -145,6 +145,7 @@ type Config struct { // Experimental flags ExperimentalCorruptCheckTime time.Duration `json:"experimental-corrupt-check-time"` + ExperimentalEnableV2V3 string `json:"experimental-enable-v2v3"` } // configYAML holds the config suitable for yaml parsing diff --git a/embed/etcd.go b/embed/etcd.go index 5959f405976..6ceb55b7930 100644 --- a/embed/etcd.go +++ b/embed/etcd.go @@ -29,6 +29,8 @@ import ( "github.com/coreos/etcd/etcdserver" "github.com/coreos/etcd/etcdserver/api/etcdhttp" "github.com/coreos/etcd/etcdserver/api/v2http" + "github.com/coreos/etcd/etcdserver/api/v2v3" + "github.com/coreos/etcd/etcdserver/api/v3client" "github.com/coreos/etcd/etcdserver/api/v3rpc" "github.com/coreos/etcd/pkg/cors" "github.com/coreos/etcd/pkg/debugutil" @@ -409,7 +411,12 @@ func (e *Etcd) serve() (err error) { // Start a client server goroutine for each listen address var h http.Handler if e.Config().EnableV2 { - h = v2http.NewClientHandler(e.Server, e.Server.Cfg.ReqTimeout()) + if len(e.Config().ExperimentalEnableV2V3) > 0 { + srv := v2v3.NewServer(v3client.New(e.Server), e.cfg.ExperimentalEnableV2V3) + h = v2http.NewClientHandler(srv, e.Server.Cfg.ReqTimeout()) + } else { + h = v2http.NewClientHandler(e.Server, e.Server.Cfg.ReqTimeout()) + } } else { mux := http.NewServeMux() etcdhttp.HandleBasic(mux, e.Server) From a0adee5209c3033d4c0ad600fbf0652e45d0785d Mon Sep 17 00:00:00 2001 From: Anthony Romano Date: Sun, 13 Aug 2017 18:12:40 -0700 Subject: [PATCH 08/11] etcdmain: add command line flag to etcdmain --- etcdmain/config.go | 1 + etcdmain/help.go | 2 ++ 2 files changed, 3 insertions(+) diff --git a/etcdmain/config.go b/etcdmain/config.go index 85c14c0d3a2..61411292363 100644 --- a/etcdmain/config.go +++ b/etcdmain/config.go @@ -158,6 +158,7 @@ func newConfig() *config { fs.BoolVar(&cfg.StrictReconfigCheck, "strict-reconfig-check", cfg.StrictReconfigCheck, "Reject reconfiguration requests that would cause quorum loss.") fs.BoolVar(&cfg.EnableV2, "enable-v2", true, "Accept etcd V2 client requests.") + fs.StringVar(&cfg.ExperimentalEnableV2V3, "experimental-enable-v2v3", cfg.ExperimentalEnableV2V3, "v3 prefix for serving emulated v2 state.") // proxy fs.Var(cfg.proxy, "proxy", fmt.Sprintf("Valid values include %s", strings.Join(cfg.proxy.Values, ", "))) diff --git a/etcdmain/help.go b/etcdmain/help.go index 152fbb649e3..37a670abdd5 100644 --- a/etcdmain/help.go +++ b/etcdmain/help.go @@ -183,5 +183,7 @@ auth flags: experimental flags: --experimental-corrupt-check-time '0s' duration of time between cluster corruption check passes. + --experimental-enable-v2v3 '' + serve v2 requests through the v3 backend under a given prefix. ` ) From e9cf07fa4d020ec3a539b9db97f46f782ba2de79 Mon Sep 17 00:00:00 2001 From: Anthony Romano Date: Fri, 18 Aug 2017 10:58:05 -0700 Subject: [PATCH 09/11] e2e: test v2v3 emulation --- e2e/cluster_test.go | 1 + e2e/v2_test.go | 19 +++++++++++++++++++ e2e/v2v3_test.go | 21 +++++++++++++++++++++ 3 files changed, 41 insertions(+) create mode 100644 e2e/v2_test.go create mode 100644 e2e/v2v3_test.go diff --git a/e2e/cluster_test.go b/e2e/cluster_test.go index 991fb13dce9..0ea7bd24bb9 100644 --- a/e2e/cluster_test.go +++ b/e2e/cluster_test.go @@ -212,6 +212,7 @@ func (cfg *etcdProcessClusterConfig) etcdServerProcessConfigs() []*etcdServerPro "--data-dir", dataDirPath, "--snapshot-count", fmt.Sprintf("%d", cfg.snapCount), } + args = addV2Args(args) if cfg.forceNewCluster { args = append(args, "--force-new-cluster") } diff --git a/e2e/v2_test.go b/e2e/v2_test.go new file mode 100644 index 00000000000..ad59211ecff --- /dev/null +++ b/e2e/v2_test.go @@ -0,0 +1,19 @@ +// Copyright 2017 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// +build !v2v3 + +package e2e + +func addV2Args(args []string) []string { return args } diff --git a/e2e/v2v3_test.go b/e2e/v2v3_test.go new file mode 100644 index 00000000000..b9c69466a45 --- /dev/null +++ b/e2e/v2v3_test.go @@ -0,0 +1,21 @@ +// Copyright 2017 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// +build v2v3 + +package e2e + +func addV2Args(args []string) []string { + return append(args, "--experimental-enable-v2v3", "v2/") +} From d4b8193c55934b8d78ec0ab9548d272996b62f4c Mon Sep 17 00:00:00 2001 From: Anthony Romano Date: Tue, 15 Aug 2017 22:26:55 -0700 Subject: [PATCH 10/11] hack/benchmark: update bench.sh to match procfile --- hack/benchmark/bench.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) mode change 100644 => 100755 hack/benchmark/bench.sh diff --git a/hack/benchmark/bench.sh b/hack/benchmark/bench.sh old mode 100644 new mode 100755 index 3955d9e6efc..d72efd3e202 --- a/hack/benchmark/bench.sh +++ b/hack/benchmark/bench.sh @@ -1,8 +1,8 @@ #!/bin/bash -e -leader=http://10.240.201.15:2379 +leader=http://localhost:2379 # assume three servers -servers=( http://10.240.201.15:2379 http://10.240.212.209:2379 http://10.240.95.3:2379 ) +servers=( http://localhost:2379 http://localhost:22379 http://localhost:32379 ) keyarray=( 64 256 ) From 32bfd9e5ab57cc7c15055c811cf779446039a6c1 Mon Sep 17 00:00:00 2001 From: Anthony Romano Date: Thu, 24 Aug 2017 22:55:41 -0700 Subject: [PATCH 11/11] test: add v2v3 store tests to integration and cov passes --- test | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/test b/test index e0cffe0f44f..6e60ced3c4f 100755 --- a/test +++ b/test @@ -87,10 +87,15 @@ function unit_pass { function integration_pass { echo "Running integration tests..." go test -timeout 15m -v -cpu 1,2,4 $@ ${REPO_PATH}/integration + integration_extra $@ +} + +function integration_extra { go test -timeout 1m -v ${RACE} -cpu 1,2,4 $@ ${REPO_PATH}/client/integration - go test -timeout 10m -v ${RACE} -cpu 1,2,4 $@ ${REPO_PATH}/clientv3/integration + go test -timeout 15m -v ${RACE} -cpu 1,2,4 $@ ${REPO_PATH}/clientv3/integration go test -timeout 1m -v -cpu 1,2,4 $@ ${REPO_PATH}/contrib/raftexample go test -timeout 1m -v ${RACE} -cpu 1,2,4 -run=Example $@ ${TEST} + go test -timeout 5m -v ${RACE} -tags v2v3 $@ ${REPO_PATH}/store } function functional_pass { @@ -162,6 +167,9 @@ function cov_pass { go test $GOCOVFLAGS -run=Test -coverprofile "$COVERDIR/${tf}.coverprofile" ${REPO_PATH}/$t || failed="$failed $t" done + # v2v3 tests + go test -tags v2v3 $GOCOVFLAGS -coverprofile "$COVERDIR/store-v2v3.coverprofile" ${REPO_PATH}/clientv3/integration || failed="$failed store-v2v3" + # proxy tests go test -tags cluster_proxy $GOCOVFLAGS -coverprofile "$COVERDIR/proxy_integration.coverprofile" ${REPO_PATH}/integration || failed="$failed proxy-integration" go test -tags cluster_proxy $GOCOVFLAGS -coverprofile "$COVERDIR/proxy_clientv3.coverprofile" ${REPO_PATH}/clientv3/integration || failed="$failed proxy-clientv3/integration" @@ -211,10 +219,7 @@ function integration_e2e_pass { intpid="$!" wait $e2epid wait $intpid - go test -timeout 1m -v ${RACE} -cpu 1,2,4 $@ ${REPO_PATH}/client/integration - go test -timeout 20m -v ${RACE} -cpu 1,2,4 $@ ${REPO_PATH}/clientv3/integration - go test -timeout 1m -v -cpu 1,2,4 $@ ${REPO_PATH}/contrib/raftexample - go test -timeout 1m -v ${RACE} -cpu 1,2,4 -run=Example $@ ${TEST} + integration_extra $@ } function grpcproxy_pass {