kvstore.go 11 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474
  1. // Copyright 2015 CoreOS, Inc.
  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. package storage
  15. import (
  16. "errors"
  17. "log"
  18. "math"
  19. "math/rand"
  20. "sync"
  21. "time"
  22. "github.com/coreos/etcd/lease"
  23. "github.com/coreos/etcd/storage/backend"
  24. "github.com/coreos/etcd/storage/storagepb"
  25. )
  26. var (
  27. keyBucketName = []byte("key")
  28. metaBucketName = []byte("meta")
  29. // markedRevBytesLen is the byte length of marked revision.
  30. // The first `revBytesLen` bytes represents a normal revision. The last
  31. // one byte is the mark.
  32. markedRevBytesLen = revBytesLen + 1
  33. markBytePosition = markedRevBytesLen - 1
  34. markTombstone byte = 't'
  35. NoLease = lease.LeaseID(0)
  36. scheduledCompactKeyName = []byte("scheduledCompactRev")
  37. finishedCompactKeyName = []byte("finishedCompactRev")
  38. ErrTxnIDMismatch = errors.New("storage: txn id mismatch")
  39. ErrCompacted = errors.New("storage: required revision has been compacted")
  40. ErrFutureRev = errors.New("storage: required revision is a future revision")
  41. ErrCanceled = errors.New("storage: watcher is canceled")
  42. )
  43. type store struct {
  44. mu sync.Mutex // guards the following
  45. b backend.Backend
  46. kvindex index
  47. currentRev revision
  48. // the main revision of the last compaction
  49. compactMainRev int64
  50. tx backend.BatchTx
  51. txnID int64 // tracks the current txnID to verify txn operations
  52. wg sync.WaitGroup
  53. stopc chan struct{}
  54. }
  55. // NewStore returns a new store. It is useful to create a store inside
  56. // storage pkg. It should only be used for testing externally.
  57. func NewStore(b backend.Backend) *store {
  58. s := &store{
  59. b: b,
  60. kvindex: newTreeIndex(),
  61. currentRev: revision{},
  62. compactMainRev: -1,
  63. stopc: make(chan struct{}),
  64. }
  65. tx := s.b.BatchTx()
  66. tx.Lock()
  67. tx.UnsafeCreateBucket(keyBucketName)
  68. tx.UnsafeCreateBucket(metaBucketName)
  69. tx.Unlock()
  70. s.b.ForceCommit()
  71. if err := s.restore(); err != nil {
  72. // TODO: return the error instead of panic here?
  73. panic("failed to recover store from backend")
  74. }
  75. return s
  76. }
  77. func (s *store) Rev() int64 {
  78. s.mu.Lock()
  79. defer s.mu.Unlock()
  80. return s.currentRev.main
  81. }
  82. func (s *store) Put(key, value []byte, lease lease.LeaseID) int64 {
  83. id := s.TxnBegin()
  84. s.put(key, value, lease)
  85. s.txnEnd(id)
  86. putCounter.Inc()
  87. return int64(s.currentRev.main)
  88. }
  89. func (s *store) Range(key, end []byte, limit, rangeRev int64) (kvs []storagepb.KeyValue, rev int64, err error) {
  90. id := s.TxnBegin()
  91. kvs, rev, err = s.rangeKeys(key, end, limit, rangeRev)
  92. s.txnEnd(id)
  93. rangeCounter.Inc()
  94. return kvs, rev, err
  95. }
  96. func (s *store) DeleteRange(key, end []byte) (n, rev int64) {
  97. id := s.TxnBegin()
  98. n = s.deleteRange(key, end)
  99. s.txnEnd(id)
  100. deleteCounter.Inc()
  101. return n, int64(s.currentRev.main)
  102. }
  103. func (s *store) TxnBegin() int64 {
  104. s.mu.Lock()
  105. s.currentRev.sub = 0
  106. s.tx = s.b.BatchTx()
  107. s.tx.Lock()
  108. s.txnID = rand.Int63()
  109. return s.txnID
  110. }
  111. func (s *store) TxnEnd(txnID int64) error {
  112. err := s.txnEnd(txnID)
  113. if err != nil {
  114. return err
  115. }
  116. txnCounter.Inc()
  117. return nil
  118. }
  119. // txnEnd is used for unlocking an internal txn. It does
  120. // not increase the txnCounter.
  121. func (s *store) txnEnd(txnID int64) error {
  122. if txnID != s.txnID {
  123. return ErrTxnIDMismatch
  124. }
  125. s.tx.Unlock()
  126. if s.currentRev.sub != 0 {
  127. s.currentRev.main += 1
  128. }
  129. s.currentRev.sub = 0
  130. dbTotalSize.Set(float64(s.b.Size()))
  131. s.mu.Unlock()
  132. return nil
  133. }
  134. func (s *store) TxnRange(txnID int64, key, end []byte, limit, rangeRev int64) (kvs []storagepb.KeyValue, rev int64, err error) {
  135. if txnID != s.txnID {
  136. return nil, 0, ErrTxnIDMismatch
  137. }
  138. return s.rangeKeys(key, end, limit, rangeRev)
  139. }
  140. func (s *store) TxnPut(txnID int64, key, value []byte, lease lease.LeaseID) (rev int64, err error) {
  141. if txnID != s.txnID {
  142. return 0, ErrTxnIDMismatch
  143. }
  144. s.put(key, value, lease)
  145. return int64(s.currentRev.main + 1), nil
  146. }
  147. func (s *store) TxnDeleteRange(txnID int64, key, end []byte) (n, rev int64, err error) {
  148. if txnID != s.txnID {
  149. return 0, 0, ErrTxnIDMismatch
  150. }
  151. n = s.deleteRange(key, end)
  152. if n != 0 || s.currentRev.sub != 0 {
  153. rev = int64(s.currentRev.main + 1)
  154. } else {
  155. rev = int64(s.currentRev.main)
  156. }
  157. return n, rev, nil
  158. }
  159. func (s *store) Compact(rev int64) error {
  160. s.mu.Lock()
  161. defer s.mu.Unlock()
  162. if rev <= s.compactMainRev {
  163. return ErrCompacted
  164. }
  165. if rev > s.currentRev.main {
  166. return ErrFutureRev
  167. }
  168. start := time.Now()
  169. s.compactMainRev = rev
  170. rbytes := newRevBytes()
  171. revToBytes(revision{main: rev}, rbytes)
  172. tx := s.b.BatchTx()
  173. tx.Lock()
  174. tx.UnsafePut(metaBucketName, scheduledCompactKeyName, rbytes)
  175. tx.Unlock()
  176. // ensure that desired compaction is persisted
  177. s.b.ForceCommit()
  178. keep := s.kvindex.Compact(rev)
  179. s.wg.Add(1)
  180. go s.scheduleCompaction(rev, keep)
  181. indexCompactionPauseDurations.Observe(float64(time.Now().Sub(start) / time.Millisecond))
  182. return nil
  183. }
  184. func (s *store) Hash() (uint32, error) {
  185. s.b.ForceCommit()
  186. return s.b.Hash()
  187. }
  188. func (s *store) Snapshot() Snapshot {
  189. s.b.ForceCommit()
  190. return s.b.Snapshot()
  191. }
  192. func (s *store) Commit() { s.b.ForceCommit() }
  193. func (s *store) Restore(b backend.Backend) error {
  194. s.mu.Lock()
  195. defer s.mu.Unlock()
  196. close(s.stopc)
  197. // TODO: restore without waiting for compaction routine to finish.
  198. // We need a way to notify that the store is finished using the old
  199. // backend though.
  200. s.wg.Wait()
  201. s.b = b
  202. s.kvindex = newTreeIndex()
  203. s.currentRev = revision{}
  204. s.compactMainRev = -1
  205. s.tx = b.BatchTx()
  206. s.txnID = -1
  207. s.stopc = make(chan struct{})
  208. return s.restore()
  209. }
  210. func (s *store) restore() error {
  211. min, max := newRevBytes(), newRevBytes()
  212. revToBytes(revision{}, min)
  213. revToBytes(revision{main: math.MaxInt64, sub: math.MaxInt64}, max)
  214. // restore index
  215. tx := s.b.BatchTx()
  216. tx.Lock()
  217. _, finishedCompactBytes := tx.UnsafeRange(metaBucketName, finishedCompactKeyName, nil, 0)
  218. if len(finishedCompactBytes) != 0 {
  219. s.compactMainRev = bytesToRev(finishedCompactBytes[0]).main
  220. log.Printf("storage: restore compact to %d", s.compactMainRev)
  221. }
  222. // TODO: limit N to reduce max memory usage
  223. keys, vals := tx.UnsafeRange(keyBucketName, min, max, 0)
  224. for i, key := range keys {
  225. var kv storagepb.KeyValue
  226. if err := kv.Unmarshal(vals[i]); err != nil {
  227. log.Fatalf("storage: cannot unmarshal event: %v", err)
  228. }
  229. rev := bytesToRev(key[:revBytesLen])
  230. // restore index
  231. switch {
  232. case isTombstone(key):
  233. s.kvindex.Tombstone(kv.Key, rev)
  234. default:
  235. s.kvindex.Restore(kv.Key, revision{kv.CreateRevision, 0}, rev, kv.Version)
  236. }
  237. // update revision
  238. s.currentRev = rev
  239. }
  240. _, scheduledCompactBytes := tx.UnsafeRange(metaBucketName, scheduledCompactKeyName, nil, 0)
  241. if len(scheduledCompactBytes) != 0 {
  242. scheduledCompact := bytesToRev(scheduledCompactBytes[0]).main
  243. if scheduledCompact > s.compactMainRev {
  244. log.Printf("storage: resume scheduled compaction at %d", scheduledCompact)
  245. go s.Compact(scheduledCompact)
  246. }
  247. }
  248. tx.Unlock()
  249. return nil
  250. }
  251. func (s *store) Close() error {
  252. close(s.stopc)
  253. s.wg.Wait()
  254. return nil
  255. }
  256. func (a *store) Equal(b *store) bool {
  257. if a.currentRev != b.currentRev {
  258. return false
  259. }
  260. if a.compactMainRev != b.compactMainRev {
  261. return false
  262. }
  263. return a.kvindex.Equal(b.kvindex)
  264. }
  265. // range is a keyword in Go, add Keys suffix.
  266. func (s *store) rangeKeys(key, end []byte, limit, rangeRev int64) (kvs []storagepb.KeyValue, rev int64, err error) {
  267. curRev := int64(s.currentRev.main)
  268. if s.currentRev.sub > 0 {
  269. curRev += 1
  270. }
  271. if rangeRev > curRev {
  272. return nil, s.currentRev.main, ErrFutureRev
  273. }
  274. if rangeRev <= 0 {
  275. rev = curRev
  276. } else {
  277. rev = rangeRev
  278. }
  279. if rev <= s.compactMainRev {
  280. return nil, 0, ErrCompacted
  281. }
  282. _, revpairs := s.kvindex.Range(key, end, int64(rev))
  283. if len(revpairs) == 0 {
  284. return nil, rev, nil
  285. }
  286. for _, revpair := range revpairs {
  287. start, end := revBytesRange(revpair)
  288. _, vs := s.tx.UnsafeRange(keyBucketName, start, end, 0)
  289. if len(vs) != 1 {
  290. log.Fatalf("storage: range cannot find rev (%d,%d)", revpair.main, revpair.sub)
  291. }
  292. var kv storagepb.KeyValue
  293. if err := kv.Unmarshal(vs[0]); err != nil {
  294. log.Fatalf("storage: cannot unmarshal event: %v", err)
  295. }
  296. kvs = append(kvs, kv)
  297. if limit > 0 && len(kvs) >= int(limit) {
  298. break
  299. }
  300. }
  301. return kvs, rev, nil
  302. }
  303. func (s *store) put(key, value []byte, lease lease.LeaseID) {
  304. rev := s.currentRev.main + 1
  305. c := rev
  306. // if the key exists before, use its previous created
  307. _, created, ver, err := s.kvindex.Get(key, rev)
  308. if err == nil {
  309. c = created.main
  310. }
  311. ibytes := newRevBytes()
  312. revToBytes(revision{main: rev, sub: s.currentRev.sub}, ibytes)
  313. ver = ver + 1
  314. kv := storagepb.KeyValue{
  315. Key: key,
  316. Value: value,
  317. CreateRevision: c,
  318. ModRevision: rev,
  319. Version: ver,
  320. Lease: int64(lease),
  321. }
  322. d, err := kv.Marshal()
  323. if err != nil {
  324. log.Fatalf("storage: cannot marshal event: %v", err)
  325. }
  326. s.tx.UnsafePut(keyBucketName, ibytes, d)
  327. s.kvindex.Put(key, revision{main: rev, sub: s.currentRev.sub})
  328. s.currentRev.sub += 1
  329. }
  330. func (s *store) deleteRange(key, end []byte) int64 {
  331. rrev := s.currentRev.main
  332. if s.currentRev.sub > 0 {
  333. rrev += 1
  334. }
  335. keys, _ := s.kvindex.Range(key, end, rrev)
  336. if len(keys) == 0 {
  337. return 0
  338. }
  339. for _, key := range keys {
  340. s.delete(key)
  341. }
  342. return int64(len(keys))
  343. }
  344. func (s *store) delete(key []byte) {
  345. mainrev := s.currentRev.main + 1
  346. ibytes := newRevBytes()
  347. revToBytes(revision{main: mainrev, sub: s.currentRev.sub}, ibytes)
  348. ibytes = appendMarkTombstone(ibytes)
  349. kv := storagepb.KeyValue{
  350. Key: key,
  351. }
  352. d, err := kv.Marshal()
  353. if err != nil {
  354. log.Fatalf("storage: cannot marshal event: %v", err)
  355. }
  356. s.tx.UnsafePut(keyBucketName, ibytes, d)
  357. err = s.kvindex.Tombstone(key, revision{main: mainrev, sub: s.currentRev.sub})
  358. if err != nil {
  359. log.Fatalf("storage: cannot tombstone an existing key (%s): %v", string(key), err)
  360. }
  361. s.currentRev.sub += 1
  362. }
  363. // appendMarkTombstone appends tombstone mark to normal revision bytes.
  364. func appendMarkTombstone(b []byte) []byte {
  365. if len(b) != revBytesLen {
  366. log.Panicf("cannot append mark to non normal revision bytes")
  367. }
  368. return append(b, markTombstone)
  369. }
  370. // isTombstone checks whether the revision bytes is a tombstone.
  371. func isTombstone(b []byte) bool {
  372. return len(b) == markedRevBytesLen && b[markBytePosition] == markTombstone
  373. }
  374. // revBytesRange returns the range of revision bytes at
  375. // the given revision.
  376. func revBytesRange(rev revision) (start, end []byte) {
  377. start = newRevBytes()
  378. revToBytes(rev, start)
  379. end = newRevBytes()
  380. endRev := revision{main: rev.main, sub: rev.sub + 1}
  381. revToBytes(endRev, end)
  382. return start, end
  383. }