aboutsummaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
-rw-r--r--consensus/dexcon/dexcon.go3
-rw-r--r--core/block_validator.go34
-rw-r--r--core/blockchain.go195
-rw-r--r--core/types/block.go35
-rw-r--r--dex/app.go213
5 files changed, 314 insertions, 166 deletions
diff --git a/consensus/dexcon/dexcon.go b/consensus/dexcon/dexcon.go
index 4017637b0..f7857c070 100644
--- a/consensus/dexcon/dexcon.go
+++ b/consensus/dexcon/dexcon.go
@@ -105,7 +105,8 @@ func (d *Dexcon) Prepare(chain consensus.ChainReader, header *types.Header) erro
// Finalize implements consensus.Engine, ensuring no uncles are set, nor block
// rewards given, and returns the final block.
func (d *Dexcon) Finalize(chain consensus.ChainReader, header *types.Header, state *state.StateDB, txs []*types.Transaction, uncles []*types.Header, receipts []*types.Receipt) (*types.Block, error) {
- blockReward := big.NewInt(100000000000)
+ // TODO(Bojie): remove it and get value from config
+ blockReward := big.NewInt(1e+18)
reward := new(big.Int).Div(blockReward, new(big.Int).SetUint64(uint64(d.config.NumChains)))
state.AddBalance(header.Coinbase, reward)
header.Root = state.IntermediateRoot(true)
diff --git a/core/block_validator.go b/core/block_validator.go
index 697944c41..65f311f9f 100644
--- a/core/block_validator.go
+++ b/core/block_validator.go
@@ -101,6 +101,40 @@ func (v *BlockValidator) ValidateState(block, parent *types.Block, statedb *stat
return nil
}
+// BlockValidator implements Validator.
+type DexonBlockValidator struct {
+ config *params.ChainConfig // Chain configuration options
+ bc *BlockChain // Canonical block chain
+ engine consensus.Engine // Consensus engine used for validating
+}
+
+// NewDexonBlockValidator returns a new block validator which is safe for re-use
+func NewDexonBlockValidator(config *params.ChainConfig, blockchain *BlockChain, engine consensus.Engine) *DexonBlockValidator {
+ validator := &DexonBlockValidator{
+ config: config,
+ engine: engine,
+ bc: blockchain,
+ }
+ return validator
+}
+
+// ValidateBody validates the given block's uncles and verifies the block
+// header's transaction and uncle roots. The headers are assumed to be already
+// validated at this point.
+func (v *DexonBlockValidator) ValidateBody(block *types.Block) error {
+ // TODO(Bojie): implement it
+ return nil
+}
+
+// ValidateState validates the various changes that happen after a state
+// transition, such as amount of used gas, the receipt roots and the state root
+// itself. ValidateState returns a database batch if the validation was a success
+// otherwise nil and an error is returned.
+func (v *DexonBlockValidator) ValidateState(block, parent *types.Block, statedb *state.StateDB, receipts types.Receipts, usedGas uint64) error {
+ // TODO(Bojie): implement it
+ return nil
+}
+
// CalcGasLimit computes the gas limit of the next block after parent. It aims
// to keep the baseline gas above the provided floor, and increase it towards the
// ceil if the blocks are full. If the ceil is exceeded, it will always decrease
diff --git a/core/blockchain.go b/core/blockchain.go
index 9775f9e16..87e586d20 100644
--- a/core/blockchain.go
+++ b/core/blockchain.go
@@ -28,6 +28,8 @@ import (
"sync/atomic"
"time"
+ "github.com/hashicorp/golang-lru"
+
coreCommon "github.com/dexon-foundation/dexon-consensus-core/common"
coreTypes "github.com/dexon-foundation/dexon-consensus-core/core/types"
@@ -47,7 +49,6 @@ import (
"github.com/dexon-foundation/dexon/params"
"github.com/dexon-foundation/dexon/rlp"
"github.com/dexon-foundation/dexon/trie"
- "github.com/hashicorp/golang-lru"
)
var (
@@ -144,6 +145,11 @@ type BlockChain struct {
confirmedBlockMu sync.Mutex
confirmedBlocks map[coreCommon.Hash]*coreTypes.Block
chainConfirmedBlocks map[uint32][]*coreTypes.Block
+
+ pendingBlocks map[uint64]struct {
+ block *types.Block
+ receipts types.Receipts
+ }
}
// NewBlockChain returns a fully initialised block chain using information
@@ -183,8 +189,12 @@ func NewBlockChain(db ethdb.Database, cacheConfig *CacheConfig, chainConfig *par
badBlocks: badBlocks,
confirmedBlocks: make(map[coreCommon.Hash]*coreTypes.Block),
chainConfirmedBlocks: make(map[uint32][]*coreTypes.Block),
+ pendingBlocks: make(map[uint64]struct {
+ block *types.Block
+ receipts types.Receipts
+ }),
}
- bc.SetValidator(NewBlockValidator(chainConfig, bc, engine))
+ bc.SetValidator(NewDexonBlockValidator(chainConfig, bc, engine))
bc.SetProcessor(NewStateProcessor(chainConfig, bc, engine))
var err error
@@ -1449,6 +1459,187 @@ func (bc *BlockChain) insertSidechain(block *types.Block, it *insertIterator) (i
return 0, nil, nil, nil
}
+func (bc *BlockChain) InsertPendingBlock(chain types.Blocks) (int, error) {
+ n, events, logs, err := bc.insertPendingBlocks(chain)
+ bc.PostChainEvents(events, logs)
+ return n, err
+}
+
+func (bc *BlockChain) insertPendingBlocks(chain types.Blocks) (int, []interface{}, []*types.Log, error) {
+ // Sanity check that we have something meaningful to import
+ if len(chain) == 0 {
+ return 0, nil, nil, nil
+ }
+ // Do a sanity check that the provided chain is actually ordered and linked
+ for i := 1; i < len(chain); i++ {
+ if chain[i].NumberU64() != chain[i-1].NumberU64()+1 || chain[i].ParentHash() != chain[i-1].Hash() {
+ // Chain broke ancestry, log a message (programming error) and skip insertion
+ log.Error("Non contiguous block insert", "number", chain[i].Number(), "hash", chain[i].Hash(),
+ "parent", chain[i].ParentHash(), "prevnumber", chain[i-1].Number(), "prevhash", chain[i-1].Hash())
+
+ return 0, nil, nil, fmt.Errorf("non contiguous insert: item %d is #%d [%x…], item %d is #%d [%x…] (parent [%x…])", i-1, chain[i-1].NumberU64(),
+ chain[i-1].Hash().Bytes()[:4], i, chain[i].NumberU64(), chain[i].Hash().Bytes()[:4], chain[i].ParentHash().Bytes()[:4])
+ }
+ }
+ // Pre-checks passed, start the full block imports
+ bc.wg.Add(1)
+ defer bc.wg.Done()
+
+ bc.chainmu.Lock()
+ defer bc.chainmu.Unlock()
+
+ // A queued approach to delivering events. This is generally
+ // faster than direct delivery and requires much less mutex
+ // acquiring.
+ var (
+ stats = insertStats{startTime: mclock.Now()}
+ events = make([]interface{}, 0, len(chain))
+ lastCanon *types.Block
+ coalescedLogs []*types.Log
+ )
+
+ // Start a parallel signature recovery (signer will fluke on fork transition, minimal perf loss)
+ senderCacher.recoverFromBlocks(types.MakeSigner(bc.chainConfig, chain[0].Number()), chain)
+
+ // Iterate over the blocks and insert when the verifier permits
+ for i, block := range chain {
+ if atomic.LoadInt32(&bc.procInterrupt) == 1 {
+ log.Debug("Premature abort during blocks processing")
+ break
+ }
+ bstart := time.Now()
+
+ currentBlock := bc.CurrentBlock()
+ if block.Header().WitnessHeight > currentBlock.NumberU64() && block.Header().WitnessHeight != 0 {
+ if bc.pendingBlocks[block.Header().WitnessHeight].block.Root() != block.Header().WitnessRoot {
+ return i, nil, nil, fmt.Errorf("invalid witness root %s vs %s", bc.pendingBlocks[block.Header().WitnessHeight].block.Root().String(), block.Header().WitnessRoot.String())
+ }
+
+ if bc.pendingBlocks[block.Header().WitnessHeight].block.ReceiptHash() != block.Header().WitnessReceiptHash {
+ return i, nil, nil, fmt.Errorf("invalid witness receipt hash %s vs %s", bc.pendingBlocks[block.Header().WitnessHeight].block.ReceiptHash().String(), block.Header().WitnessReceiptHash.String())
+ }
+ }
+
+ var parentBlock *types.Block
+ var pendingState *state.StateDB
+ var err error
+ parent, exist := bc.pendingBlocks[block.NumberU64()-1]
+ if !exist {
+ parentBlock = currentBlock
+ if parentBlock.NumberU64() != block.NumberU64()-1 {
+ return i, nil, nil, fmt.Errorf("parent block %d not exist", block.NumberU64()-1)
+ }
+ } else {
+ parentBlock = parent.block
+ }
+ block.RawHeader().ParentHash = parentBlock.Hash()
+ pendingState, err = state.New(parentBlock.Root(), bc.stateCache)
+ if err != nil {
+ return i, events, coalescedLogs, err
+ }
+
+ var (
+ receipts types.Receipts
+ usedGas = new(uint64)
+ header = block.Header()
+ allLogs []*types.Log
+ gp = new(GasPool).AddGas(block.GasLimit())
+ )
+ // Iterate over and process the individual transactions
+ for i, tx := range block.Transactions() {
+ pendingState.Prepare(tx.Hash(), block.Hash(), i)
+ receipt, _, err := ApplyTransaction(bc.chainConfig, bc, nil, gp, pendingState, header, tx, usedGas, bc.vmConfig)
+ if err != nil {
+ return i, nil, nil, fmt.Errorf("apply transaction error: %v %d", err, tx.Nonce())
+ }
+ receipts = append(receipts, receipt)
+ allLogs = append(allLogs, receipt.Logs...)
+ log.Debug("apply transaction", "tx.hash", tx.Hash(), "nonce", tx.Nonce(), "amount", tx.Value())
+ }
+ // Finalize the block, applying any consensus engine specific extras (e.g. block rewards)
+ header.GasUsed = *usedGas
+ newPendingBlock, err := bc.engine.Finalize(bc, header, pendingState, block.Transactions(), block.Uncles(), receipts)
+ if err != nil {
+ return i, events, coalescedLogs, fmt.Errorf("finalize error: %v", err)
+ }
+
+ // Validate the state using the default validator
+ err = bc.Validator().ValidateState(block, nil, pendingState, receipts, *usedGas)
+ if err != nil {
+ bc.reportBlock(block, receipts, err)
+ return i, events, coalescedLogs, fmt.Errorf("valiadte state error: %v", err)
+ }
+ proctime := time.Since(bstart)
+
+ // commit state to refresh stateCache
+ _, err = pendingState.Commit(true)
+ if err != nil {
+ return i, nil, nil, fmt.Errorf("pendingState commit error: %v", err)
+ }
+
+ // add into pending blocks
+ bc.pendingBlocks[block.NumberU64()] = struct {
+ block *types.Block
+ receipts types.Receipts
+ }{block: newPendingBlock, receipts: receipts}
+
+ // start insert available pending blocks into db
+ for pendingHeight := bc.CurrentBlock().NumberU64() + 1; pendingHeight <= block.Header().WitnessHeight; pendingHeight++ {
+ confirmedBlock, exist := bc.pendingBlocks[pendingHeight]
+ if !exist {
+ log.Debug("block has already inserted", "height", pendingHeight)
+ continue
+ }
+
+ s, err := state.New(confirmedBlock.block.Root(), bc.stateCache)
+ if err != nil {
+ return i, events, coalescedLogs, err
+ }
+
+ // Write the block to the chain and get the status.
+ log.Debug("insert pending block", "height", pendingHeight)
+ status, err := bc.WriteBlockWithState(confirmedBlock.block, confirmedBlock.receipts, s)
+ if err != nil {
+ return i, events, coalescedLogs, fmt.Errorf("WriteBlockWithState error: %v", err)
+ }
+
+ switch status {
+ case CanonStatTy:
+ log.Debug("Inserted new block", "number", block.Number(), "hash", block.Hash(), "uncles", len(block.Uncles()),
+ "txs", len(block.Transactions()), "gas", block.GasUsed(), "elapsed", common.PrettyDuration(time.Since(bstart)))
+
+ coalescedLogs = append(coalescedLogs, allLogs...)
+ blockInsertTimer.UpdateSince(bstart)
+ events = append(events, ChainEvent{confirmedBlock.block, confirmedBlock.block.Hash(), allLogs})
+ lastCanon = confirmedBlock.block
+
+ // Only count canonical blocks for GC processing time
+ bc.gcproc += proctime
+
+ case SideStatTy:
+ return i, nil, nil, fmt.Errorf("insert pending block and fork found")
+ }
+
+ delete(bc.pendingBlocks, pendingHeight)
+
+ stats.processed++
+ stats.usedGas += *usedGas
+
+ cache, _ := bc.stateCache.TrieDB().Size()
+ stats.report(chain, i, cache)
+ }
+ }
+ // Append a single chain head event if we've progressed the chain
+ if lastCanon != nil && bc.CurrentBlock().Hash() == lastCanon.Hash() {
+ events = append(events, ChainHeadEvent{lastCanon})
+ }
+ return 0, events, coalescedLogs, nil
+}
+
+func (bc *BlockChain) GetPendingBlockByHeight(height uint64) *types.Block {
+ return bc.pendingBlocks[height].block
+}
+
// reorg takes two blocks, an old chain and a new chain and will reconstruct the
// blocks and inserts them to be part of the new canonical chain and accumulates
// potential missing transactions and post an event about them.
diff --git a/core/types/block.go b/core/types/block.go
index ef2cb2c56..a28ce8ee0 100644
--- a/core/types/block.go
+++ b/core/types/block.go
@@ -68,22 +68,25 @@ func (n *BlockNonce) UnmarshalText(input []byte) error {
// Header represents a block header in the Ethereum blockchain.
type Header struct {
- ParentHash common.Hash `json:"parentHash" gencodec:"required"`
- UncleHash common.Hash `json:"sha3Uncles" gencodec:"required"`
- Coinbase common.Address `json:"miner" gencodec:"required"`
- Root common.Hash `json:"stateRoot" gencodec:"required"`
- TxHash common.Hash `json:"transactionsRoot" gencodec:"required"`
- ReceiptHash common.Hash `json:"receiptsRoot" gencodec:"required"`
- Bloom Bloom `json:"logsBloom" gencodec:"required"`
- Difficulty *big.Int `json:"difficulty" gencodec:"required"`
- Number *big.Int `json:"number" gencodec:"required"`
- GasLimit uint64 `json:"gasLimit" gencodec:"required"`
- GasUsed uint64 `json:"gasUsed" gencodec:"required"`
- Time uint64 `json:"timestamp" gencodec:"required"`
- Extra []byte `json:"extraData" gencodec:"required"`
- MixDigest common.Hash `json:"mixHash"`
- Nonce BlockNonce `json:"nonce"`
- Randomness []byte `json:"randomness" gencodec:"required"`
+ ParentHash common.Hash `json:"parentHash" gencodec:"required"`
+ UncleHash common.Hash `json:"sha3Uncles" gencodec:"required"`
+ Coinbase common.Address `json:"miner" gencodec:"required"`
+ Root common.Hash `json:"stateRoot" gencodec:"required"`
+ TxHash common.Hash `json:"transactionsRoot" gencodec:"required"`
+ ReceiptHash common.Hash `json:"receiptsRoot" gencodec:"required"`
+ Bloom Bloom `json:"logsBloom" gencodec:"required"`
+ Difficulty *big.Int `json:"difficulty" gencodec:"required"`
+ Number *big.Int `json:"number" gencodec:"required"`
+ GasLimit uint64 `json:"gasLimit" gencodec:"required"`
+ GasUsed uint64 `json:"gasUsed" gencodec:"required"`
+ Time uint64 `json:"timestamp" gencodec:"required"`
+ Extra []byte `json:"extraData" gencodec:"required"`
+ MixDigest common.Hash `json:"mixHash"`
+ Nonce BlockNonce `json:"nonce"`
+ Randomness []byte `json:"randomness" gencodec:"required"`
+ WitnessHeight uint64 `json:"witnessHeight" gencodec:"required"`
+ WitnessRoot common.Hash `json:"WitnessRoot" gencodec:"required"`
+ WitnessReceiptHash common.Hash `json:"WitnessReceiptHash" gencodec:"required"`
}
// field type overrides for gencodec
diff --git a/dex/app.go b/dex/app.go
index 85d7ea621..9b13c32b8 100644
--- a/dex/app.go
+++ b/dex/app.go
@@ -28,9 +28,7 @@ import (
coreTypes "github.com/dexon-foundation/dexon-consensus-core/core/types"
"github.com/dexon-foundation/dexon/common"
- "github.com/dexon-foundation/dexon/common/math"
"github.com/dexon-foundation/dexon/core"
- "github.com/dexon-foundation/dexon/core/rawdb"
"github.com/dexon-foundation/dexon/core/state"
"github.com/dexon-foundation/dexon/core/types"
"github.com/dexon-foundation/dexon/core/vm"
@@ -50,10 +48,13 @@ type DexconApp struct {
notifyChan map[uint64]*notify
mutex *sync.Mutex
+
+ lastHeight uint64
+ insertMu sync.Mutex
}
type notify struct {
- results []chan bool
+ results []chan uint64
}
type witnessData struct {
@@ -75,10 +76,10 @@ func NewDexconApp(txPool *core.TxPool, blockchain *core.BlockChain, gov *DexconG
}
}
-func (d *DexconApp) addNotify(height uint64) <-chan bool {
+func (d *DexconApp) addNotify(height uint64) <-chan uint64 {
d.mutex.Lock()
defer d.mutex.Unlock()
- result := make(chan bool)
+ result := make(chan uint64)
if n, exist := d.notifyChan[height]; exist {
n.results = append(n.results, result)
} else {
@@ -94,11 +95,12 @@ func (d *DexconApp) notify(height uint64) {
for h, n := range d.notifyChan {
if height >= h {
for _, ch := range n.results {
- ch <- true
+ ch <- height
}
delete(d.notifyChan, h)
}
}
+ d.lastHeight = height
}
func (d *DexconApp) checkChain(address common.Address, chainSize, chainID *big.Int) bool {
@@ -108,52 +110,41 @@ func (d *DexconApp) checkChain(address common.Address, chainSize, chainID *big.I
// PreparePayload is called when consensus core is preparing payload for block.
func (d *DexconApp) PreparePayload(position coreTypes.Position) (payload []byte, err error) {
+ d.insertMu.Lock()
+ defer d.insertMu.Unlock()
txsMap, err := d.txPool.Pending()
if err != nil {
return
}
- currentBlock := d.blockchain.CurrentBlock()
- gasLimit := core.CalcGasLimit(currentBlock, d.config.GasFloor, d.config.GasCeil)
- gp := new(core.GasPool).AddGas(gasLimit)
- stateDB, err := state.New(currentBlock.Root(), state.NewDatabase(d.chainDB))
- if err != nil {
- return
- }
-
chainID := new(big.Int).SetUint64(uint64(position.ChainID))
- chainSize := new(big.Int).SetUint64(uint64(d.gov.Configuration(position.Round).NumChains))
+ chainSize := new(big.Int).SetUint64(uint64(d.gov.GetNumChains(position.Round)))
var allTxs types.Transactions
- var totalGasUsed uint64
for addr, txs := range txsMap {
// every address's transactions will appear in fixed chain
if !d.checkChain(addr, chainSize, chainID) {
continue
}
- undeliveredTxs, err := d.blockchain.GetConfirmedTxsByAddress(position.ChainID, addr)
- if err != nil {
- return nil, err
- }
-
- for _, tx := range undeliveredTxs {
- // confirmed txs must apply successfully
- var gasUsed uint64
- gp := new(core.GasPool).AddGas(math.MaxUint64)
- _, _, err = core.ApplyTransaction(d.blockchain.Config(), d.blockchain, nil, gp, stateDB, currentBlock.Header(), tx, &gasUsed, d.vmConfig)
+ var stateDB *state.StateDB
+ if d.lastHeight > 0 {
+ stateDB, err = d.blockchain.StateAt(d.blockchain.GetPendingBlockByHeight(d.lastHeight).Root())
if err != nil {
- log.Error("apply confirmed transaction", "error", err)
- return nil, err
+ return nil, fmt.Errorf("PreparePayload d.blockchain.StateAt err %v", err)
+ }
+ } else {
+ stateDB, err = d.blockchain.State()
+ if err != nil {
+ return nil, fmt.Errorf("PreparePayload d.blockchain.State err %v", err)
}
}
for _, tx := range txs {
- // apply transaction to calculate total gas used, validate nonce and check available balance
- _, _, err = core.ApplyTransaction(d.blockchain.Config(), d.blockchain, nil, gp, stateDB, currentBlock.Header(), tx, &totalGasUsed, d.vmConfig)
- if err != nil || totalGasUsed > gasLimit {
- log.Debug("apply transaction fail", "error", err, "totalGasUsed", totalGasUsed, "gasLimit", gasLimit)
+ if tx.Nonce() != stateDB.GetNonce(addr) {
+ log.Debug("break transaction", "tx.hash", tx.Hash(), "nonce", tx.Nonce(), "expect", stateDB.GetNonce(addr))
break
}
+ log.Debug("receive transaction", "tx.hash", tx.Hash(), "nonce", tx.Nonce(), "amount", tx.Value())
allTxs = append(allTxs, tx)
}
}
@@ -167,134 +158,48 @@ func (d *DexconApp) PreparePayload(position coreTypes.Position) (payload []byte,
// PrepareWitness will return the witness data no lower than consensusHeight.
func (d *DexconApp) PrepareWitness(consensusHeight uint64) (witness coreTypes.Witness, err error) {
- var currentBlock *types.Block
- currentBlock = d.blockchain.CurrentBlock()
- if currentBlock.NumberU64() < consensusHeight {
- // wait notification
- if <-d.addNotify(consensusHeight) {
- currentBlock = d.blockchain.CurrentBlock()
- } else {
- err = fmt.Errorf("fail to wait notification")
- return
- }
+ // TODO(bojie): the witness logic need to correct
+ var witnessBlock *types.Block
+ if d.lastHeight == 0 && consensusHeight == 0 {
+ witnessBlock = d.blockchain.CurrentBlock()
+ } else if d.lastHeight >= consensusHeight {
+ witnessBlock = d.blockchain.GetPendingBlockByHeight(d.lastHeight)
+ } else if h := <-d.addNotify(consensusHeight); h >= consensusHeight {
+ witnessBlock = d.blockchain.GetPendingBlockByHeight(h)
+ } else {
+ log.Error("need pending block")
+ return witness, fmt.Errorf("need pending block")
}
witnessData, err := rlp.EncodeToBytes(&witnessData{
- Root: currentBlock.Root(),
- TxHash: currentBlock.TxHash(),
- ReceiptHash: currentBlock.ReceiptHash(),
+ Root: witnessBlock.Root(),
+ TxHash: witnessBlock.TxHash(),
+ ReceiptHash: witnessBlock.ReceiptHash(),
})
if err != nil {
return
}
return coreTypes.Witness{
- Timestamp: time.Unix(currentBlock.Time().Int64(), 0),
- Height: currentBlock.NumberU64(),
+ Timestamp: time.Unix(witnessBlock.Time().Int64(), 0),
+ Height: witnessBlock.NumberU64(),
Data: witnessData,
}, nil
}
// VerifyBlock verifies if the payloads are valid.
func (d *DexconApp) VerifyBlock(block *coreTypes.Block) bool {
- // decode payload to transactions
- var transactions types.Transactions
- err := rlp.Decode(bytes.NewReader(block.Payload), &transactions)
- if err != nil {
- return false
- }
-
- currentBlock := d.blockchain.CurrentBlock()
- chainID := new(big.Int).SetUint64(uint64(block.Position.ChainID))
- chainSize := new(big.Int).SetUint64(uint64(d.gov.Configuration(block.Position.Round).NumChains))
- stateDB, err := state.New(currentBlock.Root(), state.NewDatabase(d.chainDB))
- if err != nil {
- return false
- }
-
- // verify transactions
- addressMap := map[common.Address]interface{}{}
- gasLimit := core.CalcGasLimit(currentBlock, d.config.GasFloor, d.config.GasCeil)
- gp := new(core.GasPool).AddGas(gasLimit)
- var totalGasUsed uint64
- for _, transaction := range transactions {
- msg, err := transaction.AsMessage(types.MakeSigner(d.blockchain.Config(), currentBlock.Header().Number))
- if err != nil {
- return false
- }
-
- if !d.checkChain(msg.From(), chainSize, chainID) {
- log.Error("check chain fail", "from", msg.From().String(), "chainSize", chainSize, "chainID", chainID)
- return false
- }
-
- _, exist := addressMap[msg.From()]
- if !exist {
- txs, err := d.blockchain.GetConfirmedTxsByAddress(block.Position.ChainID, msg.From())
- if err != nil {
- log.Error("get confirmed txs by address", "error", err)
- return false
- }
-
- gp := new(core.GasPool).AddGas(math.MaxUint64)
- for _, tx := range txs {
- var gasUsed uint64
- // confirmed txs must apply successfully
- _, _, err := core.ApplyTransaction(d.blockchain.Config(), d.blockchain, nil, gp, stateDB, currentBlock.Header(), tx, &gasUsed, d.vmConfig)
- if err != nil {
- log.Error("apply confirmed transaction", "error", err)
- return false
- }
- }
- addressMap[msg.From()] = nil
- }
-
- _, _, err = core.ApplyTransaction(d.blockchain.Config(), d.blockchain, nil, gp, stateDB, currentBlock.Header(), transaction, &totalGasUsed, d.vmConfig)
- if err != nil {
- log.Error("apply block transaction", "error", err)
- return false
- }
- }
-
- if totalGasUsed > gasLimit+d.config.GasLimitTolerance {
- return false
- }
-
- witnessData := witnessData{}
- err = rlp.Decode(bytes.NewReader(block.Witness.Data), &witnessData)
- if err != nil {
- return false
- }
-
- witnessBlock := d.blockchain.GetBlockByNumber(block.Witness.Height)
- if witnessBlock == nil {
- return false
- } else if witnessBlock.Root() != witnessData.Root {
- // invalid state root of witness data
- return false
- } else if witnessBlock.ReceiptHash() != witnessData.ReceiptHash {
- // invalid receipt root of witness data
- return false
- } else if witnessBlock.TxHash() != witnessData.TxHash {
- // invalid tx root of witness data
- return false
- }
-
- for _, transaction := range witnessBlock.Transactions() {
- tx, _, _, _ := rawdb.ReadTransaction(d.chainDB, transaction.Hash())
- if tx == nil {
- return false
- }
- }
-
+ // TODO(bojie): implement this
return true
}
// BlockDelivered is called when a block is add to the compaction chain.
func (d *DexconApp) BlockDelivered(blockHash coreCommon.Hash, result coreTypes.FinalizationResult) {
+ d.insertMu.Lock()
+ defer d.insertMu.Unlock()
+
block := d.blockchain.GetConfirmedBlockByHash(blockHash)
if block == nil {
- // do something
log.Error("can not get confirmed block")
return
}
@@ -306,20 +211,34 @@ func (d *DexconApp) BlockDelivered(blockHash coreCommon.Hash, result coreTypes.F
return
}
- _, err = d.blockchain.InsertChain(
- []*types.Block{types.NewBlock(&types.Header{
- ParentHash: d.blockchain.CurrentBlock().Hash(),
- Number: new(big.Int).SetUint64(result.Height),
- Time: big.NewInt(result.Timestamp.Unix()),
- TxHash: types.DeriveSha(transactions),
- Coinbase: common.BytesToAddress(block.ProposerID.Hash[:]),
- GasLimit: 800000,
- }, transactions, nil, nil)})
+ var witnessData witnessData
+ err = rlp.Decode(bytes.NewReader(block.Witness.Data), &witnessData)
+ if err != nil {
+ log.Error("witness rlp decode", "error", err)
+ return
+ }
+
+ log.Debug("block proposer id", "hash", block.ProposerID)
+ newBlock := types.NewBlock(&types.Header{
+ Number: new(big.Int).SetUint64(result.Height),
+ Time: big.NewInt(result.Timestamp.Unix()),
+ Coinbase: common.BytesToAddress(block.ProposerID.Bytes()),
+ WitnessHeight: block.Witness.Height,
+ WitnessRoot: witnessData.Root,
+ WitnessReceiptHash: witnessData.ReceiptHash,
+ // TODO(bojie): fix it
+ GasLimit: 8000000,
+ Difficulty: big.NewInt(1),
+ }, transactions, nil, nil)
+
+ _, err = d.blockchain.InsertPendingBlock([]*types.Block{newBlock})
if err != nil {
log.Error("insert chain", "error", err)
return
}
+ log.Debug("insert pending block success", "height", result.Height)
+
d.blockchain.RemoveConfirmedBlock(blockHash)
d.notify(result.Height)
}