mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00
v2v3: ServerV2 backed by clientv3
This commit is contained in:
parent
525fbba1bd
commit
8091be6e97
31
etcdserver/api/v2v3/cluster.go
Normal file
31
etcdserver/api/v2v3/cluster.go
Normal file
@ -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") }
|
16
etcdserver/api/v2v3/doc.go
Normal file
16
etcdserver/api/v2v3/doc.go
Normal file
@ -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
|
117
etcdserver/api/v2v3/server.go
Normal file
117
etcdserver/api/v2v3/server.go
Normal file
@ -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
|
||||
}
|
621
etcdserver/api/v2v3/store.go
Normal file
621
etcdserver/api/v2v3/store.go
Normal file
@ -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))
|
||||
}
|
140
etcdserver/api/v2v3/watcher.go
Normal file
140
etcdserver/api/v2v3/watcher.go
Normal file
@ -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 }
|
Loading…
x
Reference in New Issue
Block a user