You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.

486 lines
12 KiB

  1. package kv
  2. import (
  3. "context"
  4. "errors"
  5. "fmt"
  6. "sort"
  7. "strconv"
  8. "strings"
  9. "github.com/google/orderedcode"
  10. dbm "github.com/tendermint/tm-db"
  11. abci "github.com/tendermint/tendermint/abci/types"
  12. "github.com/tendermint/tendermint/internal/pubsub/query"
  13. "github.com/tendermint/tendermint/internal/pubsub/query/syntax"
  14. "github.com/tendermint/tendermint/internal/state/indexer"
  15. "github.com/tendermint/tendermint/types"
  16. )
  17. var _ indexer.BlockIndexer = (*BlockerIndexer)(nil)
  18. // BlockerIndexer implements a block indexer, indexing BeginBlock and EndBlock
  19. // events with an underlying KV store. Block events are indexed by their height,
  20. // such that matching search criteria returns the respective block height(s).
  21. type BlockerIndexer struct {
  22. store dbm.DB
  23. }
  24. func New(store dbm.DB) *BlockerIndexer {
  25. return &BlockerIndexer{
  26. store: store,
  27. }
  28. }
  29. // Has returns true if the given height has been indexed. An error is returned
  30. // upon database query failure.
  31. func (idx *BlockerIndexer) Has(height int64) (bool, error) {
  32. key, err := heightKey(height)
  33. if err != nil {
  34. return false, fmt.Errorf("failed to create block height index key: %w", err)
  35. }
  36. return idx.store.Has(key)
  37. }
  38. // Index indexes BeginBlock and EndBlock events for a given block by its height.
  39. // The following is indexed:
  40. //
  41. // primary key: encode(block.height | height) => encode(height)
  42. // BeginBlock events: encode(eventType.eventAttr|eventValue|height|begin_block) => encode(height)
  43. // EndBlock events: encode(eventType.eventAttr|eventValue|height|end_block) => encode(height)
  44. func (idx *BlockerIndexer) Index(bh types.EventDataNewBlockHeader) error {
  45. batch := idx.store.NewBatch()
  46. defer batch.Close()
  47. height := bh.Header.Height
  48. // 1. index by height
  49. key, err := heightKey(height)
  50. if err != nil {
  51. return fmt.Errorf("failed to create block height index key: %w", err)
  52. }
  53. if err := batch.Set(key, int64ToBytes(height)); err != nil {
  54. return err
  55. }
  56. // 2. index BeginBlock events
  57. if err := idx.indexEvents(batch, bh.ResultFinalizeBlock.Events, "finalize_block", height); err != nil {
  58. return fmt.Errorf("failed to index FinalizeBlock events: %w", err)
  59. }
  60. return batch.WriteSync()
  61. }
  62. // Search performs a query for block heights that match a given BeginBlock
  63. // and Endblock event search criteria. The given query can match against zero,
  64. // one or more block heights. In the case of height queries, i.e. block.height=H,
  65. // if the height is indexed, that height alone will be returned. An error and
  66. // nil slice is returned. Otherwise, a non-nil slice and nil error is returned.
  67. func (idx *BlockerIndexer) Search(ctx context.Context, q *query.Query) ([]int64, error) {
  68. results := make([]int64, 0)
  69. select {
  70. case <-ctx.Done():
  71. return results, nil
  72. default:
  73. }
  74. conditions := q.Syntax()
  75. // If there is an exact height query, return the result immediately
  76. // (if it exists).
  77. height, ok := lookForHeight(conditions)
  78. if ok {
  79. ok, err := idx.Has(height)
  80. if err != nil {
  81. return nil, err
  82. }
  83. if ok {
  84. return []int64{height}, nil
  85. }
  86. return results, nil
  87. }
  88. var heightsInitialized bool
  89. filteredHeights := make(map[string][]byte)
  90. // conditions to skip because they're handled before "everything else"
  91. skipIndexes := make([]int, 0)
  92. // Extract ranges. If both upper and lower bounds exist, it's better to get
  93. // them in order as to not iterate over kvs that are not within range.
  94. ranges, rangeIndexes := indexer.LookForRanges(conditions)
  95. if len(ranges) > 0 {
  96. skipIndexes = append(skipIndexes, rangeIndexes...)
  97. for _, qr := range ranges {
  98. prefix, err := orderedcode.Append(nil, qr.Key)
  99. if err != nil {
  100. return nil, fmt.Errorf("failed to create prefix key: %w", err)
  101. }
  102. if !heightsInitialized {
  103. filteredHeights, err = idx.matchRange(ctx, qr, prefix, filteredHeights, true)
  104. if err != nil {
  105. return nil, err
  106. }
  107. heightsInitialized = true
  108. // Ignore any remaining conditions if the first condition resulted in no
  109. // matches (assuming implicit AND operand).
  110. if len(filteredHeights) == 0 {
  111. break
  112. }
  113. } else {
  114. filteredHeights, err = idx.matchRange(ctx, qr, prefix, filteredHeights, false)
  115. if err != nil {
  116. return nil, err
  117. }
  118. }
  119. }
  120. }
  121. // for all other conditions
  122. for i, c := range conditions {
  123. if intInSlice(i, skipIndexes) {
  124. continue
  125. }
  126. startKey, err := orderedcode.Append(nil, c.Tag, c.Arg.Value())
  127. if err != nil {
  128. return nil, err
  129. }
  130. if !heightsInitialized {
  131. filteredHeights, err = idx.match(ctx, c, startKey, filteredHeights, true)
  132. if err != nil {
  133. return nil, err
  134. }
  135. heightsInitialized = true
  136. // Ignore any remaining conditions if the first condition resulted in no
  137. // matches (assuming implicit AND operand).
  138. if len(filteredHeights) == 0 {
  139. break
  140. }
  141. } else {
  142. filteredHeights, err = idx.match(ctx, c, startKey, filteredHeights, false)
  143. if err != nil {
  144. return nil, err
  145. }
  146. }
  147. }
  148. // fetch matching heights
  149. results = make([]int64, 0, len(filteredHeights))
  150. heights:
  151. for _, hBz := range filteredHeights {
  152. h := int64FromBytes(hBz)
  153. ok, err := idx.Has(h)
  154. if err != nil {
  155. return nil, err
  156. }
  157. if ok {
  158. results = append(results, h)
  159. }
  160. select {
  161. case <-ctx.Done():
  162. break heights
  163. default:
  164. }
  165. }
  166. sort.Slice(results, func(i, j int) bool { return results[i] < results[j] })
  167. return results, nil
  168. }
  169. // matchRange returns all matching block heights that match a given QueryRange
  170. // and start key. An already filtered result (filteredHeights) is provided such
  171. // that any non-intersecting matches are removed.
  172. //
  173. // NOTE: The provided filteredHeights may be empty if no previous condition has
  174. // matched.
  175. func (idx *BlockerIndexer) matchRange(
  176. ctx context.Context,
  177. qr indexer.QueryRange,
  178. startKey []byte,
  179. filteredHeights map[string][]byte,
  180. firstRun bool,
  181. ) (map[string][]byte, error) {
  182. // A previous match was attempted but resulted in no matches, so we return
  183. // no matches (assuming AND operand).
  184. if !firstRun && len(filteredHeights) == 0 {
  185. return filteredHeights, nil
  186. }
  187. tmpHeights := make(map[string][]byte)
  188. lowerBound := qr.LowerBoundValue()
  189. upperBound := qr.UpperBoundValue()
  190. it, err := dbm.IteratePrefix(idx.store, startKey)
  191. if err != nil {
  192. return nil, fmt.Errorf("failed to create prefix iterator: %w", err)
  193. }
  194. defer it.Close()
  195. iter:
  196. for ; it.Valid(); it.Next() {
  197. var (
  198. eventValue string
  199. err error
  200. )
  201. if qr.Key == types.BlockHeightKey {
  202. eventValue, err = parseValueFromPrimaryKey(it.Key())
  203. } else {
  204. eventValue, err = parseValueFromEventKey(it.Key())
  205. }
  206. if err != nil {
  207. continue
  208. }
  209. if _, ok := qr.AnyBound().(int64); ok {
  210. v, err := strconv.ParseInt(eventValue, 10, 64)
  211. if err != nil {
  212. continue iter
  213. }
  214. include := true
  215. if lowerBound != nil && v < lowerBound.(int64) {
  216. include = false
  217. }
  218. if upperBound != nil && v > upperBound.(int64) {
  219. include = false
  220. }
  221. if include {
  222. tmpHeights[string(it.Value())] = it.Value()
  223. }
  224. }
  225. select {
  226. case <-ctx.Done():
  227. break iter
  228. default:
  229. }
  230. }
  231. if err := it.Error(); err != nil {
  232. return nil, err
  233. }
  234. if len(tmpHeights) == 0 || firstRun {
  235. // Either:
  236. //
  237. // 1. Regardless if a previous match was attempted, which may have had
  238. // results, but no match was found for the current condition, then we
  239. // return no matches (assuming AND operand).
  240. //
  241. // 2. A previous match was not attempted, so we return all results.
  242. return tmpHeights, nil
  243. }
  244. // Remove/reduce matches in filteredHashes that were not found in this
  245. // match (tmpHashes).
  246. for k := range filteredHeights {
  247. if tmpHeights[k] == nil {
  248. delete(filteredHeights, k)
  249. select {
  250. case <-ctx.Done():
  251. break
  252. default:
  253. }
  254. }
  255. }
  256. return filteredHeights, nil
  257. }
  258. // match returns all matching heights that meet a given query condition and start
  259. // key. An already filtered result (filteredHeights) is provided such that any
  260. // non-intersecting matches are removed.
  261. //
  262. // NOTE: The provided filteredHeights may be empty if no previous condition has
  263. // matched.
  264. func (idx *BlockerIndexer) match(
  265. ctx context.Context,
  266. c syntax.Condition,
  267. startKeyBz []byte,
  268. filteredHeights map[string][]byte,
  269. firstRun bool,
  270. ) (map[string][]byte, error) {
  271. // A previous match was attempted but resulted in no matches, so we return
  272. // no matches (assuming AND operand).
  273. if !firstRun && len(filteredHeights) == 0 {
  274. return filteredHeights, nil
  275. }
  276. tmpHeights := make(map[string][]byte)
  277. switch {
  278. case c.Op == syntax.TEq:
  279. it, err := dbm.IteratePrefix(idx.store, startKeyBz)
  280. if err != nil {
  281. return nil, fmt.Errorf("failed to create prefix iterator: %w", err)
  282. }
  283. defer it.Close()
  284. for ; it.Valid(); it.Next() {
  285. tmpHeights[string(it.Value())] = it.Value()
  286. if err := ctx.Err(); err != nil {
  287. break
  288. }
  289. }
  290. if err := it.Error(); err != nil {
  291. return nil, err
  292. }
  293. case c.Op == syntax.TExists:
  294. prefix, err := orderedcode.Append(nil, c.Tag)
  295. if err != nil {
  296. return nil, err
  297. }
  298. it, err := dbm.IteratePrefix(idx.store, prefix)
  299. if err != nil {
  300. return nil, fmt.Errorf("failed to create prefix iterator: %w", err)
  301. }
  302. defer it.Close()
  303. iterExists:
  304. for ; it.Valid(); it.Next() {
  305. tmpHeights[string(it.Value())] = it.Value()
  306. select {
  307. case <-ctx.Done():
  308. break iterExists
  309. default:
  310. }
  311. }
  312. if err := it.Error(); err != nil {
  313. return nil, err
  314. }
  315. case c.Op == syntax.TContains:
  316. prefix, err := orderedcode.Append(nil, c.Tag)
  317. if err != nil {
  318. return nil, err
  319. }
  320. it, err := dbm.IteratePrefix(idx.store, prefix)
  321. if err != nil {
  322. return nil, fmt.Errorf("failed to create prefix iterator: %w", err)
  323. }
  324. defer it.Close()
  325. iterContains:
  326. for ; it.Valid(); it.Next() {
  327. eventValue, err := parseValueFromEventKey(it.Key())
  328. if err != nil {
  329. continue
  330. }
  331. if strings.Contains(eventValue, c.Arg.Value()) {
  332. tmpHeights[string(it.Value())] = it.Value()
  333. }
  334. select {
  335. case <-ctx.Done():
  336. break iterContains
  337. default:
  338. }
  339. }
  340. if err := it.Error(); err != nil {
  341. return nil, err
  342. }
  343. default:
  344. return nil, errors.New("other operators should be handled already")
  345. }
  346. if len(tmpHeights) == 0 || firstRun {
  347. // Either:
  348. //
  349. // 1. Regardless if a previous match was attempted, which may have had
  350. // results, but no match was found for the current condition, then we
  351. // return no matches (assuming AND operand).
  352. //
  353. // 2. A previous match was not attempted, so we return all results.
  354. return tmpHeights, nil
  355. }
  356. // Remove/reduce matches in filteredHeights that were not found in this
  357. // match (tmpHeights).
  358. for k := range filteredHeights {
  359. if tmpHeights[k] == nil {
  360. delete(filteredHeights, k)
  361. select {
  362. case <-ctx.Done():
  363. break
  364. default:
  365. }
  366. }
  367. }
  368. return filteredHeights, nil
  369. }
  370. func (idx *BlockerIndexer) indexEvents(batch dbm.Batch, events []abci.Event, typ string, height int64) error {
  371. heightBz := int64ToBytes(height)
  372. for _, event := range events {
  373. // only index events with a non-empty type
  374. if len(event.Type) == 0 {
  375. continue
  376. }
  377. for _, attr := range event.Attributes {
  378. if len(attr.Key) == 0 {
  379. continue
  380. }
  381. // index iff the event specified index:true and it's not a reserved event
  382. compositeKey := fmt.Sprintf("%s.%s", event.Type, attr.Key)
  383. if compositeKey == types.BlockHeightKey {
  384. return fmt.Errorf("event type and attribute key \"%s\" is reserved; please use a different key", compositeKey)
  385. }
  386. if attr.GetIndex() {
  387. key, err := eventKey(compositeKey, typ, attr.Value, height)
  388. if err != nil {
  389. return fmt.Errorf("failed to create block index key: %w", err)
  390. }
  391. if err := batch.Set(key, heightBz); err != nil {
  392. return err
  393. }
  394. }
  395. }
  396. }
  397. return nil
  398. }