kvstore.go 12 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517
  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/storage/backend"
  23. "github.com/coreos/etcd/storage/storagepb"
  24. )
  25. var (
  26. batchLimit = 10000
  27. batchInterval = 100 * time.Millisecond
  28. keyBucketName = []byte("key")
  29. metaBucketName = []byte("meta")
  30. // markedRevBytesLen is the byte length of marked revision.
  31. // The first `revBytesLen` bytes represents a normal revision. The last
  32. // one byte is the mark.
  33. markedRevBytesLen = revBytesLen + 1
  34. markBytePosition = markedRevBytesLen - 1
  35. markTombstone byte = 't'
  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.RWMutex
  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. tmu sync.Mutex // protect the txnID field
  52. txnID int64 // tracks the current txnID to verify txn operations
  53. wg sync.WaitGroup
  54. stopc chan struct{}
  55. }
  56. func newStore(path string) *store {
  57. s := &store{
  58. b: backend.New(path, batchInterval, batchLimit),
  59. kvindex: newTreeIndex(),
  60. currentRev: revision{},
  61. compactMainRev: -1,
  62. stopc: make(chan struct{}),
  63. }
  64. tx := s.b.BatchTx()
  65. tx.Lock()
  66. tx.UnsafeCreateBucket(keyBucketName)
  67. tx.UnsafeCreateBucket(metaBucketName)
  68. tx.Unlock()
  69. s.b.ForceCommit()
  70. return s
  71. }
  72. func (s *store) Rev() int64 {
  73. s.mu.RLock()
  74. defer s.mu.RUnlock()
  75. return s.currentRev.main
  76. }
  77. func (s *store) Put(key, value []byte) int64 {
  78. id := s.TxnBegin()
  79. s.put(key, value)
  80. s.txnEnd(id)
  81. putCounter.Inc()
  82. return int64(s.currentRev.main)
  83. }
  84. func (s *store) Range(key, end []byte, limit, rangeRev int64) (kvs []storagepb.KeyValue, rev int64, err error) {
  85. id := s.TxnBegin()
  86. kvs, rev, err = s.rangeKeys(key, end, limit, rangeRev)
  87. s.txnEnd(id)
  88. rangeCounter.Inc()
  89. return kvs, rev, err
  90. }
  91. func (s *store) DeleteRange(key, end []byte) (n, rev int64) {
  92. id := s.TxnBegin()
  93. n = s.deleteRange(key, end)
  94. s.txnEnd(id)
  95. deleteCounter.Inc()
  96. return n, int64(s.currentRev.main)
  97. }
  98. func (s *store) TxnBegin() int64 {
  99. s.mu.Lock()
  100. s.currentRev.sub = 0
  101. s.tx = s.b.BatchTx()
  102. s.tx.Lock()
  103. s.tmu.Lock()
  104. defer s.tmu.Unlock()
  105. s.txnID = rand.Int63()
  106. return s.txnID
  107. }
  108. func (s *store) TxnEnd(txnID int64) error {
  109. err := s.txnEnd(txnID)
  110. if err != nil {
  111. return err
  112. }
  113. txnCounter.Inc()
  114. return nil
  115. }
  116. // txnEnd is used for unlocking an internal txn. It does
  117. // not increase the txnCounter.
  118. func (s *store) txnEnd(txnID int64) error {
  119. s.tmu.Lock()
  120. defer s.tmu.Unlock()
  121. if txnID != s.txnID {
  122. return ErrTxnIDMismatch
  123. }
  124. s.tx.Unlock()
  125. if s.currentRev.sub != 0 {
  126. s.currentRev.main += 1
  127. }
  128. s.currentRev.sub = 0
  129. dbTotalSize.Set(float64(s.b.Size()))
  130. s.mu.Unlock()
  131. return nil
  132. }
  133. func (s *store) TxnRange(txnID int64, key, end []byte, limit, rangeRev int64) (kvs []storagepb.KeyValue, rev int64, err error) {
  134. s.tmu.Lock()
  135. defer s.tmu.Unlock()
  136. if txnID != s.txnID {
  137. return nil, 0, ErrTxnIDMismatch
  138. }
  139. return s.rangeKeys(key, end, limit, rangeRev)
  140. }
  141. func (s *store) TxnPut(txnID int64, key, value []byte) (rev int64, err error) {
  142. s.tmu.Lock()
  143. defer s.tmu.Unlock()
  144. if txnID != s.txnID {
  145. return 0, ErrTxnIDMismatch
  146. }
  147. s.put(key, value)
  148. return int64(s.currentRev.main + 1), nil
  149. }
  150. func (s *store) TxnDeleteRange(txnID int64, key, end []byte) (n, rev int64, err error) {
  151. s.tmu.Lock()
  152. defer s.tmu.Unlock()
  153. if txnID != s.txnID {
  154. return 0, 0, ErrTxnIDMismatch
  155. }
  156. n = s.deleteRange(key, end)
  157. if n != 0 || s.currentRev.sub != 0 {
  158. rev = int64(s.currentRev.main + 1)
  159. } else {
  160. rev = int64(s.currentRev.main)
  161. }
  162. return n, rev, nil
  163. }
  164. // RangeHistory ranges the history from key to end starting from startRev.
  165. // If `end` is nil, the request only observes the events on key.
  166. // If `end` is not nil, it observes the events on key range [key, range_end).
  167. // Limit limits the number of events returned.
  168. // If startRev <=0, rangeEvents returns events from the beginning of uncompacted history.
  169. //
  170. // If the required start rev is compacted, ErrCompacted will be returned.
  171. // If the required start rev has not happened, ErrFutureRev will be returned.
  172. //
  173. // RangeHistory returns revision bytes slice and key-values that satisfy the requirement (0 <= n <= limit).
  174. // If history in the revision range has not all happened, it returns immeidately
  175. // what is available.
  176. // It also returns nextRev which indicates the start revision used for the following
  177. // RangeEvents call. The nextRev could be smaller than the given endRev if the store
  178. // has not progressed so far or it hits the event limit.
  179. //
  180. // TODO: return byte slices instead of keyValues to avoid meaningless encode and decode.
  181. // This also helps to return raw (key, val) pair directly to make API consistent.
  182. func (s *store) RangeHistory(key, end []byte, limit, startRev int64) (revbs [][]byte, kvs []storagepb.KeyValue, nextRev int64, err error) {
  183. s.mu.Lock()
  184. defer s.mu.Unlock()
  185. if startRev > 0 && startRev <= s.compactMainRev {
  186. return nil, nil, 0, ErrCompacted
  187. }
  188. if startRev > s.currentRev.main {
  189. return nil, nil, 0, ErrFutureRev
  190. }
  191. revs := s.kvindex.RangeSince(key, end, startRev)
  192. if len(revs) == 0 {
  193. return nil, nil, s.currentRev.main + 1, nil
  194. }
  195. tx := s.b.BatchTx()
  196. tx.Lock()
  197. defer tx.Unlock()
  198. // fetch events from the backend using revisions
  199. for _, rev := range revs {
  200. start, end := revBytesRange(rev)
  201. ks, vs := tx.UnsafeRange(keyBucketName, start, end, 0)
  202. if len(vs) != 1 {
  203. log.Fatalf("storage: range cannot find rev (%d,%d)", rev.main, rev.sub)
  204. }
  205. var kv storagepb.KeyValue
  206. if err := kv.Unmarshal(vs[0]); err != nil {
  207. log.Fatalf("storage: cannot unmarshal event: %v", err)
  208. }
  209. revbs = append(revbs, ks[0])
  210. kvs = append(kvs, kv)
  211. if limit > 0 && len(kvs) >= int(limit) {
  212. return revbs, kvs, rev.main + 1, nil
  213. }
  214. }
  215. return revbs, kvs, s.currentRev.main + 1, nil
  216. }
  217. func (s *store) Compact(rev int64) error {
  218. s.mu.Lock()
  219. defer s.mu.Unlock()
  220. if rev <= s.compactMainRev {
  221. return ErrCompacted
  222. }
  223. if rev > s.currentRev.main {
  224. return ErrFutureRev
  225. }
  226. start := time.Now()
  227. s.compactMainRev = rev
  228. rbytes := newRevBytes()
  229. revToBytes(revision{main: rev}, rbytes)
  230. tx := s.b.BatchTx()
  231. tx.Lock()
  232. tx.UnsafePut(metaBucketName, scheduledCompactKeyName, rbytes)
  233. tx.Unlock()
  234. // ensure that desired compaction is persisted
  235. s.b.ForceCommit()
  236. keep := s.kvindex.Compact(rev)
  237. s.wg.Add(1)
  238. go s.scheduleCompaction(rev, keep)
  239. indexCompactionPauseDurations.Observe(float64(time.Now().Sub(start) / time.Millisecond))
  240. return nil
  241. }
  242. func (s *store) Hash() (uint32, error) {
  243. s.b.ForceCommit()
  244. return s.b.Hash()
  245. }
  246. func (s *store) Snapshot() Snapshot {
  247. s.b.ForceCommit()
  248. return s.b.Snapshot()
  249. }
  250. func (s *store) Commit() { s.b.ForceCommit() }
  251. func (s *store) Restore() error {
  252. s.mu.Lock()
  253. defer s.mu.Unlock()
  254. min, max := newRevBytes(), newRevBytes()
  255. revToBytes(revision{}, min)
  256. revToBytes(revision{main: math.MaxInt64, sub: math.MaxInt64}, max)
  257. // restore index
  258. tx := s.b.BatchTx()
  259. tx.Lock()
  260. _, finishedCompactBytes := tx.UnsafeRange(metaBucketName, finishedCompactKeyName, nil, 0)
  261. if len(finishedCompactBytes) != 0 {
  262. s.compactMainRev = bytesToRev(finishedCompactBytes[0]).main
  263. log.Printf("storage: restore compact to %d", s.compactMainRev)
  264. }
  265. // TODO: limit N to reduce max memory usage
  266. keys, vals := tx.UnsafeRange(keyBucketName, min, max, 0)
  267. for i, key := range keys {
  268. var kv storagepb.KeyValue
  269. if err := kv.Unmarshal(vals[i]); err != nil {
  270. log.Fatalf("storage: cannot unmarshal event: %v", err)
  271. }
  272. rev := bytesToRev(key[:revBytesLen])
  273. // restore index
  274. switch {
  275. case isTombstone(key):
  276. s.kvindex.Tombstone(kv.Key, rev)
  277. default:
  278. s.kvindex.Restore(kv.Key, revision{kv.CreateRevision, 0}, rev, kv.Version)
  279. }
  280. // update revision
  281. s.currentRev = rev
  282. }
  283. _, scheduledCompactBytes := tx.UnsafeRange(metaBucketName, scheduledCompactKeyName, nil, 0)
  284. if len(scheduledCompactBytes) != 0 {
  285. scheduledCompact := bytesToRev(scheduledCompactBytes[0]).main
  286. if scheduledCompact > s.compactMainRev {
  287. log.Printf("storage: resume scheduled compaction at %d", scheduledCompact)
  288. go s.Compact(scheduledCompact)
  289. }
  290. }
  291. tx.Unlock()
  292. return nil
  293. }
  294. func (s *store) Close() error {
  295. close(s.stopc)
  296. s.wg.Wait()
  297. return s.b.Close()
  298. }
  299. func (a *store) Equal(b *store) bool {
  300. if a.currentRev != b.currentRev {
  301. return false
  302. }
  303. if a.compactMainRev != b.compactMainRev {
  304. return false
  305. }
  306. return a.kvindex.Equal(b.kvindex)
  307. }
  308. // range is a keyword in Go, add Keys suffix.
  309. func (s *store) rangeKeys(key, end []byte, limit, rangeRev int64) (kvs []storagepb.KeyValue, rev int64, err error) {
  310. curRev := int64(s.currentRev.main)
  311. if s.currentRev.sub > 0 {
  312. curRev += 1
  313. }
  314. if rangeRev > curRev {
  315. return nil, s.currentRev.main, ErrFutureRev
  316. }
  317. if rangeRev <= 0 {
  318. rev = curRev
  319. } else {
  320. rev = rangeRev
  321. }
  322. if rev <= s.compactMainRev {
  323. return nil, 0, ErrCompacted
  324. }
  325. _, revpairs := s.kvindex.Range(key, end, int64(rev))
  326. if len(revpairs) == 0 {
  327. return nil, rev, nil
  328. }
  329. for _, revpair := range revpairs {
  330. start, end := revBytesRange(revpair)
  331. _, vs := s.tx.UnsafeRange(keyBucketName, start, end, 0)
  332. if len(vs) != 1 {
  333. log.Fatalf("storage: range cannot find rev (%d,%d)", revpair.main, revpair.sub)
  334. }
  335. var kv storagepb.KeyValue
  336. if err := kv.Unmarshal(vs[0]); err != nil {
  337. log.Fatalf("storage: cannot unmarshal event: %v", err)
  338. }
  339. kvs = append(kvs, kv)
  340. if limit > 0 && len(kvs) >= int(limit) {
  341. break
  342. }
  343. }
  344. return kvs, rev, nil
  345. }
  346. func (s *store) put(key, value []byte) {
  347. rev := s.currentRev.main + 1
  348. c := rev
  349. // if the key exists before, use its previous created
  350. _, created, ver, err := s.kvindex.Get(key, rev)
  351. if err == nil {
  352. c = created.main
  353. }
  354. ibytes := newRevBytes()
  355. revToBytes(revision{main: rev, sub: s.currentRev.sub}, ibytes)
  356. ver = ver + 1
  357. kv := storagepb.KeyValue{
  358. Key: key,
  359. Value: value,
  360. CreateRevision: c,
  361. ModRevision: rev,
  362. Version: ver,
  363. }
  364. d, err := kv.Marshal()
  365. if err != nil {
  366. log.Fatalf("storage: cannot marshal event: %v", err)
  367. }
  368. s.tx.UnsafePut(keyBucketName, ibytes, d)
  369. s.kvindex.Put(key, revision{main: rev, sub: s.currentRev.sub})
  370. s.currentRev.sub += 1
  371. }
  372. func (s *store) deleteRange(key, end []byte) int64 {
  373. rrev := s.currentRev.main
  374. if s.currentRev.sub > 0 {
  375. rrev += 1
  376. }
  377. keys, _ := s.kvindex.Range(key, end, rrev)
  378. if len(keys) == 0 {
  379. return 0
  380. }
  381. for _, key := range keys {
  382. s.delete(key)
  383. }
  384. return int64(len(keys))
  385. }
  386. func (s *store) delete(key []byte) {
  387. mainrev := s.currentRev.main + 1
  388. ibytes := newRevBytes()
  389. revToBytes(revision{main: mainrev, sub: s.currentRev.sub}, ibytes)
  390. ibytes = appendMarkTombstone(ibytes)
  391. kv := storagepb.KeyValue{
  392. Key: key,
  393. }
  394. d, err := kv.Marshal()
  395. if err != nil {
  396. log.Fatalf("storage: cannot marshal event: %v", err)
  397. }
  398. s.tx.UnsafePut(keyBucketName, ibytes, d)
  399. err = s.kvindex.Tombstone(key, revision{main: mainrev, sub: s.currentRev.sub})
  400. if err != nil {
  401. log.Fatalf("storage: cannot tombstone an existing key (%s): %v", string(key), err)
  402. }
  403. s.currentRev.sub += 1
  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. }