tendermint/types/block.go
Dev Ojha 12fa9d1cab crypto/merkle: Remove byter in favor of plain byte slices (#2595)
* crypto/merkle: Remove byter in favor of plain byte slices

This PR is fully backwards compatible in terms of function output!
(The Go API differs though) The only test case changes was to refactor
it to be table driven.

* Update godocs per review comments
2018-10-10 12:46:09 -04:00

691 lines
19 KiB
Go

package types
import (
"bytes"
"errors"
"fmt"
"strings"
"sync"
"time"
"github.com/tendermint/tendermint/crypto/merkle"
cmn "github.com/tendermint/tendermint/libs/common"
)
const (
// MaxHeaderBytes is a maximum header size (including amino overhead).
MaxHeaderBytes int64 = 511
// MaxAminoOverheadForBlock - maximum amino overhead to encode a block (up to
// MaxBlockSizeBytes in size) not including it's parts except Data.
//
// Uvarint length of MaxBlockSizeBytes: 4 bytes
// 2 fields (2 embedded): 2 bytes
// Uvarint length of Data.Txs: 4 bytes
// Data.Txs field: 1 byte
MaxAminoOverheadForBlock int64 = 11
)
// Block defines the atomic unit of a Tendermint blockchain.
// TODO: add Version byte
type Block struct {
mtx sync.Mutex
Header `json:"header"`
Data `json:"data"`
Evidence EvidenceData `json:"evidence"`
LastCommit *Commit `json:"last_commit"`
}
// MakeBlock returns a new block with an empty header, except what can be
// computed from itself.
// It populates the same set of fields validated by ValidateBasic.
func MakeBlock(height int64, txs []Tx, lastCommit *Commit, evidence []Evidence) *Block {
block := &Block{
Header: Header{
Height: height,
NumTxs: int64(len(txs)),
},
Data: Data{
Txs: txs,
},
Evidence: EvidenceData{Evidence: evidence},
LastCommit: lastCommit,
}
block.fillHeader()
return block
}
// ValidateBasic performs basic validation that doesn't involve state data.
// It checks the internal consistency of the block.
func (b *Block) ValidateBasic() error {
if b == nil {
return errors.New("Nil blocks are invalid")
}
b.mtx.Lock()
defer b.mtx.Unlock()
newTxs := int64(len(b.Data.Txs))
if b.NumTxs != newTxs {
return fmt.Errorf(
"Wrong Block.Header.NumTxs. Expected %v, got %v",
newTxs,
b.NumTxs,
)
}
if !bytes.Equal(b.LastCommitHash, b.LastCommit.Hash()) {
return fmt.Errorf(
"Wrong Block.Header.LastCommitHash. Expected %v, got %v",
b.LastCommitHash,
b.LastCommit.Hash(),
)
}
if b.Header.Height != 1 {
if err := b.LastCommit.ValidateBasic(); err != nil {
return err
}
}
if !bytes.Equal(b.DataHash, b.Data.Hash()) {
return fmt.Errorf(
"Wrong Block.Header.DataHash. Expected %v, got %v",
b.DataHash,
b.Data.Hash(),
)
}
if !bytes.Equal(b.EvidenceHash, b.Evidence.Hash()) {
return fmt.Errorf(
"Wrong Block.Header.EvidenceHash. Expected %v, got %v",
b.EvidenceHash,
b.Evidence.Hash(),
)
}
return nil
}
// fillHeader fills in any remaining header fields that are a function of the block data
func (b *Block) fillHeader() {
if b.LastCommitHash == nil {
b.LastCommitHash = b.LastCommit.Hash()
}
if b.DataHash == nil {
b.DataHash = b.Data.Hash()
}
if b.EvidenceHash == nil {
b.EvidenceHash = b.Evidence.Hash()
}
}
// Hash computes and returns the block hash.
// If the block is incomplete, block hash is nil for safety.
func (b *Block) Hash() cmn.HexBytes {
if b == nil {
return nil
}
b.mtx.Lock()
defer b.mtx.Unlock()
if b == nil || b.LastCommit == nil {
return nil
}
b.fillHeader()
return b.Header.Hash()
}
// MakePartSet returns a PartSet containing parts of a serialized block.
// This is the form in which the block is gossipped to peers.
// CONTRACT: partSize is greater than zero.
func (b *Block) MakePartSet(partSize int) *PartSet {
if b == nil {
return nil
}
b.mtx.Lock()
defer b.mtx.Unlock()
// We prefix the byte length, so that unmarshaling
// can easily happen via a reader.
bz, err := cdc.MarshalBinary(b)
if err != nil {
panic(err)
}
return NewPartSetFromData(bz, partSize)
}
// HashesTo is a convenience function that checks if a block hashes to the given argument.
// Returns false if the block is nil or the hash is empty.
func (b *Block) HashesTo(hash []byte) bool {
if len(hash) == 0 {
return false
}
if b == nil {
return false
}
return bytes.Equal(b.Hash(), hash)
}
// Size returns size of the block in bytes.
func (b *Block) Size() int {
bz, err := cdc.MarshalBinaryBare(b)
if err != nil {
return 0
}
return len(bz)
}
// String returns a string representation of the block
func (b *Block) String() string {
return b.StringIndented("")
}
// StringIndented returns a string representation of the block
func (b *Block) StringIndented(indent string) string {
if b == nil {
return "nil-Block"
}
return fmt.Sprintf(`Block{
%s %v
%s %v
%s %v
%s %v
%s}#%v`,
indent, b.Header.StringIndented(indent+" "),
indent, b.Data.StringIndented(indent+" "),
indent, b.Evidence.StringIndented(indent+" "),
indent, b.LastCommit.StringIndented(indent+" "),
indent, b.Hash())
}
// StringShort returns a shortened string representation of the block
func (b *Block) StringShort() string {
if b == nil {
return "nil-Block"
}
return fmt.Sprintf("Block#%v", b.Hash())
}
//-----------------------------------------------------------------------------
// MaxDataBytes returns the maximum size of block's data.
//
// XXX: Panics on negative result.
func MaxDataBytes(maxBytes int64, valsCount, evidenceCount int) int64 {
maxDataBytes := maxBytes -
MaxAminoOverheadForBlock -
MaxHeaderBytes -
int64(valsCount)*MaxVoteBytes -
int64(evidenceCount)*MaxEvidenceBytes
if maxDataBytes < 0 {
panic(fmt.Sprintf(
"Negative MaxDataBytes. BlockSize.MaxBytes=%d is too small to accommodate header&lastCommit&evidence=%d",
maxBytes,
-(maxDataBytes - maxBytes),
))
}
return maxDataBytes
}
// MaxDataBytesUnknownEvidence returns the maximum size of block's data when
// evidence count is unknown. MaxEvidenceBytesPerBlock will be used as the size
// of evidence.
//
// XXX: Panics on negative result.
func MaxDataBytesUnknownEvidence(maxBytes int64, valsCount int) int64 {
maxDataBytes := maxBytes -
MaxAminoOverheadForBlock -
MaxHeaderBytes -
int64(valsCount)*MaxVoteBytes -
MaxEvidenceBytesPerBlock(maxBytes)
if maxDataBytes < 0 {
panic(fmt.Sprintf(
"Negative MaxDataBytesUnknownEvidence. BlockSize.MaxBytes=%d is too small to accommodate header&lastCommit&evidence=%d",
maxBytes,
-(maxDataBytes - maxBytes),
))
}
return maxDataBytes
}
//-----------------------------------------------------------------------------
// Header defines the structure of a Tendermint block header
// TODO: limit header size
// NOTE: changes to the Header should be duplicated in the abci Header
// and in /docs/spec/blockchain/blockchain.md
type Header struct {
// basic block info
ChainID string `json:"chain_id"`
Height int64 `json:"height"`
Time time.Time `json:"time"`
NumTxs int64 `json:"num_txs"`
TotalTxs int64 `json:"total_txs"`
// prev block info
LastBlockID BlockID `json:"last_block_id"`
// hashes of block data
LastCommitHash cmn.HexBytes `json:"last_commit_hash"` // commit from validators from the last block
DataHash cmn.HexBytes `json:"data_hash"` // transactions
// hashes from the app output from the prev block
ValidatorsHash cmn.HexBytes `json:"validators_hash"` // validators for the current block
NextValidatorsHash cmn.HexBytes `json:"next_validators_hash"` // validators for the next block
ConsensusHash cmn.HexBytes `json:"consensus_hash"` // consensus params for current block
AppHash cmn.HexBytes `json:"app_hash"` // state after txs from the previous block
LastResultsHash cmn.HexBytes `json:"last_results_hash"` // root hash of all results from the txs from the previous block
// consensus info
EvidenceHash cmn.HexBytes `json:"evidence_hash"` // evidence included in the block
ProposerAddress Address `json:"proposer_address"` // original proposer of the block
}
// Hash returns the hash of the header.
// Returns nil if ValidatorHash is missing,
// since a Header is not valid unless there is
// a ValidatorsHash (corresponding to the validator set).
func (h *Header) Hash() cmn.HexBytes {
if h == nil || len(h.ValidatorsHash) == 0 {
return nil
}
return merkle.SimpleHashFromMap(map[string][]byte{
"ChainID": cdcEncode(h.ChainID),
"Height": cdcEncode(h.Height),
"Time": cdcEncode(h.Time),
"NumTxs": cdcEncode(h.NumTxs),
"TotalTxs": cdcEncode(h.TotalTxs),
"LastBlockID": cdcEncode(h.LastBlockID),
"LastCommit": cdcEncode(h.LastCommitHash),
"Data": cdcEncode(h.DataHash),
"Validators": cdcEncode(h.ValidatorsHash),
"NextValidators": cdcEncode(h.NextValidatorsHash),
"App": cdcEncode(h.AppHash),
"Consensus": cdcEncode(h.ConsensusHash),
"Results": cdcEncode(h.LastResultsHash),
"Evidence": cdcEncode(h.EvidenceHash),
"Proposer": cdcEncode(h.ProposerAddress),
})
}
// StringIndented returns a string representation of the header
func (h *Header) StringIndented(indent string) string {
if h == nil {
return "nil-Header"
}
return fmt.Sprintf(`Header{
%s ChainID: %v
%s Height: %v
%s Time: %v
%s NumTxs: %v
%s TotalTxs: %v
%s LastBlockID: %v
%s LastCommit: %v
%s Data: %v
%s Validators: %v
%s NextValidators: %v
%s App: %v
%s Consensus: %v
%s Results: %v
%s Evidence: %v
%s Proposer: %v
%s}#%v`,
indent, h.ChainID,
indent, h.Height,
indent, h.Time,
indent, h.NumTxs,
indent, h.TotalTxs,
indent, h.LastBlockID,
indent, h.LastCommitHash,
indent, h.DataHash,
indent, h.ValidatorsHash,
indent, h.NextValidatorsHash,
indent, h.AppHash,
indent, h.ConsensusHash,
indent, h.LastResultsHash,
indent, h.EvidenceHash,
indent, h.ProposerAddress,
indent, h.Hash())
}
//-------------------------------------
// Commit contains the evidence that a block was committed by a set of validators.
// NOTE: Commit is empty for height 1, but never nil.
type Commit struct {
// NOTE: The Precommits are in order of address to preserve the bonded ValidatorSet order.
// Any peer with a block can gossip precommits by index with a peer without recalculating the
// active ValidatorSet.
BlockID BlockID `json:"block_id"`
Precommits []*Vote `json:"precommits"`
// Volatile
firstPrecommit *Vote
hash cmn.HexBytes
bitArray *cmn.BitArray
}
// FirstPrecommit returns the first non-nil precommit in the commit.
// If all precommits are nil, it returns an empty precommit with height 0.
func (commit *Commit) FirstPrecommit() *Vote {
if len(commit.Precommits) == 0 {
return nil
}
if commit.firstPrecommit != nil {
return commit.firstPrecommit
}
for _, precommit := range commit.Precommits {
if precommit != nil {
commit.firstPrecommit = precommit
return precommit
}
}
return &Vote{
Type: VoteTypePrecommit,
}
}
// Height returns the height of the commit
func (commit *Commit) Height() int64 {
if len(commit.Precommits) == 0 {
return 0
}
return commit.FirstPrecommit().Height
}
// Round returns the round of the commit
func (commit *Commit) Round() int {
if len(commit.Precommits) == 0 {
return 0
}
return commit.FirstPrecommit().Round
}
// Type returns the vote type of the commit, which is always VoteTypePrecommit
func (commit *Commit) Type() byte {
return VoteTypePrecommit
}
// Size returns the number of votes in the commit
func (commit *Commit) Size() int {
if commit == nil {
return 0
}
return len(commit.Precommits)
}
// BitArray returns a BitArray of which validators voted in this commit
func (commit *Commit) BitArray() *cmn.BitArray {
if commit.bitArray == nil {
commit.bitArray = cmn.NewBitArray(len(commit.Precommits))
for i, precommit := range commit.Precommits {
// TODO: need to check the BlockID otherwise we could be counting conflicts,
// not just the one with +2/3 !
commit.bitArray.SetIndex(i, precommit != nil)
}
}
return commit.bitArray
}
// GetByIndex returns the vote corresponding to a given validator index
func (commit *Commit) GetByIndex(index int) *Vote {
return commit.Precommits[index]
}
// IsCommit returns true if there is at least one vote
func (commit *Commit) IsCommit() bool {
return len(commit.Precommits) != 0
}
// ValidateBasic performs basic validation that doesn't involve state data.
// Does not actually check the cryptographic signatures.
func (commit *Commit) ValidateBasic() error {
if commit.BlockID.IsZero() {
return errors.New("Commit cannot be for nil block")
}
if len(commit.Precommits) == 0 {
return errors.New("No precommits in commit")
}
height, round := commit.Height(), commit.Round()
// Validate the precommits.
for _, precommit := range commit.Precommits {
// It's OK for precommits to be missing.
if precommit == nil {
continue
}
// Ensure that all votes are precommits.
if precommit.Type != VoteTypePrecommit {
return fmt.Errorf("Invalid commit vote. Expected precommit, got %v",
precommit.Type)
}
// Ensure that all heights are the same.
if precommit.Height != height {
return fmt.Errorf("Invalid commit precommit height. Expected %v, got %v",
height, precommit.Height)
}
// Ensure that all rounds are the same.
if precommit.Round != round {
return fmt.Errorf("Invalid commit precommit round. Expected %v, got %v",
round, precommit.Round)
}
}
return nil
}
// Hash returns the hash of the commit
func (commit *Commit) Hash() cmn.HexBytes {
if commit == nil {
return nil
}
if commit.hash == nil {
bs := make([][]byte, len(commit.Precommits))
for i, precommit := range commit.Precommits {
bs[i] = cdcEncode(precommit)
}
commit.hash = merkle.SimpleHashFromByteSlices(bs)
}
return commit.hash
}
// StringIndented returns a string representation of the commit
func (commit *Commit) StringIndented(indent string) string {
if commit == nil {
return "nil-Commit"
}
precommitStrings := make([]string, len(commit.Precommits))
for i, precommit := range commit.Precommits {
precommitStrings[i] = precommit.String()
}
return fmt.Sprintf(`Commit{
%s BlockID: %v
%s Precommits:
%s %v
%s}#%v`,
indent, commit.BlockID,
indent,
indent, strings.Join(precommitStrings, "\n"+indent+" "),
indent, commit.hash)
}
//-----------------------------------------------------------------------------
// SignedHeader is a header along with the commits that prove it.
type SignedHeader struct {
*Header `json:"header"`
Commit *Commit `json:"commit"`
}
// ValidateBasic does basic consistency checks and makes sure the header
// and commit are consistent.
//
// NOTE: This does not actually check the cryptographic signatures. Make
// sure to use a Verifier to validate the signatures actually provide a
// significantly strong proof for this header's validity.
func (sh SignedHeader) ValidateBasic(chainID string) error {
// Make sure the header is consistent with the commit.
if sh.Header == nil {
return errors.New("SignedHeader missing header.")
}
if sh.Commit == nil {
return errors.New("SignedHeader missing commit (precommit votes).")
}
// Check ChainID.
if sh.ChainID != chainID {
return fmt.Errorf("Header belongs to another chain '%s' not '%s'",
sh.ChainID, chainID)
}
// Check Height.
if sh.Commit.Height() != sh.Height {
return fmt.Errorf("SignedHeader header and commit height mismatch: %v vs %v",
sh.Height, sh.Commit.Height())
}
// Check Hash.
hhash := sh.Hash()
chash := sh.Commit.BlockID.Hash
if !bytes.Equal(hhash, chash) {
return fmt.Errorf("SignedHeader commit signs block %X, header is block %X",
chash, hhash)
}
// ValidateBasic on the Commit.
err := sh.Commit.ValidateBasic()
if err != nil {
return cmn.ErrorWrap(err, "commit.ValidateBasic failed during SignedHeader.ValidateBasic")
}
return nil
}
func (sh SignedHeader) String() string {
return sh.StringIndented("")
}
// StringIndented returns a string representation of the SignedHeader.
func (sh SignedHeader) StringIndented(indent string) string {
return fmt.Sprintf(`SignedHeader{
%s %v
%s %v
%s}`,
indent, sh.Header.StringIndented(indent+" "),
indent, sh.Commit.StringIndented(indent+" "),
indent)
return ""
}
//-----------------------------------------------------------------------------
// Data contains the set of transactions included in the block
type Data struct {
// Txs that will be applied by state @ block.Height+1.
// NOTE: not all txs here are valid. We're just agreeing on the order first.
// This means that block.AppHash does not include these txs.
Txs Txs `json:"txs"`
// Volatile
hash cmn.HexBytes
}
// Hash returns the hash of the data
func (data *Data) Hash() cmn.HexBytes {
if data == nil {
return (Txs{}).Hash()
}
if data.hash == nil {
data.hash = data.Txs.Hash() // NOTE: leaves of merkle tree are TxIDs
}
return data.hash
}
// StringIndented returns a string representation of the transactions
func (data *Data) StringIndented(indent string) string {
if data == nil {
return "nil-Data"
}
txStrings := make([]string, cmn.MinInt(len(data.Txs), 21))
for i, tx := range data.Txs {
if i == 20 {
txStrings[i] = fmt.Sprintf("... (%v total)", len(data.Txs))
break
}
txStrings[i] = fmt.Sprintf("%X (%d bytes)", tx.Hash(), len(tx))
}
return fmt.Sprintf(`Data{
%s %v
%s}#%v`,
indent, strings.Join(txStrings, "\n"+indent+" "),
indent, data.hash)
}
//-----------------------------------------------------------------------------
// EvidenceData contains any evidence of malicious wrong-doing by validators
type EvidenceData struct {
Evidence EvidenceList `json:"evidence"`
// Volatile
hash cmn.HexBytes
}
// Hash returns the hash of the data.
func (data *EvidenceData) Hash() cmn.HexBytes {
if data.hash == nil {
data.hash = data.Evidence.Hash()
}
return data.hash
}
// StringIndented returns a string representation of the evidence.
func (data *EvidenceData) StringIndented(indent string) string {
if data == nil {
return "nil-Evidence"
}
evStrings := make([]string, cmn.MinInt(len(data.Evidence), 21))
for i, ev := range data.Evidence {
if i == 20 {
evStrings[i] = fmt.Sprintf("... (%v total)", len(data.Evidence))
break
}
evStrings[i] = fmt.Sprintf("Evidence:%v", ev)
}
return fmt.Sprintf(`EvidenceData{
%s %v
%s}#%v`,
indent, strings.Join(evStrings, "\n"+indent+" "),
indent, data.hash)
return ""
}
//--------------------------------------------------------------------------------
// BlockID defines the unique ID of a block as its Hash and its PartSetHeader
type BlockID struct {
Hash cmn.HexBytes `json:"hash"`
PartsHeader PartSetHeader `json:"parts"`
}
// IsZero returns true if this is the BlockID for a nil-block
func (blockID BlockID) IsZero() bool {
return len(blockID.Hash) == 0 && blockID.PartsHeader.IsZero()
}
// Equals returns true if the BlockID matches the given BlockID
func (blockID BlockID) Equals(other BlockID) bool {
return bytes.Equal(blockID.Hash, other.Hash) &&
blockID.PartsHeader.Equals(other.PartsHeader)
}
// Key returns a machine-readable string representation of the BlockID
func (blockID BlockID) Key() string {
bz, err := cdc.MarshalBinaryBare(blockID.PartsHeader)
if err != nil {
panic(err)
}
return string(blockID.Hash) + string(bz)
}
// String returns a human readable string representation of the BlockID
func (blockID BlockID) String() string {
return fmt.Sprintf(`%v:%v`, blockID.Hash, blockID.PartsHeader)
}