kvstore.go 12 KB

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