go.etcd.io/etcd@v3.3.27+incompatible/contrib/raftexample/kvstore.go (about) 1 // Copyright 2015 The etcd Authors 2 // 3 // Licensed under the Apache License, Version 2.0 (the "License"); 4 // you may not use this file except in compliance with the License. 5 // You may obtain a copy of the License at 6 // 7 // http://www.apache.org/licenses/LICENSE-2.0 8 // 9 // Unless required by applicable law or agreed to in writing, software 10 // distributed under the License is distributed on an "AS IS" BASIS, 11 // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 // See the License for the specific language governing permissions and 13 // limitations under the License. 14 15 package main 16 17 import ( 18 "bytes" 19 "encoding/gob" 20 "encoding/json" 21 "log" 22 "sync" 23 24 "github.com/coreos/etcd/snap" 25 ) 26 27 // a key-value store backed by raft 28 type kvstore struct { 29 proposeC chan<- string // channel for proposing updates 30 mu sync.RWMutex 31 kvStore map[string]string // current committed key-value pairs 32 snapshotter *snap.Snapshotter 33 } 34 35 type kv struct { 36 Key string 37 Val string 38 } 39 40 func newKVStore(snapshotter *snap.Snapshotter, proposeC chan<- string, commitC <-chan *string, errorC <-chan error) *kvstore { 41 s := &kvstore{proposeC: proposeC, kvStore: make(map[string]string), snapshotter: snapshotter} 42 // replay log into key-value map 43 s.readCommits(commitC, errorC) 44 // read commits from raft into kvStore map until error 45 go s.readCommits(commitC, errorC) 46 return s 47 } 48 49 func (s *kvstore) Lookup(key string) (string, bool) { 50 s.mu.RLock() 51 v, ok := s.kvStore[key] 52 s.mu.RUnlock() 53 return v, ok 54 } 55 56 func (s *kvstore) Propose(k string, v string) { 57 var buf bytes.Buffer 58 if err := gob.NewEncoder(&buf).Encode(kv{k, v}); err != nil { 59 log.Fatal(err) 60 } 61 s.proposeC <- buf.String() 62 } 63 64 func (s *kvstore) readCommits(commitC <-chan *string, errorC <-chan error) { 65 for data := range commitC { 66 if data == nil { 67 // done replaying log; new data incoming 68 // OR signaled to load snapshot 69 snapshot, err := s.snapshotter.Load() 70 if err == snap.ErrNoSnapshot { 71 return 72 } 73 if err != nil && err != snap.ErrNoSnapshot { 74 log.Panic(err) 75 } 76 log.Printf("loading snapshot at term %d and index %d", snapshot.Metadata.Term, snapshot.Metadata.Index) 77 if err := s.recoverFromSnapshot(snapshot.Data); err != nil { 78 log.Panic(err) 79 } 80 continue 81 } 82 83 var dataKv kv 84 dec := gob.NewDecoder(bytes.NewBufferString(*data)) 85 if err := dec.Decode(&dataKv); err != nil { 86 log.Fatalf("raftexample: could not decode message (%v)", err) 87 } 88 s.mu.Lock() 89 s.kvStore[dataKv.Key] = dataKv.Val 90 s.mu.Unlock() 91 } 92 if err, ok := <-errorC; ok { 93 log.Fatal(err) 94 } 95 } 96 97 func (s *kvstore) getSnapshot() ([]byte, error) { 98 s.mu.Lock() 99 defer s.mu.Unlock() 100 return json.Marshal(s.kvStore) 101 } 102 103 func (s *kvstore) recoverFromSnapshot(snapshot []byte) error { 104 var store map[string]string 105 if err := json.Unmarshal(snapshot, &store); err != nil { 106 return err 107 } 108 s.mu.Lock() 109 s.kvStore = store 110 s.mu.Unlock() 111 return nil 112 }