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.

485 lines
11 KiB

abci: Synchronize FinalizeBlock with the updated specification (#7983) This change set implements the most recent version of `FinalizeBlock`. # What does this change actually contain? * This change set is rather large but fear not! The majority of the files touched and changes are renaming `ResponseDeliverTx` to `ExecTxResult`. This should be a pretty inoffensive change since they're effectively the same type but with a different name. * The `execBlockOnProxyApp` was totally removed since it served as just a wrapper around the logic that is now mostly encapsulated within `FinalizeBlock` * The `updateState` helper function has been made a public method on `State`. It was being exposed as a shim through the testing infrastructure, so this seemed innocuous. * Tests already existed to ensure that the application received the `ByzantineValidators` and the `ValidatorUpdates`, but one was fixed up to ensure that `LastCommitInfo` was being sent across. * Tests were removed from the `psql` indexer that seemed to search for an event in the indexer that was not being created. # Questions for reviewers * We store this [ABCIResponses](https://github.com/tendermint/tendermint/blob/5721a13ab1f4479f9807f449f0bf5c536b9a05f2/proto/tendermint/state/types.pb.go#L37) type in the data base as the block results. This type has changed since v0.35 to contain the `FinalizeBlock` response. I'm wondering if we need to do any shimming to keep the old data retrieveable? * Similarly, this change is exposed via the RPC through [ResultBlockResults](https://github.com/tendermint/tendermint/blob/5721a13ab1f4479f9807f449f0bf5c536b9a05f2/rpc/coretypes/responses.go#L69) changing. Should we somehow shim or notify for this change? closes: #7658
2 years ago
abci: Synchronize FinalizeBlock with the updated specification (#7983) This change set implements the most recent version of `FinalizeBlock`. # What does this change actually contain? * This change set is rather large but fear not! The majority of the files touched and changes are renaming `ResponseDeliverTx` to `ExecTxResult`. This should be a pretty inoffensive change since they're effectively the same type but with a different name. * The `execBlockOnProxyApp` was totally removed since it served as just a wrapper around the logic that is now mostly encapsulated within `FinalizeBlock` * The `updateState` helper function has been made a public method on `State`. It was being exposed as a shim through the testing infrastructure, so this seemed innocuous. * Tests already existed to ensure that the application received the `ByzantineValidators` and the `ValidatorUpdates`, but one was fixed up to ensure that `LastCommitInfo` was being sent across. * Tests were removed from the `psql` indexer that seemed to search for an event in the indexer that was not being created. # Questions for reviewers * We store this [ABCIResponses](https://github.com/tendermint/tendermint/blob/5721a13ab1f4479f9807f449f0bf5c536b9a05f2/proto/tendermint/state/types.pb.go#L37) type in the data base as the block results. This type has changed since v0.35 to contain the `FinalizeBlock` response. I'm wondering if we need to do any shimming to keep the old data retrieveable? * Similarly, this change is exposed via the RPC through [ResultBlockResults](https://github.com/tendermint/tendermint/blob/5721a13ab1f4479f9807f449f0bf5c536b9a05f2/rpc/coretypes/responses.go#L69) changing. Should we somehow shim or notify for this change? closes: #7658
2 years ago
abci: Synchronize FinalizeBlock with the updated specification (#7983) This change set implements the most recent version of `FinalizeBlock`. # What does this change actually contain? * This change set is rather large but fear not! The majority of the files touched and changes are renaming `ResponseDeliverTx` to `ExecTxResult`. This should be a pretty inoffensive change since they're effectively the same type but with a different name. * The `execBlockOnProxyApp` was totally removed since it served as just a wrapper around the logic that is now mostly encapsulated within `FinalizeBlock` * The `updateState` helper function has been made a public method on `State`. It was being exposed as a shim through the testing infrastructure, so this seemed innocuous. * Tests already existed to ensure that the application received the `ByzantineValidators` and the `ValidatorUpdates`, but one was fixed up to ensure that `LastCommitInfo` was being sent across. * Tests were removed from the `psql` indexer that seemed to search for an event in the indexer that was not being created. # Questions for reviewers * We store this [ABCIResponses](https://github.com/tendermint/tendermint/blob/5721a13ab1f4479f9807f449f0bf5c536b9a05f2/proto/tendermint/state/types.pb.go#L37) type in the data base as the block results. This type has changed since v0.35 to contain the `FinalizeBlock` response. I'm wondering if we need to do any shimming to keep the old data retrieveable? * Similarly, this change is exposed via the RPC through [ResultBlockResults](https://github.com/tendermint/tendermint/blob/5721a13ab1f4479f9807f449f0bf5c536b9a05f2/rpc/coretypes/responses.go#L69) changing. Should we somehow shim or notify for this change? closes: #7658
2 years ago
abci: Synchronize FinalizeBlock with the updated specification (#7983) This change set implements the most recent version of `FinalizeBlock`. # What does this change actually contain? * This change set is rather large but fear not! The majority of the files touched and changes are renaming `ResponseDeliverTx` to `ExecTxResult`. This should be a pretty inoffensive change since they're effectively the same type but with a different name. * The `execBlockOnProxyApp` was totally removed since it served as just a wrapper around the logic that is now mostly encapsulated within `FinalizeBlock` * The `updateState` helper function has been made a public method on `State`. It was being exposed as a shim through the testing infrastructure, so this seemed innocuous. * Tests already existed to ensure that the application received the `ByzantineValidators` and the `ValidatorUpdates`, but one was fixed up to ensure that `LastCommitInfo` was being sent across. * Tests were removed from the `psql` indexer that seemed to search for an event in the indexer that was not being created. # Questions for reviewers * We store this [ABCIResponses](https://github.com/tendermint/tendermint/blob/5721a13ab1f4479f9807f449f0bf5c536b9a05f2/proto/tendermint/state/types.pb.go#L37) type in the data base as the block results. This type has changed since v0.35 to contain the `FinalizeBlock` response. I'm wondering if we need to do any shimming to keep the old data retrieveable? * Similarly, this change is exposed via the RPC through [ResultBlockResults](https://github.com/tendermint/tendermint/blob/5721a13ab1f4479f9807f449f0bf5c536b9a05f2/rpc/coretypes/responses.go#L69) changing. Should we somehow shim or notify for this change? closes: #7658
2 years ago
abci: Synchronize FinalizeBlock with the updated specification (#7983) This change set implements the most recent version of `FinalizeBlock`. # What does this change actually contain? * This change set is rather large but fear not! The majority of the files touched and changes are renaming `ResponseDeliverTx` to `ExecTxResult`. This should be a pretty inoffensive change since they're effectively the same type but with a different name. * The `execBlockOnProxyApp` was totally removed since it served as just a wrapper around the logic that is now mostly encapsulated within `FinalizeBlock` * The `updateState` helper function has been made a public method on `State`. It was being exposed as a shim through the testing infrastructure, so this seemed innocuous. * Tests already existed to ensure that the application received the `ByzantineValidators` and the `ValidatorUpdates`, but one was fixed up to ensure that `LastCommitInfo` was being sent across. * Tests were removed from the `psql` indexer that seemed to search for an event in the indexer that was not being created. # Questions for reviewers * We store this [ABCIResponses](https://github.com/tendermint/tendermint/blob/5721a13ab1f4479f9807f449f0bf5c536b9a05f2/proto/tendermint/state/types.pb.go#L37) type in the data base as the block results. This type has changed since v0.35 to contain the `FinalizeBlock` response. I'm wondering if we need to do any shimming to keep the old data retrieveable? * Similarly, this change is exposed via the RPC through [ResultBlockResults](https://github.com/tendermint/tendermint/blob/5721a13ab1f4479f9807f449f0bf5c536b9a05f2/rpc/coretypes/responses.go#L69) changing. Should we somehow shim or notify for this change? closes: #7658
2 years ago
  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 FinalizeBlock
  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 FinalizeBlock events for a given block by its height.
  39. // The following is indexed:
  40. //
  41. // primary key: encode(block.height | height) => encode(height)
  42. // FinalizeBlock events: encode(eventType.eventAttr|eventValue|height|finalize_block) => encode(height)
  43. func (idx *BlockerIndexer) Index(bh types.EventDataNewBlockHeader) error {
  44. batch := idx.store.NewBatch()
  45. defer batch.Close()
  46. height := bh.Header.Height
  47. // 1. index by height
  48. key, err := heightKey(height)
  49. if err != nil {
  50. return fmt.Errorf("failed to create block height index key: %w", err)
  51. }
  52. if err := batch.Set(key, int64ToBytes(height)); err != nil {
  53. return err
  54. }
  55. // 2. index FinalizeBlock events
  56. if err := idx.indexEvents(batch, bh.ResultFinalizeBlock.Events, types.EventTypeFinalizeBlock, height); err != nil {
  57. return fmt.Errorf("failed to index FinalizeBlock events: %w", err)
  58. }
  59. return batch.WriteSync()
  60. }
  61. // Search performs a query for block heights that match a given FinalizeBlock
  62. // The given query can match against zero or more block heights. In the case
  63. // of height queries, i.e. block.height=H, if the height is indexed, that height
  64. // alone will be returned. An error and nil slice is returned. Otherwise, a
  65. // non-nil slice and nil error is returned.
  66. func (idx *BlockerIndexer) Search(ctx context.Context, q *query.Query) ([]int64, error) {
  67. results := make([]int64, 0)
  68. select {
  69. case <-ctx.Done():
  70. return results, nil
  71. default:
  72. }
  73. conditions := q.Syntax()
  74. // If there is an exact height query, return the result immediately
  75. // (if it exists).
  76. height, ok := lookForHeight(conditions)
  77. if ok {
  78. ok, err := idx.Has(height)
  79. if err != nil {
  80. return nil, err
  81. }
  82. if ok {
  83. return []int64{height}, nil
  84. }
  85. return results, nil
  86. }
  87. var heightsInitialized bool
  88. filteredHeights := make(map[string][]byte)
  89. // conditions to skip because they're handled before "everything else"
  90. skipIndexes := make([]int, 0)
  91. // Extract ranges. If both upper and lower bounds exist, it's better to get
  92. // them in order as to not iterate over kvs that are not within range.
  93. ranges, rangeIndexes := indexer.LookForRanges(conditions)
  94. if len(ranges) > 0 {
  95. skipIndexes = append(skipIndexes, rangeIndexes...)
  96. for _, qr := range ranges {
  97. prefix, err := orderedcode.Append(nil, qr.Key)
  98. if err != nil {
  99. return nil, fmt.Errorf("failed to create prefix key: %w", err)
  100. }
  101. if !heightsInitialized {
  102. filteredHeights, err = idx.matchRange(ctx, qr, prefix, filteredHeights, true)
  103. if err != nil {
  104. return nil, err
  105. }
  106. heightsInitialized = true
  107. // Ignore any remaining conditions if the first condition resulted in no
  108. // matches (assuming implicit AND operand).
  109. if len(filteredHeights) == 0 {
  110. break
  111. }
  112. } else {
  113. filteredHeights, err = idx.matchRange(ctx, qr, prefix, filteredHeights, false)
  114. if err != nil {
  115. return nil, err
  116. }
  117. }
  118. }
  119. }
  120. // for all other conditions
  121. for i, c := range conditions {
  122. if intInSlice(i, skipIndexes) {
  123. continue
  124. }
  125. startKey, err := orderedcode.Append(nil, c.Tag, c.Arg.Value())
  126. if err != nil {
  127. return nil, err
  128. }
  129. if !heightsInitialized {
  130. filteredHeights, err = idx.match(ctx, c, startKey, filteredHeights, true)
  131. if err != nil {
  132. return nil, err
  133. }
  134. heightsInitialized = true
  135. // Ignore any remaining conditions if the first condition resulted in no
  136. // matches (assuming implicit AND operand).
  137. if len(filteredHeights) == 0 {
  138. break
  139. }
  140. } else {
  141. filteredHeights, err = idx.match(ctx, c, startKey, filteredHeights, false)
  142. if err != nil {
  143. return nil, err
  144. }
  145. }
  146. }
  147. // fetch matching heights
  148. results = make([]int64, 0, len(filteredHeights))
  149. heights:
  150. for _, hBz := range filteredHeights {
  151. h := int64FromBytes(hBz)
  152. ok, err := idx.Has(h)
  153. if err != nil {
  154. return nil, err
  155. }
  156. if ok {
  157. results = append(results, h)
  158. }
  159. select {
  160. case <-ctx.Done():
  161. break heights
  162. default:
  163. }
  164. }
  165. sort.Slice(results, func(i, j int) bool { return results[i] < results[j] })
  166. return results, nil
  167. }
  168. // matchRange returns all matching block heights that match a given QueryRange
  169. // and start key. An already filtered result (filteredHeights) is provided such
  170. // that any non-intersecting matches are removed.
  171. //
  172. // NOTE: The provided filteredHeights may be empty if no previous condition has
  173. // matched.
  174. func (idx *BlockerIndexer) matchRange(
  175. ctx context.Context,
  176. qr indexer.QueryRange,
  177. startKey []byte,
  178. filteredHeights map[string][]byte,
  179. firstRun bool,
  180. ) (map[string][]byte, error) {
  181. // A previous match was attempted but resulted in no matches, so we return
  182. // no matches (assuming AND operand).
  183. if !firstRun && len(filteredHeights) == 0 {
  184. return filteredHeights, nil
  185. }
  186. tmpHeights := make(map[string][]byte)
  187. lowerBound := qr.LowerBoundValue()
  188. upperBound := qr.UpperBoundValue()
  189. it, err := dbm.IteratePrefix(idx.store, startKey)
  190. if err != nil {
  191. return nil, fmt.Errorf("failed to create prefix iterator: %w", err)
  192. }
  193. defer it.Close()
  194. iter:
  195. for ; it.Valid(); it.Next() {
  196. var (
  197. eventValue string
  198. err error
  199. )
  200. if qr.Key == types.BlockHeightKey {
  201. eventValue, err = parseValueFromPrimaryKey(it.Key())
  202. } else {
  203. eventValue, err = parseValueFromEventKey(it.Key())
  204. }
  205. if err != nil {
  206. continue
  207. }
  208. if _, ok := qr.AnyBound().(int64); ok {
  209. v, err := strconv.ParseInt(eventValue, 10, 64)
  210. if err != nil {
  211. continue iter
  212. }
  213. include := true
  214. if lowerBound != nil && v < lowerBound.(int64) {
  215. include = false
  216. }
  217. if upperBound != nil && v > upperBound.(int64) {
  218. include = false
  219. }
  220. if include {
  221. tmpHeights[string(it.Value())] = it.Value()
  222. }
  223. }
  224. select {
  225. case <-ctx.Done():
  226. break iter
  227. default:
  228. }
  229. }
  230. if err := it.Error(); err != nil {
  231. return nil, err
  232. }
  233. if len(tmpHeights) == 0 || firstRun {
  234. // Either:
  235. //
  236. // 1. Regardless if a previous match was attempted, which may have had
  237. // results, but no match was found for the current condition, then we
  238. // return no matches (assuming AND operand).
  239. //
  240. // 2. A previous match was not attempted, so we return all results.
  241. return tmpHeights, nil
  242. }
  243. // Remove/reduce matches in filteredHashes that were not found in this
  244. // match (tmpHashes).
  245. for k := range filteredHeights {
  246. if tmpHeights[k] == nil {
  247. delete(filteredHeights, k)
  248. select {
  249. case <-ctx.Done():
  250. break
  251. default:
  252. }
  253. }
  254. }
  255. return filteredHeights, nil
  256. }
  257. // match returns all matching heights that meet a given query condition and start
  258. // key. An already filtered result (filteredHeights) is provided such that any
  259. // non-intersecting matches are removed.
  260. //
  261. // NOTE: The provided filteredHeights may be empty if no previous condition has
  262. // matched.
  263. func (idx *BlockerIndexer) match(
  264. ctx context.Context,
  265. c syntax.Condition,
  266. startKeyBz []byte,
  267. filteredHeights map[string][]byte,
  268. firstRun bool,
  269. ) (map[string][]byte, error) {
  270. // A previous match was attempted but resulted in no matches, so we return
  271. // no matches (assuming AND operand).
  272. if !firstRun && len(filteredHeights) == 0 {
  273. return filteredHeights, nil
  274. }
  275. tmpHeights := make(map[string][]byte)
  276. switch {
  277. case c.Op == syntax.TEq:
  278. it, err := dbm.IteratePrefix(idx.store, startKeyBz)
  279. if err != nil {
  280. return nil, fmt.Errorf("failed to create prefix iterator: %w", err)
  281. }
  282. defer it.Close()
  283. for ; it.Valid(); it.Next() {
  284. tmpHeights[string(it.Value())] = it.Value()
  285. if err := ctx.Err(); err != nil {
  286. break
  287. }
  288. }
  289. if err := it.Error(); err != nil {
  290. return nil, err
  291. }
  292. case c.Op == syntax.TExists:
  293. prefix, err := orderedcode.Append(nil, c.Tag)
  294. if err != nil {
  295. return nil, err
  296. }
  297. it, err := dbm.IteratePrefix(idx.store, prefix)
  298. if err != nil {
  299. return nil, fmt.Errorf("failed to create prefix iterator: %w", err)
  300. }
  301. defer it.Close()
  302. iterExists:
  303. for ; it.Valid(); it.Next() {
  304. tmpHeights[string(it.Value())] = it.Value()
  305. select {
  306. case <-ctx.Done():
  307. break iterExists
  308. default:
  309. }
  310. }
  311. if err := it.Error(); err != nil {
  312. return nil, err
  313. }
  314. case c.Op == syntax.TContains:
  315. prefix, err := orderedcode.Append(nil, c.Tag)
  316. if err != nil {
  317. return nil, err
  318. }
  319. it, err := dbm.IteratePrefix(idx.store, prefix)
  320. if err != nil {
  321. return nil, fmt.Errorf("failed to create prefix iterator: %w", err)
  322. }
  323. defer it.Close()
  324. iterContains:
  325. for ; it.Valid(); it.Next() {
  326. eventValue, err := parseValueFromEventKey(it.Key())
  327. if err != nil {
  328. continue
  329. }
  330. if strings.Contains(eventValue, c.Arg.Value()) {
  331. tmpHeights[string(it.Value())] = it.Value()
  332. }
  333. select {
  334. case <-ctx.Done():
  335. break iterContains
  336. default:
  337. }
  338. }
  339. if err := it.Error(); err != nil {
  340. return nil, err
  341. }
  342. default:
  343. return nil, errors.New("other operators should be handled already")
  344. }
  345. if len(tmpHeights) == 0 || firstRun {
  346. // Either:
  347. //
  348. // 1. Regardless if a previous match was attempted, which may have had
  349. // results, but no match was found for the current condition, then we
  350. // return no matches (assuming AND operand).
  351. //
  352. // 2. A previous match was not attempted, so we return all results.
  353. return tmpHeights, nil
  354. }
  355. // Remove/reduce matches in filteredHeights that were not found in this
  356. // match (tmpHeights).
  357. for k := range filteredHeights {
  358. if tmpHeights[k] == nil {
  359. delete(filteredHeights, k)
  360. select {
  361. case <-ctx.Done():
  362. break
  363. default:
  364. }
  365. }
  366. }
  367. return filteredHeights, nil
  368. }
  369. func (idx *BlockerIndexer) indexEvents(batch dbm.Batch, events []abci.Event, typ string, height int64) error {
  370. heightBz := int64ToBytes(height)
  371. for _, event := range events {
  372. // only index events with a non-empty type
  373. if len(event.Type) == 0 {
  374. continue
  375. }
  376. for _, attr := range event.Attributes {
  377. if len(attr.Key) == 0 {
  378. continue
  379. }
  380. // index iff the event specified index:true and it's not a reserved event
  381. compositeKey := fmt.Sprintf("%s.%s", event.Type, attr.Key)
  382. if compositeKey == types.BlockHeightKey {
  383. return fmt.Errorf("event type and attribute key \"%s\" is reserved; please use a different key", compositeKey)
  384. }
  385. if attr.GetIndex() {
  386. key, err := eventKey(compositeKey, typ, attr.Value, height)
  387. if err != nil {
  388. return fmt.Errorf("failed to create block index key: %w", err)
  389. }
  390. if err := batch.Set(key, heightBz); err != nil {
  391. return err
  392. }
  393. }
  394. }
  395. }
  396. return nil
  397. }