OSDN Git Service

rename (#465)
[bytom/vapor.git] / protocol / bbft.go
index 9315ca8..d62b3ed 100644 (file)
@@ -2,239 +2,224 @@ package protocol
 
 import (
        "encoding/hex"
-       "time"
-
-       "github.com/vapor/crypto/ed25519"
-       "github.com/vapor/crypto/ed25519/chainkd"
-       "github.com/vapor/errors"
-       "github.com/vapor/math/checked"
-       "github.com/vapor/protocol/bc/types"
-       "github.com/vapor/protocol/state"
-)
+       "fmt"
 
-var (
-       errVotingOperationOverFlow = errors.New("voting operation result overflow")
+       log "github.com/sirupsen/logrus"
+
+       "github.com/bytom/vapor/config"
+       "github.com/bytom/vapor/crypto/ed25519/chainkd"
+       "github.com/bytom/vapor/errors"
+       "github.com/bytom/vapor/event"
+       "github.com/bytom/vapor/protocol/bc"
+       "github.com/bytom/vapor/protocol/bc/types"
+       "github.com/bytom/vapor/protocol/state"
 )
 
-type bbft struct {
-       consensusNodeManager *consensusNodeManager
-}
+const (
+       maxSignatureCacheSize = 10000
+)
 
-func newBbft(store Store, blockIndex *state.BlockIndex) *bbft {
-       return &bbft{
-               consensusNodeManager: newConsensusNodeManager(store, blockIndex),
-       }
-}
+var (
+       errDoubleSignBlock  = errors.New("the consensus is double sign in same height of different block")
+       errInvalidSignature = errors.New("the signature of block is invalid")
+       errSignForkChain    = errors.New("can not sign fork before the irreversible block")
+)
 
-// IsConsensusPubkey determine whether a public key is a consensus node at a specified height
-func (b *bbft) IsConsensusPubkey(height uint64, pubkey []byte) (bool, error) {
-       node, err := b.consensusNodeManager.getConsensusNode(height, hex.EncodeToString(pubkey))
-       if err != nil && err != errNotFoundConsensusNode {
-               return false, err
-       }
-       return node != nil, nil
+func signCacheKey(blockHash, pubkey string) string {
+       return fmt.Sprintf("%s:%s", blockHash, pubkey)
 }
 
-func (b *bbft) isIrreversible(block *types.Block) bool {
-       signNum, err := b.validateSign(block)
+func (c *Chain) checkDoubleSign(bh *types.BlockHeader, xPub string) error {
+       blockHashes, err := c.store.GetBlockHashesByHeight(bh.Height)
        if err != nil {
-               return false
+               return err
        }
 
-       return signNum > (numOfConsensusNode * 2 / 3)
-}
-
-// NextLeaderTime returns the start time of the specified public key as the next leader node
-func (b *bbft) NextLeaderTime(pubkey []byte, bestBlockTimestamp, bestBlockHeight uint64) (*time.Time, error) {
-       return b.consensusNodeManager.nextLeaderTime(pubkey, bestBlockTimestamp, bestBlockHeight)
-}
+       for _, blockHash := range blockHashes {
+               if *blockHash == bh.Hash() {
+                       continue
+               }
 
-func (b *bbft) ApplyBlock(voteResultMap map[uint64]*state.VoteResult, block *types.Block) (err error) {
-       voteSeq := block.Height / roundVoteBlockNums
-       voteResult := voteResultMap[voteSeq]
+               blockHeader, err := c.store.GetBlockHeader(blockHash)
+               if err != nil {
+                       return err
+               }
 
-       if voteResult == nil {
-               store := b.consensusNodeManager.store
-               voteResult, err = store.GetVoteResult(voteSeq)
-               if err != nil && err != ErrNotFoundVoteResult {
+               consensusNode, err := c.getConsensusNode(&blockHeader.PreviousBlockHash, xPub)
+               if err == errNotFoundConsensusNode {
+                       continue
+               } else if err != nil {
                        return err
                }
-       }
 
-       if voteResult == nil {
-               voteResult = &state.VoteResult{
-                       Seq:             voteSeq,
-                       NumOfVote:       make(map[string]uint64),
-                       LastBlockHeight: block.Height,
+               if blockHeader.BlockWitness.Get(consensusNode.Order) != nil {
+                       return errDoubleSignBlock
                }
        }
+       return nil
+}
 
-       voteResultMap[voteSeq] = voteResult
-
-       if voteResult.LastBlockHeight+1 != block.Height {
-               return errors.New("bbft append block error, the block height is not equals last block height plus 1 of vote result")
+func (c *Chain) checkNodeSign(bh *types.BlockHeader, consensusNode *state.ConsensusNode, signature []byte) error {
+       if !consensusNode.XPub.Verify(bh.Hash().Bytes(), signature) {
+               return errInvalidSignature
        }
 
-       for _, tx := range block.Transactions {
-               for _, input := range tx.Inputs {
-                       unVoteInput, ok := input.TypedInput.(*types.UnvoteInput)
-                       if !ok {
-                               continue
-                       }
-                       
-                       pubkey := hex.EncodeToString(unVoteInput.Vote)
-                       voteResult.NumOfVote[pubkey], ok = checked.SubUint64(voteResult.NumOfVote[pubkey], unVoteInput.Amount)
-                       if !ok {
-                               return errVotingOperationOverFlow
-                       }
-               }
-               for _, output := range tx.Outputs {
-                       voteOutput, ok := output.TypedOutput.(*types.VoteTxOutput)
-                       if !ok {
-                               continue
-                       }
+       return c.checkDoubleSign(bh, consensusNode.XPub.String())
+}
 
-                       pubkey := hex.EncodeToString(voteOutput.Vote)
-                       voteResult.NumOfVote[pubkey], ok = checked.AddUint64(voteResult.NumOfVote[pubkey], voteOutput.Amount)
-                       if !ok {
-                               return errVotingOperationOverFlow
-                       }
+func (c *Chain) isIrreversible(blockHeader *types.BlockHeader) bool {
+       consensusNodes, err := c.getConsensusNodes(&blockHeader.PreviousBlockHash)
+       if err != nil {
+               return false
+       }
+
+       signCount := 0
+       for i := 0; i < len(consensusNodes); i++ {
+               if blockHeader.BlockWitness.Get(uint64(i)) != nil {
+                       signCount++
                }
        }
 
-       voteResult.LastBlockHeight++
-       voteResult.Finalized = (block.Height+1)%roundVoteBlockNums == 0
-       return nil
+       return signCount > len(consensusNodes)*2/3
 }
 
-func (b *bbft) DetachBlock(voteResultMap map[uint64]*state.VoteResult, block *types.Block) error {
-       voteSeq := block.Height / roundVoteBlockNums
-       voteResult := voteResultMap[voteSeq]
-
-       if voteResult == nil {
-               store := b.consensusNodeManager.store
-               voteResult, err := store.GetVoteResult(voteSeq)
-               if err != nil {
-                       return err
-               }
-               voteResultMap[voteSeq] = voteResult
+func (c *Chain) updateBlockSignature(blockHeader *types.BlockHeader, nodeOrder uint64, signature []byte) error {
+       blockHeader.Set(nodeOrder, signature)
+       if err := c.store.SaveBlockHeader(blockHeader); err != nil {
+               return err
        }
 
-       if voteResult.LastBlockHeight != block.Height {
-               return errors.New("bbft detach block error, the block height is not equals last block height of vote result")
+       if !c.isIrreversible(blockHeader) || blockHeader.Height <= c.lastIrrBlockHeader.Height {
+               return nil
        }
 
-       for _, tx := range block.Transactions {
-               for _, input := range tx.Inputs {
-                       unVoteInput, ok := input.TypedInput.(*types.UnvoteInput)
-                       if !ok {
-                               continue
-                       }
-                       
-                       pubkey := hex.EncodeToString(unVoteInput.Vote)
-                       voteResult.NumOfVote[pubkey], ok = checked.AddUint64(voteResult.NumOfVote[pubkey], unVoteInput.Amount)
-                       if !ok {
-                               return errVotingOperationOverFlow
-                       }
+       if c.InMainChain(blockHeader.Hash()) {
+               if err := c.store.SaveChainStatus(c.bestBlockHeader, blockHeader, []*types.BlockHeader{}, state.NewUtxoViewpoint(), []*state.ConsensusResult{}); err != nil {
+                       return err
                }
-               for _, output := range tx.Outputs {
-                       voteOutput, ok := output.TypedOutput.(*types.VoteTxOutput)
-                       if !ok {
-                               continue
-                       }
-                       
-                       pubkey := hex.EncodeToString(voteOutput.Vote)
-                       voteResult.NumOfVote[pubkey], ok = checked.SubUint64(voteResult.NumOfVote[pubkey], voteOutput.Amount)
-                       if !ok {
-                               return errVotingOperationOverFlow
-                       }
+
+               c.lastIrrBlockHeader = blockHeader
+       } else {
+               // block is on a forked chain
+               log.WithFields(log.Fields{"module": logModule}).Info("majority votes received on forked chain")
+               tail, err := c.traceLongestChainTail(blockHeader)
+               if err != nil {
+                       return err
                }
-       }
 
-       voteResult.LastBlockHeight--
-       voteResult.Finalized = false
+               return c.reorganizeChain(tail)
+       }
        return nil
 }
 
-// ValidateBlock verify whether the block is valid
-func (b *bbft) ValidateBlock(block *types.Block) error {
-       signNum, err := b.validateSign(block)
+// validateSign verify the signatures of block, and return the number of correct signature
+// if some signature is invalid, they will be reset to nil
+// if the block does not have the signature of blocker, it will return error
+func (c *Chain) validateSign(block *types.Block) error {
+       consensusNodeMap, err := c.getConsensusNodes(&block.PreviousBlockHash)
        if err != nil {
                return err
        }
 
-       if signNum == 0 {
-               return errors.New("no valid signature")
-       }
-       return nil
-}
-
-// validateSign verify the signatures of block, and return the number of correct signature
-// if some signature is invalid, they will be reset to nil
-// if the block has not the signature of blocker, it will return error
-func (b *bbft) validateSign(block *types.Block) (uint64, error) {
-       var correctSignNum uint64
-       consensusNodeMap, err := b.consensusNodeManager.getConsensusNodesByVoteResult(block.Height)
+       blocker, err := c.GetBlocker(&block.PreviousBlockHash, block.Timestamp)
        if err != nil {
-               return 0, err
+               return err
        }
 
        hasBlockerSign := false
-       for pubkey, node := range consensusNodeMap {
-               if len(block.Witness) <= int(node.order) {
-                       continue
-               }
-
-               blocks := b.consensusNodeManager.blockIndex.NodesByHeight(block.Height)
-               for _, b := range blocks {
-                       if b.Hash == block.Hash() {
+       blockHash := block.Hash()
+       for pubKey, node := range consensusNodeMap {
+               if block.BlockWitness.Get(node.Order) == nil {
+                       cachekey := signCacheKey(blockHash.String(), pubKey)
+                       if signature, ok := c.signatureCache.Get(cachekey); ok {
+                               block.Set(node.Order, signature.([]byte))
+                               c.eventDispatcher.Post(event.BlockSignatureEvent{BlockHash: blockHash, Signature: signature.([]byte), XPub: node.XPub[:]})
+                               c.signatureCache.Remove(cachekey)
+                       } else {
                                continue
                        }
-                       if ok, err := b.BlockWitness.Test(uint32(node.order)); err != nil && ok {
-                               // Consensus node is signed twice with the same block height, discard the signature
-                               block.Witness[node.order] = nil
-                               break
-                       }
                }
 
-               if ed25519.Verify(ed25519.PublicKey(pubkey), block.Hash().Bytes(), block.Witness[node.order]) {
-                       correctSignNum++
-                       isBlocker, err := b.consensusNodeManager.isBlocker(block.Height, block.Timestamp, pubkey)
-                       if err != nil {
-                               return 0, err
-                       }
-                       if isBlocker {
-                               hasBlockerSign = true
-                       }
-               } else {
-                       // discard the invalid signature
-                       block.Witness[node.order] = nil
+               if err := c.checkNodeSign(&block.BlockHeader, node, block.Get(node.Order)); err == errDoubleSignBlock {
+                       log.WithFields(log.Fields{"module": logModule, "blockHash": blockHash.String(), "pubKey": pubKey}).Warn("the consensus node double sign the same height of different block")
+                       block.BlockWitness.Delete(node.Order)
+                       continue
+               } else if err != nil {
+                       return err
+               }
+
+               if blocker == pubKey {
+                       hasBlockerSign = true
                }
        }
+
        if !hasBlockerSign {
-               return 0, errors.New("the block has no signature of the blocker")
+               return errors.New("the block has no signature of the blocker")
        }
-       return correctSignNum, nil
+       return nil
 }
 
-// SignBlock signing the block if current node is consensus node
-func (b *bbft) SignBlock(block *types.Block) error {
-       var xprv chainkd.XPrv
-       xpub := [64]byte(xprv.XPub())
-       node, err := b.consensusNodeManager.getConsensusNode(block.Height, hex.EncodeToString(xpub[:]))
-       if err != nil && err != errNotFoundConsensusNode {
+// ProcessBlockSignature process the received block signature messages
+// return whether a block become irreversible, if so, the chain module must update status
+func (c *Chain) ProcessBlockSignature(signature, xPub []byte, blockHash *bc.Hash) error {
+       xpubStr := hex.EncodeToString(xPub[:])
+       blockHeader, _ := c.store.GetBlockHeader(blockHash)
+
+       // save the signature if the block is not exist
+       if blockHeader == nil {
+               var xPubKey chainkd.XPub
+               copy(xPubKey[:], xPub[:])
+               if !xPubKey.Verify(blockHash.Bytes(), signature) {
+                       return errInvalidSignature
+               }
+
+               cacheKey := signCacheKey(blockHash.String(), xpubStr)
+               c.signatureCache.Add(cacheKey, signature)
+               return nil
+       }
+
+       consensusNode, err := c.getConsensusNode(&blockHeader.PreviousBlockHash, xpubStr)
+       if err != nil {
                return err
        }
 
-       if node == nil {
+       if blockHeader.BlockWitness.Get(consensusNode.Order) != nil {
                return nil
        }
 
-       block.Witness[node.order] = xprv.Sign(block.Hash().Bytes())
-       return nil
+       c.cond.L.Lock()
+       defer c.cond.L.Unlock()
+       if err := c.checkNodeSign(blockHeader, consensusNode, signature); err != nil {
+               return err
+       }
+
+       if err := c.updateBlockSignature(blockHeader, consensusNode.Order, signature); err != nil {
+               return err
+       }
+       return c.eventDispatcher.Post(event.BlockSignatureEvent{BlockHash: *blockHash, Signature: signature, XPub: xPub})
 }
 
-// UpdateConsensusNodes used to update consensus node after each round of voting
-func (b *bbft) UpdateConsensusNodes(blockHeight uint64) error {
-       return b.consensusNodeManager.updateConsensusNodes(blockHeight)
+// SignBlock signing the block if current node is consensus node
+func (c *Chain) SignBlock(block *types.Block) ([]byte, error) {
+       xprv := config.CommonConfig.PrivateKey()
+       xpubStr := xprv.XPub().String()
+       node, err := c.getConsensusNode(&block.PreviousBlockHash, xpubStr)
+       if err == errNotFoundConsensusNode {
+               return nil, nil
+       } else if err != nil {
+               return nil, err
+       }
+
+       if err := c.checkDoubleSign(&block.BlockHeader, node.XPub.String()); err == errDoubleSignBlock {
+               return nil, nil
+       } else if err != nil {
+               return nil, err
+       }
+
+       signature := block.Get(node.Order)
+       if len(signature) == 0 {
+               signature = xprv.Sign(block.Hash().Bytes())
+               block.Set(node.Order, signature)
+       }
+       return signature, nil
 }