From 6df7230108a318a0ca962e2884b3f3c53052611d Mon Sep 17 00:00:00 2001
From: Alex Sharov <AskAlexSharov@gmail.com>
Date: Wed, 30 Jun 2021 16:30:21 +0700
Subject: [PATCH] Sokol v0 - better header rlp, steps to generate blocks
 (#2256)

---
 cmd/integration/commands/reset_state.go       |   6 +
 consensus/aura/aura.go                        | 482 ++++++++++++++----
 consensus/aura/aura_test.go                   | 189 +++++++
 consensus/aura/config.go                      |  32 --
 consensus/aura/oe-test/authority_round.json   |  96 ++++
 ...authority_round_block_reward_contract.json | 103 ++++
 .../oe-test/authority_round_empty_steps.json  |  51 ++
 .../authority_round_randomness_contract.json  | 100 ++++
 consensus/aura/test/authority_round.json      |  96 ++++
 ...authority_round_block_reward_contract.json |  14 +
 .../test/authority_round_empty_steps.json     |  51 ++
 .../authority_round_randomness_contract.json  | 100 ++++
 consensus/aura/test/embed.go                  |   8 +
 consensus/aura/validators.go                  |  81 ++-
 consensus/clique/clique.go                    |   6 +-
 consensus/consensus.go                        |   5 +-
 consensus/ethash/consensus.go                 |  10 +-
 core/chain_makers.go                          |   4 +-
 core/genesis.go                               |  41 +-
 core/rawdb/accessors_account.go               |   7 +-
 core/types/block.go                           |  43 +-
 eth/ethconfig/config.go                       |  13 +-
 eth/stagedsync/stage_headers.go               |   1 -
 go.mod                                        |   4 +-
 go.sum                                        |   6 -
 turbo/adapter/chain_context.go                |   8 +-
 turbo/stages/genesis_test.go                  |   8 +-
 turbo/stages/mock_sentry.go                   |  16 +
 28 files changed, 1361 insertions(+), 220 deletions(-)
 create mode 100644 consensus/aura/aura_test.go
 create mode 100644 consensus/aura/oe-test/authority_round.json
 create mode 100644 consensus/aura/oe-test/authority_round_block_reward_contract.json
 create mode 100644 consensus/aura/oe-test/authority_round_empty_steps.json
 create mode 100644 consensus/aura/oe-test/authority_round_randomness_contract.json
 create mode 100644 consensus/aura/test/authority_round.json
 create mode 100644 consensus/aura/test/authority_round_block_reward_contract.json
 create mode 100644 consensus/aura/test/authority_round_empty_steps.json
 create mode 100644 consensus/aura/test/authority_round_randomness_contract.json
 create mode 100644 consensus/aura/test/embed.go

diff --git a/cmd/integration/commands/reset_state.go b/cmd/integration/commands/reset_state.go
index dcfe4053ec..ddde060cd0 100644
--- a/cmd/integration/commands/reset_state.go
+++ b/cmd/integration/commands/reset_state.go
@@ -29,6 +29,12 @@ var cmdResetState = &cobra.Command{
 			log.Error(err.Error())
 			return err
 		}
+		if err := db.Update(ctx, func(tx ethdb.RwTx) error {
+			return clearUnwindStack(tx, ctx)
+		}); err != nil {
+			log.Error(err.Error())
+			return err
+		}
 
 		return nil
 	},
diff --git a/consensus/aura/aura.go b/consensus/aura/aura.go
index 544cc8b085..854ab95c92 100644
--- a/consensus/aura/aura.go
+++ b/consensus/aura/aura.go
@@ -14,12 +14,11 @@
 // You should have received a copy of the GNU Lesser General Public License
 // along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
 
