From 605d7a8d59bf27b3bdef21882beeb276879bca98 Mon Sep 17 00:00:00 2001
From: Alex Sharov <AskAlexSharov@gmail.com>
Date: Sat, 26 Jun 2021 01:13:40 +0700
Subject: [PATCH] Sokol v0  (#2228)

* handle chain flag in integration

* handle chain flag in integration

* handle chain flag in integration

* save

* save

* merge devel

* save

* noopWriter - one is enough

* chain spec parser

* chain spec parser

* embed

* embed

* embed

* embed

* embed

* embed

* embed

* clean

* clean

* correct alloc after reset state

* correct alloc after reset state

* correct alloc after reset state

* integration reset state now does re-apply genesis and chainConfig

* eips summary

* eips summary

* eips summary
---
 Makefile                                      |   1 +
 accounts/abi/bind/backends/simulated.go       |   2 +-
 cmd/evm/runner.go                             |   2 +-
 cmd/integration/commands/reset_state.go       |  27 +-
 cmd/integration/commands/stages.go            |  53 +-
 cmd/integration/commands/state_stages.go      |   2 +
 cmd/rpcdaemon/commands/eth_receipts.go        |   2 +-
 cmd/state/commands/opcode_tracer.go           |   4 +-
 cmd/utils/flags.go                            |   2 +-
 common/u256/big.go                            |   1 +
 consensus/aura/aura.go                        | 891 ++++++++++++++++++
 consensus/aura/auraabi/abi.go                 |   4 +
 consensus/aura/auraabi/gen_block_reward.go    | 193 ++++
 consensus/aura/auraabi/gen_validator_set.go   | 452 +++++++++
 consensus/aura/aurainterfaces/interface.go    |  58 ++
 consensus/aura/config.go                      | 292 ++++++
 consensus/aura/consensusconfig/embed.go       |   8 +
 consensus/aura/consensusconfig/poasokol.json  |  28 +
 .../contracts/authority_round_random.json     | 149 +++
 consensus/aura/contracts/block_gas_limit.json |  16 +
 consensus/aura/contracts/block_reward.json    |  29 +
 consensus/aura/contracts/embed.go             |   8 +
 consensus/aura/contracts/registrar.json       |  21 +
 .../test_authority_round_random.json          | 265 ++++++
 .../contracts/test_authority_round_random.sol | 101 ++
 .../aura/contracts/test_validator_set.json    |   8 +
 consensus/aura/contracts/tx_acl.json          |   1 +
 .../aura/contracts/tx_acl_deprecated.json     |   1 +
 .../aura/contracts/tx_acl_gas_price.json      |  83 ++
 .../aura/contracts/validator_report.json      |   5 +
 consensus/aura/contracts/validator_set.json   |  55 ++
 consensus/aura/validators.go                  | 465 +++++++++
 consensus/clique/aura.go                      | 309 ------
 consensus/clique/clique.go                    |   4 +-
 consensus/consensus.go                        |   6 +-
 consensus/ethash/consensus.go                 |   6 +-
 core/blockchain.go                            |  40 +-
 core/chain_makers.go                          |   6 +-
 core/genesis.go                               |  34 +-
 core/rawdb/accessors_metadata.go              |   5 +
 core/state_processor.go                       |  14 +-
 core/types/access_list_tx.go                  |   9 +
 core/types/dynamic_fee_tx.go                  |  10 +
 core/types/legacy_tx.go                       |  10 +
 core/types/transaction.go                     |   1 +
 eth/backend.go                                |   1 +
 eth/ethconfig/config.go                       |  18 +-
 eth/stagedsync/stage_execute.go               |   1 +
 eth/stagedsync/stage_interhashes.go           |   2 +-
 eth/stagedsync/stage_mining_exec.go           |   2 +-
 go.mod                                        |   1 +
 go.sum                                        |   2 +
 gointerfaces/types/types.pb.go                |  11 +-
 params/config.go                              |  61 +-
 turbo/adapter/chain_context.go                |   4 +-
 55 files changed, 3385 insertions(+), 401 deletions(-)
 create mode 100644 consensus/aura/aura.go
 create mode 100644 consensus/aura/auraabi/abi.go
 create mode 100644 consensus/aura/auraabi/gen_block_reward.go
 create mode 100644 consensus/aura/auraabi/gen_validator_set.go
 create mode 100644 consensus/aura/aurainterfaces/interface.go
 create mode 100644 consensus/aura/config.go
 create mode 100644 consensus/aura/consensusconfig/embed.go
 create mode 100644 consensus/aura/consensusconfig/poasokol.json
 create mode 100644 consensus/aura/contracts/authority_round_random.json
 create mode 100644 consensus/aura/contracts/block_gas_limit.json
 create mode 100644 consensus/aura/contracts/block_reward.json
 create mode 100644 consensus/aura/contracts/embed.go
 create mode 100644 consensus/aura/contracts/registrar.json
 create mode 100644 consensus/aura/contracts/test_authority_round_random.json
 create mode 100644 consensus/aura/contracts/test_authority_round_random.sol
 create mode 100644 consensus/aura/contracts/test_validator_set.json
 create mode 100644 consensus/aura/contracts/tx_acl.json
 create mode 100644 consensus/aura/contracts/tx_acl_deprecated.json
 create mode 100644 consensus/aura/contracts/tx_acl_gas_price.json
 create mode 100644 consensus/aura/contracts/validator_report.json
 create mode 100644 consensus/aura/contracts/validator_set.json
 create mode 100644 consensus/aura/validators.go
 delete mode 100644 consensus/clique/aura.go

diff --git a/Makefile b/Makefile
index 06e390c9fa..270ab4fe73 100644
--- a/Makefile
+++ b/Makefile
@@ -151,6 +151,7 @@ devtools:
 	$(GOBUILD) -o $(GOBIN)/abigen ./cmd/abigen
 	PATH=$(GOBIN):$(PATH) go generate ./common
 	PATH=$(GOBIN):$(PATH) go generate ./core/types
+	PATH=$(GOBIN):$(PATH) go generate ./consensus/aura/...
 	@type "npm" 2> /dev/null || echo 'Please install node.js and npm'
 	@type "solc" 2> /dev/null || echo 'Please install solc'
 	@type "protoc" 2> /dev/null || echo 'Please install protoc'
diff --git a/accounts/abi/bind/backends/simulated.go b/accounts/abi/bind/backends/simulated.go
index 8402dff5b9..b8c9c7b2f7 100644
--- a/accounts/abi/bind/backends/simulated.go
+++ b/accounts/abi/bind/backends/simulated.go
@@ -678,7 +678,7 @@ func (b *SimulatedBackend) SendTransaction(ctx context.Context, tx types.Transac
 
 	b.pendingState.Prepare(tx.Hash(), common.Hash{}, len(b.pendingBlock.Transactions()))
 	//fmt.Printf("==== Start producing block %d, header: %d\n", b.pendingBlock.NumberU64(), b.pendingHeader.Number.Uint64())
-	if _, err := core.ApplyTransaction(
+	if _, _, err := core.ApplyTransaction(
 		b.m.ChainConfig, b.getHeader, b.m.Engine,
 		&b.pendingHeader.Coinbase, b.gasPool,
 		b.pendingState, state.NewNoopWriter(),
diff --git a/cmd/evm/runner.go b/cmd/evm/runner.go
index d3229bc93a..02a760e521 100644
--- a/cmd/evm/runner.go
+++ b/cmd/evm/runner.go
@@ -138,7 +138,7 @@ func runCmd(ctx *cli.Context) error {
 	db := kv.NewMemDatabase()
 	if ctx.GlobalString(GenesisFlag.Name) != "" {
 		gen := readGenesis(ctx.GlobalString(GenesisFlag.Name))
-		_, _, err := gen.Commit(db, false)
+		_, err := gen.Commit(db, false)
 		if err != nil {
 			return err
 		}
diff --git a/cmd/integration/commands/reset_state.go b/cmd/integration/commands/reset_state.go
index dd66faf67c..dcfe4053ec 100644
--- a/cmd/integration/commands/reset_state.go
+++ b/cmd/integration/commands/reset_state.go
@@ -75,14 +75,10 @@ func clearUnwindStack(db ethdb.RwTx, _ context.Context) error {
 }
 
 func resetState(kv ethdb.RwKV, ctx context.Context) error {
-	fmt.Printf("Before reset: \n")
 	if err := kv.View(ctx, func(tx ethdb.Tx) error { return printStages(tx) }); err != nil {
 		return err
 	}
 	// don't reset senders here
-	if err := kv.Update(ctx, func(tx ethdb.RwTx) error { return resetExec(tx) }); err != nil {
-		return err
-	}
 	if err := kv.Update(ctx, func(tx ethdb.RwTx) error { return stagedsync.ResetHashState(tx) }); err != nil {
 		return err
 	}
@@ -108,18 +104,12 @@ func resetState(kv ethdb.RwKV, ctx context.Context) error {
 		return err
 	}
 
-	// set genesis after reset all buckets
-	tx, err := kv.BeginRw(context.Background())
-	if err != nil {
-		return err
-	}
-	if _, _, err = core.DefaultGenesisBlock().WriteGenesisState(tx, false); err != nil {
-		return err
-	}
-	if err := tx.Commit(); err != nil {
+	genesis, _ := byChain()
+	if err := kv.Update(ctx, func(tx ethdb.RwTx) error { return resetExec(tx, genesis) }); err != nil {
 		return err
 	}
 
+	// set genesis after reset all buckets
 	fmt.Printf("After reset: \n")
 	if err := kv.View(ctx, func(tx ethdb.Tx) error { return printStages(tx) }); err != nil {
 		return err
@@ -140,7 +130,7 @@ func resetSenders(tx ethdb.RwTx) error {
 	return nil
 }
 
-func resetExec(tx ethdb.RwTx) error {
+func resetExec(tx ethdb.RwTx, g *core.Genesis) error {
 	if err := tx.(ethdb.BucketMigrator).ClearBucket(dbutils.HashedAccountsBucket); err != nil {
 		return err
 	}
@@ -180,6 +170,15 @@ func resetExec(tx ethdb.RwTx) error {
 	if err := stages.SaveStageProgress(tx, stages.Execution, 0); err != nil {
 		return err
 	}
+
+	sm, err := ethdb.GetStorageModeFromDB(tx)
+	if err != nil {
+		return err
+	}
+	_, _, err = core.OverrideGenesisBlock(tx, g, sm.History)
+	if err != nil {
+		return err
+	}
 	return nil
 }
 
diff --git a/cmd/integration/commands/stages.go b/cmd/integration/commands/stages.go
index c82a58e37c..b66a427a81 100644
--- a/cmd/integration/commands/stages.go
+++ b/cmd/integration/commands/stages.go
@@ -411,9 +411,11 @@ func stageExec(db ethdb.RwKV, ctx context.Context) error {
 	sm, engine, chainConfig, vmConfig, _, sync, _, _, _ := newSync(ctx, db)
 
 	if reset {
-		if err := db.Update(ctx, func(tx ethdb.RwTx) error { return resetExec(tx) }); err != nil {
+		genesis, _ := byChain()
+		if err := db.Update(ctx, func(tx ethdb.RwTx) error { return resetExec(tx, genesis) }); err != nil {
 			return err
 		}
+		return nil
 	}
 	if txtrace {
 		// Activate tracing and writing into json files for each transaction
@@ -729,24 +731,9 @@ func removeMigration(db ethdb.RwKV, ctx context.Context) error {
 	})
 }
 
-func newSync(ctx context.Context, db ethdb.RwKV) (ethdb.StorageMode, consensus.Engine, *params.ChainConfig, *vm.Config, *core.TxPool, *stagedsync.State, *stagedsync.StagedSync, chan *types.Block, chan *types.Block) {
-	tmpdir := path.Join(datadir, etl.TmpDirName)
-	snapshotDir = path.Join(datadir, "erigon", "snapshot")
-
-	var sm ethdb.StorageMode
-
-	var err error
-	if err = db.View(context.Background(), func(tx ethdb.Tx) error {
-		sm, err = ethdb.GetStorageModeFromDB(tx)
-		if err != nil {
-			return err
-		}
-		return nil
-	}); err != nil {
-		panic(err)
-	}
-	vmConfig := &vm.Config{NoReceipts: !sm.Receipts}
+func byChain() (*core.Genesis, *params.ChainConfig) {
 	var chainConfig *params.ChainConfig
+
 	var genesis *core.Genesis
 	switch chain {
 	case "", params.MainnetChainName:
@@ -768,6 +755,35 @@ func newSync(ctx context.Context, db ethdb.RwKV) (ethdb.StorageMode, consensus.E
 		chainConfig = params.SokolChainConfig
 		genesis = core.DefaultSokolGenesisBlock()
 	}
+	return genesis, chainConfig
+}
+
+func newSync(ctx context.Context, db ethdb.RwKV) (ethdb.StorageMode, consensus.Engine, *params.ChainConfig, *vm.Config, *core.TxPool, *stagedsync.State, *stagedsync.StagedSync, chan *types.Block, chan *types.Block) {
+	tmpdir := path.Join(datadir, etl.TmpDirName)
+	snapshotDir = path.Join(datadir, "erigon", "snapshot")
+
+	var sm ethdb.StorageMode
+
+	var err error
+	if err = db.View(context.Background(), func(tx ethdb.Tx) error {
+		sm, err = ethdb.GetStorageModeFromDB(tx)
+		if err != nil {
+			return err
+		}
+		return nil
+	}); err != nil {
+		panic(err)
+	}
+	vmConfig := &vm.Config{NoReceipts: !sm.Receipts}
+
+	genesis, chainConfig := byChain()
+	var engine consensus.Engine
+	engine = ethash.NewFaker()
+	switch chain {
+	case params.SokolChainName:
+		engine = ethconfig.CreateConsensusEngine(chainConfig, &params.AuRaConfig{DBPath: path.Join(datadir, "aura")}, nil, false)
+	}
+
 	events := remotedbserver.NewEvents()
 
 	txPool := core.NewTxPool(ethconfig.Defaults.TxPool, chainConfig, db)
@@ -782,7 +798,6 @@ func newSync(ctx context.Context, db ethdb.RwKV) (ethdb.StorageMode, consensus.E
 	must(batchSize.UnmarshalText([]byte(batchSizeStr)))
 	bodyDownloadTimeoutSeconds := 30 // TODO: convert to duration, make configurable
 
-	engine := ethash.NewFaker()
 	blockDownloaderWindow := 65536
 	downloadServer, err := download.NewControlServer(db, "", chainConfig, genesisBlock.Hash(), engine, 1, nil, blockDownloaderWindow)
 	if err != nil {
diff --git a/cmd/integration/commands/state_stages.go b/cmd/integration/commands/state_stages.go
index 098b7b31b8..280446d5f6 100644
--- a/cmd/integration/commands/state_stages.go
+++ b/cmd/integration/commands/state_stages.go
@@ -59,6 +59,7 @@ Examples:
 		utils.SetupMinerCobra(cmd, &miningConfig)
 		db := openDB(path.Join(cfg.DataDir, "erigon", "chaindata"), true)
 		defer db.Close()
+
 		if err := syncBySmallSteps(db, miningConfig, ctx); err != nil {
 			log.Error("Error", "err", err)
 			return nil
@@ -295,6 +296,7 @@ func syncBySmallSteps(db ethdb.RwKV, miningConfig params.MiningConfig, ctx conte
 		defer tx.Rollback()
 
 		execAtBlock = progress(tx, stages.Execution)
+
 		if execAtBlock == stopAt {
 			break
 		}
diff --git a/cmd/rpcdaemon/commands/eth_receipts.go b/cmd/rpcdaemon/commands/eth_receipts.go
index b1b598c0bd..088cba777d 100644
--- a/cmd/rpcdaemon/commands/eth_receipts.go
+++ b/cmd/rpcdaemon/commands/eth_receipts.go
@@ -48,7 +48,7 @@ func getReceipts(ctx context.Context, tx ethdb.Tx, chainConfig *params.ChainConf
 	for i, txn := range block.Transactions() {
 		ibs.Prepare(txn.Hash(), block.Hash(), i)
 
-		receipt, err := core.ApplyTransaction(chainConfig, getHeader, ethash.NewFaker(), nil, gp, ibs, state.NewNoopWriter(), block.Header(), txn, usedGas, vm.Config{}, checkTEVM)
+		receipt, _, err := core.ApplyTransaction(chainConfig, getHeader, ethash.NewFaker(), nil, gp, ibs, state.NewNoopWriter(), block.Header(), txn, usedGas, vm.Config{}, checkTEVM)
 		if err != nil {
 			return nil, err
 		}
diff --git a/cmd/state/commands/opcode_tracer.go b/cmd/state/commands/opcode_tracer.go
index 61343f5170..484b14afca 100644
--- a/cmd/state/commands/opcode_tracer.go
+++ b/cmd/state/commands/opcode_tracer.go
@@ -677,7 +677,7 @@ func runBlock(ibs *state.IntraBlockState, txnWriter state.StateWriter, blockWrit
 	}
 	for i, tx := range block.Transactions() {
 		ibs.Prepare(tx.Hash(), block.Hash(), i)
-		receipt, err := core.ApplyTransaction(chainConfig, getHeader, engine, nil, gp, ibs, txnWriter, header, tx, usedGas, vmConfig, checkTEVM)
+		receipt, _, err := core.ApplyTransaction(chainConfig, getHeader, engine, nil, gp, ibs, txnWriter, header, tx, usedGas, vmConfig, checkTEVM)
 		if err != nil {
 			return nil, fmt.Errorf("could not apply tx %d [%x] failed: %v", i, tx.Hash(), err)
 		}
@@ -686,7 +686,7 @@ func runBlock(ibs *state.IntraBlockState, txnWriter state.StateWriter, blockWrit
 
 	if !vmConfig.ReadOnly {
 		// Finalize the block, applying any consensus engine specific extras (e.g. block rewards)
-		if _, err := engine.FinalizeAndAssemble(chainConfig, header, ibs, block.Transactions(), block.Uncles(), receipts); err != nil {
+		if _, err := engine.FinalizeAndAssemble(chainConfig, header, ibs, block.Transactions(), block.Uncles(), receipts, nil); err != nil {
 			return nil, fmt.Errorf("finalize of block %d failed: %v", block.NumberU64(), err)
 		}
 
diff --git a/cmd/utils/flags.go b/cmd/utils/flags.go
index 038464ad5f..c210a072dd 100644
--- a/cmd/utils/flags.go
+++ b/cmd/utils/flags.go
@@ -1087,7 +1087,7 @@ func setClique(ctx *cli.Context, cfg *params.SnapshotConfig, datadir string) {
 }
 
 func setAuRa(ctx *cli.Context, cfg *params.AuRaConfig, datadir string) {
-	cfg.DBPath = path.Join(datadir, "aura/db")
+	cfg.DBPath = path.Join(datadir, "aura")
 }
 
 func setMiner(ctx *cli.Context, cfg *params.MiningConfig) {
diff --git a/common/u256/big.go b/common/u256/big.go
index 0bec6a69a0..0fd6d2c9a8 100644
--- a/common/u256/big.go
+++ b/common/u256/big.go
@@ -25,6 +25,7 @@ var (
 	Num0  = uint256.NewInt(0)
 	Num1  = uint256.NewInt(1)
 	Num2  = uint256.NewInt(2)
+	Num4  = uint256.NewInt(4)
 	Num8  = uint256.NewInt(8)
 	Num27 = uint256.NewInt(27)
 	Num32 = uint256.NewInt(32)
diff --git a/consensus/aura/aura.go b/consensus/aura/aura.go
new file mode 100644
index 0000000000..3194769280
--- /dev/null
+++ b/consensus/aura/aura.go
@@ -0,0 +1,891 @@
+// Copyright 2017 The go-ethereum Authors
+// This file is part of the go-ethereum library.
+//
+// The go-ethereum library is free software: you can redistribute it and/or modify
+// it under the terms of the GNU Lesser General Public License as published by
+// the Free Software Foundation, either version 3 of the License, or
+// (at your option) any later version.
+//
+// The go-ethereum library is distributed in the hope that it will be useful,
+// but WITHOUT ANY WARRANTY; without even the implied warranty of
+// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+// GNU Lesser General Public License for more details.
+//
+// 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"
+	"fmt"
+	"math/big"
+	"sort"
+	"sync"
+	"time"
+
+	lru "github.com/hashicorp/golang-lru"
+	"github.com/holiman/uint256"
+	"github.com/ledgerwatch/erigon/accounts/abi"
+	"github.com/ledgerwatch/erigon/common"
+	"github.com/ledgerwatch/erigon/consensus"
+	"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"
+	"github.com/ledgerwatch/erigon/crypto/secp256k1"
+	"github.com/ledgerwatch/erigon/ethdb"
+	"github.com/ledgerwatch/erigon/log"
+	"github.com/ledgerwatch/erigon/params"
+	"github.com/ledgerwatch/erigon/rlp"
+	"github.com/ledgerwatch/erigon/rpc"
+	"go.uber.org/atomic"
+)
+
+type StepDurationInfo struct {
+	TransitionStep      uint64
+	TransitionTimestamp uint64
+	StepDuration        uint64
+}
+
+type Step struct {
+	calibrate bool // whether calibration is enabled.
+	inner     *atomic.Uint64
+	// Planned durations of steps.
+	durations []StepDurationInfo
+}
+
+func (s *Step) doCalibrate() {
+	if s.calibrate {
+		if !s.optCalibrate() {
+			ctr := s.inner.Load()
+			panic(fmt.Errorf("step counter under- or overflow: %d", ctr))
+		}
+	}
+}
+
+// optCalibrate Calibrates the AuRa step number according to the current time.
+func (s *Step) optCalibrate() bool {
+	now := time.Now().Second()
+	var info StepDurationInfo
+	i := 0
+	for _, d := range s.durations {
+		if d.TransitionTimestamp >= uint64(now) {
+			break
+		}
+		info = d
+		i++
+	}
+	if i == 0 {
+		panic("durations cannot be empty")
+	}
+
+	if uint64(now) < info.TransitionTimestamp {
+		return false
+	}
+
+	newStep := (uint64(now)-info.TransitionTimestamp)/info.StepDuration + info.TransitionStep
+	s.inner.Store(newStep)
+	return true
+}
+
+type PermissionedStep struct {
+	inner      *Step
+	canPropose *atomic.Bool
+}
+
+//nolint
+type EpochManager struct {
+	epochTransitionHash   common.Hash // H256,
+	epochTransitionNumber uint64      // BlockNumber
+	finalityChecker       RollingFinality
+	force                 bool
+}
+
+//nolint
+type unAssembledHeader struct {
+	h common.Hash // H256
+	n uint64      // BlockNumber
+	a []common.Address
+}
+
+// RollingFinality checker for authority round consensus.
+// Stores a chain of unfinalized hashes that can be pushed onto.
+//nolint
+type RollingFinality struct {
+	headers    []unAssembledHeader //nolint
+	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
+//nolint
+type AuRa struct {
+	db     ethdb.RwKV // Database to store and retrieve snapshot checkpoints
+	exitCh chan struct{}
+	lock   sync.RWMutex // Protects the signer fields
+
+	step              PermissionedStep
+	OurSigningAddress common.Address // Same as Etherbase in Mining
+	cfg               AuthorityRoundParams
+	EmptyStepsSet     *EmptyStepSet
+
+	//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.
+	//receivedStepHashes map[uint64]map[common.Address]common.Hash // RwLock<BTreeMap<(u64, Address), H256>>
+	//// If set, enables random number contract integration. It maps the transition block to the contract address.
+	//randomnessContractAddress map[uint64]common.Address
+	//// The addresses of contracts that determine the block gas limit.
+	//blockGasLimitContractTransitions map[uint64]common.Address
+	//// Memoized gas limit overrides, by block hash.
+	//gasLimitOverrideCache *GasLimitOverride //Mutex<LruCache<H256, Option<U256>>>,
+	//// The block number at which the consensus engine switches from AuRa to AuRa with POSDAO
+	//// modifications. For details about POSDAO, see the whitepaper:
+	//// https://www.xdaichain.com/for-validators/posdao-whitepaper
+	//posdaoTransition *uint64 // Option<BlockNumber>,
+}
+
+type GasLimitOverride struct {
+	cache *lru.Cache
+}
+
+func NewGasLimitOverride() *GasLimitOverride {
+	// The number of recent block hashes for which the gas limit override is memoized.
+	const GasLimitOverrideCacheCapacity = 10
+
+	cache, err := lru.New(GasLimitOverrideCacheCapacity)
+	if err != nil {
+		panic("error creating prefetching cache for blocks")
+	}
+	return &GasLimitOverride{cache: cache}
+}
+
+func (pb *GasLimitOverride) Pop(hash common.Hash) *uint256.Int {
+	if val, ok := pb.cache.Get(hash); ok && val != nil {
+		pb.cache.Remove(hash)
+		if v, ok := val.(*uint256.Int); ok {
+			return v
+		}
+	}
+	return nil
+}
+
+func (pb *GasLimitOverride) Add(hash common.Hash, b *uint256.Int) {
+	if b == nil {
+		return
+	}
+	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
+
+	if _, ok := auraParams.StepDurations[0]; !ok {
+		return nil, fmt.Errorf("authority Round step 0 duration is undefined")
+	}
+	for _, v := range auraParams.StepDurations {
+		if v == 0 {
+			return nil, fmt.Errorf("authority Round step 0 duration is undefined")
+		}
+	}
+	if _, ok := auraParams.StepDurations[0]; !ok {
+		return nil, fmt.Errorf("authority Round step duration cannot be 0")
+	}
+	//shouldTimeout := auraParams.StartStep == nil
+	initialStep := uint64(0)
+	if auraParams.StartStep != nil {
+		initialStep = *auraParams.StartStep
+	}
+	var durations []StepDurationInfo
+	durInfo := StepDurationInfo{
+		TransitionStep:      0,
+		TransitionTimestamp: 0,
+		StepDuration:        auraParams.StepDurations[0],
+	}
+	durations = append(durations, durInfo)
+	var i = 0
+	for time, dur := range auraParams.StepDurations {
+		if i == 0 { // skip first
+			i++
+			continue
+		}
+
+		step, t, ok := nextStepTimeDuration(durInfo, time)
+		if !ok {
+			return nil, fmt.Errorf("timestamp overflow")
+		}
+		durInfo.TransitionStep = step
+		durInfo.TransitionTimestamp = t
+		durInfo.StepDuration = dur
+		durations = append(durations, durInfo)
+	}
+	step := &Step{
+		inner:     atomic.NewUint64(initialStep),
+		calibrate: auraParams.StartStep == nil,
+		durations: durations,
+	}
+	step.doCalibrate()
+
+	/*
+		    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,
+		        received_step_hashes: RwLock::new(Default::default()),
+		        gas_limit_override_cache: Mutex::new(LruCache::new(GAS_LIMIT_OVERRIDE_CACHE_CAPACITY)),
+		    })
+			// Do not initialize timeouts for tests.
+		    if should_timeout {
+		        let handler = TransitionHandler {
+		            step: engine.step.clone(),
+		            client: engine.client.clone(),
+		        };
+		        engine
+		            .transition_service
+		            .register_handler(Arc::new(handler))?;
+		    }
+	*/
+
+	exitCh := make(chan struct{})
+
+	c := &AuRa{
+		db:                db,
+		exitCh:            exitCh,
+		step:              PermissionedStep{inner: step, canPropose: atomic.NewBool(true)},
+		OurSigningAddress: ourSigningAddress,
+		cfg:               auraParams,
+	}
+	_ = config
+
+	return c, nil
+}
+
+// A helper accumulator function mapping a step duration and a step duration transition timestamp
+// to the corresponding step number and the correct starting second of the step.
+func nextStepTimeDuration(info StepDurationInfo, time uint64) (uint64, uint64, bool) {
+	stepDiff := time + info.StepDuration
+	if stepDiff < 1 {
+		return 0, 0, false
+	}
+	stepDiff -= 1
+	if stepDiff < info.TransitionTimestamp {
+		return 0, 0, false
+	}
+	stepDiff -= info.TransitionTimestamp
+	if info.StepDuration == 0 {
+		return 0, 0, false
+	}
+	stepDiff /= info.StepDuration
+	timeDiff := stepDiff * info.StepDuration
+	return info.TransitionStep + stepDiff, info.TransitionTimestamp + timeDiff, true
+}
+
+// Author implements consensus.Engine, returning the Ethereum address recovered
+// from the signature in the header's extra-data section.
+func (c *AuRa) Author(header *types.Header) (common.Address, error) {
+	return common.Address{}, nil
+	//return ecrecover(header, c.signatures)
+}
+
+// VerifyHeader checks whether a header conforms to the consensus rules.
+func (c *AuRa) VerifyHeader(chain consensus.ChainHeaderReader, header *types.Header, _ bool) error {
+	return nil
+	//return c.verifyHeader(chain, header, nil)
+}
+
+// VerifyHeaders is similar to VerifyHeader, but verifies a batch of headers. The
+// method returns a quit channel to abort the operations and a results channel to
+// retrieve the async verifications (the order is that of the input slice).
+func (c *AuRa) VerifyHeaders(chain consensus.ChainHeaderReader, headers []*types.Header, _ []bool) error {
+	return nil
+	//if len(headers) == 0 {
+	//	return nil
+	//}
+	//for i, header := range headers {
+	//	if err := c.verifyHeader(chain, header, headers[:i]); err != nil {
+	//		return err
+	//	}
+	//}
+	//return nil
+}
+
+// VerifyUncles implements consensus.Engine, always returning an error for any
+// uncles as this consensus mechanism doesn't permit uncles.
+func (c *AuRa) VerifyUncles(chain consensus.ChainReader, header *types.Header, uncles []*types.Header) error {
+	return nil
+	//if len(uncles) > 0 {
+	//	return errors.New("uncles not allowed")
+	//}
+	//return nil
+}
+
+// VerifySeal implements consensus.Engine, checking whether the signature contained
+// in the header satisfies the consensus protocol requirements.
+func (c *AuRa) VerifySeal(chain consensus.ChainHeaderReader, header *types.Header) error {
+	return nil
+	//snap, err := c.Snapshot(chain, header.Number.Uint64(), header.Hash(), nil)
+	//if err != nil {
+	//	return err
+	//}
+	//return c.verifySeal(chain, header, snap)
+}
+
+// Prepare implements consensus.Engine, preparing all the consensus fields of the
+// header for running the transactions on top.
+func (c *AuRa) Prepare(chain consensus.ChainHeaderReader, header *types.Header) error {
+	return nil
+	/// If the block isn't a checkpoint, cast a random vote (good enough for now)
+	//header.Coinbase = common.Address{}
+	//header.Nonce = types.BlockNonce{}
+	//
+	//number := header.Number.Uint64()
+	/// Assemble the voting snapshot to check which votes make sense
+	//snap, err := c.Snapshot(chain, number-1, header.ParentHash, nil)
+	//if err != nil {
+	//	return err
+	//}
+	//if number%c.config.Epoch != 0 {
+	//	c.lock.RLock()
+	//
+	//	// Gather all the proposals that make sense voting on
+	//	addresses := make([]common.Address, 0, len(c.proposals))
+	//	for address, authorize := range c.proposals {
+	//		if snap.validVote(address, authorize) {
+	//			addresses = append(addresses, address)
+	//		}
+	//	}
+	//	// If there's pending proposals, cast a vote on them
+	//	if len(addresses) > 0 {
+	//		header.Coinbase = addresses[rand.Intn(len(addresses))]
+	//		if c.proposals[header.Coinbase] {
+	//			copy(header.Nonce[:], NonceAuthVote)
+	//		} else {
+	//			copy(header.Nonce[:], nonceDropVote)
+	//		}
+	//	}
+	//	c.lock.RUnlock()
+	//}
+	/// Set the correct difficulty
+	//header.Difficulty = calcDifficulty(snap, c.signer)
+	//
+	/// Ensure the extra data has all its components
+	//if len(header.Extra) < ExtraVanity {
+	//	header.Extra = append(header.Extra, bytes.Repeat([]byte{0x00}, ExtraVanity-len(header.Extra))...)
+	//}
+	//header.Extra = header.Extra[:ExtraVanity]
+	//
+	//if number%c.config.Epoch == 0 {
+	//	for _, signer := range snap.GetSigners() {
+	//		header.Extra = append(header.Extra, signer[:]...)
+	//	}
+	//}
+	//header.Extra = append(header.Extra, make([]byte, ExtraSeal)...)
+	//
+	/// Mix digest is reserved for now, set to empty
+	//header.MixDigest = common.Hash{}
+	//
+	/// Ensure the timestamp has the correct delay
+	//parent := chain.GetHeader(header.ParentHash, number-1)
+	//if parent == nil {
+	//	return consensus.ErrUnknownAncestor
+	//}
+	//header.Time = parent.Time + c.config.Period
+	//
+	//now := uint64(time.Now().Unix())
+	//if header.Time < now {
+	//	header.Time = now
+	//}
+	//
+	//return nil
+}
+
+// Finalize implements consensus.Engine, ensuring no uncles are set, nor block
+// rewards given.
+func (c *AuRa) Finalize(cc *params.ChainConfig, header *types.Header, state *state.IntraBlockState, txs []types.Transaction, uncles []*types.Header, syscall consensus.SystemCall) {
+	// accumulateRewards retreives rewards for a block and applies them to the coinbase accounts for miner and uncle miners
+	beneficiaries, _, rewards, err := AccumulateRewards(cc, c, header, uncles, syscall)
+	if err != nil {
+		log.Error("accumulateRewards", "err", err)
+		return
+	}
+	for i := range beneficiaries {
+		state.AddBalance(beneficiaries[i], rewards[i])
+	}
+}
+
+// FinalizeAndAssemble implements consensus.Engine, ensuring no uncles are set,
+// nor block rewards given, and returns the final block.
+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)
+
+	// Assemble and return the final block for sealing
+	return types.NewBlock(header, txs, nil, receipts), nil
+}
+
+// Authorize injects a private key into the consensus engine to mint new blocks
+// with.
+func (c *AuRa) Authorize(signer common.Address, signFn clique.SignerFn) {
+	c.lock.Lock()
+	defer c.lock.Unlock()
+
+	//c.signer = signer
+	//c.signFn = signFn
+}
+
+// Seal implements consensus.Engine, attempting to create a sealed block using
+// the local signing credentials.
+func (c *AuRa) Seal(chain consensus.ChainHeaderReader, block *types.Block, results chan<- *types.Block, stop <-chan struct{}) error {
+	return nil
+	//header := block.Header()
+	//
+	/// Sealing the genesis block is not supported
+	//number := header.Number.Uint64()
+	//if number == 0 {
+	//	return errUnknownBlock
+	//}
+	/// For 0-period chains, refuse to seal empty blocks (no reward but would spin sealing)
+	//if c.config.Period == 0 && len(block.Transactions()) == 0 {
+	//	log.Info("Sealing paused, waiting for transactions")
+	//	return nil
+	//}
+	/// Don't hold the signer fields for the entire sealing procedure
+	//c.lock.RLock()
+	//signer, signFn := c.signer, c.signFn
+	//c.lock.RUnlock()
+	//
+	/// Bail out if we're unauthorized to sign a block
+	//snap, err := c.Snapshot(chain, number-1, header.ParentHash, nil)
+	//if err != nil {
+	//	return err
+	//}
+	//if _, authorized := snap.Signers[signer]; !authorized {
+	//	return ErrUnauthorizedSigner
+	//}
+	/// If we're amongst the recent signers, wait for the next block
+	//for seen, recent := range snap.Recents {
+	//	if recent == signer {
+	//		// Signer is among RecentsRLP, only wait if the current block doesn't shift it out
+	//		if limit := uint64(len(snap.Signers)/2 + 1); number < limit || seen > number-limit {
+	//			log.Info("Signed recently, must wait for others")
+	//			return nil
+	//		}
+	//	}
+	//}
+	/// Sweet, the protocol permits us to sign the block, wait for our time
+	//delay := time.Unix(int64(header.Time), 0).Sub(time.Now()) // nolint: gosimple
+	//if header.Difficulty.Cmp(diffNoTurn) == 0 {
+	//	// It's not our turn explicitly to sign, delay it a bit
+	//	wiggle := time.Duration(len(snap.Signers)/2+1) * wiggleTime
+	//	delay += time.Duration(rand.Int63n(int64(wiggle)))
+	//
+	//	log.Trace("Out-of-turn signing requested", "wiggle", common.PrettyDuration(wiggle))
+	//}
+	/// Sign all the things!
+	//sighash, err := signFn(signer, accounts.MimetypeClique, CliqueRLP(header))
+	//if err != nil {
+	//	return err
+	//}
+	//copy(header.Extra[len(header.Extra)-ExtraSeal:], sighash)
+	/// Wait until sealing is terminated or delay timeout.
+	//log.Trace("Waiting for slot to sign and propagate", "delay", common.PrettyDuration(delay))
+	//go func() {
+	//	select {
+	//	case <-stop:
+	//		return
+	//	case <-time.After(delay):
+	//	}
+	//
+	//	select {
+	//	case results <- block.WithSeal(header):
+	//	default:
+	//		log.Warn("Sealing result is not read by miner", "sealhash", SealHash(header))
+	//	}
+	//}()
+	//
+	//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 {
+	return nil
+	//snap, err := c.Snapshot(chain, parentNumber, parentHash, nil)
+	//if err != nil {
+	//	return nil
+	//}
+	//return calcDifficulty(snap, c.signer)
+}
+
+func (c *AuRa) SealHash(header *types.Header) common.Hash {
+	return clique.SealHash(header)
+}
+
+// Close implements consensus.Engine. It's a noop for clique as there are no background threads.
+func (c *AuRa) Close() error {
+	common.SafeClose(c.exitCh)
+	return nil
+}
+
+// APIs implements consensus.Engine, returning the user facing RPC API to allow
+// controlling the signer voting.
+func (c *AuRa) APIs(chain consensus.ChainHeaderReader) []rpc.API {
+	return []rpc.API{
+		//{
+		//Namespace: "clique",
+		//Version:   "1.0",
+		//Service:   &API{chain: chain, clique: c},
+		//Public:    false,
+		//}
+	}
+}
+
+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) {
+		return res
+	}
+
+	c.EmptyStepsSet.Sort()
+	c.EmptyStepsSet.ForEach(func(i int, step *EmptyStep) {
+		if step.Less(&from) || (&to).Less(step) {
+			return
+		}
+		if step.parentHash != parentHash {
+			return
+		}
+		res = append(res, *step)
+	})
+	return res
+}
+
+// AccumulateRewards returns rewards for a given block. The mining reward consists
+// 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)
+
+	rewardContractAddress := aura.cfg.BlockRewardContractTransitions.GreaterOrEqual(header.Number.Uint64())
+	if rewardContractAddress != nil {
+		beneficiaries, rewards = callBlockRewardAbi(rewardContractAddress.Address, syscall, beneficiaries, rewardKind)
+		rewardKind = rewardKind[:len(beneficiaries)]
+		for i := 0; i < len(rewardKind); i++ {
+			rewardKind[i] = aurainterfaces.RewardExternal
+		}
+	} else {
+		// find: n <= header.number
+		var foundNum uint64
+		var found bool
+		for n := range aura.cfg.BlockReward {
+			if n > header.Number.Uint64() {
+				continue
+			}
+			if n > foundNum {
+				found = true
+				foundNum = n
+			}
+		}
+		if !found {
+			panic("Current block's reward is not found; this indicates a chain config error")
+		}
+		reward := aura.cfg.BlockReward[foundNum]
+		rewards = append(rewards, reward)
+	}
+
+	//err = aura.Validators.onCloseBlock(header, aura.OurSigningAddress)
+	//if err != nil {
+	//	return
+	//}
+	return
+}
+
+func callBlockRewardAbi(contractAddr common.Address, syscall consensus.SystemCall, beneficiaries []common.Address, rewardKind []aurainterfaces.RewardKind) ([]common.Address, []*uint256.Int) {
+	castedKind := make([]uint16, len(rewardKind))
+	for i := range rewardKind {
+		castedKind[i] = uint16(rewardKind[i])
+	}
+	packed, err := blockRewardAbi().Pack("reward", beneficiaries, castedKind)
+	if err != nil {
+		panic(err)
+	}
+	out, err := syscall(contractAddr, packed)
+	if err != nil {
+		panic(err)
+	}
+	if len(out) == 0 {
+		return nil, nil
+	}
+	res, err := blockRewardAbi().Unpack("reward", out)
+	if err != nil {
+		panic(err)
+	}
+	_ = res[0]
+	_ = res[1]
+	fmt.Printf("aaaaa: %#v, %#v\n", res[0], res[1])
+	return nil, nil
+}
+
+func blockRewardAbi() abi.ABI {
+	a, err := abi.JSON(bytes.NewReader(contracts.BlockReward))
+	if err != nil {
+		panic(err)
+	}
+	return a
+}
+
+// An empty step message that is included in a seal, the only difference is that it doesn't include
+// 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.
+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) {
+	s := headerEmptyStepsRaw(header)
+	sealedSteps := []SealedEmptyStep{}
+	err := rlp.DecodeBytes(s, &sealedSteps)
+	if err != nil {
+		return nil, err
+	}
+	steps := make([]EmptyStep, len(sealedSteps))
+	for i := range sealedSteps {
+		steps[i] = newEmptyStepFromSealed(sealedSteps[i], header.ParentHash)
+	}
+	return steps, nil
+}
+
+func newEmptyStepFromSealed(step SealedEmptyStep, parentHash common.Hash) EmptyStep {
+	return EmptyStep{
+		signature:  step.signature,
+		step:       step.step,
+		parentHash: parentHash,
+	}
+}
+
+// extracts the raw empty steps vec 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 headerEmptyStepsRaw(header *types.Header) []byte {
+	if len(header.Seal) < 3 {
+		panic("was checked with verify_block_basic; has 3 fields; qed")
+	}
+	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
+// proof.
+//
+// An empty step message is created _instead of_ a block if there are no pending transactions.
+// It cannot itself be a parent, and `parent_hash` always points to the most recent block. E.g.:
+// * Validator A creates block `bA`.
+// * Validator B has no pending transactions, so it signs an empty step message `mB`
+//   instead whose hash points to block `bA`.
+// * Validator C also has no pending transactions, so it also signs an empty step message `mC`
+//   instead whose hash points to block `bA`.
+// * Validator D creates block `bD`. The parent is block `bA`, and the header includes `mB` and `mC`.
+type EmptyStep struct {
+	// The signature of the other two fields, by the message's author.
+	signature []byte // H520
+	// This message's step number.
+	step uint64
+	// The hash of the most recent block.
+	parentHash common.Hash //     H256
+}
+
+func (s *EmptyStep) Less(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
+	//sRlp, err := EmptyStepRlp(s.step, s.parentHash)
+	//if err != nil {
+	//	return false, err
+	//}
+	//message := crypto.Keccak256(sRlp)
+
+	/*
+		let correct_proposer = step_proposer(validators, &self.parent_hash, self.step);
+
+		publickey::verify_address(&correct_proposer, &self.signature.into(), &message)
+		.map_err(|e| e.into())
+	*/
+	return true, nil
+}
+
+func (s *EmptyStep) author() (common.Address, error) {
+	sRlp, err := EmptyStepRlp(s.step, s.parentHash)
+	if err != nil {
+		return common.Address{}, err
+	}
+	message := crypto.Keccak256(sRlp)
+	public, err := secp256k1.RecoverPubkey(message, s.signature)
+	if err != nil {
+		return common.Address{}, err
+	}
+	ecdsa, err := crypto.UnmarshalPubkey(public)
+	if err != nil {
+		return common.Address{}, err
+	}
+	return crypto.PubkeyToAddress(*ecdsa), nil
+}
+
+type EmptyStepSet struct {
+	lock sync.Mutex
+	list []*EmptyStep
+}
+
+func (s *EmptyStepSet) Less(i, j int) bool { return s.list[i].Less(s.list[j]) }
+func (s *EmptyStepSet) Swap(i, j int)      { s.list[i], s.list[j] = s.list[j], s.list[i] }
+func (s *EmptyStepSet) Len() int           { return len(s.list) }
+
+func (s *EmptyStepSet) Sort() {
+	s.lock.Lock()
+	defer s.lock.Unlock()
+	sort.Stable(s)
+}
+
+func (s *EmptyStepSet) ForEach(f func(int, *EmptyStep)) {
+	s.lock.Lock()
+	defer s.lock.Unlock()
+	for i, el := range s.list {
+		f(i, el)
+	}
+}
+
+func EmptyStepFullRlp(signature []byte, emptyStepRlp []byte) ([]byte, error) {
+	type A struct {
+		s []byte
+		r []byte
+	}
+
+	return rlp.EncodeToBytes(A{s: signature, r: emptyStepRlp})
+}
+
+func EmptyStepRlp(step uint64, parentHash common.Hash) ([]byte, error) {
+	type A struct {
+		s uint64
+		h common.Hash
+	}
+	return rlp.EncodeToBytes(A{s: step, h: parentHash})
+}
diff --git a/consensus/aura/auraabi/abi.go b/consensus/aura/auraabi/abi.go
new file mode 100644
index 0000000000..8622847f48
--- /dev/null
+++ b/consensus/aura/auraabi/abi.go
@@ -0,0 +1,4 @@
+package auraabi
+
+//go:generate abigen -abi ./../contracts/block_reward.json -pkg auraabi -type block_reward -out ./gen_block_reward.go
+//go:generate abigen -abi ./../contracts/validator_set.json -pkg auraabi -type validator_set -out ./gen_validator_set.go
diff --git a/consensus/aura/auraabi/gen_block_reward.go b/consensus/aura/auraabi/gen_block_reward.go
new file mode 100644
index 0000000000..1bbdf9355d
--- /dev/null
+++ b/consensus/aura/auraabi/gen_block_reward.go
@@ -0,0 +1,193 @@
+// Code generated - DO NOT EDIT.
+// This file is a generated binding and any manual changes will be lost.
+
+package auraabi
+
+import (
+	"math/big"
+	"strings"
+
+	ethereum "github.com/ledgerwatch/erigon"
+	"github.com/ledgerwatch/erigon/accounts/abi"
+	"github.com/ledgerwatch/erigon/accounts/abi/bind"
+	"github.com/ledgerwatch/erigon/common"
+	"github.com/ledgerwatch/erigon/core/types"
+	"github.com/ledgerwatch/erigon/event"
+)
+
+// Reference imports to suppress errors if they are not otherwise used.
+var (
+	_ = big.NewInt
+	_ = strings.NewReader
+	_ = ethereum.NotFound
+	_ = bind.Bind
+	_ = common.Big1
+	_ = types.BloomLookup
+	_ = event.NewSubscription
+)
+
+// BlockRewardABI is the input ABI used to generate the binding from.
+const BlockRewardABI = "[{\"constant\":false,\"inputs\":[{\"name\":\"benefactors\",\"type\":\"address[]\"},{\"name\":\"kind\",\"type\":\"uint16[]\"}],\"name\":\"reward\",\"outputs\":[{\"name\":\"\",\"type\":\"address[]\"},{\"name\":\"\",\"type\":\"uint256[]\"}],\"payable\":false,\"stateMutability\":\"nonpayable\",\"type\":\"function\"}]"
+
+// BlockReward is an auto generated Go binding around an Ethereum contract.
+type BlockReward struct {
+	BlockRewardCaller     // Read-only binding to the contract
+	BlockRewardTransactor // Write-only binding to the contract
+	BlockRewardFilterer   // Log filterer for contract events
+}
+
+// BlockRewardCaller is an auto generated read-only Go binding around an Ethereum contract.
+type BlockRewardCaller struct {
+	contract *bind.BoundContract // Generic contract wrapper for the low level calls
+}
+
+// BlockRewardTransactor is an auto generated write-only Go binding around an Ethereum contract.
+type BlockRewardTransactor struct {
+	contract *bind.BoundContract // Generic contract wrapper for the low level calls
+}
+
+// BlockRewardFilterer is an auto generated log filtering Go binding around an Ethereum contract events.
+type BlockRewardFilterer struct {
+	contract *bind.BoundContract // Generic contract wrapper for the low level calls
+}
+
+// BlockRewardSession is an auto generated Go binding around an Ethereum contract,
+// with pre-set call and transact options.
+type BlockRewardSession struct {
+	Contract     *BlockReward      // Generic contract binding to set the session for
+	CallOpts     bind.CallOpts     // Call options to use throughout this session
+	TransactOpts bind.TransactOpts // Transaction auth options to use throughout this session
+}
+
+// BlockRewardCallerSession is an auto generated read-only Go binding around an Ethereum contract,
+// with pre-set call options.
+type BlockRewardCallerSession struct {
+	Contract *BlockRewardCaller // Generic contract caller binding to set the session for
+	CallOpts bind.CallOpts      // Call options to use throughout this session
+}
+
+// BlockRewardTransactorSession is an auto generated write-only Go binding around an Ethereum contract,
+// with pre-set transact options.
+type BlockRewardTransactorSession struct {
+	Contract     *BlockRewardTransactor // Generic contract transactor binding to set the session for
+	TransactOpts bind.TransactOpts      // Transaction auth options to use throughout this session
+}
+
+// BlockRewardRaw is an auto generated low-level Go binding around an Ethereum contract.
+type BlockRewardRaw struct {
+	Contract *BlockReward // Generic contract binding to access the raw methods on
+}
+
+// BlockRewardCallerRaw is an auto generated low-level read-only Go binding around an Ethereum contract.
+type BlockRewardCallerRaw struct {
+	Contract *BlockRewardCaller // Generic read-only contract binding to access the raw methods on
+}
+
+// BlockRewardTransactorRaw is an auto generated low-level write-only Go binding around an Ethereum contract.
+type BlockRewardTransactorRaw struct {
+	Contract *BlockRewardTransactor // Generic write-only contract binding to access the raw methods on
+}
+
+// NewBlockReward creates a new instance of BlockReward, bound to a specific deployed contract.
+func NewBlockReward(address common.Address, backend bind.ContractBackend) (*BlockReward, error) {
+	contract, err := bindBlockReward(address, backend, backend, backend)
+	if err != nil {
+		return nil, err
+	}
+	return &BlockReward{BlockRewardCaller: BlockRewardCaller{contract: contract}, BlockRewardTransactor: BlockRewardTransactor{contract: contract}, BlockRewardFilterer: BlockRewardFilterer{contract: contract}}, nil
+}
+
+// NewBlockRewardCaller creates a new read-only instance of BlockReward, bound to a specific deployed contract.
+func NewBlockRewardCaller(address common.Address, caller bind.ContractCaller) (*BlockRewardCaller, error) {
+	contract, err := bindBlockReward(address, caller, nil, nil)
+	if err != nil {
+		return nil, err
+	}
+	return &BlockRewardCaller{contract: contract}, nil
+}
+
+// NewBlockRewardTransactor creates a new write-only instance of BlockReward, bound to a specific deployed contract.
+func NewBlockRewardTransactor(address common.Address, transactor bind.ContractTransactor) (*BlockRewardTransactor, error) {
+	contract, err := bindBlockReward(address, nil, transactor, nil)
+	if err != nil {
+		return nil, err
+	}
+	return &BlockRewardTransactor{contract: contract}, nil
+}
+
+// NewBlockRewardFilterer creates a new log filterer instance of BlockReward, bound to a specific deployed contract.
+func NewBlockRewardFilterer(address common.Address, filterer bind.ContractFilterer) (*BlockRewardFilterer, error) {
+	contract, err := bindBlockReward(address, nil, nil, filterer)
+	if err != nil {
+		return nil, err
+	}
+	return &BlockRewardFilterer{contract: contract}, nil
+}
+
+// bindBlockReward binds a generic wrapper to an already deployed contract.
+func bindBlockReward(address common.Address, caller bind.ContractCaller, transactor bind.ContractTransactor, filterer bind.ContractFilterer) (*bind.BoundContract, error) {
+	parsed, err := abi.JSON(strings.NewReader(BlockRewardABI))
+	if err != nil {
+		return nil, err
+	}
+	return bind.NewBoundContract(address, parsed, caller, transactor, filterer), nil
+}
+
+// Call invokes the (constant) contract method with params as input values and
+// sets the output to result. The result type might be a single field for simple
+// returns, a slice of interfaces for anonymous returns and a struct for named
+// returns.
+func (_BlockReward *BlockRewardRaw) Call(opts *bind.CallOpts, result *[]interface{}, method string, params ...interface{}) error {
+	return _BlockReward.Contract.BlockRewardCaller.contract.Call(opts, result, method, params...)
+}
+
+// Transfer initiates a plain transaction to move funds to the contract, calling
+// its default method if one is available.
+func (_BlockReward *BlockRewardRaw) Transfer(opts *bind.TransactOpts) (types.Transaction, error) {
+	return _BlockReward.Contract.BlockRewardTransactor.contract.Transfer(opts)
+}
+
+// Transact invokes the (paid) contract method with params as input values.
+func (_BlockReward *BlockRewardRaw) Transact(opts *bind.TransactOpts, method string, params ...interface{}) (types.Transaction, error) {
+	return _BlockReward.Contract.BlockRewardTransactor.contract.Transact(opts, method, params...)
+}
+
+// Call invokes the (constant) contract method with params as input values and
+// sets the output to result. The result type might be a single field for simple
+// returns, a slice of interfaces for anonymous returns and a struct for named
+// returns.
+func (_BlockReward *BlockRewardCallerRaw) Call(opts *bind.CallOpts, result *[]interface{}, method string, params ...interface{}) error {
+	return _BlockReward.Contract.contract.Call(opts, result, method, params...)
+}
+
+// Transfer initiates a plain transaction to move funds to the contract, calling
+// its default method if one is available.
+func (_BlockReward *BlockRewardTransactorRaw) Transfer(opts *bind.TransactOpts) (types.Transaction, error) {
+	return _BlockReward.Contract.contract.Transfer(opts)
+}
+
+// Transact invokes the (paid) contract method with params as input values.
+func (_BlockReward *BlockRewardTransactorRaw) Transact(opts *bind.TransactOpts, method string, params ...interface{}) (types.Transaction, error) {
+	return _BlockReward.Contract.contract.Transact(opts, method, params...)
+}
+
+// Reward is a paid mutator transaction binding the contract method 0xf91c2898.
+//
+// Solidity: function reward(address[] benefactors, uint16[] kind) returns(address[], uint256[])
+func (_BlockReward *BlockRewardTransactor) Reward(opts *bind.TransactOpts, benefactors []common.Address, kind []uint16) (types.Transaction, error) {
+	return _BlockReward.contract.Transact(opts, "reward", benefactors, kind)
+}
+
+// Reward is a paid mutator transaction binding the contract method 0xf91c2898.
+//
+// Solidity: function reward(address[] benefactors, uint16[] kind) returns(address[], uint256[])
+func (_BlockReward *BlockRewardSession) Reward(benefactors []common.Address, kind []uint16) (types.Transaction, error) {
+	return _BlockReward.Contract.Reward(&_BlockReward.TransactOpts, benefactors, kind)
+}
+
+// Reward is a paid mutator transaction binding the contract method 0xf91c2898.
+//
+// Solidity: function reward(address[] benefactors, uint16[] kind) returns(address[], uint256[])
+func (_BlockReward *BlockRewardTransactorSession) Reward(benefactors []common.Address, kind []uint16) (types.Transaction, error) {
+	return _BlockReward.Contract.Reward(&_BlockReward.TransactOpts, benefactors, kind)
+}
diff --git a/consensus/aura/auraabi/gen_validator_set.go b/consensus/aura/auraabi/gen_validator_set.go
new file mode 100644
index 0000000000..2e1d545804
--- /dev/null
+++ b/consensus/aura/auraabi/gen_validator_set.go
@@ -0,0 +1,452 @@
+// Code generated - DO NOT EDIT.
+// This file is a generated binding and any manual changes will be lost.
+
+package auraabi
+
+import (
+	"math/big"
+	"strings"
+
+	ethereum "github.com/ledgerwatch/erigon"
+	"github.com/ledgerwatch/erigon/accounts/abi"
+	"github.com/ledgerwatch/erigon/accounts/abi/bind"
+	"github.com/ledgerwatch/erigon/common"
+	"github.com/ledgerwatch/erigon/core/types"
+	"github.com/ledgerwatch/erigon/event"
+)
+
+// Reference imports to suppress errors if they are not otherwise used.
+var (
+	_ = big.NewInt
+	_ = strings.NewReader
+	_ = ethereum.NotFound
+	_ = bind.Bind
+	_ = common.Big1
+	_ = types.BloomLookup
+	_ = event.NewSubscription
+)
+
+// ValidatorSetABI is the input ABI used to generate the binding from.
+const ValidatorSetABI = "[{\"constant\":false,\"inputs\":[],\"name\":\"finalizeChange\",\"outputs\":[],\"payable\":false,\"type\":\"function\"},{\"constant\":true,\"inputs\":[],\"name\":\"getValidators\",\"outputs\":[{\"name\":\"validators\",\"type\":\"address[]\"}],\"payable\":false,\"type\":\"function\"},{\"anonymous\":false,\"inputs\":[{\"indexed\":true,\"name\":\"_parent_hash\",\"type\":\"bytes32\"},{\"indexed\":false,\"name\":\"_new_set\",\"type\":\"address[]\"}],\"name\":\"InitiateChange\",\"type\":\"event\"},{\"constant\":true,\"inputs\":[],\"name\":\"emitInitiateChangeCallable\",\"outputs\":[{\"name\":\"\",\"type\":\"bool\"}],\"payable\":false,\"stateMutability\":\"view\",\"type\":\"function\"},{\"constant\":false,\"inputs\":[],\"name\":\"emitInitiateChange\",\"outputs\":[],\"payable\":false,\"stateMutability\":\"nonpayable\",\"type\":\"function\"},{\"constant\":true,\"inputs\":[{\"name\":\"_reportingValidator\",\"type\":\"address\"},{\"name\":\"_maliciousValidator\",\"type\":\"address\"},{\"name\":\"_blockNumber\",\"type\":\"uint256\"}],\"name\":\"shouldValidatorReport\",\"outputs\":[{\"name\":\"\",\"type\":\"bool\"}],\"payable\":false,\"stateMutability\":\"view\",\"type\":\"function\"}]"
+
+// ValidatorSet is an auto generated Go binding around an Ethereum contract.
+type ValidatorSet struct {
+	ValidatorSetCaller     // Read-only binding to the contract
+	ValidatorSetTransactor // Write-only binding to the contract
+	ValidatorSetFilterer   // Log filterer for contract events
+}
+
+// ValidatorSetCaller is an auto generated read-only Go binding around an Ethereum contract.
+type ValidatorSetCaller struct {
+	contract *bind.BoundContract // Generic contract wrapper for the low level calls
+}
+
+// ValidatorSetTransactor is an auto generated write-only Go binding around an Ethereum contract.
+type ValidatorSetTransactor struct {
+	contract *bind.BoundContract // Generic contract wrapper for the low level calls
+}
+
+// ValidatorSetFilterer is an auto generated log filtering Go binding around an Ethereum contract events.
+type ValidatorSetFilterer struct {
+	contract *bind.BoundContract // Generic contract wrapper for the low level calls
+}
+
+// ValidatorSetSession is an auto generated Go binding around an Ethereum contract,
+// with pre-set call and transact options.
+type ValidatorSetSession struct {
+	Contract     *ValidatorSet     // Generic contract binding to set the session for
+	CallOpts     bind.CallOpts     // Call options to use throughout this session
+	TransactOpts bind.TransactOpts // Transaction auth options to use throughout this session
+}
+
+// ValidatorSetCallerSession is an auto generated read-only Go binding around an Ethereum contract,
+// with pre-set call options.
+type ValidatorSetCallerSession struct {
+	Contract *ValidatorSetCaller // Generic contract caller binding to set the session for
+	CallOpts bind.CallOpts       // Call options to use throughout this session
+}
+
+// ValidatorSetTransactorSession is an auto generated write-only Go binding around an Ethereum contract,
+// with pre-set transact options.
+type ValidatorSetTransactorSession struct {
+	Contract     *ValidatorSetTransactor // Generic contract transactor binding to set the session for
+	TransactOpts bind.TransactOpts       // Transaction auth options to use throughout this session
+}
+
+// ValidatorSetRaw is an auto generated low-level Go binding around an Ethereum contract.
+type ValidatorSetRaw struct {
+	Contract *ValidatorSet // Generic contract binding to access the raw methods on
+}
+
+// ValidatorSetCallerRaw is an auto generated low-level read-only Go binding around an Ethereum contract.
+type ValidatorSetCallerRaw struct {
+	Contract *ValidatorSetCaller // Generic read-only contract binding to access the raw methods on
+}
+
+// ValidatorSetTransactorRaw is an auto generated low-level write-only Go binding around an Ethereum contract.
+type ValidatorSetTransactorRaw struct {
+	Contract *ValidatorSetTransactor // Generic write-only contract binding to access the raw methods on
+}
+
+// NewValidatorSet creates a new instance of ValidatorSet, bound to a specific deployed contract.
+func NewValidatorSet(address common.Address, backend bind.ContractBackend) (*ValidatorSet, error) {
+	contract, err := bindValidatorSet(address, backend, backend, backend)
+	if err != nil {
+		return nil, err
+	}
+	return &ValidatorSet{ValidatorSetCaller: ValidatorSetCaller{contract: contract}, ValidatorSetTransactor: ValidatorSetTransactor{contract: contract}, ValidatorSetFilterer: ValidatorSetFilterer{contract: contract}}, nil
+}
+
+// NewValidatorSetCaller creates a new read-only instance of ValidatorSet, bound to a specific deployed contract.
+func NewValidatorSetCaller(address common.Address, caller bind.ContractCaller) (*ValidatorSetCaller, error) {
+	contract, err := bindValidatorSet(address, caller, nil, nil)
+	if err != nil {
+		return nil, err
+	}
+	return &ValidatorSetCaller{contract: contract}, nil
+}
+
+// NewValidatorSetTransactor creates a new write-only instance of ValidatorSet, bound to a specific deployed contract.
+func NewValidatorSetTransactor(address common.Address, transactor bind.ContractTransactor) (*ValidatorSetTransactor, error) {
+	contract, err := bindValidatorSet(address, nil, transactor, nil)
+	if err != nil {
+		return nil, err
+	}
+	return &ValidatorSetTransactor{contract: contract}, nil
+}
+
+// NewValidatorSetFilterer creates a new log filterer instance of ValidatorSet, bound to a specific deployed contract.
+func NewValidatorSetFilterer(address common.Address, filterer bind.ContractFilterer) (*ValidatorSetFilterer, error) {
+	contract, err := bindValidatorSet(address, nil, nil, filterer)
+	if err != nil {
+		return nil, err
+	}
+	return &ValidatorSetFilterer{contract: contract}, nil
+}
+
+// bindValidatorSet binds a generic wrapper to an already deployed contract.
+func bindValidatorSet(address common.Address, caller bind.ContractCaller, transactor bind.ContractTransactor, filterer bind.ContractFilterer) (*bind.BoundContract, error) {
+	parsed, err := abi.JSON(strings.NewReader(ValidatorSetABI))
+	if err != nil {
+		return nil, err
+	}
+	return bind.NewBoundContract(address, parsed, caller, transactor, filterer), nil
+}
+
+// Call invokes the (constant) contract method with params as input values and
+// sets the output to result. The result type might be a single field for simple
+// returns, a slice of interfaces for anonymous returns and a struct for named
+// returns.
+func (_ValidatorSet *ValidatorSetRaw) Call(opts *bind.CallOpts, result *[]interface{}, method string, params ...interface{}) error {
+	return _ValidatorSet.Contract.ValidatorSetCaller.contract.Call(opts, result, method, params...)
+}
+
+// Transfer initiates a plain transaction to move funds to the contract, calling
+// its default method if one is available.
+func (_ValidatorSet *ValidatorSetRaw) Transfer(opts *bind.TransactOpts) (types.Transaction, error) {
+	return _ValidatorSet.Contract.ValidatorSetTransactor.contract.Transfer(opts)
+}
+
+// Transact invokes the (paid) contract method with params as input values.
+func (_ValidatorSet *ValidatorSetRaw) Transact(opts *bind.TransactOpts, method string, params ...interface{}) (types.Transaction, error) {
+	return _ValidatorSet.Contract.ValidatorSetTransactor.contract.Transact(opts, method, params...)
+}
+
+// Call invokes the (constant) contract method with params as input values and
+// sets the output to result. The result type might be a single field for simple
+// returns, a slice of interfaces for anonymous returns and a struct for named
+// returns.
+func (_ValidatorSet *ValidatorSetCallerRaw) Call(opts *bind.CallOpts, result *[]interface{}, method string, params ...interface{}) error {
+	return _ValidatorSet.Contract.contract.Call(opts, result, method, params...)
+}
+
+// Transfer initiates a plain transaction to move funds to the contract, calling
+// its default method if one is available.
+func (_ValidatorSet *ValidatorSetTransactorRaw) Transfer(opts *bind.TransactOpts) (types.Transaction, error) {
+	return _ValidatorSet.Contract.contract.Transfer(opts)
+}
+
+// Transact invokes the (paid) contract method with params as input values.
+func (_ValidatorSet *ValidatorSetTransactorRaw) Transact(opts *bind.TransactOpts, method string, params ...interface{}) (types.Transaction, error) {
+	return _ValidatorSet.Contract.contract.Transact(opts, method, params...)
+}
+
+// EmitInitiateChangeCallable is a free data retrieval call binding the contract method 0x3d3b5458.
+//
+// Solidity: function emitInitiateChangeCallable() view returns(bool)
+func (_ValidatorSet *ValidatorSetCaller) EmitInitiateChangeCallable(opts *bind.CallOpts) (bool, error) {
+	var out []interface{}
+	err := _ValidatorSet.contract.Call(opts, &out, "emitInitiateChangeCallable")
+
+	if err != nil {
+		return *new(bool), err
+	}
+
+	out0 := *abi.ConvertType(out[0], new(bool)).(*bool)
+
+	return out0, err
+
+}
+
+// EmitInitiateChangeCallable is a free data retrieval call binding the contract method 0x3d3b5458.
+//
+// Solidity: function emitInitiateChangeCallable() view returns(bool)
+func (_ValidatorSet *ValidatorSetSession) EmitInitiateChangeCallable() (bool, error) {
+	return _ValidatorSet.Contract.EmitInitiateChangeCallable(&_ValidatorSet.CallOpts)
+}
+
+// EmitInitiateChangeCallable is a free data retrieval call binding the contract method 0x3d3b5458.
+//
+// Solidity: function emitInitiateChangeCallable() view returns(bool)
+func (_ValidatorSet *ValidatorSetCallerSession) EmitInitiateChangeCallable() (bool, error) {
+	return _ValidatorSet.Contract.EmitInitiateChangeCallable(&_ValidatorSet.CallOpts)
+}
+
+// GetValidators is a free data retrieval call binding the contract method 0xb7ab4db5.
+//
+// Solidity: function getValidators() returns(address[] validators)
+func (_ValidatorSet *ValidatorSetCaller) GetValidators(opts *bind.CallOpts) ([]common.Address, error) {
+	var out []interface{}
+	err := _ValidatorSet.contract.Call(opts, &out, "getValidators")
+
+	if err != nil {
+		return *new([]common.Address), err
+	}
+
+	out0 := *abi.ConvertType(out[0], new([]common.Address)).(*[]common.Address)
+
+	return out0, err
+
+}
+
+// GetValidators is a free data retrieval call binding the contract method 0xb7ab4db5.
+//
+// Solidity: function getValidators() returns(address[] validators)
+func (_ValidatorSet *ValidatorSetSession) GetValidators() ([]common.Address, error) {
+	return _ValidatorSet.Contract.GetValidators(&_ValidatorSet.CallOpts)
+}
+
+// GetValidators is a free data retrieval call binding the contract method 0xb7ab4db5.
+//
+// Solidity: function getValidators() returns(address[] validators)
+func (_ValidatorSet *ValidatorSetCallerSession) GetValidators() ([]common.Address, error) {
+	return _ValidatorSet.Contract.GetValidators(&_ValidatorSet.CallOpts)
+}
+
+// ShouldValidatorReport is a free data retrieval call binding the contract method 0xcbd2d528.
+//
+// Solidity: function shouldValidatorReport(address _reportingValidator, address _maliciousValidator, uint256 _blockNumber) view returns(bool)
+func (_ValidatorSet *ValidatorSetCaller) ShouldValidatorReport(opts *bind.CallOpts, _reportingValidator common.Address, _maliciousValidator common.Address, _blockNumber *big.Int) (bool, error) {
+	var out []interface{}
+	err := _ValidatorSet.contract.Call(opts, &out, "shouldValidatorReport", _reportingValidator, _maliciousValidator, _blockNumber)
+
+	if err != nil {
+		return *new(bool), err
+	}
+
+	out0 := *abi.ConvertType(out[0], new(bool)).(*bool)
+
+	return out0, err
+
+}
+
+// ShouldValidatorReport is a free data retrieval call binding the contract method 0xcbd2d528.
+//
+// Solidity: function shouldValidatorReport(address _reportingValidator, address _maliciousValidator, uint256 _blockNumber) view returns(bool)
+func (_ValidatorSet *ValidatorSetSession) ShouldValidatorReport(_reportingValidator common.Address, _maliciousValidator common.Address, _blockNumber *big.Int) (bool, error) {
+	return _ValidatorSet.Contract.ShouldValidatorReport(&_ValidatorSet.CallOpts, _reportingValidator, _maliciousValidator, _blockNumber)
+}
+
+// ShouldValidatorReport is a free data retrieval call binding the contract method 0xcbd2d528.
+//
+// Solidity: function shouldValidatorReport(address _reportingValidator, address _maliciousValidator, uint256 _blockNumber) view returns(bool)
+func (_ValidatorSet *ValidatorSetCallerSession) ShouldValidatorReport(_reportingValidator common.Address, _maliciousValidator common.Address, _blockNumber *big.Int) (bool, error) {
+	return _ValidatorSet.Contract.ShouldValidatorReport(&_ValidatorSet.CallOpts, _reportingValidator, _maliciousValidator, _blockNumber)
+}
+
+// EmitInitiateChange is a paid mutator transaction binding the contract method 0x93b4e25e.
+//
+// Solidity: function emitInitiateChange() returns()
+func (_ValidatorSet *ValidatorSetTransactor) EmitInitiateChange(opts *bind.TransactOpts) (types.Transaction, error) {
+	return _ValidatorSet.contract.Transact(opts, "emitInitiateChange")
+}
+
+// EmitInitiateChange is a paid mutator transaction binding the contract method 0x93b4e25e.
+//
+// Solidity: function emitInitiateChange() returns()
+func (_ValidatorSet *ValidatorSetSession) EmitInitiateChange() (types.Transaction, error) {
+	return _ValidatorSet.Contract.EmitInitiateChange(&_ValidatorSet.TransactOpts)
+}
+
+// EmitInitiateChange is a paid mutator transaction binding the contract method 0x93b4e25e.
+//
+// Solidity: function emitInitiateChange() returns()
+func (_ValidatorSet *ValidatorSetTransactorSession) EmitInitiateChange() (types.Transaction, error) {
+	return _ValidatorSet.Contract.EmitInitiateChange(&_ValidatorSet.TransactOpts)
+}
+
+// FinalizeChange is a paid mutator transaction binding the contract method 0x75286211.
+//
+// Solidity: function finalizeChange() returns()
+func (_ValidatorSet *ValidatorSetTransactor) FinalizeChange(opts *bind.TransactOpts) (types.Transaction, error) {
+	return _ValidatorSet.contract.Transact(opts, "finalizeChange")
+}
+
+// FinalizeChange is a paid mutator transaction binding the contract method 0x75286211.
+//
+// Solidity: function finalizeChange() returns()
+func (_ValidatorSet *ValidatorSetSession) FinalizeChange() (types.Transaction, error) {
+	return _ValidatorSet.Contract.FinalizeChange(&_ValidatorSet.TransactOpts)
+}
+
+// FinalizeChange is a paid mutator transaction binding the contract method 0x75286211.
+//
+// Solidity: function finalizeChange() returns()
+func (_ValidatorSet *ValidatorSetTransactorSession) FinalizeChange() (types.Transaction, error) {
+	return _ValidatorSet.Contract.FinalizeChange(&_ValidatorSet.TransactOpts)
+}
+
+// ValidatorSetInitiateChangeIterator is returned from FilterInitiateChange and is used to iterate over the raw logs and unpacked data for InitiateChange events raised by the ValidatorSet contract.
+type ValidatorSetInitiateChangeIterator struct {
+	Event *ValidatorSetInitiateChange // Event containing the contract specifics and raw log
+
+	contract *bind.BoundContract // Generic contract to use for unpacking event data
+	event    string              // Event name to use for unpacking event data
+
+	logs chan types.Log        // Log channel receiving the found contract events
+	sub  ethereum.Subscription // Subscription for errors, completion and termination
+	done bool                  // Whether the subscription completed delivering logs
+	fail error                 // Occurred error to stop iteration
+}
+
+// Next advances the iterator to the subsequent event, returning whether there
+// are any more events found. In case of a retrieval or parsing error, false is
+// returned and Error() can be queried for the exact failure.
+func (it *ValidatorSetInitiateChangeIterator) Next() bool {
+	// If the iterator failed, stop iterating
+	if it.fail != nil {
+		return false
+	}
+	// If the iterator completed, deliver directly whatever's available
+	if it.done {
+		select {
+		case log := <-it.logs:
+			it.Event = new(ValidatorSetInitiateChange)
+			if err := it.contract.UnpackLog(it.Event, it.event, log); err != nil {
+				it.fail = err
+				return false
+			}
+			it.Event.Raw = log
+			return true
+
+		default:
+			return false
+		}
+	}
+	// Iterator still in progress, wait for either a data or an error event
+	select {
+	case log := <-it.logs:
+		it.Event = new(ValidatorSetInitiateChange)
+		if err := it.contract.UnpackLog(it.Event, it.event, log); err != nil {
+			it.fail = err
+			return false
+		}
+		it.Event.Raw = log
+		return true
+
+	case err := <-it.sub.Err():
+		it.done = true
+		it.fail = err
+		return it.Next()
+	}
+}
+
+// Error returns any retrieval or parsing error occurred during filtering.
+func (it *ValidatorSetInitiateChangeIterator) Error() error {
+	return it.fail
+}
+
+// Close terminates the iteration process, releasing any pending underlying
+// resources.
+func (it *ValidatorSetInitiateChangeIterator) Close() error {
+	it.sub.Unsubscribe()
+	return nil
+}
+
+// ValidatorSetInitiateChange represents a InitiateChange event raised by the ValidatorSet contract.
+type ValidatorSetInitiateChange struct {
+	ParentHash [32]byte
+	NewSet     []common.Address
+	Raw        types.Log // Blockchain specific contextual infos
+}
+
+// FilterInitiateChange is a free log retrieval operation binding the contract event 0x55252fa6eee4741b4e24a74a70e9c11fd2c2281df8d6ea13126ff845f7825c89.
+//
+// Solidity: event InitiateChange(bytes32 indexed _parent_hash, address[] _new_set)
+func (_ValidatorSet *ValidatorSetFilterer) FilterInitiateChange(opts *bind.FilterOpts, _parent_hash [][32]byte) (*ValidatorSetInitiateChangeIterator, error) {
+
+	var _parent_hashRule []interface{}
+	for _, _parent_hashItem := range _parent_hash {
+		_parent_hashRule = append(_parent_hashRule, _parent_hashItem)
+	}
+
+	logs, sub, err := _ValidatorSet.contract.FilterLogs(opts, "InitiateChange", _parent_hashRule)
+	if err != nil {
+		return nil, err
+	}
+	return &ValidatorSetInitiateChangeIterator{contract: _ValidatorSet.contract, event: "InitiateChange", logs: logs, sub: sub}, nil
+}
+
+// WatchInitiateChange is a free log subscription operation binding the contract event 0x55252fa6eee4741b4e24a74a70e9c11fd2c2281df8d6ea13126ff845f7825c89.
+//
+// Solidity: event InitiateChange(bytes32 indexed _parent_hash, address[] _new_set)
+func (_ValidatorSet *ValidatorSetFilterer) WatchInitiateChange(opts *bind.WatchOpts, sink chan<- *ValidatorSetInitiateChange, _parent_hash [][32]byte) (event.Subscription, error) {
+
+	var _parent_hashRule []interface{}
+	for _, _parent_hashItem := range _parent_hash {
+		_parent_hashRule = append(_parent_hashRule, _parent_hashItem)
+	}
+
+	logs, sub, err := _ValidatorSet.contract.WatchLogs(opts, "InitiateChange", _parent_hashRule)
+	if err != nil {
+		return nil, err
+	}
+	return event.NewSubscription(func(quit <-chan struct{}) error {
+		defer sub.Unsubscribe()
+		for {
+			select {
+			case log := <-logs:
+				// New log arrived, parse the event and forward to the user
+				event := new(ValidatorSetInitiateChange)
+				if err := _ValidatorSet.contract.UnpackLog(event, "InitiateChange", log); err != nil {
+					return err
+				}
+				event.Raw = log
+
+				select {
+				case sink <- event:
+				case err := <-sub.Err():
+					return err
+				case <-quit:
+					return nil
+				}
+			case err := <-sub.Err():
+				return err
+			case <-quit:
+				return nil
+			}
+		}
+	}), nil
+}
+
+// ParseInitiateChange is a log parse operation binding the contract event 0x55252fa6eee4741b4e24a74a70e9c11fd2c2281df8d6ea13126ff845f7825c89.
+//
+// Solidity: event InitiateChange(bytes32 indexed _parent_hash, address[] _new_set)
+func (_ValidatorSet *ValidatorSetFilterer) ParseInitiateChange(log types.Log) (*ValidatorSetInitiateChange, error) {
+	event := new(ValidatorSetInitiateChange)
+	if err := _ValidatorSet.contract.UnpackLog(event, "InitiateChange", log); err != nil {
+		return nil, err
+	}
+	event.Raw = log
+	return event, nil
+}
diff --git a/consensus/aura/aurainterfaces/interface.go b/consensus/aura/aurainterfaces/interface.go
new file mode 100644
index 0000000000..367e346518
--- /dev/null
+++ b/consensus/aura/aurainterfaces/interface.go
@@ -0,0 +1,58 @@
+package aurainterfaces
+
+import (
+	"github.com/holiman/uint256"
+	"github.com/ledgerwatch/erigon/common"
+)
+
+// see openethereum/crates/ethcore/res/contracts/block_reward.json
+type BlockRewardABI interface {
+	Reward(benefactors []common.Address, kind []RewardKind) ([]common.Address, []*uint256.Int, error)
+}
+
+type abiDecoder func([]byte, interface{}) error
+
+// see openethereum/crates/ethcore/res/contracts/validator_set.json
+type ValidatorSetABI interface {
+	GetValidators() ([]byte, abiDecoder)
+	ShouldValidatorReport(ourAddr, maliciousValidatorAddress common.Address, blockNum uint64) ([]byte, abiDecoder)
+}
+
+// RewardKind - The kind of block reward.
+// Depending on the consensus engine the allocated block reward might have
+// different semantics which could lead e.g. to different reward values.
+type RewardKind uint16
+
+const (
+	// RewardAuthor - attributed to the block author.
+	RewardAuthor RewardKind = 0
+	// RewardEmptyStep - attributed to the author(s) of empty step(s) included in the block (AuthorityRound engine).
+	RewardEmptyStep RewardKind = 1
+	// RewardExternal - attributed by an external protocol (e.g. block reward contract).
+	RewardExternal RewardKind = 2
+	// RewardUncle - attributed to the block uncle(s) with given difference.
+	RewardUncle RewardKind = 3
+)
+
+type SealKind [][]byte
+
+// Proposal seal; should be broadcasted, but not inserted into blockchain.
+type SealProposal SealKind
+
+// Regular block seal; should be part of the blockchain.
+type SealRegular SealKind
+
+// Engine does not generate seal for this block right now.
+type None SealKind
+
+/// The type of sealing the engine is currently able to perform.
+type SealingState uint8
+
+const (
+	/// The engine is ready to seal a block.
+	SealingStateReady SealingState = 0
+	/// The engine can't seal at the moment, and no block should be prepared and queued.
+	SealingStateNotReady SealingState = 1
+	/// The engine does not seal internally.
+	SealingStateExternal SealingState = 2
+)
diff --git a/consensus/aura/config.go b/consensus/aura/config.go
new file mode 100644
index 0000000000..f5b733f275
--- /dev/null
+++ b/consensus/aura/config.go
@@ -0,0 +1,292 @@
+// Copyright 2017 The go-ethereum Authors
+// This file is part of the go-ethereum library.
+//
+// The go-ethereum library is free software: you can redistribute it and/or modify
+// it under the terms of the GNU Lesser General Public License as published by
+// the Free Software Foundation, either version 3 of the License, or
+// (at your option) any later version.
+//
+// The go-ethereum library is distributed in the hope that it will be useful,
+// but WITHOUT ANY WARRANTY; without even the implied warranty of
+// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+// GNU Lesser General Public License for more details.
+//
+// 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 (
+	"sort"
+
+	"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"
+)
+
+// Draws an validator nonce modulo number of validators.
+func GetFromValidatorSet(set ValidatorSet, parent common.Hash, nonce uint) (common.Address, error) {
+	d, err := set.defaultCaller(parent)
+	if err != nil {
+		return common.Address{}, err
+	}
+	return set.getWithCaller(parent, nonce, d)
+}
+
+// Different ways of specifying validators.
+type ValidatorSetJson struct {
+	// A simple list of authorities.
+	List []common.Address `json:"list"`
+	// Address of a contract that indicates the list of authorities.
+	SafeContract *common.Address `json:"safeContract"`
+	// Address of a contract that indicates the list of authorities and enables reporting of theor misbehaviour using transactions.
+	Contract *common.Address `json:"contract"`
+	// A map of starting blocks for each validator set.
+	Multi map[uint64]*ValidatorSetJson `json:"multi"`
+}
+
+func newValidatorSetFromJson(j *ValidatorSetJson, posdaoTransition *uint64) ValidatorSet {
+	if j.List != nil {
+		return &SimpleList{validators: j.List}
+	}
+	if j.SafeContract != nil {
+		return &ValidatorSafeContract{contractAddress: *j.SafeContract}
+	}
+	if j.Contract != nil {
+		return &ValidatorContract{contractAddress: *j.SafeContract, posdaoTransition: posdaoTransition}
+	}
+	if j.Multi != nil {
+		l := map[uint64]ValidatorSet{}
+		for block, set := range j.Multi {
+			l[block] = newValidatorSetFromJson(set, posdaoTransition)
+		}
+		return NewMulti(l)
+	}
+
+	return nil
+}
+
+//TODO: StepDuration and BlockReward - now are uint64, but it can be an object in non-sokol consensus
+type JsonSpec struct {
+	StepDuration *uint64           `json:"stepDuration"` // Block duration, in seconds.
+	Validators   *ValidatorSetJson `json:"validators"`   // Valid authorities
+
+	// Starting step. Determined automatically if not specified.
+	// To be used for testing only.
+	StartStep               *uint64         `json:"startStep"`
+	ValidateScoreTransition *uint64         `json:"validateScoreTransition"` // Block at which score validation should start.
+	ValidateStepTransition  *uint64         `json:"validateStepTransition"`  // Block from which monotonic steps start.
+	ImmediateTransitions    *bool           `json:"immediateTransitions"`    // Whether transitions should be immediate.
+	BlockReward             *hexutil.Uint64 `json:"blockReward"`             // Reward per block in wei.
+	// Block at which the block reward contract should start being used. This option allows one to
+	// add a single block reward contract transition and is compatible with the multiple address
+	// option `block_reward_contract_transitions` below.
+	BlockRewardContractTransition *uint64 `json:"blockRewardContractTransition"`
+	/// Block reward contract address which overrides the `block_reward` setting. This option allows
+	/// one to add a single block reward contract address and is compatible with the multiple
+	/// address option `block_reward_contract_transitions` below.
+	BlockRewardContractAddress *common.Address `json:"blockRewardContractAddress"`
+	// Block reward contract addresses with their associated starting block numbers.
+	//
+	// Setting the block reward contract overrides `block_reward`. If the single block reward
+	// contract address is also present then it is added into the map at the block number stored in
+	// `block_reward_contract_transition` or 0 if that block number is not provided. Therefore both
+	// a single block reward contract transition and a map of reward contract transitions can be
+	// used simulataneously in the same configuration. In such a case the code requires that the
+	// block number of the single transition is strictly less than any of the block numbers in the
+	// map.
+	BlockRewardContractTransitions map[uint]common.Address `json:"blockRewardContractTransitions"`
+	/// Block reward code. This overrides the block reward contract address.
+	BlockRewardContractCode []byte `json:"blockRewardContractCode"`
+	// Block at which maximum uncle count should be considered.
+	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
+	// associated with each of those contracts.
+	BlockGasLimitContractTransitions map[uint64]common.Address `json:"blockGasLimitContractTransitions"`
+	// The block number at which the consensus engine switches from AuRa to AuRa with POSDAO
+	// modifications.
+	PosdaoTransition *uint64 `json:"PosdaoTransition"`
+}
+
+type Code struct {
+	Code     []byte
+	CodeHash common.Hash
+}
+
+type BlockRewardContract struct {
+	BlockNum uint64
+	Address  common.Address // On-chain address.
+}
+
+func NewBlockRewardContract(address common.Address) *BlockRewardContract {
+	return &BlockRewardContract{Address: address}
+}
+
+type BlockRewardContractList []*BlockRewardContract
+
+func (r BlockRewardContractList) Less(i, j int) bool { return r[i].BlockNum < r[j].BlockNum }
+func (r BlockRewardContractList) Len() int           { return len(r) }
+func (r BlockRewardContractList) Swap(i, j int)      { r[i], r[j] = r[j], r[i] }
+func (r BlockRewardContractList) GreaterOrEqual(block uint64) *BlockRewardContract {
+	return r[sort.Search(len(r), func(i int) bool { return block >= r[i].BlockNum })-1]
+}
+
+type AuthorityRoundParams struct {
+	// A map defining intervals of blocks with the given times (in seconds) to wait before next
+	// block or authority switching. The keys in the map are steps of starting blocks of those
+	// periods. The entry at `0` should be defined.
+	//
+	// Wait times (durations) are additionally required to be less than 65535 since larger values
+	// lead to slow block issuance.
+	StepDurations map[uint64]uint64
+	// Starting step,
+	StartStep *uint64
+	// Valid validators.
+	Validators ValidatorSet
+	// Chain score validation transition block.
+	ValidateScoreTransition uint64
+	// Monotonic step validation transition block.
+	ValidateStepTransition uint64
+	// Immediate transitions.
+	ImmediateTransitions bool
+	// Block reward in base units.
+	BlockReward map[uint64]*uint256.Int
+	// Block reward contract addresses with their associated starting block numbers.
+	BlockRewardContractTransitions BlockRewardContractList
+	// Number of accepted uncles transition block.
+	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.
+	RandomnessContractAddress map[uint64]common.Address
+	// The addresses of contracts that determine the block gas limit with their associated block
+	// numbers.
+	BlockGasLimitContractTransitions map[uint64]common.Address
+	// If set, this is the block number at which the consensus engine switches from AuRa to AuRa
+	// with POSDAO modifications.
+	PosdaoTransition *uint64
+}
+
+func FromJson(jsonParams JsonSpec) (AuthorityRoundParams, error) {
+	params := AuthorityRoundParams{
+		Validators:                       newValidatorSetFromJson(jsonParams.Validators, jsonParams.PosdaoTransition),
+		StartStep:                        jsonParams.StartStep,
+		RandomnessContractAddress:        jsonParams.RandomnessContractAddress,
+		BlockGasLimitContractTransitions: jsonParams.BlockGasLimitContractTransitions,
+		PosdaoTransition:                 jsonParams.PosdaoTransition,
+	}
+	params.StepDurations = map[uint64]uint64{}
+	if jsonParams.StepDuration != nil {
+		params.StepDurations[0] = *jsonParams.StepDuration
+	}
+
+	//TODO: jsonParams.BlockRewardContractTransitions
+	/*
+			   let mut br_transitions: BTreeMap<_, _> = p
+		           .block_reward_contract_transitions
+		           .unwrap_or_default()
+		           .into_iter()
+		           .map(|(block_num, address)| {
+		               (
+		                   block_num.into(),
+		                   BlockRewardContract::new_from_address(address.into()),
+		               )
+		           })
+		           .collect();
+	*/
+
+	transitionBlockNum := uint64(0)
+	if jsonParams.BlockRewardContractTransition != nil {
+		transitionBlockNum = *jsonParams.BlockRewardContractTransition
+	}
+	/*
+	   if (p.block_reward_contract_code.is_some() || p.block_reward_contract_address.is_some())
+	        && br_transitions
+	            .keys()
+	            .next()
+	            .map_or(false, |&block_num| block_num <= transition_block_num)
+	    {
+	        let s = "blockRewardContractTransition";
+	        panic!("{} should be less than any of the keys in {}s", s, s);
+	    }
+	*/
+	if jsonParams.BlockRewardContractCode != nil {
+		/* TODO: support hard-coded reward contract
+		    br_transitions.insert(
+		       transition_block_num,
+		       BlockRewardContract::new_from_code(Arc::new(code.into())),
+		   );
+		*/
+	} else if jsonParams.BlockRewardContractAddress != nil {
+		params.BlockRewardContractTransitions = append(params.BlockRewardContractTransitions, &BlockRewardContract{BlockNum: transitionBlockNum, Address: *jsonParams.BlockRewardContractAddress})
+	}
+
+	if jsonParams.ValidateScoreTransition != nil {
+		params.ValidateScoreTransition = *jsonParams.ValidateScoreTransition
+	}
+	if jsonParams.ValidateStepTransition != nil {
+		params.ValidateStepTransition = *jsonParams.ValidateStepTransition
+	}
+	if jsonParams.ImmediateTransitions != nil {
+		params.ImmediateTransitions = *jsonParams.ImmediateTransitions
+	}
+	if jsonParams.MaximumUncleCount != nil {
+		params.MaximumUncleCount = *jsonParams.MaximumUncleCount
+	}
+	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 {
+		params.BlockReward[0] = u256.Num0
+	} else {
+		if jsonParams.BlockReward != nil {
+			params.BlockReward[0] = uint256.NewInt(uint64(*jsonParams.BlockReward))
+		}
+	}
+
+	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/consensusconfig/embed.go b/consensus/aura/consensusconfig/embed.go
new file mode 100644
index 0000000000..2ab017dd74
--- /dev/null
+++ b/consensus/aura/consensusconfig/embed.go
@@ -0,0 +1,8 @@
+package consensusconfig
+
+import (
+	_ "embed"
+)
+
+//go:embed poasokol.json
+var Sokol []byte
diff --git a/consensus/aura/consensusconfig/poasokol.json b/consensus/aura/consensusconfig/poasokol.json
new file mode 100644
index 0000000000..0c8775312a
--- /dev/null
+++ b/consensus/aura/consensusconfig/poasokol.json
@@ -0,0 +1,28 @@
+{
+  "stepDuration": 5,
+  "blockReward": "0xDE0B6B3A7640000",
+  "maximumUncleCountTransition": 0,
+  "maximumUncleCount": 0,
+  "validators": {
+    "multi": {
+      "0": {
+        "safeContract": "0x8bf38d4764929064f2d4d3a56520a76ab3df415b"
+      },
+      "362296": {
+        "safeContract": "0xf5cE3f5D0366D6ec551C74CCb1F67e91c56F2e34"
+      },
+      "509355": {
+        "safeContract": "0x03048F666359CFD3C74a1A5b9a97848BF71d5038"
+      },
+      "4622420": {
+        "safeContract": "0x4c6a159659CCcb033F4b2e2Be0C16ACC62b89DDB"
+      }
+    }
+  },
+  "blockRewardContractAddress": "0x3145197AD50D7083D0222DE4fCCf67d9BD05C30D",
+  "blockRewardContractTransition": 4639000,
+  "randomnessContractAddress": {
+    "13391641": "0x8f2b78169B0970F11a762e56659Db52B59CBCf1B"
+  }
+}
+
diff --git a/consensus/aura/contracts/authority_round_random.json b/consensus/aura/contracts/authority_round_random.json
new file mode 100644
index 0000000000..90766c93ef
--- /dev/null
+++ b/consensus/aura/contracts/authority_round_random.json
@@ -0,0 +1,149 @@
+[
+  {
+    "constant": false,
+    "inputs": [
+      {
+        "name": "_secretHash",
+        "type": "bytes32"
+      },
+      {
+        "name": "_cipher",
+        "type": "bytes"
+      }
+    ],
+    "name": "commitHash",
+    "outputs": [],
+    "payable": false,
+    "stateMutability": "nonpayable",
+    "type": "function"
+  },
+  {
+    "constant": false,
+    "inputs": [
+      {
+        "name": "_number",
+        "type": "uint256"
+      }
+    ],
+    "name": "revealNumber",
+    "outputs": [],
+    "payable": false,
+    "stateMutability": "nonpayable",
+    "type": "function"
+  },
+  {
+    "constant": true,
+    "inputs": [],
+    "name": "currentCollectRound",
+    "outputs": [
+      {
+        "name": "",
+        "type": "uint256"
+      }
+    ],
+    "payable": false,
+    "stateMutability": "view",
+    "type": "function"
+  },
+  {
+    "constant": true,
+    "inputs": [
+      {
+        "name": "_collectRound",
+        "type": "uint256"
+      },
+      {
+        "name": "_miningAddress",
+        "type": "address"
+      }
+    ],
+    "name": "getCommitAndCipher",
+    "outputs": [
+      {
+        "name": "",
+        "type": "bytes32"
+      },
+      {
+        "name": "",
+        "type": "bytes"
+      }
+    ],
+    "payable": false,
+    "stateMutability": "view",
+    "type": "function"
+  },
+  {
+    "constant": true,
+    "inputs": [
+      {
+        "name": "_collectRound",
+        "type": "uint256"
+      },
+      {
+        "name": "_validator",
+        "type": "address"
+      }
+    ],
+    "name": "isCommitted",
+    "outputs": [
+      {
+        "name": "",
+        "type": "bool"
+      }
+    ],
+    "payable": false,
+    "stateMutability": "view",
+    "type": "function"
+  },
+  {
+    "constant": true,
+    "inputs": [],
+    "name": "isCommitPhase",
+    "outputs": [
+      {
+        "name": "",
+        "type": "bool"
+      }
+    ],
+    "payable": false,
+    "stateMutability": "view",
+    "type": "function"
+  },
+  {
+    "constant": true,
+    "inputs": [],
+    "name": "isRevealPhase",
+    "outputs": [
+      {
+        "name": "",
+        "type": "bool"
+      }
+    ],
+    "payable": false,
+    "stateMutability": "view",
+    "type": "function"
+  },
+  {
+    "constant": true,
+    "inputs": [
+      {
+        "name": "_collectRound",
+        "type": "uint256"
+      },
+      {
+        "name": "_validator",
+        "type": "address"
+      }
+    ],
+    "name": "sentReveal",
+    "outputs": [
+      {
+        "name": "",
+        "type": "bool"
+      }
+    ],
+    "payable": false,
+    "stateMutability": "view",
+    "type": "function"
+  }
+]
\ No newline at end of file
diff --git a/consensus/aura/contracts/block_gas_limit.json b/consensus/aura/contracts/block_gas_limit.json
new file mode 100644
index 0000000000..8ca9cf9694
--- /dev/null
+++ b/consensus/aura/contracts/block_gas_limit.json
@@ -0,0 +1,16 @@
+[
+	{
+		"constant": true,
+		"inputs": [],
+		"name": "blockGasLimit",
+		"outputs": [
+			{
+				"name": "",
+				"type": "uint256"
+			}
+		],
+		"payable": false,
+		"stateMutability": "view",
+		"type": "function"
+	}
+]
\ No newline at end of file
diff --git a/consensus/aura/contracts/block_reward.json b/consensus/aura/contracts/block_reward.json
new file mode 100644
index 0000000000..9209967f3a
--- /dev/null
+++ b/consensus/aura/contracts/block_reward.json
@@ -0,0 +1,29 @@
+[
+  {
+    "constant": false,
+    "inputs": [
+      {
+        "name": "benefactors",
+        "type": "address[]"
+      },
+      {
+        "name": "kind",
+        "type": "uint16[]"
+      }
+    ],
+    "name": "reward",
+    "outputs": [
+      {
+        "name": "",
+        "type": "address[]"
+      },
+      {
+        "name": "",
+        "type": "uint256[]"
+      }
+    ],
+    "payable": false,
+    "stateMutability": "nonpayable",
+    "type": "function"
+  }
+]
diff --git a/consensus/aura/contracts/embed.go b/consensus/aura/contracts/embed.go
new file mode 100644
index 0000000000..002b62a4cb
--- /dev/null
+++ b/consensus/aura/contracts/embed.go
@@ -0,0 +1,8 @@
+package contracts
+
+import (
+	_ "embed"
+)
+
+//go:embed block_reward.json
+var BlockReward []byte
diff --git a/consensus/aura/contracts/registrar.json b/consensus/aura/contracts/registrar.json
new file mode 100644
index 0000000000..2f4aab1fdb
--- /dev/null
+++ b/consensus/aura/contracts/registrar.json
@@ -0,0 +1,21 @@
+[
+	{"constant":false,"inputs":[{"name":"_new","type":"address"}],"name":"setOwner","outputs":[],"type":"function"},
+	{"constant":false,"inputs":[{"name":"_name","type":"string"}],"name":"confirmReverse","outputs":[{"name":"success","type":"bool"}],"type":"function"},
+	{"constant":false,"inputs":[{"name":"_name","type":"bytes32"}],"name":"reserve","outputs":[{"name":"success","type":"bool"}],"type":"function"},
+	{"constant":false,"inputs":[{"name":"_name","type":"bytes32"},{"name":"_key","type":"string"},{"name":"_value","type":"bytes32"}],"name":"set","outputs":[{"name":"success","type":"bool"}],"type":"function"},
+	{"constant":false,"inputs":[{"name":"_name","type":"bytes32"}],"name":"drop","outputs":[{"name":"success","type":"bool"}],"type":"function"},
+	{"constant":true,"inputs":[{"name":"_name","type":"bytes32"},{"name":"_key","type":"string"}],"name":"getAddress","outputs":[{"name":"","type":"address"}],"type":"function"},
+	{"constant":false,"inputs":[{"name":"_amount","type":"uint256"}],"name":"setFee","outputs":[],"type":"function"},
+	{"constant":false,"inputs":[{"name":"_name","type":"bytes32"},{"name":"_to","type":"address"}],"name":"transfer","outputs":[{"name":"success","type":"bool"}],"type":"function"},
+	{"constant":true,"inputs":[],"name":"owner","outputs":[{"name":"","type":"address"}],"type":"function"},
+	{"constant":true,"inputs":[{"name":"_name","type":"bytes32"}],"name":"reserved","outputs":[{"name":"reserved","type":"bool"}],"type":"function"},
+	{"constant":false,"inputs":[],"name":"drain","outputs":[],"type":"function"},
+	{"constant":false,"inputs":[{"name":"_name","type":"string"},{"name":"_who","type":"address"}],"name":"proposeReverse","outputs":[{"name":"success","type":"bool"}],"type":"function"},
+	{"constant":true,"inputs":[{"name":"_name","type":"bytes32"},{"name":"_key","type":"string"}],"name":"getUint","outputs":[{"name":"","type":"uint256"}],"type":"function"},
+	{"constant":true,"inputs":[{"name":"_name","type":"bytes32"},{"name":"_key","type":"string"}],"name":"get","outputs":[{"name":"","type":"bytes32"}],"type":"function"},
+	{"constant":true,"inputs":[],"name":"fee","outputs":[{"name":"","type":"uint256"}],"type":"function"},
+	{"constant":true,"inputs":[{"name":"","type":"address"}],"name":"reverse","outputs":[{"name":"","type":"string"}],"type":"function"},
+	{"constant":false,"inputs":[{"name":"_name","type":"bytes32"},{"name":"_key","type":"string"},{"name":"_value","type":"uint256"}],"name":"setUint","outputs":[{"name":"success","type":"bool"}],"type":"function"},
+	{"constant":false,"inputs":[],"name":"removeReverse","outputs":[],"type":"function"},
+	{"constant":false,"inputs":[{"name":"_name","type":"bytes32"},{"name":"_key","type":"string"},{"name":"_value","type":"address"}],"name":"setAddress","outputs":[{"name":"success","type":"bool"}],"type":"function"},{"anonymous":false,"inputs":[{"indexed":false,"name":"amount","type":"uint256"}],"name":"Drained","type":"event"},{"anonymous":false,"inputs":[{"indexed":false,"name":"amount","type":"uint256"}],"name":"FeeChanged","type":"event"},{"anonymous":false,"inputs":[{"indexed":true,"name":"name","type":"bytes32"},{"indexed":true,"name":"owner","type":"address"}],"name":"Reserved","type":"event"},{"anonymous":false,"inputs":[{"indexed":true,"name":"name","type":"bytes32"},{"indexed":true,"name":"oldOwner","type":"address"},{"indexed":true,"name":"newOwner","type":"address"}],"name":"Transferred","type":"event"},{"anonymous":false,"inputs":[{"indexed":true,"name":"name","type":"bytes32"},{"indexed":true,"name":"owner","type":"address"}],"name":"Dropped","type":"event"},{"anonymous":false,"inputs":[{"indexed":true,"name":"name","type":"bytes32"},{"indexed":true,"name":"owner","type":"address"},{"indexed":true,"name":"key","type":"string"}],"name":"DataChanged","type":"event"},{"anonymous":false,"inputs":[{"indexed":true,"name":"name","type":"string"},{"indexed":true,"name":"reverse","type":"address"}],"name":"ReverseProposed","type":"event"},{"anonymous":false,"inputs":[{"indexed":true,"name":"name","type":"string"},{"indexed":true,"name":"reverse","type":"address"}],"name":"ReverseConfirmed","type":"event"},{"anonymous":false,"inputs":[{"indexed":true,"name":"name","type":"string"},{"indexed":true,"name":"reverse","type":"address"}],"name":"ReverseRemoved","type":"event"},{"anonymous":false,"inputs":[{"indexed":true,"name":"old","type":"address"},{"indexed":true,"name":"current","type":"address"}],"name":"NewOwner","type":"event"}
+]
\ No newline at end of file
diff --git a/consensus/aura/contracts/test_authority_round_random.json b/consensus/aura/contracts/test_authority_round_random.json
new file mode 100644
index 0000000000..beab82ab48
--- /dev/null
+++ b/consensus/aura/contracts/test_authority_round_random.json
@@ -0,0 +1,265 @@
+[
+	{
+		"constant": true,
+		"inputs": [
+			{
+				"name": "",
+				"type": "uint256"
+			},
+			{
+				"name": "",
+				"type": "address"
+			}
+		],
+		"name": "ciphers",
+		"outputs": [
+			{
+				"name": "",
+				"type": "bytes"
+			}
+		],
+		"payable": false,
+		"stateMutability": "view",
+		"type": "function"
+	},
+	{
+		"constant": true,
+		"inputs": [
+			{
+				"name": "_collectRound",
+				"type": "uint256"
+			},
+			{
+				"name": "_miningAddress",
+				"type": "address"
+			}
+		],
+		"name": "getCipher",
+		"outputs": [
+			{
+				"name": "",
+				"type": "bytes"
+			}
+		],
+		"payable": false,
+		"stateMutability": "view",
+		"type": "function"
+	},
+	{
+		"constant": false,
+		"inputs": [
+			{
+				"name": "_secretHash",
+				"type": "bytes32"
+			},
+			{
+				"name": "_cipher",
+				"type": "bytes"
+			}
+		],
+		"name": "commitHash",
+		"outputs": [],
+		"payable": false,
+		"stateMutability": "nonpayable",
+		"type": "function"
+	},
+	{
+		"constant": true,
+		"inputs": [],
+		"name": "getValue",
+		"outputs": [
+			{
+				"name": "",
+				"type": "uint256"
+			}
+		],
+		"payable": false,
+		"stateMutability": "view",
+		"type": "function"
+	},
+	{
+		"constant": true,
+		"inputs": [
+			{
+				"name": "",
+				"type": "uint256"
+			},
+			{
+				"name": "",
+				"type": "address"
+			}
+		],
+		"name": "hashes",
+		"outputs": [
+			{
+				"name": "",
+				"type": "bytes32"
+			}
+		],
+		"payable": false,
+		"stateMutability": "view",
+		"type": "function"
+	},
+	{
+		"constant": true,
+		"inputs": [],
+		"name": "value",
+		"outputs": [
+			{
+				"name": "",
+				"type": "uint256"
+			}
+		],
+		"payable": false,
+		"stateMutability": "view",
+		"type": "function"
+	},
+	{
+		"constant": true,
+		"inputs": [
+			{
+				"name": "",
+				"type": "uint256"
+			},
+			{
+				"name": "",
+				"type": "address"
+			}
+		],
+		"name": "secrets",
+		"outputs": [
+			{
+				"name": "",
+				"type": "uint256"
+			}
+		],
+		"payable": false,
+		"stateMutability": "view",
+		"type": "function"
+	},
+	{
+		"constant": true,
+		"inputs": [
+			{
+				"name": "_collectRound",
+				"type": "uint256"
+			},
+			{
+				"name": "_miningAddress",
+				"type": "address"
+			}
+		],
+		"name": "sentReveal",
+		"outputs": [
+			{
+				"name": "",
+				"type": "bool"
+			}
+		],
+		"payable": false,
+		"stateMutability": "view",
+		"type": "function"
+	},
+	{
+		"constant": true,
+		"inputs": [],
+		"name": "isCommitPhase",
+		"outputs": [
+			{
+				"name": "",
+				"type": "bool"
+			}
+		],
+		"payable": false,
+		"stateMutability": "view",
+		"type": "function"
+	},
+	{
+		"constant": false,
+		"inputs": [
+			{
+				"name": "_number",
+				"type": "uint256"
+			}
+		],
+		"name": "revealNumber",
+		"outputs": [],
+		"payable": false,
+		"stateMutability": "nonpayable",
+		"type": "function"
+	},
+	{
+		"constant": true,
+		"inputs": [],
+		"name": "getRound",
+		"outputs": [
+			{
+				"name": "",
+				"type": "uint256"
+			}
+		],
+		"payable": false,
+		"stateMutability": "view",
+		"type": "function"
+	},
+	{
+		"constant": true,
+		"inputs": [
+			{
+				"name": "_collectRound",
+				"type": "uint256"
+			},
+			{
+				"name": "_miningAddress",
+				"type": "address"
+			}
+		],
+		"name": "isCommitted",
+		"outputs": [
+			{
+				"name": "",
+				"type": "bool"
+			}
+		],
+		"payable": false,
+		"stateMutability": "view",
+		"type": "function"
+	},
+	{
+		"constant": true,
+		"inputs": [],
+		"name": "isRevealPhase",
+		"outputs": [
+			{
+				"name": "",
+				"type": "bool"
+			}
+		],
+		"payable": false,
+		"stateMutability": "view",
+		"type": "function"
+	},
+	{
+		"constant": true,
+		"inputs": [
+			{
+				"name": "_collectRound",
+				"type": "uint256"
+			},
+			{
+				"name": "_miningAddress",
+				"type": "address"
+			}
+		],
+		"name": "getCommit",
+		"outputs": [
+			{
+				"name": "",
+				"type": "bytes32"
+			}
+		],
+		"payable": false,
+		"stateMutability": "view",
+		"type": "function"
+	}
+]
\ No newline at end of file
diff --git a/consensus/aura/contracts/test_authority_round_random.sol b/consensus/aura/contracts/test_authority_round_random.sol
new file mode 100644
index 0000000000..66c9717863
--- /dev/null
+++ b/consensus/aura/contracts/test_authority_round_random.sol
@@ -0,0 +1,101 @@
+pragma solidity 0.5.10;
+
+/// @dev Randomness test contract based on https://github.com/poanetwork/posdao-contracts.
+/// Generates and stores random numbers in a RANDAO manner and accumulates a random seed.
+contract Random {
+    mapping(uint256 => mapping(address => bytes32)) public hashes;
+    mapping(uint256 => mapping(address => bytes)) public ciphers;
+    mapping(uint256 => mapping(address => uint256)) public secrets;
+    uint256 public value;
+
+    /// @dev Called by the validator's node to store a hash and a cipher of the validator's secret on each collection
+    /// round. The validator's node must use its mining address to call this function.
+    /// This function can only be called once per collection round (during the `commits phase`).
+    /// @param _secretHash The Keccak-256 hash of the validator's secret.
+    /// @param _cipher The cipher of the validator's secret. Can be used by the node to decrypt and reveal.
+    function commitHash(bytes32 _secretHash, bytes calldata _cipher) external {
+        require(block.coinbase == msg.sender);
+        require(_isCommitPhase(block.number - 1));
+        uint256 round = _collectRound(block.number - 1);
+        require(!isCommitted(round, msg.sender));
+        hashes[round][msg.sender] = _secretHash;
+        ciphers[round][msg.sender] = _cipher;
+    }
+
+    /// @dev Called by the validator's node to XOR its secret with the current random seed.
+    /// The validator's node must use its mining address to call this function.
+    /// This function can only be called once per collection round (during the `reveals phase`).
+    /// @param _number The validator's secret.
+    function revealNumber(uint256 _number) external {
+        require(block.coinbase == msg.sender);
+        require(_isRevealPhase(block.number - 1));
+        uint256 round = _collectRound(block.number - 1);
+        require(!sentReveal(round, msg.sender));
+        require(hashes[round][msg.sender] == keccak256(abi.encodePacked(_number)));
+        secrets[round][msg.sender] = _number;
+        value ^= _number;
+    }
+
+	/// @dev Returns the Keccak-256 hash and cipher of the validator's secret for the specified collection round
+    /// and the specified validator stored by the validator through the `commitHash` function.
+    /// @param _collectRound The serial number of the collection round for which hash and cipher should be retrieved.
+    /// @param _miningAddress The mining address of validator.
+    function getCommitAndCipher(
+        uint256 _collectRound,
+        address _miningAddress
+    ) public view returns(bytes32, bytes memory) {
+        return (hashes[_collectRound][_miningAddress], ciphers[_collectRound][_miningAddress]);
+    }
+
+    /// @dev Returns a boolean flag indicating whether the specified validator has committed their secret's hash for the
+    /// specified collection round.
+    /// @param _collectRound The serial number of the collection round for which the checkup should be done.
+    /// @param _miningAddress The mining address of the validator.
+    function isCommitted(uint256 _collectRound, address _miningAddress) public view returns(bool) {
+        return hashes[_collectRound][_miningAddress] != bytes32(0);
+    }
+
+    /// @dev Returns a boolean flag indicating whether the current phase of the current collection round
+    /// is a `commits phase`. Used by the validator's node to determine if it should commit the hash of
+    /// the secret during the current collection round.
+    function isCommitPhase() public view returns(bool) {
+        return _isCommitPhase(block.number);
+    }
+
+    /// @dev Returns a boolean flag indicating whether the current phase of the current collection round
+    /// is a `reveals phase`. Used by the validator's node to determine if it should reveal the secret during
+    /// the current collection round.
+    function isRevealPhase() public view returns(bool) {
+        return _isRevealPhase(block.number);
+    }
+
+    /// @dev Returns a boolean flag of whether the specified validator has revealed their secret for the
+    /// specified collection round.
+    /// @param _collectRound The serial number of the collection round for which the checkup should be done.
+    /// @param _miningAddress The mining address of the validator.
+    function sentReveal(uint256 _collectRound, address _miningAddress) public view returns(bool) {
+        return secrets[_collectRound][_miningAddress] != uint256(0);
+    }
+
+    /// @dev Returns the current collect round number.
+    function currentCollectRound() public view returns(uint256) {
+        return _collectRound(block.number);
+    }
+
+    /// @dev Returns the current random value.
+    function getValue() public view returns(uint256) {
+        return value;
+    }
+
+    function _collectRound(uint256 blockNumber) private pure returns(uint256) {
+        return blockNumber / 6;
+    }
+
+    function _isCommitPhase(uint256 blockNumber) private pure returns(bool) {
+        return blockNumber % 6 < 3;
+    }
+
+    function _isRevealPhase(uint256 blockNumber) private pure returns(bool) {
+        return blockNumber % 6 >= 3;
+    }
+}
diff --git a/consensus/aura/contracts/test_validator_set.json b/consensus/aura/contracts/test_validator_set.json
new file mode 100644
index 0000000000..7ed6a000fe
--- /dev/null
+++ b/consensus/aura/contracts/test_validator_set.json
@@ -0,0 +1,8 @@
+[
+	{"constant":false,"inputs":[{"name":"_validators","type":"address[]"}],"name":"setValidators","outputs":[],"payable":false,"type":"function"},
+	{"constant":false,"inputs":[{"name":"","type":"address"},{"name":"","type":"bytes"}],"name":"reportMalicious","outputs":[],"payable":false,"type":"function"},
+	{"constant":false,"inputs":[],"name":"finalizeChange","outputs":[],"payable":false,"type":"function"},
+	{"constant":true,"inputs":[],"name":"getValidators","outputs":[{"name":"_validators","type":"address[]"}],"payable":false,"type":"function"},
+	{"constant":false,"inputs":[{"name":"","type":"address"}],"name":"reportBenign","outputs":[],"payable":false,"type":"function"},
+	{"anonymous":false,"inputs":[{"indexed":true,"name":"_parent_hash","type":"bytes32"},{"indexed":false,"name":"_new_set","type":"address[]"}],"name":"InitiateChange","type":"event"}
+]
diff --git a/consensus/aura/contracts/tx_acl.json b/consensus/aura/contracts/tx_acl.json
new file mode 100644
index 0000000000..e110797d96
--- /dev/null
+++ b/consensus/aura/contracts/tx_acl.json
@@ -0,0 +1 @@
+[ { "constant": true, "inputs": [], "name": "contractNameHash", "outputs": [ { "name": "", "type": "bytes32" } ], "payable": false, "stateMutability": "view", "type": "function" }, { "constant": true, "inputs": [], "name": "contractName", "outputs": [ { "name": "", "type": "string" } ], "payable": false, "stateMutability": "view", "type": "function" }, { "constant": true, "inputs": [], "name": "contractVersion", "outputs": [ { "name": "", "type": "uint256" } ], "payable": false, "stateMutability": "view", "type": "function" }, { "constant": true, "inputs": [ { "name": "sender", "type": "address" }, { "name": "to", "type": "address" }, { "name": "value", "type": "uint256" } ], "name": "allowedTxTypes", "outputs": [ { "name": "", "type": "uint32" }, { "name": "", "type": "bool" } ], "payable": false, "stateMutability": "view", "type": "function" } ]
diff --git a/consensus/aura/contracts/tx_acl_deprecated.json b/consensus/aura/contracts/tx_acl_deprecated.json
new file mode 100644
index 0000000000..cc924cafb8
--- /dev/null
+++ b/consensus/aura/contracts/tx_acl_deprecated.json
@@ -0,0 +1 @@
+[{"constant":true,"inputs":[{"name":"sender","type":"address"}],"name":"allowedTxTypes","outputs":[{"name":"","type":"uint32"}],"payable":false,"stateMutability":"nonpayable","type":"function"}]
diff --git a/consensus/aura/contracts/tx_acl_gas_price.json b/consensus/aura/contracts/tx_acl_gas_price.json
new file mode 100644
index 0000000000..37b08e9f0d
--- /dev/null
+++ b/consensus/aura/contracts/tx_acl_gas_price.json
@@ -0,0 +1,83 @@
+[
+	{
+		"constant": true,
+		"inputs": [],
+		"name": "contractNameHash",
+		"outputs": [
+			{
+				"name": "",
+				"type": "bytes32"
+			}
+		],
+		"payable": false,
+		"stateMutability": "view",
+		"type": "function"
+	},
+	{
+		"constant": true,
+		"inputs": [],
+		"name": "contractName",
+		"outputs": [
+			{
+				"name": "",
+				"type": "string"
+			}
+		],
+		"payable": false,
+		"stateMutability": "view",
+		"type": "function"
+	},
+	{
+		"constant": true,
+		"inputs": [],
+		"name": "contractVersion",
+		"outputs": [
+			{
+				"name": "",
+				"type": "uint256"
+			}
+		],
+		"payable": false,
+		"stateMutability": "view",
+		"type": "function"
+	},
+	{
+		"constant": true,
+		"inputs": [
+			{
+				"name": "sender",
+				"type": "address"
+			},
+			{
+				"name": "to",
+				"type": "address"
+			},
+			{
+				"name": "value",
+				"type": "uint256"
+			},
+			{
+				"name": "gasPrice",
+				"type": "uint256"
+			},
+			{
+				"name": "data",
+				"type": "bytes"
+			}
+		],
+		"name": "allowedTxTypes",
+		"outputs": [
+			{
+				"name": "",
+				"type": "uint32"
+			},
+			{
+				"name": "",
+				"type": "bool"
+			}
+		],
+		"payable": false,
+		"stateMutability": "view",
+		"type": "function"
+	}
+]
\ No newline at end of file
diff --git a/consensus/aura/contracts/validator_report.json b/consensus/aura/contracts/validator_report.json
new file mode 100644
index 0000000000..e0c0114326
--- /dev/null
+++ b/consensus/aura/contracts/validator_report.json
@@ -0,0 +1,5 @@
+[
+	{"constant":false,"inputs":[{"name":"validator","type":"address"},{"name":"blockNumber","type":"uint256"},{"name":"proof","type":"bytes"}],"name":"reportMalicious","outputs":[],"payable":false,"type":"function"},
+	{"constant":false,"inputs":[{"name":"validator","type":"address"},{"name":"blockNumber","type":"uint256"}],"name":"reportBenign","outputs":[],"payable":false,"type":"function"},
+	{"constant": true, "inputs": [ { "name": "validator", "type": "address" }, { "name": "blockNum", "type": "uint256" } ], "name": "maliceReportedForBlock", "outputs": [ { "name": "", "type": "address[]" } ], "payable": false, "stateMutability": "view", "type": "function" }
+]
diff --git a/consensus/aura/contracts/validator_set.json b/consensus/aura/contracts/validator_set.json
new file mode 100644
index 0000000000..660e8b614b
--- /dev/null
+++ b/consensus/aura/contracts/validator_set.json
@@ -0,0 +1,55 @@
+[
+	{"constant":false,"inputs":[],"name":"finalizeChange","outputs":[],"payable":false,"type":"function"},
+	{"constant":true,"inputs":[],"name":"getValidators","outputs":[{"name":"validators","type":"address[]"}],"payable":false,"type":"function"},
+	{"anonymous":false,"inputs":[{"indexed":true,"name":"_parent_hash","type":"bytes32"},{"indexed":false,"name":"_new_set","type":"address[]"}],"name":"InitiateChange","type":"event"},
+			{
+				"constant": true,
+				"inputs": [],
+				"name": "emitInitiateChangeCallable",
+				"outputs": [
+					{
+						"name": "",
+						"type": "bool"
+					}
+				],
+				"payable": false,
+				"stateMutability": "view",
+				"type": "function"
+			},
+			{
+				"constant": false,
+				"inputs": [],
+				"name": "emitInitiateChange",
+				"outputs": [],
+				"payable": false,
+				"stateMutability": "nonpayable",
+				"type": "function"
+			},
+			{
+			  "constant": true,
+			  "inputs": [
+				{
+				  "name": "_reportingValidator",
+				  "type": "address"
+				},
+				{
+				  "name": "_maliciousValidator",
+				  "type": "address"
+				},
+				{
+					"name": "_blockNumber",
+					"type": "uint256"
+				}
+			  ],
+			  "name": "shouldValidatorReport",
+			  "outputs": [
+				{
+				  "name": "",
+				  "type": "bool"
+				}
+			  ],
+			  "payable": false,
+			  "stateMutability": "view",
+			  "type": "function"
+			}
+		]
\ No newline at end of file
diff --git a/consensus/aura/validators.go b/consensus/aura/validators.go
new file mode 100644
index 0000000000..4a1220f514
--- /dev/null
+++ b/consensus/aura/validators.go
@@ -0,0 +1,465 @@
+package aura
+
+import (
+	"container/list"
+	"fmt"
+	"sort"
+	"sync"
+
+	lru "github.com/hashicorp/golang-lru"
+	"github.com/ledgerwatch/erigon/common"
+	"github.com/ledgerwatch/erigon/consensus/aura/aurainterfaces"
+	"github.com/ledgerwatch/erigon/core/types"
+	"github.com/ledgerwatch/erigon/log"
+	"go.uber.org/atomic"
+)
+
+/// Kind of SystemOrCodeCall, this is either an on-chain address, or code.
+type SystemOrCodeCallKind uint8
+
+const (
+	SystemCallOnChain SystemOrCodeCallKind = 0
+	CallHardCodedCode SystemOrCodeCallKind = 1
+)
+
+//nolint
+type CallResults struct {
+	data      []byte
+	proof     [][]byte
+	execError string
+}
+
+// Type alias for a function we can make calls through synchronously.
+// Returns the call result and state proof for each call.
+type Call func(common.Address, []byte) (CallResults, error)
+
+// A system-calling closure. Enacts calls on a block's state from the system address.
+type SystemCall func(common.Address, []byte) (CallResults, error)
+
+type client interface {
+	CallAtBlockHash(common.Hash, common.Address, []byte) (CallResults, error)
+	CallAtLatestBlock(common.Address, []byte) (CallResults, error)
+	SystemCallAtBlockHash(blockHash common.Hash, contract common.Address, data []byte) (CallResults, error)
+}
+
+type ValidatorSet interface {
+
+	// Get the default "Call" helper, for use in general operation.
+	// TODO [keorn]: this is a hack intended to migrate off of
+	// a strict dependency on state always being available.
+	defaultCaller(blockHash common.Hash) (Call, error)
+
+	// Called for each new block this node is creating.  If this block is
+	// the first block of an epoch, this is called *after* `on_epoch_begin()`,
+	// but with the same parameters.
+	//
+	// 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>
+
+	// Called on the close of every block.
+	onCloseBlock(_header *types.Header, _address common.Address) error
+
+	// Draws an validator nonce modulo number of validators.
+	getWithCaller(parentHash common.Hash, nonce uint, caller Call) (common.Address, error)
+	/*
+	 // Returns the current number of validators.
+	    fn count(&self, parent: &H256) -> usize {
+	        let default = self.default_caller(BlockId::Hash(*parent));
+	        self.count_with_caller(parent, &*default)
+	    }
+
+	    // Signalling that a new epoch has begun.
+	    //
+	    // All calls here will be from the `SYSTEM_ADDRESS`: 2^160 - 2
+	    // and will have an effect on the block's state.
+	    // The caller provided here may not generate proofs.
+	    //
+	    // `first` is true if this is the first block in the set.
+	    fn on_epoch_begin(
+	        &self,
+	        _first: bool,
+	        _header: &Header,
+	        _call: &mut SystemCall,
+	    ) -> Result<(), ::error::Error> {
+	        Ok(())
+	    }
+
+	    // Extract genesis epoch data from the genesis state and header.
+	    fn genesis_epoch_data(&self, _header: &Header, _call: &Call) -> Result<Vec<u8>, String> {
+	        Ok(Vec::new())
+	    }
+
+	    // Whether this block is the last one in its epoch.
+	    //
+	    // Indicates that the validator set changed at the given block in a manner
+	    // that doesn't require finality.
+	    //
+	    // `first` is true if this is the first block in the set.
+	    fn is_epoch_end(&self, first: bool, chain_head: &Header) -> Option<Vec<u8>>;
+
+	    // Whether the given block signals the end of an epoch, but change won't take effect
+	    // until finality.
+	    //
+	    // Engine should set `first` only if the header is genesis. Multiplexing validator
+	    // sets can set `first` to internal changes.
+	    fn signals_epoch_end(
+	        &self,
+	        first: bool,
+	        header: &Header,
+	        aux: AuxiliaryData,
+	    ) -> ::engines::EpochChange<EthereumMachine>;
+
+	    // 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.
+	    fn epoch_set(
+	        &self,
+	        first: bool,
+	        machine: &EthereumMachine,
+	        number: BlockNumber,
+	        proof: &[u8],
+	    ) -> Result<(SimpleList, Option<H256>), ::error::Error>;
+
+	    // Checks if a given address is a validator, with the given function
+	    // for executing synchronous calls to contracts.
+	    fn contains_with_caller(
+	        &self,
+	        parent_block_hash: &H256,
+	        address: &Address,
+	        caller: &Call,
+	    ) -> bool;
+
+	    // Draws an validator nonce modulo number of validators.
+	    fn get_with_caller(&self, parent_block_hash: &H256, nonce: usize, caller: &Call) -> Address;
+
+	    // Returns the current number of validators.
+	    fn count_with_caller(&self, parent_block_hash: &H256, caller: &Call) -> usize;
+
+	    // Notifies about malicious behaviour.
+	    fn report_malicious(
+	        &self,
+	        _validator: &Address,
+	        _set_block: BlockNumber,
+	        _block: BlockNumber,
+	        _proof: Bytes,
+	    ) {
+	    }
+	    // Notifies about benign misbehaviour.
+	    fn report_benign(&self, _validator: &Address, _set_block: BlockNumber, _block: BlockNumber) {}
+	*/
+}
+
+func get(s ValidatorSet, h common.Hash, nonce uint) (common.Address, error) {
+	d, err := s.defaultCaller(h)
+	if err != nil {
+		return common.Address{}, err
+	}
+	return s.getWithCaller(h, nonce, d)
+}
+
+//nolint
+type MultiItem struct {
+	num  uint64
+	hash common.Hash
+	set  ValidatorSet
+}
+
+type Multi struct {
+	sorted []MultiItem
+	parent func(common.Hash) *types.Header
+}
+
+func (s *Multi) Less(i, j int) bool { return s.sorted[i].num < s.sorted[j].num }
+func (s *Multi) Len() int           { return len(s.sorted) }
+func (s *Multi) Swap(i, j int)      { s.sorted[i], s.sorted[j] = s.sorted[j], s.sorted[i] }
+
+func NewMulti(m map[uint64]ValidatorSet) *Multi {
+	if _, ok := m[0]; !ok {
+		panic("ValidatorSet has to be specified from block 0")
+	}
+	list := make([]MultiItem, len(m))
+	i := 0
+	for n, v := range m {
+		list[i] = MultiItem{num: n, set: v}
+		i++
+	}
+	multi := &Multi{sorted: list}
+	sort.Sort(multi)
+	return multi
+}
+
+func (s *Multi) defaultCaller(blockHash common.Hash) (Call, error) {
+	set, ok := s.correctSet(blockHash)
+	if !ok {
+		return nil, fmt.Errorf("no validator set for given blockHash: %x", blockHash)
+	}
+	return set.defaultCaller(blockHash)
+}
+
+func (s *Multi) getWithCaller(parentHash common.Hash, nonce uint, caller Call) (common.Address, error) {
+	panic("not implemented")
+}
+
+func (s *Multi) correctSet(blockHash common.Hash) (ValidatorSet, bool) {
+	parent := s.parent(blockHash)
+	if parent == nil {
+		return nil, false
+	}
+	_, set := s.correctSetByNumber(parent.Number.Uint64())
+	return set, set != nil
+}
+
+func (s *Multi) correctSetByNumber(parentNumber uint64) (uint64, ValidatorSet) {
+	// get correct set by block number, along with block number at which
+	// this set was activated.
+	for i := len(s.sorted); i >= 0; i-- {
+		if s.sorted[i].num <= parentNumber+1 {
+			return s.sorted[i].num, s.sorted[i].set
+		}
+	}
+	panic("constructor validation ensures that there is at least one validator set for block 0; block 0 is less than any uint; qed")
+}
+
+func (s *Multi) get(num uint64) (first bool, set ValidatorSet) {
+	block, set := s.correctSetByNumber(num)
+	first = block == num
+	return first, set
+}
+
+func (s *Multi) onCloseBlock(header *types.Header, address common.Address) error {
+	_, set := s.get(header.Number.Uint64())
+	return set.onCloseBlock(header, address)
+}
+
+//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)
+//}
+
+type SimpleList struct {
+	validators []common.Address
+}
+
+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")
+}
+func (s *SimpleList) getWithCaller(parentHash common.Hash, nonce uint, caller Call) (common.Address, error) {
+	if len(s.validators) == 0 {
+		return common.Address{}, fmt.Errorf("cannot operate with an empty validator set")
+	}
+	return s.validators[nonce%uint(len(s.validators))], nil
+}
+
+// Draws an validator nonce modulo number of validators.
+
+func NewSimpleList(validators []common.Address) *SimpleList {
+	return &SimpleList{validators: validators}
+}
+
+//nolint
+type ReportQueueItem struct {
+	addr     common.Address
+	blockNum uint64
+	data     []byte
+}
+type ReportQueue struct {
+	sync.RWMutex
+	list *list.List
+}
+
+//nolint
+func (q *ReportQueue) push(addr common.Address, blockNum uint64, data []byte) {
+	q.Lock()
+	defer q.Unlock()
+	q.list.PushBack(&ReportQueueItem{addr: addr, blockNum: blockNum, data: data})
+}
+
+// Filters reports of validators that have already been reported or are banned.
+
+func (q *ReportQueue) filter(abi aurainterfaces.ValidatorSetABI, client client, ourAddr, contractAddr common.Address) error {
+	q.Lock()
+	defer q.Unlock()
+	for e := q.list.Front(); e != nil; e = e.Next() {
+		el := e.Value.(*ReportQueueItem)
+		// Check if the validator should be reported.
+		maliciousValidatorAddress := el.addr
+		data, decoder := abi.ShouldValidatorReport(ourAddr, maliciousValidatorAddress, el.blockNum)
+		res, err := client.CallAtLatestBlock(contractAddr, data)
+		if err != nil {
+			return err
+		}
+		if res.execError != "" {
+			log.Warn("Failed to query report status, dropping pending report.", "reason", res.execError)
+			continue
+		}
+		var shouldReport bool
+		err = decoder(res.data, &res)
+		if err != nil {
+			return err
+		}
+		if !shouldReport {
+			q.list.Remove(e)
+		}
+	}
+	return nil
+}
+
+// Removes reports from the queue if it contains more than `MAX_QUEUED_REPORTS` entries.
+func (q *ReportQueue) truncate() {
+	// The maximum number of reports to keep queued.
+	const MaxQueuedReports = 10
+
+	q.RLock()
+	defer q.RUnlock()
+	// Removes reports from the queue if it contains more than `MAX_QUEUED_REPORTS` entries.
+	if q.list.Len() > MaxQueuedReports {
+		log.Warn("Removing reports from report cache, even though it has not been finalized", "amount", q.list.Len()-MaxQueuedReports)
+	}
+	i := 0
+	for e := q.list.Front(); e != nil; e = e.Next() {
+		if i > MaxQueuedReports {
+			q.list.Remove(e)
+		}
+		i++
+	}
+}
+
+// The validator contract should have the following interface:
+//nolint
+type ValidatorSafeContract struct {
+	contractAddress common.Address
+	validators      *lru.Cache  // RwLock<MemoryLruCache<H256, SimpleList>>,
+	reportQueue     ReportQueue //Mutex<ReportQueue>,
+	// The block number where we resent the queued reports last time.
+	resentReportsInBlock atomic.Uint64
+	// If set, this is the block number at which the consensus engine switches from AuRa to AuRa
+	// with POSDAO modifications.
+	posdaoTransition *uint64
+
+	abi    aurainterfaces.ValidatorSetABI
+	client client
+}
+
+func NewValidatorSafeContract(contractAddress common.Address, posdaoTransition *uint64, abi aurainterfaces.ValidatorSetABI, client client) *ValidatorSafeContract {
+	const MemoizeCapacity = 500
+	c, err := lru.New(MemoizeCapacity)
+	if err != nil {
+		panic("error creating ValidatorSafeContract cache")
+	}
+	return &ValidatorSafeContract{contractAddress: contractAddress, posdaoTransition: posdaoTransition, validators: c, client: client}
+}
+
+// Called for each new block this node is creating.  If this block is
+// the first block of an epoch, this is called *after* `on_epoch_begin()`,
+// but with the same parameters.
+//
+// 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) defaultCaller(blockHash common.Hash) (Call, error) {
+	return func(addr common.Address, data []byte) (CallResults, error) {
+		return s.client.CallAtBlockHash(blockHash, addr, data)
+	}, nil
+}
+func (s *ValidatorSafeContract) getWithCaller(blockHash common.Hash, nonce uint, caller Call) (common.Address, error) {
+	set, ok := s.validators.Get(blockHash)
+	if ok {
+		return get(set.(ValidatorSet), blockHash, nonce)
+	}
+
+	list, ok := s.getList(caller)
+	if !ok {
+		return common.Address{}, nil
+	}
+	return get(list, blockHash, nonce)
+}
+
+func (s *ValidatorSafeContract) getList(caller Call) (*SimpleList, bool) {
+	code, decoder := s.abi.GetValidators()
+	callResult, err := caller(s.contractAddress, code)
+	if err != nil {
+		log.Debug("Set of validators could not be updated: ", "err", err)
+		return nil, false
+	}
+	if callResult.execError != "" {
+		log.Debug("Set of validators could not be updated: ", "err", callResult.execError)
+		return nil, false
+	}
+	var res []common.Address
+	err = decoder(callResult.data, &res)
+	if err != nil {
+		log.Debug("Set of validators could not be updated: ", "err", err)
+		return nil, false
+	}
+	return NewSimpleList(res), true
+}
+
+func (s *ValidatorSafeContract) onCloseBlock(header *types.Header, ourAddress common.Address) error {
+	// Skip the rest of the function unless there has been a transition to POSDAO AuRa.
+	if s.posdaoTransition != nil && header.Number.Uint64() < *s.posdaoTransition {
+		log.Trace("Skipping resending of queued malicious behavior reports")
+		return nil
+	}
+	err := s.reportQueue.filter(s.abi, s.client, ourAddress, s.contractAddress)
+	if err != nil {
+		return err
+	}
+	s.reportQueue.truncate()
+	/*
+	   let mut resent_reports_in_block = self.resent_reports_in_block.lock();
+
+	   // Skip at least one block after sending malicious reports last time.
+	   if header.number() > *resent_reports_in_block + REPORTS_SKIP_BLOCKS {
+	       *resent_reports_in_block = header.number();
+	       let mut nonce = client.latest_nonce(our_address);
+	       for (address, block, data) in report_queue.iter() {
+	           debug!(target: "engine", "Retrying to report validator {} for misbehavior on block {} with nonce {}.",
+	              address, block, nonce);
+	           while match self.transact(data.clone(), nonce) {
+	               Ok(()) => false,
+	               Err(EthcoreError(
+	                   EthcoreErrorKind::Transaction(transaction::Error::Old),
+	                   _,
+	               )) => true,
+	               Err(err) => {
+	                   warn!(target: "engine", "Cannot report validator {} for misbehavior on block {}: {}",
+	                     address, block, err);
+	                   false
+	               }
+	           } {
+	               warn!(target: "engine", "Nonce {} already used. Incrementing.", nonce);
+	               nonce += U256::from(1);
+	           }
+	           nonce += U256::from(1);
+	       }
+	   }
+
+	   Ok(())
+
+	*/
+	return nil
+}
+
+// A validator contract with reporting.
+type ValidatorContract struct {
+	contractAddress  common.Address
+	validators       ValidatorSafeContract
+	posdaoTransition *uint64
+}
+
+func (s *ValidatorContract) defaultCaller(blockHash common.Hash) (Call, error) {
+	return s.validators.defaultCaller(blockHash)
+}
+
+func (s *ValidatorContract) getWithCaller(parentHash common.Hash, nonce uint, caller Call) (common.Address, error) {
+	return s.validators.getWithCaller(parentHash, nonce, caller)
+}
+func (s *ValidatorContract) onCloseBlock(header *types.Header, address common.Address) error {
+	return s.validators.onCloseBlock(header, address)
+}
diff --git a/consensus/clique/aura.go b/consensus/clique/aura.go
deleted file mode 100644
index 31197de903..0000000000
--- a/consensus/clique/aura.go
+++ /dev/null
@@ -1,309 +0,0 @@
-// Copyright 2017 The go-ethereum Authors
-// This file is part of the go-ethereum library.
-//
-// The go-ethereum library is free software: you can redistribute it and/or modify
-// it under the terms of the GNU Lesser General Public License as published by
-// the Free Software Foundation, either version 3 of the License, or
-// (at your option) any later version.
-//
-// The go-ethereum library is distributed in the hope that it will be useful,
-// but WITHOUT ANY WARRANTY; without even the implied warranty of
-// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
-// GNU Lesser General Public License for more details.
-//
-// 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 clique
-
-import (
-	"math/big"
-	"sync"
-
-	"github.com/ledgerwatch/erigon/common"
-	"github.com/ledgerwatch/erigon/consensus"
-	"github.com/ledgerwatch/erigon/core/state"
-	"github.com/ledgerwatch/erigon/core/types"
-	"github.com/ledgerwatch/erigon/ethdb"
-	"github.com/ledgerwatch/erigon/params"
-	"github.com/ledgerwatch/erigon/rpc"
-)
-
-// AuRa
-type AuRa struct {
-	db     ethdb.RwKV   // Database to store and retrieve snapshot checkpoints
-	lock   sync.RWMutex // Protects the signer fields
-	exitCh chan struct{}
-}
-
-// 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) *AuRa {
-	config := cfg.Aura
-
-	exitCh := make(chan struct{})
-
-	c := &AuRa{
-		db:     db,
-		exitCh: exitCh,
-	}
-	_ = config
-
-	return c
-}
-
-// Author implements consensus.Engine, returning the Ethereum address recovered
-// from the signature in the header's extra-data section.
-func (c *AuRa) Author(header *types.Header) (common.Address, error) {
-	return common.Address{}, nil
-	//return ecrecover(header, c.signatures)
-}
-
-// VerifyHeader checks whether a header conforms to the consensus rules.
-func (c *AuRa) VerifyHeader(chain consensus.ChainHeaderReader, header *types.Header, _ bool) error {
-	return nil
-	//return c.verifyHeader(chain, header, nil)
-}
-
-// VerifyHeaders is similar to VerifyHeader, but verifies a batch of headers. The
-// method returns a quit channel to abort the operations and a results channel to
-// retrieve the async verifications (the order is that of the input slice).
-func (c *AuRa) VerifyHeaders(chain consensus.ChainHeaderReader, headers []*types.Header, _ []bool) error {
-	return nil
-	//if len(headers) == 0 {
-	//	return nil
-	//}
-	//for i, header := range headers {
-	//	if err := c.verifyHeader(chain, header, headers[:i]); err != nil {
-	//		return err
-	//	}
-	//}
-	//return nil
-}
-
-// VerifyUncles implements consensus.Engine, always returning an error for any
-// uncles as this consensus mechanism doesn't permit uncles.
-func (c *AuRa) VerifyUncles(chain consensus.ChainReader, header *types.Header, uncles []*types.Header) error {
-	return nil
-	//if len(uncles) > 0 {
-	//	return errors.New("uncles not allowed")
-	//}
-	//return nil
-}
-
-// VerifySeal implements consensus.Engine, checking whether the signature contained
-// in the header satisfies the consensus protocol requirements.
-func (c *AuRa) VerifySeal(chain consensus.ChainHeaderReader, header *types.Header) error {
-	return nil
-	//snap, err := c.Snapshot(chain, header.Number.Uint64(), header.Hash(), nil)
-	//if err != nil {
-	//	return err
-	//}
-	//return c.verifySeal(chain, header, snap)
-}
-
-// Prepare implements consensus.Engine, preparing all the consensus fields of the
-// header for running the transactions on top.
-func (c *AuRa) Prepare(chain consensus.ChainHeaderReader, header *types.Header) error {
-	return nil
-	//// If the block isn't a checkpoint, cast a random vote (good enough for now)
-	//header.Coinbase = common.Address{}
-	//header.Nonce = types.BlockNonce{}
-	//
-	//number := header.Number.Uint64()
-	//// Assemble the voting snapshot to check which votes make sense
-	//snap, err := c.Snapshot(chain, number-1, header.ParentHash, nil)
-	//if err != nil {
-	//	return err
-	//}
-	//if number%c.config.Epoch != 0 {
-	//	c.lock.RLock()
-	//
-	//	// Gather all the proposals that make sense voting on
-	//	addresses := make([]common.Address, 0, len(c.proposals))
-	//	for address, authorize := range c.proposals {
-	//		if snap.validVote(address, authorize) {
-	//			addresses = append(addresses, address)
-	//		}
-	//	}
-	//	// If there's pending proposals, cast a vote on them
-	//	if len(addresses) > 0 {
-	//		header.Coinbase = addresses[rand.Intn(len(addresses))]
-	//		if c.proposals[header.Coinbase] {
-	//			copy(header.Nonce[:], NonceAuthVote)
-	//		} else {
-	//			copy(header.Nonce[:], nonceDropVote)
-	//		}
-	//	}
-	//	c.lock.RUnlock()
-	//}
-	//// Set the correct difficulty
-	//header.Difficulty = calcDifficulty(snap, c.signer)
-	//
-	//// Ensure the extra data has all its components
-	//if len(header.Extra) < ExtraVanity {
-	//	header.Extra = append(header.Extra, bytes.Repeat([]byte{0x00}, ExtraVanity-len(header.Extra))...)
-	//}
-	//header.Extra = header.Extra[:ExtraVanity]
-	//
-	//if number%c.config.Epoch == 0 {
-	//	for _, signer := range snap.GetSigners() {
-	//		header.Extra = append(header.Extra, signer[:]...)
-	//	}
-	//}
-	//header.Extra = append(header.Extra, make([]byte, ExtraSeal)...)
-	//
-	//// Mix digest is reserved for now, set to empty
-	//header.MixDigest = common.Hash{}
-	//
-	//// Ensure the timestamp has the correct delay
-	//parent := chain.GetHeader(header.ParentHash, number-1)
-	//if parent == nil {
-	//	return consensus.ErrUnknownAncestor
-	//}
-	//header.Time = parent.Time + c.config.Period
-	//
-	//now := uint64(time.Now().Unix())
-	//if header.Time < now {
-	//	header.Time = now
-	//}
-	//
-	//return nil
-}
-
-// Finalize implements consensus.Engine, ensuring no uncles are set, nor block
-// rewards given.
-func (c *AuRa) Finalize(_ *params.ChainConfig, header *types.Header, state *state.IntraBlockState, txs []types.Transaction, uncles []*types.Header) {
-	// No block rewards in PoA, so the state remains as is and uncles are dropped
-	header.UncleHash = types.CalcUncleHash(nil)
-}
-
-// FinalizeAndAssemble implements consensus.Engine, ensuring no uncles are set,
-// nor block rewards given, and returns the final block.
-func (c *AuRa) FinalizeAndAssemble(chainConfig *params.ChainConfig, header *types.Header, state *state.IntraBlockState, txs []types.Transaction, uncles []*types.Header, receipts []*types.Receipt) (*types.Block, error) {
-	// No block rewards in PoA, so the state remains as is and uncles are dropped
-	//header.UncleHash = types.CalcUncleHash(nil)
-
-	// Assemble and return the final block for sealing
-	return types.NewBlock(header, txs, nil, receipts), nil
-}
-
-// Authorize injects a private key into the consensus engine to mint new blocks
-// with.
-func (c *AuRa) Authorize(signer common.Address, signFn SignerFn) {
-	c.lock.Lock()
-	defer c.lock.Unlock()
-
-	//c.signer = signer
-	//c.signFn = signFn
-}
-
-// Seal implements consensus.Engine, attempting to create a sealed block using
-// the local signing credentials.
-func (c *AuRa) Seal(chain consensus.ChainHeaderReader, block *types.Block, results chan<- *types.Block, stop <-chan struct{}) error {
-	return nil
-	//header := block.Header()
-	//
-	//// Sealing the genesis block is not supported
-	//number := header.Number.Uint64()
-	//if number == 0 {
-	//	return errUnknownBlock
-	//}
-	//// For 0-period chains, refuse to seal empty blocks (no reward but would spin sealing)
-	//if c.config.Period == 0 && len(block.Transactions()) == 0 {
-	//	log.Info("Sealing paused, waiting for transactions")
-	//	return nil
-	//}
-	//// Don't hold the signer fields for the entire sealing procedure
-	//c.lock.RLock()
-	//signer, signFn := c.signer, c.signFn
-	//c.lock.RUnlock()
-	//
-	//// Bail out if we're unauthorized to sign a block
-	//snap, err := c.Snapshot(chain, number-1, header.ParentHash, nil)
-	//if err != nil {
-	//	return err
-	//}
-	//if _, authorized := snap.Signers[signer]; !authorized {
-	//	return ErrUnauthorizedSigner
-	//}
-	//// If we're amongst the recent signers, wait for the next block
-	//for seen, recent := range snap.Recents {
-	//	if recent == signer {
-	//		// Signer is among RecentsRLP, only wait if the current block doesn't shift it out
-	//		if limit := uint64(len(snap.Signers)/2 + 1); number < limit || seen > number-limit {
-	//			log.Info("Signed recently, must wait for others")
-	//			return nil
-	//		}
-	//	}
-	//}
-	//// Sweet, the protocol permits us to sign the block, wait for our time
-	//delay := time.Unix(int64(header.Time), 0).Sub(time.Now()) // nolint: gosimple
-	//if header.Difficulty.Cmp(diffNoTurn) == 0 {
-	//	// It's not our turn explicitly to sign, delay it a bit
-	//	wiggle := time.Duration(len(snap.Signers)/2+1) * wiggleTime
-	//	delay += time.Duration(rand.Int63n(int64(wiggle)))
-	//
-	//	log.Trace("Out-of-turn signing requested", "wiggle", common.PrettyDuration(wiggle))
-	//}
-	//// Sign all the things!
-	//sighash, err := signFn(signer, accounts.MimetypeClique, CliqueRLP(header))
-	//if err != nil {
-	//	return err
-	//}
-	//copy(header.Extra[len(header.Extra)-ExtraSeal:], sighash)
-	//// Wait until sealing is terminated or delay timeout.
-	//log.Trace("Waiting for slot to sign and propagate", "delay", common.PrettyDuration(delay))
-	//go func() {
-	//	select {
-	//	case <-stop:
-	//		return
-	//	case <-time.After(delay):
-	//	}
-	//
-	//	select {
-	//	case results <- block.WithSeal(header):
-	//	default:
-	//		log.Warn("Sealing result is not read by miner", "sealhash", SealHash(header))
-	//	}
-	//}()
-	//
-	//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 {
-	return nil
-	//snap, err := c.Snapshot(chain, parentNumber, parentHash, nil)
-	//if err != nil {
-	//	return nil
-	//}
-	//return calcDifficulty(snap, c.signer)
-}
-
-func (c *AuRa) SealHash(header *types.Header) common.Hash {
-	return SealHash(header)
-}
-
-// Close implements consensus.Engine. It's a noop for clique as there are no background threads.
-func (c *AuRa) Close() error {
-	common.SafeClose(c.exitCh)
-	return nil
-}
-
-// APIs implements consensus.Engine, returning the user facing RPC API to allow
-// controlling the signer voting.
-func (c *AuRa) APIs(chain consensus.ChainHeaderReader) []rpc.API {
-	return []rpc.API{
-		//{
-		//Namespace: "clique",
-		//Version:   "1.0",
-		//Service:   &API{chain: chain, clique: c},
-		//Public:    false,
-		//}
-	}
-}
diff --git a/consensus/clique/clique.go b/consensus/clique/clique.go
index 5d9434104e..87b55fbbc8 100644
--- a/consensus/clique/clique.go
+++ b/consensus/clique/clique.go
@@ -365,14 +365,14 @@ func (c *Clique) Prepare(chain consensus.ChainHeaderReader, header *types.Header
 
 // Finalize implements consensus.Engine, ensuring no uncles are set, nor block
 // rewards given.
-func (c *Clique) Finalize(_ *params.ChainConfig, header *types.Header, state *state.IntraBlockState, txs []types.Transaction, uncles []*types.Header) {
+func (c *Clique) Finalize(_ *params.ChainConfig, header *types.Header, state *state.IntraBlockState, txs []types.Transaction, uncles []*types.Header, syscall consensus.SystemCall) {
 	// No block rewards in PoA, so the state remains as is and uncles are dropped
 	header.UncleHash = types.CalcUncleHash(nil)
 }
 
 // FinalizeAndAssemble implements consensus.Engine, ensuring no uncles are set,
 // nor block rewards given, and returns the final block.
-func (c *Clique) FinalizeAndAssemble(chainConfig *params.ChainConfig, header *types.Header, state *state.IntraBlockState, txs []types.Transaction, uncles []*types.Header, receipts []*types.Receipt) (*types.Block, error) {
+func (c *Clique) 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)
 
diff --git a/consensus/consensus.go b/consensus/consensus.go
index d69b46478b..3e252065fd 100644
--- a/consensus/consensus.go
+++ b/consensus/consensus.go
@@ -58,6 +58,8 @@ type ChainReader interface {
 	HasBlock(hash common.Hash, number uint64) bool
 }
 
+type SystemCall func(address common.Address, in []byte) ([]byte, error)
+
 // Engine is an algorithm agnostic consensus engine.
 type Engine interface {
 	// Author retrieves the Ethereum address of the account that minted the given
@@ -89,7 +91,7 @@ type Engine interface {
 	//
 	// Note: The block header and state database might be updated to reflect any
 	// consensus rules that happen at finalization (e.g. block rewards).
-	Finalize(config *params.ChainConfig, header *types.Header, state *state.IntraBlockState, txs []types.Transaction, uncles []*types.Header)
+	Finalize(config *params.ChainConfig, header *types.Header, state *state.IntraBlockState, txs []types.Transaction, uncles []*types.Header, syscall SystemCall)
 
 	// FinalizeAndAssemble runs any post-transaction state modifications (e.g. block
 	// rewards) and assembles the final block.
@@ -97,7 +99,7 @@ type Engine interface {
 	// Note: The block header and state database might be updated to reflect any
 	// consensus rules that happen at finalization (e.g. block rewards).
 	FinalizeAndAssemble(config *params.ChainConfig, header *types.Header, state *state.IntraBlockState, txs []types.Transaction,
-		uncles []*types.Header, receipts []*types.Receipt) (*types.Block, error)
+		uncles []*types.Header, receipts []*types.Receipt, syscall SystemCall) (*types.Block, error)
 
 	// Seal generates a new sealing request for the given input block and pushes
 	// the result into the given channel.
diff --git a/consensus/ethash/consensus.go b/consensus/ethash/consensus.go
index 932b5138ba..ef3153d833 100644
--- a/consensus/ethash/consensus.go
+++ b/consensus/ethash/consensus.go
@@ -594,17 +594,17 @@ func (ethash *Ethash) Prepare(chain consensus.ChainHeaderReader, header *types.H
 
 // Finalize implements consensus.Engine, accumulating the block and uncle rewards,
 // setting the final state on the header
-func (ethash *Ethash) Finalize(config *params.ChainConfig, header *types.Header, state *state.IntraBlockState, txs []types.Transaction, uncles []*types.Header) {
+func (ethash *Ethash) Finalize(config *params.ChainConfig, header *types.Header, state *state.IntraBlockState, txs []types.Transaction, uncles []*types.Header, _ consensus.SystemCall) {
 	// Accumulate any block and uncle rewards and commit the final state root
 	accumulateRewards(config, state, header, uncles)
 }
 
 // FinalizeAndAssemble implements consensus.Engine, accumulating the block and
 // uncle rewards, setting the final state and assembling the block.
-func (ethash *Ethash) FinalizeAndAssemble(chainConfig *params.ChainConfig, header *types.Header, state *state.IntraBlockState, txs []types.Transaction, uncles []*types.Header, receipts []*types.Receipt) (*types.Block, error) {
+func (ethash *Ethash) 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) {
 
 	// Finalize block
-	ethash.Finalize(chainConfig, header, state, txs, uncles)
+	ethash.Finalize(chainConfig, header, state, txs, uncles, syscall)
 	// Header seems complete, assemble into a block and return
 	return types.NewBlock(header, txs, uncles, receipts), nil
 }
diff --git a/core/blockchain.go b/core/blockchain.go
index bea7329640..47b8fba13c 100644
--- a/core/blockchain.go
+++ b/core/blockchain.go
@@ -24,8 +24,10 @@ import (
 	"os"
 	"time"
 
+	"github.com/holiman/uint256"
 	"github.com/ledgerwatch/erigon/common"
 	"github.com/ledgerwatch/erigon/common/mclock"
+	"github.com/ledgerwatch/erigon/common/u256"
 	"github.com/ledgerwatch/erigon/consensus"
 	"github.com/ledgerwatch/erigon/consensus/misc"
 	"github.com/ledgerwatch/erigon/core/state"
@@ -117,7 +119,7 @@ func ExecuteBlockEphemerally(
 			writeTrace = true
 		}
 
-		receipt, err := ApplyTransaction(chainConfig, getHeader, engine, nil, gp, ibs, noop, header, tx, usedGas, *vmConfig, checkTEVM)
+		receipt, _, err := ApplyTransaction(chainConfig, getHeader, engine, nil, gp, ibs, noop, header, tx, usedGas, *vmConfig, checkTEVM)
 		if writeTrace {
 			w, err1 := os.Create(fmt.Sprintf("txtrace_%x.txt", tx.Hash()))
 			if err1 != nil {
@@ -166,9 +168,43 @@ func ExecuteBlockEphemerally(
 	return receipts, nil
 }
 
+func CallContract(contract common.Address, data []byte, chainConfig params.ChainConfig, ibs *state.IntraBlockState, header *types.Header, engine consensus.Engine) (result []byte, err error) {
+	gp := new(GasPool)
+	gp.AddGas(50_000_000)
+	var gasUsed uint64
+
+	if chainConfig.DAOForkSupport && chainConfig.DAOForkBlock != nil && chainConfig.DAOForkBlock.Cmp(header.Number) == 0 {
+		misc.ApplyDAOHardFork(ibs)
+	}
+	noop := state.NewNoopWriter()
+	tx, err := CallContractTx(contract, data, ibs)
+	if err != nil {
+		return nil, fmt.Errorf("CallContract: %w ", err)
+	}
+	// Set infinite balance to the fake caller account.
+	from := ibs.GetOrNewStateObject(common.Address{})
+	from.SetBalance(uint256.NewInt(0).SetAllOne())
+
+	_, result, err = ApplyTransaction(&chainConfig, nil, engine, nil, gp, ibs, noop, header, tx, &gasUsed, vm.Config{}, nil)
+	if err != nil {
+		return result, fmt.Errorf("CallContract: %w ", err)
+	}
+	return result, nil
+}
+
+// from the null sender, with 50M gas.
+func CallContractTx(contract common.Address, data []byte, ibs *state.IntraBlockState) (tx types.Transaction, err error) {
+	var from common.Address
+	nonce := ibs.GetNonce(from)
+	tx = types.NewTransaction(nonce, contract, u256.Num0, 50_000_000, u256.Num1, data)
+	return tx.FakeSign(from)
+}
+
 func FinalizeBlockExecution(engine consensus.Engine, header *types.Header, txs types.Transactions, uncles []*types.Header, stateWriter state.WriterWithChangeSets, cc *params.ChainConfig, ibs *state.IntraBlockState) error {
 	// Finalize the block, applying any consensus engine specific extras (e.g. block rewards)
-	engine.Finalize(cc, header, ibs, txs, uncles)
+	engine.Finalize(cc, header, ibs, txs, uncles, func(contract common.Address, data []byte) ([]byte, error) {
+		return CallContract(contract, data, *cc, ibs, header, engine)
+	})
 
 	ctx := cc.WithEIPsFlags(context.Background(), header.Number.Uint64())
 	if err := ibs.CommitBlock(ctx, stateWriter); err != nil {
diff --git a/core/chain_makers.go b/core/chain_makers.go
index 71ff6f03f5..306593b63d 100644
--- a/core/chain_makers.go
+++ b/core/chain_makers.go
@@ -111,7 +111,7 @@ func (b *BlockGen) AddTxWithChain(getHeader func(hash common.Hash, number uint64
 	}
 	b.ibs.Prepare(tx.Hash(), common.Hash{}, len(b.txs))
 	checkTEVM := func(_ common.Hash) (bool, error) { return false, nil }
-	receipt, err := ApplyTransaction(b.config, getHeader, engine, &b.header.Coinbase, b.gasPool, b.ibs, state.NewNoopWriter(), b.header, tx, &b.header.GasUsed, vm.Config{}, checkTEVM)
+	receipt, _, err := ApplyTransaction(b.config, getHeader, engine, &b.header.Coinbase, b.gasPool, b.ibs, state.NewNoopWriter(), b.header, tx, &b.header.GasUsed, vm.Config{}, checkTEVM)
 	if err != nil {
 		panic(err)
 	}
@@ -125,7 +125,7 @@ func (b *BlockGen) AddFailedTxWithChain(getHeader func(hash common.Hash, number
 	}
 	b.ibs.Prepare(tx.Hash(), common.Hash{}, len(b.txs))
 	checkTEVM := func(common.Hash) (bool, error) { return false, nil }
-	receipt, err := ApplyTransaction(b.config, getHeader, engine, &b.header.Coinbase, b.gasPool, b.ibs, state.NewNoopWriter(), b.header, tx, &b.header.GasUsed, vm.Config{}, checkTEVM)
+	receipt, _, err := ApplyTransaction(b.config, getHeader, engine, &b.header.Coinbase, b.gasPool, b.ibs, state.NewNoopWriter(), b.header, tx, &b.header.GasUsed, vm.Config{}, checkTEVM)
 	_ = err // accept failed transactions
 	b.txs = append(b.txs, tx)
 	b.receipts = append(b.receipts, receipt)
@@ -276,7 +276,7 @@ func GenerateChain(config *params.ChainConfig, parent *types.Block, engine conse
 		}
 		if b.engine != nil {
 			// Finalize and seal the block
-			if _, err := b.engine.FinalizeAndAssemble(config, b.header, ibs, b.txs, b.uncles, b.receipts); err != nil {
+			if _, err := b.engine.FinalizeAndAssemble(config, b.header, ibs, b.txs, b.uncles, b.receipts, nil); err != nil {
 				return nil, nil, fmt.Errorf("call to FinaliseAndAssemble: %w", err)
 			}
 			ctx := config.WithEIPsFlags(context.Background(), b.header.Number.Uint64())
diff --git a/core/genesis.go b/core/genesis.go
index 88ce6ee141..24839a58ff 100644
--- a/core/genesis.go
+++ b/core/genesis.go
@@ -186,6 +186,22 @@ func MustCommitGenesisBlock(db ethdb.RwKV, genesis *Genesis, history bool) (*par
 	return c, b
 }
 
+func OverrideGenesisBlock(db ethdb.RwTx, genesis *Genesis, history bool) (*params.ChainConfig, *types.Block, error) {
+	stored, err := rawdb.ReadCanonicalHash(db, 0)
+	if err != nil {
+		return nil, nil, err
+	}
+	err = rawdb.DeleteCanonicalHash(db, 0)
+	if err != nil {
+		return nil, nil, err
+	}
+	err = rawdb.DeleteChainConfig(db, stored)
+	if err != nil {
+		return nil, nil, err
+	}
+	return WriteGenesisBlock(db, genesis, history)
+}
+
 func WriteGenesisBlock(db ethdb.RwTx, genesis *Genesis, history bool) (*params.ChainConfig, *types.Block, error) {
 	if genesis != nil && genesis.Config == nil {
 		return params.AllEthashProtocolChanges, nil, ErrGenesisNoConfig
@@ -309,7 +325,10 @@ func (g *Genesis) ToBlock() (*types.Block, *state.IntraBlockState, error) {
 		r, w := state.NewDbStateReader(kv.WrapIntoTxDB(tx)), state.NewDbStateWriter(kv.WrapIntoTxDB(tx), 0)
 		statedb = state.New(r)
 		for addr, account := range g.Alloc {
-			balance, _ := uint256.FromBig(account.Balance)
+			balance, overflow := uint256.FromBig(account.Balance)
+			if overflow {
+				panic("overflow at genesis allocs")
+			}
 			statedb.AddBalance(addr, balance)
 			statedb.SetCode(addr, account.Code)
 			statedb.SetNonce(addr, account.Nonce)
@@ -392,7 +411,6 @@ func (g *Genesis) WriteGenesisState(tx ethdb.RwTx, history bool) (*types.Block,
 	if err := statedb.CommitBlock(context.Background(), blockWriter); err != nil {
 		return nil, statedb, fmt.Errorf("cannot write state: %v", err)
 	}
-
 	if err := blockWriter.WriteChangeSets(); err != nil {
 		return nil, statedb, fmt.Errorf("cannot write change sets: %v", err)
 	}
@@ -451,20 +469,20 @@ func (g *Genesis) Write(tx ethdb.RwTx, history bool) (*types.Block, *state.Intra
 	return block, statedb, nil
 }
 
-func (g *Genesis) Commit(db ethdb.Database, history bool) (*types.Block, *state.IntraBlockState, error) {
+func (g *Genesis) Commit(db ethdb.Database, history bool) (*types.Block, error) {
 	tx, err := db.Begin(context.Background(), ethdb.RW)
 	if err != nil {
-		return nil, nil, err
+		return nil, err
 	}
 	defer tx.Rollback()
-	block, statedb, err := g.Write(tx.(ethdb.HasTx).Tx().(ethdb.RwTx), history)
+	block, _, err := g.Write(tx.(ethdb.HasTx).Tx().(ethdb.RwTx), history)
 	if err != nil {
-		return block, statedb, err
+		return block, err
 	}
 	if err := tx.Commit(); err != nil {
-		return block, statedb, err
+		return block, err
 	}
-	return block, statedb, nil
+	return block, nil
 }
 
 // MustCommit writes the genesis block and state to db, panicking on error.
diff --git a/core/rawdb/accessors_metadata.go b/core/rawdb/accessors_metadata.go
index 281a0afbc4..4ded1ceb23 100644
--- a/core/rawdb/accessors_metadata.go
+++ b/core/rawdb/accessors_metadata.go
@@ -57,3 +57,8 @@ func WriteChainConfig(db ethdb.Putter, hash common.Hash, cfg *params.ChainConfig
 	}
 	return nil
 }
+
+// DeleteChainConfig retrieves the consensus settings based on the given genesis hash.
+func DeleteChainConfig(db ethdb.Deleter, hash common.Hash) error {
+	return db.Delete(dbutils.ConfigPrefix, hash[:], nil)
+}
diff --git a/core/state_processor.go b/core/state_processor.go
index a649f49d61..7f868ef454 100644
--- a/core/state_processor.go
+++ b/core/state_processor.go
@@ -85,14 +85,13 @@ func FormatLogs(logs []vm.StructLog) []StructLogRes {
 // and uses the input parameters for its environment. It returns the receipt
 // for the transaction, gas used and an error if the transaction failed,
 // indicating the block was invalid.
-func applyTransaction(config *params.ChainConfig, gp *GasPool, statedb *state.IntraBlockState, stateWriter state.StateWriter, header *types.Header, tx types.Transaction, usedGas *uint64, evm *vm.EVM, cfg vm.Config) (*types.Receipt, error) {
+func applyTransaction(config *params.ChainConfig, gp *GasPool, statedb *state.IntraBlockState, stateWriter state.StateWriter, header *types.Header, tx types.Transaction, usedGas *uint64, evm *vm.EVM, cfg vm.Config) (*types.Receipt, []byte, error) {
 	msg, err := tx.AsMessage(*types.MakeSigner(config, header.Number.Uint64()), header.BaseFee)
 	if err != nil {
-		return nil, err
+		return nil, nil, err
 	}
 
 	ctx := config.WithEIPsFlags(context.Background(), header.Number.Uint64())
-	// Create a new context to be used in the EVM environment
 	txContext := NewEVMTxContext(msg)
 	if cfg.TraceJumpDest {
 		txContext.TxHash = tx.Hash()
@@ -106,11 +105,11 @@ func applyTransaction(config *params.ChainConfig, gp *GasPool, statedb *state.In
 	// If the transaction created a contract, store the creation address in the receipt.
 	result, err := ApplyMessage(evm, msg, gp, true /* refunds */, false /* gasBailout */)
 	if err != nil {
-		return nil, err
+		return nil, nil, err
 	}
 	// Update the state with pending changes
 	if err = statedb.FinalizeTx(ctx, stateWriter); err != nil {
-		return nil, err
+		return nil, nil, err
 	}
 
 	*usedGas += result.UsedGas
@@ -138,16 +137,17 @@ func applyTransaction(config *params.ChainConfig, gp *GasPool, statedb *state.In
 		receipt.BlockNumber = header.Number
 		receipt.TransactionIndex = uint(statedb.TxIndex())
 	}
-	return receipt, err
+	return receipt, result.ReturnData, err
 }
 
 // ApplyTransaction attempts to apply a transaction to the given state database
 // and uses the input parameters for its environment. It returns the receipt
 // for the transaction, gas used and an error if the transaction failed,
 // indicating the block was invalid.
-func ApplyTransaction(config *params.ChainConfig, getHeader func(hash common.Hash, number uint64) *types.Header, engine consensus.Engine, author *common.Address, gp *GasPool, ibs *state.IntraBlockState, stateWriter state.StateWriter, header *types.Header, tx types.Transaction, usedGas *uint64, cfg vm.Config, checkTEVM func(contractHash common.Hash) (bool, error)) (*types.Receipt, error) {
+func ApplyTransaction(config *params.ChainConfig, getHeader func(hash common.Hash, number uint64) *types.Header, engine consensus.Engine, author *common.Address, gp *GasPool, ibs *state.IntraBlockState, stateWriter state.StateWriter, header *types.Header, tx types.Transaction, usedGas *uint64, cfg vm.Config, checkTEVM func(contractHash common.Hash) (bool, error)) (*types.Receipt, []byte, error) {
 	// Create a new context to be used in the EVM environment
 	blockContext := NewEVMBlockContext(header, getHeader, engine, author, checkTEVM)
 	vmenv := vm.NewEVM(blockContext, vm.TxContext{}, ibs, config, cfg)
+
 	return applyTransaction(config, gp, ibs, stateWriter, header, tx, usedGas, vmenv, cfg)
 }
diff --git a/core/types/access_list_tx.go b/core/types/access_list_tx.go
index 4fa94e195d..0a79b589fb 100644
--- a/core/types/access_list_tx.go
+++ b/core/types/access_list_tx.go
@@ -26,6 +26,7 @@ import (
 
 	"github.com/holiman/uint256"
 	"github.com/ledgerwatch/erigon/common"
+	"github.com/ledgerwatch/erigon/common/u256"
 	"github.com/ledgerwatch/erigon/rlp"
 )
 
@@ -569,6 +570,14 @@ func (tx *AccessListTx) WithSignature(signer Signer, sig []byte) (Transaction, e
 	cpy.ChainID = signer.ChainID()
 	return cpy, nil
 }
+func (tx *AccessListTx) FakeSign(address common.Address) (Transaction, error) {
+	cpy := tx.copy()
+	cpy.R.Set(u256.Num1)
+	cpy.S.Set(u256.Num1)
+	cpy.V.Set(u256.Num4)
+	cpy.from.Store(address)
+	return cpy, nil
+}
 
 // Hash computes the hash (but not for signatures!)
 func (tx AccessListTx) Hash() common.Hash {
diff --git a/core/types/dynamic_fee_tx.go b/core/types/dynamic_fee_tx.go
index 63f062f2f7..e7d993e30a 100644
--- a/core/types/dynamic_fee_tx.go
+++ b/core/types/dynamic_fee_tx.go
@@ -25,6 +25,7 @@ import (
 
 	"github.com/holiman/uint256"
 	"github.com/ledgerwatch/erigon/common"
+	"github.com/ledgerwatch/erigon/common/u256"
 	"github.com/ledgerwatch/erigon/rlp"
 )
 
@@ -233,6 +234,15 @@ func (tx *DynamicFeeTransaction) WithSignature(signer Signer, sig []byte) (Trans
 	return cpy, nil
 }
 
+func (tx *DynamicFeeTransaction) FakeSign(address common.Address) (Transaction, error) {
+	cpy := tx.copy()
+	cpy.R.Set(u256.Num1)
+	cpy.S.Set(u256.Num1)
+	cpy.V.Set(u256.Num4)
+	cpy.from.Store(address)
+	return cpy, nil
+}
+
 // MarshalBinary returns the canonical encoding of the transaction.
 // For legacy transactions, it returns the RLP encoding. For EIP-2718 typed
 // transactions, it returns the type and payload.
diff --git a/core/types/legacy_tx.go b/core/types/legacy_tx.go
index 78d8f73de6..7fddc1c3bc 100644
--- a/core/types/legacy_tx.go
+++ b/core/types/legacy_tx.go
@@ -25,6 +25,7 @@ import (
 
 	"github.com/holiman/uint256"
 	"github.com/ledgerwatch/erigon/common"
+	"github.com/ledgerwatch/erigon/common/u256"
 	"github.com/ledgerwatch/erigon/rlp"
 )
 
@@ -457,6 +458,15 @@ func (tx *LegacyTx) WithSignature(signer Signer, sig []byte) (Transaction, error
 	return cpy, nil
 }
 
+func (tx *LegacyTx) FakeSign(address common.Address) (Transaction, error) {
+	cpy := tx.copy()
+	cpy.R.Set(u256.Num1)
+	cpy.S.Set(u256.Num1)
+	cpy.V.Set(u256.Num4)
+	cpy.from.Store(address)
+	return cpy, nil
+}
+
 // Hash computes the hash (but not for signatures!)
 func (tx LegacyTx) Hash() common.Hash {
 	if hash := tx.hash.Load(); hash != nil {
diff --git a/core/types/transaction.go b/core/types/transaction.go
index a33bf95ab8..db0a308de3 100644
--- a/core/types/transaction.go
+++ b/core/types/transaction.go
@@ -62,6 +62,7 @@ type Transaction interface {
 	GetTo() *common.Address
 	AsMessage(s Signer, baseFee *big.Int) (Message, error)
 	WithSignature(signer Signer, sig []byte) (Transaction, error)
+	FakeSign(address common.Address) (Transaction, error)
 	Hash() common.Hash
 	SigningHash(chainID *big.Int) common.Hash
 	Size() common.StorageSize
diff --git a/eth/backend.go b/eth/backend.go
index fcff82593a..c3b6ee13cc 100644
--- a/eth/backend.go
+++ b/eth/backend.go
@@ -187,6 +187,7 @@ func New(stack *node.Node, config *ethconfig.Config) (*Ethereum, error) {
 	if chainConfig.Clique != nil {
 		consensusConfig = &config.Clique
 	} else if chainConfig.Aura != nil {
+		config.Aura.Etherbase = config.Miner.Etherbase
 		consensusConfig = &config.Aura
 	} else {
 		consensusConfig = &config.Ethash
diff --git a/eth/ethconfig/config.go b/eth/ethconfig/config.go
index 11a61bc45c..1dd181c64f 100644
--- a/eth/ethconfig/config.go
+++ b/eth/ethconfig/config.go
@@ -18,6 +18,7 @@
 package ethconfig
 
 import (
+	"encoding/json"
 	"math/big"
 	"os"
 	"os/user"
@@ -27,6 +28,9 @@ import (
 
 	"github.com/c2h5oh/datasize"
 	"github.com/davecgh/go-spew/spew"
+	"github.com/ledgerwatch/erigon/consensus/aura"
+	"github.com/ledgerwatch/erigon/consensus/aura/consensusconfig"
+
 	"github.com/ledgerwatch/erigon/common"
 	"github.com/ledgerwatch/erigon/consensus"
 	"github.com/ledgerwatch/erigon/consensus/clique"
@@ -208,7 +212,19 @@ func CreateConsensusEngine(chainConfig *params.ChainConfig, config interface{},
 		}
 	case *params.AuRaConfig:
 		if chainConfig.Aura != nil {
-			eng = clique.NewAuRa(chainConfig, db.OpenDatabase(consensusCfg.DBPath, consensusCfg.InMemory))
+			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)
+			if err != nil {
+				panic(err)
+			}
 		}
 	}
 
diff --git a/eth/stagedsync/stage_execute.go b/eth/stagedsync/stage_execute.go
index fcb18b519f..387679051a 100644
--- a/eth/stagedsync/stage_execute.go
+++ b/eth/stagedsync/stage_execute.go
@@ -118,6 +118,7 @@ func executeBlock(
 		cfg.vmConfig.Debug = true
 		cfg.vmConfig.Tracer = callTracer
 	}
+
 	receipts, err := core.ExecuteBlockEphemerally(cfg.chainConfig, cfg.vmConfig, getHeader, cfg.engine, block, stateReader, stateWriter, checkTEVM)
 	if err != nil {
 		return err
diff --git a/eth/stagedsync/stage_interhashes.go b/eth/stagedsync/stage_interhashes.go
index b0ca82c572..805b25b26e 100644
--- a/eth/stagedsync/stage_interhashes.go
+++ b/eth/stagedsync/stage_interhashes.go
@@ -88,7 +88,7 @@ func SpawnIntermediateHashesStage(s *StageState, u Unwinder, tx ethdb.RwTx, cfg
 
 	if err == nil {
 		if cfg.checkRoot && root != expectedRootHash {
-			log.Error(fmt.Sprintf("[%s] Wrong trie root: %x, expected (from header): %x", logPrefix, root, expectedRootHash))
+			log.Error(fmt.Sprintf("[%s] Wrong trie root of block %d: %x, expected (from header): %x", logPrefix, to, root, expectedRootHash))
 			if to > s.BlockNumber {
 				log.Warn("Unwinding due to incorrect root hash", "to", to-1)
 				if err = u.UnwindTo(to-1, tx, headerHash); err != nil {
diff --git a/eth/stagedsync/stage_mining_exec.go b/eth/stagedsync/stage_mining_exec.go
index d829ece614..7ee33456b1 100644
--- a/eth/stagedsync/stage_mining_exec.go
+++ b/eth/stagedsync/stage_mining_exec.go
@@ -152,7 +152,7 @@ func addTransactionsToMiningBlock(current *miningBlock, chainConfig params.Chain
 
 	var miningCommitTx = func(txn types.Transaction, coinbase common.Address, vmConfig *vm.Config, chainConfig params.ChainConfig, ibs *state.IntraBlockState, current *miningBlock) ([]*types.Log, error) {
 		snap := ibs.Snapshot()
-		receipt, err := core.ApplyTransaction(&chainConfig, getHeader, engine, &coinbase, gasPool, ibs, noop, header, txn, &header.GasUsed, *vmConfig, checkTEVM)
+		receipt, _, err := core.ApplyTransaction(&chainConfig, getHeader, engine, &coinbase, gasPool, ibs, noop, header, txn, &header.GasUsed, *vmConfig, checkTEVM)
 		if err != nil {
 			ibs.RevertToSnapshot(snap)
 			return nil, err
diff --git a/go.mod b/go.mod
index 77c6724902..d4619fb9f8 100644
--- a/go.mod
+++ b/go.mod
@@ -59,6 +59,7 @@ require (
 	github.com/urfave/cli v1.22.4
 	github.com/valyala/fastjson v1.6.3
 	github.com/wcharczuk/go-chart v2.0.1+incompatible
+	go.uber.org/atomic v1.8.0
 	golang.org/x/crypto v0.0.0-20210322153248-0c34fe9e7dc2
 	golang.org/x/sync v0.0.0-20201207232520-09787c993a3a
 	golang.org/x/sys v0.0.0-20210420205809-ac73e9fd8988
diff --git a/go.sum b/go.sum
index 7a9acce2e7..54865c71e7 100644
--- a/go.sum
+++ b/go.sum
@@ -1038,6 +1038,8 @@ go.uber.org/atomic v1.3.2/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE=
 go.uber.org/atomic v1.4.0/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE=
 go.uber.org/atomic v1.5.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ=
 go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ=
+go.uber.org/atomic v1.8.0 h1:CUhrE4N1rqSE6FM9ecihEjRkLQu8cDfgDyoOs83mEY4=
+go.uber.org/atomic v1.8.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc=
 go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0=
 go.uber.org/multierr v1.3.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4=
 go.uber.org/multierr v1.5.0/go.mod h1:FeouvMocqHpRaaGuG9EjoKcStLC43Zu/fmqdUMPcKYU=
diff --git a/gointerfaces/types/types.pb.go b/gointerfaces/types/types.pb.go
index 70944af3de..66accbf5f6 100644
--- a/gointerfaces/types/types.pb.go
+++ b/gointerfaces/types/types.pb.go
@@ -7,11 +7,12 @@
 package types
 
 import (
-	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
-	protoimpl "google.golang.org/protobuf/runtime/protoimpl"
-	descriptorpb "google.golang.org/protobuf/types/descriptorpb"
-	reflect "reflect"
-	sync "sync"
+	"reflect"
+	"sync"
+
+	"google.golang.org/protobuf/reflect/protoreflect"
+	"google.golang.org/protobuf/runtime/protoimpl"
+	"google.golang.org/protobuf/types/descriptorpb"
 )
 
 const (
diff --git a/params/config.go b/params/config.go
index 7bca2fea29..5f70d3eb2e 100644
--- a/params/config.go
+++ b/params/config.go
@@ -285,21 +285,49 @@ var (
 	}
 
 	SokolChainConfig = &ChainConfig{
-		ChainName:           SokolChainName,
-		ChainID:             big.NewInt(77),
-		HomesteadBlock:      big.NewInt(0),
-		DAOForkBlock:        nil,
-		DAOForkSupport:      true,
-		EIP150Block:         big.NewInt(0),
-		EIP155Block:         big.NewInt(0),
-		EIP158Block:         big.NewInt(0),
+		ChainName:      SokolChainName,
+		ChainID:        big.NewInt(77),
+		HomesteadBlock: big.NewInt(0),
+		DAOForkBlock:   nil,
+		DAOForkSupport: true,
+		EIP150Block:    big.NewInt(0),
+		EIP155Block:    big.NewInt(0),
+		EIP158Block:    big.NewInt(0),
+		/*
+							"eip140Transition": "0x0",  in Byzantium
+							"eip211Transition": "0x0",  in Byzantium
+							"eip214Transition": "0x0",  in Byzantium
+							"eip658Transition": "0x0",  in Byzantium
+			Byzantium also has EIP-100, EIP-196, EIP-197, EIP-198, EIP-649
+
+							"eip145Transition": 6464300,  in Constantinople
+							"eip1014Transition": 6464300, in Constantinople
+							"eip1052Transition": 6464300, in Constantinople
+							"eip1283Transition": 6464300, in Constantinople
+			Constantinople also has EIP-1234 (bomb)
+
+							"eip1283DisableTransition": 7026400, in Petersburg
+			Petersburg has nothing else
+
+							"eip1283ReenableTransition": 12095200, ????
+
+							"eip1344Transition": 12095200, in Istanbul
+							"eip1706Transition": 12095200,  ???? [EIP-2200] has superseded [EIP-1706]
+							"eip1884Transition": 12095200, in Istanbul
+							"eip2028Transition": 12095200, in Istanbul
+			Istanbul also has 152, 1108, 2200
+
+							"eip2929Transition": 21050600, in Berlin ?
+							"eip2930Transition": 21050600  in Berlin ?
+			Berlin also has  663, 1057, 1380, 1702, 1962, 1985, 2045, 2046
+			London : not in list
+		*/
 		ByzantiumBlock:      big.NewInt(0),
-		ConstantinopleBlock: big.NewInt(0),
-		PetersburgBlock:     big.NewInt(6464300),
-
-		IstanbulBlock:    big.NewInt(12095200),
-		MuirGlacierBlock: nil,
-		BerlinBlock:      big.NewInt(21050600),
+		ConstantinopleBlock: big.NewInt(6464300),
+		PetersburgBlock:     big.NewInt(7026400),
+		IstanbulBlock:       big.NewInt(12095200),
+		MuirGlacierBlock:    nil,
+		BerlinBlock:         big.NewInt(21050600),
 		//LondonBlock:         big.NewInt(21050600),
 		Aura: &AuRaConfig{},
 	}
@@ -472,8 +500,9 @@ type CliqueConfig struct {
 
 // AuRaConfig is the consensus engine configs for proof-of-authority based sealing.
 type AuRaConfig struct {
-	DBPath   string
-	InMemory bool
+	DBPath    string
+	InMemory  bool
+	Etherbase common.Address // same as miner etherbase
 }
 
 // String implements the stringer interface, returning the consensus engine details.
diff --git a/turbo/adapter/chain_context.go b/turbo/adapter/chain_context.go
index 20ce9de223..2b446e4a00 100644
--- a/turbo/adapter/chain_context.go
+++ b/turbo/adapter/chain_context.go
@@ -42,11 +42,11 @@ func (c *powEngine) VerifySeal(chain consensus.ChainHeaderReader, header *types.
 func (c *powEngine) Prepare(chain consensus.ChainHeaderReader, header *types.Header) error {
 	panic("must not be called")
 }
-func (c *powEngine) Finalize(chainConfig *params.ChainConfig, header *types.Header, state *state.IntraBlockState, txs []types.Transaction, uncles []*types.Header) {
+func (c *powEngine) Finalize(chainConfig *params.ChainConfig, header *types.Header, state *state.IntraBlockState, txs []types.Transaction, uncles []*types.Header, syscall consensus.SystemCall) {
 	panic("must not be called")
 }
 func (c *powEngine) FinalizeAndAssemble(chainConfig *params.ChainConfig, header *types.Header, state *state.IntraBlockState, txs []types.Transaction,
-	uncles []*types.Header, receipts []*types.Receipt) (*types.Block, error) {
+	uncles []*types.Header, receipts []*types.Receipt, syscall consensus.SystemCall) (*types.Block, error) {
 	panic("must not be called")
 }
 
-- 
GitLab