OSDN Git Service

clean
[bytom/vapor.git] / protocol / bbft.go
index 9315ca8..4fdb607 100644 (file)
@@ -2,239 +2,232 @@ package protocol
 
 import (
        "encoding/hex"
-       "time"
+       "fmt"
 
-       "github.com/vapor/crypto/ed25519"
-       "github.com/vapor/crypto/ed25519/chainkd"
+       log "github.com/sirupsen/logrus"
+
+       "github.com/vapor/config"
        "github.com/vapor/errors"
-       "github.com/vapor/math/checked"
+       "github.com/vapor/protocol/bc"
        "github.com/vapor/protocol/bc/types"
        "github.com/vapor/protocol/state"
+       "github.com/vapor/event"
+)
+
+const (
+       maxSignatureCacheSize = 10000
 )
 
 var (
        errVotingOperationOverFlow = errors.New("voting operation result overflow")
+       errDoubleSignBlock         = errors.New("the consensus is double sign in same height of different block")
+       errInvalidSignature        = errors.New("the signature of block is invalid")
 )
 
-type bbft struct {
-       consensusNodeManager *consensusNodeManager
+func signCacheKey(blockHash, pubkey string) string {
+       return fmt.Sprintf("%s:%s", blockHash, pubkey)
 }
 
-func newBbft(store Store, blockIndex *state.BlockIndex) *bbft {
-       return &bbft{
-               consensusNodeManager: newConsensusNodeManager(store, blockIndex),
-       }
-}
-
-// 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 (b *bbft) isIrreversible(block *types.Block) bool {
-       signNum, err := b.validateSign(block)
+func (c *Chain) isIrreversible(blockNode *state.BlockNode) bool {
+       consensusNodes, err := c.consensusNodeManager.getConsensusNodes(&blockNode.Parent.Hash)
        if err != nil {
                return false
        }
 
-       return signNum > (numOfConsensusNode * 2 / 3)
+       signCount := 0
+       for i := 0; i < len(consensusNodes); i++ {
+               if ok, _ := blockNode.BlockWitness.Test(uint32(i)); ok {
+                       signCount++
+               }
+       }
+
+       return signCount > len(consensusNodes)*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)
+func (c *Chain) IsBlocker(prevBlockHash *bc.Hash, pubkey string, timeStamp uint64) (bool, error) {
+       return c.consensusNodeManager.isBlocker(prevBlockHash, pubkey, timeStamp)
 }
 
-func (b *bbft) ApplyBlock(voteResultMap map[uint64]*state.VoteResult, block *types.Block) (err error) {
-       voteSeq := block.Height / roundVoteBlockNums
-       voteResult := voteResultMap[voteSeq]
-
-       if voteResult == nil {
-               store := b.consensusNodeManager.store
-               voteResult, err = store.GetVoteResult(voteSeq)
-               if err != nil && err != ErrNotFoundVoteResult {
-                       return err
-               }
+// 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 []byte, xPub [64]byte, blockHash *bc.Hash) error {
+       xpubStr := hex.EncodeToString(xPub[:])
+       blockNode := c.index.GetNode(blockHash)
+       // save the signature if the block is not exist
+       if blockNode == nil {
+               cacheKey := signCacheKey(blockHash.String(), xpubStr)
+               c.signatureCache.Add(cacheKey, signature)
+               return nil
        }
 
-       if voteResult == nil {
-               voteResult = &state.VoteResult{
-                       Seq:             voteSeq,
-                       NumOfVote:       make(map[string]uint64),
-                       LastBlockHeight: block.Height,
-               }
+       consensusNode, err := c.consensusNodeManager.getConsensusNode(&blockNode.Parent.Hash, xpubStr)
+       if err != nil {
+               return err
        }
 
-       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")
+       if exist, err := blockNode.BlockWitness.Test(uint32(consensusNode.Order)); err != nil && exist {
+               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.SubUint64(voteResult.NumOfVote[pubkey], unVoteInput.Amount)
-                       if !ok {
-                               return errVotingOperationOverFlow
-                       }
-               }
-               for _, output := range tx.Outputs {
-                       voteOutput, ok := output.TypedOutput.(*types.VoteTxOutput)
-                       if !ok {
-                               continue
-                       }
-
-                       pubkey := hex.EncodeToString(voteOutput.Vote)
-                       voteResult.NumOfVote[pubkey], ok = checked.AddUint64(voteResult.NumOfVote[pubkey], voteOutput.Amount)
-                       if !ok {
-                               return errVotingOperationOverFlow
-                       }
-               }
+       if !consensusNode.XPub.Verify(blockHash.Bytes(), signature) {
+               return errInvalidSignature
        }
 
-       voteResult.LastBlockHeight++
-       voteResult.Finalized = (block.Height+1)%roundVoteBlockNums == 0
-       return nil
-}
-
-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
+       isDoubleSign, err := c.checkDoubleSign(consensusNode.Order, blockNode.Height, *blockHash)
+       if 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 isDoubleSign {
+               return errDoubleSignBlock
        }
 
-       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
-                       }
-               }
-               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
-                       }
-               }
+       if err := c.updateBlockSignature(&blockNode.Hash, consensusNode.Order, signature); err != nil {
+               return err
        }
 
