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.

935 lines
26 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
lite2: light client with weak subjectivity (#3989) Refs #1771 ADR: https://github.com/tendermint/tendermint/blob/master/docs/architecture/adr-044-lite-client-with-weak-subjectivity.md ## Commits: * add Verifier and VerifyCommitTrusting * add two more checks make trustLevel an option * float32 for trustLevel * check newHeader time * started writing lite Client * unify Verify methods * ensure h2.Header.bfttime < h1.Header.bfttime + tp * move trust checks into Verify function * add more comments * more docs * started writing tests * unbonding period failures * tests are green * export ErrNewHeaderTooFarIntoFuture * make golangci happy * test for non-adjusted headers * more precision * providers and stores * VerifyHeader and VerifyHeaderAtHeight funcs * fix compile errors * remove lastVerifiedHeight, persist new trusted header * sequential verification * remove TrustedStore option * started writing tests for light client * cover basic cases for linear verification * bisection tests PASS * rename BisectingVerification to SkippingVerification * refactor the code * add TrustedHeader method * consolidate sequential verification tests * consolidate skipping verification tests * rename trustedVals to trustedNextVals * start writing docs * ValidateTrustLevel func and ErrOldHeaderExpired error * AutoClient and example tests * fix errors * update doc * remove ErrNewHeaderTooFarIntoFuture This check is unnecessary given existing a) ErrOldHeaderExpired b) h2.Time > now checks. * return an error if we're at more recent height * add comments * add LastSignedHeaderHeight method to Store I think it's fine if Store tracks last height * copy over proxy from old lite package * make TrustedHeader return latest if height=0 * modify LastSignedHeaderHeight to return an error if no headers exist * copy over proxy impl * refactor proxy and start http lite client * Tx and BlockchainInfo methods * Block method * commit method * code compiles again * lite client compiles * extract updateLiteClientIfNeededTo func * move final parts * add placeholder for tests * force usage of lite http client in proxy * comment out query tests for now * explicitly mention tp: trusting period * verify nextVals in VerifyHeader * refactor bisection * move the NextValidatorsHash check into updateTrustedHeaderAndVals + update the comment * add ConsensusParams method to RPC client * add ConsensusParams to rpc/mock/client * change trustLevel type to a new cmn.Fraction type + update SkippingVerification comment * stress out trustLevel is only used for non-adjusted headers * fixes after Fede's review Co-authored-by: Federico Kunze <31522760+fedekunze@users.noreply.github.com> * compare newHeader with a header from an alternative provider * save pivot header Refs https://github.com/tendermint/tendermint/pull/3989#discussion_r349122824 * check header can still be trusted in TrustedHeader Refs https://github.com/tendermint/tendermint/pull/3989#discussion_r349101424 * lite: update Validators and Block endpoints - Block no longer contains BlockMeta - Validators now accept two additional params: page and perPage * make linter happy
5 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 = 632
  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. // ValidateBasic performs basic validation that doesn't involve state data.
  38. // It checks the internal consistency of the block.
  39. // Further validation is done using state#ValidateBlock.
  40. func (b *Block) ValidateBasic() error {
  41. if b == nil {
  42. return errors.New("nil block")
  43. }
  44. b.mtx.Lock()
  45. defer b.mtx.Unlock()
  46. if len(b.ChainID) > MaxChainIDLen {
  47. return fmt.Errorf("chainID is too long. Max is %d, got %d", MaxChainIDLen, len(b.ChainID))
  48. }
  49. if b.Height < 0 {
  50. return errors.New("negative Header.Height")
  51. } else if b.Height == 0 {
  52. return errors.New("zero Header.Height")
  53. }
  54. // NOTE: Timestamp validation is subtle and handled elsewhere.
  55. if err := b.LastBlockID.ValidateBasic(); err != nil {
  56. return fmt.Errorf("wrong Header.LastBlockID: %v", err)
  57. }
  58. // Validate the last commit and its hash.
  59. if b.Header.Height > 1 {
  60. if b.LastCommit == nil {
  61. return errors.New("nil LastCommit")
  62. }
  63. if err := b.LastCommit.ValidateBasic(); err != nil {
  64. return fmt.Errorf("wrong LastCommit: %v", err)
  65. }
  66. }
  67. if err := ValidateHash(b.LastCommitHash); err != nil {
  68. return fmt.Errorf("wrong Header.LastCommitHash: %v", err)
  69. }
  70. if !bytes.Equal(b.LastCommitHash, b.LastCommit.Hash()) {
  71. return fmt.Errorf("wrong Header.LastCommitHash. Expected %v, got %v",
  72. b.LastCommit.Hash(),
  73. b.LastCommitHash,
  74. )
  75. }
  76. // Validate the hash of the transactions.
  77. // NOTE: b.Data.Txs may be nil, but b.Data.Hash()
  78. // still works fine
  79. if err := ValidateHash(b.DataHash); err != nil {
  80. return fmt.Errorf("wrong Header.DataHash: %v", err)
  81. }
  82. if !bytes.Equal(b.DataHash, b.Data.Hash()) {
  83. return fmt.Errorf(
  84. "wrong Header.DataHash. Expected %v, got %v",
  85. b.Data.Hash(),
  86. b.DataHash,
  87. )
  88. }
  89. // Basic validation of hashes related to application data.
  90. // Will validate fully against state in state#ValidateBlock.
  91. if err := ValidateHash(b.ValidatorsHash); err != nil {
  92. return fmt.Errorf("wrong Header.ValidatorsHash: %v", err)
  93. }
  94. if err := ValidateHash(b.NextValidatorsHash); err != nil {
  95. return fmt.Errorf("wrong Header.NextValidatorsHash: %v", err)
  96. }
  97. if err := ValidateHash(b.ConsensusHash); err != nil {
  98. return fmt.Errorf("wrong Header.ConsensusHash: %v", err)
  99. }
  100. // NOTE: AppHash is arbitrary length
  101. if err := ValidateHash(b.LastResultsHash); err != nil {
  102. return fmt.Errorf("wrong Header.LastResultsHash: %v", err)
  103. }
  104. // Validate evidence and its hash.
  105. if err := ValidateHash(b.EvidenceHash); err != nil {
  106. return fmt.Errorf("wrong Header.EvidenceHash: %v", err)
  107. }
  108. // NOTE: b.Evidence.Evidence may be nil, but we're just looping.
  109. for i, ev := range b.Evidence.Evidence {
  110. if err := ev.ValidateBasic(); err != nil {
  111. return fmt.Errorf("invalid evidence (#%d): %v", i, err)
  112. }
  113. }
  114. if !bytes.Equal(b.EvidenceHash, b.Evidence.Hash()) {
  115. return fmt.Errorf("wrong Header.EvidenceHash. Expected %v, got %v",
  116. b.EvidenceHash,
  117. b.Evidence.Hash(),
  118. )
  119. }
  120. if len(b.ProposerAddress) != crypto.AddressSize {
  121. return fmt.Errorf("expected len(Header.ProposerAddress) to be %d, got %d",
  122. crypto.AddressSize, len(b.ProposerAddress))
  123. }
  124. return nil
  125. }
  126. // fillHeader fills in any remaining header fields that are a function of the block data
  127. func (b *Block) fillHeader() {
  128. if b.LastCommitHash == nil {
  129. b.LastCommitHash = b.LastCommit.Hash()
  130. }
  131. if b.DataHash == nil {
  132. b.DataHash = b.Data.Hash()
  133. }
  134. if b.EvidenceHash == nil {
  135. b.EvidenceHash = b.Evidence.Hash()
  136. }
  137. }
  138. // Hash computes and returns the block hash.
  139. // If the block is incomplete, block hash is nil for safety.
  140. func (b *Block) Hash() cmn.HexBytes {
  141. if b == nil {
  142. return nil
  143. }
  144. b.mtx.Lock()
  145. defer b.mtx.Unlock()
  146. if b.LastCommit == nil {
  147. return nil
  148. }
  149. b.fillHeader()
  150. return b.Header.Hash()
  151. }
  152. // MakePartSet returns a PartSet containing parts of a serialized block.
  153. // This is the form in which the block is gossipped to peers.
  154. // CONTRACT: partSize is greater than zero.
  155. func (b *Block) MakePartSet(partSize int) *PartSet {
  156. if b == nil {
  157. return nil
  158. }
  159. b.mtx.Lock()
  160. defer b.mtx.Unlock()
  161. // We prefix the byte length, so that unmarshaling
  162. // can easily happen via a reader.
  163. bz, err := cdc.MarshalBinaryLengthPrefixed(b)
  164. if err != nil {
  165. panic(err)
  166. }
  167. return NewPartSetFromData(bz, partSize)
  168. }
  169. // HashesTo is a convenience function that checks if a block hashes to the given argument.
  170. // Returns false if the block is nil or the hash is empty.
  171. func (b *Block) HashesTo(hash []byte) bool {
  172. if len(hash) == 0 {
  173. return false
  174. }
  175. if b == nil {
  176. return false
  177. }
  178. return bytes.Equal(b.Hash(), hash)
  179. }
  180. // Size returns size of the block in bytes.
  181. func (b *Block) Size() int {
  182. bz, err := cdc.MarshalBinaryBare(b)
  183. if err != nil {
  184. return 0
  185. }
  186. return len(bz)
  187. }
  188. // String returns a string representation of the block
  189. func (b *Block) String() string {
  190. return b.StringIndented("")
  191. }
  192. // StringIndented returns a string representation of the block
  193. func (b *Block) StringIndented(indent string) string {
  194. if b == nil {
  195. return "nil-Block"
  196. }
  197. return fmt.Sprintf(`Block{
  198. %s %v
  199. %s %v
  200. %s %v
  201. %s %v
  202. %s}#%v`,
  203. indent, b.Header.StringIndented(indent+" "),
  204. indent, b.Data.StringIndented(indent+" "),
  205. indent, b.Evidence.StringIndented(indent+" "),
  206. indent, b.LastCommit.StringIndented(indent+" "),
  207. indent, b.Hash())
  208. }
  209. // StringShort returns a shortened string representation of the block
  210. func (b *Block) StringShort() string {
  211. if b == nil {
  212. return "nil-Block"
  213. }
  214. return fmt.Sprintf("Block#%v", b.Hash())
  215. }
  216. //-----------------------------------------------------------
  217. // These methods are for Protobuf Compatibility
  218. // Marshal returns the amino encoding.
  219. func (b *Block) Marshal() ([]byte, error) {
  220. return cdc.MarshalBinaryBare(b)
  221. }
  222. // MarshalTo calls Marshal and copies to the given buffer.
  223. func (b *Block) MarshalTo(data []byte) (int, error) {
  224. bs, err := b.Marshal()
  225. if err != nil {
  226. return -1, err
  227. }
  228. return copy(data, bs), nil
  229. }
  230. // Unmarshal deserializes from amino encoded form.
  231. func (b *Block) Unmarshal(bs []byte) error {
  232. return cdc.UnmarshalBinaryBare(bs, b)
  233. }
  234. //-----------------------------------------------------------------------------
  235. // MaxDataBytes returns the maximum size of block's data.
  236. //
  237. // XXX: Panics on negative result.
  238. func MaxDataBytes(maxBytes int64, valsCount, evidenceCount int) int64 {
  239. maxDataBytes := maxBytes -
  240. MaxAminoOverheadForBlock -
  241. MaxHeaderBytes -
  242. int64(valsCount)*MaxVoteBytes -
  243. int64(evidenceCount)*MaxEvidenceBytes
  244. if maxDataBytes < 0 {
  245. panic(fmt.Sprintf(
  246. "Negative MaxDataBytes. Block.MaxBytes=%d is too small to accommodate header&lastCommit&evidence=%d",
  247. maxBytes,
  248. -(maxDataBytes - maxBytes),
  249. ))
  250. }
  251. return maxDataBytes
  252. }
  253. // MaxDataBytesUnknownEvidence returns the maximum size of block's data when
  254. // evidence count is unknown. MaxEvidencePerBlock will be used for the size
  255. // of evidence.
  256. //
  257. // XXX: Panics on negative result.
  258. func MaxDataBytesUnknownEvidence(maxBytes int64, valsCount int) int64 {
  259. _, maxEvidenceBytes := MaxEvidencePerBlock(maxBytes)
  260. maxDataBytes := maxBytes -
  261. MaxAminoOverheadForBlock -
  262. MaxHeaderBytes -
  263. int64(valsCount)*MaxVoteBytes -
  264. maxEvidenceBytes
  265. if maxDataBytes < 0 {
  266. panic(fmt.Sprintf(
  267. "Negative MaxDataBytesUnknownEvidence. Block.MaxBytes=%d is too small to accommodate header&lastCommit&evidence=%d",
  268. maxBytes,
  269. -(maxDataBytes - maxBytes),
  270. ))
  271. }
  272. return maxDataBytes
  273. }
  274. //-----------------------------------------------------------------------------
  275. // Header defines the structure of a Tendermint block header.
  276. // NOTE: changes to the Header should be duplicated in:
  277. // - header.Hash()
  278. // - abci.Header
  279. // - /docs/spec/blockchain/blockchain.md
  280. type Header struct {
  281. // basic block info
  282. Version version.Consensus `json:"version"`
  283. ChainID string `json:"chain_id"`
  284. Height int64 `json:"height"`
  285. Time time.Time `json:"time"`
  286. // prev block info
  287. LastBlockID BlockID `json:"last_block_id"`
  288. // hashes of block data
  289. LastCommitHash cmn.HexBytes `json:"last_commit_hash"` // commit from validators from the last block
  290. DataHash cmn.HexBytes `json:"data_hash"` // transactions
  291. // hashes from the app output from the prev block
  292. ValidatorsHash cmn.HexBytes `json:"validators_hash"` // validators for the current block
  293. NextValidatorsHash cmn.HexBytes `json:"next_validators_hash"` // validators for the next block
  294. ConsensusHash cmn.HexBytes `json:"consensus_hash"` // consensus params for current block
  295. AppHash cmn.HexBytes `json:"app_hash"` // state after txs from the previous block
  296. // root hash of all results from the txs from the previous block
  297. LastResultsHash cmn.HexBytes `json:"last_results_hash"`
  298. // consensus info
  299. EvidenceHash cmn.HexBytes `json:"evidence_hash"` // evidence included in the block
  300. ProposerAddress Address `json:"proposer_address"` // original proposer of the block
  301. }
  302. // Populate the Header with state-derived data.
  303. // Call this after MakeBlock to complete the Header.
  304. func (h *Header) Populate(
  305. version version.Consensus, chainID string,
  306. timestamp time.Time, lastBlockID BlockID,
  307. valHash, nextValHash []byte,
  308. consensusHash, appHash, lastResultsHash []byte,
  309. proposerAddress Address,
  310. ) {
  311. h.Version = version
  312. h.ChainID = chainID
  313. h.Time = timestamp
  314. h.LastBlockID = lastBlockID
  315. h.ValidatorsHash = valHash
  316. h.NextValidatorsHash = nextValHash
  317. h.ConsensusHash = consensusHash
  318. h.AppHash = appHash
  319. h.LastResultsHash = lastResultsHash
  320. h.ProposerAddress = proposerAddress
  321. }
  322. // Hash returns the hash of the header.
  323. // It computes a Merkle tree from the header fields
  324. // ordered as they appear in the Header.
  325. // Returns nil if ValidatorHash is missing,
  326. // since a Header is not valid unless there is
  327. // a ValidatorsHash (corresponding to the validator set).
  328. func (h *Header) Hash() cmn.HexBytes {
  329. if h == nil || len(h.ValidatorsHash) == 0 {
  330. return nil
  331. }
  332. return merkle.SimpleHashFromByteSlices([][]byte{
  333. cdcEncode(h.Version),
  334. cdcEncode(h.ChainID),
  335. cdcEncode(h.Height),
  336. cdcEncode(h.Time),
  337. cdcEncode(h.LastBlockID),
  338. cdcEncode(h.LastCommitHash),
  339. cdcEncode(h.DataHash),
  340. cdcEncode(h.ValidatorsHash),
  341. cdcEncode(h.NextValidatorsHash),
  342. cdcEncode(h.ConsensusHash),
  343. cdcEncode(h.AppHash),
  344. cdcEncode(h.LastResultsHash),
  345. cdcEncode(h.EvidenceHash),
  346. cdcEncode(h.ProposerAddress),
  347. })
  348. }
  349. // StringIndented returns a string representation of the header
  350. func (h *Header) StringIndented(indent string) string {
  351. if h == nil {
  352. return "nil-Header"
  353. }
  354. return fmt.Sprintf(`Header{
  355. %s Version: %v
  356. %s ChainID: %v
  357. %s Height: %v
  358. %s Time: %v
  359. %s LastBlockID: %v
  360. %s LastCommit: %v
  361. %s Data: %v
  362. %s Validators: %v
  363. %s NextValidators: %v
  364. %s App: %v
  365. %s Consensus: %v
  366. %s Results: %v
  367. %s Evidence: %v
  368. %s Proposer: %v
  369. %s}#%v`,
  370. indent, h.Version,
  371. indent, h.ChainID,
  372. indent, h.Height,
  373. indent, h.Time,
  374. indent, h.LastBlockID,
  375. indent, h.LastCommitHash,
  376. indent, h.DataHash,
  377. indent, h.ValidatorsHash,
  378. indent, h.NextValidatorsHash,
  379. indent, h.AppHash,
  380. indent, h.ConsensusHash,
  381. indent, h.LastResultsHash,
  382. indent, h.EvidenceHash,
  383. indent, h.ProposerAddress,
  384. indent, h.Hash())
  385. }
  386. //-------------------------------------
  387. // BlockIDFlag indicates which BlockID the signature is for.
  388. type BlockIDFlag byte
  389. const (
  390. // BlockIDFlagAbsent - no vote was received from a validator.
  391. BlockIDFlagAbsent BlockIDFlag = iota + 1
  392. // BlockIDFlagCommit - voted for the Commit.BlockID.
  393. BlockIDFlagCommit
  394. // BlockIDFlagNil - voted for nil.
  395. BlockIDFlagNil
  396. )
  397. // CommitSig is a part of the Vote included in a Commit.
  398. type CommitSig struct {
  399. BlockIDFlag BlockIDFlag `json:"block_id_flag"`
  400. ValidatorAddress Address `json:"validator_address"`
  401. Timestamp time.Time `json:"timestamp"`
  402. Signature []byte `json:"signature"`
  403. }
  404. // NewCommitSigForBlock returns new CommitSig with BlockIDFlagCommit.
  405. func NewCommitSigForBlock(signature []byte, valAddr Address, ts time.Time) CommitSig {
  406. return CommitSig{
  407. BlockIDFlag: BlockIDFlagCommit,
  408. ValidatorAddress: valAddr,
  409. Timestamp: ts,
  410. Signature: signature,
  411. }
  412. }
  413. // NewCommitSigAbsent returns new CommitSig with BlockIDFlagAbsent. Other
  414. // fields are all empty.
  415. func NewCommitSigAbsent() CommitSig {
  416. return CommitSig{
  417. BlockIDFlag: BlockIDFlagAbsent,
  418. }
  419. }
  420. // Absent returns true if CommitSig is absent.
  421. func (cs CommitSig) Absent() bool {
  422. return cs.BlockIDFlag == BlockIDFlagAbsent
  423. }
  424. func (cs CommitSig) String() string {
  425. return fmt.Sprintf("CommitSig{%X by %X on %v @ %s}",
  426. cmn.Fingerprint(cs.Signature),
  427. cmn.Fingerprint(cs.ValidatorAddress),
  428. cs.BlockIDFlag,
  429. CanonicalTime(cs.Timestamp))
  430. }
  431. // BlockID returns the Commit's BlockID if CommitSig indicates signing,
  432. // otherwise - empty BlockID.
  433. func (cs CommitSig) BlockID(commitBlockID BlockID) BlockID {
  434. var blockID BlockID
  435. switch cs.BlockIDFlag {
  436. case BlockIDFlagAbsent:
  437. blockID = BlockID{}
  438. case BlockIDFlagCommit:
  439. blockID = commitBlockID
  440. case BlockIDFlagNil:
  441. blockID = BlockID{}
  442. default:
  443. panic(fmt.Sprintf("Unknown BlockIDFlag: %v", cs.BlockIDFlag))
  444. }
  445. return blockID
  446. }
  447. // ValidateBasic performs basic validation.
  448. func (cs CommitSig) ValidateBasic() error {
  449. switch cs.BlockIDFlag {
  450. case BlockIDFlagAbsent:
  451. case BlockIDFlagCommit:
  452. case BlockIDFlagNil:
  453. default:
  454. return fmt.Errorf("unknown BlockIDFlag: %v", cs.BlockIDFlag)
  455. }
  456. switch cs.BlockIDFlag {
  457. case BlockIDFlagAbsent:
  458. if len(cs.ValidatorAddress) != 0 {
  459. return errors.New("validator address is present")
  460. }
  461. if !cs.Timestamp.IsZero() {
  462. return errors.New("time is present")
  463. }
  464. if len(cs.Signature) != 0 {
  465. return errors.New("signature is present")
  466. }
  467. default:
  468. if len(cs.ValidatorAddress) != crypto.AddressSize {
  469. return fmt.Errorf("expected ValidatorAddress size to be %d bytes, got %d bytes",
  470. crypto.AddressSize,
  471. len(cs.ValidatorAddress),
  472. )
  473. }
  474. // NOTE: Timestamp validation is subtle and handled elsewhere.
  475. if len(cs.Signature) == 0 {
  476. return errors.New("signature is missing")
  477. }
  478. if len(cs.Signature) > MaxSignatureSize {
  479. return fmt.Errorf("signature is too big (max: %d)", MaxSignatureSize)
  480. }
  481. }
  482. return nil
  483. }
  484. //-------------------------------------
  485. // Commit contains the evidence that a block was committed by a set of validators.
  486. // NOTE: Commit is empty for height 1, but never nil.
  487. type Commit struct {
  488. // NOTE: The signatures are in order of address to preserve the bonded
  489. // ValidatorSet order.
  490. // Any peer with a block can gossip signatures by index with a peer without
  491. // recalculating the active ValidatorSet.
  492. Height int64 `json:"height"`
  493. Round int `json:"round"`
  494. BlockID BlockID `json:"block_id"`
  495. Signatures []CommitSig `json:"signatures"`
  496. // Memoized in first call to corresponding method.
  497. // NOTE: can't memoize in constructor because constructor isn't used for
  498. // unmarshaling.
  499. hash cmn.HexBytes
  500. bitArray *cmn.BitArray
  501. }
  502. // NewCommit returns a new Commit.
  503. func NewCommit(height int64, round int, blockID BlockID, commitSigs []CommitSig) *Commit {
  504. return &Commit{
  505. Height: height,
  506. Round: round,
  507. BlockID: blockID,
  508. Signatures: commitSigs,
  509. }
  510. }
  511. // CommitToVoteSet constructs a VoteSet from the Commit and validator set.
  512. // Panics if signatures from the commit can't be added to the voteset.
  513. // Inverse of VoteSet.MakeCommit().
  514. func CommitToVoteSet(chainID string, commit *Commit, vals *ValidatorSet) *VoteSet {
  515. voteSet := NewVoteSet(chainID, commit.Height, commit.Round, PrecommitType, vals)
  516. for idx, commitSig := range commit.Signatures {
  517. if commitSig.Absent() {
  518. continue // OK, some precommits can be missing.
  519. }
  520. added, err := voteSet.AddVote(commit.GetVote(idx))
  521. if !added || err != nil {
  522. panic(fmt.Sprintf("Failed to reconstruct LastCommit: %v", err))
  523. }
  524. }
  525. return voteSet
  526. }
  527. // GetVote converts the CommitSig for the given valIdx to a Vote.
  528. // Returns nil if the precommit at valIdx is nil.
  529. // Panics if valIdx >= commit.Size().
  530. func (commit *Commit) GetVote(valIdx int) *Vote {
  531. commitSig := commit.Signatures[valIdx]
  532. return &Vote{
  533. Type: PrecommitType,
  534. Height: commit.Height,
  535. Round: commit.Round,
  536. BlockID: commitSig.BlockID(commit.BlockID),
  537. Timestamp: commitSig.Timestamp,
  538. ValidatorAddress: commitSig.ValidatorAddress,
  539. ValidatorIndex: valIdx,
  540. Signature: commitSig.Signature,
  541. }
  542. }
  543. // VoteSignBytes constructs the SignBytes for the given CommitSig.
  544. // The only unique part of the SignBytes is the Timestamp - all other fields
  545. // signed over are otherwise the same for all validators.
  546. // Panics if valIdx >= commit.Size().
  547. func (commit *Commit) VoteSignBytes(chainID string, valIdx int) []byte {
  548. return commit.GetVote(valIdx).SignBytes(chainID)
  549. }
  550. // Type returns the vote type of the commit, which is always VoteTypePrecommit
  551. // Implements VoteSetReader.
  552. func (commit *Commit) Type() byte {
  553. return byte(PrecommitType)
  554. }
  555. // GetHeight returns height of the commit.
  556. // Implements VoteSetReader.
  557. func (commit *Commit) GetHeight() int64 {
  558. return commit.Height
  559. }
  560. // GetRound returns height of the commit.
  561. // Implements VoteSetReader.
  562. func (commit *Commit) GetRound() int {
  563. return commit.Round
  564. }
  565. // Size returns the number of signatures in the commit.
  566. // Implements VoteSetReader.
  567. func (commit *Commit) Size() int {
  568. if commit == nil {
  569. return 0
  570. }
  571. return len(commit.Signatures)
  572. }
  573. // BitArray returns a BitArray of which validators voted for BlockID or nil in this commit.
  574. // Implements VoteSetReader.
  575. func (commit *Commit) BitArray() *cmn.BitArray {
  576. if commit.bitArray == nil {
  577. commit.bitArray = cmn.NewBitArray(len(commit.Signatures))
  578. for i, commitSig := range commit.Signatures {
  579. // TODO: need to check the BlockID otherwise we could be counting conflicts,
  580. // not just the one with +2/3 !
  581. commit.bitArray.SetIndex(i, !commitSig.Absent())
  582. }
  583. }
  584. return commit.bitArray
  585. }
  586. // GetByIndex returns the vote corresponding to a given validator index.
  587. // Panics if `index >= commit.Size()`.
  588. // Implements VoteSetReader.
  589. func (commit *Commit) GetByIndex(valIdx int) *Vote {
  590. return commit.GetVote(valIdx)
  591. }
  592. // IsCommit returns true if there is at least one signature.
  593. // Implements VoteSetReader.
  594. func (commit *Commit) IsCommit() bool {
  595. return len(commit.Signatures) != 0
  596. }
  597. // ValidateBasic performs basic validation that doesn't involve state data.
  598. // Does not actually check the cryptographic signatures.
  599. func (commit *Commit) ValidateBasic() error {
  600. if commit.Height < 0 {
  601. return errors.New("negative Height")
  602. }
  603. if commit.Round < 0 {
  604. return errors.New("negative Round")
  605. }
  606. if commit.BlockID.IsZero() {
  607. return errors.New("commit cannot be for nil block")
  608. }
  609. if len(commit.Signatures) == 0 {
  610. return errors.New("no signatures in commit")
  611. }
  612. for i, commitSig := range commit.Signatures {
  613. if err := commitSig.ValidateBasic(); err != nil {
  614. return fmt.Errorf("wrong CommitSig #%d: %v", i, err)
  615. }
  616. }
  617. return nil
  618. }
  619. // Hash returns the hash of the commit
  620. func (commit *Commit) Hash() cmn.HexBytes {
  621. if commit == nil {
  622. return nil
  623. }
  624. if commit.hash == nil {
  625. bs := make([][]byte, len(commit.Signatures))
  626. for i, commitSig := range commit.Signatures {
  627. bs[i] = cdcEncode(commitSig)
  628. }
  629. commit.hash = merkle.SimpleHashFromByteSlices(bs)
  630. }
  631. return commit.hash
  632. }
  633. // StringIndented returns a string representation of the commit
  634. func (commit *Commit) StringIndented(indent string) string {
  635. if commit == nil {
  636. return "nil-Commit"
  637. }
  638. commitSigStrings := make([]string, len(commit.Signatures))
  639. for i, commitSig := range commit.Signatures {
  640. commitSigStrings[i] = commitSig.String()
  641. }
  642. return fmt.Sprintf(`Commit{
  643. %s Height: %d
  644. %s Round: %d
  645. %s BlockID: %v
  646. %s Signatures:
  647. %s %v
  648. %s}#%v`,
  649. indent, commit.Height,
  650. indent, commit.Round,
  651. indent, commit.BlockID,
  652. indent,
  653. indent, strings.Join(commitSigStrings, "\n"+indent+" "),
  654. indent, commit.hash)
  655. }
  656. //-----------------------------------------------------------------------------
  657. // SignedHeader is a header along with the commits that prove it.
  658. // It is the basis of the lite client.
  659. type SignedHeader struct {
  660. *Header `json:"header"`
  661. Commit *Commit `json:"commit"`
  662. }
  663. // ValidateBasic does basic consistency checks and makes sure the header
  664. // and commit are consistent.
  665. //
  666. // NOTE: This does not actually check the cryptographic signatures. Make
  667. // sure to use a Verifier to validate the signatures actually provide a
  668. // significantly strong proof for this header's validity.
  669. func (sh SignedHeader) ValidateBasic(chainID string) error {
  670. // Make sure the header is consistent with the commit.
  671. if sh.Header == nil {
  672. return errors.New("signedHeader missing header")
  673. }
  674. if sh.Commit == nil {
  675. return errors.New("signedHeader missing commit (precommit votes)")
  676. }
  677. // Check ChainID.
  678. if sh.ChainID != chainID {
  679. return fmt.Errorf("signedHeader belongs to another chain '%s' not '%s'",
  680. sh.ChainID, chainID)
  681. }
  682. // Check Height.
  683. if sh.Commit.Height != sh.Height {
  684. return fmt.Errorf("signedHeader header and commit height mismatch: %v vs %v",
  685. sh.Height, sh.Commit.Height)
  686. }
  687. // Check Hash.
  688. hhash := sh.Hash()
  689. chash := sh.Commit.BlockID.Hash
  690. if !bytes.Equal(hhash, chash) {
  691. return fmt.Errorf("signedHeader commit signs block %X, header is block %X",
  692. chash, hhash)
  693. }
  694. // ValidateBasic on the Commit.
  695. err := sh.Commit.ValidateBasic()
  696. if err != nil {
  697. return errors.Wrap(err, "commit.ValidateBasic failed during SignedHeader.ValidateBasic")
  698. }
  699. return nil
  700. }
  701. func (sh SignedHeader) String() string {
  702. return sh.StringIndented("")
  703. }
  704. // StringIndented returns a string representation of the SignedHeader.
  705. func (sh SignedHeader) StringIndented(indent string) string {
  706. return fmt.Sprintf(`SignedHeader{
  707. %s %v
  708. %s %v
  709. %s}`,
  710. indent, sh.Header.StringIndented(indent+" "),
  711. indent, sh.Commit.StringIndented(indent+" "),
  712. indent)
  713. }
  714. //-----------------------------------------------------------------------------
  715. // Data contains the set of transactions included in the block
  716. type Data struct {
  717. // Txs that will be applied by state @ block.Height+1.
  718. // NOTE: not all txs here are valid. We're just agreeing on the order first.
  719. // This means that block.AppHash does not include these txs.
  720. Txs Txs `json:"txs"`
  721. // Volatile
  722. hash cmn.HexBytes
  723. }
  724. // Hash returns the hash of the data
  725. func (data *Data) Hash() cmn.HexBytes {
  726. if data == nil {
  727. return (Txs{}).Hash()
  728. }
  729. if data.hash == nil {
  730. data.hash = data.Txs.Hash() // NOTE: leaves of merkle tree are TxIDs
  731. }
  732. return data.hash
  733. }
  734. // StringIndented returns a string representation of the transactions
  735. func (data *Data) StringIndented(indent string) string {
  736. if data == nil {
  737. return "nil-Data"
  738. }
  739. txStrings := make([]string, cmn.MinInt(len(data.Txs), 21))
  740. for i, tx := range data.Txs {
  741. if i == 20 {
  742. txStrings[i] = fmt.Sprintf("... (%v total)", len(data.Txs))
  743. break
  744. }
  745. txStrings[i] = fmt.Sprintf("%X (%d bytes)", tx.Hash(), len(tx))
  746. }
  747. return fmt.Sprintf(`Data{
  748. %s %v
  749. %s}#%v`,
  750. indent, strings.Join(txStrings, "\n"+indent+" "),
  751. indent, data.hash)
  752. }
  753. //-----------------------------------------------------------------------------
  754. // EvidenceData contains any evidence of malicious wrong-doing by validators
  755. type EvidenceData struct {
  756. Evidence EvidenceList `json:"evidence"`
  757. // Volatile
  758. hash cmn.HexBytes
  759. }
  760. // Hash returns the hash of the data.
  761. func (data *EvidenceData) Hash() cmn.HexBytes {
  762. if data.hash == nil {
  763. data.hash = data.Evidence.Hash()
  764. }
  765. return data.hash
  766. }
  767. // StringIndented returns a string representation of the evidence.
  768. func (data *EvidenceData) StringIndented(indent string) string {
  769. if data == nil {
  770. return "nil-Evidence"
  771. }
  772. evStrings := make([]string, cmn.MinInt(len(data.Evidence), 21))
  773. for i, ev := range data.Evidence {
  774. if i == 20 {
  775. evStrings[i] = fmt.Sprintf("... (%v total)", len(data.Evidence))
  776. break
  777. }
  778. evStrings[i] = fmt.Sprintf("Evidence:%v", ev)
  779. }
  780. return fmt.Sprintf(`EvidenceData{
  781. %s %v
  782. %s}#%v`,
  783. indent, strings.Join(evStrings, "\n"+indent+" "),
  784. indent, data.hash)
  785. }
  786. //--------------------------------------------------------------------------------
  787. // BlockID defines the unique ID of a block as its Hash and its PartSetHeader
  788. type BlockID struct {
  789. Hash cmn.HexBytes `json:"hash"`
  790. PartsHeader PartSetHeader `json:"parts"`
  791. }
  792. // Equals returns true if the BlockID matches the given BlockID
  793. func (blockID BlockID) Equals(other BlockID) bool {
  794. return bytes.Equal(blockID.Hash, other.Hash) &&
  795. blockID.PartsHeader.Equals(other.PartsHeader)
  796. }
  797. // Key returns a machine-readable string representation of the BlockID
  798. func (blockID BlockID) Key() string {
  799. bz, err := cdc.MarshalBinaryBare(blockID.PartsHeader)
  800. if err != nil {
  801. panic(err)
  802. }
  803. return string(blockID.Hash) + string(bz)
  804. }
  805. // ValidateBasic performs basic validation.
  806. func (blockID BlockID) ValidateBasic() error {
  807. // Hash can be empty in case of POLBlockID in Proposal.
  808. if err := ValidateHash(blockID.Hash); err != nil {
  809. return fmt.Errorf("wrong Hash")
  810. }
  811. if err := blockID.PartsHeader.ValidateBasic(); err != nil {
  812. return fmt.Errorf("wrong PartsHeader: %v", err)
  813. }
  814. return nil
  815. }
  816. // IsZero returns true if this is the BlockID of a nil block.
  817. func (blockID BlockID) IsZero() bool {
  818. return len(blockID.Hash) == 0 &&
  819. blockID.PartsHeader.IsZero()
  820. }
  821. // IsComplete returns true if this is a valid BlockID of a non-nil block.
  822. func (blockID BlockID) IsComplete() bool {
  823. return len(blockID.Hash) == tmhash.Size &&
  824. blockID.PartsHeader.Total > 0 &&
  825. len(blockID.PartsHeader.Hash) == tmhash.Size
  826. }
  827. // String returns a human readable string representation of the BlockID
  828. func (blockID BlockID) String() string {
  829. return fmt.Sprintf(`%v:%v`, blockID.Hash, blockID.PartsHeader)
  830. }