etcd/store/store.go

509 lines
13 KiB
Go
Raw Normal View History

2013-09-29 03:26:19 +04:00
package store
2013-09-03 22:30:42 +04:00
import (
2013-09-14 01:10:40 +04:00
"encoding/json"
2013-09-05 23:38:22 +04:00
"fmt"
"path"
2013-09-09 05:14:31 +04:00
"sort"
"strconv"
2013-09-03 22:30:42 +04:00
"strings"
2013-09-29 03:58:57 +04:00
"sync"
2013-09-03 22:30:42 +04:00
"time"
2013-09-29 03:26:19 +04:00
etcdErr "github.com/coreos/etcd/error"
2013-09-03 22:30:42 +04:00
)
2013-10-31 04:36:15 +04:00
// The default version to set when the store is first initialized.
const defaultVersion = 2
2013-10-14 21:12:30 +04:00
type Store interface {
2013-10-31 04:36:15 +04:00
Version() int
CommandFactory() CommandFactory
2013-10-14 21:12:30 +04:00
Get(nodePath string, recursive, sorted bool, index uint64, term uint64) (*Event, error)
2013-10-15 10:04:21 +04:00
Set(nodePath string, value string, expireTime time.Time, index uint64, term uint64) (*Event, error)
2013-10-16 10:18:03 +04:00
Update(nodePath string, newValue string, expireTime time.Time, index uint64, term uint64) (*Event, error)
2013-10-15 10:04:21 +04:00
Create(nodePath string, value string, incrementalSuffix bool, expireTime time.Time,
index uint64, term uint64) (*Event, error)
2013-10-15 09:22:23 +04:00
CompareAndSwap(nodePath string, prevValue string, prevIndex uint64,
2013-10-14 21:12:30 +04:00
value string, expireTime time.Time, index uint64, term uint64) (*Event, error)
Delete(nodePath string, recursive bool, index uint64, term uint64) (*Event, error)
Watch(prefix string, recursive bool, sinceIndex uint64, index uint64, term uint64) (<-chan *Event, error)
Save() ([]byte, error)
Recovery(state []byte) error
2013-10-31 02:14:34 +04:00
TotalTransactions() uint64
2013-10-14 21:12:30 +04:00
JsonStats() []byte
}
type store struct {
2013-10-31 04:36:15 +04:00
Root *Node
WatcherHub *watcherHub
2013-11-05 08:56:41 +04:00
TTLKeyHeap *TTLKeyHeap
2013-10-31 04:36:15 +04:00
Index uint64
Term uint64
Stats *Stats
CurrentVersion int
worldLock sync.RWMutex // stop the world lock
2013-09-03 22:30:42 +04:00
}
2013-10-14 21:12:30 +04:00
func New() Store {
return newStore()
}
func newStore() *store {
s := new(store)
2013-10-31 04:36:15 +04:00
s.CurrentVersion = defaultVersion
2013-10-14 21:12:30 +04:00
s.Root = newDir(s, "/", UndefIndex, UndefTerm, nil, "", Permanent)
2013-09-29 03:26:19 +04:00
s.Stats = newStats()
s.WatcherHub = newWatchHub(1000)
2013-11-05 08:56:41 +04:00
s.TTLKeyHeap = newTTLKeyHeap()
2013-09-29 03:26:19 +04:00
return s
2013-09-03 22:30:42 +04:00
}
2013-10-31 04:36:15 +04:00
// Version retrieves current version of the store.
func (s *store) Version() int {
return s.CurrentVersion
}
// CommandFactory retrieves the command factory for the current version of the store.
func (s *store) CommandFactory() CommandFactory {
return GetCommandFactory(s.Version())
}
// Get function returns a get event.
// If recursive is true, it will return all the content under the node path.
// If sorted is true, it will sort the content by keys.
2013-10-14 21:12:30 +04:00
func (s *store) Get(nodePath string, recursive, sorted bool, index uint64, term uint64) (*Event, error) {
2013-09-29 03:58:57 +04:00
s.worldLock.RLock()
defer s.worldLock.RUnlock()
2013-09-14 23:13:33 +04:00
nodePath = path.Clean(path.Join("/", nodePath))
2013-09-29 03:58:57 +04:00
n, err := s.internalGet(nodePath, index, term)
2013-09-03 22:30:42 +04:00
2013-09-04 05:27:46 +04:00
if err != nil {
2013-09-29 03:26:19 +04:00
s.Stats.Inc(GetFail)
2013-09-04 05:27:46 +04:00
return nil, err
}
2013-09-03 22:30:42 +04:00
e := newEvent(Get, nodePath, index, term)
2013-09-03 22:30:42 +04:00
if n.IsDir() { // node is a directory
2013-09-07 07:36:11 +04:00
e.Dir = true
2013-09-03 22:30:42 +04:00
2013-09-06 07:10:41 +04:00
children, _ := n.List()
e.KVPairs = make([]KeyValuePair, len(children))
// we do not use the index in the children slice directly
// we need to skip the hidden one
2013-09-04 05:27:46 +04:00
i := 0
2013-09-03 22:30:42 +04:00
2013-09-06 07:10:41 +04:00
for _, child := range children {
if child.IsHidden() { // get will not return hidden nodes
2013-09-04 06:35:25 +04:00
continue
}
e.KVPairs[i] = child.Pair(recursive, sorted)
2013-09-04 05:27:46 +04:00
i++
}
2013-09-04 07:10:33 +04:00
// eliminate hidden nodes
e.KVPairs = e.KVPairs[:i]
if sorted {
sort.Sort(e.KVPairs)
2013-09-09 05:14:31 +04:00
}
2013-10-09 08:49:10 +04:00
} else { // node is a file
e.Value, _ = n.Read()
2013-09-03 22:30:42 +04:00
}
2013-10-09 08:49:10 +04:00
e.Expiration, e.TTL = n.ExpirationAndTTL()
2013-09-14 23:13:33 +04:00
2013-09-29 03:26:19 +04:00
s.Stats.Inc(GetSuccess)
2013-09-04 05:27:46 +04:00
return e, nil
2013-09-03 22:30:42 +04:00
}
// Create function creates the Node at nodePath. Create will help to create intermediate directories with no ttl.
// If the node has already existed, create will fail.
// If any node on the path is a file, create will fail.
func (s *store) Create(nodePath string, value string, unique bool,
expireTime time.Time, index uint64, term uint64) (*Event, error) {
2013-10-06 22:23:52 +04:00
nodePath = path.Clean(path.Join("/", nodePath))
2013-10-01 10:18:52 +04:00
s.worldLock.Lock()
defer s.worldLock.Unlock()
2013-10-16 10:18:03 +04:00
e, err := s.internalCreate(nodePath, value, unique, false, expireTime, index, term, Create)
if err == nil {
s.Stats.Inc(CreateSuccess)
} else {
s.Stats.Inc(CreateFail)
}
return e, err
2013-10-15 10:04:21 +04:00
}
// Set function creates or replace the Node at nodePath.
func (s *store) Set(nodePath string, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) {
nodePath = path.Clean(path.Join("/", nodePath))
s.worldLock.Lock()
defer s.worldLock.Unlock()
2013-10-16 10:18:03 +04:00
e, err := s.internalCreate(nodePath, value, false, true, expireTime, index, term, Set)
if err == nil {
s.Stats.Inc(SetSuccess)
} else {
s.Stats.Inc(SetFail)
}
return e, err
2013-09-03 22:30:42 +04:00
}
2013-10-15 09:22:23 +04:00
func (s *store) CompareAndSwap(nodePath string, prevValue string, prevIndex uint64,
value string, expireTime time.Time, index uint64, term uint64) (*Event, error) {
2013-10-06 22:23:52 +04:00
nodePath = path.Clean(path.Join("/", nodePath))
2013-10-01 10:18:52 +04:00
s.worldLock.Lock()
defer s.worldLock.Unlock()
2013-09-29 03:58:57 +04:00
n, err := s.internalGet(nodePath, index, term)
2013-09-03 22:30:42 +04:00
2013-09-05 23:38:22 +04:00
if err != nil {
s.Stats.Inc(CompareAndSwapFail)
2013-09-05 23:38:22 +04:00
return nil, err
}
2013-09-03 22:30:42 +04:00
if n.IsDir() { // can only test and set file
s.Stats.Inc(CompareAndSwapFail)
2013-10-06 22:23:52 +04:00
return nil, etcdErr.NewError(etcdErr.EcodeNotFile, nodePath, index, term)
2013-09-05 23:38:22 +04:00
}
2013-10-14 23:57:35 +04:00
// If both of the prevValue and prevIndex are given, we will test both of them.
// Command will be executed, only if both of the tests are successful.
if (prevValue == "" || n.Value == prevValue) && (prevIndex == 0 || n.ModifiedIndex == prevIndex) {
e := newEvent(CompareAndSwap, nodePath, index, term)
2013-10-09 09:10:18 +04:00
e.PrevValue = n.Value
2013-09-05 23:38:22 +04:00
// if test succeed, write the value
n.Write(value, index, term)
2013-10-14 21:12:30 +04:00
n.UpdateTTL(expireTime)
e.Value = value
2013-10-09 08:49:10 +04:00
e.Expiration, e.TTL = n.ExpirationAndTTL()
2013-10-06 22:23:52 +04:00
2013-09-29 03:26:19 +04:00
s.WatcherHub.notify(e)
s.Stats.Inc(CompareAndSwapSuccess)
2013-09-05 23:38:22 +04:00
return e, nil
}
cause := fmt.Sprintf("[%v != %v] [%v != %v]", prevValue, n.Value, prevIndex, n.ModifiedIndex)
s.Stats.Inc(CompareAndSwapFail)
2013-10-06 22:23:52 +04:00
return nil, etcdErr.NewError(etcdErr.EcodeTestFailed, cause, index, term)
2013-09-03 22:30:42 +04:00
}
// Delete function deletes the node at the given path.
// If the node is a directory, recursive must be true to delete it.
2013-10-14 21:12:30 +04:00
func (s *store) Delete(nodePath string, recursive bool, index uint64, term uint64) (*Event, error) {
2013-10-06 22:23:52 +04:00
nodePath = path.Clean(path.Join("/", nodePath))
2013-10-01 10:18:52 +04:00
s.worldLock.Lock()
defer s.worldLock.Unlock()
2013-09-29 03:58:57 +04:00
n, err := s.internalGet(nodePath, index, term)
2013-09-03 22:30:42 +04:00
if err != nil { // if the node does not exist, return error
2013-09-29 03:26:19 +04:00
s.Stats.Inc(DeleteFail)
2013-09-04 06:35:25 +04:00
return nil, err
}
e := newEvent(Delete, nodePath, index, term)
2013-09-04 06:35:25 +04:00
if n.IsDir() {
e.Dir = true
} else {
e.PrevValue = n.Value
}
callback := func(path string) { // notify function
2013-10-08 09:17:58 +04:00
// notify the watchers with delted set true
s.WatcherHub.notifyWatchers(e, path, true)
2013-09-07 09:05:11 +04:00
}
err = n.Remove(recursive, callback)
2013-09-07 09:05:11 +04:00
if err != nil {
2013-09-29 03:26:19 +04:00
s.Stats.Inc(DeleteFail)
2013-09-07 09:05:11 +04:00
return nil, err
}
2013-09-29 03:26:19 +04:00
s.WatcherHub.notify(e)
s.Stats.Inc(DeleteSuccess)
2013-09-07 09:05:11 +04:00
2013-09-04 06:35:25 +04:00
return e, nil
2013-09-03 22:30:42 +04:00
}
2013-10-14 21:12:30 +04:00
func (s *store) Watch(prefix string, recursive bool, sinceIndex uint64, index uint64, term uint64) (<-chan *Event, error) {
2013-10-06 22:23:52 +04:00
prefix = path.Clean(path.Join("/", prefix))
2013-09-29 03:58:57 +04:00
s.worldLock.RLock()
defer s.worldLock.RUnlock()
2013-09-29 03:26:19 +04:00
s.Index, s.Term = index, term
2013-09-14 23:13:33 +04:00
2013-10-06 22:23:52 +04:00
var c <-chan *Event
var err *etcdErr.Error
2013-09-14 23:13:33 +04:00
if sinceIndex == 0 {
2013-10-06 22:23:52 +04:00
c, err = s.WatcherHub.watch(prefix, recursive, index+1)
} else {
c, err = s.WatcherHub.watch(prefix, recursive, sinceIndex)
2013-09-14 23:13:33 +04:00
}
2013-10-06 22:23:52 +04:00
if err != nil {
err.Index = index
err.Term = term
return nil, err
2013-10-06 22:23:52 +04:00
}
return c, nil
2013-09-14 23:13:33 +04:00
}
// walk function walks all the nodePath and apply the walkFunc on each directory
2013-10-14 21:12:30 +04:00
func (s *store) walk(nodePath string, walkFunc func(prev *Node, component string) (*Node, *etcdErr.Error)) (*Node, *etcdErr.Error) {
components := strings.Split(nodePath, "/")
2013-09-03 22:30:42 +04:00
2013-09-29 03:26:19 +04:00
curr := s.Root
2013-10-06 22:23:52 +04:00
var err *etcdErr.Error
2013-09-03 22:30:42 +04:00
for i := 1; i < len(components); i++ {
if len(components[i]) == 0 { // ignore empty string
return curr, nil
}
curr, err = walkFunc(curr, components[i])
if err != nil {
return nil, err
}
}
return curr, nil
}
2013-10-15 09:22:23 +04:00
// Update function updates the value/ttl of the node.
// If the node is a file, the value and the ttl can be updated.
// If the node is a directory, only the ttl can be updated.
2013-10-16 10:18:03 +04:00
func (s *store) Update(nodePath string, newValue string, expireTime time.Time, index uint64, term uint64) (*Event, error) {
2013-10-15 09:22:23 +04:00
s.worldLock.Lock()
defer s.worldLock.Unlock()
nodePath = path.Clean(path.Join("/", nodePath))
n, err := s.internalGet(nodePath, index, term)
if err != nil { // if the node does not exist, return error
s.Stats.Inc(UpdateFail)
return nil, err
}
e := newEvent(Update, nodePath, s.Index, s.Term)
if len(newValue) != 0 {
if n.IsDir() {
// if the node is a directory, we cannot update value
s.Stats.Inc(UpdateFail)
return nil, etcdErr.NewError(etcdErr.EcodeNotFile, nodePath, index, term)
}
e.PrevValue = n.Value
n.Write(newValue, index, term)
}
// update ttl
n.UpdateTTL(expireTime)
2013-10-19 22:51:41 +04:00
e.Value = newValue
2013-10-15 09:22:23 +04:00
e.Expiration, e.TTL = n.ExpirationAndTTL()
s.WatcherHub.notify(e)
s.Stats.Inc(UpdateSuccess)
return e, nil
}
func (s *store) internalCreate(nodePath string, value string, unique bool, replace bool,
2013-10-06 22:23:52 +04:00
expireTime time.Time, index uint64, term uint64, action string) (*Event, error) {
s.Index, s.Term = index, term
if unique { // append unique item under the node path
nodePath += "/" + strconv.FormatUint(index, 10)
}
2013-10-03 09:15:12 +04:00
nodePath = path.Clean(path.Join("/", nodePath))
2013-10-06 22:23:52 +04:00
dir, newNodeName := path.Split(nodePath)
2013-10-03 09:15:12 +04:00
// walk through the nodePath, create dirs and get the last directory node
d, err := s.walk(dir, s.checkDir)
if err != nil {
s.Stats.Inc(SetFail)
err.Index, err.Term = s.Index, s.Term
2013-10-03 09:15:12 +04:00
return nil, err
}
e := newEvent(action, nodePath, s.Index, s.Term)
n, _ := d.GetChild(newNodeName)
// force will try to replace a existing file
if n != nil {
2013-10-15 10:15:31 +04:00
if replace {
2013-10-06 22:23:52 +04:00
if n.IsDir() {
return nil, etcdErr.NewError(etcdErr.EcodeNotFile, nodePath, index, term)
}
e.PrevValue, _ = n.Read()
n.Remove(false, nil)
} else {
return nil, etcdErr.NewError(etcdErr.EcodeNodeExist, nodePath, index, term)
2013-10-06 22:23:52 +04:00
}
}
2013-10-03 09:15:12 +04:00
if len(value) != 0 { // create file
e.Value = value
2013-10-14 21:12:30 +04:00
n = newKV(s, nodePath, value, index, term, d, "", expireTime)
2013-10-03 09:15:12 +04:00
} else { // create directory
e.Dir = true
2013-10-14 21:12:30 +04:00
n = newDir(s, nodePath, index, term, d, "", expireTime)
2013-10-03 09:15:12 +04:00
}
2013-10-16 10:18:03 +04:00
// we are sure d is a directory and does not have the children with name n.Name
d.Add(n)
2013-10-03 09:15:12 +04:00
// Node with TTL
2013-11-05 08:56:41 +04:00
if !n.IsPermanent() {
2013-10-14 21:12:30 +04:00
n.Expire()
2013-10-09 08:49:10 +04:00
e.Expiration, e.TTL = n.ExpirationAndTTL()
2013-10-03 09:15:12 +04:00
}
s.WatcherHub.notify(e)
return e, nil
}
// InternalGet function get the node of the given nodePath.
2013-10-14 21:12:30 +04:00
func (s *store) internalGet(nodePath string, index uint64, term uint64) (*Node, *etcdErr.Error) {
2013-09-14 23:13:33 +04:00
nodePath = path.Clean(path.Join("/", nodePath))
2013-09-04 05:27:46 +04:00
// update file system known index and term
2013-10-11 02:40:05 +04:00
if index > s.Index {
s.Index, s.Term = index, term
}
2013-09-04 05:27:46 +04:00
2013-10-06 22:23:52 +04:00
walkFunc := func(parent *Node, name string) (*Node, *etcdErr.Error) {
2013-09-05 23:38:22 +04:00
if !parent.IsDir() {
2013-10-06 22:23:52 +04:00
err := etcdErr.NewError(etcdErr.EcodeNotDir, parent.Path, index, term)
return nil, err
2013-09-05 23:38:22 +04:00
}
child, ok := parent.Children[name]
2013-09-04 05:27:46 +04:00
if ok {
return child, nil
}
2013-10-06 22:23:52 +04:00
return nil, etcdErr.NewError(etcdErr.EcodeKeyNotFound, path.Join(parent.Path, name), index, term)
2013-09-04 05:27:46 +04:00
}
2013-09-29 03:26:19 +04:00
f, err := s.walk(nodePath, walkFunc)
2013-09-04 05:27:46 +04:00
if err != nil {
return nil, err
}
return f, nil
}
2013-09-03 22:30:42 +04:00
// checkDir function will check whether the component is a directory under parent node.
// If it is a directory, this function will return the pointer to that node.
// If it does not exist, this function will create a new directory and return the pointer to that node.
// If it is a file, this function will return error.
2013-10-14 21:12:30 +04:00
func (s *store) checkDir(parent *Node, dirName string) (*Node, *etcdErr.Error) {
node, ok := parent.Children[dirName]
2013-09-03 22:30:42 +04:00
if ok {
if node.IsDir() {
return node, nil
}
return nil, etcdErr.NewError(etcdErr.EcodeNotDir, parent.Path, UndefIndex, UndefTerm)
2013-09-03 22:30:42 +04:00
}
2013-10-14 21:12:30 +04:00
n := newDir(s, path.Join(parent.Path, dirName), s.Index, s.Term, parent, parent.ACL, Permanent)
2013-09-03 22:30:42 +04:00
parent.Children[dirName] = n
return n, nil
}
2013-09-14 01:10:40 +04:00
// Save function saves the static state of the store system.
// Save function will not be able to save the state of watchers.
// Save function will not save the parent field of the node. Or there will
// be cyclic dependencies issue for the json package.
2013-10-14 21:12:30 +04:00
func (s *store) Save() ([]byte, error) {
2013-09-29 03:58:57 +04:00
s.worldLock.Lock()
2013-10-14 21:12:30 +04:00
clonedStore := newStore()
2013-09-29 03:26:19 +04:00
clonedStore.Index = s.Index
clonedStore.Term = s.Term
2013-09-29 04:41:45 +04:00
clonedStore.Root = s.Root.Clone()
clonedStore.WatcherHub = s.WatcherHub.clone()
2013-09-29 03:58:57 +04:00
clonedStore.Stats = s.Stats.clone()
2013-10-31 04:36:15 +04:00
clonedStore.CurrentVersion = s.CurrentVersion
2013-09-29 03:58:57 +04:00
s.worldLock.Unlock()
2013-09-14 01:10:40 +04:00
2013-09-29 03:26:19 +04:00
b, err := json.Marshal(clonedStore)
2013-09-14 01:10:40 +04:00
if err != nil {
2013-09-16 06:28:42 +04:00
return nil, err
2013-09-14 01:10:40 +04:00
}
2013-09-16 06:28:42 +04:00
return b, nil
2013-09-14 01:10:40 +04:00
}
// recovery function recovery the store system from a static state.
// It needs to recovery the parent field of the nodes.
// It needs to delete the expired nodes since the saved time and also
// need to create monitor go routines.
2013-10-14 21:12:30 +04:00
func (s *store) Recovery(state []byte) error {
2013-09-29 03:58:57 +04:00
s.worldLock.Lock()
defer s.worldLock.Unlock()
2013-09-29 03:26:19 +04:00
err := json.Unmarshal(state, s)
2013-09-14 01:10:40 +04:00
if err != nil {
2013-09-16 06:28:42 +04:00
return err
2013-09-14 01:10:40 +04:00
}
2013-10-14 21:12:30 +04:00
s.Root.recoverAndclean()
2013-09-16 06:28:42 +04:00
return nil
2013-09-14 01:10:40 +04:00
}
2013-09-29 03:26:19 +04:00
2013-10-14 21:12:30 +04:00
func (s *store) JsonStats() []byte {
2013-09-29 03:26:19 +04:00
s.Stats.Watchers = uint64(s.WatcherHub.count)
return s.Stats.toJson()
}
2013-10-31 02:14:34 +04:00
func (s *store) TotalTransactions() uint64 {
return s.Stats.TotalTranscations()
}