-       voteResult.LastBlockHeight--
-       voteResult.Finalized = false
-       return nil
-}
+       if c.isIrreversible(blockNode) && blockNode.Height > c.bestIrreversibleNode.Height {
+               bestIrreversibleNode := c.index.GetNode(blockHash)
+               if err := c.store.SaveChainNodeStatus(c.bestNode, bestIrreversibleNode); err != nil {
+                       return err
+               }
 
-// ValidateBlock verify whether the block is valid
-func (b *bbft) ValidateBlock(block *types.Block) error {
-       signNum, err := b.validateSign(block)
-       if err != nil {
-               return err
+               c.bestIrreversibleNode = bestIrreversibleNode
        }
 
-       if signNum == 0 {
-               return errors.New("no valid signature")
-       }
-       return nil
+       return c.eventDispatcher.Post(event.BlockSignatureEvent{BlockHash: *blockHash, Signature: signature, XPub: xPub})
 }
 
 // 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)
+func (c *Chain) validateSign(block *types.Block) (uint64, error) {
+       consensusNodeMap, err := c.consensusNodeManager.getConsensusNodes(&block.PreviousBlockHash)
        if err != nil {
                return 0, err
        }
 
        hasBlockerSign := false
-       for pubkey, node := range consensusNodeMap {
-               if len(block.Witness) <= int(node.order) {
+       signCount := uint64(0)
+       blockHash := block.Hash()
+       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() {
+               if block.Witness[node.Order] == nil {
+                       cachekey := signCacheKey(blockHash.String(), pubKey)
+                       if signature, ok := c.signatureCache.Get(cachekey); ok {
+                               block.Witness[node.Order] = signature.([]byte)
+                       } 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 ok := node.XPub.Verify(blockHash.Bytes(), block.Witness[node.Order]); !ok {
+                       block.Witness[node.Order] = nil
+                       continue
+               }
+
+               isDoubleSign, err := c.checkDoubleSign(node.Order, block.Height, block.Hash())
+               if err != nil {
+                       return 0, err
                }
+
+               if isDoubleSign {
+                       // Consensus node is signed twice with the same block height, discard the signature
+                       log.WithFields(log.Fields{"module": logModule, "blockHash": blockHash.String(), "pubKey": pubKey}).Warn("the consensus node double sign the same height of different block")
+                       block.Witness[node.Order] = nil
+                       continue
+               }
+
+               signCount++
+               isBlocker, err := c.consensusNodeManager.isBlocker(&block.PreviousBlockHash, pubKey, block.Timestamp)
+               if err != nil {
+                       return 0, err
+               }
+
+               if isBlocker {
+                       hasBlockerSign = true
+               }
+
        }
+
        if !hasBlockerSign {
                return 0, errors.New("the block has no signature of the blocker")
        }
-       return correctSignNum, nil
+       return signCount, nil
+}
+
+func (c *Chain) checkDoubleSign(nodeOrder, blockHeight uint64, blockHash bc.Hash) (bool, error) {
+       blockNodes := c.consensusNodeManager.blockIndex.NodesByHeight(blockHeight)
+       for _, blockNode := range blockNodes {
+               if blockNode.Hash == blockHash {
+                       continue
+               }
+               if ok, err := blockNode.BlockWitness.Test(uint32(nodeOrder)); err != nil && ok {
+                       if err := c.updateBlockSignature(&blockHash, nodeOrder, nil); err != nil {
+                               return false, err
+                       }
+
+                       return true, nil
+               }
+       }
+       return false, nil
 }
 
 // SignBlock signing the block if current node is consensus node
-func (b *bbft) SignBlock(block *types.Block) error {
-       var xprv chainkd.XPrv
+func (c *Chain) SignBlock(block *types.Block) ([]byte, error) {
+       xprv := config.CommonConfig.PrivateKey()
        xpub := [64]byte(xprv.XPub())
-       node, err := b.consensusNodeManager.getConsensusNode(block.Height, hex.EncodeToString(xpub[:]))
+       node, err := c.consensusNodeManager.getConsensusNode(&block.PreviousBlockHash, hex.EncodeToString(xpub[:]))
        if err != nil && err != errNotFoundConsensusNode {
-               return err
+               return nil, err
        }
 
        if node == nil {
-               return nil
+               return nil, nil
        }
 
-       block.Witness[node.order] = xprv.Sign(block.Hash().Bytes())
-       return nil
+       blockNodes := c.consensusNodeManager.blockIndex.NodesByHeight(block.Height)
+       for _, blockNode := range blockNodes {
+               // Has already signed the same height block
+               if ok, err := blockNode.BlockWitness.Test(uint32(node.Order)); err != nil && ok {
+                       return nil, nil
+               }
+       }
+
+       signature := block.Witness[node.Order]
+       if len(signature) == 0 {
+               signature = xprv.Sign(block.Hash().Bytes())
+               block.Witness[node.Order] = signature
+       }
+       return signature, nil
 }
 
-// UpdateConsensusNodes used to update consensus node after each round of voting
-func (b *bbft) UpdateConsensusNodes(blockHeight uint64) error {
-       return b.consensusNodeManager.updateConsensusNodes(blockHeight)
+func (c *Chain) updateBlockSignature(blockHash *bc.Hash, nodeOrder uint64, signature []byte) error {
+       blockNode := c.consensusNodeManager.blockIndex.GetNode(blockHash)
+       if len(signature) != 0 {
+               if err := blockNode.BlockWitness.Set(uint32(nodeOrder)); err != nil {
+                       return err
+               }
+       } else {
+               if err := blockNode.BlockWitness.Clean(uint32(nodeOrder)); err != nil {
+                       return err
+               }
+       }
+
+       block, err := c.store.GetBlock(blockHash)
+       if err != nil {
+               return err
+       }
+
+       block.Witness[nodeOrder] = signature
+       txStatus, err := c.consensusNodeManager.store.GetTransactionStatus(blockHash)
+       if err != nil {
+               return err
+       }
+
+       return c.consensusNodeManager.store.SaveBlock(block, txStatus)
 }