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.

422 lines
12 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
add support for block pruning via ABCI Commit response (#4588) * Added BlockStore.DeleteBlock() * Added initial block pruner prototype * wip * Added BlockStore.PruneBlocks() * Added consensus setting for block pruning * Added BlockStore base * Error on replay if base does not have blocks * Handle missing blocks when sending VoteSetMaj23Message * Error message tweak * Properly update blockstore state * Error message fix again * blockchain: ignore peer missing blocks * Added FIXME * Added test for block replay with truncated history * Handle peer base in blockchain reactor * Improved replay error handling * Added tests for Store.PruneBlocks() * Fix non-RPC handling of truncated block history * Panic on missing block meta in needProofBlock() * Updated changelog * Handle truncated block history in RPC layer * Added info about earliest block in /status RPC * Reorder height and base in blockchain reactor messages * Updated changelog * Fix tests * Appease linter * Minor review fixes * Non-empty BlockStores should always have base > 0 * Update code to assume base > 0 invariant * Added blockstore tests for pruning to 0 * Make sure we don't prune below the current base * Added BlockStore.Size() * config: added retain_blocks recommendations * Update v1 blockchain reactor to handle blockstore base * Added state database pruning * Propagate errors on missing validator sets * Comment tweaks * Improved error message Co-Authored-By: Anton Kaliaev <anton.kalyaev@gmail.com> * use ABCI field ResponseCommit.retain_height instead of retain-blocks config option * remove State.RetainHeight, return value instead * fix minor issues * rename pruneHeights() to pruneBlocks() * noop to fix GitHub borkage Co-authored-by: Anton Kaliaev <anton.kalyaev@gmail.com>
4 years ago
add support for block pruning via ABCI Commit response (#4588) * Added BlockStore.DeleteBlock() * Added initial block pruner prototype * wip * Added BlockStore.PruneBlocks() * Added consensus setting for block pruning * Added BlockStore base * Error on replay if base does not have blocks * Handle missing blocks when sending VoteSetMaj23Message * Error message tweak * Properly update blockstore state * Error message fix again * blockchain: ignore peer missing blocks * Added FIXME * Added test for block replay with truncated history * Handle peer base in blockchain reactor * Improved replay error handling * Added tests for Store.PruneBlocks() * Fix non-RPC handling of truncated block history * Panic on missing block meta in needProofBlock() * Updated changelog * Handle truncated block history in RPC layer * Added info about earliest block in /status RPC * Reorder height and base in blockchain reactor messages * Updated changelog * Fix tests * Appease linter * Minor review fixes * Non-empty BlockStores should always have base > 0 * Update code to assume base > 0 invariant * Added blockstore tests for pruning to 0 * Make sure we don't prune below the current base * Added BlockStore.Size() * config: added retain_blocks recommendations * Update v1 blockchain reactor to handle blockstore base * Added state database pruning * Propagate errors on missing validator sets * Comment tweaks * Improved error message Co-Authored-By: Anton Kaliaev <anton.kalyaev@gmail.com> * use ABCI field ResponseCommit.retain_height instead of retain-blocks config option * remove State.RetainHeight, return value instead * fix minor issues * rename pruneHeights() to pruneBlocks() * noop to fix GitHub borkage Co-authored-by: Anton Kaliaev <anton.kalyaev@gmail.com>
4 years ago
add support for block pruning via ABCI Commit response (#4588) * Added BlockStore.DeleteBlock() * Added initial block pruner prototype * wip * Added BlockStore.PruneBlocks() * Added consensus setting for block pruning * Added BlockStore base * Error on replay if base does not have blocks * Handle missing blocks when sending VoteSetMaj23Message * Error message tweak * Properly update blockstore state * Error message fix again * blockchain: ignore peer missing blocks * Added FIXME * Added test for block replay with truncated history * Handle peer base in blockchain reactor * Improved replay error handling * Added tests for Store.PruneBlocks() * Fix non-RPC handling of truncated block history * Panic on missing block meta in needProofBlock() * Updated changelog * Handle truncated block history in RPC layer * Added info about earliest block in /status RPC * Reorder height and base in blockchain reactor messages * Updated changelog * Fix tests * Appease linter * Minor review fixes * Non-empty BlockStores should always have base > 0 * Update code to assume base > 0 invariant * Added blockstore tests for pruning to 0 * Make sure we don't prune below the current base * Added BlockStore.Size() * config: added retain_blocks recommendations * Update v1 blockchain reactor to handle blockstore base * Added state database pruning * Propagate errors on missing validator sets * Comment tweaks * Improved error message Co-Authored-By: Anton Kaliaev <anton.kalyaev@gmail.com> * use ABCI field ResponseCommit.retain_height instead of retain-blocks config option * remove State.RetainHeight, return value instead * fix minor issues * rename pruneHeights() to pruneBlocks() * noop to fix GitHub borkage Co-authored-by: Anton Kaliaev <anton.kalyaev@gmail.com>
4 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 coretypes
  2. import (
  3. "encoding/json"
  4. "errors"
  5. "time"
  6. abci "github.com/tendermint/tendermint/abci/types"
  7. "github.com/tendermint/tendermint/crypto"
  8. "github.com/tendermint/tendermint/internal/jsontypes"
  9. "github.com/tendermint/tendermint/libs/bytes"
  10. tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
  11. "github.com/tendermint/tendermint/types"
  12. )
  13. // List of standardized errors used across RPC
  14. var (
  15. ErrZeroOrNegativePerPage = errors.New("zero or negative per_page")
  16. ErrPageOutOfRange = errors.New("page should be within range")
  17. ErrZeroOrNegativeHeight = errors.New("height must be greater than zero")
  18. ErrHeightExceedsChainHead = errors.New("height must be less than or equal to the head of the node's blockchain")
  19. ErrHeightNotAvailable = errors.New("height is not available")
  20. // ErrInvalidRequest is used as a wrapper to cover more specific cases where the user has
  21. // made an invalid request
  22. ErrInvalidRequest = errors.New("invalid request")
  23. )
  24. // List of blocks
  25. type ResultBlockchainInfo struct {
  26. LastHeight int64 `json:"last_height,string"`
  27. BlockMetas []*types.BlockMeta `json:"block_metas"`
  28. }
  29. // Genesis file
  30. type ResultGenesis struct {
  31. Genesis *types.GenesisDoc `json:"genesis"`
  32. }
  33. // ResultGenesisChunk is the output format for the chunked/paginated
  34. // interface. These chunks are produced by converting the genesis
  35. // document to JSON and then splitting the resulting payload into
  36. // 16 megabyte blocks and then base64 encoding each block.
  37. type ResultGenesisChunk struct {
  38. ChunkNumber int `json:"chunk,string"`
  39. TotalChunks int `json:"total,string"`
  40. Data string `json:"data"`
  41. }
  42. // Single block (with meta)
  43. type ResultBlock struct {
  44. BlockID types.BlockID `json:"block_id"`
  45. Block *types.Block `json:"block"`
  46. }
  47. // ResultHeader represents the response for a Header RPC Client query
  48. type ResultHeader struct {
  49. Header *types.Header `json:"header"`
  50. }
  51. // Commit and Header
  52. type ResultCommit struct {
  53. types.SignedHeader `json:"signed_header"`
  54. CanonicalCommit bool `json:"canonical"`
  55. }
  56. // ABCI results from a block
  57. type ResultBlockResults struct {
  58. Height int64 `json:"height,string"`
  59. TxsResults []*abci.ExecTxResult `json:"txs_results"`
  60. TotalGasUsed int64 `json:"total_gas_used,string"`
  61. FinalizeBlockEvents []abci.Event `json:"finalize_block_events"`
  62. ValidatorUpdates []abci.ValidatorUpdate `json:"validator_updates"`
  63. ConsensusParamUpdates *tmproto.ConsensusParams `json:"consensus_param_updates"`
  64. }
  65. // NewResultCommit is a helper to initialize the ResultCommit with
  66. // the embedded struct
  67. func NewResultCommit(header *types.Header, commit *types.Commit,
  68. canonical bool) *ResultCommit {
  69. return &ResultCommit{
  70. SignedHeader: types.SignedHeader{
  71. Header: header,
  72. Commit: commit,
  73. },
  74. CanonicalCommit: canonical,
  75. }
  76. }
  77. // Info about the node's syncing state
  78. type SyncInfo struct {
  79. LatestBlockHash bytes.HexBytes `json:"latest_block_hash"`
  80. LatestAppHash bytes.HexBytes `json:"latest_app_hash"`
  81. LatestBlockHeight int64 `json:"latest_block_height,string"`
  82. LatestBlockTime time.Time `json:"latest_block_time"`
  83. EarliestBlockHash bytes.HexBytes `json:"earliest_block_hash"`
  84. EarliestAppHash bytes.HexBytes `json:"earliest_app_hash"`
  85. EarliestBlockHeight int64 `json:"earliest_block_height,string"`
  86. EarliestBlockTime time.Time `json:"earliest_block_time"`
  87. MaxPeerBlockHeight int64 `json:"max_peer_block_height,string"`
  88. CatchingUp bool `json:"catching_up"`
  89. TotalSyncedTime time.Duration `json:"total_synced_time,string"`
  90. RemainingTime time.Duration `json:"remaining_time,string"`
  91. TotalSnapshots int64 `json:"total_snapshots,string"`
  92. ChunkProcessAvgTime time.Duration `json:"chunk_process_avg_time,string"`
  93. SnapshotHeight int64 `json:"snapshot_height,string"`
  94. SnapshotChunksCount int64 `json:"snapshot_chunks_count,string"`
  95. SnapshotChunksTotal int64 `json:"snapshot_chunks_total,string"`
  96. BackFilledBlocks int64 `json:"backfilled_blocks,string"`
  97. BackFillBlocksTotal int64 `json:"backfill_blocks_total,string"`
  98. }
  99. type ApplicationInfo struct {
  100. Version string `json:"version"`
  101. }
  102. // Info about the node's validator
  103. type ValidatorInfo struct {
  104. Address bytes.HexBytes
  105. PubKey crypto.PubKey
  106. VotingPower int64
  107. }
  108. type validatorInfoJSON struct {
  109. Address bytes.HexBytes `json:"address"`
  110. PubKey json.RawMessage `json:"pub_key"`
  111. VotingPower int64 `json:"voting_power,string"`
  112. }
  113. func (v ValidatorInfo) MarshalJSON() ([]byte, error) {
  114. pk, err := jsontypes.Marshal(v.PubKey)
  115. if err != nil {
  116. return nil, err
  117. }
  118. return json.Marshal(validatorInfoJSON{
  119. Address: v.Address, PubKey: pk, VotingPower: v.VotingPower,
  120. })
  121. }
  122. func (v *ValidatorInfo) UnmarshalJSON(data []byte) error {
  123. var val validatorInfoJSON
  124. if err := json.Unmarshal(data, &val); err != nil {
  125. return err
  126. }
  127. if err := jsontypes.Unmarshal(val.PubKey, &v.PubKey); err != nil {
  128. return err
  129. }
  130. v.Address = val.Address
  131. v.VotingPower = val.VotingPower
  132. return nil
  133. }
  134. // Node Status
  135. type ResultStatus struct {
  136. NodeInfo types.NodeInfo `json:"node_info"`
  137. ApplicationInfo ApplicationInfo `json:"application_info,omitempty"`
  138. SyncInfo SyncInfo `json:"sync_info"`
  139. ValidatorInfo ValidatorInfo `json:"validator_info"`
  140. LightClientInfo types.LightClientInfo `json:"light_client_info,omitempty"`
  141. }
  142. // Is TxIndexing enabled
  143. func (s *ResultStatus) TxIndexEnabled() bool {
  144. if s == nil {
  145. return false
  146. }
  147. return s.NodeInfo.Other.TxIndex == "on"
  148. }
  149. // Info about peer connections
  150. type ResultNetInfo struct {
  151. Listening bool `json:"listening"`
  152. Listeners []string `json:"listeners"`
  153. NPeers int `json:"n_peers,string"`
  154. Peers []Peer `json:"peers"`
  155. }
  156. // Log from dialing seeds
  157. type ResultDialSeeds struct {
  158. Log string `json:"log"`
  159. }
  160. // Log from dialing peers
  161. type ResultDialPeers struct {
  162. Log string `json:"log"`
  163. }
  164. // A peer
  165. type Peer struct {
  166. ID types.NodeID `json:"node_id"`
  167. URL string `json:"url"`
  168. }
  169. // Validators for a height.
  170. type ResultValidators struct {
  171. BlockHeight int64 `json:"block_height,string"`
  172. Validators []*types.Validator `json:"validators"`
  173. Count int `json:"count,string"` // Count of actual validators in this result
  174. Total int `json:"total,string"` // Total number of validators
  175. }
  176. // ConsensusParams for given height
  177. type ResultConsensusParams struct {
  178. BlockHeight int64 `json:"block_height,string"`
  179. ConsensusParams types.ConsensusParams `json:"consensus_params"`
  180. }
  181. // Info about the consensus state.
  182. // UNSTABLE
  183. type ResultDumpConsensusState struct {
  184. RoundState json.RawMessage `json:"round_state"`
  185. Peers []PeerStateInfo `json:"peers"`
  186. }
  187. // UNSTABLE
  188. type PeerStateInfo struct {
  189. NodeAddress string `json:"node_address"`
  190. PeerState json.RawMessage `json:"peer_state"`
  191. }
  192. // UNSTABLE
  193. type ResultConsensusState struct {
  194. RoundState json.RawMessage `json:"round_state"`
  195. }
  196. // CheckTx result
  197. type ResultBroadcastTx struct {
  198. Code uint32 `json:"code"`
  199. Data bytes.HexBytes `json:"data"`
  200. Log string `json:"log"`
  201. Codespace string `json:"codespace"`
  202. MempoolError string `json:"mempool_error"`
  203. Hash bytes.HexBytes `json:"hash"`
  204. }
  205. // CheckTx and DeliverTx results
  206. type ResultBroadcastTxCommit struct {
  207. CheckTx abci.ResponseCheckTx `json:"check_tx"`
  208. TxResult abci.ExecTxResult `json:"tx_result"`
  209. Hash bytes.HexBytes `json:"hash"`
  210. Height int64 `json:"height,string"`
  211. }
  212. // ResultCheckTx wraps abci.ResponseCheckTx.
  213. type ResultCheckTx struct {
  214. abci.ResponseCheckTx
  215. }
  216. // Result of querying for a tx
  217. type ResultTx struct {
  218. Hash bytes.HexBytes `json:"hash"`
  219. Height int64 `json:"height,string"`
  220. Index uint32 `json:"index"`
  221. TxResult abci.ExecTxResult `json:"tx_result"`
  222. Tx types.Tx `json:"tx"`
  223. Proof types.TxProof `json:"proof,omitempty"`
  224. }
  225. // Result of searching for txs
  226. type ResultTxSearch struct {
  227. Txs []*ResultTx `json:"txs"`
  228. TotalCount int `json:"total_count,string"`
  229. }
  230. // ResultBlockSearch defines the RPC response type for a block search by events.
  231. type ResultBlockSearch struct {
  232. Blocks []*ResultBlock `json:"blocks"`
  233. TotalCount int `json:"total_count,string"`
  234. }
  235. // List of mempool txs
  236. type ResultUnconfirmedTxs struct {
  237. Count int `json:"n_txs,string"`
  238. Total int `json:"total,string"`
  239. TotalBytes int64 `json:"total_bytes,string"`
  240. Txs []types.Tx `json:"txs"`
  241. }
  242. // Info abci msg
  243. type ResultABCIInfo struct {
  244. Response abci.ResponseInfo `json:"response"`
  245. }
  246. // Query abci msg
  247. type ResultABCIQuery struct {
  248. Response abci.ResponseQuery `json:"response"`
  249. }
  250. // Result of broadcasting evidence
  251. type ResultBroadcastEvidence struct {
  252. Hash []byte `json:"hash"`
  253. }
  254. // empty results
  255. type (
  256. ResultUnsafeFlushMempool struct{}
  257. ResultUnsafeProfile struct{}
  258. ResultSubscribe struct{}
  259. ResultUnsubscribe struct{}
  260. ResultHealth struct{}
  261. )
  262. // Event data from a subscription
  263. type ResultEvent struct {
  264. SubscriptionID string
  265. Query string
  266. Data types.EventData
  267. Events []abci.Event
  268. }
  269. type resultEventJSON struct {
  270. SubscriptionID string `json:"subscription_id"`
  271. Query string `json:"query"`
  272. Data json.RawMessage `json:"data"`
  273. Events []abci.Event `json:"events"`
  274. }
  275. func (r ResultEvent) MarshalJSON() ([]byte, error) {
  276. evt, err := jsontypes.Marshal(r.Data)
  277. if err != nil {
  278. return nil, err
  279. }
  280. return json.Marshal(resultEventJSON{
  281. SubscriptionID: r.SubscriptionID,
  282. Query: r.Query,
  283. Data: evt,
  284. Events: r.Events,
  285. })
  286. }
  287. func (r *ResultEvent) UnmarshalJSON(data []byte) error {
  288. var res resultEventJSON
  289. if err := json.Unmarshal(data, &res); err != nil {
  290. return err
  291. }
  292. if err := jsontypes.Unmarshal(res.Data, &r.Data); err != nil {
  293. return err
  294. }
  295. r.SubscriptionID = res.SubscriptionID
  296. r.Query = res.Query
  297. r.Events = res.Events
  298. return nil
  299. }
  300. // Evidence is an argument wrapper for a types.Evidence value, that handles
  301. // encoding and decoding through JSON.
  302. type Evidence struct {
  303. Value types.Evidence
  304. }
  305. func (e Evidence) MarshalJSON() ([]byte, error) { return jsontypes.Marshal(e.Value) }
  306. func (e *Evidence) UnmarshalJSON(data []byte) error { return jsontypes.Unmarshal(data, &e.Value) }
  307. // RequestEvents is the argument for the "/events" RPC endpoint.
  308. type RequestEvents struct {
  309. // Optional filter spec. If nil or empty, all items are eligible.
  310. Filter *EventFilter `json:"filter"`
  311. // The maximum number of eligible items to return.
  312. // If zero or negative, the server will report a default number.
  313. MaxItems int `json:"maxItems"`
  314. // Return only items after this cursor. If empty, the limit is just
  315. // before the the beginning of the event log.
  316. After string `json:"after"`
  317. // Return only items before this cursor. If empty, the limit is just
  318. // after the head of the event log.
  319. Before string `json:"before"`
  320. // Wait for up to this long for events to be available.
  321. WaitTime time.Duration `json:"waitTime"`
  322. }
  323. // An EventFilter specifies which events are selected by an /events request.
  324. type EventFilter struct {
  325. Query string `json:"query"`
  326. }
  327. // ResultEvents is the response from the "/events" RPC endpoint.
  328. type ResultEvents struct {
  329. // The items matching the request parameters, from newest
  330. // to oldest, if any were available within the timeout.
  331. Items []*EventItem `json:"items"`
  332. // This is true if there is at least one older matching item
  333. // available in the log that was not returned.
  334. More bool `json:"more"`
  335. // The cursor of the oldest item in the log at the time of this reply,
  336. // or "" if the log is empty.
  337. Oldest string `json:"oldest"`
  338. // The cursor of the newest item in the log at the time of this reply,
  339. // or "" if the log is empty.
  340. Newest string `json:"newest"`
  341. }
  342. type EventItem struct {
  343. // The cursor of this item.
  344. Cursor string `json:"cursor"`
  345. // The event label of this item (for example, "Vote").
  346. Event string `json:"event,omitempty"`
  347. // The encoded event data for this item. The content is a JSON object with
  348. // the following structure:
  349. //
  350. // {
  351. // "type": "type-tag",
  352. // "value": <json-encoded-value>
  353. // }
  354. //
  355. // The known type tags are defined by the tendermint/types package.
  356. Data json.RawMessage `json:"data"`
  357. }