OSDN Git Service

split block (#180) v0.1_block_cache_of_sign
authorwz <mars@bytom.io>
Sat, 15 Jun 2019 07:42:12 +0000 (15:42 +0800)
committerPaladz <yzhu101@uottawa.ca>
Sat, 15 Jun 2019 07:42:12 +0000 (15:42 +0800)
* split block

* fix test

* fix test

* rm code

* fix review

* fix test

* fix test

* fix cache

* modify func name

* modify import

* fix review

* fit text

database/cache.go
database/cache_test.go
database/store.go
database/store_test.go
protocol/bbft.go
protocol/bc/types/block.go
protocol/bc/types/block_header.go
protocol/block.go
protocol/consensus_node_manager.go
protocol/store.go
protocol/txpool_test.go

index c963f25..a525262 100644 (file)
@@ -2,67 +2,111 @@ package database
 
 import (
        "fmt"
-       "sync"
 
-       "github.com/golang/groupcache/lru"
        "github.com/golang/groupcache/singleflight"
 
+       "github.com/vapor/common"
        "github.com/vapor/protocol/bc"
        "github.com/vapor/protocol/bc/types"
 )
 
-const maxCachedBlocks = 30
+const (
+       maxCachedBlockHeaders      = 1000
+       maxCachedBlockTransactions = 1000
+)
+
+type fillBlockHeaderFn func(hash *bc.Hash, height uint64) (*types.BlockHeader, error)
+type fillBlockTransactionsFn func(hash *bc.Hash) ([]*types.Tx, error)
 
-func newBlockCache(fillFn func(hash *bc.Hash) (*types.Block, error)) blockCache {
+func newBlockCache(fillBlockHeader fillBlockHeaderFn, fillBlockTxs fillBlockTransactionsFn) blockCache {
        return blockCache{
-               lru:    lru.New(maxCachedBlocks),
-               fillFn: fillFn,
+               lruBlockHeaders: common.NewCache(maxCachedBlockHeaders),
+               lruBlockTxs:     common.NewCache(maxCachedBlockTransactions),
+
+               fillBlockHeaderFn:      fillBlockHeader,
+               fillBlockTransactionFn: fillBlockTxs,
        }
 }
 
 type blockCache struct {
-       mu     sync.Mutex
-       lru    *lru.Cache
-       fillFn func(hash *bc.Hash) (*types.Block, error)
-       single singleflight.Group
+       lruBlockHeaders *common.Cache
+       lruBlockTxs     *common.Cache
+
+       fillBlockHeaderFn      func(hash *bc.Hash, height uint64) (*types.BlockHeader, error)
+       fillBlockTransactionFn func(hash *bc.Hash) ([]*types.Tx, error)
+
+       singleBlockHeader singleflight.Group
+       singleBlockTxs    singleflight.Group
 }
 
-func (c *blockCache) lookup(hash *bc.Hash) (*types.Block, error) {
-       if b, ok := c.get(hash); ok {
-               return b, nil
+func (c *blockCache) lookupBlockHeader(hash *bc.Hash, height uint64) (*types.BlockHeader, error) {
+       if bH, ok := c.getBlockHeader(hash); ok {
+               return bH, nil
        }
 
-       block, err := c.single.Do(hash.String(), func() (interface{}, error) {
-               b, err := c.fillFn(hash)
+       blockHeader, err := c.singleBlockHeader.Do(hash.String(), func() (interface{}, error) {
+               bH, err := c.fillBlockHeaderFn(hash, height)
                if err != nil {
                        return nil, err
                }
 
-               if b == nil {
-                       return nil, fmt.Errorf("There are no block with given hash %s", hash.String())
+               if bH == nil {
+                       return nil, fmt.Errorf("There are no blockHeader with given hash %s", hash.String())
                }
 
-               c.add(b)
-               return b, nil
+               c.addBlockHeader(bH)
+               return bH, nil
        })
        if err != nil {
                return nil, err
        }
-       return block.(*types.Block), nil
+       return blockHeader.(*types.BlockHeader), nil
 }
 
-func (c *blockCache) get(hash *bc.Hash) (*types.Block, bool) {
-       c.mu.Lock()
-       block, ok := c.lru.Get(*hash)
-       c.mu.Unlock()
-       if block == nil {
+func (c *blockCache) lookupBlockTxs(hash *bc.Hash) ([]*types.Tx, error) {
+       if bTxs, ok := c.getBlockTransactions(hash); ok {
+               return bTxs, nil
+       }
+
+       blockTransactions, err := c.singleBlockTxs.Do(hash.String(), func() (interface{}, error) {
+               bTxs, err := c.fillBlockTransactionFn(hash)
+               if err != nil {
+                       return nil, err
+               }
+
+               if bTxs == nil {
+                       return nil, fmt.Errorf("There are no block transactions with given hash %s", hash.String())
+               }
+
+               c.addBlockTxs(*hash, bTxs)
+               return bTxs, nil
+       })
+       if err != nil {
+               return nil, err
+       }
+       return blockTransactions.([]*types.Tx), nil
+}
+
+func (c *blockCache) getBlockHeader(hash *bc.Hash) (*types.BlockHeader, bool) {
+       blockHeader, ok := c.lruBlockHeaders.Get(*hash)
+       if blockHeader == nil {
                return nil, ok
        }
-       return block.(*types.Block), ok
+       return blockHeader.(*types.BlockHeader), ok
+}
+
+func (c *blockCache) getBlockTransactions(hash *bc.Hash) ([]*types.Tx, bool) {
+       txs, ok := c.lruBlockTxs.Get(*hash)
+       if txs == nil {
+               return nil, ok
+       }
+       return txs.([]*types.Tx), ok
+}
+
+func (c *blockCache) addBlockHeader(blockHeader *types.BlockHeader) {
+       c.lruBlockHeaders.Add(blockHeader.Hash(), blockHeader)
 }
 
-func (c *blockCache) add(block *types.Block) {
-       c.mu.Lock()
-       c.lru.Add(block.Hash(), block)
-       c.mu.Unlock()
+func (c *blockCache) addBlockTxs(hash bc.Hash, txs []*types.Tx) {
+       c.lruBlockTxs.Add(hash, txs)
 }
index 673ab7d..d7116ec 100644 (file)
@@ -16,33 +16,39 @@ func TestBlockCache(t *testing.T) {
                }
        }
        blocks := make(map[bc.Hash]*types.Block)
-       for i := 0; i < maxCachedBlocks+10; i++ {
+       for i := 0; i < maxCachedBlockHeaders+10; i++ {
                block := newBlock(uint64(i))
                blocks[block.Hash()] = block
        }
 
-       cache := newBlockCache(func(hash *bc.Hash) (*types.Block, error) {
-               return blocks[*hash], nil
-       })
+       fillBlockHeaderFn := func(hash *bc.Hash, height uint64) (*types.BlockHeader, error) {
+               return &blocks[*hash].BlockHeader, nil
+       }
+
+       fillBlockTxsFn := func(hash *bc.Hash) ([]*types.Tx, error) {
+               return blocks[*hash].Transactions, nil
+       }
+
+       cache := newBlockCache(fillBlockHeaderFn, fillBlockTxsFn)
 
-       for i := 0; i < maxCachedBlocks+10; i++ {
+       for i := 0; i < maxCachedBlockHeaders+10; i++ {
                block := newBlock(uint64(i))
                hash := block.Hash()
-               cache.lookup(&hash)
+               cache.lookupBlockHeader(&hash, block.Height)
        }
 
        for i := 0; i < 10; i++ {
                block := newBlock(uint64(i))
                hash := block.Hash()
-               if b, _ := cache.get(&hash); b != nil {
+               if b, _ := cache.getBlockHeader(&hash); b != nil {
                        t.Fatalf("find old block")
                }
        }
 
-       for i := 10; i < maxCachedBlocks+10; i++ {
+       for i := 10; i < maxCachedBlockHeaders+10; i++ {
                block := newBlock(uint64(i))
                hash := block.Hash()
-               if b, _ := cache.get(&hash); b == nil {
+               if b, _ := cache.getBlockHeader(&hash); b == nil {
                        t.Fatalf("can't find new block")
                }
        }
index b7cd2eb..fd53c90 100644 (file)
@@ -21,11 +21,11 @@ import (
 const logModule = "leveldb"
 
 var (
-       blockStoreKey     = []byte("blockStore")
-       blockPrefix       = []byte("B:")
-       blockHeaderPrefix = []byte("BH:")
-       txStatusPrefix    = []byte("BTS:")
-       voteResultPrefix  = []byte("VR:")
+       blockStoreKey          = []byte("blockStore")
+       blockHeaderPrefix      = []byte("BH:")
+       blockTransactonsPrefix = []byte("BTXS:")
+       txStatusPrefix         = []byte("BTS:")
+       voteResultPrefix       = []byte("VR:")
 )
 
 func loadBlockStoreStateJSON(db dbm.DB) *protocol.BlockStoreState {
@@ -48,10 +48,6 @@ type Store struct {
        cache blockCache
 }
 
-func calcBlockKey(hash *bc.Hash) []byte {
-       return append(blockPrefix, hash.Bytes()...)
-}
-
 func calcBlockHeaderKey(height uint64, hash *bc.Hash) []byte {
        buf := [8]byte{}
        binary.BigEndian.PutUint64(buf[:], height)
@@ -59,6 +55,10 @@ func calcBlockHeaderKey(height uint64, hash *bc.Hash) []byte {
        return append(key, hash.Bytes()...)
 }
 
+func calcBlockTransactionsKey(hash *bc.Hash) []byte {
+       return append(blockTransactonsPrefix, hash.Bytes()...)
+}
+
 func calcTxStatusKey(hash *bc.Hash) []byte {
        return append(txStatusPrefix, hash.Bytes()...)
 }
@@ -69,23 +69,45 @@ func calcVoteResultKey(seq uint64) []byte {
        return append(voteResultPrefix, buf[:]...)
 }
 
-// GetBlock return the block by given hash
-func GetBlock(db dbm.DB, hash *bc.Hash) (*types.Block, error) {
-       bytez := db.Get(calcBlockKey(hash))
-       if bytez == nil {
+// GetBlockHeader return the block header by given hash and height
+func GetBlockHeader(db dbm.DB, hash *bc.Hash, height uint64) (*types.BlockHeader, error) {
+       block := &types.Block{}
+       binaryBlockHeader := db.Get(calcBlockHeaderKey(height, hash))
+       if binaryBlockHeader == nil {
                return nil, nil
        }
+       if err := block.UnmarshalText(binaryBlockHeader); err != nil {
+               return nil, err
+       }
 
+       return &block.BlockHeader, nil
+}
+
+// GetBlockTransactions return the block transactions by given hash
+func GetBlockTransactions(db dbm.DB, hash *bc.Hash) ([]*types.Tx, error) {
        block := &types.Block{}
-       err := block.UnmarshalText(bytez)
-       return block, err
+       binaryBlockTxs := db.Get(calcBlockTransactionsKey(hash))
+       if binaryBlockTxs == nil {
+               return nil, errors.New("The transactions in the block is empty")
+       }
+
+       if err := block.UnmarshalText(binaryBlockTxs); err != nil {
+               return nil, err
+       }
+       return block.Transactions, nil
 }
 
 // NewStore creates and returns a new Store object.
 func NewStore(db dbm.DB) *Store {
-       cache := newBlockCache(func(hash *bc.Hash) (*types.Block, error) {
-               return GetBlock(db, hash)
-       })
+       fillBlockHeaderFn := func(hash *bc.Hash, height uint64) (*types.BlockHeader, error) {
+               return GetBlockHeader(db, hash, height)
+       }
+
+       fillBlockTxsFn := func(hash *bc.Hash) ([]*types.Tx, error) {
+               return GetBlockTransactions(db, hash)
+       }
+
+       cache := newBlockCache(fillBlockHeaderFn, fillBlockTxsFn)
        return &Store{
                db:    db,
                cache: cache,
@@ -98,14 +120,45 @@ func (s *Store) GetUtxo(hash *bc.Hash) (*storage.UtxoEntry, error) {
 }
 
 // BlockExist check if the block is stored in disk
-func (s *Store) BlockExist(hash *bc.Hash) bool {
-       block, err := s.cache.lookup(hash)
-       return err == nil && block != nil
+func (s *Store) BlockExist(hash *bc.Hash, height uint64) bool {
+       blockHeader, err := s.cache.lookupBlockHeader(hash, height)
+       return err == nil && blockHeader != nil
 }
 
 // GetBlock return the block by given hash
-func (s *Store) GetBlock(hash *bc.Hash) (*types.Block, error) {
-       return s.cache.lookup(hash)
+func (s *Store) GetBlock(hash *bc.Hash, height uint64) (*types.Block, error) {
+       blockHeader, err := s.GetBlockHeader(hash, height)
+       if err != nil {
+               return nil, err
+       }
+
+       txs, err := s.GetBlockTransactions(hash)
+       if err != nil {
+               return nil, err
+       }
+
+       return &types.Block{
+               BlockHeader:  *blockHeader,
+               Transactions: txs,
+       }, nil
+}
+
+// GetBlockHeader return the BlockHeader by given hash
+func (s *Store) GetBlockHeader(hash *bc.Hash, height uint64) (*types.BlockHeader, error) {
+       blockHeader, err := s.cache.lookupBlockHeader(hash, height)
+       if err != nil {
+               return nil, err
+       }
+       return blockHeader, nil
+}
+
+// GetBlockTransactions return the Block transactions by given hash
+func (s *Store) GetBlockTransactions(hash *bc.Hash) ([]*types.Tx, error) {
+       txs, err := s.cache.lookupBlockTxs(hash)
+       if err != nil {
+               return nil, err
+       }
+       return txs, nil
 }
 
 // GetTransactionsUtxo will return all the utxo that related to the input txs
@@ -192,14 +245,15 @@ func (s *Store) LoadBlockIndex(stateBestHeight uint64) (*state.BlockIndex, error
 // SaveBlock persists a new block in the protocol.
 func (s *Store) SaveBlock(block *types.Block, ts *bc.TransactionStatus) error {
        startTime := time.Now()
-       binaryBlock, err := block.MarshalText()
+
+       binaryBlockHeader, err := block.MarshalTextForBlockHeader()
        if err != nil {
-               return errors.Wrap(err, "Marshal block meta")
+               return errors.Wrap(err, "Marshal block header")
        }
 
-       binaryBlockHeader, err := block.BlockHeader.MarshalText()
+       binaryBlockTxs, err := block.MarshalTextForTransactions()
        if err != nil {
-               return errors.Wrap(err, "Marshal block header")
+               return errors.Wrap(err, "Marshal block transactions")
        }
 
        binaryTxStatus, err := proto.Marshal(ts)
@@ -209,8 +263,8 @@ func (s *Store) SaveBlock(block *types.Block, ts *bc.TransactionStatus) error {
 
        blockHash := block.Hash()
        batch := s.db.NewBatch()
-       batch.Set(calcBlockKey(&blockHash), binaryBlock)
        batch.Set(calcBlockHeaderKey(block.Height, &blockHash), binaryBlockHeader)
+       batch.Set(calcBlockTransactionsKey(&blockHash), binaryBlockTxs)
        batch.Set(calcTxStatusKey(&blockHash), binaryTxStatus)
        batch.Write()
 
@@ -223,6 +277,29 @@ func (s *Store) SaveBlock(block *types.Block, ts *bc.TransactionStatus) error {
        return nil
 }
 
+// SaveBlockHeader persists a new block header in the protocol.
+func (s *Store) SaveBlockHeader(blockHeader *types.BlockHeader) error {
+       startTime := time.Now()
+
+       binaryBlockHeader, err := blockHeader.MarshalText()
+       if err != nil {
+               return errors.Wrap(err, "Marshal block header")
+       }
+
+       blockHash := blockHeader.Hash()
+       batch := s.db.NewBatch()
+       batch.Set(calcBlockHeaderKey(blockHeader.Height, &blockHash), binaryBlockHeader)
+       batch.Write()
+
+       log.WithFields(log.Fields{
+               "module":   logModule,
+               "height":   blockHeader.Height,
+               "hash":     blockHash.String(),
+               "duration": time.Since(startTime),
+       }).Info("blockHeader saved on disk")
+       return nil
+}
+
 // SaveChainStatus save the core's newest status && delete old status
 func (s *Store) SaveChainStatus(node, irreversibleNode *state.BlockNode, view *state.UtxoViewpoint, voteResults []*state.VoteResult) error {
        batch := s.db.NewBatch()
index 68f62c3..ea009eb 100644 (file)
@@ -221,7 +221,7 @@ func TestSaveBlock(t *testing.T) {
        }
 
        blockHash := block.Hash()
-       gotBlock, err := store.GetBlock(&blockHash)
+       gotBlock, err := store.GetBlock(&blockHash, block.Height)
        if err != nil {
                t.Fatal(err)
        }
index c6fc3b7..6c61949 100644 (file)
@@ -216,19 +216,14 @@ func (c *Chain) updateBlockSignature(blockNode *state.BlockNode, nodeOrder uint6
                return err
        }
 
-       block, err := c.store.GetBlock(&blockNode.Hash)
+       blockHeader, err := c.store.GetBlockHeader(&blockNode.Hash, blockNode.Height)
        if err != nil {
                return err
        }
 
-       block.Set(nodeOrder, signature)
+       blockHeader.Set(nodeOrder, signature)
 
-       txStatus, err := c.store.GetTransactionStatus(&blockNode.Hash)
-       if err != nil {
-               return err
-       }
-
-       if err := c.store.SaveBlock(block, txStatus); err != nil {
+       if err := c.store.SaveBlockHeader(blockHeader); err != nil {
                return err
        }
 
index dbb301a..4e1130b 100644 (file)
@@ -58,13 +58,49 @@ func (b *Block) UnmarshalText(text []byte) error {
        return nil
 }
 
+// MarshalTextForBlockHeader fulfills the json.Marshaler interface.
+func (b *Block) MarshalTextForBlockHeader() ([]byte, error) {
+       buf := bufpool.Get()
+       defer bufpool.Put(buf)
+
+       ew := errors.NewWriter(buf)
+       if err := b.writeTo(ew, SerBlockHeader); err != nil {
+               return nil, err
+       }
+       if err := ew.Err(); err != nil {
+               return nil, err
+       }
+
+       enc := make([]byte, hex.EncodedLen(buf.Len()))
+       hex.Encode(enc, buf.Bytes())
+       return enc, nil
+}
+
+// MarshalTextForTransactions fulfills the json.Marshaler interface.
+func (b *Block) MarshalTextForTransactions() ([]byte, error) {
+       buf := bufpool.Get()
+       defer bufpool.Put(buf)
+
+       ew := errors.NewWriter(buf)
+       if err := b.writeTo(ew, SerBlockTransactions); err != nil {
+               return nil, err
+       }
+       if err := ew.Err(); err != nil {
+               return nil, err
+       }
+
+       enc := make([]byte, hex.EncodedLen(buf.Len()))
+       hex.Encode(enc, buf.Bytes())
+       return enc, nil
+}
+
 func (b *Block) readFrom(r *blockchain.Reader) error {
-       serflags, err := b.BlockHeader.readFrom(r)
+       serflag, err := b.BlockHeader.readFrom(r)
        if err != nil {
                return err
        }
 
-       if serflags == SerBlockHeader {
+       if serflag == SerBlockHeader {
                return nil
        }
 
@@ -94,14 +130,20 @@ func (b *Block) WriteTo(w io.Writer) (int64, error) {
 }
 
 func (b *Block) writeTo(w io.Writer, serflags uint8) error {
-       if err := b.BlockHeader.writeTo(w, serflags); err != nil {
-               return err
+       if serflags == SerBlockHeader || serflags == SerBlockFull {
+               if err := b.BlockHeader.writeTo(w, serflags); err != nil {
+                       return err
+               }
        }
 
        if serflags == SerBlockHeader {
                return nil
        }
 
+       if serflags != SerBlockFull {
+               w.Write([]byte{serflags})
+       }
+
        if _, err := blockchain.WriteVarint31(w, uint64(len(b.Transactions))); err != nil {
                return err
        }
index 88bdddb..392b3ec 100644 (file)
@@ -56,8 +56,15 @@ func (bh *BlockHeader) UnmarshalText(text []byte) error {
                return err
        }
 
-       _, err := bh.readFrom(blockchain.NewReader(decoded))
-       return err
+       serflag, err := bh.readFrom(blockchain.NewReader(decoded))
+       if err != nil {
+               return err
+       }
+
+       if serflag == SerBlockTransactions {
+               return fmt.Errorf("unsupported serialization flags 0x%02x", serflag)
+       }
+       return nil
 }
 
 func (bh *BlockHeader) readFrom(r *blockchain.Reader) (serflag uint8, err error) {
@@ -66,6 +73,8 @@ func (bh *BlockHeader) readFrom(r *blockchain.Reader) (serflag uint8, err error)
        serflag = serflags[0]
        switch serflag {
        case SerBlockHeader, SerBlockFull:
+       case SerBlockTransactions:
+               return
        default:
                return 0, fmt.Errorf("unsupported serialization flags 0x%x", serflags)
        }
index 8d87804..d5ef5cf 100644 (file)
@@ -28,7 +28,11 @@ func (c *Chain) BlockExist(hash *bc.Hash) bool {
 
 // GetBlockByHash return a block by given hash
 func (c *Chain) GetBlockByHash(hash *bc.Hash) (*types.Block, error) {
-       return c.store.GetBlock(hash)
+       node := c.index.GetNode(hash)
+       if node == nil {
+               return nil, errors.New("can't find block in given hash")
+       }
+       return c.store.GetBlock(hash, node.Height)
 }
 
 // GetBlockByHeight return a block header by given height
@@ -37,7 +41,7 @@ func (c *Chain) GetBlockByHeight(height uint64) (*types.Block, error) {
        if node == nil {
                return nil, errors.New("can't find block in given height")
        }
-       return c.store.GetBlock(&node.Hash)
+       return c.store.GetBlock(&node.Hash, height)
 }
 
 // GetHeaderByHash return a block header by given hash
@@ -125,7 +129,7 @@ func (c *Chain) reorganizeChain(node *state.BlockNode) error {
        }
 
        for _, detachNode := range detachNodes {
-               b, err := c.store.GetBlock(&detachNode.Hash)
+               b, err := c.store.GetBlock(&detachNode.Hash, detachNode.Height)
                if err != nil {
                        return err
                }
@@ -152,7 +156,7 @@ func (c *Chain) reorganizeChain(node *state.BlockNode) error {
        }
 
        for _, attachNode := range attachNodes {
-               b, err := c.store.GetBlock(&attachNode.Hash)
+               b, err := c.store.GetBlock(&attachNode.Hash, attachNode.Height)
                if err != nil {
                        return err
                }
index 4fe6638..66d1826 100644 (file)
@@ -66,7 +66,7 @@ func getBlockerOrder(startTimestamp, blockTimestamp, numOfConsensusNode uint64)
        // The start time of the last round of product block
        lastRoundStartTime := startTimestamp + (blockTimestamp-startTimestamp)/roundBlockTime*roundBlockTime
        // Order of blocker
-       return (blockTimestamp - lastRoundStartTime)/(consensus.BlockNumEachNode*consensus.BlockTimeInterval)
+       return (blockTimestamp - lastRoundStartTime) / (consensus.BlockNumEachNode * consensus.BlockTimeInterval)
 }
 
 func (c *consensusNodeManager) getPrevRoundLastBlock(prevBlockHash *bc.Hash) (*state.BlockNode, error) {
@@ -114,7 +114,7 @@ func (c *consensusNodeManager) getBestVoteResult() (*state.VoteResult, error) {
 // getVoteResult return the vote result
 // seq represent the sequence of vote
 // blockNode represent the chain in which the result of the vote is located
-// Voting results need to be adjusted according to the chain 
+// Voting results need to be adjusted according to the chain
 func (c *consensusNodeManager) getVoteResult(seq uint64, blockNode *state.BlockNode) (*state.VoteResult, error) {
        voteResult, err := c.store.GetVoteResult(seq)
        if err != nil {
@@ -136,7 +136,7 @@ func (c *consensusNodeManager) reorganizeVoteResult(voteResult *state.VoteResult
                var forChainRollback, mainChainRollBack bool
                if forChainRollback = forkChainNode.Height >= mainChainNode.Height; forChainRollback {
                        attachNodes = append([]*state.BlockNode{forkChainNode}, attachNodes...)
-               } 
+               }
                if mainChainRollBack = forkChainNode.Height <= mainChainNode.Height; mainChainRollBack {
                        detachNodes = append(detachNodes, mainChainNode)
                }
@@ -149,7 +149,7 @@ func (c *consensusNodeManager) reorganizeVoteResult(voteResult *state.VoteResult
        }
 
        for _, node := range detachNodes {
-               block, err := c.store.GetBlock(&node.Hash)
+               block, err := c.store.GetBlock(&node.Hash, node.Height)
                if err != nil {
                        return err
                }
@@ -160,7 +160,7 @@ func (c *consensusNodeManager) reorganizeVoteResult(voteResult *state.VoteResult
        }
 
        for _, node := range attachNodes {
-               block, err := c.store.GetBlock(&node.Hash)
+               block, err := c.store.GetBlock(&node.Hash, node.Height)
                if err != nil {
                        return err
                }
index a0f6b03..b3909ff 100644 (file)
@@ -15,9 +15,10 @@ var (
 
 // Store provides storage interface for blockchain data
 type Store interface {
-       BlockExist(*bc.Hash) bool
+       BlockExist(*bc.Hash, uint64) bool
 
-       GetBlock(*bc.Hash) (*types.Block, error)
+       GetBlock(*bc.Hash, uint64) (*types.Block, error)
+       GetBlockHeader(*bc.Hash, uint64) (*types.BlockHeader, error)
        GetStoreStatus() *BlockStoreState
        GetTransactionStatus(*bc.Hash) (*bc.TransactionStatus, error)
        GetTransactionsUtxo(*state.UtxoViewpoint, []*bc.Tx) error
@@ -26,6 +27,7 @@ type Store interface {
 
        LoadBlockIndex(uint64) (*state.BlockIndex, error)
        SaveBlock(*types.Block, *bc.TransactionStatus) error
+       SaveBlockHeader(*types.BlockHeader) error
        SaveChainStatus(*state.BlockNode, *state.BlockNode, *state.UtxoViewpoint, []*state.VoteResult) error
 }
 
index 59e9d25..338d57f 100644 (file)
@@ -111,8 +111,9 @@ var testTxs = []*types.Tx{
 
 type mockStore struct{}
 
-func (s *mockStore) BlockExist(hash *bc.Hash) bool                                { return false }
-func (s *mockStore) GetBlock(*bc.Hash) (*types.Block, error)                      { return nil, nil }
+func (s *mockStore) BlockExist(hash *bc.Hash, height uint64) bool                 { return false }
+func (s *mockStore) GetBlock(*bc.Hash, uint64) (*types.Block, error)              { return nil, nil }
+func (s *mockStore) GetBlockHeader(*bc.Hash, uint64) (*types.BlockHeader, error)  { return nil, nil }
 func (s *mockStore) GetStoreStatus() *BlockStoreState                             { return nil }
 func (s *mockStore) GetTransactionStatus(*bc.Hash) (*bc.TransactionStatus, error) { return nil, nil }
 func (s *mockStore) GetTransactionsUtxo(*state.UtxoViewpoint, []*bc.Tx) error     { return nil }
@@ -120,6 +121,7 @@ func (s *mockStore) GetUtxo(*bc.Hash) (*storage.UtxoEntry, error)
 func (s *mockStore) GetVoteResult(uint64) (*state.VoteResult, error)              { return nil, nil }
 func (s *mockStore) LoadBlockIndex(uint64) (*state.BlockIndex, error)             { return nil, nil }
 func (s *mockStore) SaveBlock(*types.Block, *bc.TransactionStatus) error          { return nil }
+func (s *mockStore) SaveBlockHeader(*types.BlockHeader) error                     { return nil }
 func (s *mockStore) SaveChainStatus(*state.BlockNode, *state.BlockNode, *state.UtxoViewpoint, []*state.VoteResult) error {
        return nil
 }
@@ -653,8 +655,9 @@ func TestRemoveOrphan(t *testing.T) {
 
 type mockStore1 struct{}
 
-func (s *mockStore1) BlockExist(hash *bc.Hash) bool                                { return false }
-func (s *mockStore1) GetBlock(*bc.Hash) (*types.Block, error)                      { return nil, nil }
+func (s *mockStore1) BlockExist(hash *bc.Hash, height uint64) bool                 { return false }
+func (s *mockStore1) GetBlock(*bc.Hash, uint64) (*types.Block, error)              { return nil, nil }
+func (s *mockStore1) GetBlockHeader(*bc.Hash, uint64) (*types.BlockHeader, error)  { return nil, nil }
 func (s *mockStore1) GetStoreStatus() *BlockStoreState                             { return nil }
 func (s *mockStore1) GetTransactionStatus(*bc.Hash) (*bc.TransactionStatus, error) { return nil, nil }
 func (s *mockStore1) GetTransactionsUtxo(utxoView *state.UtxoViewpoint, tx []*bc.Tx) error {
@@ -668,6 +671,7 @@ func (s *mockStore1) GetUtxo(*bc.Hash) (*storage.UtxoEntry, error)        { retu
 func (s *mockStore1) GetVoteResult(uint64) (*state.VoteResult, error)     { return nil, nil }
 func (s *mockStore1) LoadBlockIndex(uint64) (*state.BlockIndex, error)    { return nil, nil }
 func (s *mockStore1) SaveBlock(*types.Block, *bc.TransactionStatus) error { return nil }
+func (s *mockStore1) SaveBlockHeader(*types.BlockHeader) error            { return nil }
 func (s *mockStore1) SaveChainStatus(*state.BlockNode, *state.BlockNode, *state.UtxoViewpoint, []*state.VoteResult) error {
        return nil
 }