X-Git-Url: http://git.osdn.net/view?a=blobdiff_plain;f=protocol%2Fconsensus_node_manager.go;h=5bd9ee3aca8f503b639fa9618b7abd055856e0ec;hb=c748b385a655fd8b1b1b30fb6bd5dda030649318;hp=be5e6bb92df04d62c0de862d8ee06f1373834876;hpb=2d27cad41ef79da3a46142401b5efcf92133ea1a;p=bytom%2Fvapor.git diff --git a/protocol/consensus_node_manager.go b/protocol/consensus_node_manager.go index be5e6bb9..5bd9ee3a 100644 --- a/protocol/consensus_node_manager.go +++ b/protocol/consensus_node_manager.go @@ -1,204 +1,172 @@ package protocol import ( - "encoding/hex" - "sort" - "sync" - "time" - + "github.com/vapor/config" + "github.com/vapor/consensus" "github.com/vapor/errors" + "github.com/vapor/protocol/bc" "github.com/vapor/protocol/state" ) -const ( - numOfConsensusNode = 21 - roundVoteBlockNums = 1000 - - // BlockTimeInterval indicate product one block per 500 milliseconds - BlockTimeInterval = 500 - BlockNumEachNode = 3 -) - var ( - errHasNoChanceProductBlock = errors.New("the node has no chance to product a block in this round of voting") - errNotFoundConsensusNode = errors.New("can not found consensus node") - errVoteResultIsNotfinalized = errors.New("vote result is not finalized") - errPublicKeyIsNotConsensusNode = errors.New("public key is not consensus node") + errNotFoundConsensusNode = errors.New("can not found consensus node") + errNotFoundBlockNode = errors.New("can not find block node") ) -type consensusNode struct { - pubkey string - voteNum uint64 - order uint64 -} - -type consensusNodeSlice []*consensusNode - -func (c consensusNodeSlice) Len() int { return len(c) } -func (c consensusNodeSlice) Less(i, j int) bool { return c[i].voteNum > c[j].voteNum } -func (c consensusNodeSlice) Swap(i, j int) { c[i], c[j] = c[j], c[i] } - type consensusNodeManager struct { - consensusNodeMap map[string]*consensusNode - effectiveStartHeight uint64 - store Store - blockIndex *state.BlockIndex - sync.RWMutex + store Store + blockIndex *state.BlockIndex } func newConsensusNodeManager(store Store, blockIndex *state.BlockIndex) *consensusNodeManager { return &consensusNodeManager{ - consensusNodeMap: make(map[string]*consensusNode), - effectiveStartHeight: 1, - store: store, - blockIndex: blockIndex, + store: store, + blockIndex: blockIndex, } } -func (c *consensusNodeManager) getConsensusNode(height uint64, pubkey string) (*consensusNode, error) { - defer c.RUnlock() - c.RLock() - if height >= c.effectiveStartHeight+roundVoteBlockNums { - return nil, errors.New("the vote has not been completed for the specified block height ") - } - - var err error - consensusNodeMap := c.consensusNodeMap - // query history vote result - if height < c.effectiveStartHeight { - consensusNodeMap, err = c.getConsensusNodesByVoteResult(height) - if err != nil { - return nil, err - } +func (c *consensusNodeManager) getConsensusNode(prevBlockHash *bc.Hash, pubkey string) (*state.ConsensusNode, error) { + consensusNodeMap, err := c.getConsensusNodes(prevBlockHash) + if err != nil { + return nil, err } node, exist := consensusNodeMap[pubkey] if !exist { - return node, errNotFoundConsensusNode + return nil, errNotFoundConsensusNode } return node, nil } -func (c *consensusNodeManager) isBlocker(height uint64, blockTimestamp uint64, pubkey string) (bool, error) { - prevVoteRoundLastBlock := c.blockIndex.NodeByHeight(height - 1) - startTimestamp := prevVoteRoundLastBlock.Timestamp + BlockTimeInterval - - consensusNodeMap, err := c.getConsensusNodesByVoteResult(height) +func (c *consensusNodeManager) isBlocker(prevBlockHash *bc.Hash, pubKey string, timeStamp uint64) (bool, error) { + consensusNodeMap, err := c.getConsensusNodes(prevBlockHash) if err != nil { return false, err } - blockerNode, exist := consensusNodeMap[pubkey] - if !exist { + consensusNode := consensusNodeMap[pubKey] + if consensusNode == nil { return false, nil } - begin := getLastBlockTimeInTimeRange(startTimestamp, blockTimestamp, blockerNode.order) - end := begin + BlockNumEachNode*BlockTimeInterval - return blockTimestamp >= begin && blockTimestamp < end, nil -} - -func (c *consensusNodeManager) nextLeaderTimeRange(pubkey []byte, bestBlockTimestamp, bestBlockHeight uint64) (uint64, uint64, error) { - defer c.RUnlock() - c.RLock() - - startHeight := c.effectiveStartHeight - prevRoundLastBlock := c.blockIndex.NodeByHeight(startHeight - 1) - startTime := prevRoundLastBlock.Timestamp + BlockTimeInterval - endTime := bestBlockTimestamp + (roundVoteBlockNums-bestBlockHeight%roundVoteBlockNums)*BlockTimeInterval - - consensusNode, exist := c.consensusNodeMap[hex.EncodeToString(pubkey)] - if !exist { - return 0, 0, errPublicKeyIsNotConsensusNode - } - - nextLeaderTime, err := nextLeaderTimeHelper(startTime, endTime, uint64(time.Now().UnixNano()/1e6), consensusNode.order) + prevVoteRoundLastBlock, err := c.getPrevRoundLastBlock(prevBlockHash) if err != nil { - return 0, 0, err + return false, err } - return nextLeaderTime, nextLeaderTime + BlockNumEachNode*BlockTimeInterval, nil + startTimestamp := prevVoteRoundLastBlock.Timestamp + consensus.BlockTimeInterval + begin := getLastBlockTimeInTimeRange(startTimestamp, timeStamp, consensusNode.Order, uint64(len(consensusNodeMap))) + end := begin + consensus.BlockNumEachNode*consensus.BlockTimeInterval + return timeStamp >= begin && timeStamp < end, nil } -func nextLeaderTimeHelper(startTime, endTime, now, nodeOrder uint64) (uint64, error) { - nextLeaderTimestamp := getLastBlockTimeInTimeRange(startTime, now, nodeOrder) - roundBlockTime := uint64(BlockNumEachNode * numOfConsensusNode * BlockTimeInterval) +func getLastBlockTimeInTimeRange(startTimestamp, endTimestamp, order, numOfConsensusNode uint64) uint64 { + // One round of product block time for all consensus nodes + roundBlockTime := consensus.BlockNumEachNode * numOfConsensusNode * consensus.BlockTimeInterval + // The start time of the last round of product block + lastRoundStartTime := startTimestamp + (endTimestamp-startTimestamp)/roundBlockTime*roundBlockTime + // The time of product block of the consensus in last round + return lastRoundStartTime + order*(consensus.BlockNumEachNode*consensus.BlockTimeInterval) +} - if int64(now-nextLeaderTimestamp) >= BlockNumEachNode*BlockTimeInterval { - nextLeaderTimestamp += roundBlockTime - if nextLeaderTimestamp >= endTime { - return 0, errHasNoChanceProductBlock - } +func (c *consensusNodeManager) getPrevRoundLastBlock(prevBlockHash *bc.Hash) (*state.BlockNode, error) { + node := c.blockIndex.GetNode(prevBlockHash) + if node == nil { + return nil, errNotFoundBlockNode } - return nextLeaderTimestamp, nil + for node.Height%consensus.RoundVoteBlockNums != 0 { + node = node.Parent + } + return node, nil } -// updateConsensusNodes used to update consensus node after each round of voting -func (c *consensusNodeManager) updateConsensusNodes(bestBlockHeight uint64) error { - defer c.Unlock() - c.Lock() - - consensusNodeMap, err := c.getConsensusNodesByVoteResult(bestBlockHeight) - if err != nil && err != errVoteResultIsNotfinalized { - return err +func (c *consensusNodeManager) getConsensusNodes(prevBlockHash *bc.Hash) (map[string]*state.ConsensusNode, error) { + prevBlockNode := c.blockIndex.GetNode(prevBlockHash) + if prevBlockNode == nil { + return nil, errNotFoundBlockNode } - if err == errVoteResultIsNotfinalized { - return nil + preSeq := state.CalcVoteSeq(prevBlockNode.Height + 1) - 1 + if bestSeq := state.CalcVoteSeq(c.blockIndex.BestNode().Height); preSeq > bestSeq { + preSeq = bestSeq } - c.consensusNodeMap = consensusNodeMap - c.effectiveStartHeight = bestBlockHeight / roundVoteBlockNums * roundVoteBlockNums - return nil -} + voteResult, err := c.store.GetVoteResult(preSeq) + if err != nil { + return nil, err + } -func getLastBlockTimeInTimeRange(startTimestamp, endTimestamp, order uint64) uint64 { - // One round of product block time for all consensus nodes - roundBlockTime := uint64(BlockNumEachNode * numOfConsensusNode * BlockTimeInterval) - // The start time of the last round of product block - lastRoundStartTime := startTimestamp + (endTimestamp-startTimestamp)/roundBlockTime*roundBlockTime - // The time of product block of the consensus in last round - return lastRoundStartTime + order*(BlockNumEachNode*BlockTimeInterval) -} + lastBlockNode, err := c.getPrevRoundLastBlock(prevBlockHash) + if err != nil { + return nil, err + } -func (c *consensusNodeManager) getConsensusNodesByVoteResult(blockHeight uint64) (map[string]*consensusNode, error) { - defer c.RUnlock() - c.RLock() - if blockHeight >= c.effectiveStartHeight+roundVoteBlockNums { - return nil, errors.New("the given block height is greater than current vote start height") + if err := c.reorganizeVoteResult(voteResult, lastBlockNode); err != nil { + return nil, err } - if blockHeight >= c.effectiveStartHeight { - return c.consensusNodeMap, nil + if len(voteResult.NumOfVote) == 0 { + return federationNodes(), nil } + return voteResult.ConsensusNodes() +} - voteResult, err := c.store.GetVoteResult(blockHeight / roundVoteBlockNums) +func (c *consensusNodeManager) getBestVoteResult() (*state.VoteResult, error) { + blockNode := c.blockIndex.BestNode() + seq := state.CalcVoteSeq(blockNode.Height) + voteResult, err := c.store.GetVoteResult(seq) if err != nil { return nil, err } - if !voteResult.Finalized { - return nil, errVoteResultIsNotfinalized + if err := c.reorganizeVoteResult(voteResult, blockNode); err != nil { + return nil, err } - var nodes []*consensusNode - for pubkey, voteNum := range voteResult.NumOfVote { - nodes = append(nodes, &consensusNode{ - pubkey: pubkey, - voteNum: voteNum, - }) + return voteResult, nil +} + +func (c *consensusNodeManager) reorganizeVoteResult(voteResult *state.VoteResult, node *state.BlockNode) error { + mainChainNode := c.blockIndex.GetNode(&voteResult.BlockHash) + var attachNodes []*state.BlockNode + var detachNodes []*state.BlockNode + for forkChainNode := node; mainChainNode != forkChainNode; node = node.Parent { + if forkChainNode.Height == mainChainNode.Height { + detachNodes = append(detachNodes, mainChainNode) + mainChainNode = mainChainNode.Parent + } + attachNodes = append([]*state.BlockNode{forkChainNode}, attachNodes...) + } + + for _, node := range detachNodes { + block, err := c.store.GetBlock(&node.Hash) + if err != nil { + return err + } + + if err := voteResult.DetachBlock(block); err != nil { + return err + } } - // In principle, there is no need to sort all voting nodes. - // if there is a performance problem, consider the optimization later. - // TODO not consider the same number of votes - sort.Sort(consensusNodeSlice(nodes)) - result := make(map[string]*consensusNode) - for i := 0; i < numOfConsensusNode; i++ { - node := nodes[i] - node.order = uint64(i) - result[node.pubkey] = node + for _, node := range attachNodes { + block, err := c.store.GetBlock(&node.Hash) + if err != nil { + return err + } + + if err := voteResult.ApplyBlock(block); err != nil { + return err + } + } + return nil +} + +func federationNodes() map[string]*state.ConsensusNode { + voteResult := map[string]*state.ConsensusNode{} + for i, xpub := range config.CommonConfig.Federation.Xpubs { + voteResult[xpub.String()] = &state.ConsensusNode{XPub: xpub, VoteNum: 0, Order: uint64(i)} } - return result, nil + return voteResult }