engine.go 19 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603
  1. package qbftengine
  2. import (
  3. "bytes"
  4. "fmt"
  5. "math/big"
  6. "time"
  7. "github.com/ethereum/go-ethereum/accounts/abi/bind"
  8. "github.com/ethereum/go-ethereum/common"
  9. "github.com/ethereum/go-ethereum/consensus"
  10. "github.com/ethereum/go-ethereum/consensus/istanbul"
  11. "github.com/ethereum/go-ethereum/consensus/istanbul/backend/contract"
  12. istanbulcommon "github.com/ethereum/go-ethereum/consensus/istanbul/common"
  13. "github.com/ethereum/go-ethereum/consensus/istanbul/validator"
  14. "github.com/ethereum/go-ethereum/core/state"
  15. "github.com/ethereum/go-ethereum/core/types"
  16. "github.com/ethereum/go-ethereum/log"
  17. "github.com/ethereum/go-ethereum/params"
  18. "github.com/ethereum/go-ethereum/rlp"
  19. "github.com/ethereum/go-ethereum/trie"
  20. "golang.org/x/crypto/sha3"
  21. )
  22. var (
  23. nilUncleHash = types.CalcUncleHash(nil) // Always Keccak256(RLP([])) as uncles are meaningless outside of PoW.
  24. )
  25. type SignerFn func(data []byte) ([]byte, error)
  26. type Engine struct {
  27. cfg *istanbul.Config
  28. signer common.Address // Ethereum address of the signing key
  29. sign SignerFn // Signer function to authorize hashes with
  30. }
  31. func NewEngine(cfg *istanbul.Config, signer common.Address, sign SignerFn) *Engine {
  32. return &Engine{
  33. cfg: cfg,
  34. signer: signer,
  35. sign: sign,
  36. }
  37. }
  38. func (e *Engine) Author(header *types.Header) (common.Address, error) {
  39. return header.Coinbase, nil
  40. }
  41. func (e *Engine) CommitHeader(header *types.Header, seals [][]byte, round *big.Int) error {
  42. return ApplyHeaderQBFTExtra(
  43. header,
  44. writeCommittedSeals(seals),
  45. writeRoundNumber(round),
  46. )
  47. }
  48. // writeCommittedSeals writes the extra-data field of a block header with given committed seals.
  49. func writeCommittedSeals(committedSeals [][]byte) ApplyQBFTExtra {
  50. return func(qbftExtra *types.QBFTExtra) error {
  51. if len(committedSeals) == 0 {
  52. return istanbulcommon.ErrInvalidCommittedSeals
  53. }
  54. for _, seal := range committedSeals {
  55. if len(seal) != types.IstanbulExtraSeal {
  56. return istanbulcommon.ErrInvalidCommittedSeals
  57. }
  58. }
  59. qbftExtra.CommittedSeal = make([][]byte, len(committedSeals))
  60. copy(qbftExtra.CommittedSeal, committedSeals)
  61. return nil
  62. }
  63. }
  64. // writeRoundNumber writes the extra-data field of a block header with given round.
  65. func writeRoundNumber(round *big.Int) ApplyQBFTExtra {
  66. return func(qbftExtra *types.QBFTExtra) error {
  67. qbftExtra.Round = uint32(round.Uint64())
  68. return nil
  69. }
  70. }
  71. func (e *Engine) VerifyBlockProposal(chain consensus.ChainHeaderReader, block *types.Block, validators istanbul.ValidatorSet) (time.Duration, error) {
  72. // check block body
  73. txnHash := types.DeriveSha(block.Transactions(), new(trie.Trie))
  74. if txnHash != block.Header().TxHash {
  75. return 0, istanbulcommon.ErrMismatchTxhashes
  76. }
  77. uncleHash := types.CalcUncleHash(block.Uncles())
  78. if uncleHash != nilUncleHash {
  79. return 0, istanbulcommon.ErrInvalidUncleHash
  80. }
  81. // verify the header of proposed block
  82. err := e.VerifyHeader(chain, block.Header(), nil, validators)
  83. if err == nil || err == istanbulcommon.ErrEmptyCommittedSeals {
  84. // ignore errEmptyCommittedSeals error because we don't have the committed seals yet
  85. return 0, nil
  86. } else if err == consensus.ErrFutureBlock {
  87. return time.Until(time.Unix(int64(block.Header().Time), 0)), consensus.ErrFutureBlock
  88. }
  89. parentHeader := chain.GetHeaderByHash(block.ParentHash())
  90. config := e.cfg.GetConfig(parentHeader.Number)
  91. if config.EmptyBlockPeriod > config.BlockPeriod && len(block.Transactions()) == 0 {
  92. if block.Header().Time < parentHeader.Time+config.EmptyBlockPeriod {
  93. return 0, fmt.Errorf("empty block verification fail")
  94. }
  95. }
  96. return 0, err
  97. }
  98. func (e *Engine) VerifyHeader(chain consensus.ChainHeaderReader, header *types.Header, parents []*types.Header, validators istanbul.ValidatorSet) error {
  99. return e.verifyHeader(chain, header, parents, validators)
  100. }
  101. // verifyHeader checks whether a header conforms to the consensus rules.The
  102. // caller may optionally pass in a batch of parents (ascending order) to avoid
  103. // looking those up from the database. This is useful for concurrently verifying
  104. // a batch of new headers.
  105. func (e *Engine) verifyHeader(chain consensus.ChainHeaderReader, header *types.Header, parents []*types.Header, validators istanbul.ValidatorSet) error {
  106. if header.Number == nil {
  107. return istanbulcommon.ErrUnknownBlock
  108. }
  109. // Don't waste time checking blocks from the future (adjusting for allowed threshold)
  110. adjustedTimeNow := time.Now().Add(time.Duration(e.cfg.AllowedFutureBlockTime) * time.Second).Unix()
  111. if header.Time > uint64(adjustedTimeNow) {
  112. return consensus.ErrFutureBlock
  113. }
  114. if _, err := types.ExtractQBFTExtra(header); err != nil {
  115. return istanbulcommon.ErrInvalidExtraDataFormat
  116. }
  117. // Ensure that the mix digest is zero as we don't have fork protection currently
  118. if header.MixDigest != types.IstanbulDigest {
  119. return istanbulcommon.ErrInvalidMixDigest
  120. }
  121. // Ensure that the block doesn't contain any uncles which are meaningless in Istanbul
  122. if header.UncleHash != nilUncleHash {
  123. return istanbulcommon.ErrInvalidUncleHash
  124. }
  125. // Ensure that the block's difficulty is meaningful (may not be correct at this point)
  126. if header.Difficulty == nil || header.Difficulty.Cmp(istanbulcommon.DefaultDifficulty) != 0 {
  127. return istanbulcommon.ErrInvalidDifficulty
  128. }
  129. return e.verifyCascadingFields(chain, header, validators, parents)
  130. }
  131. func (e *Engine) VerifyHeaders(chain consensus.ChainHeaderReader, headers []*types.Header, seals []bool, validators istanbul.ValidatorSet) (chan<- struct{}, <-chan error) {
  132. abort := make(chan struct{})
  133. results := make(chan error, len(headers))
  134. go func() {
  135. errored := false
  136. for i, header := range headers {
  137. var err error
  138. if errored {
  139. err = consensus.ErrUnknownAncestor
  140. } else {
  141. err = e.verifyHeader(chain, header, headers[:i], validators)
  142. }
  143. if err != nil {
  144. errored = true
  145. }
  146. select {
  147. case <-abort:
  148. return
  149. case results <- err:
  150. }
  151. }
  152. }()
  153. return abort, results
  154. }
  155. // verifyCascadingFields verifies all the header fields that are not standalone,
  156. // rather depend on a batch of previous headers. The caller may optionally pass
  157. // in a batch of parents (ascending order) to avoid looking those up from the
  158. // database. This is useful for concurrently verifying a batch of new headers.
  159. func (e *Engine) verifyCascadingFields(chain consensus.ChainHeaderReader, header *types.Header, validators istanbul.ValidatorSet, parents []*types.Header) error {
  160. // The genesis block is the always valid dead-end
  161. number := header.Number.Uint64()
  162. if number == 0 {
  163. return nil
  164. }
  165. // Check parent
  166. var parent *types.Header
  167. if len(parents) > 0 {
  168. parent = parents[len(parents)-1]
  169. } else {
  170. parent = chain.GetHeader(header.ParentHash, number-1)
  171. }
  172. // Ensure that the block's parent has right number and hash
  173. if parent == nil || parent.Number.Uint64() != number-1 || parent.Hash() != header.ParentHash {
  174. return consensus.ErrUnknownAncestor
  175. }
  176. // Ensure that the block's timestamp isn't too close to it's parent
  177. // When the BlockPeriod is reduced it is reduced for the proposal.
  178. // e.g when blockperiod is 1 from block 10 the block period between 9 and 10 is 1
  179. if parent.Time+e.cfg.GetConfig(header.Number).BlockPeriod > header.Time {
  180. return istanbulcommon.ErrInvalidTimestamp
  181. }
  182. // Verify signer
  183. if err := e.verifySigner(chain, header, parents, validators); err != nil {
  184. return err
  185. }
  186. return e.verifyCommittedSeals(chain, header, parents, validators)
  187. }
  188. func (e *Engine) verifySigner(chain consensus.ChainHeaderReader, header *types.Header, parents []*types.Header, validators istanbul.ValidatorSet) error {
  189. // Verifying the genesis block is not supported
  190. number := header.Number.Uint64()
  191. if number == 0 {
  192. return istanbulcommon.ErrUnknownBlock
  193. }
  194. // Resolve the authorization key and check against signers
  195. signer, err := e.Author(header)
  196. if err != nil {
  197. return err
  198. }
  199. // Signer should be in the validator set of previous block's extraData.
  200. if _, v := validators.GetByAddress(signer); v == nil {
  201. return istanbulcommon.ErrUnauthorized
  202. }
  203. return nil
  204. }
  205. // verifyCommittedSeals checks whether every committed seal is signed by one of the parent's validators
  206. func (e *Engine) verifyCommittedSeals(chain consensus.ChainHeaderReader, header *types.Header, parents []*types.Header, validators istanbul.ValidatorSet) error {
  207. number := header.Number.Uint64()
  208. if number == 0 {
  209. // We don't need to verify committed seals in the genesis block
  210. return nil
  211. }
  212. extra, err := types.ExtractQBFTExtra(header)
  213. if err != nil {
  214. return err
  215. }
  216. committedSeal := extra.CommittedSeal
  217. // The length of Committed seals should be larger than 0
  218. if len(committedSeal) == 0 {
  219. return istanbulcommon.ErrEmptyCommittedSeals
  220. }
  221. validatorsCpy := validators.Copy()
  222. // Check whether the committed seals are generated by validators
  223. validSeal := 0
  224. committers, err := e.Signers(header)
  225. if err != nil {
  226. return err
  227. }
  228. for _, addr := range committers {
  229. if validatorsCpy.RemoveValidator(addr) {
  230. validSeal++
  231. continue
  232. }
  233. return istanbulcommon.ErrInvalidCommittedSeals
  234. }
  235. // The length of validSeal should be larger than number of faulty node + 1
  236. if validSeal <= validators.F() {
  237. return istanbulcommon.ErrInvalidCommittedSeals
  238. }
  239. return nil
  240. }
  241. // VerifyUncles verifies that the given block's uncles conform to the consensus
  242. // rules of a given engine.
  243. func (e *Engine) VerifyUncles(chain consensus.ChainReader, block *types.Block) error {
  244. if len(block.Uncles()) > 0 {
  245. return istanbulcommon.ErrInvalidUncleHash
  246. }
  247. return nil
  248. }
  249. // VerifySeal checks whether the crypto seal on a header is valid according to
  250. // the consensus rules of the given engine.
  251. func (e *Engine) VerifySeal(chain consensus.ChainHeaderReader, header *types.Header, validators istanbul.ValidatorSet) error {
  252. // get parent header and ensure the signer is in parent's validator set
  253. number := header.Number.Uint64()
  254. if number == 0 {
  255. return istanbulcommon.ErrUnknownBlock
  256. }
  257. // ensure that the difficulty equals to istanbulcommon.DefaultDifficulty
  258. if header.Difficulty.Cmp(istanbulcommon.DefaultDifficulty) != 0 {
  259. return istanbulcommon.ErrInvalidDifficulty
  260. }
  261. return e.verifySigner(chain, header, nil, validators)
  262. }
  263. func (e *Engine) Prepare(chain consensus.ChainHeaderReader, header *types.Header, validators istanbul.ValidatorSet) error {
  264. header.Coinbase = common.Address{}
  265. header.Nonce = istanbulcommon.EmptyBlockNonce
  266. header.MixDigest = types.IstanbulDigest
  267. // copy the parent extra data as the header extra data
  268. number := header.Number.Uint64()
  269. parent := chain.GetHeader(header.ParentHash, number-1)
  270. if parent == nil {
  271. return consensus.ErrUnknownAncestor
  272. }
  273. // use the same difficulty for all blocks
  274. header.Difficulty = istanbulcommon.DefaultDifficulty
  275. // set header's timestamp
  276. header.Time = parent.Time + e.cfg.GetConfig(header.Number).BlockPeriod
  277. if header.Time < uint64(time.Now().Unix()) {
  278. header.Time = uint64(time.Now().Unix())
  279. }
  280. currentBlockNumber := big.NewInt(0).SetUint64(number - 1)
  281. validatorContract := e.cfg.GetValidatorContractAddress(currentBlockNumber)
  282. if validatorContract != (common.Address{}) && e.cfg.GetValidatorSelectionMode(currentBlockNumber) == params.ContractMode {
  283. return ApplyHeaderQBFTExtra(
  284. header,
  285. WriteValidators([]common.Address{}),
  286. )
  287. } else {
  288. for _, transition := range e.cfg.Transitions {
  289. if transition.Block.Cmp(currentBlockNumber) == 0 && len(transition.Validators) > 0 {
  290. toRemove := make([]istanbul.Validator, 0, validators.Size())
  291. l := validators.List()
  292. for i := range l {
  293. toRemove = append(toRemove, l[i])
  294. }
  295. for i := range toRemove {
  296. validators.RemoveValidator(toRemove[i].Address())
  297. }
  298. for i := range transition.Validators {
  299. validators.AddValidator(transition.Validators[i])
  300. }
  301. break
  302. }
  303. }
  304. validatorsList := validator.SortedAddresses(validators.List())
  305. // add validators in snapshot to extraData's validators section
  306. return ApplyHeaderQBFTExtra(
  307. header,
  308. WriteValidators(validatorsList),
  309. )
  310. }
  311. }
  312. func WriteValidators(validators []common.Address) ApplyQBFTExtra {
  313. return func(qbftExtra *types.QBFTExtra) error {
  314. qbftExtra.Validators = validators
  315. return nil
  316. }
  317. }
  318. // Finalize runs any post-transaction state modifications (e.g. block rewards)
  319. // and assembles the final block.
  320. //
  321. // Note, the block header and state database might be updated to reflect any
  322. // consensus rules that happen at finalization (e.g. block rewards).
  323. func (e *Engine) Finalize(chain consensus.ChainHeaderReader, header *types.Header, state *state.StateDB, txs []*types.Transaction, uncles []*types.Header) {
  324. // Accumulate any block and uncle rewards and commit the final state root
  325. e.accumulateRewards(chain, state, header)
  326. header.Root = state.IntermediateRoot(chain.Config().IsEIP158(header.Number))
  327. header.UncleHash = nilUncleHash
  328. }
  329. // FinalizeAndAssemble implements consensus.Engine, ensuring no uncles are set,
  330. // nor block rewards given, and returns the final block.
  331. func (e *Engine) FinalizeAndAssemble(chain consensus.ChainHeaderReader, header *types.Header, state *state.StateDB, txs []*types.Transaction, uncles []*types.Header, receipts []*types.Receipt) (*types.Block, error) {
  332. e.Finalize(chain, header, state, txs, uncles)
  333. // Assemble and return the final block for sealing
  334. return types.NewBlock(header, txs, nil, receipts, new(trie.Trie)), nil
  335. }
  336. // Seal generates a new block for the given input block with the local miner's
  337. // seal place on top.
  338. func (e *Engine) Seal(chain consensus.ChainHeaderReader, block *types.Block, validators istanbul.ValidatorSet) (*types.Block, error) {
  339. if _, v := validators.GetByAddress(e.signer); v == nil {
  340. return block, istanbulcommon.ErrUnauthorized
  341. }
  342. header := block.Header()
  343. parent := chain.GetHeader(header.ParentHash, header.Number.Uint64()-1)
  344. if parent == nil {
  345. return block, consensus.ErrUnknownAncestor
  346. }
  347. // Set Coinbase
  348. header.Coinbase = e.signer
  349. return block.WithSeal(header), nil
  350. }
  351. func (e *Engine) SealHash(header *types.Header) common.Hash {
  352. header.Coinbase = e.signer
  353. return sigHash(header)
  354. }
  355. func (e *Engine) CalcDifficulty(chain consensus.ChainHeaderReader, time uint64, parent *types.Header) *big.Int {
  356. return new(big.Int)
  357. }
  358. func (e *Engine) ExtractGenesisValidators(header *types.Header) ([]common.Address, error) {
  359. extra, err := types.ExtractQBFTExtra(header)
  360. if err != nil {
  361. return nil, err
  362. }
  363. return extra.Validators, nil
  364. }
  365. func (e *Engine) Signers(header *types.Header) ([]common.Address, error) {
  366. extra, err := types.ExtractQBFTExtra(header)
  367. if err != nil {
  368. return []common.Address{}, err
  369. }
  370. committedSeal := extra.CommittedSeal
  371. proposalSeal := PrepareCommittedSeal(header, extra.Round)
  372. var addrs []common.Address
  373. // 1. Get committed seals from current header
  374. for _, seal := range committedSeal {
  375. // 2. Get the original address by seal and parent block hash
  376. addr, err := istanbul.GetSignatureAddressNoHashing(proposalSeal, seal)
  377. if err != nil {
  378. return nil, istanbulcommon.ErrInvalidSignature
  379. }
  380. addrs = append(addrs, addr)
  381. }
  382. return addrs, nil
  383. }
  384. func (e *Engine) Address() common.Address {
  385. return e.signer
  386. }
  387. // FIXME: Need to update this for Istanbul
  388. // sigHash returns the hash which is used as input for the Istanbul
  389. // signing. It is the hash of the entire header apart from the 65 byte signature
  390. // contained at the end of the extra data.
  391. //
  392. // Note, the method requires the extra data to be at least 65 bytes, otherwise it
  393. // panics. This is done to avoid accidentally using both forms (signature present
  394. // or not), which could be abused to produce different hashes for the same header.
  395. func sigHash(header *types.Header) (hash common.Hash) {
  396. hasher := sha3.NewLegacyKeccak256()
  397. rlp.Encode(hasher, types.QBFTFilteredHeader(header))
  398. hasher.Sum(hash[:0])
  399. return hash
  400. }
  401. // PrepareCommittedSeal returns a committed seal for the given hash
  402. func PrepareCommittedSeal(header *types.Header, round uint32) []byte {
  403. h := types.CopyHeader(header)
  404. return h.QBFTHashWithRoundNumber(round).Bytes()
  405. }
  406. func (e *Engine) WriteVote(header *types.Header, candidate common.Address, authorize bool) error {
  407. return ApplyHeaderQBFTExtra(
  408. header,
  409. WriteVote(candidate, authorize),
  410. )
  411. }
  412. func WriteVote(candidate common.Address, authorize bool) ApplyQBFTExtra {
  413. return func(qbftExtra *types.QBFTExtra) error {
  414. voteType := types.QBFTDropVote
  415. if authorize {
  416. voteType = types.QBFTAuthVote
  417. }
  418. vote := &types.ValidatorVote{RecipientAddress: candidate, VoteType: voteType}
  419. qbftExtra.Vote = vote
  420. return nil
  421. }
  422. }
  423. func (e *Engine) ReadVote(header *types.Header) (candidate common.Address, authorize bool, err error) {
  424. qbftExtra, err := getExtra(header)
  425. if err != nil {
  426. return common.Address{}, false, err
  427. }
  428. var vote *types.ValidatorVote
  429. if qbftExtra.Vote == nil {
  430. vote = &types.ValidatorVote{RecipientAddress: common.Address{}, VoteType: types.QBFTDropVote}
  431. } else {
  432. vote = qbftExtra.Vote
  433. }
  434. // Tally up the new vote from the validator
  435. switch {
  436. case vote.VoteType == types.QBFTAuthVote:
  437. authorize = true
  438. case vote.VoteType == types.QBFTDropVote:
  439. authorize = false
  440. default:
  441. return common.Address{}, false, istanbulcommon.ErrInvalidVote
  442. }
  443. return vote.RecipientAddress, authorize, nil
  444. }
  445. func getExtra(header *types.Header) (*types.QBFTExtra, error) {
  446. if len(header.Extra) < types.IstanbulExtraVanity {
  447. // In this scenario, the header extradata only contains client specific information, hence create a new qbftExtra and set vanity
  448. vanity := append(header.Extra, bytes.Repeat([]byte{0x00}, types.IstanbulExtraVanity-len(header.Extra))...)
  449. return &types.QBFTExtra{
  450. VanityData: vanity,
  451. Validators: []common.Address{},
  452. CommittedSeal: [][]byte{},
  453. Round: 0,
  454. Vote: nil,
  455. }, nil
  456. }
  457. // This is the case when Extra has already been set
  458. return types.ExtractQBFTExtra(header)
  459. }
  460. func setExtra(h *types.Header, qbftExtra *types.QBFTExtra) error {
  461. payload, err := rlp.EncodeToBytes(qbftExtra)
  462. if err != nil {
  463. return err
  464. }
  465. h.Extra = payload
  466. return nil
  467. }
  468. func (e *Engine) validatorsList(genesis *types.Header, config istanbul.Config) ([]common.Address, error) {
  469. var validators []common.Address
  470. if config.ValidatorContract != (common.Address{}) && config.GetValidatorSelectionMode(big.NewInt(0)) == params.ContractMode {
  471. log.Info("Initialising snap with contract validators", "address", config.ValidatorContract, "client", config.Client)
  472. validatorContractCaller, err := contract.NewValidatorContractInterfaceCaller(config.ValidatorContract, config.Client)
  473. if err != nil {
  474. return nil, fmt.Errorf("invalid smart contract in genesis alloc: %w", err)
  475. }
  476. opts := bind.CallOpts{
  477. Pending: false,
  478. BlockNumber: big.NewInt(0),
  479. }
  480. validators, err = validatorContractCaller.GetValidators(&opts)
  481. if err != nil {
  482. log.Error("QBFT: invalid smart contract in genesis alloc", "err", err)
  483. return nil, err
  484. }
  485. } else {
  486. // Get the validators from genesis to create a snapshot
  487. var err error
  488. validators, err = e.ExtractGenesisValidators(genesis)
  489. if err != nil {
  490. log.Error("BFT: invalid genesis block", "err", err)
  491. return nil, err
  492. }
  493. }
  494. return validators, nil
  495. }
  496. // AccumulateRewards credits the beneficiary of the given block with a reward.
  497. func (e *Engine) accumulateRewards(chain consensus.ChainHeaderReader, state *state.StateDB, header *types.Header) {
  498. blockReward := chain.Config().GetBlockReward(header.Number)
  499. if blockReward.Cmp(big.NewInt(0)) > 0 {
  500. coinbase := header.Coinbase
  501. if (coinbase == common.Address{}) {
  502. coinbase = e.signer
  503. }
  504. rewardAccount, _ := chain.Config().GetRewardAccount(header.Number, coinbase)
  505. log.Trace("QBFT: accumulate rewards to", "rewardAccount", rewardAccount, "blockReward", blockReward)
  506. state.AddBalance(rewardAccount, &blockReward)
  507. }
  508. }