core, consensus: pluggable consensus engines (#3817)
This commit adds pluggable consensus engines to go-ethereum. In short, it introduces a generic consensus interface, and refactors the entire codebase to use this interface.
This commit is contained in:
committed by
Felix Lange
parent
e50a5b7771
commit
09777952ee
@ -17,56 +17,49 @@
|
||||
package miner
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"sync"
|
||||
|
||||
"sync/atomic"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/core/types"
|
||||
"github.com/ethereum/go-ethereum/consensus"
|
||||
"github.com/ethereum/go-ethereum/log"
|
||||
"github.com/ethereum/go-ethereum/pow"
|
||||
)
|
||||
|
||||
type CpuAgent struct {
|
||||
mu sync.Mutex
|
||||
|
||||
workCh chan *Work
|
||||
quit chan struct{}
|
||||
stop chan struct{}
|
||||
quitCurrentOp chan struct{}
|
||||
returnCh chan<- *Result
|
||||
|
||||
index int
|
||||
pow pow.PoW
|
||||
chain consensus.ChainReader
|
||||
engine consensus.Engine
|
||||
|
||||
isMining int32 // isMining indicates whether the agent is currently mining
|
||||
}
|
||||
|
||||
func NewCpuAgent(index int, pow pow.PoW) *CpuAgent {
|
||||
func NewCpuAgent(chain consensus.ChainReader, engine consensus.Engine) *CpuAgent {
|
||||
miner := &CpuAgent{
|
||||
pow: pow,
|
||||
index: index,
|
||||
quit: make(chan struct{}),
|
||||
chain: chain,
|
||||
engine: engine,
|
||||
stop: make(chan struct{}, 1),
|
||||
workCh: make(chan *Work, 1),
|
||||
}
|
||||
|
||||
return miner
|
||||
}
|
||||
|
||||
func (self *CpuAgent) Work() chan<- *Work { return self.workCh }
|
||||
func (self *CpuAgent) Pow() pow.PoW { return self.pow }
|
||||
func (self *CpuAgent) SetReturnCh(ch chan<- *Result) { self.returnCh = ch }
|
||||
|
||||
func (self *CpuAgent) Stop() {
|
||||
close(self.quit)
|
||||
self.stop <- struct{}{}
|
||||
}
|
||||
|
||||
func (self *CpuAgent) Start() {
|
||||
|
||||
if !atomic.CompareAndSwapInt32(&self.isMining, 0, 1) {
|
||||
return // agent already started
|
||||
}
|
||||
|
||||
go self.update()
|
||||
}
|
||||
|
||||
@ -82,7 +75,7 @@ out:
|
||||
self.quitCurrentOp = make(chan struct{})
|
||||
go self.mine(work, self.quitCurrentOp)
|
||||
self.mu.Unlock()
|
||||
case <-self.quit:
|
||||
case <-self.stop:
|
||||
self.mu.Lock()
|
||||
if self.quitCurrentOp != nil {
|
||||
close(self.quitCurrentOp)
|
||||
@ -99,27 +92,27 @@ done:
|
||||
select {
|
||||
case <-self.workCh:
|
||||
default:
|
||||
close(self.workCh)
|
||||
break done
|
||||
}
|
||||
}
|
||||
|
||||
atomic.StoreInt32(&self.isMining, 0)
|
||||
}
|
||||
|
||||
func (self *CpuAgent) mine(work *Work, stop <-chan struct{}) {
|
||||
log.Debug(fmt.Sprintf("(re)started agent[%d]. mining...\n", self.index))
|
||||
|
||||
// Mine
|
||||
nonce, mixDigest := self.pow.Search(work.Block, stop)
|
||||
if nonce != 0 {
|
||||
block := work.Block.WithMiningResult(types.EncodeNonce(nonce), common.BytesToHash(mixDigest))
|
||||
self.returnCh <- &Result{work, block}
|
||||
if result, err := self.engine.Seal(self.chain, work.Block, stop); result != nil {
|
||||
log.Info("Successfully sealed new block", "number", result.Number(), "hash", result.Hash())
|
||||
self.returnCh <- &Result{work, result}
|
||||
} else {
|
||||
if err != nil {
|
||||
log.Warn("Block sealing failed", "err", err)
|
||||
}
|
||||
self.returnCh <- nil
|
||||
}
|
||||
}
|
||||
|
||||
func (self *CpuAgent) GetHashRate() int64 {
|
||||
return int64(self.pow.Hashrate())
|
||||
if pow, ok := self.engine.(consensus.PoW); ok {
|
||||
return int64(pow.Hashrate())
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
@ -24,6 +24,7 @@ import (
|
||||
|
||||
"github.com/ethereum/go-ethereum/accounts"
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/consensus"
|
||||
"github.com/ethereum/go-ethereum/core"
|
||||
"github.com/ethereum/go-ethereum/core/state"
|
||||
"github.com/ethereum/go-ethereum/core/types"
|
||||
@ -32,7 +33,6 @@ import (
|
||||
"github.com/ethereum/go-ethereum/event"
|
||||
"github.com/ethereum/go-ethereum/log"
|
||||
"github.com/ethereum/go-ethereum/params"
|
||||
"github.com/ethereum/go-ethereum/pow"
|
||||
)
|
||||
|
||||
// Backend wraps all methods required for mining.
|
||||
@ -49,24 +49,24 @@ type Miner struct {
|
||||
|
||||
worker *worker
|
||||
|
||||
threads int
|
||||
coinbase common.Address
|
||||
mining int32
|
||||
eth Backend
|
||||
pow pow.PoW
|
||||
engine consensus.Engine
|
||||
|
||||
canStart int32 // can start indicates whether we can start the mining operation
|
||||
shouldStart int32 // should start indicates whether we should start after sync
|
||||
}
|
||||
|
||||
func New(eth Backend, config *params.ChainConfig, mux *event.TypeMux, pow pow.PoW) *Miner {
|
||||
func New(eth Backend, config *params.ChainConfig, mux *event.TypeMux, engine consensus.Engine) *Miner {
|
||||
miner := &Miner{
|
||||
eth: eth,
|
||||
mux: mux,
|
||||
pow: pow,
|
||||
worker: newWorker(config, common.Address{}, eth, mux),
|
||||
engine: engine,
|
||||
worker: newWorker(config, engine, common.Address{}, eth, mux),
|
||||
canStart: 1,
|
||||
}
|
||||
miner.Register(NewCpuAgent(eth.BlockChain(), engine))
|
||||
go miner.update()
|
||||
|
||||
return miner
|
||||
@ -86,7 +86,7 @@ out:
|
||||
if self.Mining() {
|
||||
self.Stop()
|
||||
atomic.StoreInt32(&self.shouldStart, 1)
|
||||
log.Info(fmt.Sprint("Mining operation aborted due to sync operation"))
|
||||
log.Info("Mining aborted due to sync")
|
||||
}
|
||||
case downloader.DoneEvent, downloader.FailedEvent:
|
||||
shouldStart := atomic.LoadInt32(&self.shouldStart) == 1
|
||||
@ -94,7 +94,7 @@ out:
|
||||
atomic.StoreInt32(&self.canStart, 1)
|
||||
atomic.StoreInt32(&self.shouldStart, 0)
|
||||
if shouldStart {
|
||||
self.Start(self.coinbase, self.threads)
|
||||
self.Start(self.coinbase)
|
||||
}
|
||||
// unsubscribe. we're only interested in this event once
|
||||
events.Unsubscribe()
|
||||
@ -116,23 +116,18 @@ func (m *Miner) SetGasPrice(price *big.Int) {
|
||||
m.worker.setGasPrice(price)
|
||||
}
|
||||
|
||||
func (self *Miner) Start(coinbase common.Address, threads int) {
|
||||
func (self *Miner) Start(coinbase common.Address) {
|
||||
atomic.StoreInt32(&self.shouldStart, 1)
|
||||
self.worker.setEtherbase(coinbase)
|
||||
self.coinbase = coinbase
|
||||
self.threads = threads
|
||||
|
||||
if atomic.LoadInt32(&self.canStart) == 0 {
|
||||
log.Info(fmt.Sprint("Can not start mining operation due to network sync (starts when finished)"))
|
||||
log.Info("Network syncing, will start miner afterwards")
|
||||
return
|
||||
}
|
||||
atomic.StoreInt32(&self.mining, 1)
|
||||
|
||||
for i := 0; i < threads; i++ {
|
||||
self.worker.register(NewCpuAgent(i, self.pow))
|
||||
}
|
||||
|
||||
log.Info(fmt.Sprintf("Starting mining operation (CPU=%d TOT=%d)\n", threads, len(self.worker.agents)))
|
||||
log.Info("Starting mining operation")
|
||||
self.worker.start()
|
||||
self.worker.commitNewWork()
|
||||
}
|
||||
@ -159,7 +154,9 @@ func (self *Miner) Mining() bool {
|
||||
}
|
||||
|
||||
func (self *Miner) HashRate() (tot int64) {
|
||||
tot += int64(self.pow.Hashrate())
|
||||
if pow, ok := self.engine.(consensus.PoW); ok {
|
||||
tot += int64(pow.Hashrate())
|
||||
}
|
||||
// do we care this might race? is it worth we're rewriting some
|
||||
// aspects of the worker/locking up agents so we can get an accurate
|
||||
// hashrate?
|
||||
|
@ -18,16 +18,16 @@ package miner
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"math/big"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/consensus"
|
||||
"github.com/ethereum/go-ethereum/consensus/ethash"
|
||||
"github.com/ethereum/go-ethereum/core/types"
|
||||
"github.com/ethereum/go-ethereum/log"
|
||||
"github.com/ethereum/go-ethereum/pow"
|
||||
)
|
||||
|
||||
type hashrate struct {
|
||||
@ -42,7 +42,8 @@ type RemoteAgent struct {
|
||||
workCh chan *Work
|
||||
returnCh chan<- *Result
|
||||
|
||||
pow pow.PoW
|
||||
chain consensus.ChainReader
|
||||
engine consensus.Engine
|
||||
currentWork *Work
|
||||
work map[common.Hash]*Work
|
||||
|
||||
@ -52,9 +53,10 @@ type RemoteAgent struct {
|
||||
running int32 // running indicates whether the agent is active. Call atomically
|
||||
}
|
||||
|
||||
func NewRemoteAgent(pow pow.PoW) *RemoteAgent {
|
||||
func NewRemoteAgent(chain consensus.ChainReader, engine consensus.Engine) *RemoteAgent {
|
||||
return &RemoteAgent{
|
||||
pow: pow,
|
||||
chain: chain,
|
||||
engine: engine,
|
||||
work: make(map[common.Hash]*Work),
|
||||
hashrate: make(map[common.Hash]hashrate),
|
||||
}
|
||||
@ -114,7 +116,7 @@ func (a *RemoteAgent) GetWork() ([3]string, error) {
|
||||
block := a.currentWork.Block
|
||||
|
||||
res[0] = block.HashNoNonce().Hex()
|
||||
seedHash := pow.EthashSeedHash(block.NumberU64())
|
||||
seedHash := ethash.SeedHash(block.NumberU64())
|
||||
res[1] = common.BytesToHash(seedHash).Hex()
|
||||
// Calculate the "target" to be returned to the external miner
|
||||
n := big.NewInt(1)
|
||||
@ -129,8 +131,8 @@ func (a *RemoteAgent) GetWork() ([3]string, error) {
|
||||
return res, errors.New("No work available yet, don't panic.")
|
||||
}
|
||||
|
||||
// SubmitWork tries to inject a PoW solution tinto the remote agent, returning
|
||||
// whether the solution was acceted or not (not can be both a bad PoW as well as
|
||||
// SubmitWork tries to inject a pow solution into the remote agent, returning
|
||||
// whether the solution was accepted or not (not can be both a bad pow as well as
|
||||
// any other error, like no work pending).
|
||||
func (a *RemoteAgent) SubmitWork(nonce types.BlockNonce, mixDigest, hash common.Hash) bool {
|
||||
a.mu.Lock()
|
||||
@ -139,15 +141,20 @@ func (a *RemoteAgent) SubmitWork(nonce types.BlockNonce, mixDigest, hash common.
|
||||
// Make sure the work submitted is present
|
||||
work := a.work[hash]
|
||||
if work == nil {
|
||||
log.Info(fmt.Sprintf("Work was submitted for %x but no pending work found", hash))
|
||||
log.Info("Work submitted but none pending", "hash", hash)
|
||||
return false
|
||||
}
|
||||
// Make sure the PoW solutions is indeed valid
|
||||
block := work.Block.WithMiningResult(nonce, mixDigest)
|
||||
if err := a.pow.Verify(block); err != nil {
|
||||
log.Warn(fmt.Sprintf("Invalid PoW submitted for %x: %v", hash, err))
|
||||
// Make sure the Engine solutions is indeed valid
|
||||
result := work.Block.Header()
|
||||
result.Nonce = nonce
|
||||
result.MixDigest = mixDigest
|
||||
|
||||
if err := a.engine.VerifySeal(a.chain, result); err != nil {
|
||||
log.Warn("Invalid proof-of-work submitted", "hash", hash, "err", err)
|
||||
return false
|
||||
}
|
||||
block := work.Block.WithSeal(result)
|
||||
|
||||
// Solutions seems to be valid, return to the miner and notify acceptance
|
||||
a.returnCh <- &Result{work, block}
|
||||
delete(a.work, hash)
|
||||
|
@ -18,7 +18,6 @@ package miner
|
||||
|
||||
import (
|
||||
"container/ring"
|
||||
"fmt"
|
||||
"sync"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
@ -80,7 +79,7 @@ func (set *unconfirmedBlocks) Insert(index uint64, hash common.Hash) {
|
||||
set.blocks.Move(-1).Link(item)
|
||||
}
|
||||
// Display a log for the user to notify of a new mined block unconfirmed
|
||||
log.Info(fmt.Sprintf("🔨 mined potential block #%d [%x…], waiting for %d blocks to confirm", index, hash.Bytes()[:4], set.depth))
|
||||
log.Info("🔨 mined potential block", "number", index, "hash", hash)
|
||||
}
|
||||
|
||||
// Shift drops all unconfirmed blocks from the set which exceed the unconfirmed sets depth
|
||||
@ -100,11 +99,11 @@ func (set *unconfirmedBlocks) Shift(height uint64) {
|
||||
header := set.chain.GetHeaderByNumber(next.index)
|
||||
switch {
|
||||
case header == nil:
|
||||
log.Warn(fmt.Sprintf("failed to retrieve header of mined block #%d [%x…]", next.index, next.hash.Bytes()[:4]))
|
||||
log.Warn("Failed to retrieve header of mined block", "number", next.index, "hash", next.hash)
|
||||
case header.Hash() == next.hash:
|
||||
log.Info(fmt.Sprintf("🔗 mined block #%d [%x…] reached canonical chain", next.index, next.hash.Bytes()[:4]))
|
||||
log.Info("🔗 block reached canonical chain", "number", next.index, "hash", next.hash)
|
||||
default:
|
||||
log.Info(fmt.Sprintf("⑂ mined block #%d [%x…] became a side fork", next.index, next.hash.Bytes()[:4]))
|
||||
log.Info("⑂ block became a side fork", "number", next.index, "hash", next.hash)
|
||||
}
|
||||
// Drop the block out of the ring
|
||||
if set.blocks.Value == set.blocks.Next().Value {
|
||||
|
@ -26,6 +26,8 @@ import (
|
||||
|
||||
"github.com/ethereum/go-ethereum/accounts"
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/consensus"
|
||||
"github.com/ethereum/go-ethereum/consensus/misc"
|
||||
"github.com/ethereum/go-ethereum/core"
|
||||
"github.com/ethereum/go-ethereum/core/state"
|
||||
"github.com/ethereum/go-ethereum/core/types"
|
||||
@ -34,7 +36,6 @@ import (
|
||||
"github.com/ethereum/go-ethereum/event"
|
||||
"github.com/ethereum/go-ethereum/log"
|
||||
"github.com/ethereum/go-ethereum/params"
|
||||
"github.com/ethereum/go-ethereum/pow"
|
||||
"gopkg.in/fatih/set.v0"
|
||||
)
|
||||
|
||||
@ -84,6 +85,7 @@ type Result struct {
|
||||
// worker is the main object which takes care of applying messages to the new state
|
||||
type worker struct {
|
||||
config *params.ChainConfig
|
||||
engine consensus.Engine
|
||||
|
||||
mu sync.Mutex
|
||||
|
||||
@ -94,7 +96,6 @@ type worker struct {
|
||||
|
||||
agents map[Agent]struct{}
|
||||
recv chan *Result
|
||||
pow pow.PoW
|
||||
|
||||
eth Backend
|
||||
chain *core.BlockChain
|
||||
@ -123,9 +124,10 @@ type worker struct {
|
||||
fullValidation bool
|
||||
}
|
||||
|
||||
func newWorker(config *params.ChainConfig, coinbase common.Address, eth Backend, mux *event.TypeMux) *worker {
|
||||
func newWorker(config *params.ChainConfig, engine consensus.Engine, coinbase common.Address, eth Backend, mux *event.TypeMux) *worker {
|
||||
worker := &worker{
|
||||
config: config,
|
||||
engine: engine,
|
||||
eth: eth,
|
||||
mux: mux,
|
||||
chainDb: eth.ChainDb(),
|
||||
@ -209,16 +211,10 @@ func (self *worker) stop() {
|
||||
self.mu.Lock()
|
||||
defer self.mu.Unlock()
|
||||
if atomic.LoadInt32(&self.mining) == 1 {
|
||||
// Stop all agents.
|
||||
for agent := range self.agents {
|
||||
agent.Stop()
|
||||
// Remove CPU agents.
|
||||
if _, ok := agent.(*CpuAgent); ok {
|
||||
delete(self.agents, agent)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
atomic.StoreInt32(&self.mining, 0)
|
||||
atomic.StoreInt32(&self.atWork, 0)
|
||||
}
|
||||
@ -277,7 +273,7 @@ func (self *worker) wait() {
|
||||
|
||||
if self.fullValidation {
|
||||
if _, err := self.chain.InsertChain(types.Blocks{block}); err != nil {
|
||||
log.Error(fmt.Sprint("mining err", err))
|
||||
log.Error("Mined invalid block", "err", err)
|
||||
continue
|
||||
}
|
||||
go self.mux.Post(core.NewMinedBlockEvent{Block: block})
|
||||
@ -285,19 +281,16 @@ func (self *worker) wait() {
|
||||
work.state.Commit(self.config.IsEIP158(block.Number()))
|
||||
parent := self.chain.GetBlock(block.ParentHash(), block.NumberU64()-1)
|
||||
if parent == nil {
|
||||
log.Error(fmt.Sprint("Invalid block found during mining"))
|
||||
log.Error("Invalid block found during mining")
|
||||
continue
|
||||
}
|
||||
|
||||
auxValidator := self.eth.BlockChain().AuxValidator()
|
||||
if err := core.ValidateHeader(self.config, auxValidator, block.Header(), parent.Header(), true, false); err != nil && err != core.BlockFutureErr {
|
||||
log.Error(fmt.Sprint("Invalid header on mined block:", err))
|
||||
if err := self.engine.VerifyHeader(self.chain, block.Header(), false); err != nil {
|
||||
log.Error("Invalid header on mined block", "err", err)
|
||||
continue
|
||||
}
|
||||
|
||||
stat, err := self.chain.WriteBlock(block)
|
||||
if err != nil {
|
||||
log.Error(fmt.Sprint("error writing block to chain", err))
|
||||
log.Error("Failed writing block to chain", "err", err)
|
||||
continue
|
||||
}
|
||||
|
||||
@ -333,7 +326,7 @@ func (self *worker) wait() {
|
||||
self.mux.Post(logs)
|
||||
}
|
||||
if err := core.WriteBlockReceipts(self.chainDb, block.Hash(), block.NumberU64(), receipts); err != nil {
|
||||
log.Warn(fmt.Sprint("error writing block receipts:", err))
|
||||
log.Warn("Failed writing block receipts", "err", err)
|
||||
}
|
||||
}(block, work.state.Logs(), work.receipts)
|
||||
}
|
||||
@ -424,9 +417,9 @@ func (self *worker) commitNewWork() {
|
||||
tstamp = parent.Time().Int64() + 1
|
||||
}
|
||||
// this will ensure we're not going off too far in the future
|
||||
if now := time.Now().Unix(); tstamp > now+4 {
|
||||
if now := time.Now().Unix(); tstamp > now+1 {
|
||||
wait := time.Duration(tstamp-now) * time.Second
|
||||
log.Info(fmt.Sprint("We are too far in the future. Waiting for", wait))
|
||||
log.Info("Mining too far in the future", "wait", common.PrettyDuration(wait))
|
||||
time.Sleep(wait)
|
||||
}
|
||||
|
||||
@ -434,13 +427,19 @@ func (self *worker) commitNewWork() {
|
||||
header := &types.Header{
|
||||
ParentHash: parent.Hash(),
|
||||
Number: num.Add(num, common.Big1),
|
||||
Difficulty: core.CalcDifficulty(self.config, uint64(tstamp), parent.Time().Uint64(), parent.Number(), parent.Difficulty()),
|
||||
GasLimit: core.CalcGasLimit(parent),
|
||||
GasUsed: new(big.Int),
|
||||
Coinbase: self.coinbase,
|
||||
Extra: self.extra,
|
||||
Time: big.NewInt(tstamp),
|
||||
}
|
||||
// Only set the coinbase if we are mining (avoid spurious block rewards)
|
||||
if atomic.LoadInt32(&self.mining) == 1 {
|
||||
header.Coinbase = self.coinbase
|
||||
}
|
||||
if err := self.engine.Prepare(self.chain, header); err != nil {
|
||||
log.Error("Failed to prepare header for mining", "err", err)
|
||||
return
|
||||
}
|
||||
// If we are care about TheDAO hard-fork check whether to override the extra-data or not
|
||||
if daoBlock := self.config.DAOForkBlock; daoBlock != nil {
|
||||
// Check whether the block is among the fork extra-override range
|
||||
@ -457,21 +456,19 @@ func (self *worker) commitNewWork() {
|
||||
// Could potentially happen if starting to mine in an odd state.
|
||||
err := self.makeCurrent(parent, header)
|
||||
if err != nil {
|
||||
log.Info(fmt.Sprint("Could not create new env for mining, retrying on next block."))
|
||||
log.Error("Failed to create mining context", "err", err)
|
||||
return
|
||||
}
|
||||
// Create the current work task and check any fork transitions needed
|
||||
work := self.current
|
||||
if self.config.DAOForkSupport && self.config.DAOForkBlock != nil && self.config.DAOForkBlock.Cmp(header.Number) == 0 {
|
||||
core.ApplyDAOHardFork(work.state)
|
||||
misc.ApplyDAOHardFork(work.state)
|
||||
}
|
||||
|
||||
pending, err := self.eth.TxPool().Pending()
|
||||
if err != nil {
|
||||
log.Error(fmt.Sprintf("Could not fetch pending transactions: %v", err))
|
||||
log.Error("Failed to fetch pending transactions", "err", err)
|
||||
return
|
||||
}
|
||||
|
||||
txs := types.NewTransactionsByPriceAndNonce(pending)
|
||||
work.commitTransactions(self.mux, txs, self.gasPrice, self.chain)
|
||||
|
||||
@ -488,31 +485,26 @@ func (self *worker) commitNewWork() {
|
||||
break
|
||||
}
|
||||
if err := self.commitUncle(work, uncle.Header()); err != nil {
|
||||
log.Trace(fmt.Sprintf("Bad uncle found and will be removed (%x)\n", hash[:4]))
|
||||
log.Trace("Bad uncle found and will be removed", "hash", hash)
|
||||
log.Trace(fmt.Sprint(uncle))
|
||||
|
||||
badUncles = append(badUncles, hash)
|
||||
} else {
|
||||
log.Debug(fmt.Sprintf("committing %x as uncle\n", hash[:4]))
|
||||
log.Debug("Committing new uncle to block", "hash", hash)
|
||||
uncles = append(uncles, uncle.Header())
|
||||
}
|
||||
}
|
||||
for _, hash := range badUncles {
|
||||
delete(self.possibleUncles, hash)
|
||||
}
|
||||
|
||||
if atomic.LoadInt32(&self.mining) == 1 {
|
||||
// commit state root after all state transitions.
|
||||
core.AccumulateRewards(work.state, header, uncles)
|
||||
header.Root = work.state.IntermediateRoot(self.config.IsEIP158(header.Number))
|
||||
// Create the new block to seal with the consensus engine
|
||||
if work.Block, err = self.engine.Finalize(self.chain, header, work.state, work.txs, uncles, work.receipts); err != nil {
|
||||
log.Error("Failed to finalize block for sealing", "err", err)
|
||||
return
|
||||
}
|
||||
|
||||
// create the new block whose nonce will be mined.
|
||||
work.Block = types.NewBlock(header, work.txs, uncles, work.receipts)
|
||||
|
||||
// We only care about logging if we're actually mining.
|
||||
if atomic.LoadInt32(&self.mining) == 1 {
|
||||
log.Info(fmt.Sprintf("commit new work on block %v with %d txs & %d uncles. Took %v\n", work.Block.Number(), work.tcount, len(uncles), time.Since(tstart)))
|
||||
log.Info("Commit new mining work", "number", work.Block.Number(), "txs", work.tcount, "uncles", len(uncles), "elapsed", common.PrettyDuration(time.Since(tstart)))
|
||||
self.unconfirmed.Shift(work.Block.NumberU64() - 1)
|
||||
}
|
||||
self.push(work)
|
||||
@ -521,13 +513,13 @@ func (self *worker) commitNewWork() {
|
||||
func (self *worker) commitUncle(work *Work, uncle *types.Header) error {
|
||||
hash := uncle.Hash()
|
||||
if work.uncles.Has(hash) {
|
||||
return core.UncleError("Uncle not unique")
|
||||
return core.UncleError("uncle not unique")
|
||||
}
|
||||
if !work.ancestors.Has(uncle.ParentHash) {
|
||||
return core.UncleError(fmt.Sprintf("Uncle's parent unknown (%x)", uncle.ParentHash[0:4]))
|
||||
return core.UncleError(fmt.Sprintf("uncle's parent unknown (%x)", uncle.ParentHash[0:4]))
|
||||
}
|
||||
if work.family.Has(hash) {
|
||||
return core.UncleError(fmt.Sprintf("Uncle already in family (%x)", hash))
|
||||
return core.UncleError(fmt.Sprintf("uncle already in family (%x)", hash))
|
||||
}
|
||||
work.uncles.Add(uncle.Hash())
|
||||
return nil
|
||||
@ -552,7 +544,7 @@ func (env *Work) commitTransactions(mux *event.TypeMux, txs *types.TransactionsB
|
||||
// Check whether the tx is replay protected. If we're not in the EIP155 hf
|
||||
// phase, start ignoring the sender until we do.
|
||||
if tx.Protected() && !env.config.IsEIP155(env.header.Number) {
|
||||
log.Trace(fmt.Sprintf("Transaction (%x) is replay protected, but we haven't yet hardforked. Transaction will be ignored until we hardfork.\n", tx.Hash()))
|
||||
log.Trace("Ignoring reply protected transaction", "hash", tx.Hash(), "eip155", env.config.EIP155Block)
|
||||
|
||||
txs.Pop()
|
||||
continue
|
||||
@ -561,7 +553,7 @@ func (env *Work) commitTransactions(mux *event.TypeMux, txs *types.TransactionsB
|
||||
// Ignore any transactions (and accounts subsequently) with low gas limits
|
||||
if tx.GasPrice().Cmp(gasPrice) < 0 && !env.ownedAccounts.Has(from) {
|
||||
// Pop the current low-priced transaction without shifting in the next from the account
|
||||
log.Info(fmt.Sprintf("Transaction (%x) below gas price (tx=%dwei ask=%dwei). All sequential txs from this address(%x) will be ignored\n", tx.Hash().Bytes()[:4], tx.GasPrice(), gasPrice, from[:4]))
|
||||
log.Warn("Transaction below gas price", "sender", from, "hash", tx.Hash(), "have", tx.GasPrice(), "want", gasPrice)
|
||||
|
||||
env.lowGasTxs = append(env.lowGasTxs, tx)
|
||||
txs.Pop()
|
||||
@ -575,12 +567,12 @@ func (env *Work) commitTransactions(mux *event.TypeMux, txs *types.TransactionsB
|
||||
switch {
|
||||
case core.IsGasLimitErr(err):
|
||||
// Pop the current out-of-gas transaction without shifting in the next from the account
|
||||
log.Trace(fmt.Sprintf("Gas limit reached for (%x) in this block. Continue to try smaller txs\n", from[:4]))
|
||||
log.Trace("Gas limit exceeded for current block", "sender", from)
|
||||
txs.Pop()
|
||||
|
||||
case err != nil:
|
||||
// Pop the current failed transaction without shifting in the next from the account
|
||||
log.Trace(fmt.Sprintf("Transaction (%x) failed, will be removed: %v\n", tx.Hash().Bytes()[:4], err))
|
||||
log.Trace("Transaction failed, will be removed", "hash", tx.Hash(), "err", err)
|
||||
env.failedTxs = append(env.failedTxs, tx)
|
||||
txs.Pop()
|
||||
|
||||
|
Reference in New Issue
Block a user