-// Package clique implements the proof-of-authority consensus engine.
 package aura
 
 import (
 	"bytes"
-	"context"
+	"encoding/json"
 	"fmt"
 	"math/big"
 	"sort"
@@ -34,7 +33,6 @@ import (
 	"github.com/ledgerwatch/erigon/consensus/aura/aurainterfaces"
 	"github.com/ledgerwatch/erigon/consensus/aura/contracts"
 	"github.com/ledgerwatch/erigon/consensus/clique"
-	"github.com/ledgerwatch/erigon/core/rawdb"
 	"github.com/ledgerwatch/erigon/core/state"
 	"github.com/ledgerwatch/erigon/core/types"
 	"github.com/ledgerwatch/erigon/crypto"
@@ -47,6 +45,13 @@ import (
 	"go.uber.org/atomic"
 )
 
+/*
+Not implemented features from OS:
+ - two_thirds_majority_transition - because no chains in OE where this is != MaxUint64 - means 1/2 majority used everywhere
+ - emptyStepsTransition - same
+
+*/
+
 type StepDurationInfo struct {
 	TransitionStep      uint64
 	TransitionTimestamp uint64
@@ -107,6 +112,137 @@ type EpochManager struct {
 	force                 bool
 }
 
+// zoomValidators - Zooms to the epoch after the header with the given hash. Returns true if succeeded, false otherwise.
+// It's analog of zoom_to_after function in OE, but doesn't require external locking
+//nolint
+func (e *EpochManager) zoom(chain consensus.ChainHeaderReader, validators ValidatorSet, h *types.Header) (RollingFinality, uint64, bool) {
+	var lastWasParent bool
+	if e.finalityChecker.lastPushed != nil {
+		lastWasParent = *e.finalityChecker.lastPushed == h.Hash()
+	}
+
+	// early exit for current target == chain head, but only if the epochs are
+	// the same.
+	if lastWasParent && !e.force {
+		return e.finalityChecker, e.epochTransitionNumber, true
+	}
+	e.force = false
+
+	// epoch_transition_for can be an expensive call, but in the absence of
+	// forks it will only need to be called for the block directly after
+	// epoch transition, in which case it will be O(1) and require a single
+	// DB lookup.
+	last_transition, ok := epochTransitionFor(chain, h.ParentHash)
+	if !ok {
+		return e.finalityChecker, e.epochTransitionNumber, false
+	}
+	_ = last_transition
+	/*
+	       // extract other epoch set if it's not the same as the last.
+	       if last_transition.block_hash != self.epoch_transition_hash {
+	           let (signal_number, set_proof, _) = destructure_proofs(&last_transition.proof)
+	               .expect("proof produced by this engine; therefore it is valid; qed");
+
+	           trace!(
+	               target: "engine",
+	               "extracting epoch validator set for epoch ({}, {}) signalled at #{}",
+	               last_transition.block_number, last_transition.block_hash, signal_number
+	           );
+
+	           let first = signal_number == 0;
+	           let (list, _) = validators
+	               .epoch_set(
+	                   first,
+	                   machine,
+	                   signal_number, // use signal number so multi-set first calculation is correct.
+	                   set_proof,
+	               )
+	               .expect("proof produced by this engine; therefore it is valid; qed");
+	           trace!(
+	               target: "engine",
+	               "Updating finality checker with new validator set extracted from epoch ({}, {}): {:?}",
+	               last_transition.block_number, last_transition.block_hash, &list
+	           );
+	           let epoch_set = list.into_inner();
+	           self.finality_checker =
+	               RollingFinality::blank(epoch_set);
+	       }
+
+	       self.epoch_transition_hash = last_transition.block_hash;
+	       self.epoch_transition_number = last_transition.block_number;
+
+	       true
+	   }
+	*/
+	return e.finalityChecker, e.epochTransitionNumber, true
+}
+
+/// Get the transition to the epoch the given parent hash is part of
+/// or transitions to.
+/// This will give the epoch that any children of this parent belong to.
+///
+/// The block corresponding the the parent hash must be stored already.
+//nolint
+func epochTransitionFor(chain consensus.ChainHeaderReader, parentHash common.Hash) (common.Hash, bool) {
+	// slow path: loop back block by block
+	for {
+		h := chain.GetHeaderByHash(parentHash)
+		if h == nil {
+			return parentHash, false
+		}
+
+		// look for transition in database.
+		transitionHash, ok := epochTransition(h.Number.Uint64(), h.Hash())
+		if ok {
+			return transitionHash, true
+		}
+
+		// canonical hash -> fast breakout:
+		// get the last epoch transition up to this block.
+		//
+		// if `block_hash` is canonical it will only return transitions up to
+		// the parent.
+		canonical := chain.GetHeaderByNumber(h.Number.Uint64())
+		if canonical == nil {
+			return parentHash, false
+		}
+		//nolint
+		if canonical.Hash() == parentHash {
+			/* TODO: whaaaat????
+			   return self
+			       .epoch_transitions()
+			       .map(|(_, t)| t)
+			       .take_while(|t| t.block_number <= details.number)
+			       .last();
+			*/
+		}
+
+		parentHash = h.Hash()
+	}
+}
+
+// epochTransition get a specific epoch transition by block number and provided block hash.
+//nolint
+func epochTransition(blockNum uint64, blockHash common.Hash) (common.Hash, bool) {
+	return blockHash, true
+	/*
+		pub fn epoch_transition(&self, block_num: u64, block_hash: H256) -> Option<EpochTransition> {
+		   trace!(target: "blockchain", "Loading epoch transition at block {}, {}",
+		    block_num, block_hash);
+
+		   self.db
+		       .key_value()
+		       .read(db::COL_EXTRA, &block_num)
+		       .and_then(|transitions: EpochTransitions| {
+		           transitions
+		               .candidates
+		               .into_iter()
+		               .find(|c| c.block_hash == block_hash)
+		       })
+		}
+	*/
+}
+
 //nolint
 type unAssembledHeader struct {
 	h common.Hash // H256
@@ -122,8 +258,6 @@ type RollingFinality struct {
 	signers    SimpleList
 	signCount  map[common.Address]uint
 	lastPushed *common.Hash // Option<H256>,
-	// First block for which a 2/3 quorum (instead of 1/2) is required.
-	twoThirdsMajorityTransition uint64 //BlockNumber
 }
 
 // AuRa
@@ -137,19 +271,16 @@ type AuRa struct {
 	OurSigningAddress common.Address // Same as Etherbase in Mining
 	cfg               AuthorityRoundParams
 	EmptyStepsSet     *EmptyStepSet
+	EpochManager      EpochManager // Mutex<EpochManager>,
 
 	//Validators                     ValidatorSet
 	//ValidateScoreTransition        uint64
 	//ValidateStepTransition         uint64
-	//EpochManager                   EpochManager // Mutex<EpochManager>,
 	//immediateTransitions           bool
 	//blockReward                    map[uint64]*uint256.Int
 	//blockRewardContractTransitions BlockRewardContractList
 	//maximumUncleCountTransition    uint64
 	//maximumUncleCount              uint
-	//emptyStepsTransition           uint64
-	//strictEmptyStepsTransition     uint64
-	//twoThirdsMajorityTransition    uint64 //  BlockNumber
 	//maximumEmptySteps              uint
 	////machine: EthereumMachine,
 	//// History of step hashes recently received from peers.
@@ -198,10 +329,16 @@ func (pb *GasLimitOverride) Add(hash common.Hash, b *uint256.Int) {
 	pb.cache.ContainsOrAdd(hash, b)
 }
 
-// NewAuRa creates a Clique proof-of-authority consensus engine with the initial
-// signers set to the ones provided by the user.
-func NewAuRa(cfg *params.ChainConfig, db ethdb.RwKV, ourSigningAddress common.Address, auraParams AuthorityRoundParams) (*AuRa, error) {
-	config := cfg.Aura
+func NewAuRa(config *params.AuRaConfig, db ethdb.RwKV, ourSigningAddress common.Address, engineParamsJson []byte) (*AuRa, error) {
+	spec := JsonSpec{}
+	err := json.Unmarshal(engineParamsJson, &spec)
+	if err != nil {
+		return nil, err
+	}
+	auraParams, err := FromJson(spec)
+	if err != nil {
+		return nil, err
+	}
 
 	if _, ok := auraParams.StepDurations[0]; !ok {
 		return nil, fmt.Errorf("authority Round step 0 duration is undefined")
@@ -251,29 +388,7 @@ func NewAuRa(cfg *params.ChainConfig, db ethdb.RwKV, ourSigningAddress common.Ad
 
 	/*
 		    let engine = Arc::new(AuthorityRound {
-		        transition_service: IoService::<()>::start("AuRa")?,
-		        step: Arc::new(PermissionedStep {
-		            inner: step,
-		            can_propose: AtomicBool::new(true),
-		        }),
-		        client: Arc::new(RwLock::new(None)),
-		        signer: RwLock::new(None),
-		        validators: our_params.validators,
-		        validate_score_transition: our_params.validate_score_transition,
-		        validate_step_transition: our_params.validate_step_transition,
-		        empty_steps: Default::default(),
-		        epoch_manager: Mutex::new(EpochManager::blank(
-		            our_params.two_thirds_majority_transition,
-		        )),
-		        immediate_transitions: our_params.immediate_transitions,
-		        block_reward: our_params.block_reward,
-		        block_reward_contract_transitions: our_params.block_reward_contract_transitions,
-		        maximum_uncle_count_transition: our_params.maximum_uncle_count_transition,
-		        maximum_uncle_count: our_params.maximum_uncle_count,
-		        empty_steps_transition: our_params.empty_steps_transition,
-		        maximum_empty_steps: our_params.maximum_empty_steps,
-		        two_thirds_majority_transition: our_params.two_thirds_majority_transition,
-		        machine: machine,
+		        epoch_manager: Mutex::new(EpochManager::blank()),
 		        received_step_hashes: RwLock::new(Default::default()),
 		        gas_limit_override_cache: Mutex::new(LruCache::new(GAS_LIMIT_OVERRIDE_CACHE_CAPACITY)),
 		    })
@@ -456,14 +571,12 @@ func (c *AuRa) Finalize(cc *params.ChainConfig, header *types.Header, state *sta
 	}
 }
 
-// FinalizeAndAssemble implements consensus.Engine, ensuring no uncles are set,
-// nor block rewards given, and returns the final block.
+// FinalizeAndAssemble implements consensus.Engine
 func (c *AuRa) FinalizeAndAssemble(chainConfig *params.ChainConfig, header *types.Header, state *state.IntraBlockState, txs []types.Transaction, uncles []*types.Header, receipts []*types.Receipt, syscall consensus.SystemCall) (*types.Block, error) {
-	// No block rewards in PoA, so the state remains as is and uncles are dropped
-	//header.UncleHash = types.CalcUncleHash(nil)
+	c.Finalize(chainConfig, header, state, txs, uncles, syscall)
 
 	// Assemble and return the final block for sealing
-	return types.NewBlock(header, txs, nil, receipts), nil
+	return types.NewBlock(header, txs, uncles, receipts), nil
 }
 
 // Authorize injects a private key into the consensus engine to mint new blocks
@@ -549,17 +662,204 @@ func (c *AuRa) Seal(chain consensus.ChainHeaderReader, block *types.Block, resul
 	//return nil
 }
 
-// CalcDifficulty is the difficulty adjustment algorithm. It returns the difficulty
-// that a new block should have:
-// * DIFF_NOTURN(2) if BLOCK_NUMBER % SIGNER_COUNT != SIGNER_INDEX
-// * DIFF_INTURN(1) if BLOCK_NUMBER % SIGNER_COUNT == SIGNER_INDEX
-func (c *AuRa) CalcDifficulty(chain consensus.ChainHeaderReader, _, _ uint64, _ *big.Int, parentNumber uint64, parentHash, _ common.Hash) *big.Int {
+func stepProposer(validators ValidatorSet, blockHash common.Hash, step uint64) (common.Address, error) {
+	c, err := validators.defaultCaller(blockHash)
+	if err != nil {
+		return common.Address{}, err
+	}
+	return validators.getWithCaller(blockHash, uint(step), c)
+}
+
+// GenerateSeal - Attempt to seal the block internally.
+//
+// This operation is synchronous and may (quite reasonably) not be available, in which case
+// `Seal::None` will be returned.
+func (c *AuRa) GenerateSeal(chain consensus.ChainHeaderReader, current, parent *types.Header) []rlp.RawValue {
+	// first check to avoid generating signature most of the time
+	// (but there's still a race to the `compare_exchange`)
+	if !c.step.canPropose.Load() {
+		log.Trace("[engine] Aborting seal generation. Can't propose.")
+		return nil
+	}
+	parentStep, err := headerStep(parent)
+	if err != nil {
+		panic(err)
+	}
+	step := c.step.inner.inner.Load()
+
+	// filter messages from old and future steps and different parents
+	expectedDiff := calculateScore(parentStep, step, 0)
+	if current.Difficulty.Cmp(expectedDiff.ToBig()) != 0 {
+		log.Debug(fmt.Sprintf("[engine] Aborting seal generation. The step or empty_steps have changed in the meantime. %d != %d", current.Difficulty, expectedDiff))
+		return nil
+	}
+
+	if parentStep > step {
+		log.Warn(fmt.Sprintf("[engine] Aborting seal generation for invalid step: %d > %d", parentStep, step))
+		return nil
+	}
+
+	validators, setNumber, err := c.epochSet(chain, current)
+	if err != nil {
+		log.Warn("[engine] Unable to generate seal", "err", err)
+		return nil
+	}
+
+	stepProposerAddr, err := stepProposer(validators, current.ParentHash, step)
+	if err != nil {
+		log.Warn("[engine] Unable to get stepProposer", "err", err)
+		return nil
+	}
+	if stepProposerAddr != current.Coinbase {
+		return nil
+	}
+
+	// this is guarded against by `can_propose` unless the block was signed
+	// on the same step (implies same key) and on a different node.
+	if parentStep == step {
+		log.Warn("Attempted to seal block on the same step as parent. Is this authority sealing with more than one node?")
+		return nil
+	}
+
+	_ = setNumber
+	/*
+		signature, err := c.sign(current.bareHash())
+			if err != nil {
+				log.Warn("[engine] generate_seal: FAIL: Accounts secret key unavailable.", "err", err)
+				return nil
+			}
+	*/
+
+	/*
+		  // only issue the seal if we were the first to reach the compare_exchange.
+		  if self
+			  .step
+			  .can_propose
+			  .compare_exchange(true, false, AtomicOrdering::SeqCst, AtomicOrdering::SeqCst)
+			  .is_ok()
+		  {
+			  // we can drop all accumulated empty step messages that are
+			  // older than the parent step since we're including them in
+			  // the seal
+			  self.clear_empty_steps(parent_step);
+
+			  // report any skipped primaries between the parent block and
+			  // the block we're sealing, unless we have empty steps enabled
+			  if header.number() < self.empty_steps_transition {
+				  self.report_skipped(header, step, parent_step, &*validators, set_number);
+			  }
+
+			  let mut fields =
+				  vec![encode(&step), encode(&(H520::from(signature).as_bytes()))];
+
+			  if let Some(empty_steps_rlp) = empty_steps_rlp {
+				  fields.push(empty_steps_rlp);
+			  }
+
+			  return Seal::Regular(fields);
+		  }
+	*/
 	return nil
-	//snap, err := c.Snapshot(chain, parentNumber, parentHash, nil)
-	//if err != nil {
-	//	return nil
-	//}
-	//return calcDifficulty(snap, c.signer)
+}
+
+// epochSet fetch correct validator set for epoch at header, taking into account
+// finality of previous transitions.
+func (c *AuRa) epochSet(chain consensus.ChainHeaderReader, h *types.Header) (ValidatorSet, uint64, error) {
+	if c.cfg.ImmediateTransitions {
+		return c.cfg.Validators, h.Number.Uint64(), nil
+	}
+
+	//TODO: hardcode for now
+	if h.Number.Uint64() <= 671 {
+		return &SimpleList{validators: []common.Address{
+			common.HexToAddress("0xe8ddc5c7a2d2f0d7a9798459c0104fdf5e987aca"),
+		}}, 0, nil
+	}
+	return &SimpleList{validators: []common.Address{
+		common.HexToAddress("0xe8ddc5c7a2d2f0d7a9798459c0104fdf5e987aca"),
+		common.HexToAddress("0x82e4e61e7f5139ff0a4157a5bc687ef42294c248"),
+	}}, 672, nil
+	/*
+		finalityChecker, epochTransitionNumber, ok := c.EpochManager.zoom(chain, c.cfg.Validators, h)
+		if !ok {
+			return nil, 0, fmt.Errorf("Unable to zoom to epoch.")
+		}
+		return finalityChecker.validators(), epochTransitionNumber
+	*/
+
+	/*
+			fn epoch_set<'a>(
+		        &'a self,
+		        header: &Header,
+		    ) -> Result<(CowLike<dyn ValidatorSet, SimpleList>, BlockNumber), Error> {
+		        Ok(if self.immediate_transitions {
+		            (CowLike::Borrowed(&*self.validators), header.number())
+		        } else {
+		            let mut epoch_manager = self.epoch_manager.lock();
+		            let client = self.upgrade_client_or("Unable to verify sig")?;
+
+		            if !epoch_manager.zoom_to_after(
+		                &*client,
+		                &self.machine,
+		                &*self.validators,
+		                *header.parent_hash(),
+		            ) {
+		                debug!(target: "engine", "Unable to zoom to epoch.");
+		                return Err(EngineError::RequiresClient.into());
+		            }
+
+		            (
+		                CowLike::Owned(epoch_manager.validators().clone()),
+		                epoch_manager.epoch_transition_number,
+		            )
+		        })
+		    }
+	*/
+}
+
+//nolint
+func headerStep(current *types.Header) (val uint64, err error) {
+	if len(current.Seal) < 1 {
+		panic("was either checked with verify_block_basic or is genesis; has 2 fields; qed (Make sure the spec file has a correct genesis seal)")
+	}
+	err = rlp.Decode(bytes.NewReader(current.Seal[0]), &val)
+	if err != nil {
+		return val, err
+	}
+	return val, err
+}
+
+func (c *AuRa) CalcDifficulty(chain consensus.ChainHeaderReader, time, parentTime uint64, parentDifficulty *big.Int, parentNumber uint64, parentHash, parentUncleHash common.Hash, parentSeal []rlp.RawValue) *big.Int {
+	var parentStep uint64
+	err := rlp.Decode(bytes.NewReader(parentSeal[0]), &parentStep)
+	if err != nil {
+		panic(err)
+	}
+	currentStep := c.step.inner.inner.Load()
+	currentEmptyStepsLen := 0
+	return calculateScore(parentStep, currentStep, uint64(currentEmptyStepsLen)).ToBig()
+
+	/* TODO: do I need gasLimit override logic here ?
+	if let Some(gas_limit) = self.gas_limit_override(header) {
+		trace!(target: "engine", "Setting gas limit to {} for block {}.", gas_limit, header.number());
+		let parent_gas_limit = *parent.gas_limit();
+		header.set_gas_limit(gas_limit);
+		if parent_gas_limit != gas_limit {
+			info!(target: "engine", "Block gas limit was changed from {} to {}.", parent_gas_limit, gas_limit);
+		}
+	}
+	*/
+}
+
+// calculateScore - analog of PoW difficulty:
+//    sqrt(U256::max_value()) + parent_step - current_step + current_empty_steps
+func calculateScore(parentStep, currentStep, currentEmptySteps uint64) *uint256.Int {
+	maxU128 := uint256.NewInt(0).SetAllOne()
+	maxU128 = maxU128.Rsh(maxU128, 128)
+	res := maxU128.Add(maxU128, uint256.NewInt(parentStep))
+	res = res.Sub(res, uint256.NewInt(currentStep))
+	res = res.Add(res, uint256.NewInt(currentEmptySteps))
+	return res
 }
 
 func (c *AuRa) SealHash(header *types.Header) common.Hash {
@@ -585,11 +885,12 @@ func (c *AuRa) APIs(chain consensus.ChainHeaderReader) []rpc.API {
 	}
 }
 
-func (c *AuRa) EmptySteps(fromStep, toStep uint64, parentHash common.Hash) []EmptyStep {
+//nolint
+func (c *AuRa) emptySteps(fromStep, toStep uint64, parentHash common.Hash) []EmptyStep {
 	from := EmptyStep{step: fromStep + 1, parentHash: parentHash}
 	to := EmptyStep{step: toStep}
 	res := []EmptyStep{}
-	if to.Less(&from) {
+	if to.LessOrEqual(&from) {
 		return res
 	}
 
@@ -610,49 +911,6 @@ func (c *AuRa) EmptySteps(fromStep, toStep uint64, parentHash common.Hash) []Emp
 // of the static blockReward plus a reward for each included uncle (if any). Individual
 // uncle rewards are also returned in an array.
 func AccumulateRewards(_ *params.ChainConfig, aura *AuRa, header *types.Header, _ []*types.Header, syscall consensus.SystemCall) (beneficiaries []common.Address, rewardKind []aurainterfaces.RewardKind, rewards []*uint256.Int, err error) {
-	if header.Number.Uint64() == aura.cfg.TwoThirdsMajorityTransition {
-		log.Info("Transitioning to 2/3 quorum", "block", aura.cfg.TwoThirdsMajorityTransition)
-	}
-	if header.Number.Uint64() >= aura.cfg.EmptyStepsTransition {
-		var emptySteps []EmptyStep
-		if len(header.Seal) == 0 {
-			// this is a new block, calculate rewards based on the empty steps messages we have accumulated
-			if err = aura.db.View(context.Background(), func(tx ethdb.Tx) error {
-				parent := rawdb.ReadHeader(tx, header.ParentHash, header.Number.Uint64())
-				if parent == nil {
-					return fmt.Errorf("parent not found: %d,%x\n", header.Number.Uint64(), header.ParentHash)
-				}
-				parentStep, err := headerStep(parent, aura.cfg.EmptyStepsTransition)
-				if err != nil {
-					return err
-				}
-				currentStep := aura.step.inner.inner.Load()
-				emptySteps = aura.EmptySteps(parentStep, currentStep, parent.Hash())
-
-				return nil
-			}); err != nil {
-				return
-			}
-		} else {
-			// we're verifying a block, extract empty steps from the seal
-			emptySteps, err = headerEmptySteps(header)
-			if err != nil {
-				return
-			}
-		}
-
-		for _, s := range emptySteps {
-			var author common.Address
-			author, err = s.author()
-			if err != nil {
-				return
-			}
-
-			beneficiaries = append(beneficiaries, author)
-			rewardKind = append(rewardKind, aurainterfaces.RewardEmptyStep)
-		}
-	}
-
 	beneficiaries = append(beneficiaries, header.Coinbase)
 	rewardKind = append(rewardKind, aurainterfaces.RewardAuthor)
 
@@ -728,11 +986,13 @@ func blockRewardAbi() abi.ABI {
 // the `parent_hash` in order to save space. The included signature is of the original empty step
 // message, which can be reconstructed by using the parent hash of the block in which this sealed
 // empty message is inc    luded.
+//nolint
 type SealedEmptyStep struct {
 	signature []byte // H520
 	step      uint64
 }
 
+/*
 // extracts the empty steps from the header seal. should only be called when there are 3 fields in the seal
 // (i.e. header.number() >= self.empty_steps_transition).
 func headerEmptySteps(header *types.Header) ([]EmptyStep, error) {
@@ -765,22 +1025,7 @@ func headerEmptyStepsRaw(header *types.Header) []byte {
 	}
 	return header.Seal[2]
 }
-
-func headerStep(header *types.Header, emptyStepsTransition uint64) (uint64, error) {
-	if len(header.Seal) == 0 {
-		panic(fmt.Errorf("was either checked with verify_block_basic or is genesis; has %v fields; qed (Make sure the spec file has a correct genesis seal)", headerExpectedSealFields(header, emptyStepsTransition)))
-	}
-	var val uint64
-	err := rlp.DecodeBytes(header.Seal[0], &val)
-	return val, err
-}
-
-func headerExpectedSealFields(header *types.Header, emptyStepsTransition uint64) uint {
-	if header.Number.Uint64() >= emptyStepsTransition {
-		return 3
-	}
-	return 2
-}
+*/
 
 // A message broadcast by authorities when it's their turn to seal a block but there are no
 // transactions. Other authorities accumulate these messages and later include them in the seal as
@@ -815,6 +1060,18 @@ func (s *EmptyStep) Less(other *EmptyStep) bool {
 	}
 	return false
 }
+func (s *EmptyStep) LessOrEqual(other *EmptyStep) bool {
+	if s.step <= other.step {
+		return true
+	}
+	if bytes.Compare(s.parentHash[:], other.parentHash[:]) <= 0 {
+		return true
+	}
+	if bytes.Compare(s.signature, other.signature) <= 0 {
+		return true
+	}
+	return false
+}
 
 // Returns `true` if the message has a valid signature by the expected proposer in the message's step.
 func (s *EmptyStep) verify(validators ValidatorSet) (bool, error) { //nolint
@@ -833,6 +1090,7 @@ func (s *EmptyStep) verify(validators ValidatorSet) (bool, error) { //nolint
 	return true, nil
 }
 
+//nolint
 func (s *EmptyStep) author() (common.Address, error) {
 	sRlp, err := EmptyStepRlp(s.step, s.parentHash)
 	if err != nil {
diff --git a/consensus/aura/aura_test.go b/consensus/aura/aura_test.go
new file mode 100644
index 0000000000..144381f8f0
--- /dev/null
+++ b/consensus/aura/aura_test.go
@@ -0,0 +1,189 @@
+package aura_test
+
+import (
+	"context"
+	"fmt"
+	"testing"
+
+	"github.com/ledgerwatch/erigon/common"
+	"github.com/ledgerwatch/erigon/consensus/aura"
+	"github.com/ledgerwatch/erigon/consensus/aura/test"
+	"github.com/ledgerwatch/erigon/core"
+	"github.com/ledgerwatch/erigon/core/rawdb"
+	"github.com/ledgerwatch/erigon/core/types/accounts"
+	"github.com/ledgerwatch/erigon/ethdb"
+	"github.com/ledgerwatch/erigon/ethdb/kv"
+	"github.com/ledgerwatch/erigon/turbo/stages"
+	"github.com/stretchr/testify/require"
+)
+
+/*
+ #[test]
+    fn block_reward_contract() {
+        let spec = Spec::new_test_round_block_reward_contract();
+        let tap = Arc::new(AccountProvider::transient_provider());
+
+        let addr1 = tap.insert_account(keccak("1").into(), &"1".into()).unwrap();
+
+        let engine = &*spec.engine;
+        let genesis_header = spec.genesis_header();
+        let db1 = spec
+            .ensure_db_good(get_temp_state_db(), &Default::default())
+            .unwrap();
+        let db2 = spec
+            .ensure_db_good(get_temp_state_db(), &Default::default())
+            .unwrap();
+
+        let last_hashes = Arc::new(vec![genesis_header.hash()]);
+
+        let client = generate_dummy_client_with_spec(Spec::new_test_round_block_reward_contract);
+        engine.register_client(Arc::downgrade(&client) as _);
+
+        // step 2
+        let b1 = OpenBlock::new(
+            engine,
+            Default::default(),
+            false,
+            db1,
+            &genesis_header,
+            last_hashes.clone(),
+            addr1,
+            (3141562.into(), 31415620.into()),
+            vec![],
+            false,
+            None,
+        )
+        .unwrap();
+        let b1 = b1.close_and_lock().unwrap();
+
+        // since the block is empty it isn't sealed and we generate empty steps
+        engine.set_signer(Some(Box::new((tap.clone(), addr1, "1".into()))));
+        assert_eq!(engine.generate_seal(&b1, &genesis_header), Seal::None);
+        engine.step();
+
+        // step 3
+        // the signer of the accumulated empty step message should be rewarded
+        let b2 = OpenBlock::new(
+            engine,
+            Default::default(),
+            false,
+            db2,
+            &genesis_header,
+            last_hashes.clone(),
+            addr1,
+            (3141562.into(), 31415620.into()),
+            vec![],
+            false,
+            None,
+        )
+        .unwrap();
+        let addr1_balance = b2.state.balance(&addr1).unwrap();
+
+        // after closing the block `addr1` should be reward twice, one for the included empty step
+        // message and another for block creation
+        let b2 = b2.close_and_lock().unwrap();
+
+        // the contract rewards (1000 + kind) for each benefactor/reward kind
+        assert_eq!(
+            b2.state.balance(&addr1).unwrap(),
+            addr1_balance + (1000 + 0) + (1000 + 2),
+        )
+    }
+*/
+func TestRewardContract(t *testing.T) {
+	t.Skip("not ready yet")
+	auraDB, require := kv.NewTestKV(t), require.New(t)
+	engine, err := aura.NewAuRa(nil, auraDB, common.Address{}, test.AuthorityRoundBlockRewardContract)
+	require.NoError(err)
+	m := stages.MockWithGenesisEngine(t, core.DefaultSokolGenesisBlock(), engine)
+	m.EnableLogs()
+
+	var accBefore *accounts.Account
+	err = auraDB.View(context.Background(), func(tx ethdb.Tx) (err error) { _, err = rawdb.ReadAccount(tx, m.Address, accBefore); return err })
+	require.NoError(err)
+
+	chain, err := core.GenerateChain(m.ChainConfig, m.Genesis, m.Engine, m.DB, 2, func(i int, gen *core.BlockGen) {
+		gen.SetCoinbase(m.Address)
+	}, false /* intermediateHashes */)
+	require.NoError(err)
+
+	err = m.InsertChain(chain)
+	require.NoError(err)
+
+	var accAfter *accounts.Account
+	err = auraDB.View(context.Background(), func(tx ethdb.Tx) (err error) { _, err = rawdb.ReadAccount(tx, m.Address, accAfter); return err })
+	require.NoError(err)
+
+	fmt.Printf("balance: %d\n", accAfter.Balance.Uint64())
+	/*
+
+	   	let spec = Spec::new_test_round_block_reward_contract();
+	              let tap = Arc::new(AccountProvider::transient_provider());
+
+	              let addr1 = tap.insert_account(keccak("1").into(), &"1".into()).unwrap();
+
+	              let engine = &*spec.engine;
+	              let genesis_header = spec.genesis_header();
+	              let db1 = spec
+	                  .ensure_db_good(get_temp_state_db(), &Default::default())
+	                  .unwrap();
+	              let db2 = spec
+	                  .ensure_db_good(get_temp_state_db(), &Default::default())
+	                  .unwrap();
+
+	              let last_hashes = Arc::new(vec![genesis_header.hash()]);
+
+	              let client = generate_dummy_client_with_spec(Spec::new_test_round_block_reward_contract);
+	              engine.register_client(Arc::downgrade(&client) as _);
+
+	              // step 2
+	              let b1 = OpenBlock::new(
+	                  engine,
+	                  Default::default(),
+	                  false,
+	                  db1,
+	                  &genesis_header,
+	                  last_hashes.clone(),
+	                  addr1,
+	                  (3141562.into(), 31415620.into()),
+	                  vec![],
+	                  false,
+	                  None,
+	              )
+	              .unwrap();
+	              let b1 = b1.close_and_lock().unwrap();
+
+	              // since the block is empty it isn't sealed and we generate empty steps
+	              engine.set_signer(Some(Box::new((tap.clone(), addr1, "1".into()))));
+	              assert_eq!(engine.generate_seal(&b1, &genesis_header), Seal::None);
+	              engine.step();
+
+	              // step 3
+	              // the signer of the accumulated empty step message should be rewarded
+	              let b2 = OpenBlock::new(
+	                  engine,
+	                  Default::default(),
+	                  false,
+	                  db2,
+	                  &genesis_header,
+	                  last_hashes.clone(),
+	                  addr1,
+	                  (3141562.into(), 31415620.into()),
+	                  vec![],
+	                  false,
+	                  None,
+	              )
+	              .unwrap();
+	              let addr1_balance = b2.state.balance(&addr1).unwrap();
+
+	              // after closing the block `addr1` should be reward twice, one for the included empty step
+	              // message and another for block creation
+	              let b2 = b2.close_and_lock().unwrap();
+
+	              // the contract rewards (1000 + kind) for each benefactor/reward kind
+	              assert_eq!(
+	                  b2.state.balance(&addr1).unwrap(),
+	                  addr1_balance + (1000 + 0) + (1000 + 2),
+	              )
+	*/
+}
diff --git a/consensus/aura/config.go b/consensus/aura/config.go
index f5b733f275..48aecb2cda 100644
--- a/consensus/aura/config.go
+++ b/consensus/aura/config.go
@@ -23,7 +23,6 @@ import (
 	"github.com/holiman/uint256"
 	"github.com/ledgerwatch/erigon/common"
 	"github.com/ledgerwatch/erigon/common/hexutil"
-	"github.com/ledgerwatch/erigon/common/math"
 	"github.com/ledgerwatch/erigon/common/u256"
 )
 
@@ -105,14 +104,8 @@ type JsonSpec struct {
 	MaximumUncleCountTransition *uint64 `json:"maximumUncleCountTransition"`
 	// Maximum number of accepted uncles.
 	MaximumUncleCount *uint `json:"maximumUncleCount"`
-	// Block at which empty step messages should start.
-	EmptyStepsTransition *uint64 `json:"emptyStepsTransition"`
-	// Maximum number of accepted empty steps.
-	MaximumEmptySteps *uint `json:"maximumEmptySteps"`
 	// Strict validation of empty steps transition block.
 	StrictEmptyStepsTransition *uint `json:"strictEmptyStepsTransition"`
-	// First block for which a 2/3 quorum (instead of 1/2) is required.
-	TwoThirdsMajorityTransition *uint64 `json:"twoThirdsMajorityTransition"`
 	// The random number contract's address, or a map of contract transitions.
 	RandomnessContractAddress map[uint64]common.Address `json:"randomnessContractAddress"`
 	// The addresses of contracts that determine the block gas limit starting from the block number
@@ -172,12 +165,6 @@ type AuthorityRoundParams struct {
 	MaximumUncleCountTransition uint64
 	// Number of accepted uncles.
 	MaximumUncleCount uint
-	// Empty step messages transition block.
-	EmptyStepsTransition uint64
-	// First block for which a 2/3 quorum (instead of 1/2) is required.
-	TwoThirdsMajorityTransition uint64
-	// Number of accepted empty steps.
-	MaximumEmptySteps uint
 	// Transition block to strict empty steps validation.
 	StrictEmptyStepsTransition uint64
 	// If set, enables random number contract integration. It maps the transition block to the contract address.
@@ -259,12 +246,6 @@ func FromJson(jsonParams JsonSpec) (AuthorityRoundParams, error) {
 	if jsonParams.MaximumUncleCountTransition != nil {
 		params.MaximumUncleCountTransition = *jsonParams.MaximumUncleCountTransition
 	}
-	if jsonParams.MaximumEmptySteps != nil {
-		params.MaximumEmptySteps = *jsonParams.MaximumEmptySteps
-	}
-	if jsonParams.EmptyStepsTransition != nil {
-		params.EmptyStepsTransition = *jsonParams.EmptyStepsTransition
-	}
 
 	params.BlockReward = map[uint64]*uint256.Int{}
 	if jsonParams.BlockReward == nil {
@@ -275,18 +256,5 @@ func FromJson(jsonParams JsonSpec) (AuthorityRoundParams, error) {
 		}
 	}
 
-	params.EmptyStepsTransition = math.MaxUint64
-	if jsonParams.EmptyStepsTransition != nil {
-		if *jsonParams.EmptyStepsTransition < 1 {
-			params.EmptyStepsTransition = 1
-		} else {
-			params.EmptyStepsTransition = *jsonParams.EmptyStepsTransition
-		}
-	}
-
-	params.TwoThirdsMajorityTransition = math.MaxUint64
-	if jsonParams.TwoThirdsMajorityTransition != nil {
-		params.TwoThirdsMajorityTransition = *jsonParams.TwoThirdsMajorityTransition
-	}
 	return params, nil
 }
diff --git a/consensus/aura/oe-test/authority_round.json b/consensus/aura/oe-test/authority_round.json
new file mode 100644
index 0000000000..292760444d
--- /dev/null
+++ b/consensus/aura/oe-test/authority_round.json
@@ -0,0 +1,96 @@
+{
+	"name": "TestAuthorityRound",
+	"engine": {
+		"authorityRound": {
+			"params": {
+				"stepDuration": 1,
+				"startStep": 2,
+				"validators": {
+					"list": [
+						"0x7d577a597b2742b498cb5cf0c26cdcd726d39e6e",
+						"0x82a978b3f5962a5b0957d9ee9eef472ee55b42f1"
+					]
+				},
+				"immediateTransitions": true
+			}
+		}
+	},
+	"params": {
+		"gasLimitBoundDivisor": "0x0400",
+		"accountStartNonce": "0x0",
+		"maximumExtraDataSize": "0x20",
+		"minGasLimit": "0x1388",
+		"networkID" : "0x69",
+		"eip140Transition": "0x0",
+		"eip211Transition": "0x0",
+		"eip214Transition": "0x0",
+		"eip658Transition": "0x0"
+	},
+	"genesis": {
+		"seal": {
+			"authorityRound": {
+				"step": "0x0",
+				"signature": "0x0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000"
+			}
+		},
+		"difficulty": "0x20000",
+		"author": "0x0000000000000000000000000000000000000000",
+		"timestamp": "0x00",
+		"parentHash": "0x0000000000000000000000000000000000000000000000000000000000000000",
+		"extraData": "0x",
+		"gasLimit": "0x222222"
+	},
+	"accounts": {
+		"0000000000000000000000000000000000000001": { "balance": "1", "nonce": "1048576", "builtin": { "name": "ecrecover", "pricing": { "linear": { "base": 3000, "word": 0 } } } },
+		"0000000000000000000000000000000000000002": { "balance": "1", "nonce": "1048576", "builtin": { "name": "sha256", "pricing": { "linear": { "base": 60, "word": 12 } } } },
+		"0000000000000000000000000000000000000003": { "balance": "1", "nonce": "1048576", "builtin": { "name": "ripemd160", "pricing": { "linear": { "base": 600, "word": 120 } } } },
+		"0000000000000000000000000000000000000004": { "balance": "1", "nonce": "1048576", "builtin": { "name": "identity", "pricing": { "linear": { "base": 15, "word": 3 } } } },
+		"0000000000000000000000000000000000000005": { "balance": "1", "builtin": { "name": "modexp", "activate_at": 0, "pricing": { "modexp": { "divisor": 20 } } } },
+		"0000000000000000000000000000000000000006": {
+			"balance": "1",
+			"builtin": {
+				"name": "alt_bn128_add",
+				"pricing": {
+					"0": {
+						"price": { "alt_bn128_const_operations": { "price": 500 }}
+					},
+					"0x7fffffffffffff": {
+						"info": "EIP 1108 transition",
+						"price": { "alt_bn128_const_operations": { "price": 150 }}
+					}
+				}
+			}
+		},
+		"0000000000000000000000000000000000000007": {
+			"balance": "1",
+			"builtin": {
+				"name": "alt_bn128_mul",
+				"pricing": {
+					"0": {
+						"price": { "alt_bn128_const_operations": { "price": 40000 }}
+					},
+					"0x7fffffffffffff": {
+						"info": "EIP 1108 transition",
+						"price": { "alt_bn128_const_operations": { "price": 6000 }}
+					}
+				}
+			}
+		},
+		"0000000000000000000000000000000000000008": {
+			"balance": "1",
+			"builtin": {
+				"name": "alt_bn128_pairing",
+				"pricing": {
+					"0": {
+						"price": { "alt_bn128_pairing": { "base": 100000, "pair": 80000 }}
+					},
+					"0x7fffffffffffff": {
+						"info": "EIP 1108 transition",
+						"price": { "alt_bn128_pairing": { "base": 45000, "pair": 34000 }}
+					}
+				}
+			}
+		},
+		"9cce34f7ab185c7aba1b7c8140d620b4bda941d6": { "balance": "1606938044258990275541962092341162602522202993782792835301376", "nonce": "1048576" }
+	}
+}
diff --git a/consensus/aura/oe-test/authority_round_block_reward_contract.json b/consensus/aura/oe-test/authority_round_block_reward_contract.json
new file mode 100644
index 0000000000..4adb9a8d43
--- /dev/null
+++ b/consensus/aura/oe-test/authority_round_block_reward_contract.json
@@ -0,0 +1,103 @@
+{
+	"name": "TestAuthorityRoundBlockRewardContract",
+	"engine": {
+		"authorityRound": {
+			"params": {
+				"stepDuration": 1,
+				"startStep": 2,
+				"validators": {
+					"list": [
+						"0x7d577a597b2742b498cb5cf0c26cdcd726d39e6e",
+						"0x82a978b3f5962a5b0957d9ee9eef472ee55b42f1"
+					]
+				},
+				"immediateTransitions": true,
+				"emptyStepsTransition": "1",
+				"maximumEmptySteps": "2",
+				"blockRewardContractAddress": "0x0000000000000000000000000000000000000042"
+			}
+		}
+	},
+	"params": {
+		"gasLimitBoundDivisor": "0x0400",
+		"accountStartNonce": "0x0",
+		"maximumExtraDataSize": "0x20",
+		"minGasLimit": "0x1388",
+		"networkID" : "0x69",
+		"eip140Transition": "0x0",
+		"eip211Transition": "0x0",
+		"eip214Transition": "0x0",
+		"eip658Transition": "0x0"
+	},
+	"genesis": {
+		"seal": {
+			"authorityRound": {
+				"step": "0x0",
+				"signature": "0x0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000"
+			}
+		},
+		"difficulty": "0x20000",
+		"author": "0x0000000000000000000000000000000000000000",
+		"timestamp": "0x00",
+		"parentHash": "0x0000000000000000000000000000000000000000000000000000000000000000",
+		"extraData": "0x",
+		"gasLimit": "0x222222"
+	},
+	"accounts": {
+		"0000000000000000000000000000000000000001": { "balance": "1", "nonce": "1048576", "builtin": { "name": "ecrecover", "pricing": { "linear": { "base": 3000, "word": 0 } } } },
+		"0000000000000000000000000000000000000002": { "balance": "1", "nonce": "1048576", "builtin": { "name": "sha256", "pricing": { "linear": { "base": 60, "word": 12 } } } },
+		"0000000000000000000000000000000000000003": { "balance": "1", "nonce": "1048576", "builtin": { "name": "ripemd160", "pricing": { "linear": { "base": 600, "word": 120 } } } },
+		"0000000000000000000000000000000000000004": { "balance": "1", "nonce": "1048576", "builtin": { "name": "identity", "pricing": { "linear": { "base": 15, "word": 3 } } } },
+		"0000000000000000000000000000000000000005": { "balance": "1", "builtin": { "name": "modexp", "activate_at": 0, "pricing": { "modexp": { "divisor": 20 } } } },
+		"0000000000000000000000000000000000000006": {
+			"balance": "1",
+			"builtin": {
+				"name": "alt_bn128_add",
+				"pricing": {
+					"0": {
+						"price": { "alt_bn128_const_operations": { "price": 500 }}
+					},
+					"0x7fffffffffffff": {
+						"info": "EIP 1108 transition",
+						"price": { "alt_bn128_const_operations": { "price": 150 }}
+					}
+				}
+			}
+		},
+		"0000000000000000000000000000000000000007": {
+			"balance": "1",
+			"builtin": {
+				"name": "alt_bn128_mul",
+				"pricing": {
+					"0": {
+						"price": { "alt_bn128_const_operations": { "price": 40000 }}
+					},
+					"0x7fffffffffffff": {
+						"info": "EIP 1108 transition",
+						"price": { "alt_bn128_const_operations": { "price": 6000 }}
+					}
+				}
+			}
+		},
+		"0000000000000000000000000000000000000008": {
+			"balance": "1",
+			"builtin": {
+				"name": "alt_bn128_pairing",
+				"pricing": {
+					"0": {
+						"price": { "alt_bn128_pairing": { "base": 100000, "pair": 80000 }}
+					},
+					"0x7fffffffffffff": {
+						"info": "EIP 1108 transition",
+						"price": { "alt_bn128_pairing": { "base": 45000, "pair": 34000 }}
+					}
+				}
+			}
+		},
+		"9cce34f7ab185c7aba1b7c8140d620b4bda941d6": { "balance": "1606938044258990275541962092341162602522202993782792835301376", "nonce": "1048576" },
+		"0000000000000000000000000000000000000042": {
+			"balance": "1",
+			"constructor": "6060604052341561000f57600080fd5b6102b88061001e6000396000f300606060405260043610610041576000357c0100000000000000000000000000000000000000000000000000000000900463ffffffff168063f91c289814610046575b600080fd5b341561005157600080fd5b610086600480803590602001908201803590602001919091929080359060200190820180359060200191909192905050610125565b604051808060200180602001838103835285818151815260200191508051906020019060200280838360005b838110156100cd5780820151818401526020810190506100b2565b50505050905001838103825284818151815260200191508051906020019060200280838360005b8381101561010f5780820151818401526020810190506100f4565b5050505090500194505050505060405180910390f35b61012d610264565b610135610278565b61013d610278565b600073fffffffffffffffffffffffffffffffffffffffe73ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff1614151561018d57600080fd5b85859050888890501415156101a157600080fd5b878790506040518059106101b25750595b90808252806020026020018201604052509150600090505b815181101561021d5785858281811015156101e157fe5b9050602002013561ffff166103e80161ffff16828281518110151561020257fe5b906020019060200201818152505080806001019150506101ca565b878783828280806020026020016040519081016040528093929190818152602001838360200280828437820191505050505050915090915093509350505094509492505050565b602060405190810160405280600081525090565b6020604051908101604052806000815250905600a165627a7a723058201da0f164e75517fb8baf51f030b904032cb748334938e7386f63025bfb23f3de0029"
+		}
+	}
+}
diff --git a/consensus/aura/oe-test/authority_round_empty_steps.json b/consensus/aura/oe-test/authority_round_empty_steps.json
new file mode 100644
index 0000000000..b884e8e15a
--- /dev/null
+++ b/consensus/aura/oe-test/authority_round_empty_steps.json
@@ -0,0 +1,51 @@
+{
+	"name": "TestAuthorityRoundEmptySteps",
+	"engine": {
+		"authorityRound": {
+			"params": {
+				"stepDuration": 1,
+				"startStep": 2,
+				"validators": {
+					"list": [
+						"0x7d577a597b2742b498cb5cf0c26cdcd726d39e6e",
+						"0x82a978b3f5962a5b0957d9ee9eef472ee55b42f1"
+					]
+				},
+				"blockReward": "10",
+				"immediateTransitions": true,
+				"emptyStepsTransition": "1",
+				"maximumEmptySteps": "2"
+			}
+		}
+	},
+	"params": {
+		"gasLimitBoundDivisor": "0x0400",
+		"accountStartNonce": "0x0",
+		"maximumExtraDataSize": "0x20",
+		"minGasLimit": "0x1388",
+		"networkID" : "0x69"
+	},
+	"genesis": {
+		"seal": {
+			"authorityRound": {
+				"step": "0x0",
+				"signature": "0x0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000"
+			}
+		},
+		"difficulty": "0x20000",
+		"author": "0x0000000000000000000000000000000000000000",
+		"timestamp": "0x00",
+		"parentHash": "0x0000000000000000000000000000000000000000000000000000000000000000",
+		"extraData": "0x",
+		"gasLimit": "0x222222"
+	},
+	"accounts": {
+		"0000000000000000000000000000000000000001": { "balance": "1", "nonce": "1048576", "builtin": { "name": "ecrecover", "pricing": { "linear": { "base": 3000, "word": 0 } } } },
+		"0000000000000000000000000000000000000002": { "balance": "1", "nonce": "1048576", "builtin": { "name": "sha256", "pricing": { "linear": { "base": 60, "word": 12 } } } },
+		"0000000000000000000000000000000000000003": { "balance": "1", "nonce": "1048576", "builtin": { "name": "ripemd160", "pricing": { "linear": { "base": 600, "word": 120 } } } },
+		"0000000000000000000000000000000000000004": { "balance": "1", "nonce": "1048576", "builtin": { "name": "identity", "pricing": { "linear": { "base": 15, "word": 3 } } } },
+		"9cce34f7ab185c7aba1b7c8140d620b4bda941d6": { "balance": "1606938044258990275541962092341162602522202993782792835301376", "nonce": "1048576" },
+		"7d577a597b2742b498cb5cf0c26cdcd726d39e6e": { "balance": "1000000000" },
+		"82a978b3f5962a5b0957d9ee9eef472ee55b42f1": { "balance": "1000000000" }
+	}
+}
diff --git a/consensus/aura/oe-test/authority_round_randomness_contract.json b/consensus/aura/oe-test/authority_round_randomness_contract.json
new file mode 100644
index 0000000000..2c820ee771
--- /dev/null
+++ b/consensus/aura/oe-test/authority_round_randomness_contract.json
@@ -0,0 +1,100 @@
+{
+	"name": "TestAuthorityRoundRandomnessContract",
+	"engine": {
+		"authorityRound": {
+			"params": {
+				"stepDuration": 1,
+				"startStep": 2,
+				"validators": {
+					"list": [
+						"0x7d577a597b2742b498cb5cf0c26cdcd726d39e6e"
+					]
+				},
+				"immediateTransitions": true,
+				"maximumEmptySteps": "2",
+				"randomnessContractAddress": {
+					"0": "0x0000000000000000000000000000000000000042"
+				}
+			}
+		}
+	},
+	"params": {
+		"gasLimitBoundDivisor": "0x0400",
+		"accountStartNonce": "0x0",
+		"maximumExtraDataSize": "0x20",
+		"minGasLimit": "0x1388",
+		"networkID" : "0x69",
+		"eip140Transition": "0x0",
+		"eip211Transition": "0x0",
+		"eip214Transition": "0x0",
+		"eip658Transition": "0x0"
+	},
+	"genesis": {
+		"seal": {
+			"authorityRound": {
+				"step": "0x0",
+				"signature": "0x0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000"
+			}
+		},
+		"difficulty": "0x20000",
+		"author": "0x0000000000000000000000000000000000000000",
+		"timestamp": "0x00",
+		"parentHash": "0x0000000000000000000000000000000000000000000000000000000000000000",
+		"extraData": "0x",
+		"gasLimit": "0x222222"
+	},
+	"accounts": {
+		"0x7d577a597b2742b498cb5cf0c26cdcd726d39e6e": { "balance": "100000000000" },
+		"0000000000000000000000000000000000000001": { "balance": "1", "nonce": "1048576", "builtin": { "name": "ecrecover", "pricing": { "linear": { "base": 3000, "word": 0 } } } },
+		"0000000000000000000000000000000000000002": { "balance": "1", "nonce": "1048576", "builtin": { "name": "sha256", "pricing": { "linear": { "base": 60, "word": 12 } } } },
+		"0000000000000000000000000000000000000003": { "balance": "1", "nonce": "1048576", "builtin": { "name": "ripemd160", "pricing": { "linear": { "base": 600, "word": 120 } } } },
+		"0000000000000000000000000000000000000004": { "balance": "1", "nonce": "1048576", "builtin": { "name": "identity", "pricing": { "linear": { "base": 15, "word": 3 } } } },
+		"0000000000000000000000000000000000000005": { "balance": "1", "builtin": { "name": "modexp", "activate_at": 0, "pricing": { "modexp": { "divisor": 20 } } } },
+		"0000000000000000000000000000000000000006": {
+			"balance": "1",
+			"builtin": {
+				"name": "alt_bn128_add",
+				"pricing": {
+					"0x0": {
+						"price": { "linear": { "base": 500, "word": 0 }}
+					},
+					"0x7fffffffffffff": {
+						"price": { "linear": { "base": 150, "word": 0 }}
+					}
+				}
+			}
+		},
+		"0000000000000000000000000000000000000007": {
+			"balance": "1",
+			"builtin": {
+				"name": "alt_bn128_mul",
+				"pricing": {
+					"0x0": {
+						"price": { "linear": { "base": 40000, "word": 0 }}
+					},
+					"0x7fffffffffffff": {
+						"price": { "linear": { "base": 6000, "word": 0 }}
+					}
+				}
+			}
+		},
+		"0000000000000000000000000000000000000008": {
+			"balance": "1",
+			"builtin": {
+				"name": "alt_bn128_pairing",
+				"pricing": {
+					"0x0": {
+						"price": { "alt_bn128_pairing": { "base": 100000, "pair": 80000 }}
+					},
+					"0x7fffffffffffff": {
+						"price": { "alt_bn128_pairing": { "base": 45000, "pair": 34000 }}
+					}
+				}
+			}
+		},
+		"0000000000000000000000000000000000000042": {
+			"balance": "1",
+			"constructor": "608060405234801561001057600080fd5b50610820806100206000396000f3fe608060405234801561001057600080fd5b50600436106100ec576000357c01000000000000000000000000000000000000000000000000000000009004806363f160e6116100a95780637a3e286b116100835780637a3e286b14610378578063baf11cab14610380578063c358ced0146103ac578063fe7d567d146103b4576100ec565b806363f160e614610285578063695e89f6146102c557806374ce906714610370576100ec565b806304fdb016146100f15780630b61ba8514610192578063209652551461020b5780632e8a8dd5146102255780633fa4f245146102515780635580e58b14610259575b600080fd5b61011d6004803603604081101561010757600080fd5b5080359060200135600160a060020a03166103d1565b6040805160208082528351818301528351919283929083019185019080838360005b8381101561015757818101518382015260200161013f565b50505050905090810190601f1680156101845780820380516001836020036101000a031916815260200191505b509250505060405180910390f35b610209600480360360408110156101a857600080fd5b813591908101906040810160208201356401000000008111156101ca57600080fd5b8201836020820111156101dc57600080fd5b803590602001918460018302840111640100000000831117156101fe57600080fd5b509092509050610475565b005b6102136104fa565b60408051918252519081900360200190f35b6102136004803603604081101561023b57600080fd5b5080359060200135600160a060020a0316610501565b61021361051b565b6102136004803603604081101561026f57600080fd5b5080359060200135600160a060020a0316610521565b6102b16004803603604081101561029b57600080fd5b5080359060200135600160a060020a031661053e565b604080519115158252519081900360200190f35b6102f1600480360360408110156102db57600080fd5b5080359060200135600160a060020a0316610568565b6040518083815260200180602001828103825283818151815260200191508051906020019080838360005b8381101561033457818101518382015260200161031c565b50505050905090810190601f1680156103615780820380516001836020036101000a031916815260200191505b50935050505060405180910390f35b6102b1610639565b610213610649565b6102b16004803603604081101561039657600080fd5b5080359060200135600160a060020a0316610654565b6102b161067c565b610209600480360360208110156103ca57600080fd5b5035610687565b600160208181526000938452604080852082529284529282902080548351600293821615610100026000190190911692909204601f8101859004850283018501909352828252909290919083018282801561046d5780601f106104425761010080835404028352916020019161046d565b820191906000526020600020905b81548152906001019060200180831161045057829003601f168201915b505050505081565b41331461048157600080fd5b61048d60014303610735565b61049657600080fd5b60006104a460014303610740565b90506104b08133610654565b156104ba57600080fd5b600081815260208181526040808320338085529083528184208890558484526001835281842090845290915290206104f3908484610753565b5050505050565b6003545b90565b600060208181529281526040808220909352908152205481565b60035481565b600260209081526000928352604080842090915290825290205481565b6000918252600260209081526040808420600160a060020a03939093168452919052902054151590565b600082815260208181526040808320600160a060020a03851680855290835281842054868552600180855283862092865291845282852080548451600294821615610100026000190190911693909304601f810186900486028401860190945283835260609491939092918391908301828280156106275780601f106105fc57610100808354040283529160200191610627565b820191906000526020600020905b81548152906001019060200180831161060a57829003601f168201915b50505050509050915091509250929050565b600061064443610735565b905090565b600061064443610740565b600091825260208281526040808420600160a060020a03939093168452919052902054151590565b600061064443610747565b41331461069357600080fd5b61069f60014303610747565b6106a857600080fd5b60006106b660014303610740565b90506106c2813361053e565b156106cc57600080fd5b60408051602080820185905282518083038201815291830183528151918101919091206000848152808352838120338252909252919020541461070e57600080fd5b60009081526002602090815260408083203384529091529020819055600380549091189055565b600360069091061090565b6006900490565b60036006909106101590565b828054600181600116156101000203166002900490600052602060002090601f016020900481019282601f106107945782800160ff198235161785556107c1565b828001600101855582156107c1579182015b828111156107c15782358255916020019190600101906107a6565b506107cd9291506107d1565b5090565b6104fe91905b808211156107cd57600081556001016107d756fea265627a7a7230582008bb7311af9026bd70ddb998741333d414a366275b9b433a2943bbd6bedc27ae64736f6c634300050a0032"
+		}
+	}
+}
\ No newline at end of file
diff --git a/consensus/aura/test/authority_round.json b/consensus/aura/test/authority_round.json
new file mode 100644
index 0000000000..292760444d
--- /dev/null
+++ b/consensus/aura/test/authority_round.json
@@ -0,0 +1,96 @@
+{
+	"name": "TestAuthorityRound",
+	"engine": {
+		"authorityRound": {
+			"params": {
+				"stepDuration": 1,
+				"startStep": 2,
+				"validators": {
+					"list": [
+						"0x7d577a597b2742b498cb5cf0c26cdcd726d39e6e",
+						"0x82a978b3f5962a5b0957d9ee9eef472ee55b42f1"
+					]
+				},
+				"immediateTransitions": true
+			}
+		}
+	},
+	"params": {
+		"gasLimitBoundDivisor": "0x0400",
+		"accountStartNonce": "0x0",
+		"maximumExtraDataSize": "0x20",
+		"minGasLimit": "0x1388",
+		"networkID" : "0x69",
+		"eip140Transition": "0x0",
+		"eip211Transition": "0x0",
+		"eip214Transition": "0x0",
+		"eip658Transition": "0x0"
+	},
+	"genesis": {
+		"seal": {
+			"authorityRound": {
+				"step": "0x0",
+				"signature": "0x0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000"
+			}
+		},
+		"difficulty": "0x20000",
+		"author": "0x0000000000000000000000000000000000000000",
+		"timestamp": "0x00",
+		"parentHash": "0x0000000000000000000000000000000000000000000000000000000000000000",
+		"extraData": "0x",
+		"gasLimit": "0x222222"
+	},
+	"accounts": {
+		"0000000000000000000000000000000000000001": { "balance": "1", "nonce": "1048576", "builtin": { "name": "ecrecover", "pricing": { "linear": { "base": 3000, "word": 0 } } } },
+		"0000000000000000000000000000000000000002": { "balance": "1", "nonce": "1048576", "builtin": { "name": "sha256", "pricing": { "linear": { "base": 60, "word": 12 } } } },
+		"0000000000000000000000000000000000000003": { "balance": "1", "nonce": "1048576", "builtin": { "name": "ripemd160", "pricing": { "linear": { "base": 600, "word": 120 } } } },
+		"0000000000000000000000000000000000000004": { "balance": "1", "nonce": "1048576", "builtin": { "name": "identity", "pricing": { "linear": { "base": 15, "word": 3 } } } },
+		"0000000000000000000000000000000000000005": { "balance": "1", "builtin": { "name": "modexp", "activate_at": 0, "pricing": { "modexp": { "divisor": 20 } } } },
+		"0000000000000000000000000000000000000006": {
+			"balance": "1",
+			"builtin": {
+				"name": "alt_bn128_add",
+				"pricing": {
+					"0": {
+						"price": { "alt_bn128_const_operations": { "price": 500 }}
+					},
+					"0x7fffffffffffff": {
+						"info": "EIP 1108 transition",
+						"price": { "alt_bn128_const_operations": { "price": 150 }}
+					}
+				}
+			}
+		},
+		"0000000000000000000000000000000000000007": {
+			"balance": "1",
+			"builtin": {
+				"name": "alt_bn128_mul",
+				"pricing": {
+					"0": {
+						"price": { "alt_bn128_const_operations": { "price": 40000 }}
+					},
+					"0x7fffffffffffff": {
+						"info": "EIP 1108 transition",
+						"price": { "alt_bn128_const_operations": { "price": 6000 }}
+					}
+				}
+			}
+		},
+		"0000000000000000000000000000000000000008": {
+			"balance": "1",
+			"builtin": {
+				"name": "alt_bn128_pairing",
+				"pricing": {
+					"0": {
+						"price": { "alt_bn128_pairing": { "base": 100000, "pair": 80000 }}
+					},
+					"0x7fffffffffffff": {
+						"info": "EIP 1108 transition",
+						"price": { "alt_bn128_pairing": { "base": 45000, "pair": 34000 }}
+					}
+				}
+			}
+		},
+		"9cce34f7ab185c7aba1b7c8140d620b4bda941d6": { "balance": "1606938044258990275541962092341162602522202993782792835301376", "nonce": "1048576" }
+	}
+}
diff --git a/consensus/aura/test/authority_round_block_reward_contract.json b/consensus/aura/test/authority_round_block_reward_contract.json
new file mode 100644
index 0000000000..d3b62b76d5
--- /dev/null
+++ b/consensus/aura/test/authority_round_block_reward_contract.json
@@ -0,0 +1,14 @@
+{
+  "stepDuration": 1,
+  "startStep": 2,
+  "validators": {
+    "list": [
+      "0x7d577a597b2742b498cb5cf0c26cdcd726d39e6e",
+      "0x82a978b3f5962a5b0957d9ee9eef472ee55b42f1"
+    ]
+  },
+  "immediateTransitions": true,
+  "emptyStepsTransition": 1,
+  "maximumEmptySteps": 2,
+  "blockRewardContractAddress": "0x0000000000000000000000000000000000000042"
+}
\ No newline at end of file
diff --git a/consensus/aura/test/authority_round_empty_steps.json b/consensus/aura/test/authority_round_empty_steps.json
new file mode 100644
index 0000000000..b884e8e15a
--- /dev/null
+++ b/consensus/aura/test/authority_round_empty_steps.json
@@ -0,0 +1,51 @@
+{
+	"name": "TestAuthorityRoundEmptySteps",
+	"engine": {
+		"authorityRound": {
+			"params": {
+				"stepDuration": 1,
+				"startStep": 2,
+				"validators": {
+					"list": [
+						"0x7d577a597b2742b498cb5cf0c26cdcd726d39e6e",
+						"0x82a978b3f5962a5b0957d9ee9eef472ee55b42f1"
+					]
+				},
+				"blockReward": "10",
+				"immediateTransitions": true,
+				"emptyStepsTransition": "1",
+				"maximumEmptySteps": "2"
+			}
+		}
+	},
+	"params": {
+		"gasLimitBoundDivisor": "0x0400",
+		"accountStartNonce": "0x0",
+		"maximumExtraDataSize": "0x20",
+		"minGasLimit": "0x1388",
+		"networkID" : "0x69"
+	},
+	"genesis": {
+		"seal": {
+			"authorityRound": {
+				"step": "0x0",
+				"signature": "0x0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000"
+			}
+		},
+		"difficulty": "0x20000",
+		"author": "0x0000000000000000000000000000000000000000",
+		"timestamp": "0x00",
+		"parentHash": "0x0000000000000000000000000000000000000000000000000000000000000000",
+		"extraData": "0x",
+		"gasLimit": "0x222222"
+	},
+	"accounts": {
+		"0000000000000000000000000000000000000001": { "balance": "1", "nonce": "1048576", "builtin": { "name": "ecrecover", "pricing": { "linear": { "base": 3000, "word": 0 } } } },
+		"0000000000000000000000000000000000000002": { "balance": "1", "nonce": "1048576", "builtin": { "name": "sha256", "pricing": { "linear": { "base": 60, "word": 12 } } } },
+		"0000000000000000000000000000000000000003": { "balance": "1", "nonce": "1048576", "builtin": { "name": "ripemd160", "pricing": { "linear": { "base": 600, "word": 120 } } } },
+		"0000000000000000000000000000000000000004": { "balance": "1", "nonce": "1048576", "builtin": { "name": "identity", "pricing": { "linear": { "base": 15, "word": 3 } } } },
+		"9cce34f7ab185c7aba1b7c8140d620b4bda941d6": { "balance": "1606938044258990275541962092341162602522202993782792835301376", "nonce": "1048576" },
+		"7d577a597b2742b498cb5cf0c26cdcd726d39e6e": { "balance": "1000000000" },
+		"82a978b3f5962a5b0957d9ee9eef472ee55b42f1": { "balance": "1000000000" }
+	}
+}
diff --git a/consensus/aura/test/authority_round_randomness_contract.json b/consensus/aura/test/authority_round_randomness_contract.json
new file mode 100644
index 0000000000..2c820ee771
--- /dev/null
+++ b/consensus/aura/test/authority_round_randomness_contract.json
@@ -0,0 +1,100 @@
+{
+	"name": "TestAuthorityRoundRandomnessContract",
+	"engine": {
+		"authorityRound": {
+			"params": {
+				"stepDuration": 1,
+				"startStep": 2,
+				"validators": {
+					"list": [
+						"0x7d577a597b2742b498cb5cf0c26cdcd726d39e6e"
+					]
+				},
+				"immediateTransitions": true,
+				"maximumEmptySteps": "2",
+				"randomnessContractAddress": {
+					"0": "0x0000000000000000000000000000000000000042"
+				}
+			}
+		}
+	},
+	"params": {
+		"gasLimitBoundDivisor": "0x0400",
+		"accountStartNonce": "0x0",
+		"maximumExtraDataSize": "0x20",
+		"minGasLimit": "0x1388",
+		"networkID" : "0x69",
+		"eip140Transition": "0x0",
+		"eip211Transition": "0x0",
+		"eip214Transition": "0x0",
+		"eip658Transition": "0x0"
+	},
+	"genesis": {
+		"seal": {
+			"authorityRound": {
+				"step": "0x0",
+				"signature": "0x0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000"
+			}
+		},
+		"difficulty": "0x20000",
+		"author": "0x0000000000000000000000000000000000000000",
+		"timestamp": "0x00",
+		"parentHash": "0x0000000000000000000000000000000000000000000000000000000000000000",
+		"extraData": "0x",
+		"gasLimit": "0x222222"
+	},
+	"accounts": {
+		"0x7d577a597b2742b498cb5cf0c26cdcd726d39e6e": { "balance": "100000000000" },
+		"0000000000000000000000000000000000000001": { "balance": "1", "nonce": "1048576", "builtin": { "name": "ecrecover", "pricing": { "linear": { "base": 3000, "word": 0 } } } },
+		"0000000000000000000000000000000000000002": { "balance": "1", "nonce": "1048576", "builtin": { "name": "sha256", "pricing": { "linear": { "base": 60, "word": 12 } } } },
+		"0000000000000000000000000000000000000003": { "balance": "1", "nonce": "1048576", "builtin": { "name": "ripemd160", "pricing": { "linear": { "base": 600, "word": 120 } } } },
+		"0000000000000000000000000000000000000004": { "balance": "1", "nonce": "1048576", "builtin": { "name": "identity", "pricing": { "linear": { "base": 15, "word": 3 } } } },
+		"0000000000000000000000000000000000000005": { "balance": "1", "builtin": { "name": "modexp", "activate_at": 0, "pricing": { "modexp": { "divisor": 20 } } } },
+		"0000000000000000000000000000000000000006": {
+			"balance": "1",
+			"builtin": {
+				"name": "alt_bn128_add",
+				"pricing": {
+					"0x0": {
+						"price": { "linear": { "base": 500, "word": 0 }}
+					},
+					"0x7fffffffffffff": {
+						"price": { "linear": { "base": 150, "word": 0 }}
+					}
+				}
+			}
+		},
+		"0000000000000000000000000000000000000007": {
+			"balance": "1",
+			"builtin": {
+				"name": "alt_bn128_mul",
+				"pricing": {
+					"0x0": {
+						"price": { "linear": { "base": 40000, "word": 0 }}
+					},
+					"0x7fffffffffffff": {
+						"price": { "linear": { "base": 6000, "word": 0 }}
+					}
+				}
+			}
+		},
+		"0000000000000000000000000000000000000008": {
+			"balance": "1",
+			"builtin": {
+				"name": "alt_bn128_pairing",
+				"pricing": {
+					"0x0": {
+						"price": { "alt_bn128_pairing": { "base": 100000, "pair": 80000 }}
+					},
+					"0x7fffffffffffff": {
+						"price": { "alt_bn128_pairing": { "base": 45000, "pair": 34000 }}
+					}
+				}
+			}
+		},
+		"0000000000000000000000000000000000000042": {
+			"balance": "1",
+			"constructor": "608060405234801561001057600080fd5b50610820806100206000396000f3fe608060405234801561001057600080fd5b50600436106100ec576000357c01000000000000000000000000000000000000000000000000000000009004806363f160e6116100a95780637a3e286b116100835780637a3e286b14610378578063baf11cab14610380578063c358ced0146103ac578063fe7d567d146103b4576100ec565b806363f160e614610285578063695e89f6146102c557806374ce906714610370576100ec565b806304fdb016146100f15780630b61ba8514610192578063209652551461020b5780632e8a8dd5146102255780633fa4f245146102515780635580e58b14610259575b600080fd5b61011d6004803603604081101561010757600080fd5b5080359060200135600160a060020a03166103d1565b6040805160208082528351818301528351919283929083019185019080838360005b8381101561015757818101518382015260200161013f565b50505050905090810190601f1680156101845780820380516001836020036101000a031916815260200191505b509250505060405180910390f35b610209600480360360408110156101a857600080fd5b813591908101906040810160208201356401000000008111156101ca57600080fd5b8201836020820111156101dc57600080fd5b803590602001918460018302840111640100000000831117156101fe57600080fd5b509092509050610475565b005b6102136104fa565b60408051918252519081900360200190f35b6102136004803603604081101561023b57600080fd5b5080359060200135600160a060020a0316610501565b61021361051b565b6102136004803603604081101561026f57600080fd5b5080359060200135600160a060020a0316610521565b6102b16004803603604081101561029b57600080fd5b5080359060200135600160a060020a031661053e565b604080519115158252519081900360200190f35b6102f1600480360360408110156102db57600080fd5b5080359060200135600160a060020a0316610568565b6040518083815260200180602001828103825283818151815260200191508051906020019080838360005b8381101561033457818101518382015260200161031c565b50505050905090810190601f1680156103615780820380516001836020036101000a031916815260200191505b50935050505060405180910390f35b6102b1610639565b610213610649565b6102b16004803603604081101561039657600080fd5b5080359060200135600160a060020a0316610654565b6102b161067c565b610209600480360360208110156103ca57600080fd5b5035610687565b600160208181526000938452604080852082529284529282902080548351600293821615610100026000190190911692909204601f8101859004850283018501909352828252909290919083018282801561046d5780601f106104425761010080835404028352916020019161046d565b820191906000526020600020905b81548152906001019060200180831161045057829003601f168201915b505050505081565b41331461048157600080fd5b61048d60014303610735565b61049657600080fd5b60006104a460014303610740565b90506104b08133610654565b156104ba57600080fd5b600081815260208181526040808320338085529083528184208890558484526001835281842090845290915290206104f3908484610753565b5050505050565b6003545b90565b600060208181529281526040808220909352908152205481565b60035481565b600260209081526000928352604080842090915290825290205481565b6000918252600260209081526040808420600160a060020a03939093168452919052902054151590565b600082815260208181526040808320600160a060020a03851680855290835281842054868552600180855283862092865291845282852080548451600294821615610100026000190190911693909304601f810186900486028401860190945283835260609491939092918391908301828280156106275780601f106105fc57610100808354040283529160200191610627565b820191906000526020600020905b81548152906001019060200180831161060a57829003601f168201915b50505050509050915091509250929050565b600061064443610735565b905090565b600061064443610740565b600091825260208281526040808420600160a060020a03939093168452919052902054151590565b600061064443610747565b41331461069357600080fd5b61069f60014303610747565b6106a857600080fd5b60006106b660014303610740565b90506106c2813361053e565b156106cc57600080fd5b60408051602080820185905282518083038201815291830183528151918101919091206000848152808352838120338252909252919020541461070e57600080fd5b60009081526002602090815260408083203384529091529020819055600380549091189055565b600360069091061090565b6006900490565b60036006909106101590565b828054600181600116156101000203166002900490600052602060002090601f016020900481019282601f106107945782800160ff198235161785556107c1565b828001600101855582156107c1579182015b828111156107c15782358255916020019190600101906107a6565b506107cd9291506107d1565b5090565b6104fe91905b808211156107cd57600081556001016107d756fea265627a7a7230582008bb7311af9026bd70ddb998741333d414a366275b9b433a2943bbd6bedc27ae64736f6c634300050a0032"
+		}
+	}
+}
\ No newline at end of file
diff --git a/consensus/aura/test/embed.go b/consensus/aura/test/embed.go
new file mode 100644
index 0000000000..8d9a4bc5e8
--- /dev/null
+++ b/consensus/aura/test/embed.go
@@ -0,0 +1,8 @@
+package test
+
+import (
+	_ "embed"
+)
+
+//go:embed authority_round_block_reward_contract.json
+var AuthorityRoundBlockRewardContract []byte
diff --git a/consensus/aura/validators.go b/consensus/aura/validators.go
index 4a1220f514..f95b54e0e7 100644
--- a/consensus/aura/validators.go
+++ b/consensus/aura/validators.go
@@ -61,6 +61,16 @@ type ValidatorSet interface {
 
 	// Draws an validator nonce modulo number of validators.
 	getWithCaller(parentHash common.Hash, nonce uint, caller Call) (common.Address, error)
+
+	// Recover the validator set from the given proof, the block number, and
+	// whether this header is first in its set.
+	//
+	// May fail if the given header doesn't kick off an epoch or
+	// the proof is invalid.
+	//
+	// Returns the set, along with a flag indicating whether finality of a specific
+	// hash should be proven.
+	epochSet(first bool, num uint64, proof []byte) (SimpleList, *common.Hash, error)
 	/*
 	 // Returns the current number of validators.
 	    fn count(&self, parent: &H256) -> usize {
@@ -236,6 +246,14 @@ func (s *Multi) onCloseBlock(header *types.Header, address common.Address) error
 	return set.onCloseBlock(header, address)
 }
 
+// TODO: do we need add `proof` argument?
+//nolint
+func (s *Multi) epochSet(first bool, num uint64, proof []byte) (SimpleList, *common.Hash, error) {
+	setBlock, set := s.correctSetByNumber(num)
+	first = setBlock == num
+	return set.epochSet(first, num, proof)
+}
+
 //func (s *Multi) onEpochBegin(first bool, header *types.Header, call SysCall) error {
 //	first, set := s.get(header.Number.Uint64())
 //	return set.onEpochBegin(first,header, address)
@@ -245,6 +263,9 @@ type SimpleList struct {
 	validators []common.Address
 }
 
+func (s *SimpleList) epochSet(first bool, num uint64, proof []byte) (SimpleList, *common.Hash, error) {
+	return *s, nil, nil
+}
 func (s *SimpleList) onCloseBlock(_header *types.Header, _address common.Address) error { return nil }
 func (s *SimpleList) defaultCaller(blockHash common.Hash) (Call, error) {
 	return nil, fmt.Errorf("simple list doesn't require calls")
@@ -362,6 +383,60 @@ func NewValidatorSafeContract(contractAddress common.Address, posdaoTransition *
 // Returns a list of contract calls to be pushed onto the new block.
 //func generateEngineTransactions(_first bool, _header *types.Header, _call SystemCall) -> Result<Vec<(Address, Bytes)>, EthcoreError>
 
+func (s *ValidatorSafeContract) epochSet(first bool, num uint64, proof []byte) (SimpleList, *common.Hash, error) {
+	return SimpleList{}, nil, fmt.Errorf("ValidatorSafeContract.epochSet not implemented")
+	/*
+		    fn epoch_set(
+		        &self,
+		        first: bool,
+		        machine: &EthereumMachine,
+		        _number: ::types::BlockNumber,
+		        proof: &[u8],
+		    ) -> Result<(SimpleList, Option<H256>), ::error::Error> {
+		        let rlp = Rlp::new(proof);
+
+		        if first {
+		            trace!(target: "engine", "Recovering initial epoch set");
+
+		            let (old_header, state_items) = decode_first_proof(&rlp)?;
+		            let number = old_header.number();
+		            let old_hash = old_header.hash();
+		            let addresses =
+		                check_first_proof(machine, self.contract_address, old_header, &state_items)
+		                    .map_err(::engines::EngineError::InsufficientProof)?;
+
+		            trace!(target: "engine", "extracted epoch set at #{}: {} addresses",
+						number, addresses.len());
+
+		            Ok((SimpleList::new(addresses), Some(old_hash)))
+		        } else {
+		            let (old_header, receipts) = decode_proof(&rlp)?;
+
+		            // ensure receipts match header.
+		            // TODO: optimize? these were just decoded.
+		            let found_root = ::triehash::ordered_trie_root(receipts.iter().map(|r| r.encode()));
+		            if found_root != *old_header.receipts_root() {
+		                return Err(::error::BlockError::InvalidReceiptsRoot(Mismatch {
+		                    expected: *old_header.receipts_root(),
+		                    found: found_root,
+		                })
+		                .into());
+		            }
+
+		            let bloom = self.expected_bloom(&old_header);
+
+		            match self.extract_from_event(bloom, &old_header, &receipts) {
+		                Some(list) => Ok((list, Some(old_header.hash()))),
+		                None => Err(::engines::EngineError::InsufficientProof(
+		                    "No log event in proof.".into(),
+		                )
+		                .into()),
+		            }
+		        }
+		    }
+	*/
+}
+
 func (s *ValidatorSafeContract) defaultCaller(blockHash common.Hash) (Call, error) {
 	return func(addr common.Address, data []byte) (CallResults, error) {
 		return s.client.CallAtBlockHash(blockHash, addr, data)
@@ -446,13 +521,17 @@ func (s *ValidatorSafeContract) onCloseBlock(header *types.Header, ourAddress co
 	return nil
 }
 
-// A validator contract with reporting.
+// ValidatorContract a validator contract with reporting.
 type ValidatorContract struct {
 	contractAddress  common.Address
 	validators       ValidatorSafeContract
 	posdaoTransition *uint64
 }
 
+func (s *ValidatorContract) epochSet(first bool, num uint64, proof []byte) (SimpleList, *common.Hash, error) {
+	return s.validators.epochSet(first, num, proof)
+}
+
 func (s *ValidatorContract) defaultCaller(blockHash common.Hash) (Call, error) {
 	return s.validators.defaultCaller(blockHash)
 }
diff --git a/consensus/clique/clique.go b/consensus/clique/clique.go
index 87b55fbbc8..ef3aae2918 100644
--- a/consensus/clique/clique.go
+++ b/consensus/clique/clique.go
@@ -464,11 +464,15 @@ func (c *Clique) Seal(chain consensus.ChainHeaderReader, block *types.Block, res
 	return nil
 }
 
+func (c *Clique) GenerateSeal(chain consensus.ChainHeaderReader, currnt, parent *types.Header) []rlp.RawValue {
+	return nil
+}
+
 // CalcDifficulty is the difficulty adjustment algorithm. It returns the difficulty
 // that a new block should have:
 // * DIFF_NOTURN(2) if BLOCK_NUMBER % SIGNER_COUNT != SIGNER_INDEX
 // * DIFF_INTURN(1) if BLOCK_NUMBER % SIGNER_COUNT == SIGNER_INDEX
-func (c *Clique) CalcDifficulty(chain consensus.ChainHeaderReader, _, _ uint64, _ *big.Int, parentNumber uint64, parentHash, _ common.Hash) *big.Int {
+func (c *Clique) CalcDifficulty(chain consensus.ChainHeaderReader, _, _ uint64, _ *big.Int, parentNumber uint64, parentHash, _ common.Hash, _ []rlp.RawValue) *big.Int {
 
 	snap, err := c.Snapshot(chain, parentNumber, parentHash, nil)
 	if err != nil {
diff --git a/consensus/consensus.go b/consensus/consensus.go
index 3e252065fd..74161d8ee5 100644
--- a/consensus/consensus.go
+++ b/consensus/consensus.go
@@ -24,6 +24,7 @@ import (
 	"github.com/ledgerwatch/erigon/core/state"
 	"github.com/ledgerwatch/erigon/core/types"
 	"github.com/ledgerwatch/erigon/params"
+	"github.com/ledgerwatch/erigon/rlp"
 	"github.com/ledgerwatch/erigon/rpc"
 )
 
@@ -113,7 +114,9 @@ type Engine interface {
 
 	// CalcDifficulty is the difficulty adjustment algorithm. It returns the difficulty
 	// that a new block should have.
-	CalcDifficulty(chain ChainHeaderReader, time, parentTime uint64, parentDifficulty *big.Int, parentNumber uint64, parentHash, parentUncleHash common.Hash) *big.Int
+	CalcDifficulty(chain ChainHeaderReader, time, parentTime uint64, parentDifficulty *big.Int, parentNumber uint64, parentHash, parentUncleHash common.Hash, parentSeal []rlp.RawValue) *big.Int
+
+	GenerateSeal(chain ChainHeaderReader, currnt, parent *types.Header) []rlp.RawValue
 
 	// APIs returns the RPC APIs this consensus engine provides.
 	APIs(chain ChainHeaderReader) []rpc.API
diff --git a/consensus/ethash/consensus.go b/consensus/ethash/consensus.go
index ef3153d833..c623f08c41 100644
--- a/consensus/ethash/consensus.go
+++ b/consensus/ethash/consensus.go
@@ -274,7 +274,7 @@ func (ethash *Ethash) verifyHeader(chain consensus.ChainHeaderReader, header, pa
 		return errOlderBlockTime
 	}
 	// Verify the block's difficulty based on its timestamp and parent's difficulty
-	expected := ethash.CalcDifficulty(chain, header.Time, parent.Time, parent.Difficulty, parent.Number.Uint64(), parent.Hash(), parent.UncleHash)
+	expected := ethash.CalcDifficulty(chain, header.Time, parent.Time, parent.Difficulty, parent.Number.Uint64(), parent.Hash(), parent.UncleHash, parent.Seal)
 
 	if expected.Cmp(header.Difficulty) != 0 {
 		return fmt.Errorf("invalid difficulty: have %v, want %v", header.Difficulty, expected)
@@ -328,10 +328,14 @@ func (ethash *Ethash) verifyHeader(chain consensus.ChainHeaderReader, header, pa
 	return nil
 }
 
+func (ethash *Ethash) GenerateSeal(chain consensus.ChainHeaderReader, currnt, parent *types.Header) []rlp.RawValue {
+	return nil
+}
+
 // CalcDifficulty is the difficulty adjustment algorithm. It returns
 // the difficulty that a new block should have when created at time
 // given the parent block's time and difficulty.
-func (ethash *Ethash) CalcDifficulty(chain consensus.ChainHeaderReader, time, parentTime uint64, parentDifficulty *big.Int, parentNumber uint64, _, parentUncleHash common.Hash) *big.Int {
+func (ethash *Ethash) CalcDifficulty(chain consensus.ChainHeaderReader, time, parentTime uint64, parentDifficulty *big.Int, parentNumber uint64, _, parentUncleHash common.Hash, _ []rlp.RawValue) *big.Int {
 	return CalcDifficulty(chain.Config(), time, parentTime, parentDifficulty, parentNumber, parentUncleHash)
 }
 
@@ -588,7 +592,7 @@ func (ethash *Ethash) Prepare(chain consensus.ChainHeaderReader, header *types.H
 	if parent == nil {
 		return consensus.ErrUnknownAncestor
 	}
-	header.Difficulty = ethash.CalcDifficulty(chain, header.Time, parent.Time, parent.Difficulty, parent.Number.Uint64(), parent.Hash(), parent.UncleHash)
+	header.Difficulty = ethash.CalcDifficulty(chain, header.Time, parent.Time, parent.Difficulty, parent.Number.Uint64(), parent.Hash(), parent.UncleHash, parent.Seal)
 	return nil
 }
 
diff --git a/core/chain_makers.go b/core/chain_makers.go
index 306593b63d..7f10c0b33c 100644
--- a/core/chain_makers.go
+++ b/core/chain_makers.go
@@ -191,7 +191,7 @@ func (b *BlockGen) OffsetTime(seconds int64) {
 	}
 	chainreader := &FakeChainReader{Cfg: b.config}
 	parent := b.parent.Header()
-	b.header.Difficulty = b.engine.CalcDifficulty(chainreader, b.header.Time, parent.Time, parent.Difficulty, parent.Number.Uint64(), parent.Hash(), parent.UncleHash)
+	b.header.Difficulty = b.engine.CalcDifficulty(chainreader, b.header.Time, parent.Time, parent.Difficulty, parent.Number.Uint64(), parent.Hash(), parent.UncleHash, parent.Seal)
 }
 
 func (b *BlockGen) GetHeader() *types.Header {
@@ -403,11 +403,13 @@ func makeHeader(chain consensus.ChainReader, parent *types.Block, state *state.I
 			parent.Number().Uint64(),
 			parent.Hash(),
 			parent.UncleHash(),
+			parent.Header().Seal,
 		),
 		GasLimit: CalcGasLimit(parent.GasUsed(), parent.GasLimit(), parent.GasLimit(), parent.GasLimit()),
 		Number:   new(big.Int).Add(parent.Number(), common.Big1),
 		Time:     time,
 	}
+	header.Seal = engine.GenerateSeal(chain, header, parent.Header())
 
 	if chain.Config().IsLondon(header.Number.Uint64()) {
 		header.BaseFee = misc.CalcBaseFee(chain.Config(), parent.Header())
diff --git a/core/genesis.go b/core/genesis.go
index 24839a58ff..cab47eb6ed 100644
--- a/core/genesis.go
+++ b/core/genesis.go
@@ -41,6 +41,7 @@ import (
 	"github.com/ledgerwatch/erigon/ethdb/kv"
 	"github.com/ledgerwatch/erigon/log"
 	"github.com/ledgerwatch/erigon/params"
+	"github.com/ledgerwatch/erigon/rlp"
 	"github.com/ledgerwatch/erigon/turbo/trie"
 )
 
@@ -64,7 +65,7 @@ type Genesis struct {
 	Mixhash    common.Hash         `json:"mixHash"`
 	Coinbase   common.Address      `json:"coinbase"`
 	Alloc      GenesisAlloc        `json:"alloc"      gencodec:"required"`
-	Seal       [][]byte            `json:"seal"`
+	SealRlp    []byte              `json:"sealRlp"`
 
 	// These fields are used for consensus tests. Please don't use them
 	// in actual genesis blocks.
@@ -77,6 +78,13 @@ type Genesis struct {
 // GenesisAlloc specifies the initial state that is part of the genesis block.
 type GenesisAlloc map[common.Address]GenesisAccount
 
+type AuthorityRoundSeal struct {
+	/// Seal step.
+	Step uint64 `json:"step"`
+	/// Seal signature.
+	Signature common.Hash `json:"signature"`
+}
+
 func (ga *GenesisAlloc) UnmarshalJSON(data []byte) error {
 	m := make(map[common.UnprefixedAddress]GenesisAccount)
 	if err := json.Unmarshal(data, &m); err != nil {
@@ -351,6 +359,16 @@ func (g *Genesis) ToBlock() (*types.Block, *state.IntraBlockState, error) {
 		}
 	}()
 	wg.Wait()
+	decodeSeal := func(in []byte) (seal []rlp.RawValue) {
+		if len(in) == 0 {
+			return nil
+		}
+		err := rlp.Decode(bytes.NewReader(in), &seal)
+		if err != nil {
+			panic(err)
+		}
+		return seal
+	}
 
 	head := &types.Header{
 		Number:     new(big.Int).SetUint64(g.Number),
@@ -365,8 +383,8 @@ func (g *Genesis) ToBlock() (*types.Block, *state.IntraBlockState, error) {
 		Coinbase:   g.Coinbase,
 		Root:       root,
 		BaseFee:    g.BaseFee,
-		Seal:       g.Seal,
-		WithSeal:   g.Seal != nil,
+		Seal:       decodeSeal(g.SealRlp),
+		WithSeal:   g.SealRlp != nil,
 	}
 	if g.GasLimit == 0 {
 		head.GasLimit = params.GenesisGasLimit
@@ -599,14 +617,17 @@ func DefaultSokolGenesisBlock() *Genesis {
 	/*
 		header rlp: f9020da00000000000000000000000000000000000000000000000000000000000000000a01dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347940000000000000000000000000000000000000000a0fad4af258fd11939fae0c6c6eec9d340b1caac0b0196fd9a1bc3f489c5bf00b3a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421b9010000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000830200008083663be080808080b8410000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000
 	*/
-
+	sealRlp, err := rlp.EncodeToBytes([][]byte{
+		common.FromHex(""),
+		common.FromHex("0x0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000"),
+	})
+	if err != nil {
+		panic(err)
+	}
 	return &Genesis{
-		Config:    params.SokolChainConfig,
-		Timestamp: 0x0,
-		Seal: [][]byte{
-			common.FromHex(""),
-			common.FromHex("0x0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000"),
-		},
+		Config:     params.SokolChainConfig,
+		Timestamp:  0x0,
+		SealRlp:    sealRlp,
 		GasLimit:   0x663BE0,
 		Difficulty: big.NewInt(0x20000),
 		Alloc:      readPrealloc("allocs/sokol.json"),
diff --git a/core/rawdb/accessors_account.go b/core/rawdb/accessors_account.go
index 37cfb52a99..9d031419a1 100644
--- a/core/rawdb/accessors_account.go
+++ b/core/rawdb/accessors_account.go
@@ -36,12 +36,15 @@ func ReadAccountDeprecated(db ethdb.DatabaseReader, addrHash common.Hash, acc *a
 	return true, nil
 }
 
-func ReadAccount(db ethdb.Tx, addrHash common.Hash, acc *accounts.Account) (bool, error) {
+func ReadAccount(db ethdb.Tx, addrHash common.Address, acc *accounts.Account) (bool, error) {
 	addrHashBytes := addrHash[:]
-	enc, err := db.GetOne(dbutils.HashedAccountsBucket, addrHashBytes)
+	enc, err := db.GetOne(dbutils.PlainStateBucket, addrHashBytes)
 	if err != nil {
 		return false, err
 	}
+	if len(enc) == 0 {
+		return false, nil
+	}
 	if err = acc.DecodeForStorage(enc); err != nil {
 		return false, err
 	}
diff --git a/core/types/block.go b/core/types/block.go
index 1dbcee8a10..e556fc75f7 100644
--- a/core/types/block.go
+++ b/core/types/block.go
@@ -88,7 +88,7 @@ type Header struct {
 	Nonce       BlockNonce     `json:"nonce"`
 	BaseFee     *big.Int       `json:"baseFeePerGas"`
 	Eip1559     bool           // to avoid relying on BaseFee != nil for that
-	Seal        [][]byte       // AuRa POA network field
+	Seal        []rlp.RawValue // AuRa POA network field
 	WithSeal    bool           // to avoid relying on Seal != nil for that
 }
 
@@ -99,19 +99,7 @@ func (h Header) EncodingSize() int {
 	var sealListLen int
 	if h.WithSeal {
 		for i := range h.Seal {
-			sealListLen++
-			switch len(h.Seal[i]) {
-			case 0:
-			case 1:
-				if h.Seal[i][0] >= 128 {
-					sealListLen++
-				}
-			default:
-				if len(h.Seal[i]) >= 56 {
-					sealListLen += (bits.Len(uint(len(h.Seal[i]))) + 7) / 8
-				}
-				sealListLen += len(h.Seal[i])
-			}
+			sealListLen += len(h.Seal[i])
 		}
 		encodingSize += sealListLen
 	} else {
@@ -182,19 +170,7 @@ func (h Header) EncodeRLP(w io.Writer) error {
 	var sealListLen int
 	if h.WithSeal {
 		for i := range h.Seal {
-			sealListLen++
-			switch len(h.Seal[i]) {
-			case 0:
-			case 1:
-				if h.Seal[i][0] >= 128 {
-					sealListLen++
-				}
-			default:
-				if len(h.Seal[i]) >= 56 {
-					sealListLen += (bits.Len(uint(len(h.Seal[i]))) + 7) / 8
-				}
-				sealListLen += len(h.Seal[i])
-			}
+			sealListLen += len(h.Seal[i])
 		}
 		encodingSize += sealListLen
 	} else {
@@ -207,24 +183,28 @@ func (h Header) EncodeRLP(w io.Writer) error {
 		diffLen = (h.Difficulty.BitLen() + 7) / 8
 	}
 	encodingSize += diffLen
+
 	encodingSize++
 	var numberLen int
 	if h.Number != nil && h.Number.BitLen() >= 8 {
 		numberLen = (h.Number.BitLen() + 7) / 8
 	}
 	encodingSize += numberLen
+
 	encodingSize++
 	var gasLimitLen int
 	if h.GasLimit >= 128 {
 		gasLimitLen = (bits.Len64(h.GasLimit) + 7) / 8
 	}
 	encodingSize += gasLimitLen
+
 	encodingSize++
 	var gasUsedLen int
 	if h.GasUsed >= 128 {
 		gasUsedLen = (bits.Len64(h.GasUsed) + 7) / 8
 	}
 	encodingSize += gasUsedLen
+
 	encodingSize++
 	var timeLen int
 	if h.Time >= 128 {
@@ -377,7 +357,7 @@ func (h Header) EncodeRLP(w io.Writer) error {
 
 	if h.WithSeal {
 		for i := range h.Seal {
-			if err := EncodeString(h.Seal[i], w, b[:]); err != nil {
+			if _, err := w.Write(h.Seal[i]); err != nil {
 				return err
 			}
 		}
@@ -503,9 +483,8 @@ func (h *Header) DecodeRLP(s *rlp.Stream) error {
 
 	if h.WithSeal {
 		h.WithSeal = true
-		for b, err = s.Bytes(); err == nil; b, err = s.Bytes() {
-			h.Seal = append(h.Seal, make(rlp.RawValue, len(b)))
-			copy(h.Seal[len(h.Seal)-1][:], b)
+		for b, err = s.Raw(); err == nil; b, err = s.Raw() {
+			h.Seal = append(h.Seal, b)
 		}
 		if !errors.Is(err, rlp.EOL) {
 			return fmt.Errorf("open accessTuple: %d %w", len(h.Seal), err)
@@ -984,7 +963,7 @@ func CopyHeader(h *Header) *Header {
 		copy(cpy.Extra, h.Extra)
 	}
 	if len(h.Seal) > 0 {
-		cpy.Seal = make([][]byte, len(h.Seal))
+		cpy.Seal = make([]rlp.RawValue, len(h.Seal))
 		for i := range h.Seal {
 			cpy.Seal[i] = common.CopyBytes(h.Seal[i])
 		}
diff --git a/eth/ethconfig/config.go b/eth/ethconfig/config.go
index 1dd181c64f..deaf9e315a 100644
--- a/eth/ethconfig/config.go
+++ b/eth/ethconfig/config.go
@@ -18,7 +18,6 @@
 package ethconfig
 
 import (
-	"encoding/json"
 	"math/big"
 	"os"
 	"os/user"
@@ -212,16 +211,8 @@ func CreateConsensusEngine(chainConfig *params.ChainConfig, config interface{},
 		}
 	case *params.AuRaConfig:
 		if chainConfig.Aura != nil {
-			spec := aura.JsonSpec{}
-			err := json.Unmarshal(consensusconfig.Sokol, &spec)
-			if err != nil {
-				panic(err)
-			}
-			cfg, err := aura.FromJson(spec)
-			if err != nil {
-				panic(err)
-			}
-			eng, err = aura.NewAuRa(chainConfig, db.OpenDatabase(consensusCfg.DBPath, consensusCfg.InMemory), chainConfig.Aura.Etherbase, cfg)
+			var err error
+			eng, err = aura.NewAuRa(chainConfig.Aura, db.OpenDatabase(consensusCfg.DBPath, consensusCfg.InMemory), chainConfig.Aura.Etherbase, consensusconfig.Sokol)
 			if err != nil {
 				panic(err)
 			}
diff --git a/eth/stagedsync/stage_headers.go b/eth/stagedsync/stage_headers.go
index b21fd32ad6..1cb9032de2 100644
--- a/eth/stagedsync/stage_headers.go
+++ b/eth/stagedsync/stage_headers.go
@@ -114,7 +114,6 @@ func HeadersForward(
 	var peer []byte
 	stopped := false
 	prevProgress := headerProgress
-
 	for !stopped {
 		currentTime := uint64(time.Now().Unix())
 		req, penalties := cfg.hd.RequestMoreHeaders(currentTime)
diff --git a/go.mod b/go.mod
index 099213ade4..00672add7f 100644
--- a/go.mod
+++ b/go.mod
@@ -41,8 +41,8 @@ require (
 	github.com/json-iterator/go v1.1.11
 	github.com/julienschmidt/httprouter v1.3.0
 	github.com/kevinburke/go-bindata v3.21.0+incompatible
-	github.com/ledgerwatch/erigon-lib v0.0.0-20210626115532-b71c118e1758 // indirect
-	github.com/ledgerwatch/secp256k1 v0.0.0-20210626115225-cd5cd00ed72d // indirect
+	github.com/ledgerwatch/erigon-lib v0.0.0-20210626115532-b71c118e1758
+	github.com/ledgerwatch/secp256k1 v0.0.0-20210626115225-cd5cd00ed72d
 	github.com/logrusorgru/aurora v2.0.3+incompatible
 	github.com/mattn/go-colorable v0.1.7
 	github.com/mattn/go-isatty v0.0.12
diff --git a/go.sum b/go.sum
index 5f4ad6ee34..5706650120 100644
--- a/go.sum
+++ b/go.sum
@@ -635,8 +635,6 @@ github.com/leanovate/gopter v0.2.9 h1:fQjYxZaynp97ozCzfOyOuAGOU4aU/z37zf/tOujFk7
 github.com/leanovate/gopter v0.2.9/go.mod h1:U2L/78B+KVFIx2VmW6onHJQzXtFb+p5y3y2Sh+Jxxv8=
 github.com/ledgerwatch/erigon-lib v0.0.0-20210626115532-b71c118e1758 h1:x3EJ/h5n2f7XGVBEJO2UtsyyKtSnlFXETijTPypolVE=
 github.com/ledgerwatch/erigon-lib v0.0.0-20210626115532-b71c118e1758/go.mod h1:nrVFoWX5dK/VeV5h1fy9LPHveykqevMRoh1PHKgH7ow=
-github.com/ledgerwatch/secp256k1 v0.0.0-20210620094219-1f13e1400c9b h1:pVxTy+0G5MBgzgq/Jr8FuAWvTQhFo76vEAgREm+KNXo=
-github.com/ledgerwatch/secp256k1 v0.0.0-20210620094219-1f13e1400c9b/go.mod h1:SPmqJFciiF/Q0mPt2jVs2dTr/1TZBTIA+kPMmKgBAak=
 github.com/ledgerwatch/secp256k1 v0.0.0-20210626115225-cd5cd00ed72d h1:/IKMrJdfRsoYNc36PXqP4xMH3vhW/8IQyBKGQbKZUno=
 github.com/ledgerwatch/secp256k1 v0.0.0-20210626115225-cd5cd00ed72d/go.mod h1:SPmqJFciiF/Q0mPt2jVs2dTr/1TZBTIA+kPMmKgBAak=
 github.com/lib/pq v1.0.0/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo=
@@ -1083,8 +1081,6 @@ golang.org/x/crypto v0.0.0-20200709230013-948cd5f35899/go.mod h1:LzIPMQfyMNhhGPh
 golang.org/x/crypto v0.0.0-20201016220609-9e8e0b390897/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto=
 golang.org/x/crypto v0.0.0-20201203163018-be400aefbc4c/go.mod h1:jdWPYTVW3xRLrWPugEBEK3UY2ZEsg3UU495nc5E+M+I=
 golang.org/x/crypto v0.0.0-20201221181555-eec23a3978ad/go.mod h1:jdWPYTVW3xRLrWPugEBEK3UY2ZEsg3UU495nc5E+M+I=
-golang.org/x/crypto v0.0.0-20210322153248-0c34fe9e7dc2 h1:It14KIkyBFYkHkwZ7k45minvA9aorojkyjGk9KJ5B/w=
-golang.org/x/crypto v0.0.0-20210322153248-0c34fe9e7dc2/go.mod h1:T9bdIzuCu7OtxOm1hfPfRQxPLYneinmdGuTeoZ9dtd4=
 golang.org/x/crypto v0.0.0-20210616213533-5ff15b29337e h1:gsTQYXdTw2Gq7RBsWvlQ91b+aEQ6bXFUngBGuR8sPpI=
 golang.org/x/crypto v0.0.0-20210616213533-5ff15b29337e/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc=
 golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA=
@@ -1263,8 +1259,6 @@ golang.org/x/sys v0.0.0-20201214210602-f9fddec55a1e/go.mod h1:h1NjWce9XRLGQEsW7w
 golang.org/x/sys v0.0.0-20210105210732-16f7687f5001/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
 golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
 golang.org/x/sys v0.0.0-20210217105451-b926d437f341/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20210420205809-ac73e9fd8988 h1:EjgCl+fVlIaPJSori0ikSz3uV0DOHKWOJFpv1sAAhBM=
-golang.org/x/sys v0.0.0-20210420205809-ac73e9fd8988/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
 golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1 h1:SrN+KX8Art/Sf4HNj6Zcz06G7VEz+7w9tdXTPOZ7+l4=
 golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
 golang.org/x/term v0.0.0-20201117132131-f5c789dd3221/go.mod h1:Nr5EML6q2oocZ2LXRh80K7BxOlk5/8JxuGnuhpl+muw=
diff --git a/turbo/adapter/chain_context.go b/turbo/adapter/chain_context.go
index 2b446e4a00..2f7c608ec1 100644
--- a/turbo/adapter/chain_context.go
+++ b/turbo/adapter/chain_context.go
@@ -10,6 +10,7 @@ import (
 	"github.com/ledgerwatch/erigon/core/types"
 	"github.com/ledgerwatch/erigon/ethdb"
 	"github.com/ledgerwatch/erigon/params"
+	"github.com/ledgerwatch/erigon/rlp"
 	"github.com/ledgerwatch/erigon/rpc"
 )
 
@@ -56,7 +57,12 @@ func (c *powEngine) Seal(chain consensus.ChainHeaderReader, block *types.Block,
 func (c *powEngine) SealHash(header *types.Header) common.Hash {
 	panic("must not be called")
 }
-func (c *powEngine) CalcDifficulty(chain consensus.ChainHeaderReader, time, parentTime uint64, parentDifficulty *big.Int, parentNumber uint64, parentHash, parentUncleHash common.Hash) *big.Int {
+
+func (c *powEngine) GenerateSeal(chain consensus.ChainHeaderReader, currnt, parent *types.Header) []rlp.RawValue {
+	return nil
+}
+
+func (c *powEngine) CalcDifficulty(chain consensus.ChainHeaderReader, time, parentTime uint64, parentDifficulty *big.Int, parentNumber uint64, parentHash, parentUncleHash common.Hash, _ []rlp.RawValue) *big.Int {
 	panic("must not be called")
 }
 func (c *powEngine) APIs(chain consensus.ChainHeaderReader) []rpc.API {
diff --git a/turbo/stages/genesis_test.go b/turbo/stages/genesis_test.go
index 0212b90f5d..5745ee092d 100644
--- a/turbo/stages/genesis_test.go
+++ b/turbo/stages/genesis_test.go
@@ -86,8 +86,8 @@ func TestSokolHeaderRLP(t *testing.T) {
 		require.NoError(err)
 		require.Equal(2, len(h.Seal))
 
-		expectSeal2 := common.FromHex("0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000")
-		require.Equal(common.FromHex(""), h.Seal[0])
+		expectSeal2 := rlp.RawValue(common.FromHex("0xb8410000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000"))
+		require.Equal(rlp.RawValue(common.FromHex("0x80")), h.Seal[0])
 		require.Equal(expectSeal2, h.Seal[1])
 		enc, err := rlp.EncodeToBytes(h)
 		require.NoError(err)
@@ -100,9 +100,9 @@ func TestSokolHeaderRLP(t *testing.T) {
 		err := rlp.DecodeBytes(enc, h)
 		require.NoError(err)
 		require.Equal(2, len(h.Seal))
-		require.Equal(common.FromHex("2"), h.Seal[0])
+		require.Equal(rlp.RawValue(common.FromHex("0x2")), h.Seal[0])
 
-		expectSeal2 := common.FromHex("0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000001")
+		expectSeal2 := rlp.RawValue(common.FromHex("0xb8410000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000001"))
 		require.Equal(expectSeal2, h.Seal[1])
 
 		res, err := rlp.EncodeToBytes(h) // after encode getting source bytes
diff --git a/turbo/stages/mock_sentry.go b/turbo/stages/mock_sentry.go
index 9d7678b44b..107d564d12 100644
--- a/turbo/stages/mock_sentry.go
+++ b/turbo/stages/mock_sentry.go
@@ -24,12 +24,14 @@ import (
 	"github.com/ledgerwatch/erigon/eth/fetcher"
 	"github.com/ledgerwatch/erigon/eth/protocols/eth"
 	"github.com/ledgerwatch/erigon/eth/stagedsync"
+	"github.com/ledgerwatch/erigon/eth/stagedsync/stages"
 	"github.com/ledgerwatch/erigon/ethdb"
 	"github.com/ledgerwatch/erigon/ethdb/kv"
 	"github.com/ledgerwatch/erigon/ethdb/remote/remotedbserver"
 	"github.com/ledgerwatch/erigon/gointerfaces"
 	proto_sentry "github.com/ledgerwatch/erigon/gointerfaces/sentry"
 	ptypes "github.com/ledgerwatch/erigon/gointerfaces/types"
+	"github.com/ledgerwatch/erigon/log"
 	"github.com/ledgerwatch/erigon/params"
 	"github.com/ledgerwatch/erigon/rlp"
 	"github.com/ledgerwatch/erigon/turbo/remote"
@@ -324,6 +326,13 @@ func Mock(t *testing.T) *MockSentry {
 	return MockWithGenesis(t, gspec, key)
 }
 
+func (ms *MockSentry) EnableLogs() {
+	log.Root().SetHandler(log.LvlFilterHandler(log.LvlInfo, log.StreamHandler(os.Stderr, log.TerminalFormat(true))))
+	ms.t.Cleanup(func() {
+		log.Root().SetHandler(log.Root().GetHandler())
+	})
+}
+
 func (ms *MockSentry) InsertChain(chain *core.ChainPack) error {
 	// Send NewBlock message
 	b, err := rlp.EncodeToBytes(&eth.NewBlockPacket{
@@ -383,6 +392,13 @@ func (ms *MockSentry) InsertChain(chain *core.ChainPack) error {
 		if rawdb.ReadHeader(tx, chain.TopBlock.Hash(), chain.TopBlock.NumberU64()) == nil {
 			return fmt.Errorf("did not import block %d %x", chain.TopBlock.NumberU64(), chain.TopBlock.Hash())
 		}
+		execAt, err := stages.GetStageProgress(tx, stages.Execution)
+		if err != nil {
+			return err
+		}
+		if execAt == 0 {
+			return fmt.Errorf("sentryMock.InsertChain end up with Execution stage progress = 0")
+		}
 		return nil
 	}); err != nil {
 		return err
-- 
GitLab