kvstore.go 12 KB

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