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.

878 lines
25 KiB

max-bytes PR follow-up (#2318) * ReapMaxTxs: return all txs if max is negative this mirrors ReapMaxBytes behavior See https://github.com/tendermint/tendermint/pull/2184#discussion_r214439950 * increase MaxAminoOverheadForBlock tested with: ``` func TestMaxAminoOverheadForBlock(t *testing.T) { maxChainID := "" for i := 0; i < MaxChainIDLen; i++ { maxChainID += "𠜎" } h := Header{ ChainID: maxChainID, Height: 10, Time: time.Now().UTC(), NumTxs: 100, TotalTxs: 200, LastBlockID: makeBlockID(make([]byte, 20), 300, make([]byte, 20)), LastCommitHash: tmhash.Sum([]byte("last_commit_hash")), DataHash: tmhash.Sum([]byte("data_hash")), ValidatorsHash: tmhash.Sum([]byte("validators_hash")), NextValidatorsHash: tmhash.Sum([]byte("next_validators_hash")), ConsensusHash: tmhash.Sum([]byte("consensus_hash")), AppHash: tmhash.Sum([]byte("app_hash")), LastResultsHash: tmhash.Sum([]byte("last_results_hash")), EvidenceHash: tmhash.Sum([]byte("evidence_hash")), ProposerAddress: tmhash.Sum([]byte("proposer_address")), } b := Block{ Header: h, Data: Data{Txs: makeTxs(10000, 100)}, Evidence: EvidenceData{}, LastCommit: &Commit{}, } bz, err := cdc.MarshalBinary(b) require.NoError(t, err) assert.Equal(t, MaxHeaderBytes+MaxAminoOverheadForBlock-2, len(bz)-1000000-20000-1) } ``` * fix MaxYYY constants calculation by using math.MaxInt64 See https://github.com/tendermint/tendermint/pull/2184#discussion_r214444244 * pass mempool filter as an option See https://github.com/tendermint/tendermint/pull/2184#discussion_r214445869 * fixes after Dev's comments
6 years ago
max-bytes PR follow-up (#2318) * ReapMaxTxs: return all txs if max is negative this mirrors ReapMaxBytes behavior See https://github.com/tendermint/tendermint/pull/2184#discussion_r214439950 * increase MaxAminoOverheadForBlock tested with: ``` func TestMaxAminoOverheadForBlock(t *testing.T) { maxChainID := "" for i := 0; i < MaxChainIDLen; i++ { maxChainID += "𠜎" } h := Header{ ChainID: maxChainID, Height: 10, Time: time.Now().UTC(), NumTxs: 100, TotalTxs: 200, LastBlockID: makeBlockID(make([]byte, 20), 300, make([]byte, 20)), LastCommitHash: tmhash.Sum([]byte("last_commit_hash")), DataHash: tmhash.Sum([]byte("data_hash")), ValidatorsHash: tmhash.Sum([]byte("validators_hash")), NextValidatorsHash: tmhash.Sum([]byte("next_validators_hash")), ConsensusHash: tmhash.Sum([]byte("consensus_hash")), AppHash: tmhash.Sum([]byte("app_hash")), LastResultsHash: tmhash.Sum([]byte("last_results_hash")), EvidenceHash: tmhash.Sum([]byte("evidence_hash")), ProposerAddress: tmhash.Sum([]byte("proposer_address")), } b := Block{ Header: h, Data: Data{Txs: makeTxs(10000, 100)}, Evidence: EvidenceData{}, LastCommit: &Commit{}, } bz, err := cdc.MarshalBinary(b) require.NoError(t, err) assert.Equal(t, MaxHeaderBytes+MaxAminoOverheadForBlock-2, len(bz)-1000000-20000-1) } ``` * fix MaxYYY constants calculation by using math.MaxInt64 See https://github.com/tendermint/tendermint/pull/2184#discussion_r214444244 * pass mempool filter as an option See https://github.com/tendermint/tendermint/pull/2184#discussion_r214445869 * fixes after Dev's comments
6 years ago
7 years ago
10 years ago
10 years ago
6 years ago
7 years ago
10 years ago
9 years ago
10 years ago
  1. package types
  2. import (
  3. "bytes"
  4. "fmt"
  5. "strings"
  6. "sync"
  7. "time"
  8. "github.com/pkg/errors"
  9. "github.com/tendermint/tendermint/crypto"
  10. "github.com/tendermint/tendermint/crypto/merkle"
  11. "github.com/tendermint/tendermint/crypto/tmhash"
  12. cmn "github.com/tendermint/tendermint/libs/common"
  13. "github.com/tendermint/tendermint/version"
  14. )
  15. const (
  16. // MaxHeaderBytes is a maximum header size (including amino overhead).
  17. MaxHeaderBytes int64 = 653
  18. // MaxAminoOverheadForBlock - maximum amino overhead to encode a block (up to
  19. // MaxBlockSizeBytes in size) not including it's parts except Data.
  20. // This means it also excludes the overhead for individual transactions.
  21. // To compute individual transactions' overhead use types.ComputeAminoOverhead(tx types.Tx, fieldNum int).
  22. //
  23. // Uvarint length of MaxBlockSizeBytes: 4 bytes
  24. // 2 fields (2 embedded): 2 bytes
  25. // Uvarint length of Data.Txs: 4 bytes
  26. // Data.Txs field: 1 byte
  27. MaxAminoOverheadForBlock int64 = 11
  28. )
  29. // Block defines the atomic unit of a Tendermint blockchain.
  30. type Block struct {
  31. mtx sync.Mutex
  32. Header `json:"header"`
  33. Data `json:"data"`
  34. Evidence EvidenceData `json:"evidence"`
  35. LastCommit *Commit `json:"last_commit"`
  36. }
  37. // MakeBlock returns a new block with an empty header, except what can be
  38. // computed from itself.
  39. // It populates the same set of fields validated by ValidateBasic.
  40. func MakeBlock(height int64, txs []Tx, lastCommit *Commit, evidence []Evidence) *Block {
  41. block := &Block{
  42. Header: Header{
  43. Height: height,
  44. NumTxs: int64(len(txs)),
  45. },
  46. Data: Data{
  47. Txs: txs,
  48. },
  49. Evidence: EvidenceData{Evidence: evidence},
  50. LastCommit: lastCommit,
  51. }
  52. block.fillHeader()
  53. return block
  54. }
  55. // ValidateBasic performs basic validation that doesn't involve state data.
  56. // It checks the internal consistency of the block.
  57. // Further validation is done using state#ValidateBlock.
  58. func (b *Block) ValidateBasic() error {
  59. if b == nil {
  60. return errors.New("nil block")
  61. }
  62. b.mtx.Lock()
  63. defer b.mtx.Unlock()
  64. if len(b.ChainID) > MaxChainIDLen {
  65. return fmt.Errorf("ChainID is too long. Max is %d, got %d", MaxChainIDLen, len(b.ChainID))
  66. }
  67. if b.Height < 0 {
  68. return errors.New("Negative Header.Height")
  69. } else if b.Height == 0 {
  70. return errors.New("Zero Header.Height")
  71. }
  72. // NOTE: Timestamp validation is subtle and handled elsewhere.
  73. newTxs := int64(len(b.Data.Txs))
  74. if b.NumTxs != newTxs {
  75. return fmt.Errorf("Wrong Header.NumTxs. Expected %v, got %v",
  76. newTxs,
  77. b.NumTxs,
  78. )
  79. }
  80. // TODO: fix tests so we can do this
  81. /*if b.TotalTxs < b.NumTxs {
  82. return fmt.Errorf("Header.TotalTxs (%d) is less than Header.NumTxs (%d)", b.TotalTxs, b.NumTxs)
  83. }*/
  84. if b.TotalTxs < 0 {
  85. return errors.New("Negative Header.TotalTxs")
  86. }
  87. if err := b.LastBlockID.ValidateBasic(); err != nil {
  88. return fmt.Errorf("Wrong Header.LastBlockID: %v", err)
  89. }
  90. // Validate the last commit and its hash.
  91. if b.Header.Height > 1 {
  92. if b.LastCommit == nil {
  93. return errors.New("nil LastCommit")
  94. }
  95. if err := b.LastCommit.ValidateBasic(); err != nil {
  96. return fmt.Errorf("Wrong LastCommit")
  97. }
  98. }
  99. if err := ValidateHash(b.LastCommitHash); err != nil {
  100. return fmt.Errorf("Wrong Header.LastCommitHash: %v", err)
  101. }
  102. if !bytes.Equal(b.LastCommitHash, b.LastCommit.Hash()) {
  103. return fmt.Errorf("Wrong Header.LastCommitHash. Expected %v, got %v",
  104. b.LastCommit.Hash(),
  105. b.LastCommitHash,
  106. )
  107. }
  108. // Validate the hash of the transactions.
  109. // NOTE: b.Data.Txs may be nil, but b.Data.Hash()
  110. // still works fine
  111. if err := ValidateHash(b.DataHash); err != nil {
  112. return fmt.Errorf("Wrong Header.DataHash: %v", err)
  113. }
  114. if !bytes.Equal(b.DataHash, b.Data.Hash()) {
  115. return fmt.Errorf(
  116. "Wrong Header.DataHash. Expected %v, got %v",
  117. b.Data.Hash(),
  118. b.DataHash,
  119. )
  120. }
  121. // Basic validation of hashes related to application data.
  122. // Will validate fully against state in state#ValidateBlock.
  123. if err := ValidateHash(b.ValidatorsHash); err != nil {
  124. return fmt.Errorf("Wrong Header.ValidatorsHash: %v", err)
  125. }
  126. if err := ValidateHash(b.NextValidatorsHash); err != nil {
  127. return fmt.Errorf("Wrong Header.NextValidatorsHash: %v", err)
  128. }
  129. if err := ValidateHash(b.ConsensusHash); err != nil {
  130. return fmt.Errorf("Wrong Header.ConsensusHash: %v", err)
  131. }
  132. // NOTE: AppHash is arbitrary length
  133. if err := ValidateHash(b.LastResultsHash); err != nil {
  134. return fmt.Errorf("Wrong Header.LastResultsHash: %v", err)
  135. }
  136. // Validate evidence and its hash.
  137. if err := ValidateHash(b.EvidenceHash); err != nil {
  138. return fmt.Errorf("Wrong Header.EvidenceHash: %v", err)
  139. }
  140. // NOTE: b.Evidence.Evidence may be nil, but we're just looping.
  141. for i, ev := range b.Evidence.Evidence {
  142. if err := ev.ValidateBasic(); err != nil {
  143. return fmt.Errorf("Invalid evidence (#%d): %v", i, err)
  144. }
  145. }
  146. if !bytes.Equal(b.EvidenceHash, b.Evidence.Hash()) {
  147. return fmt.Errorf("Wrong Header.EvidenceHash. Expected %v, got %v",
  148. b.EvidenceHash,
  149. b.Evidence.Hash(),
  150. )
  151. }
  152. if len(b.ProposerAddress) != crypto.AddressSize {
  153. return fmt.Errorf("Expected len(Header.ProposerAddress) to be %d, got %d",
  154. crypto.AddressSize, len(b.ProposerAddress))
  155. }
  156. return nil
  157. }
  158. // fillHeader fills in any remaining header fields that are a function of the block data
  159. func (b *Block) fillHeader() {
  160. if b.LastCommitHash == nil {
  161. b.LastCommitHash = b.LastCommit.Hash()
  162. }
  163. if b.DataHash == nil {
  164. b.DataHash = b.Data.Hash()
  165. }
  166. if b.EvidenceHash == nil {
  167. b.EvidenceHash = b.Evidence.Hash()
  168. }
  169. }
  170. // Hash computes and returns the block hash.
  171. // If the block is incomplete, block hash is nil for safety.
  172. func (b *Block) Hash() cmn.HexBytes {
  173. if b == nil {
  174. return nil
  175. }
  176. b.mtx.Lock()
  177. defer b.mtx.Unlock()
  178. if b == nil || b.LastCommit == nil {
  179. return nil
  180. }
  181. b.fillHeader()
  182. return b.Header.Hash()
  183. }
  184. // MakePartSet returns a PartSet containing parts of a serialized block.
  185. // This is the form in which the block is gossipped to peers.
  186. // CONTRACT: partSize is greater than zero.
  187. func (b *Block) MakePartSet(partSize int) *PartSet {
  188. if b == nil {
  189. return nil
  190. }
  191. b.mtx.Lock()
  192. defer b.mtx.Unlock()
  193. // We prefix the byte length, so that unmarshaling
  194. // can easily happen via a reader.
  195. bz, err := cdc.MarshalBinaryLengthPrefixed(b)
  196. if err != nil {
  197. panic(err)
  198. }
  199. return NewPartSetFromData(bz, partSize)
  200. }
  201. // HashesTo is a convenience function that checks if a block hashes to the given argument.
  202. // Returns false if the block is nil or the hash is empty.
  203. func (b *Block) HashesTo(hash []byte) bool {
  204. if len(hash) == 0 {
  205. return false
  206. }
  207. if b == nil {
  208. return false
  209. }
  210. return bytes.Equal(b.Hash(), hash)
  211. }
  212. // Size returns size of the block in bytes.
  213. func (b *Block) Size() int {
  214. bz, err := cdc.MarshalBinaryBare(b)
  215. if err != nil {
  216. return 0
  217. }
  218. return len(bz)
  219. }
  220. // String returns a string representation of the block
  221. func (b *Block) String() string {
  222. return b.StringIndented("")
  223. }
  224. // StringIndented returns a string representation of the block
  225. func (b *Block) StringIndented(indent string) string {
  226. if b == nil {
  227. return "nil-Block"
  228. }
  229. return fmt.Sprintf(`Block{
  230. %s %v
  231. %s %v
  232. %s %v
  233. %s %v
  234. %s}#%v`,
  235. indent, b.Header.StringIndented(indent+" "),
  236. indent, b.Data.StringIndented(indent+" "),
  237. indent, b.Evidence.StringIndented(indent+" "),
  238. indent, b.LastCommit.StringIndented(indent+" "),
  239. indent, b.Hash())
  240. }
  241. // StringShort returns a shortened string representation of the block
  242. func (b *Block) StringShort() string {
  243. if b == nil {
  244. return "nil-Block"
  245. }
  246. return fmt.Sprintf("Block#%v", b.Hash())
  247. }
  248. //-----------------------------------------------------------
  249. // These methods are for Protobuf Compatibility
  250. // Marshal returns the amino encoding.
  251. func (b *Block) Marshal() ([]byte, error) {
  252. return cdc.MarshalBinaryBare(b)
  253. }
  254. // MarshalTo calls Marshal and copies to the given buffer.
  255. func (b *Block) MarshalTo(data []byte) (int, error) {
  256. bs, err := b.Marshal()
  257. if err != nil {
  258. return -1, err
  259. }
  260. return copy(data, bs), nil
  261. }
  262. // Unmarshal deserializes from amino encoded form.
  263. func (b *Block) Unmarshal(bs []byte) error {
  264. return cdc.UnmarshalBinaryBare(bs, b)
  265. }
  266. //-----------------------------------------------------------------------------
  267. // MaxDataBytes returns the maximum size of block's data.
  268. //
  269. // XXX: Panics on negative result.
  270. func MaxDataBytes(maxBytes int64, valsCount, evidenceCount int) int64 {
  271. maxDataBytes := maxBytes -
  272. MaxAminoOverheadForBlock -
  273. MaxHeaderBytes -
  274. int64(valsCount)*MaxVoteBytes -
  275. int64(evidenceCount)*MaxEvidenceBytes
  276. if maxDataBytes < 0 {
  277. panic(fmt.Sprintf(
  278. "Negative MaxDataBytes. BlockSize.MaxBytes=%d is too small to accommodate header&lastCommit&evidence=%d",
  279. maxBytes,
  280. -(maxDataBytes - maxBytes),
  281. ))
  282. }
  283. return maxDataBytes
  284. }
  285. // MaxDataBytesUnknownEvidence returns the maximum size of block's data when
  286. // evidence count is unknown. MaxEvidencePerBlock will be used for the size
  287. // of evidence.
  288. //
  289. // XXX: Panics on negative result.
  290. func MaxDataBytesUnknownEvidence(maxBytes int64, valsCount int) int64 {
  291. _, maxEvidenceBytes := MaxEvidencePerBlock(maxBytes)
  292. maxDataBytes := maxBytes -
  293. MaxAminoOverheadForBlock -
  294. MaxHeaderBytes -
  295. int64(valsCount)*MaxVoteBytes -
  296. maxEvidenceBytes
  297. if maxDataBytes < 0 {
  298. panic(fmt.Sprintf(
  299. "Negative MaxDataBytesUnknownEvidence. BlockSize.MaxBytes=%d is too small to accommodate header&lastCommit&evidence=%d",
  300. maxBytes,
  301. -(maxDataBytes - maxBytes),
  302. ))
  303. }
  304. return maxDataBytes
  305. }
  306. //-----------------------------------------------------------------------------
  307. // Header defines the structure of a Tendermint block header.
  308. // NOTE: changes to the Header should be duplicated in:
  309. // - header.Hash()
  310. // - abci.Header
  311. // - /docs/spec/blockchain/blockchain.md
  312. type Header struct {
  313. // basic block info
  314. Version version.Consensus `json:"version"`
  315. ChainID string `json:"chain_id"`
  316. Height int64 `json:"height"`
  317. Time time.Time `json:"time"`
  318. NumTxs int64 `json:"num_txs"`
  319. TotalTxs int64 `json:"total_txs"`
  320. // prev block info
  321. LastBlockID BlockID `json:"last_block_id"`
  322. // hashes of block data
  323. LastCommitHash cmn.HexBytes `json:"last_commit_hash"` // commit from validators from the last block
  324. DataHash cmn.HexBytes `json:"data_hash"` // transactions
  325. // hashes from the app output from the prev block
  326. ValidatorsHash cmn.HexBytes `json:"validators_hash"` // validators for the current block
  327. NextValidatorsHash cmn.HexBytes `json:"next_validators_hash"` // validators for the next block
  328. ConsensusHash cmn.HexBytes `json:"consensus_hash"` // consensus params for current block
  329. AppHash cmn.HexBytes `json:"app_hash"` // state after txs from the previous block
  330. LastResultsHash cmn.HexBytes `json:"last_results_hash"` // root hash of all results from the txs from the previous block
  331. // consensus info
  332. EvidenceHash cmn.HexBytes `json:"evidence_hash"` // evidence included in the block
  333. ProposerAddress Address `json:"proposer_address"` // original proposer of the block
  334. }
  335. // Populate the Header with state-derived data.
  336. // Call this after MakeBlock to complete the Header.
  337. func (h *Header) Populate(
  338. version version.Consensus, chainID string,
  339. timestamp time.Time, lastBlockID BlockID, totalTxs int64,
  340. valHash, nextValHash []byte,
  341. consensusHash, appHash, lastResultsHash []byte,
  342. proposerAddress Address,
  343. ) {
  344. h.Version = version
  345. h.ChainID = chainID
  346. h.Time = timestamp
  347. h.LastBlockID = lastBlockID
  348. h.TotalTxs = totalTxs
  349. h.ValidatorsHash = valHash
  350. h.NextValidatorsHash = nextValHash
  351. h.ConsensusHash = consensusHash
  352. h.AppHash = appHash
  353. h.LastResultsHash = lastResultsHash
  354. h.ProposerAddress = proposerAddress
  355. }
  356. // Hash returns the hash of the header.
  357. // It computes a Merkle tree from the header fields
  358. // ordered as they appear in the Header.
  359. // Returns nil if ValidatorHash is missing,
  360. // since a Header is not valid unless there is
  361. // a ValidatorsHash (corresponding to the validator set).
  362. func (h *Header) Hash() cmn.HexBytes {
  363. if h == nil || len(h.ValidatorsHash) == 0 {
  364. return nil
  365. }
  366. return merkle.SimpleHashFromByteSlices([][]byte{
  367. cdcEncode(h.Version),
  368. cdcEncode(h.ChainID),
  369. cdcEncode(h.Height),
  370. cdcEncode(h.Time),
  371. cdcEncode(h.NumTxs),
  372. cdcEncode(h.TotalTxs),
  373. cdcEncode(h.LastBlockID),
  374. cdcEncode(h.LastCommitHash),
  375. cdcEncode(h.DataHash),
  376. cdcEncode(h.ValidatorsHash),
  377. cdcEncode(h.NextValidatorsHash),
  378. cdcEncode(h.ConsensusHash),
  379. cdcEncode(h.AppHash),
  380. cdcEncode(h.LastResultsHash),
  381. cdcEncode(h.EvidenceHash),
  382. cdcEncode(h.ProposerAddress),
  383. })
  384. }
  385. // StringIndented returns a string representation of the header
  386. func (h *Header) StringIndented(indent string) string {
  387. if h == nil {
  388. return "nil-Header"
  389. }
  390. return fmt.Sprintf(`Header{
  391. %s Version: %v
  392. %s ChainID: %v
  393. %s Height: %v
  394. %s Time: %v
  395. %s NumTxs: %v
  396. %s TotalTxs: %v
  397. %s LastBlockID: %v
  398. %s LastCommit: %v
  399. %s Data: %v
  400. %s Validators: %v
  401. %s NextValidators: %v
  402. %s App: %v
  403. %s Consensus: %v
  404. %s Results: %v
  405. %s Evidence: %v
  406. %s Proposer: %v
  407. %s}#%v`,
  408. indent, h.Version,
  409. indent, h.ChainID,
  410. indent, h.Height,
  411. indent, h.Time,
  412. indent, h.NumTxs,
  413. indent, h.TotalTxs,
  414. indent, h.LastBlockID,
  415. indent, h.LastCommitHash,
  416. indent, h.DataHash,
  417. indent, h.ValidatorsHash,
  418. indent, h.NextValidatorsHash,
  419. indent, h.AppHash,
  420. indent, h.ConsensusHash,
  421. indent, h.LastResultsHash,
  422. indent, h.EvidenceHash,
  423. indent, h.ProposerAddress,
  424. indent, h.Hash())
  425. }
  426. //-------------------------------------
  427. // CommitSig is a vote included in a Commit.
  428. // For now, it is identical to a vote,
  429. // but in the future it will contain fewer fields
  430. // to eliminate the redundancy in commits.
  431. // See https://github.com/tendermint/tendermint/issues/1648.
  432. type CommitSig Vote
  433. // String returns the underlying Vote.String()
  434. func (cs *CommitSig) String() string {
  435. return cs.toVote().String()
  436. }
  437. // toVote converts the CommitSig to a vote.
  438. // Once CommitSig has fewer fields than vote,
  439. // converting to a Vote will require more information.
  440. func (cs *CommitSig) toVote() *Vote {
  441. if cs == nil {
  442. return nil
  443. }
  444. v := Vote(*cs)
  445. return &v
  446. }
  447. // Commit contains the evidence that a block was committed by a set of validators.
  448. // NOTE: Commit is empty for height 1, but never nil.
  449. type Commit struct {
  450. // NOTE: The Precommits are in order of address to preserve the bonded ValidatorSet order.
  451. // Any peer with a block can gossip precommits by index with a peer without recalculating the
  452. // active ValidatorSet.
  453. BlockID BlockID `json:"block_id"`
  454. Precommits []*CommitSig `json:"precommits"`
  455. // Volatile
  456. height int64
  457. round int
  458. hash cmn.HexBytes
  459. bitArray *cmn.BitArray
  460. }
  461. // VoteSignBytes constructs the SignBytes for the given CommitSig.
  462. // The only unique part of the SignBytes is the Timestamp - all other fields
  463. // signed over are otherwise the same for all validators.
  464. func (commit *Commit) VoteSignBytes(chainID string, cs *CommitSig) []byte {
  465. return cs.toVote().SignBytes(chainID)
  466. }
  467. // memoizeHeightRound memoizes the height and round of the commit using
  468. // the first non-nil vote.
  469. func (commit *Commit) memoizeHeightRound() {
  470. if len(commit.Precommits) == 0 {
  471. return
  472. }
  473. if commit.height > 0 {
  474. return
  475. }
  476. for _, precommit := range commit.Precommits {
  477. if precommit != nil {
  478. commit.height = precommit.Height
  479. commit.round = precommit.Round
  480. return
  481. }
  482. }
  483. }
  484. // ToVote converts a CommitSig to a Vote.
  485. // If the CommitSig is nil, the Vote will be nil.
  486. // When CommitSig is reduced to contain fewer fields,
  487. // this will need access to the ValidatorSet to properly
  488. // reconstruct the vote.
  489. func (commit *Commit) ToVote(cs *CommitSig) *Vote {
  490. return cs.toVote()
  491. }
  492. // Height returns the height of the commit
  493. func (commit *Commit) Height() int64 {
  494. if len(commit.Precommits) == 0 {
  495. return 0
  496. }
  497. commit.memoizeHeightRound()
  498. return commit.height
  499. }
  500. // Round returns the round of the commit
  501. func (commit *Commit) Round() int {
  502. if len(commit.Precommits) == 0 {
  503. return 0
  504. }
  505. commit.memoizeHeightRound()
  506. return commit.round
  507. }
  508. // Type returns the vote type of the commit, which is always VoteTypePrecommit
  509. func (commit *Commit) Type() byte {
  510. return byte(PrecommitType)
  511. }
  512. // Size returns the number of votes in the commit
  513. func (commit *Commit) Size() int {
  514. if commit == nil {
  515. return 0
  516. }
  517. return len(commit.Precommits)
  518. }
  519. // BitArray returns a BitArray of which validators voted in this commit
  520. func (commit *Commit) BitArray() *cmn.BitArray {
  521. if commit.bitArray == nil {
  522. commit.bitArray = cmn.NewBitArray(len(commit.Precommits))
  523. for i, precommit := range commit.Precommits {
  524. // TODO: need to check the BlockID otherwise we could be counting conflicts,
  525. // not just the one with +2/3 !
  526. commit.bitArray.SetIndex(i, precommit != nil)
  527. }
  528. }
  529. return commit.bitArray
  530. }
  531. // GetByIndex returns the vote corresponding to a given validator index.
  532. // Implements VoteSetReader.
  533. func (commit *Commit) GetByIndex(index int) *Vote {
  534. return commit.Precommits[index].toVote()
  535. }
  536. // IsCommit returns true if there is at least one vote.
  537. func (commit *Commit) IsCommit() bool {
  538. return len(commit.Precommits) != 0
  539. }
  540. // ValidateBasic performs basic validation that doesn't involve state data.
  541. // Does not actually check the cryptographic signatures.
  542. func (commit *Commit) ValidateBasic() error {
  543. if commit.BlockID.IsZero() {
  544. return errors.New("Commit cannot be for nil block")
  545. }
  546. if len(commit.Precommits) == 0 {
  547. return errors.New("No precommits in commit")
  548. }
  549. height, round := commit.Height(), commit.Round()
  550. // Validate the precommits.
  551. for _, precommit := range commit.Precommits {
  552. // It's OK for precommits to be missing.
  553. if precommit == nil {
  554. continue
  555. }
  556. // Ensure that all votes are precommits.
  557. if precommit.Type != PrecommitType {
  558. return fmt.Errorf("Invalid commit vote. Expected precommit, got %v",
  559. precommit.Type)
  560. }
  561. // Ensure that all heights are the same.
  562. if precommit.Height != height {
  563. return fmt.Errorf("Invalid commit precommit height. Expected %v, got %v",
  564. height, precommit.Height)
  565. }
  566. // Ensure that all rounds are the same.
  567. if precommit.Round != round {
  568. return fmt.Errorf("Invalid commit precommit round. Expected %v, got %v",
  569. round, precommit.Round)
  570. }
  571. }
  572. return nil
  573. }
  574. // Hash returns the hash of the commit
  575. func (commit *Commit) Hash() cmn.HexBytes {
  576. if commit == nil {
  577. return nil
  578. }
  579. if commit.hash == nil {
  580. bs := make([][]byte, len(commit.Precommits))
  581. for i, precommit := range commit.Precommits {
  582. bs[i] = cdcEncode(precommit)
  583. }
  584. commit.hash = merkle.SimpleHashFromByteSlices(bs)
  585. }
  586. return commit.hash
  587. }
  588. // StringIndented returns a string representation of the commit
  589. func (commit *Commit) StringIndented(indent string) string {
  590. if commit == nil {
  591. return "nil-Commit"
  592. }
  593. precommitStrings := make([]string, len(commit.Precommits))
  594. for i, precommit := range commit.Precommits {
  595. precommitStrings[i] = precommit.String()
  596. }
  597. return fmt.Sprintf(`Commit{
  598. %s BlockID: %v
  599. %s Precommits:
  600. %s %v
  601. %s}#%v`,
  602. indent, commit.BlockID,
  603. indent,
  604. indent, strings.Join(precommitStrings, "\n"+indent+" "),
  605. indent, commit.hash)
  606. }
  607. //-----------------------------------------------------------------------------
  608. // SignedHeader is a header along with the commits that prove it.
  609. // It is the basis of the lite client.
  610. type SignedHeader struct {
  611. *Header `json:"header"`
  612. Commit *Commit `json:"commit"`
  613. }
  614. // ValidateBasic does basic consistency checks and makes sure the header
  615. // and commit are consistent.
  616. //
  617. // NOTE: This does not actually check the cryptographic signatures. Make
  618. // sure to use a Verifier to validate the signatures actually provide a
  619. // significantly strong proof for this header's validity.
  620. func (sh SignedHeader) ValidateBasic(chainID string) error {
  621. // Make sure the header is consistent with the commit.
  622. if sh.Header == nil {
  623. return errors.New("SignedHeader missing header.")
  624. }
  625. if sh.Commit == nil {
  626. return errors.New("SignedHeader missing commit (precommit votes).")
  627. }
  628. // Check ChainID.
  629. if sh.ChainID != chainID {
  630. return fmt.Errorf("Header belongs to another chain '%s' not '%s'",
  631. sh.ChainID, chainID)
  632. }
  633. // Check Height.
  634. if sh.Commit.Height() != sh.Height {
  635. return fmt.Errorf("SignedHeader header and commit height mismatch: %v vs %v",
  636. sh.Height, sh.Commit.Height())
  637. }
  638. // Check Hash.
  639. hhash := sh.Hash()
  640. chash := sh.Commit.BlockID.Hash
  641. if !bytes.Equal(hhash, chash) {
  642. return fmt.Errorf("SignedHeader commit signs block %X, header is block %X",
  643. chash, hhash)
  644. }
  645. // ValidateBasic on the Commit.
  646. err := sh.Commit.ValidateBasic()
  647. if err != nil {
  648. return cmn.ErrorWrap(err, "commit.ValidateBasic failed during SignedHeader.ValidateBasic")
  649. }
  650. return nil
  651. }
  652. func (sh SignedHeader) String() string {
  653. return sh.StringIndented("")
  654. }
  655. // StringIndented returns a string representation of the SignedHeader.
  656. func (sh SignedHeader) StringIndented(indent string) string {
  657. return fmt.Sprintf(`SignedHeader{
  658. %s %v
  659. %s %v
  660. %s}`,
  661. indent, sh.Header.StringIndented(indent+" "),
  662. indent, sh.Commit.StringIndented(indent+" "),
  663. indent)
  664. }
  665. //-----------------------------------------------------------------------------
  666. // Data contains the set of transactions included in the block
  667. type Data struct {
  668. // Txs that will be applied by state @ block.Height+1.
  669. // NOTE: not all txs here are valid. We're just agreeing on the order first.
  670. // This means that block.AppHash does not include these txs.
  671. Txs Txs `json:"txs"`
  672. // Volatile
  673. hash cmn.HexBytes
  674. }
  675. // Hash returns the hash of the data
  676. func (data *Data) Hash() cmn.HexBytes {
  677. if data == nil {
  678. return (Txs{}).Hash()
  679. }
  680. if data.hash == nil {
  681. data.hash = data.Txs.Hash() // NOTE: leaves of merkle tree are TxIDs
  682. }
  683. return data.hash
  684. }
  685. // StringIndented returns a string representation of the transactions
  686. func (data *Data) StringIndented(indent string) string {
  687. if data == nil {
  688. return "nil-Data"
  689. }
  690. txStrings := make([]string, cmn.MinInt(len(data.Txs), 21))
  691. for i, tx := range data.Txs {
  692. if i == 20 {
  693. txStrings[i] = fmt.Sprintf("... (%v total)", len(data.Txs))
  694. break
  695. }
  696. txStrings[i] = fmt.Sprintf("%X (%d bytes)", tx.Hash(), len(tx))
  697. }
  698. return fmt.Sprintf(`Data{
  699. %s %v
  700. %s}#%v`,
  701. indent, strings.Join(txStrings, "\n"+indent+" "),
  702. indent, data.hash)
  703. }
  704. //-----------------------------------------------------------------------------
  705. // EvidenceData contains any evidence of malicious wrong-doing by validators
  706. type EvidenceData struct {
  707. Evidence EvidenceList `json:"evidence"`
  708. // Volatile
  709. hash cmn.HexBytes
  710. }
  711. // Hash returns the hash of the data.
  712. func (data *EvidenceData) Hash() cmn.HexBytes {
  713. if data.hash == nil {
  714. data.hash = data.Evidence.Hash()
  715. }
  716. return data.hash
  717. }
  718. // StringIndented returns a string representation of the evidence.
  719. func (data *EvidenceData) StringIndented(indent string) string {
  720. if data == nil {
  721. return "nil-Evidence"
  722. }
  723. evStrings := make([]string, cmn.MinInt(len(data.Evidence), 21))
  724. for i, ev := range data.Evidence {
  725. if i == 20 {
  726. evStrings[i] = fmt.Sprintf("... (%v total)", len(data.Evidence))
  727. break
  728. }
  729. evStrings[i] = fmt.Sprintf("Evidence:%v", ev)
  730. }
  731. return fmt.Sprintf(`EvidenceData{
  732. %s %v
  733. %s}#%v`,
  734. indent, strings.Join(evStrings, "\n"+indent+" "),
  735. indent, data.hash)
  736. }
  737. //--------------------------------------------------------------------------------
  738. // BlockID defines the unique ID of a block as its Hash and its PartSetHeader
  739. type BlockID struct {
  740. Hash cmn.HexBytes `json:"hash"`
  741. PartsHeader PartSetHeader `json:"parts"`
  742. }
  743. // Equals returns true if the BlockID matches the given BlockID
  744. func (blockID BlockID) Equals(other BlockID) bool {
  745. return bytes.Equal(blockID.Hash, other.Hash) &&
  746. blockID.PartsHeader.Equals(other.PartsHeader)
  747. }
  748. // Key returns a machine-readable string representation of the BlockID
  749. func (blockID BlockID) Key() string {
  750. bz, err := cdc.MarshalBinaryBare(blockID.PartsHeader)
  751. if err != nil {
  752. panic(err)
  753. }
  754. return string(blockID.Hash) + string(bz)
  755. }
  756. // ValidateBasic performs basic validation.
  757. func (blockID BlockID) ValidateBasic() error {
  758. // Hash can be empty in case of POLBlockID in Proposal.
  759. if err := ValidateHash(blockID.Hash); err != nil {
  760. return fmt.Errorf("Wrong Hash")
  761. }
  762. if err := blockID.PartsHeader.ValidateBasic(); err != nil {
  763. return fmt.Errorf("Wrong PartsHeader: %v", err)
  764. }
  765. return nil
  766. }
  767. // IsZero returns true if this is the BlockID of a nil block.
  768. func (blockID BlockID) IsZero() bool {
  769. return len(blockID.Hash) == 0 &&
  770. blockID.PartsHeader.IsZero()
  771. }
  772. // IsComplete returns true if this is a valid BlockID of a non-nil block.
  773. func (blockID BlockID) IsComplete() bool {
  774. return len(blockID.Hash) == tmhash.Size &&
  775. blockID.PartsHeader.Total > 0 &&
  776. len(blockID.PartsHeader.Hash) == tmhash.Size
  777. }
  778. // String returns a human readable string representation of the BlockID
  779. func (blockID BlockID) String() string {
  780. return fmt.Sprintf(`%v:%v`, blockID.Hash, blockID.PartsHeader)
  781. }