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.

1336 lines
35 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
10 years ago
10 years ago
6 years ago
6 years ago
6 years ago
6 years ago
10 years ago
10 years ago
  1. package types
  2. import (
  3. "bytes"
  4. "errors"
  5. "fmt"
  6. "strings"
  7. "sync"
  8. "time"
  9. "github.com/gogo/protobuf/proto"
  10. gogotypes "github.com/gogo/protobuf/types"
  11. "github.com/tendermint/tendermint/crypto"
  12. "github.com/tendermint/tendermint/crypto/merkle"
  13. "github.com/tendermint/tendermint/crypto/tmhash"
  14. "github.com/tendermint/tendermint/libs/bits"
  15. tmbytes "github.com/tendermint/tendermint/libs/bytes"
  16. tmmath "github.com/tendermint/tendermint/libs/math"
  17. tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
  18. tmversion "github.com/tendermint/tendermint/proto/tendermint/version"
  19. )
  20. const (
  21. // MaxHeaderBytes is a maximum header size.
  22. MaxHeaderBytes int64 = 626
  23. // MaxOverheadForBlock - maximum overhead to encode a block (up to
  24. // MaxBlockSizeBytes in size) not including it's parts except Data.
  25. // This means it also excludes the overhead for individual transactions.
  26. //
  27. // Uvarint length of MaxBlockSizeBytes: 4 bytes
  28. // 2 fields (2 embedded): 2 bytes
  29. // Uvarint length of Data.Txs: 4 bytes
  30. // Data.Txs field: 1 byte
  31. MaxOverheadForBlock int64 = 11
  32. )
  33. // Block defines the atomic unit of a Tendermint blockchain.
  34. type Block struct {
  35. mtx sync.Mutex
  36. Header `json:"header"`
  37. Data `json:"data"`
  38. Evidence EvidenceData `json:"evidence"`
  39. LastCommit *Commit `json:"last_commit"`
  40. }
  41. // ValidateBasic performs basic validation that doesn't involve state data.
  42. // It checks the internal consistency of the block.
  43. // Further validation is done using state#ValidateBlock.
  44. func (b *Block) ValidateBasic() error {
  45. if b == nil {
  46. return errors.New("nil block")
  47. }
  48. b.mtx.Lock()
  49. defer b.mtx.Unlock()
  50. if err := b.Header.ValidateBasic(); err != nil {
  51. return fmt.Errorf("invalid header: %w", err)
  52. }
  53. // Validate the last commit and its hash.
  54. if b.LastCommit == nil {
  55. return errors.New("nil LastCommit")
  56. }
  57. if b.Header.Height > 1 {
  58. if err := b.LastCommit.ValidateBasic(); err != nil {
  59. return fmt.Errorf("wrong LastCommit: %v", err)
  60. }
  61. }
  62. if !bytes.Equal(b.LastCommitHash, b.LastCommit.Hash()) {
  63. return fmt.Errorf("wrong Header.LastCommitHash. Expected %v, got %v",
  64. b.LastCommit.Hash(),
  65. b.LastCommitHash,
  66. )
  67. }
  68. // NOTE: b.Data.Txs may be nil, but b.Data.Hash() still works fine.
  69. if !bytes.Equal(b.DataHash, b.Data.Hash()) {
  70. return fmt.Errorf(
  71. "wrong Header.DataHash. Expected %v, got %v",
  72. b.Data.Hash(),
  73. b.DataHash,
  74. )
  75. }
  76. // NOTE: b.Evidence.Evidence may be nil, but we're just looping.
  77. for i, ev := range b.Evidence.Evidence {
  78. switch ev.(type) {
  79. case *ConflictingHeadersEvidence:
  80. // ConflictingHeadersEvidence must be broken up in pieces and never
  81. // committed as a single piece.
  82. return fmt.Errorf("found ConflictingHeadersEvidence (#%d)", i)
  83. case *PotentialAmnesiaEvidence:
  84. // PotentialAmnesiaEvidence does not contribute to anything on its own, so
  85. // reject it as well.
  86. return fmt.Errorf("found PotentialAmnesiaEvidence (#%d)", i)
  87. }
  88. if err := ev.ValidateBasic(); err != nil {
  89. return fmt.Errorf("invalid evidence (#%d): %v", i, err)
  90. }
  91. }
  92. if !bytes.Equal(b.EvidenceHash, b.Evidence.Hash()) {
  93. return fmt.Errorf("wrong Header.EvidenceHash. Expected %v, got %v",
  94. b.EvidenceHash,
  95. b.Evidence.Hash(),
  96. )
  97. }
  98. return nil
  99. }
  100. // fillHeader fills in any remaining header fields that are a function of the block data
  101. func (b *Block) fillHeader() {
  102. if b.LastCommitHash == nil {
  103. b.LastCommitHash = b.LastCommit.Hash()
  104. }
  105. if b.DataHash == nil {
  106. b.DataHash = b.Data.Hash()
  107. }
  108. if b.EvidenceHash == nil {
  109. b.EvidenceHash = b.Evidence.Hash()
  110. }
  111. }
  112. // Hash computes and returns the block hash.
  113. // If the block is incomplete, block hash is nil for safety.
  114. func (b *Block) Hash() tmbytes.HexBytes {
  115. if b == nil {
  116. return nil
  117. }
  118. b.mtx.Lock()
  119. defer b.mtx.Unlock()
  120. if b.LastCommit == nil {
  121. return nil
  122. }
  123. b.fillHeader()
  124. return b.Header.Hash()
  125. }
  126. // MakePartSet returns a PartSet containing parts of a serialized block.
  127. // This is the form in which the block is gossipped to peers.
  128. // CONTRACT: partSize is greater than zero.
  129. func (b *Block) MakePartSet(partSize uint32) *PartSet {
  130. if b == nil {
  131. return nil
  132. }
  133. b.mtx.Lock()
  134. defer b.mtx.Unlock()
  135. pbb, err := b.ToProto()
  136. if err != nil {
  137. panic(err)
  138. }
  139. bz, err := proto.Marshal(pbb)
  140. if err != nil {
  141. panic(err)
  142. }
  143. return NewPartSetFromData(bz, partSize)
  144. }
  145. // HashesTo is a convenience function that checks if a block hashes to the given argument.
  146. // Returns false if the block is nil or the hash is empty.
  147. func (b *Block) HashesTo(hash []byte) bool {
  148. if len(hash) == 0 {
  149. return false
  150. }
  151. if b == nil {
  152. return false
  153. }
  154. return bytes.Equal(b.Hash(), hash)
  155. }
  156. // Size returns size of the block in bytes.
  157. func (b *Block) Size() int {
  158. pbb, err := b.ToProto()
  159. if err != nil {
  160. return 0
  161. }
  162. return pbb.Size()
  163. }
  164. // String returns a string representation of the block
  165. func (b *Block) String() string {
  166. return b.StringIndented("")
  167. }
  168. // StringIndented returns a string representation of the block
  169. func (b *Block) StringIndented(indent string) string {
  170. if b == nil {
  171. return "nil-Block"
  172. }
  173. return fmt.Sprintf(`Block{
  174. %s %v
  175. %s %v
  176. %s %v
  177. %s %v
  178. %s}#%v`,
  179. indent, b.Header.StringIndented(indent+" "),
  180. indent, b.Data.StringIndented(indent+" "),
  181. indent, b.Evidence.StringIndented(indent+" "),
  182. indent, b.LastCommit.StringIndented(indent+" "),
  183. indent, b.Hash())
  184. }
  185. // StringShort returns a shortened string representation of the block
  186. func (b *Block) StringShort() string {
  187. if b == nil {
  188. return "nil-Block"
  189. }
  190. return fmt.Sprintf("Block#%v", b.Hash())
  191. }
  192. // ToProto converts Block to protobuf
  193. func (b *Block) ToProto() (*tmproto.Block, error) {
  194. if b == nil {
  195. return nil, errors.New("nil Block")
  196. }
  197. pb := new(tmproto.Block)
  198. pb.Header = *b.Header.ToProto()
  199. pb.LastCommit = b.LastCommit.ToProto()
  200. pb.Data = b.Data.ToProto()
  201. protoEvidence, err := b.Evidence.ToProto()
  202. if err != nil {
  203. return nil, err
  204. }
  205. pb.Evidence = *protoEvidence
  206. return pb, nil
  207. }
  208. // FromProto sets a protobuf Block to the given pointer.
  209. // It returns an error if the block is invalid.
  210. func BlockFromProto(bp *tmproto.Block) (*Block, error) {
  211. if bp == nil {
  212. return nil, errors.New("nil block")
  213. }
  214. b := new(Block)
  215. h, err := HeaderFromProto(&bp.Header)
  216. if err != nil {
  217. return nil, err
  218. }
  219. b.Header = h
  220. data, err := DataFromProto(&bp.Data)
  221. if err != nil {
  222. return nil, err
  223. }
  224. b.Data = data
  225. b.Evidence.FromProto(&bp.Evidence)
  226. if bp.LastCommit != nil {
  227. lc, err := CommitFromProto(bp.LastCommit)
  228. if err != nil {
  229. return nil, err
  230. }
  231. b.LastCommit = lc
  232. }
  233. return b, b.ValidateBasic()
  234. }
  235. //-----------------------------------------------------------------------------
  236. // MaxDataBytes returns the maximum size of block's data.
  237. //
  238. // XXX: Panics on negative result.
  239. func MaxDataBytes(maxBytes int64, valsCount, evidenceCount int) int64 {
  240. maxDataBytes := maxBytes -
  241. MaxOverheadForBlock -
  242. MaxHeaderBytes -
  243. int64(valsCount)*MaxVoteBytes -
  244. int64(evidenceCount)*MaxEvidenceBytes
  245. if maxDataBytes < 0 {
  246. panic(fmt.Sprintf(
  247. "Negative MaxDataBytes. Block.MaxBytes=%d is too small to accommodate header&lastCommit&evidence=%d",
  248. maxBytes,
  249. -(maxDataBytes - maxBytes),
  250. ))
  251. }
  252. return maxDataBytes
  253. }
  254. // MaxDataBytesUnknownEvidence returns the maximum size of block's data when
  255. // evidence count is unknown. MaxEvidencePerBlock will be used for the size
  256. // of evidence.
  257. //
  258. // XXX: Panics on negative result.
  259. func MaxDataBytesUnknownEvidence(maxBytes int64, valsCount int, maxNumEvidence uint32) int64 {
  260. maxEvidenceBytes := int64(maxNumEvidence) * MaxEvidenceBytes
  261. maxDataBytes := maxBytes -
  262. MaxOverheadForBlock -
  263. MaxHeaderBytes -
  264. int64(valsCount)*MaxVoteBytes -
  265. maxEvidenceBytes
  266. if maxDataBytes < 0 {
  267. panic(fmt.Sprintf(
  268. "Negative MaxDataBytesUnknownEvidence. Block.MaxBytes=%d is too small to accommodate header&lastCommit&evidence=%d",
  269. maxBytes,
  270. -(maxDataBytes - maxBytes),
  271. ))
  272. }
  273. return maxDataBytes
  274. }
  275. //-----------------------------------------------------------------------------
  276. // Header defines the structure of a Tendermint block header.
  277. // NOTE: changes to the Header should be duplicated in:
  278. // - header.Hash()
  279. // - abci.Header
  280. // - https://github.com/tendermint/spec/blob/master/spec/blockchain/blockchain.md
  281. type Header struct {
  282. // basic block info
  283. Version tmversion.Consensus `json:"version"`
  284. ChainID string `json:"chain_id"`
  285. Height int64 `json:"height"`
  286. Time time.Time `json:"time"`
  287. // prev block info
  288. LastBlockID BlockID `json:"last_block_id"`
  289. // hashes of block data
  290. LastCommitHash tmbytes.HexBytes `json:"last_commit_hash"` // commit from validators from the last block
  291. DataHash tmbytes.HexBytes `json:"data_hash"` // transactions
  292. // hashes from the app output from the prev block
  293. ValidatorsHash tmbytes.HexBytes `json:"validators_hash"` // validators for the current block
  294. NextValidatorsHash tmbytes.HexBytes `json:"next_validators_hash"` // validators for the next block
  295. ConsensusHash tmbytes.HexBytes `json:"consensus_hash"` // consensus params for current block
  296. AppHash tmbytes.HexBytes `json:"app_hash"` // state after txs from the previous block
  297. // root hash of all results from the txs from the previous block
  298. LastResultsHash tmbytes.HexBytes `json:"last_results_hash"`
  299. // consensus info
  300. EvidenceHash tmbytes.HexBytes `json:"evidence_hash"` // evidence included in the block
  301. ProposerAddress Address `json:"proposer_address"` // original proposer of the block
  302. }
  303. // Populate the Header with state-derived data.
  304. // Call this after MakeBlock to complete the Header.
  305. func (h *Header) Populate(
  306. version tmversion.Consensus, chainID string,
  307. timestamp time.Time, lastBlockID BlockID,
  308. valHash, nextValHash []byte,
  309. consensusHash, appHash, lastResultsHash []byte,
  310. proposerAddress Address,
  311. ) {
  312. h.Version = version
  313. h.ChainID = chainID
  314. h.Time = timestamp
  315. h.LastBlockID = lastBlockID
  316. h.ValidatorsHash = valHash
  317. h.NextValidatorsHash = nextValHash
  318. h.ConsensusHash = consensusHash
  319. h.AppHash = appHash
  320. h.LastResultsHash = lastResultsHash
  321. h.ProposerAddress = proposerAddress
  322. }
  323. // ValidateBasic performs stateless validation on a Header returning an error
  324. // if any validation fails.
  325. //
  326. // NOTE: Timestamp validation is subtle and handled elsewhere.
  327. func (h Header) ValidateBasic() error {
  328. if len(h.ChainID) > MaxChainIDLen {
  329. return fmt.Errorf("chainID is too long; got: %d, max: %d", len(h.ChainID), MaxChainIDLen)
  330. }
  331. if h.Height < 0 {
  332. return errors.New("negative Height")
  333. } else if h.Height == 0 {
  334. return errors.New("zero Height")
  335. }
  336. if err := h.LastBlockID.ValidateBasic(); err != nil {
  337. return fmt.Errorf("wrong LastBlockID: %w", err)
  338. }
  339. if err := ValidateHash(h.LastCommitHash); err != nil {
  340. return fmt.Errorf("wrong LastCommitHash: %v", err)
  341. }
  342. if err := ValidateHash(h.DataHash); err != nil {
  343. return fmt.Errorf("wrong DataHash: %v", err)
  344. }
  345. if err := ValidateHash(h.EvidenceHash); err != nil {
  346. return fmt.Errorf("wrong EvidenceHash: %v", err)
  347. }
  348. if len(h.ProposerAddress) != crypto.AddressSize {
  349. return fmt.Errorf(
  350. "invalid ProposerAddress length; got: %d, expected: %d",
  351. len(h.ProposerAddress), crypto.AddressSize,
  352. )
  353. }
  354. // Basic validation of hashes related to application data.
  355. // Will validate fully against state in state#ValidateBlock.
  356. if err := ValidateHash(h.ValidatorsHash); err != nil {
  357. return fmt.Errorf("wrong ValidatorsHash: %v", err)
  358. }
  359. if err := ValidateHash(h.NextValidatorsHash); err != nil {
  360. return fmt.Errorf("wrong NextValidatorsHash: %v", err)
  361. }
  362. if err := ValidateHash(h.ConsensusHash); err != nil {
  363. return fmt.Errorf("wrong ConsensusHash: %v", err)
  364. }
  365. // NOTE: AppHash is arbitrary length
  366. if err := ValidateHash(h.LastResultsHash); err != nil {
  367. return fmt.Errorf("wrong LastResultsHash: %v", err)
  368. }
  369. return nil
  370. }
  371. // Hash returns the hash of the header.
  372. // It computes a Merkle tree from the header fields
  373. // ordered as they appear in the Header.
  374. // Returns nil if ValidatorHash is missing,
  375. // since a Header is not valid unless there is
  376. // a ValidatorsHash (corresponding to the validator set).
  377. func (h *Header) Hash() tmbytes.HexBytes {
  378. if h == nil || len(h.ValidatorsHash) == 0 {
  379. return nil
  380. }
  381. hbz, err := h.Version.Marshal()
  382. if err != nil {
  383. return nil
  384. }
  385. pbt, err := gogotypes.StdTimeMarshal(h.Time)
  386. if err != nil {
  387. return nil
  388. }
  389. pbbi := h.LastBlockID.ToProto()
  390. bzbi, err := pbbi.Marshal()
  391. if err != nil {
  392. return nil
  393. }
  394. return merkle.HashFromByteSlices([][]byte{
  395. hbz,
  396. cdcEncode(h.ChainID),
  397. cdcEncode(h.Height),
  398. pbt,
  399. bzbi,
  400. cdcEncode(h.LastCommitHash),
  401. cdcEncode(h.DataHash),
  402. cdcEncode(h.ValidatorsHash),
  403. cdcEncode(h.NextValidatorsHash),
  404. cdcEncode(h.ConsensusHash),
  405. cdcEncode(h.AppHash),
  406. cdcEncode(h.LastResultsHash),
  407. cdcEncode(h.EvidenceHash),
  408. cdcEncode(h.ProposerAddress),
  409. })
  410. }
  411. // StringIndented returns a string representation of the header
  412. func (h *Header) StringIndented(indent string) string {
  413. if h == nil {
  414. return "nil-Header"
  415. }
  416. return fmt.Sprintf(`Header{
  417. %s Version: %v
  418. %s ChainID: %v
  419. %s Height: %v
  420. %s Time: %v
  421. %s LastBlockID: %v
  422. %s LastCommit: %v
  423. %s Data: %v
  424. %s Validators: %v
  425. %s NextValidators: %v
  426. %s App: %v
  427. %s Consensus: %v
  428. %s Results: %v
  429. %s Evidence: %v
  430. %s Proposer: %v
  431. %s}#%v`,
  432. indent, h.Version,
  433. indent, h.ChainID,
  434. indent, h.Height,
  435. indent, h.Time,
  436. indent, h.LastBlockID,
  437. indent, h.LastCommitHash,
  438. indent, h.DataHash,
  439. indent, h.ValidatorsHash,
  440. indent, h.NextValidatorsHash,
  441. indent, h.AppHash,
  442. indent, h.ConsensusHash,
  443. indent, h.LastResultsHash,
  444. indent, h.EvidenceHash,
  445. indent, h.ProposerAddress,
  446. indent, h.Hash())
  447. }
  448. // ToProto converts Header to protobuf
  449. func (h *Header) ToProto() *tmproto.Header {
  450. if h == nil {
  451. return nil
  452. }
  453. return &tmproto.Header{
  454. Version: h.Version,
  455. ChainID: h.ChainID,
  456. Height: h.Height,
  457. Time: h.Time,
  458. LastBlockId: h.LastBlockID.ToProto(),
  459. ValidatorsHash: h.ValidatorsHash,
  460. NextValidatorsHash: h.NextValidatorsHash,
  461. ConsensusHash: h.ConsensusHash,
  462. AppHash: h.AppHash,
  463. DataHash: h.DataHash,
  464. EvidenceHash: h.EvidenceHash,
  465. LastResultsHash: h.LastResultsHash,
  466. LastCommitHash: h.LastCommitHash,
  467. ProposerAddress: h.ProposerAddress,
  468. }
  469. }
  470. // FromProto sets a protobuf Header to the given pointer.
  471. // It returns an error if the header is invalid.
  472. func HeaderFromProto(ph *tmproto.Header) (Header, error) {
  473. if ph == nil {
  474. return Header{}, errors.New("nil Header")
  475. }
  476. h := new(Header)
  477. bi, err := BlockIDFromProto(&ph.LastBlockId)
  478. if err != nil {
  479. return Header{}, err
  480. }
  481. h.Version = ph.Version
  482. h.ChainID = ph.ChainID
  483. h.Height = ph.Height
  484. h.Time = ph.Time
  485. h.Height = ph.Height
  486. h.LastBlockID = *bi
  487. h.ValidatorsHash = ph.ValidatorsHash
  488. h.NextValidatorsHash = ph.NextValidatorsHash
  489. h.ConsensusHash = ph.ConsensusHash
  490. h.AppHash = ph.AppHash
  491. h.DataHash = ph.DataHash
  492. h.EvidenceHash = ph.EvidenceHash
  493. h.LastResultsHash = ph.LastResultsHash
  494. h.LastCommitHash = ph.LastCommitHash
  495. h.ProposerAddress = ph.ProposerAddress
  496. return *h, h.ValidateBasic()
  497. }
  498. //-------------------------------------
  499. // BlockIDFlag indicates which BlockID the signature is for.
  500. type BlockIDFlag byte
  501. const (
  502. // BlockIDFlagAbsent - no vote was received from a validator.
  503. BlockIDFlagAbsent BlockIDFlag = iota + 1
  504. // BlockIDFlagCommit - voted for the Commit.BlockID.
  505. BlockIDFlagCommit
  506. // BlockIDFlagNil - voted for nil.
  507. BlockIDFlagNil
  508. )
  509. // CommitSig is a part of the Vote included in a Commit.
  510. type CommitSig struct {
  511. BlockIDFlag BlockIDFlag `json:"block_id_flag"`
  512. ValidatorAddress Address `json:"validator_address"`
  513. Timestamp time.Time `json:"timestamp"`
  514. Signature []byte `json:"signature"`
  515. }
  516. // NewCommitSigForBlock returns new CommitSig with BlockIDFlagCommit.
  517. func NewCommitSigForBlock(signature []byte, valAddr Address, ts time.Time) CommitSig {
  518. return CommitSig{
  519. BlockIDFlag: BlockIDFlagCommit,
  520. ValidatorAddress: valAddr,
  521. Timestamp: ts,
  522. Signature: signature,
  523. }
  524. }
  525. // ForBlock returns true if CommitSig is for the block.
  526. func (cs CommitSig) ForBlock() bool {
  527. return cs.BlockIDFlag == BlockIDFlagCommit
  528. }
  529. // NewCommitSigAbsent returns new CommitSig with BlockIDFlagAbsent. Other
  530. // fields are all empty.
  531. func NewCommitSigAbsent() CommitSig {
  532. return CommitSig{
  533. BlockIDFlag: BlockIDFlagAbsent,
  534. }
  535. }
  536. // Absent returns true if CommitSig is absent.
  537. func (cs CommitSig) Absent() bool {
  538. return cs.BlockIDFlag == BlockIDFlagAbsent
  539. }
  540. func (cs CommitSig) String() string {
  541. return fmt.Sprintf("CommitSig{%X by %X on %v @ %s}",
  542. tmbytes.Fingerprint(cs.Signature),
  543. tmbytes.Fingerprint(cs.ValidatorAddress),
  544. cs.BlockIDFlag,
  545. CanonicalTime(cs.Timestamp))
  546. }
  547. // BlockID returns the Commit's BlockID if CommitSig indicates signing,
  548. // otherwise - empty BlockID.
  549. func (cs CommitSig) BlockID(commitBlockID BlockID) BlockID {
  550. var blockID BlockID
  551. switch cs.BlockIDFlag {
  552. case BlockIDFlagAbsent:
  553. blockID = BlockID{}
  554. case BlockIDFlagCommit:
  555. blockID = commitBlockID
  556. case BlockIDFlagNil:
  557. blockID = BlockID{}
  558. default:
  559. panic(fmt.Sprintf("Unknown BlockIDFlag: %v", cs.BlockIDFlag))
  560. }
  561. return blockID
  562. }
  563. // ValidateBasic performs basic validation.
  564. func (cs CommitSig) ValidateBasic() error {
  565. switch cs.BlockIDFlag {
  566. case BlockIDFlagAbsent:
  567. case BlockIDFlagCommit:
  568. case BlockIDFlagNil:
  569. default:
  570. return fmt.Errorf("unknown BlockIDFlag: %v", cs.BlockIDFlag)
  571. }
  572. switch cs.BlockIDFlag {
  573. case BlockIDFlagAbsent:
  574. if len(cs.ValidatorAddress) != 0 {
  575. return errors.New("validator address is present")
  576. }
  577. if !cs.Timestamp.IsZero() {
  578. return errors.New("time is present")
  579. }
  580. if len(cs.Signature) != 0 {
  581. return errors.New("signature is present")
  582. }
  583. default:
  584. if len(cs.ValidatorAddress) != crypto.AddressSize {
  585. return fmt.Errorf("expected ValidatorAddress size to be %d bytes, got %d bytes",
  586. crypto.AddressSize,
  587. len(cs.ValidatorAddress),
  588. )
  589. }
  590. // NOTE: Timestamp validation is subtle and handled elsewhere.
  591. if len(cs.Signature) == 0 {
  592. return errors.New("signature is missing")
  593. }
  594. if len(cs.Signature) > MaxSignatureSize {
  595. return fmt.Errorf("signature is too big (max: %d)", MaxSignatureSize)
  596. }
  597. }
  598. return nil
  599. }
  600. // ToProto converts CommitSig to protobuf
  601. func (cs *CommitSig) ToProto() *tmproto.CommitSig {
  602. if cs == nil {
  603. return nil
  604. }
  605. return &tmproto.CommitSig{
  606. BlockIdFlag: tmproto.BlockIDFlag(cs.BlockIDFlag),
  607. ValidatorAddress: cs.ValidatorAddress,
  608. Timestamp: cs.Timestamp,
  609. Signature: cs.Signature,
  610. }
  611. }
  612. // FromProto sets a protobuf CommitSig to the given pointer.
  613. // It returns an error if the CommitSig is invalid.
  614. func (cs *CommitSig) FromProto(csp tmproto.CommitSig) error {
  615. cs.BlockIDFlag = BlockIDFlag(csp.BlockIdFlag)
  616. cs.ValidatorAddress = csp.ValidatorAddress
  617. cs.Timestamp = csp.Timestamp
  618. cs.Signature = csp.Signature
  619. return cs.ValidateBasic()
  620. }
  621. //-------------------------------------
  622. // Commit contains the evidence that a block was committed by a set of validators.
  623. // NOTE: Commit is empty for height 1, but never nil.
  624. type Commit struct {
  625. // NOTE: The signatures are in order of address to preserve the bonded
  626. // ValidatorSet order.
  627. // Any peer with a block can gossip signatures by index with a peer without
  628. // recalculating the active ValidatorSet.
  629. Height int64 `json:"height"`
  630. Round int32 `json:"round"`
  631. BlockID BlockID `json:"block_id"`
  632. Signatures []CommitSig `json:"signatures"`
  633. // Memoized in first call to corresponding method.
  634. // NOTE: can't memoize in constructor because constructor isn't used for
  635. // unmarshaling.
  636. hash tmbytes.HexBytes
  637. bitArray *bits.BitArray
  638. }
  639. // NewCommit returns a new Commit.
  640. func NewCommit(height int64, round int32, blockID BlockID, commitSigs []CommitSig) *Commit {
  641. return &Commit{
  642. Height: height,
  643. Round: round,
  644. BlockID: blockID,
  645. Signatures: commitSigs,
  646. }
  647. }
  648. // CommitToVoteSet constructs a VoteSet from the Commit and validator set.
  649. // Panics if signatures from the commit can't be added to the voteset.
  650. // Inverse of VoteSet.MakeCommit().
  651. func CommitToVoteSet(chainID string, commit *Commit, vals *ValidatorSet) *VoteSet {
  652. voteSet := NewVoteSet(chainID, commit.Height, commit.Round, tmproto.PrecommitType, vals)
  653. for idx, commitSig := range commit.Signatures {
  654. if commitSig.Absent() {
  655. continue // OK, some precommits can be missing.
  656. }
  657. added, err := voteSet.AddVote(commit.GetVote(int32(idx)))
  658. if !added || err != nil {
  659. panic(fmt.Sprintf("Failed to reconstruct LastCommit: %v", err))
  660. }
  661. }
  662. return voteSet
  663. }
  664. // GetVote converts the CommitSig for the given valIdx to a Vote.
  665. // Returns nil if the precommit at valIdx is nil.
  666. // Panics if valIdx >= commit.Size().
  667. func (commit *Commit) GetVote(valIdx int32) *Vote {
  668. commitSig := commit.Signatures[valIdx]
  669. return &Vote{
  670. Type: tmproto.PrecommitType,
  671. Height: commit.Height,
  672. Round: commit.Round,
  673. BlockID: commitSig.BlockID(commit.BlockID),
  674. Timestamp: commitSig.Timestamp,
  675. ValidatorAddress: commitSig.ValidatorAddress,
  676. ValidatorIndex: valIdx,
  677. Signature: commitSig.Signature,
  678. }
  679. }
  680. // VoteSignBytes returns the bytes of the Vote corresponding to valIdx for
  681. // signing.
  682. //
  683. // The only unique part is the Timestamp - all other fields signed over are
  684. // otherwise the same for all validators.
  685. //
  686. // Panics if valIdx >= commit.Size().
  687. //
  688. // See VoteSignBytes
  689. func (commit *Commit) VoteSignBytes(chainID string, valIdx int32) []byte {
  690. v := commit.GetVote(valIdx).ToProto()
  691. return VoteSignBytes(chainID, v)
  692. }
  693. // Type returns the vote type of the commit, which is always VoteTypePrecommit
  694. // Implements VoteSetReader.
  695. func (commit *Commit) Type() byte {
  696. return byte(tmproto.PrecommitType)
  697. }
  698. // GetHeight returns height of the commit.
  699. // Implements VoteSetReader.
  700. func (commit *Commit) GetHeight() int64 {
  701. return commit.Height
  702. }
  703. // GetRound returns height of the commit.
  704. // Implements VoteSetReader.
  705. func (commit *Commit) GetRound() int32 {
  706. return commit.Round
  707. }
  708. // Size returns the number of signatures in the commit.
  709. // Implements VoteSetReader.
  710. func (commit *Commit) Size() int {
  711. if commit == nil {
  712. return 0
  713. }
  714. return len(commit.Signatures)
  715. }
  716. // BitArray returns a BitArray of which validators voted for BlockID or nil in this commit.
  717. // Implements VoteSetReader.
  718. func (commit *Commit) BitArray() *bits.BitArray {
  719. if commit.bitArray == nil {
  720. commit.bitArray = bits.NewBitArray(len(commit.Signatures))
  721. for i, commitSig := range commit.Signatures {
  722. // TODO: need to check the BlockID otherwise we could be counting conflicts,
  723. // not just the one with +2/3 !
  724. commit.bitArray.SetIndex(i, !commitSig.Absent())
  725. }
  726. }
  727. return commit.bitArray
  728. }
  729. // GetByIndex returns the vote corresponding to a given validator index.
  730. // Panics if `index >= commit.Size()`.
  731. // Implements VoteSetReader.
  732. func (commit *Commit) GetByIndex(valIdx int32) *Vote {
  733. return commit.GetVote(valIdx)
  734. }
  735. // IsCommit returns true if there is at least one signature.
  736. // Implements VoteSetReader.
  737. func (commit *Commit) IsCommit() bool {
  738. return len(commit.Signatures) != 0
  739. }
  740. // ValidateBasic performs basic validation that doesn't involve state data.
  741. // Does not actually check the cryptographic signatures.
  742. func (commit *Commit) ValidateBasic() error {
  743. if commit.Height < 0 {
  744. return errors.New("negative Height")
  745. }
  746. if commit.Round < 0 {
  747. return errors.New("negative Round")
  748. }
  749. if commit.Height >= 1 {
  750. if commit.BlockID.IsZero() {
  751. return errors.New("commit cannot be for nil block")
  752. }
  753. if len(commit.Signatures) == 0 {
  754. return errors.New("no signatures in commit")
  755. }
  756. for i, commitSig := range commit.Signatures {
  757. if err := commitSig.ValidateBasic(); err != nil {
  758. return fmt.Errorf("wrong CommitSig #%d: %v", i, err)
  759. }
  760. }
  761. }
  762. return nil
  763. }
  764. // Hash returns the hash of the commit
  765. func (commit *Commit) Hash() tmbytes.HexBytes {
  766. if commit == nil {
  767. return nil
  768. }
  769. if commit.hash == nil {
  770. bs := make([][]byte, len(commit.Signatures))
  771. for i, commitSig := range commit.Signatures {
  772. pbcs := commitSig.ToProto()
  773. bz, err := pbcs.Marshal()
  774. if err != nil {
  775. panic(err)
  776. }
  777. bs[i] = bz
  778. }
  779. commit.hash = merkle.HashFromByteSlices(bs)
  780. }
  781. return commit.hash
  782. }
  783. // StringIndented returns a string representation of the commit
  784. func (commit *Commit) StringIndented(indent string) string {
  785. if commit == nil {
  786. return "nil-Commit"
  787. }
  788. commitSigStrings := make([]string, len(commit.Signatures))
  789. for i, commitSig := range commit.Signatures {
  790. commitSigStrings[i] = commitSig.String()
  791. }
  792. return fmt.Sprintf(`Commit{
  793. %s Height: %d
  794. %s Round: %d
  795. %s BlockID: %v
  796. %s Signatures:
  797. %s %v
  798. %s}#%v`,
  799. indent, commit.Height,
  800. indent, commit.Round,
  801. indent, commit.BlockID,
  802. indent,
  803. indent, strings.Join(commitSigStrings, "\n"+indent+" "),
  804. indent, commit.hash)
  805. }
  806. // ToProto converts Commit to protobuf
  807. func (commit *Commit) ToProto() *tmproto.Commit {
  808. if commit == nil {
  809. return nil
  810. }
  811. c := new(tmproto.Commit)
  812. sigs := make([]tmproto.CommitSig, len(commit.Signatures))
  813. for i := range commit.Signatures {
  814. sigs[i] = *commit.Signatures[i].ToProto()
  815. }
  816. c.Signatures = sigs
  817. c.Height = commit.Height
  818. c.Round = commit.Round
  819. c.BlockID = commit.BlockID.ToProto()
  820. if commit.hash != nil {
  821. c.Hash = commit.hash
  822. }
  823. c.BitArray = commit.bitArray.ToProto()
  824. return c
  825. }
  826. // FromProto sets a protobuf Commit to the given pointer.
  827. // It returns an error if the commit is invalid.
  828. func CommitFromProto(cp *tmproto.Commit) (*Commit, error) {
  829. if cp == nil {
  830. return nil, errors.New("nil Commit")
  831. }
  832. var (
  833. commit = new(Commit)
  834. bitArray *bits.BitArray
  835. )
  836. bi, err := BlockIDFromProto(&cp.BlockID)
  837. if err != nil {
  838. return nil, err
  839. }
  840. bitArray.FromProto(cp.BitArray)
  841. sigs := make([]CommitSig, len(cp.Signatures))
  842. for i := range cp.Signatures {
  843. if err := sigs[i].FromProto(cp.Signatures[i]); err != nil {
  844. return nil, err
  845. }
  846. }
  847. commit.Signatures = sigs
  848. commit.Height = cp.Height
  849. commit.Round = cp.Round
  850. commit.BlockID = *bi
  851. commit.hash = cp.Hash
  852. commit.bitArray = bitArray
  853. return commit, commit.ValidateBasic()
  854. }
  855. //-----------------------------------------------------------------------------
  856. // SignedHeader is a header along with the commits that prove it.
  857. // It is the basis of the light client.
  858. type SignedHeader struct {
  859. *Header `json:"header"`
  860. Commit *Commit `json:"commit"`
  861. }
  862. // ValidateBasic does basic consistency checks and makes sure the header
  863. // and commit are consistent.
  864. //
  865. // NOTE: This does not actually check the cryptographic signatures. Make sure
  866. // to use a Verifier to validate the signatures actually provide a
  867. // significantly strong proof for this header's validity.
  868. func (sh SignedHeader) ValidateBasic(chainID string) error {
  869. if sh.Header == nil {
  870. return errors.New("missing header")
  871. }
  872. if sh.Commit == nil {
  873. return errors.New("missing commit")
  874. }
  875. if err := sh.Header.ValidateBasic(); err != nil {
  876. return fmt.Errorf("invalid header: %w", err)
  877. }
  878. if err := sh.Commit.ValidateBasic(); err != nil {
  879. return fmt.Errorf("invalid commit: %w", err)
  880. }
  881. if sh.ChainID != chainID {
  882. return fmt.Errorf("header belongs to another chain %q, not %q", sh.ChainID, chainID)
  883. }
  884. // Make sure the header is consistent with the commit.
  885. if sh.Commit.Height != sh.Height {
  886. return fmt.Errorf("header and commit height mismatch: %d vs %d", sh.Height, sh.Commit.Height)
  887. }
  888. if hhash, chash := sh.Hash(), sh.Commit.BlockID.Hash; !bytes.Equal(hhash, chash) {
  889. return fmt.Errorf("commit signs block %X, header is block %X", chash, hhash)
  890. }
  891. return nil
  892. }
  893. func (sh SignedHeader) String() string {
  894. return sh.StringIndented("")
  895. }
  896. // StringIndented returns a string representation of the SignedHeader.
  897. func (sh SignedHeader) StringIndented(indent string) string {
  898. return fmt.Sprintf(`SignedHeader{
  899. %s %v
  900. %s %v
  901. %s}`,
  902. indent, sh.Header.StringIndented(indent+" "),
  903. indent, sh.Commit.StringIndented(indent+" "),
  904. indent)
  905. }
  906. // ToProto converts SignedHeader to protobuf
  907. func (sh *SignedHeader) ToProto() *tmproto.SignedHeader {
  908. if sh == nil {
  909. return nil
  910. }
  911. psh := new(tmproto.SignedHeader)
  912. if sh.Header != nil {
  913. psh.Header = sh.Header.ToProto()
  914. }
  915. if sh.Commit != nil {
  916. psh.Commit = sh.Commit.ToProto()
  917. }
  918. return psh
  919. }
  920. // FromProto sets a protobuf SignedHeader to the given pointer.
  921. // It returns an error if the hader or the commit is invalid.
  922. func SignedHeaderFromProto(shp *tmproto.SignedHeader) (*SignedHeader, error) {
  923. if shp == nil {
  924. return nil, errors.New("nil SignedHeader")
  925. }
  926. sh := new(SignedHeader)
  927. if shp.Header != nil {
  928. h, err := HeaderFromProto(shp.Header)
  929. if err != nil {
  930. return nil, err
  931. }
  932. sh.Header = &h
  933. }
  934. if shp.Commit != nil {
  935. c, err := CommitFromProto(shp.Commit)
  936. if err != nil {
  937. return nil, err
  938. }
  939. sh.Commit = c
  940. }
  941. return sh, nil
  942. }
  943. //-----------------------------------------------------------------------------
  944. // Data contains the set of transactions included in the block
  945. type Data struct {
  946. // Txs that will be applied by state @ block.Height+1.
  947. // NOTE: not all txs here are valid. We're just agreeing on the order first.
  948. // This means that block.AppHash does not include these txs.
  949. Txs Txs `json:"txs"`
  950. // Volatile
  951. hash tmbytes.HexBytes
  952. }
  953. // Hash returns the hash of the data
  954. func (data *Data) Hash() tmbytes.HexBytes {
  955. if data == nil {
  956. return (Txs{}).Hash()
  957. }
  958. if data.hash == nil {
  959. data.hash = data.Txs.Hash() // NOTE: leaves of merkle tree are TxIDs
  960. }
  961. return data.hash
  962. }
  963. // StringIndented returns a string representation of the transactions
  964. func (data *Data) StringIndented(indent string) string {
  965. if data == nil {
  966. return "nil-Data"
  967. }
  968. txStrings := make([]string, tmmath.MinInt(len(data.Txs), 21))
  969. for i, tx := range data.Txs {
  970. if i == 20 {
  971. txStrings[i] = fmt.Sprintf("... (%v total)", len(data.Txs))
  972. break
  973. }
  974. txStrings[i] = fmt.Sprintf("%X (%d bytes)", tx.Hash(), len(tx))
  975. }
  976. return fmt.Sprintf(`Data{
  977. %s %v
  978. %s}#%v`,
  979. indent, strings.Join(txStrings, "\n"+indent+" "),
  980. indent, data.hash)
  981. }
  982. // ToProto converts Data to protobuf
  983. func (data *Data) ToProto() tmproto.Data {
  984. tp := new(tmproto.Data)
  985. if len(data.Txs) > 0 {
  986. txBzs := make([][]byte, len(data.Txs))
  987. for i := range data.Txs {
  988. txBzs[i] = data.Txs[i]
  989. }
  990. tp.Txs = txBzs
  991. }
  992. if data.hash != nil {
  993. tp.Hash = data.hash
  994. }
  995. return *tp
  996. }
  997. // DataFromProto takes a protobuf representation of Data &
  998. // returns the native type.
  999. func DataFromProto(dp *tmproto.Data) (Data, error) {
  1000. if dp == nil {
  1001. return Data{}, errors.New("nil data")
  1002. }
  1003. data := new(Data)
  1004. if len(dp.Txs) > 0 {
  1005. txBzs := make(Txs, len(dp.Txs))
  1006. for i := range dp.Txs {
  1007. txBzs[i] = Tx(dp.Txs[i])
  1008. }
  1009. data.Txs = txBzs
  1010. } else {
  1011. data.Txs = Txs{}
  1012. }
  1013. data.hash = dp.Hash
  1014. return *data, nil
  1015. }
  1016. //-----------------------------------------------------------------------------
  1017. // EvidenceData contains any evidence of malicious wrong-doing by validators
  1018. type EvidenceData struct {
  1019. Evidence EvidenceList `json:"evidence"`
  1020. // Volatile
  1021. hash tmbytes.HexBytes
  1022. }
  1023. // Hash returns the hash of the data.
  1024. func (data *EvidenceData) Hash() tmbytes.HexBytes {
  1025. if data.hash == nil {
  1026. data.hash = data.Evidence.Hash()
  1027. }
  1028. return data.hash
  1029. }
  1030. // StringIndented returns a string representation of the evidence.
  1031. func (data *EvidenceData) StringIndented(indent string) string {
  1032. if data == nil {
  1033. return "nil-Evidence"
  1034. }
  1035. evStrings := make([]string, tmmath.MinInt(len(data.Evidence), 21))
  1036. for i, ev := range data.Evidence {
  1037. if i == 20 {
  1038. evStrings[i] = fmt.Sprintf("... (%v total)", len(data.Evidence))
  1039. break
  1040. }
  1041. evStrings[i] = fmt.Sprintf("Evidence:%v", ev)
  1042. }
  1043. return fmt.Sprintf(`EvidenceData{
  1044. %s %v
  1045. %s}#%v`,
  1046. indent, strings.Join(evStrings, "\n"+indent+" "),
  1047. indent, data.hash)
  1048. }
  1049. // ToProto converts EvidenceData to protobuf
  1050. func (data *EvidenceData) ToProto() (*tmproto.EvidenceData, error) {
  1051. if data == nil {
  1052. return nil, errors.New("nil evidence data")
  1053. }
  1054. evi := new(tmproto.EvidenceData)
  1055. eviBzs := make([]tmproto.Evidence, len(data.Evidence))
  1056. for i := range data.Evidence {
  1057. protoEvi, err := EvidenceToProto(data.Evidence[i])
  1058. if err != nil {
  1059. return nil, err
  1060. }
  1061. eviBzs[i] = *protoEvi
  1062. }
  1063. evi.Evidence = eviBzs
  1064. if data.hash != nil {
  1065. evi.Hash = data.hash
  1066. }
  1067. return evi, nil
  1068. }
  1069. // FromProto sets a protobuf EvidenceData to the given pointer.
  1070. func (data *EvidenceData) FromProto(eviData *tmproto.EvidenceData) error {
  1071. if eviData == nil {
  1072. return errors.New("nil evidenceData")
  1073. }
  1074. eviBzs := make(EvidenceList, len(eviData.Evidence))
  1075. for i := range eviData.Evidence {
  1076. evi, err := EvidenceFromProto(&eviData.Evidence[i])
  1077. if err != nil {
  1078. return err
  1079. }
  1080. eviBzs[i] = evi
  1081. }
  1082. data.Evidence = eviBzs
  1083. data.hash = eviData.GetHash()
  1084. return nil
  1085. }
  1086. //--------------------------------------------------------------------------------
  1087. // BlockID
  1088. type BlockID struct {
  1089. Hash tmbytes.HexBytes `json:"hash"`
  1090. PartSetHeader PartSetHeader `json:"parts"`
  1091. }
  1092. // Equals returns true if the BlockID matches the given BlockID
  1093. func (blockID BlockID) Equals(other BlockID) bool {
  1094. return bytes.Equal(blockID.Hash, other.Hash) &&
  1095. blockID.PartSetHeader.Equals(other.PartSetHeader)
  1096. }
  1097. // Key returns a machine-readable string representation of the BlockID
  1098. func (blockID BlockID) Key() string {
  1099. pbph := blockID.PartSetHeader.ToProto()
  1100. bz, err := pbph.Marshal()
  1101. if err != nil {
  1102. panic(err)
  1103. }
  1104. return string(blockID.Hash) + string(bz)
  1105. }
  1106. // ValidateBasic performs basic validation.
  1107. func (blockID BlockID) ValidateBasic() error {
  1108. // Hash can be empty in case of POLBlockID in Proposal.
  1109. if err := ValidateHash(blockID.Hash); err != nil {
  1110. return fmt.Errorf("wrong Hash")
  1111. }
  1112. if err := blockID.PartSetHeader.ValidateBasic(); err != nil {
  1113. return fmt.Errorf("wrong PartSetHeader: %v", err)
  1114. }
  1115. return nil
  1116. }
  1117. // IsZero returns true if this is the BlockID of a nil block.
  1118. func (blockID BlockID) IsZero() bool {
  1119. return len(blockID.Hash) == 0 &&
  1120. blockID.PartSetHeader.IsZero()
  1121. }
  1122. // IsComplete returns true if this is a valid BlockID of a non-nil block.
  1123. func (blockID BlockID) IsComplete() bool {
  1124. return len(blockID.Hash) == tmhash.Size &&
  1125. blockID.PartSetHeader.Total > 0 &&
  1126. len(blockID.PartSetHeader.Hash) == tmhash.Size
  1127. }
  1128. // String returns a human readable string representation of the BlockID
  1129. func (blockID BlockID) String() string {
  1130. return fmt.Sprintf(`%v:%v`, blockID.Hash, blockID.PartSetHeader)
  1131. }
  1132. // ToProto converts BlockID to protobuf
  1133. func (blockID *BlockID) ToProto() tmproto.BlockID {
  1134. if blockID == nil {
  1135. return tmproto.BlockID{}
  1136. }
  1137. return tmproto.BlockID{
  1138. Hash: blockID.Hash,
  1139. PartSetHeader: blockID.PartSetHeader.ToProto(),
  1140. }
  1141. }
  1142. // FromProto sets a protobuf BlockID to the given pointer.
  1143. // It returns an error if the block id is invalid.
  1144. func BlockIDFromProto(bID *tmproto.BlockID) (*BlockID, error) {
  1145. if bID == nil {
  1146. return nil, errors.New("nil BlockID")
  1147. }
  1148. blockID := new(BlockID)
  1149. ph, err := PartSetHeaderFromProto(&bID.PartSetHeader)
  1150. if err != nil {
  1151. return nil, err
  1152. }
  1153. blockID.PartSetHeader = *ph
  1154. blockID.Hash = bID.Hash
  1155. return blockID, blockID.ValidateBasic()
  1156. }