From d2552196ad7ea95b732468566155138a395eb074 Mon Sep 17 00:00:00 2001 From: Alex Sharov Date: Wed, 21 Jul 2021 18:13:26 +0700 Subject: [PATCH] Sokol v0: support for first epoch-set transition (#2411) --- .golangci.yml | 1 + cmd/hack/hack.go | 2 +- cmd/integration/commands/reset_state.go | 8 +- cmd/integration/commands/state_stages.go | 3 + cmd/snapshots/debug/debug_test.go | 2 +- cmd/state/commands/opcode_tracer.go | 2 +- common/dbutils/bucket.go | 4 +- common/etl/collector.go | 2 +- consensus/aura/aura.go | 566 +++++++++++++++++------ consensus/aura/config.go | 15 +- consensus/aura/finality_test.go | 72 +++ consensus/aura/validators.go | 440 +++++++++--------- consensus/clique/clique.go | 12 +- consensus/consensus.go | 11 +- consensus/ethash/consensus.go | 14 +- core/blockchain.go | 22 +- core/chain_makers.go | 4 +- core/rawdb/accessors_chain.go | 47 +- eth/stagedsync/stage_call_traces.go | 1 + eth/stagedsync/stage_execute.go | 2 +- eth/stagedsync/stage_headers.go | 9 + eth/stagedsync/stage_mining_exec.go | 2 +- eth/stagedsync/stage_txlookup.go | 4 +- migrations/receipt_repair.go | 2 +- turbo/adapter/chain_context.go | 14 +- 25 files changed, 865 insertions(+), 396 deletions(-) create mode 100644 consensus/aura/finality_test.go diff --git a/.golangci.yml b/.golangci.yml index 1b30d347a..97e3f9557 100644 --- a/.golangci.yml +++ b/.golangci.yml @@ -15,6 +15,7 @@ linters: - unused - varcheck - gocritic +# - forcetypeassert linters-settings: gocritic: diff --git a/cmd/hack/hack.go b/cmd/hack/hack.go index 307f21c2c..948e603ce 100644 --- a/cmd/hack/hack.go +++ b/cmd/hack/hack.go @@ -2257,7 +2257,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, nil, nil); err != nil { + if _, err := engine.FinalizeAndAssemble(chainConfig, header, ibs, block.Transactions(), block.Uncles(), receipts, nil, nil, nil, nil); err != nil { return nil, fmt.Errorf("finalize of block %d failed: %v", block.NumberU64(), err) } diff --git a/cmd/integration/commands/reset_state.go b/cmd/integration/commands/reset_state.go index 77a698284..afbca3295 100644 --- a/cmd/integration/commands/reset_state.go +++ b/cmd/integration/commands/reset_state.go @@ -138,6 +138,9 @@ func resetExec(tx ethdb.RwTx, g *core.Genesis) error { if err := tx.(ethdb.BucketMigrator).ClearBucket(dbutils.Epoch); err != nil { return err } + if err := tx.(ethdb.BucketMigrator).ClearBucket(dbutils.PendingEpoch); err != nil { + return err + } if err := stages.SaveStageProgress(tx, stages.Execution, 0); err != nil { return err } @@ -247,6 +250,7 @@ func printStages(db ethdb.KVGetter) error { defer w.Flush() w.Init(os.Stdout, 8, 8, 0, '\t', 0) fmt.Fprintf(w, "Note: prune_at doesn't mean 'all data before were deleted' - it just mean stage.Prune function were run to this block. Because 1 stage may prune multiple data types to different prune distance.\n") + fmt.Fprint(w, "\n \t stage_at \t prune_at\n") for _, stage := range stages.AllStages { if progress, err = stages.GetStageProgress(db, stage); err != nil { return err @@ -255,13 +259,13 @@ func printStages(db ethdb.KVGetter) error { if err != nil { return err } - fmt.Fprintf(w, "%s \t %d \t prune_at=%d\n", string(stage), progress, prunedTo) + fmt.Fprintf(w, "%s \t %d \t %d\n", string(stage), progress, prunedTo) } pm, err := prune.Get(db) if err != nil { return err } fmt.Fprintf(w, "--\n") - fmt.Fprintf(w, "prune distance: %#v\n", pm) + fmt.Fprintf(w, "prune distance: %s\n\n", pm.ToString()) return nil } diff --git a/cmd/integration/commands/state_stages.go b/cmd/integration/commands/state_stages.go index 1fa961a48..d6a8aa703 100644 --- a/cmd/integration/commands/state_stages.go +++ b/cmd/integration/commands/state_stages.go @@ -156,6 +156,9 @@ func syncBySmallSteps(db ethdb.RwKV, miningConfig params.MiningConfig, ctx conte expectedAccountChanges := make(map[uint64]*changeset.ChangeSet) expectedStorageChanges := make(map[uint64]*changeset.ChangeSet) changeSetHook := func(blockNum uint64, csw *state.ChangeSetWriter) { + if csw == nil { + return + } accountChanges, err := csw.GetAccountChanges() if err != nil { panic(err) diff --git a/cmd/snapshots/debug/debug_test.go b/cmd/snapshots/debug/debug_test.go index ec0e6eca1..fdc734138 100644 --- a/cmd/snapshots/debug/debug_test.go +++ b/cmd/snapshots/debug/debug_test.go @@ -120,7 +120,7 @@ func TestMatreshkaStream(t *testing.T) { checkTEVM := ethdb.GetCheckTEVM(tx) - _, err = core.ExecuteBlockEphemerally(chainConfig, &vm.Config{NoReceipts: true}, getHeader, ethash.NewFaker(), block, stateReaderWriter, stateReaderWriter, nil, checkTEVM) + _, err = core.ExecuteBlockEphemerally(chainConfig, &vm.Config{NoReceipts: true}, getHeader, ethash.NewFaker(), block, stateReaderWriter, stateReaderWriter, nil, nil, checkTEVM) if err != nil { t.Fatal(err, currentBlock) } diff --git a/cmd/state/commands/opcode_tracer.go b/cmd/state/commands/opcode_tracer.go index 826b35596..5b650cd10 100644 --- a/cmd/state/commands/opcode_tracer.go +++ b/cmd/state/commands/opcode_tracer.go @@ -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, nil, nil); err != nil { + if _, err := engine.FinalizeAndAssemble(chainConfig, header, ibs, block.Transactions(), block.Uncles(), receipts, nil, nil, nil, nil); err != nil { return nil, fmt.Errorf("finalize of block %d failed: %v", block.NumberU64(), err) } diff --git a/common/dbutils/bucket.go b/common/dbutils/bucket.go index 5d3acafa1..c47ec60b3 100644 --- a/common/dbutils/bucket.go +++ b/common/dbutils/bucket.go @@ -278,7 +278,8 @@ const ( Sequence = "Sequence" // tbl_name -> seq_u64 HeadHeaderKey = "LastHeader" - Epoch = "DevEpoch" // block_num_u64+block_hash->transition_proof + Epoch = "DevEpoch" // block_num_u64+block_hash->transition_proof + PendingEpoch = "DevPendingEpoch" // block_num_u64+block_hash->transition_proof ) // Keys @@ -348,6 +349,7 @@ var Buckets = []string{ HeadersBucket, HeaderTDBucket, Epoch, + PendingEpoch, } // DeprecatedBuckets - list of buckets which can be programmatically deleted - for example after migration diff --git a/common/etl/collector.go b/common/etl/collector.go index 9a47bbe23..6618dbea2 100644 --- a/common/etl/collector.go +++ b/common/etl/collector.go @@ -131,7 +131,7 @@ func (c *Collector) Close(logPrefix string) { totalSize += p.Dispose() } if totalSize > 0 { - log.Info(fmt.Sprintf("[%s] etl: temp files removed successfully", logPrefix), "total size", datasize.ByteSize(totalSize).HumanReadable()) + log.Info(fmt.Sprintf("[%s] etl: temp files removed", logPrefix), "total size", datasize.ByteSize(totalSize).HumanReadable()) } } diff --git a/consensus/aura/aura.go b/consensus/aura/aura.go index 8ea293a45..630d1dfc8 100644 --- a/consensus/aura/aura.go +++ b/consensus/aura/aura.go @@ -18,6 +18,7 @@ package aura import ( "bytes" + "container/list" "encoding/json" "fmt" "math/big" @@ -45,6 +46,8 @@ import ( "go.uber.org/atomic" ) +const DEBUG_LOG_FROM = 999_999_999 + /* Not implemented features from OS: - two_thirds_majority_transition - because no chains in OE where this is != MaxUint64 - means 1/2 majority used everywhere @@ -59,18 +62,26 @@ type StepDurationInfo struct { StepDuration uint64 } -// Holds 2 proofs inside: ValidatorSetProof and FinalityProof +// EpochTransitionProof - Holds 2 proofs inside: ValidatorSetProof and FinalityProof type EpochTransitionProof struct { SignalNumber uint64 SetProof []byte FinalityProof []byte } -// SetProof - validator set proof +// ValidatorSetProof - validator set proof type ValidatorSetProof struct { Header *types.Header Receipts types.Receipts } + +// FirstValidatorSetProof state-dependent proofs for the safe contract: +// only "first" proofs are such. +type FirstValidatorSetProof struct { // TODO: whaaat? here is no state! + ContractAddress common.Address + Header *types.Header +} + type EpochTransition struct { /// Block hash at which the transition occurred. BlockHash common.Hash @@ -161,7 +172,7 @@ func (r ReceivedStepHashes) dropAncient(step uint64) { type EpochManager struct { epochTransitionHash common.Hash // H256, epochTransitionNumber uint64 // BlockNumber - finalityChecker RollingFinality + finalityChecker *RollingFinality force bool } @@ -172,10 +183,12 @@ func NewEpochManager() *EpochManager { } } +func (e *EpochManager) noteNewEpoch() { e.force = true } + // zoomValidators - Zooms to the epoch after the header with the given hash. Returns true if succeeded, false otherwise. // It's analog of zoom_to_after function in OE, but doesn't require external locking //nolint -func (e *EpochManager) zoomToAfter(chain consensus.ChainHeaderReader, validators ValidatorSet, hash common.Hash) (RollingFinality, uint64, bool) { +func (e *EpochManager) zoomToAfter(chain consensus.ChainHeaderReader, er consensus.EpochReader, validators ValidatorSet, hash common.Hash, call consensus.SystemCall) (*RollingFinality, uint64, bool) { var lastWasParent bool if e.finalityChecker.lastPushed != nil { lastWasParent = *e.finalityChecker.lastPushed == hash @@ -192,37 +205,45 @@ func (e *EpochManager) zoomToAfter(chain consensus.ChainHeaderReader, validators // forks it will only need to be called for the block directly after // epoch transition, in which case it will be O(1) and require a single // DB lookup. - lastTransition, ok := epochTransitionFor(chain, hash) + lastTransition, ok := epochTransitionFor2(chain, er, hash) if !ok { + if lastTransition.BlockNumber > DEBUG_LOG_FROM { + fmt.Printf("zoom1: %d\n", lastTransition.BlockNumber) + } return e.finalityChecker, e.epochTransitionNumber, false } + // extract other epoch set if it's not the same as the last. if lastTransition.BlockHash != e.epochTransitionHash { - proof, err := destructure_proofs(lastTransition.ProofRlp) - if err != nil { + proof := &EpochTransitionProof{} + if err := rlp.DecodeBytes(lastTransition.ProofRlp, proof); err != nil { panic(err) } first := proof.SignalNumber == 0 + if lastTransition.BlockNumber > DEBUG_LOG_FROM { + fmt.Printf("zoom2: %d,%d\n", lastTransition.BlockNumber, len(proof.SetProof)) + } + // use signal number so multi-set first calculation is correct. - list, _, err := validators.epochSet(first, proof.SignalNumber, proof.SetProof) + list, _, err := validators.epochSet(first, proof.SignalNumber, proof.SetProof, call) if err != nil { - panic(fmt.Errorf("proof produced by this engine; therefore it is valid; qed. %w", err)) + panic(fmt.Errorf("proof produced by this engine is invalid: %w", err)) } epochSet := list.validators + log.Debug("[aura] Updating finality checker with new validator set extracted from epoch", "num", lastTransition.BlockNumber) e.finalityChecker = NewRollingFinality(epochSet) + if proof.SignalNumber >= DEBUG_LOG_FROM { + fmt.Printf("new rolling finality: %d\n", proof.SignalNumber) + for i := 0; i < len(epochSet); i++ { + fmt.Printf("\t%x\n", epochSet[i]) + } + } } + e.epochTransitionHash = lastTransition.BlockHash e.epochTransitionNumber = lastTransition.BlockNumber return e.finalityChecker, e.epochTransitionNumber, true } -func destructure_proofs(b []byte) (EpochTransitionProof, error) { - res := &EpochTransitionProof{} - err := rlp.DecodeBytes(b, res) - if err != nil { - return EpochTransitionProof{}, err - } - return *res, nil -} /// Get the transition to the epoch the given parent hash is part of /// or transitions to. @@ -230,7 +251,24 @@ func destructure_proofs(b []byte) (EpochTransitionProof, error) { /// /// The block corresponding the the parent hash must be stored already. //nolint -func epochTransitionFor(chain consensus.ChainHeaderReader, parentHash common.Hash) (transition EpochTransition, ok bool) { +func epochTransitionFor2(chain consensus.ChainHeaderReader, e consensus.EpochReader, parentHash common.Hash) (transition EpochTransition, ok bool) { + //TODO: probably this version of func doesn't support non-canonical epoch transitions + h := chain.GetHeaderByHash(parentHash) + if h == nil { + return transition, false + } + num, hash, transitionProof, err := e.FindBeforeOrEqualNumber(h.Number.Uint64()) + if err != nil { + panic(err) + } + if transitionProof == nil { + panic("genesis epoch transition must already be set") + } + return EpochTransition{BlockNumber: num, BlockHash: hash, ProofRlp: transitionProof}, true +} + +//nolint +func epochTransitionFor(chain consensus.ChainHeaderReader, e consensus.EpochReader, parentHash common.Hash) (transition EpochTransition, ok bool) { // slow path: loop back block by block for { h := chain.GetHeaderByHash(parentHash) @@ -239,9 +277,17 @@ func epochTransitionFor(chain consensus.ChainHeaderReader, parentHash common.Has } // look for transition in database. - transition, ok = epochTransition(h.Number.Uint64(), h.Hash()) - if ok { - return transition, true + transitionProof, err := e.GetEpoch(h.Hash(), h.Number.Uint64()) + if err != nil { + panic(err) + } + + if transitionProof != nil { + return EpochTransition{ + BlockNumber: h.Number.Uint64(), + BlockHash: h.Hash(), + ProofRlp: transitionProof, + }, true } // canonical hash -> fast breakout: @@ -255,7 +301,6 @@ func epochTransitionFor(chain consensus.ChainHeaderReader, parentHash common.Has } //nolint if canonical.Hash() == parentHash { - return EpochTransition{ BlockNumber: 0, BlockHash: common.HexToHash("0x5b28c1bfd3a15230c9a46b399cd0f9a6920d432e85381cc6a140b06e8410112f"), @@ -274,61 +319,6 @@ func epochTransitionFor(chain consensus.ChainHeaderReader, parentHash common.Has } } -// epochTransition get a specific epoch transition by block number and provided block hash. -//nolint -func epochTransition(blockNum uint64, blockHash common.Hash) (transition EpochTransition, ok bool) { - if blockNum == 0 { - return EpochTransition{BlockNumber: 0, BlockHash: params.SokolGenesisHash, ProofRlp: params.SokolGenesisEpochProof}, true - } - if blockNum > 672 { - return EpochTransition{BlockNumber: 672, BlockHash: common.HexToHash("0x6254cbb380ddbac9415773d668486a9cc69daf62a91022f653702a4e4db804e8"), ProofRlp: common.FromHex("0xf9071b8202a0b904cff904ccf9023da089e215eb0c3efcba1a9bc4f7d00245a5d918879480fc2f00c560f5defbe86e02a01dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d4934794e8ddc5c7a2d2f0d7a9798459c0104fdf5e987acaa0742fc98a1b575fb5f8d5cd475d0e53e35c2630993420ffc5b216e427bb8a54dfa0c8bbec82f1ace26614aa8379a3498fa4c2e5d6624167096f434e6955f5d67896a0ecf05cd0986446a335960ba00ccf2083e08b55103a20a874fb6215395e28dce4b901000000001000000010000000000000000000000000000100000000000002000000000000400000000040000000000000000000000000000000000000800000000020000200000000000000000000000000000002000000000000000000000000002000000000000000000000000000000100000000028000000000000000000000000000000000000000400400000000000000000000000000000000000000000000020000000000000008000000000012000000000200000000800000040000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000001000000090fffffffffffffffffffffffffffffffe8202a0837a12008302ed2e845a3445bc96d5830108048650617269747986312e32322e31826c6984120a7459b84106cec1622008b07bc3fb6c0cb7781a69f4101fe99c59960b2b0aee6b2a8f34b17f07001e5a8d83c85e7584a48c90906b308a71681b6a500018f80868fc6dd98a01f90289f90286018302ed2eb9010000000010000000100000000000000000000000000001000000000000020000000000004000000000400000000000000000000000000000000000008000000000200002000000000000000000000000000000020000000000000000000000000020000000000000000000000000000001000000000280000000000000000000000000000000000000004004000000000000000000000000000000000000000000000200000000000000080000000000120000000002000000008000000400000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000010000000f9017bf8db948bf38d4764929064f2d4d3a56520a76ab3df415bf842a055252fa6eee4741b4e24a74a70e9c11fd2c2281df8d6ea13126ff845f7825c89a089e215eb0c3efcba1a9bc4f7d00245a5d918879480fc2f00c560f5defbe86e02b88000000000000000000000000000000000000000000000000000000000000000200000000000000000000000000000000000000000000000000000000000000002000000000000000000000000e8ddc5c7a2d2f0d7a9798459c0104fdf5e987aca00000000000000000000000082e4e61e7f5139ff0a4157a5bc687ef42294c248f89c9488a34124bfffa27ef3e052c8dd2908e212643771f884a009b714df46e3a39ff284866b80612984e10c731561c157cd03aca91e436808a5a000000000000000000000000082e4e61e7f5139ff0a4157a5bc687ef42294c248a00000000000000000000000009a1f6adb5bd804b5d8bd21dd7aeb44edecbaa313a000000000000000000000000096604b744e9c5a72680d33e7faf9629e19c97eaa80b90243f90240f9023da089e215eb0c3efcba1a9bc4f7d00245a5d918879480fc2f00c560f5defbe86e02a01dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d4934794e8ddc5c7a2d2f0d7a9798459c0104fdf5e987acaa0742fc98a1b575fb5f8d5cd475d0e53e35c2630993420ffc5b216e427bb8a54dfa0c8bbec82f1ace26614aa8379a3498fa4c2e5d6624167096f434e6955f5d67896a0ecf05cd0986446a335960ba00ccf2083e08b55103a20a874fb6215395e28dce4b901000000001000000010000000000000000000000000000100000000000002000000000000400000000040000000000000000000000000000000000000800000000020000200000000000000000000000000000002000000000000000000000000002000000000000000000000000000000100000000028000000000000000000000000000000000000000400400000000000000000000000000000000000000000000020000000000000008000000000012000000000200000000800000040000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000001000000090fffffffffffffffffffffffffffffffe8202a0837a12008302ed2e845a3445bc96d5830108048650617269747986312e32322e31826c6984120a7459b84106cec1622008b07bc3fb6c0cb7781a69f4101fe99c59960b2b0aee6b2a8f34b17f07001e5a8d83c85e7584a48c90906b308a71681b6a500018f80868fc6dd98a01")}, true - } - return EpochTransition{}, false - /* - pub fn epoch_transition(&self, block_num: u64, block_hash: H256) -> Option { - trace!(target: "blockchain", "Loading epoch transition at block {}, {}", - block_num, block_hash); - - self.db - .key_value() - .read(db::COL_EXTRA, &block_num) - .and_then(|transitions: EpochTransitions| { - transitions - .candidates - .into_iter() - .find(|c| c.block_hash == block_hash) - }) - } - */ -} - -//nolint -type unAssembledHeader struct { - h common.Hash // H256 - 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, -} - -/// Create a blank finality checker under the given validator set. -func NewRollingFinality(signers []common.Address) RollingFinality { - return RollingFinality{ - signers: NewSimpleList(signers), - //headers: VecDeque::new(), - //sign_count: HashMap::new(), - //last_pushed: None - } -} - // AuRa //nolint type AuRa struct { @@ -551,21 +541,32 @@ func (c *AuRa) insertReceivedStepHashes(step uint64, author common.Address, newH //nolint func (c *AuRa) VerifyFamily(chain consensus.ChainHeaderReader, header *types.Header) error { return nil +} + +//nolit +func (c *AuRa) verifyFamily(chain consensus.ChainHeaderReader, e consensus.EpochReader, header *types.Header, call consensus.Call, syscall consensus.SystemCall) error { + // TODO: I call it from Initialize - because looks like no much reason to have separated "verifyFamily" call + + //nolint step, err := headerStep(header) if err != nil { return err } parent := chain.GetHeader(header.ParentHash, header.Number.Uint64()-1) + //nolint parentStep, err := headerStep(parent) if err != nil { return err } - validators, setNumber, err := c.epochSet(chain, header) + //nolint + validators, setNumber, err := c.epochSet(chain, e, header, syscall) if err != nil { return err } + return nil // Ensure header is from the step after parent. + //nolint if step == parentStep || (header.Number.Uint64() >= c.cfg.ValidateStepTransition && step <= parentStep) { log.Debug("[engine] Multiple blocks proposed for step", "num", parentStep) @@ -599,7 +600,7 @@ func (c *AuRa) VerifyFamily(chain consensus.ChainHeaderReader, header *types.Hea // Remove hash records older than two full rounds of steps (picked as a reasonable trade-off between // memory consumption and fault-tolerance). - cnt, err := count(validators, parent.Hash()) + cnt, err := count(validators, parent.Hash(), call) if err != nil { return err } @@ -798,69 +799,194 @@ func (c *AuRa) Prepare(chain consensus.ChainHeaderReader, header *types.Header) //return nil } -func (c *AuRa) Initialize(cc *params.ChainConfig, e consensus.EpochReader, header *types.Header, txs []types.Transaction, uncles []*types.Header, syscall consensus.SystemCall) { +func (c *AuRa) Initialize(cc *params.ChainConfig, chain consensus.ChainHeaderReader, e consensus.EpochReader, header *types.Header, txs []types.Transaction, uncles []*types.Header, + syscall consensus.SystemCall, call consensus.Call) { //TODO: hardcoded boolean!!! // let is_epoch_begin = chain.epoch_transition(parent.number(), *header.parent_hash()).is_some(); if header.Number.Uint64() == 1 { - _ = e.PutEpoch(header.ParentHash, 0, []byte{1}) //TODO: block 0 hardcoded - need fix it inside validators + proof, err := c.GenesisEpochData(header, call) + if err != nil { + panic(err) + } + err = e.PutEpoch(header.ParentHash, 0, proof) //TODO: block 0 hardcoded - need fix it inside validators + if err != nil { + panic(err) + } } + if err := c.verifyFamily(chain, e, header, call, syscall); err != nil { //TODO: OE has it as a separate engine call? why? + panic(err) + } + + // check_and_lock_block -> check_epoch_end_signal epoch, err := e.GetEpoch(header.ParentHash, header.Number.Uint64()-1) if err != nil { - log.Warn("aura initialize block: on epoch begin", "err", err) + log.Warn("[aura] initialize block: on epoch begin", "err", err) return } - isEpochBegin := epoch != nil - //var isEpochBegin bool - //sokolTransitions := []uint64{ - // 1, 673, 194808, 200642, - //} - //for _, t := range sokolTransitions { - // if header.Number.Uint64() == t { - // isEpochBegin = true - // break - // } - //} if !isEpochBegin { return } err = c.cfg.Validators.onEpochBegin(isEpochBegin, header, syscall) if err != nil { - log.Warn("aura initialize block: on epoch begin", "err", err) + log.Warn("[aura] initialize block: on epoch begin", "err", err) return } + // check_and_lock_block -> check_epoch_end_signal END (before enact) + } -func (c *AuRa) Finalize(cc *params.ChainConfig, header *types.Header, state *state.IntraBlockState, txs []types.Transaction, uncles []*types.Header, r types.Receipts, e consensus.EpochReader, syscall consensus.SystemCall) { +//word `signal epoch` == word `pending epoch` +func (c *AuRa) Finalize(cc *params.ChainConfig, header *types.Header, state *state.IntraBlockState, + txs []types.Transaction, uncles []*types.Header, r types.Receipts, + e consensus.EpochReader, chain consensus.ChainHeaderReader, syscall consensus.SystemCall, call consensus.Call) error { // accumulateRewards retrieves 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 + return fmt.Errorf("buildAncestrySubChain: %w", err) } for i := range beneficiaries { //fmt.Printf("beneficiary: n=%d, %x,%d\n", header.Number.Uint64(), beneficiaries[i], rewards[i]) state.AddBalance(beneficiaries[i], rewards[i]) } - // t_nb 9.13 check epoch end. Related only to AuRa and it seems light engine - //c.check_epoch_end(&header, &finalized, &chain, client) - //c.cfg.Validators.isEpochEnd() - _, ok := c.cfg.Validators.signalEpochEnd(header.Number.Uint64() == 0, header, r) - if ok { - //fmt.Printf("put echo: %d\n", header.Number.Uint64()) - _ = e.PutEpoch(header.Hash(), header.Number.Uint64(), []byte{1}) //TODO: put correct proof + // check_and_lock_block -> check_epoch_end_signal (after enact) + if header.Number.Uint64() >= DEBUG_LOG_FROM { + fmt.Printf("finalize1: %d,%d\n", header.Number.Uint64(), len(r)) } + pendingTransitionProof, err := c.cfg.Validators.signalEpochEnd(header.Number.Uint64() == 0, header, r) + if err != nil { + return err + } + if pendingTransitionProof != nil { + if header.Number.Uint64() >= DEBUG_LOG_FROM { + fmt.Printf("insert_pending_trancition: %d,receipts=%d, lenProof=%d\n", header.Number.Uint64(), len(r), len(pendingTransitionProof)) + } + if err = e.PutPendingEpoch(header.Hash(), header.Number.Uint64(), pendingTransitionProof); err != nil { + return err + } + } + // check_and_lock_block -> check_epoch_end_signal END + + finalized := buildFinality(c.EpochManager, chain, e, c.cfg.Validators, header, call, syscall) + c.EpochManager.finalityChecker.print(header.Number.Uint64()) + epochEndProof, err := isEpochEnd(chain, e, finalized, header) + if err != nil { + return err + } + if epochEndProof != nil { + c.EpochManager.noteNewEpoch() + log.Info("[aura] epoch transition", "block_num", header.Number.Uint64()) + if err := e.PutEpoch(header.Hash(), header.Number.Uint64(), epochEndProof); err != nil { + return err + } + } + + return nil +} + +func buildFinality(e *EpochManager, chain consensus.ChainHeaderReader, er consensus.EpochReader, validators ValidatorSet, header *types.Header, call consensus.Call, syscall consensus.SystemCall) []unAssembledHeader { + // commit_block -> aura.build_finality + _, _, ok := e.zoomToAfter(chain, er, validators, header.ParentHash, syscall) + if !ok { + return []unAssembledHeader{} + } + if e.finalityChecker.lastPushed == nil || *e.finalityChecker.lastPushed != header.ParentHash { + if err := e.finalityChecker.buildAncestrySubChain(func(hash common.Hash) ([]common.Address, common.Hash, common.Hash, uint64, bool) { + h := chain.GetHeaderByHash(hash) + if h == nil { + return nil, common.Hash{}, common.Hash{}, 0, false + } + return []common.Address{h.Coinbase}, h.Hash(), h.ParentHash, h.Number.Uint64(), true + }, header.ParentHash, e.epochTransitionHash); err != nil { + //log.Warn("[aura] buildAncestrySubChain", "err", err) + return []unAssembledHeader{} + } + } + + res, err := e.finalityChecker.push(header.Hash(), header.Number.Uint64(), []common.Address{header.Coinbase}) + if err != nil { + //log.Warn("[aura] finalityChecker.push", "err", err) + return []unAssembledHeader{} + } + return res +} + +func isEpochEnd(chain consensus.ChainHeaderReader, e consensus.EpochReader, finalized []unAssembledHeader, header *types.Header) ([]byte, error) { + // commit_block -> aura.is_epoch_end + for i := range finalized { + pendingTransitionProof, err := e.GetPendingEpoch(finalized[i].hash, finalized[i].number) + if err != nil { + return nil, err + } + if pendingTransitionProof == nil { + continue + } + if header.Number.Uint64() >= DEBUG_LOG_FROM { + fmt.Printf("pending transition: %d,%x,len=%d\n", finalized[i].number, finalized[i].hash, len(pendingTransitionProof)) + } + + finalityProof := allHeadersUntil(chain, header, finalized[i].hash) + var finalizedHeader *types.Header + if finalized[i].hash == header.Hash() { + finalizedHeader = header + } else { + finalizedHeader = chain.GetHeader(finalized[i].hash, finalized[i].number) + } + signalNumber := finalizedHeader.Number + finalityProof = append(finalityProof, finalizedHeader) + for i, j := 0, len(finalityProof)-1; i < j; i, j = i+1, j-1 { // reverse + finalityProof[i], finalityProof[j] = finalityProof[j], finalityProof[i] + } + finalityProofRLP, err := rlp.EncodeToBytes(finalityProof) + if err != nil { + return nil, err + } + /* + // We turn off can_propose here because upon validator set change there can + // be two valid proposers for a single step: one from the old set and + // one from the new. + // + // This way, upon encountering an epoch change, the proposer from the + // new set will be forced to wait until the next step to avoid sealing a + // block that breaks the invariant that the parent's step < the block's step. + self.step.can_propose.store(false, AtomicOrdering::SeqCst); + */ + return rlp.EncodeToBytes(EpochTransitionProof{SignalNumber: signalNumber.Uint64(), SetProof: pendingTransitionProof, FinalityProof: finalityProofRLP}) + } + return nil, nil +} + +// allHeadersUntil walk the chain backwards from current head until finalized_hash +// to construct transition proof. author == ec_recover(sig) known +// since the blocks are in the DB. +func allHeadersUntil(chain consensus.ChainHeaderReader, from *types.Header, to common.Hash) (out []*types.Header) { + var header = from + for { + header = chain.GetHeader(header.ParentHash, header.Number.Uint64()-1) + if header == nil { + panic("not found header") + } + if header.Number.Uint64() == 0 { + break + } + if to == header.Hash() { + break + } + out = append(out, header) + } + return out } //func (c *AuRa) check_epoch_end(cc *params.ChainConfig, header *types.Header, state *state.IntraBlockState, txs []types.Transaction, uncles []*types.Header, syscall consensus.SystemCall) { //} // FinalizeAndAssemble implements consensus.Engine -func (c *AuRa) FinalizeAndAssemble(chainConfig *params.ChainConfig, header *types.Header, state *state.IntraBlockState, txs []types.Transaction, uncles []*types.Header, r types.Receipts, e consensus.EpochReader, syscall consensus.SystemCall) (*types.Block, error) { - c.Finalize(chainConfig, header, state, txs, uncles, r, e, syscall) +func (c *AuRa) FinalizeAndAssemble(chainConfig *params.ChainConfig, header *types.Header, state *state.IntraBlockState, txs []types.Transaction, uncles []*types.Header, r types.Receipts, + e consensus.EpochReader, chain consensus.ChainHeaderReader, syscall consensus.SystemCall, call consensus.Call) (*types.Block, error) { + c.Finalize(chainConfig, header, state, txs, uncles, r, e, chain, syscall, call) // Assemble and return the final block for sealing return types.NewBlock(header, txs, uncles, r), nil @@ -876,29 +1002,19 @@ func (c *AuRa) Authorize(signer common.Address, signFn clique.SignerFn) { //c.signFn = signFn } -func (c *AuRa) GenesisEpochData(header *types.Header, caller Call) ([]byte, error) { - proof, err := c.cfg.Validators.genesisEpochData(header, caller) +func (c *AuRa) GenesisEpochData(header *types.Header, caller consensus.Call) ([]byte, error) { + setProof, err := c.cfg.Validators.genesisEpochData(header, caller) if err != nil { return nil, err } - return combineProofs(0, proof, []byte{}), nil -} -func combineProofs(signalNumber uint64, setProof []byte, finalityProof []byte) []byte { - //if signalNumber == 672 { - return common.FromHex("0xf91a8c80b91a87f91a84f9020da00000000000000000000000000000000000000000000000000000000000000000a01dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347940000000000000000000000000000000000000000a0fad4af258fd11939fae0c6c6eec9d340b1caac0b0196fd9a1bc3f489c5bf00b3a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421b9010000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000830200008083663be080808080b8410000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000f91871b853f851808080a07bb75cabebdcbd1dbb4331054636d0c6d7a2b08483b9e04df057395a7434c9e080808080808080a0e61e567237b49c44d8f906ceea49027260b4010c10a547b38d8b131b9d3b6f848080808080b86bf869a02080c7b7ae81a58eb98d9c78de4a1fd7fd9535fc953ed2be602daaa41767312ab846f8448080a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421a0c5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470b86bf869a033aa5d69545785694b808840be50c182dad2ec3636dfccbe6572fb69828742c0b846f8440101a0663ce0d171e545a26aa67e4ca66f72ba96bb48287dbcc03beea282867f80d44ba01f0e7726926cb43c03a0abf48197dba78522ec8ba1b158e2aa30da7d2a2c6f9ea3e2a02052222313e28459528d920b65115c16c04f3efc82aaedc97be59f3f377c0d3f01b914c26060604052600436106100fc576000357c0100000000000000000000000000000000000000000000000000000000900463ffffffff16806303aca79214610101578063108552691461016457806340a141ff1461019d57806340c9cdeb146101d65780634110a489146101ff57806345199e0a1461025757806349285b58146102c15780634d238c8e14610316578063752862111461034f578063900eb5a8146103645780639a573786146103c7578063a26a47d21461041c578063ae4b1b5b14610449578063b3f05b971461049e578063b7ab4db5146104cb578063d3e848f114610535578063fa81b2001461058a578063facd743b146105df575b600080fd5b341561010c57600080fd5b6101226004808035906020019091905050610630565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b341561016f57600080fd5b61019b600480803573ffffffffffffffffffffffffffffffffffffffff1690602001909190505061066f565b005b34156101a857600080fd5b6101d4600480803573ffffffffffffffffffffffffffffffffffffffff16906020019091905050610807565b005b34156101e157600080fd5b6101e9610bb7565b6040518082815260200191505060405180910390f35b341561020a57600080fd5b610236600480803573ffffffffffffffffffffffffffffffffffffffff16906020019091905050610bbd565b60405180831515151581526020018281526020019250505060405180910390f35b341561026257600080fd5b61026a610bee565b6040518080602001828103825283818151815260200191508051906020019060200280838360005b838110156102ad578082015181840152602081019050610292565b505050509050019250505060405180910390f35b34156102cc57600080fd5b6102d4610c82565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b341561032157600080fd5b61034d600480803573ffffffffffffffffffffffffffffffffffffffff16906020019091905050610d32565b005b341561035a57600080fd5b610362610fcc565b005b341561036f57600080fd5b61038560048080359060200190919050506110fc565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b34156103d257600080fd5b6103da61113b565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b341561042757600080fd5b61042f6111eb565b604051808215151515815260200191505060405180910390f35b341561045457600080fd5b61045c6111fe565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b34156104a957600080fd5b6104b1611224565b604051808215151515815260200191505060405180910390f35b34156104d657600080fd5b6104de611237565b6040518080602001828103825283818151815260200191508051906020019060200280838360005b83811015610521578082015181840152602081019050610506565b505050509050019250505060405180910390f35b341561054057600080fd5b6105486112cb565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b341561059557600080fd5b61059d6112f1565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b34156105ea57600080fd5b610616600480803573ffffffffffffffffffffffffffffffffffffffff16906020019091905050611317565b604051808215151515815260200191505060405180910390f35b60078181548110151561063f57fe5b90600052602060002090016000915054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b600460029054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff161415156106cb57600080fd5b600460019054906101000a900460ff161515156106e757600080fd5b600073ffffffffffffffffffffffffffffffffffffffff168173ffffffffffffffffffffffffffffffffffffffff161415151561072357600080fd5b80600a60006101000a81548173ffffffffffffffffffffffffffffffffffffffff021916908373ffffffffffffffffffffffffffffffffffffffff1602179055506001600460016101000a81548160ff0219169083151502179055507f600bcf04a13e752d1e3670a5a9f1c21177ca2a93c6f5391d4f1298d098097c22600a60009054906101000a900473ffffffffffffffffffffffffffffffffffffffff16604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390a150565b600080600061081461113b565b73ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff1614151561084d57600080fd5b83600960008273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200190815260200160002060000160009054906101000a900460ff1615156108a957600080fd5b600960008673ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff168152602001908152602001600020600101549350600160078054905003925060078381548110151561090857fe5b906000526020600020900160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1691508160078581548110151561094657fe5b906000526020600020900160006101000a81548173ffffffffffffffffffffffffffffffffffffffff021916908373ffffffffffffffffffffffffffffffffffffffff16021790555083600960008473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff168152602001908152602001600020600101819055506007838154811015156109e557fe5b906000526020600020900160006101000a81549073ffffffffffffffffffffffffffffffffffffffff02191690556000600780549050111515610a2757600080fd5b6007805480919060019003610a3c9190611370565b506000600960008773ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff168152602001908152602001600020600101819055506000600960008773ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200190815260200160002060000160006101000a81548160ff0219169083151502179055506000600460006101000a81548160ff0219169083151502179055506001430340600019167f55252fa6eee4741b4e24a74a70e9c11fd2c2281df8d6ea13126ff845f7825c89600760405180806020018281038252838181548152602001915080548015610ba257602002820191906000526020600020905b8160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff1681526020019060010190808311610b58575b50509250505060405180910390a25050505050565b60085481565b60096020528060005260406000206000915090508060000160009054906101000a900460ff16908060010154905082565b610bf661139c565b6007805480602002602001604051908101604052809291908181526020018280548015610c7857602002820191906000526020600020905b8160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff1681526020019060010190808311610c2e575b5050505050905090565b6000600a60009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff166349285b586000604051602001526040518163ffffffff167c0100000000000000000000000000000000000000000000000000000000028152600401602060405180830381600087803b1515610d1257600080fd5b6102c65a03f11515610d2357600080fd5b50505060405180519050905090565b610d3a61113b565b73ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff16141515610d7357600080fd5b80600960008273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200190815260200160002060000160009054906101000a900460ff16151515610dd057600080fd5b600073ffffffffffffffffffffffffffffffffffffffff168273ffffffffffffffffffffffffffffffffffffffff1614151515610e0c57600080fd5b6040805190810160405280600115158152602001600780549050815250600960008473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200190815260200160002060008201518160000160006101000a81548160ff0219169083151502179055506020820151816001015590505060078054806001018281610ea991906113b0565b9160005260206000209001600084909190916101000a81548173ffffffffffffffffffffffffffffffffffffffff021916908373ffffffffffffffffffffffffffffffffffffffff160217905550506000600460006101000a81548160ff0219169083151502179055506001430340600019167f55252fa6eee4741b4e24a74a70e9c11fd2c2281df8d6ea13126ff845f7825c89600760405180806020018281038252838181548152602001915080548015610fba57602002820191906000526020600020905b8160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff1681526020019060010190808311610f70575b50509250505060405180910390a25050565b600560009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff161480156110365750600460009054906101000a900460ff16155b151561104157600080fd5b6001600460006101000a81548160ff0219169083151502179055506007600690805461106e9291906113dc565b506006805490506008819055507f8564cd629b15f47dc310d45bcbfc9bcf5420b0d51bf0659a16c67f91d27632536110a4611237565b6040518080602001828103825283818151815260200191508051906020019060200280838360005b838110156110e75780820151818401526020810190506110cc565b505050509050019250505060405180910390a1565b60068181548110151561110b57fe5b90600052602060002090016000915054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b6000600a60009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16639a5737866000604051602001526040518163ffffffff167c0100000000000000000000000000000000000000000000000000000000028152600401602060405180830381600087803b15156111cb57600080fd5b6102c65a03f115156111dc57600080fd5b50505060405180519050905090565b600460019054906101000a900460ff1681565b600a60009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b600460009054906101000a900460ff1681565b61123f61139c565b60068054806020026020016040519081016040528092919081815260200182805480156112c157602002820191906000526020600020905b8160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff1681526020019060010190808311611277575b5050505050905090565b600560009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b600460029054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b6000600960008373ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200190815260200160002060000160009054906101000a900460ff169050919050565b81548183558181151161139757818360005260206000209182019101611396919061142e565b5b505050565b602060405190810160405280600081525090565b8154818355818115116113d7578183600052602060002091820191016113d6919061142e565b5b505050565b82805482825590600052602060002090810192821561141d5760005260206000209182015b8281111561141c578254825591600101919060010190611401565b5b50905061142a9190611453565b5090565b61145091905b8082111561144c576000816000905550600101611434565b5090565b90565b61149391905b8082111561148f57600081816101000a81549073ffffffffffffffffffffffffffffffffffffffff021916905550600101611459565b5090565b905600a165627a7a7230582036ea35935c8246b68074adece2eab70c40e69a0193c08a6277ce06e5b25188510029b8f3f8f1a08023c0d95fc2364e0bf7593f5ff32e1db8ef9f4b41c0bd474eae62d1af896e99808080a0b47b4f0b3e73b5edc8f9a9da1cbcfed562eb06bf54619b6aefeadebf5b3604c280a0da6ec08940a924cb08c947dd56cdb40076b29a6f0ea4dba4e2d02d9a9a72431b80a030cc4138c9e74b6cf79d624b4b5612c0fd888e91f55316cfee7d1694e1a90c0b80a0c5d54b915b56a888eee4e6eeb3141e778f9b674d1d322962eed900f02c29990aa017256b36ef47f907c6b1378a2636942ce894c17075e56fc054d4283f6846659e808080a03340bbaeafcda3a8672eb83099231dbbfab8dae02a1e8ec2f7180538fac207e080b853f851808080a0a87d9bb950836582673aa0eecc0ff64aac607870637a2dd2012b8b1b31981f698080a08da6d5c36a404670c553a2c9052df7cd604f04e3863c4c7b9e0027bfd54206d680808080808080808080b838f7a03868bdfa8727775661e4ccf117824a175a33f8703d728c04488fbfffcafda9f99594e8ddc5c7a2d2f0d7a9798459c0104fdf5e987acab8d3f8d1a0dc277c93a9f9dcee99aac9b8ba3cfa4c51821998522469c37715644e8fbac0bfa0ab8cdb808c8303bb61fb48e276217be9770fa83ecf3f90f2234d558885f5abf1808080a0fe137c3a474fbde41d89a59dd76da4c55bf696b86d3af64a55632f76cf30786780808080a06301b39b2ea8a44df8b0356120db64b788e71f52e1d7a6309d0d2e5b86fee7cb80a0da5d8b08dea0c5a4799c0f44d8a24d7cdf209f9b7a5588c1ecafb5361f6b9f07a01b7779e149cadf24d4ffb77ca7e11314b8db7097e4d70b2a173493153ca2e5a080808080") - //} else if signalNumber == 194807 { - // return common.FromHex("0xf9097c8302f8f6b904f0f904edf9023ea0f2ddffeefdda4c62e6dd5fe1d489c2afa5b60cef25fbea75d71a97e6237a7b2ea01dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d4934794e8ddc5c7a2d2f0d7a9798459c0104fdf5e987acaa065eb3b2f364e19dd070fcbbe25177ea0e0394be1d7f3846de432fab7f514f412a0c67d3e89a146214538d0a11d9fcbf32c8fdd2aec1b08c0d5c980cf9ce90f4b12a070c8149e15fee19ac37cc4c0fcd91b3fa299b2954f4bda0ea1850ea052f862e4b901000000000000000010000020000000000000000000000100000000000002000000000000001000000000000000000000002000000000000000000000800000000000000000200000000000000100000000000002000000804000000000000000002000000000000000000000000000000100000000008000000000000000000000000000000000000000000400000000000000000000000000000000000000000008000000000000002008000000000012000000000008000000000000000000000000000000000000000000000000020000000000000000000000000000000000000000000000000000000000000000000000000000000000200000000000000090fffffffffffffffffffffffffffffffe8302f8f6837a12008302ef15845a43183696d5830108048650617269747986312e32322e31826c6984120d6b3eb8415efd749e75311427f77a482d57c059aa26aaf5adfeb1dc0c49054ef3b50bdc232edf6d89964d98dfa8b26698de7dee7458381b263f4156122ba17309bd2b710301f902a9f902a6018302ef15b9010000000000000000100000200000000000000000000001000000000000020000000000000010000000000000000000000020000000000000000000008000000000000000002000000000000001000000000000020000008040000000000000000020000000000000000000000000000001000000000080000000000000000000000000000000000000000004000000000000000000000000000000000000000000080000000000000020080000000000120000000000080000000000000000000000000000000000000000000000000200000000000000000000000000000000000000000000000000000000000000000000000000000000002000000000000000f9019bf8fb948bf38d4764929064f2d4d3a56520a76ab3df415bf842a055252fa6eee4741b4e24a74a70e9c11fd2c2281df8d6ea13126ff845f7825c89a0f2ddffeefdda4c62e6dd5fe1d489c2afa5b60cef25fbea75d71a97e6237a7b2eb8a000000000000000000000000000000000000000000000000000000000000000200000000000000000000000000000000000000000000000000000000000000003000000000000000000000000e8ddc5c7a2d2f0d7a9798459c0104fdf5e987aca00000000000000000000000082e4e61e7f5139ff0a4157a5bc687ef42294c24800000000000000000000000023468db7f4b8ff4924b0709ab7ae0971b3bc8d28f89c9488a34124bfffa27ef3e052c8dd2908e212643771f884a009b714df46e3a39ff284866b80612984e10c731561c157cd03aca91e436808a5a000000000000000000000000023468db7f4b8ff4924b0709ab7ae0971b3bc8d28a00000000000000000000000007b4352a6a703bcf581f30c3259ef0078dfed8d90a0000000000000000000000000743f62ea80bebd2151c190bb611634cfacef12d480b90482f9047ff9023ea0f2ddffeefdda4c62e6dd5fe1d489c2afa5b60cef25fbea75d71a97e6237a7b2ea01dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d4934794e8ddc5c7a2d2f0d7a9798459c0104fdf5e987acaa065eb3b2f364e19dd070fcbbe25177ea0e0394be1d7f3846de432fab7f514f412a0c67d3e89a146214538d0a11d9fcbf32c8fdd2aec1b08c0d5c980cf9ce90f4b12a070c8149e15fee19ac37cc4c0fcd91b3fa299b2954f4bda0ea1850ea052f862e4b901000000000000000010000020000000000000000000000100000000000002000000000000001000000000000000000000002000000000000000000000800000000000000000200000000000000100000000000002000000804000000000000000002000000000000000000000000000000100000000008000000000000000000000000000000000000000000400000000000000000000000000000000000000000008000000000000002008000000000012000000000008000000000000000000000000000000000000000000000000020000000000000000000000000000000000000000000000000000000000000000000000000000000000200000000000000090fffffffffffffffffffffffffffffffe8302f8f6837a12008302ef15845a43183696d5830108048650617269747986312e32322e31826c6984120d6b3eb8415efd749e75311427f77a482d57c059aa26aaf5adfeb1dc0c49054ef3b50bdc232edf6d89964d98dfa8b26698de7dee7458381b263f4156122ba17309bd2b710301f9023ba0ffbe26248da7c29ab616e860abdae2d946995b2f49a53eda3cf491a37e926a2ea01dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d493479482e4e61e7f5139ff0a4157a5bc687ef42294c248a02e8491780355b3ad277d61b844f8d3bb5150e09be443fc740da17ebaaa58fc39a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421b901000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000090fffffffffffffffffffffffffffffffe8302f8f7837a120080845a43183a96d5830108048650617269747986312e32322e31826c6984120d6b3fb841d558d52af10da51c559706d4c5d79bec19fe1f9d473a8da0ccde3ab006d1dbe13ad1f69c27b59203ddb31d0f9533dbfb71692609040ca254588b64f20da0a88700") - // last_transition: 194807,87cf344d370c79a20e154b97d4e7fc9b287b91eb823f79955e00b1f3aa9f9d1b,f9097c8302f8f6b904f0f904edf9023ea0f2ddffeefdda4c62e6dd5fe1d489c2afa5b60cef25fbea75d71a97e6237a7b2ea01dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d4934794e8ddc5c7a2d2f0d7a9798459c0104fdf5e987acaa065eb3b2f364e19dd070fcbbe25177ea0e0394be1d7f3846de432fab7f514f412a0c67d3e89a146214538d0a11d9fcbf32c8fdd2aec1b08c0d5c980cf9ce90f4b12a070c8149e15fee19ac37cc4c0fcd91b3fa299b2954f4bda0ea1850ea052f862e4b901000000000000000010000020000000000000000000000100000000000002000000000000001000000000000000000000002000000000000000000000800000000000000000200000000000000100000000000002000000804000000000000000002000000000000000000000000000000100000000008000000000000000000000000000000000000000000400000000000000000000000000000000000000000008000000000000002008000000000012000000000008000000000000000000000000000000000000000000000000020000000000000000000000000000000000000000000000000000000000000000000000000000000000200000000000000090fffffffffffffffffffffffffffffffe8302f8f6837a12008302ef15845a43183696d5830108048650617269747986312e32322e31826c6984120d6b3eb8415efd749e75311427f77a482d57c059aa26aaf5adfeb1dc0c49054ef3b50bdc232edf6d89964d98dfa8b26698de7dee7458381b263f4156122ba17309bd2b710301f902a9f902a6018302ef15b9010000000000000000100000200000000000000000000001000000000000020000000000000010000000000000000000000020000000000000000000008000000000000000002000000000000001000000000000020000008040000000000000000020000000000000000000000000000001000000000080000000000000000000000000000000000000000004000000000000000000000000000000000000000000080000000000000020080000000000120000000000080000000000000000000000000000000000000000000000000200000000000000000000000000000000000000000000000000000000000000000000000000000000002000000000000000f9019bf8fb948bf38d4764929064f2d4d3a56520a76ab3df415bf842a055252fa6eee4741b4e24a74a70e9c11fd2c2281df8d6ea13126ff845f7825c89a0f2ddffeefdda4c62e6dd5fe1d489c2afa5b60cef25fbea75d71a97e6237a7b2eb8a000000000000000000000000000000000000000000000000000000000000000200000000000000000000000000000000000000000000000000000000000000003000000000000000000000000e8ddc5c7a2d2f0d7a9798459c0104fdf5e987aca00000000000000000000000082e4e61e7f5139ff0a4157a5bc687ef42294c24800000000000000000000000023468db7f4b8ff4924b0709ab7ae0971b3bc8d28f89c9488a34124bfffa27ef3e052c8dd2908e212643771f884a009b714df46e3a39ff284866b80612984e10c731561c157cd03aca91e436808a5a000000000000000000000000023468db7f4b8ff4924b0709ab7ae0971b3bc8d28a00000000000000000000000007b4352a6a703bcf581f30c3259ef0078dfed8d90a0000000000000000000000000743f62ea80bebd2151c190bb611634cfacef12d480b90482f9047ff9023ea0f2ddffeefdda4c62e6dd5fe1d489c2afa5b60cef25fbea75d71a97e6237a7b2ea01dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d4934794e8ddc5c7a2d2f0d7a9798459c0104fdf5e987acaa065eb3b2f364e19dd070fcbbe25177ea0e0394be1d7f3846de432fab7f514f412a0c67d3e89a146214538d0a11d9fcbf32c8fdd2aec1b08c0d5c980cf9ce90f4b12a070c8149e15fee19ac37cc4c0fcd91b3fa299b2954f4bda0ea1850ea052f862e4b901000000000000000010000020000000000000000000000100000000000002000000000000001000000000000000000000002000000000000000000000800000000000000000200000000000000100000000000002000000804000000000000000002000000000000000000000000000000100000000008000000000000000000000000000000000000000000400000000000000000000000000000000000000000008000000000000002008000000000012000000000008000000000000000000000000000000000000000000000000020000000000000000000000000000000000000000000000000000000000000000000000000000000000200000000000000090fffffffffffffffffffffffffffffffe8302f8f6837a12008302ef15845a43183696d5830108048650617269747986312e32322e31826c6984120d6b3eb8415efd749e75311427f77a482d57c059aa26aaf5adfeb1dc0c49054ef3b50bdc232edf6d89964d98dfa8b26698de7dee7458381b263f4156122ba17309bd2b710301f9023ba0ffbe26248da7c29ab616e860abdae2d946995b2f49a53eda3cf491a37e926a2ea01dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d493479482e4e61e7f5139ff0a4157a5bc687ef42294c248a02e8491780355b3ad277d61b844f8d3bb5150e09be443fc740da17ebaaa58fc39a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421b901000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000090fffffffffffffffffffffffffffffffe8302f8f7837a120080845a43183a96d5830108048650617269747986312e32322e31826c6984120d6b3fb841d558d52af10da51c559706d4c5d79bec19fe1f9d473a8da0ccde3ab006d1dbe13ad1f69c27b59203ddb31d0f9533dbfb71692609040ca254588b64f20da0a88700 - //} - //return nil - /* - let mut stream = RlpStream::new_list(3); - stream - .append(&signal_number) - .append(&set_proof) - .append(&finality_proof); - stream.out() - */ + res, err := rlp.EncodeToBytes(EpochTransitionProof{SignalNumber: 0, SetProof: setProof, FinalityProof: []byte{}}) + if err != nil { + panic(err) + } + //fmt.Printf("reere: %x\n", res) + //f91a84f9020da00000000000000000000000000000000000000000000000000000000000000000a01dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347940000000000000000000000000000000000000000a0fad4af258fd11939fae0c6c6eec9d340b1caac0b0196fd9a1bc3f489c5bf00b3a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421b9010000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000830200008083663be080808080b8410000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000f91871b914c26060604052600436106100fc576000357c0100000000000000000000000000000000000000000000000000000000900463ffffffff16806303aca79214610101578063108552691461016457806340a141ff1461019d57806340c9cdeb146101d65780634110a489146101ff57806345199e0a1461025757806349285b58146102c15780634d238c8e14610316578063752862111461034f578063900eb5a8146103645780639a573786146103c7578063a26a47d21461041c578063ae4b1b5b14610449578063b3f05b971461049e578063b7ab4db5146104cb578063d3e848f114610535578063fa81b2001461058a578063facd743b146105df575b600080fd5b341561010c57600080fd5b6101226004808035906020019091905050610630565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b341561016f57600080fd5b61019b600480803573ffffffffffffffffffffffffffffffffffffffff1690602001909190505061066f565b005b34156101a857600080fd5b6101d4600480803573ffffffffffffffffffffffffffffffffffffffff16906020019091905050610807565b005b34156101e157600080fd5b6101e9610bb7565b6040518082815260200191505060405180910390f35b341561020a57600080fd5b610236600480803573ffffffffffffffffffffffffffffffffffffffff16906020019091905050610bbd565b60405180831515151581526020018281526020019250505060405180910390f35b341561026257600080fd5b61026a610bee565b6040518080602001828103825283818151815260200191508051906020019060200280838360005b838110156102ad578082015181840152602081019050610292565b505050509050019250505060405180910390f35b34156102cc57600080fd5b6102d4610c82565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b341561032157600080fd5b61034d600480803573ffffffffffffffffffffffffffffffffffffffff16906020019091905050610d32565b005b341561035a57600080fd5b610362610fcc565b005b341561036f57600080fd5b61038560048080359060200190919050506110fc565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b34156103d257600080fd5b6103da61113b565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b341561042757600080fd5b61042f6111eb565b604051808215151515815260200191505060405180910390f35b341561045457600080fd5b61045c6111fe565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b34156104a957600080fd5b6104b1611224565b604051808215151515815260200191505060405180910390f35b34156104d657600080fd5b6104de611237565b6040518080602001828103825283818151815260200191508051906020019060200280838360005b83811015610521578082015181840152602081019050610506565b505050509050019250505060405180910390f35b341561054057600080fd5b6105486112cb565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b341561059557600080fd5b61059d6112f1565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b34156105ea57600080fd5b610616600480803573ffffffffffffffffffffffffffffffffffffffff16906020019091905050611317565b604051808215151515815260200191505060405180910390f35b60078181548110151561063f57fe5b90600052602060002090016000915054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b600460029054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff161415156106cb57600080fd5b600460019054906101000a900460ff161515156106e757600080fd5b600073ffffffffffffffffffffffffffffffffffffffff168173ffffffffffffffffffffffffffffffffffffffff161415151561072357600080fd5b80600a60006101000a81548173ffffffffffffffffffffffffffffffffffffffff021916908373ffffffffffffffffffffffffffffffffffffffff1602179055506001600460016101000a81548160ff0219169083151502179055507f600bcf04a13e752d1e3670a5a9f1c21177ca2a93c6f5391d4f1298d098097c22600a60009054906101000a900473ffffffffffffffffffffffffffffffffffffffff16604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390a150565b600080600061081461113b565b73ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff1614151561084d57600080fd5b83600960008273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200190815260200160002060000160009054906101000a900460ff1615156108a957600080fd5b600960008673ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff168152602001908152602001600020600101549350600160078054905003925060078381548110151561090857fe5b906000526020600020900160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1691508160078581548110151561094657fe5b906000526020600020900160006101000a81548173ffffffffffffffffffffffffffffffffffffffff021916908373ffffffffffffffffffffffffffffffffffffffff16021790555083600960008473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff168152602001908152602001600020600101819055506007838154811015156109e557fe5b906000526020600020900160006101000a81549073ffffffffffffffffffffffffffffffffffffffff02191690556000600780549050111515610a2757600080fd5b6007805480919060019003610a3c9190611370565b506000600960008773ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff168152602001908152602001600020600101819055506000600960008773ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200190815260200160002060000160006101000a81548160ff0219169083151502179055506000600460006101000a81548160ff0219169083151502179055506001430340600019167f55252fa6eee4741b4e24a74a70e9c11fd2c2281df8d6ea13126ff845f7825c89600760405180806020018281038252838181548152602001915080548015610ba257602002820191906000526020600020905b8160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff1681526020019060010190808311610b58575b50509250505060405180910390a25050505050565b60085481565b60096020528060005260406000206000915090508060000160009054906101000a900460ff16908060010154905082565b610bf661139c565b6007805480602002602001604051908101604052809291908181526020018280548015610c7857602002820191906000526020600020905b8160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff1681526020019060010190808311610c2e575b5050505050905090565b6000600a60009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff166349285b586000604051602001526040518163ffffffff167c0100000000000000000000000000000000000000000000000000000000028152600401602060405180830381600087803b1515610d1257600080fd5b6102c65a03f11515610d2357600080fd5b50505060405180519050905090565b610d3a61113b565b73ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff16141515610d7357600080fd5b80600960008273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200190815260200160002060000160009054906101000a900460ff16151515610dd057600080fd5b600073ffffffffffffffffffffffffffffffffffffffff168273ffffffffffffffffffffffffffffffffffffffff1614151515610e0c57600080fd5b6040805190810160405280600115158152602001600780549050815250600960008473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200190815260200160002060008201518160000160006101000a81548160ff0219169083151502179055506020820151816001015590505060078054806001018281610ea991906113b0565b9160005260206000209001600084909190916101000a81548173ffffffffffffffffffffffffffffffffffffffff021916908373ffffffffffffffffffffffffffffffffffffffff160217905550506000600460006101000a81548160ff0219169083151502179055506001430340600019167f55252fa6eee4741b4e24a74a70e9c11fd2c2281df8d6ea13126ff845f7825c89600760405180806020018281038252838181548152602001915080548015610fba57602002820191906000526020600020905b8160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff1681526020019060010190808311610f70575b50509250505060405180910390a25050565b600560009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff161480156110365750600460009054906101000a900460ff16155b151561104157600080fd5b6001600460006101000a81548160ff0219169083151502179055506007600690805461106e9291906113dc565b506006805490506008819055507f8564cd629b15f47dc310d45bcbfc9bcf5420b0d51bf0659a16c67f91d27632536110a4611237565b6040518080602001828103825283818151815260200191508051906020019060200280838360005b838110156110e75780820151818401526020810190506110cc565b505050509050019250505060405180910390a1565b60068181548110151561110b57fe5b90600052602060002090016000915054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b6000600a60009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16639a5737866000604051602001526040518163ffffffff167c0100000000000000000000000000000000000000000000000000000000028152600401602060405180830381600087803b15156111cb57600080fd5b6102c65a03f115156111dc57600080fd5b50505060405180519050905090565b600460019054906101000a900460ff1681565b600a60009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b600460009054906101000a900460ff1681565b61123f61139c565b60068054806020026020016040519081016040528092919081815260200182805480156112c157602002820191906000526020600020905b8160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff1681526020019060010190808311611277575b5050505050905090565b600560009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b600460029054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b6000600960008373ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200190815260200160002060000160009054906101000a900460ff169050919050565b81548183558181151161139757818360005260206000209182019101611396919061142e565b5b505050565b602060405190810160405280600081525090565b8154818355818115116113d7578183600052602060002091820191016113d6919061142e565b5b505050565b82805482825590600052602060002090810192821561141d5760005260206000209182015b8281111561141c578254825591600101919060010190611401565b5b50905061142a9190611453565b5090565b61145091905b8082111561144c576000816000905550600101611434565b5090565b90565b61149391905b8082111561148f57600081816101000a81549073ffffffffffffffffffffffffffffffffffffffff021916905550600101611459565b5090565b905600a165627a7a7230582036ea35935c8246b68074adece2eab70c40e69a0193c08a6277ce06e5b25188510029b8f3f8f1a08023c0d95fc2364e0bf7593f5ff32e1db8ef9f4b41c0bd474eae62d1af896e99808080a0b47b4f0b3e73b5edc8f9a9da1cbcfed562eb06bf54619b6aefeadebf5b3604c280a0da6ec08940a924cb08c947dd56cdb40076b29a6f0ea4dba4e2d02d9a9a72431b80a030cc4138c9e74b6cf79d624b4b5612c0fd888e91f55316cfee7d1694e1a90c0b80a0c5d54b915b56a888eee4e6eeb3141e778f9b674d1d322962eed900f02c29990aa017256b36ef47f907c6b1378a2636942ce894c17075e56fc054d4283f6846659e808080a03340bbaeafcda3a8672eb83099231dbbfab8dae02a1e8ec2f7180538fac207e080b86bf869a033aa5d69545785694b808840be50c182dad2ec3636dfccbe6572fb69828742c0b846f8440101a0663ce0d171e545a26aa67e4ca66f72ba96bb48287dbcc03beea282867f80d44ba01f0e7726926cb43c03a0abf48197dba78522ec8ba1b158e2aa30da7d2a2c6f9eb838f7a03868bdfa8727775661e4ccf117824a175a33f8703d728c04488fbfffcafda9f99594e8ddc5c7a2d2f0d7a9798459c0104fdf5e987acaa3e2a02052222313e28459528d920b65115c16c04f3efc82aaedc97be59f3f377c0d3f01b853f851808080a07bb75cabebdcbd1dbb4331054636d0c6d7a2b08483b9e04df057395a7434c9e080808080808080a0e61e567237b49c44d8f906ceea49027260b4010c10a547b38d8b131b9d3b6f848080808080b8d3f8d1a0dc277c93a9f9dcee99aac9b8ba3cfa4c51821998522469c37715644e8fbac0bfa0ab8cdb808c8303bb61fb48e276217be9770fa83ecf3f90f2234d558885f5abf1808080a0fe137c3a474fbde41d89a59dd76da4c55bf696b86d3af64a55632f76cf30786780808080a06301b39b2ea8a44df8b0356120db64b788e71f52e1d7a6309d0d2e5b86fee7cb80a0da5d8b08dea0c5a4799c0f44d8a24d7cdf209f9b7a5588c1ecafb5361f6b9f07a01b7779e149cadf24d4ffb77ca7e11314b8db7097e4d70b2a173493153ca2e5a0808080b853f851808080a0a87d9bb950836582673aa0eecc0ff64aac607870637a2dd2012b8b1b31981f698080a08da6d5c36a404670c553a2c9052df7cd604f04e3863c4c7b9e0027bfd54206d680808080808080808080b86bf869a02080c7b7ae81a58eb98d9c78de4a1fd7fd9535fc953ed2be602daaa41767312ab846f8448080a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421a0c5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470 + //f91a8c80b91a87f91a84f9020da00000000000000000000000000000000000000000000000000000000000000000a01dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347940000000000000000000000000000000000000000a0fad4af258fd11939fae0c6c6eec9d340b1caac0b0196fd9a1bc3f489c5bf00b3a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421b9010000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000830200008083663be080808080b8410000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000f91871b914c26060604052600436106100fc576000357c0100000000000000000000000000000000000000000000000000000000900463ffffffff16806303aca79214610101578063108552691461016457806340a141ff1461019d57806340c9cdeb146101d65780634110a489146101ff57806345199e0a1461025757806349285b58146102c15780634d238c8e14610316578063752862111461034f578063900eb5a8146103645780639a573786146103c7578063a26a47d21461041c578063ae4b1b5b14610449578063b3f05b971461049e578063b7ab4db5146104cb578063d3e848f114610535578063fa81b2001461058a578063facd743b146105df575b600080fd5b341561010c57600080fd5b6101226004808035906020019091905050610630565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b341561016f57600080fd5b61019b600480803573ffffffffffffffffffffffffffffffffffffffff1690602001909190505061066f565b005b34156101a857600080fd5b6101d4600480803573ffffffffffffffffffffffffffffffffffffffff16906020019091905050610807565b005b34156101e157600080fd5b6101e9610bb7565b6040518082815260200191505060405180910390f35b341561020a57600080fd5b610236600480803573ffffffffffffffffffffffffffffffffffffffff16906020019091905050610bbd565b60405180831515151581526020018281526020019250505060405180910390f35b341561026257600080fd5b61026a610bee565b6040518080602001828103825283818151815260200191508051906020019060200280838360005b838110156102ad578082015181840152602081019050610292565b505050509050019250505060405180910390f35b34156102cc57600080fd5b6102d4610c82565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b341561032157600080fd5b61034d600480803573ffffffffffffffffffffffffffffffffffffffff16906020019091905050610d32565b005b341561035a57600080fd5b610362610fcc565b005b341561036f57600080fd5b61038560048080359060200190919050506110fc565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b34156103d257600080fd5b6103da61113b565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b341561042757600080fd5b61042f6111eb565b604051808215151515815260200191505060405180910390f35b341561045457600080fd5b61045c6111fe565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b34156104a957600080fd5b6104b1611224565b604051808215151515815260200191505060405180910390f35b34156104d657600080fd5b6104de611237565b6040518080602001828103825283818151815260200191508051906020019060200280838360005b83811015610521578082015181840152602081019050610506565b505050509050019250505060405180910390f35b341561054057600080fd5b6105486112cb565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b341561059557600080fd5b61059d6112f1565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b34156105ea57600080fd5b610616600480803573ffffffffffffffffffffffffffffffffffffffff16906020019091905050611317565b604051808215151515815260200191505060405180910390f35b60078181548110151561063f57fe5b90600052602060002090016000915054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b600460029054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff161415156106cb57600080fd5b600460019054906101000a900460ff161515156106e757600080fd5b600073ffffffffffffffffffffffffffffffffffffffff168173ffffffffffffffffffffffffffffffffffffffff161415151561072357600080fd5b80600a60006101000a81548173ffffffffffffffffffffffffffffffffffffffff021916908373ffffffffffffffffffffffffffffffffffffffff1602179055506001600460016101000a81548160ff0219169083151502179055507f600bcf04a13e752d1e3670a5a9f1c21177ca2a93c6f5391d4f1298d098097c22600a60009054906101000a900473ffffffffffffffffffffffffffffffffffffffff16604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390a150565b600080600061081461113b565b73ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff1614151561084d57600080fd5b83600960008273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200190815260200160002060000160009054906101000a900460ff1615156108a957600080fd5b600960008673ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff168152602001908152602001600020600101549350600160078054905003925060078381548110151561090857fe5b906000526020600020900160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1691508160078581548110151561094657fe5b906000526020600020900160006101000a81548173ffffffffffffffffffffffffffffffffffffffff021916908373ffffffffffffffffffffffffffffffffffffffff16021790555083600960008473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff168152602001908152602001600020600101819055506007838154811015156109e557fe5b906000526020600020900160006101000a81549073ffffffffffffffffffffffffffffffffffffffff02191690556000600780549050111515610a2757600080fd5b6007805480919060019003610a3c9190611370565b506000600960008773ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff168152602001908152602001600020600101819055506000600960008773ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200190815260200160002060000160006101000a81548160ff0219169083151502179055506000600460006101000a81548160ff0219169083151502179055506001430340600019167f55252fa6eee4741b4e24a74a70e9c11fd2c2281df8d6ea13126ff845f7825c89600760405180806020018281038252838181548152602001915080548015610ba257602002820191906000526020600020905b8160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff1681526020019060010190808311610b58575b50509250505060405180910390a25050505050565b60085481565b60096020528060005260406000206000915090508060000160009054906101000a900460ff16908060010154905082565b610bf661139c565b6007805480602002602001604051908101604052809291908181526020018280548015610c7857602002820191906000526020600020905b8160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff1681526020019060010190808311610c2e575b5050505050905090565b6000600a60009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff166349285b586000604051602001526040518163ffffffff167c0100000000000000000000000000000000000000000000000000000000028152600401602060405180830381600087803b1515610d1257600080fd5b6102c65a03f11515610d2357600080fd5b50505060405180519050905090565b610d3a61113b565b73ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff16141515610d7357600080fd5b80600960008273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200190815260200160002060000160009054906101000a900460ff16151515610dd057600080fd5b600073ffffffffffffffffffffffffffffffffffffffff168273ffffffffffffffffffffffffffffffffffffffff1614151515610e0c57600080fd5b6040805190810160405280600115158152602001600780549050815250600960008473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200190815260200160002060008201518160000160006101000a81548160ff0219169083151502179055506020820151816001015590505060078054806001018281610ea991906113b0565b9160005260206000209001600084909190916101000a81548173ffffffffffffffffffffffffffffffffffffffff021916908373ffffffffffffffffffffffffffffffffffffffff160217905550506000600460006101000a81548160ff0219169083151502179055506001430340600019167f55252fa6eee4741b4e24a74a70e9c11fd2c2281df8d6ea13126ff845f7825c89600760405180806020018281038252838181548152602001915080548015610fba57602002820191906000526020600020905b8160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff1681526020019060010190808311610f70575b50509250505060405180910390a25050565b600560009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff161480156110365750600460009054906101000a900460ff16155b151561104157600080fd5b6001600460006101000a81548160ff0219169083151502179055506007600690805461106e9291906113dc565b506006805490506008819055507f8564cd629b15f47dc310d45bcbfc9bcf5420b0d51bf0659a16c67f91d27632536110a4611237565b6040518080602001828103825283818151815260200191508051906020019060200280838360005b838110156110e75780820151818401526020810190506110cc565b505050509050019250505060405180910390a1565b60068181548110151561110b57fe5b90600052602060002090016000915054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b6000600a60009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16639a5737866000604051602001526040518163ffffffff167c0100000000000000000000000000000000000000000000000000000000028152600401602060405180830381600087803b15156111cb57600080fd5b6102c65a03f115156111dc57600080fd5b50505060405180519050905090565b600460019054906101000a900460ff1681565b600a60009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b600460009054906101000a900460ff1681565b61123f61139c565b60068054806020026020016040519081016040528092919081815260200182805480156112c157602002820191906000526020600020905b8160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff1681526020019060010190808311611277575b5050505050905090565b600560009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b600460029054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b6000600960008373ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200190815260200160002060000160009054906101000a900460ff169050919050565b81548183558181151161139757818360005260206000209182019101611396919061142e565b5b505050565b602060405190810160405280600081525090565b8154818355818115116113d7578183600052602060002091820191016113d6919061142e565b5b505050565b82805482825590600052602060002090810192821561141d5760005260206000209182015b8281111561141c578254825591600101919060010190611401565b5b50905061142a9190611453565b5090565b61145091905b8082111561144c576000816000905550600101611434565b5090565b90565b61149391905b8082111561148f57600081816101000a81549073ffffffffffffffffffffffffffffffffffffffff021916905550600101611459565b5090565b905600a165627a7a7230582036ea35935c8246b68074adece2eab70c40e69a0193c08a6277ce06e5b25188510029b8f3f8f1a08023c0d95fc2364e0bf7593f5ff32e1db8ef9f4b41c0bd474eae62d1af896e99808080a0b47b4f0b3e73b5edc8f9a9da1cbcfed562eb06bf54619b6aefeadebf5b3604c280a0da6ec08940a924cb08c947dd56cdb40076b29a6f0ea4dba4e2d02d9a9a72431b80a030cc4138c9e74b6cf79d624b4b5612c0fd888e91f55316cfee7d1694e1a90c0b80a0c5d54b915b56a888eee4e6eeb3141e778f9b674d1d322962eed900f02c29990aa017256b36ef47f907c6b1378a2636942ce894c17075e56fc054d4283f6846659e808080a03340bbaeafcda3a8672eb83099231dbbfab8dae02a1e8ec2f7180538fac207e080b86bf869a033aa5d69545785694b808840be50c182dad2ec3636dfccbe6572fb69828742c0b846f8440101a0663ce0d171e545a26aa67e4ca66f72ba96bb48287dbcc03beea282867f80d44ba01f0e7726926cb43c03a0abf48197dba78522ec8ba1b158e2aa30da7d2a2c6f9eb838f7a03868bdfa8727775661e4ccf117824a175a33f8703d728c04488fbfffcafda9f99594e8ddc5c7a2d2f0d7a9798459c0104fdf5e987acaa3e2a02052222313e28459528d920b65115c16c04f3efc82aaedc97be59f3f377c0d3f01b853f851808080a07bb75cabebdcbd1dbb4331054636d0c6d7a2b08483b9e04df057395a7434c9e080808080808080a0e61e567237b49c44d8f906ceea49027260b4010c10a547b38d8b131b9d3b6f848080808080b8d3f8d1a0dc277c93a9f9dcee99aac9b8ba3cfa4c51821998522469c37715644e8fbac0bfa0ab8cdb808c8303bb61fb48e276217be9770fa83ecf3f90f2234d558885f5abf1808080a0fe137c3a474fbde41d89a59dd76da4c55bf696b86d3af64a55632f76cf30786780808080a06301b39b2ea8a44df8b0356120db64b788e71f52e1d7a6309d0d2e5b86fee7cb80a0da5d8b08dea0c5a4799c0f44d8a24d7cdf209f9b7a5588c1ecafb5361f6b9f07a01b7779e149cadf24d4ffb77ca7e11314b8db7097e4d70b2a173493153ca2e5a0808080b853f851808080a0a87d9bb950836582673aa0eecc0ff64aac607870637a2dd2012b8b1b31981f698080a08da6d5c36a404670c553a2c9052df7cd604f04e3863c4c7b9e0027bfd54206d680808080808080808080b86bf869a02080c7b7ae81a58eb98d9c78de4a1fd7fd9535fc953ed2be602daaa41767312ab846f8448080a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421a0c5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a47080 + return res, nil } func (c *AuRa) Seal(chain consensus.ChainHeaderReader, block *types.Block, results chan<- *types.Block, stop <-chan struct{}) error { @@ -972,19 +1088,19 @@ func (c *AuRa) Seal(chain consensus.ChainHeaderReader, block *types.Block, resul //return nil } -func stepProposer(validators ValidatorSet, blockHash common.Hash, step uint64) (common.Address, error) { - c, err := validators.defaultCaller(blockHash) - if err != nil { - return common.Address{}, err - } - return validators.getWithCaller(blockHash, uint(step), c) +func stepProposer(validators ValidatorSet, blockHash common.Hash, step uint64, call consensus.Call) (common.Address, error) { + //c, err := validators.defaultCaller(blockHash) + //if err != nil { + // return common.Address{}, err + //} + return validators.getWithCaller(blockHash, uint(step), call) } // GenerateSeal - Attempt to seal the block internally. // // This operation is synchronous and may (quite reasonably) not be available, in which case // `Seal::None` will be returned. -func (c *AuRa) GenerateSeal(chain consensus.ChainHeaderReader, current, parent *types.Header) []rlp.RawValue { +func (c *AuRa) GenerateSeal(chain consensus.ChainHeaderReader, current, parent *types.Header, call consensus.Call) []rlp.RawValue { // first check to avoid generating signature most of the time // (but there's still a race to the `compare_exchange`) if !c.step.canPropose.Load() { @@ -1009,13 +1125,13 @@ func (c *AuRa) GenerateSeal(chain consensus.ChainHeaderReader, current, parent * return nil } - validators, setNumber, err := c.epochSet(chain, current) + validators, setNumber, err := c.epochSet(chain, nil, current, nil) if err != nil { log.Warn("[engine] Unable to generate seal", "err", err) return nil } - stepProposerAddr, err := stepProposer(validators, current.ParentHash, step) + stepProposerAddr, err := stepProposer(validators, current.ParentHash, step, call) if err != nil { log.Warn("[engine] Unable to get stepProposer", "err", err) return nil @@ -1074,12 +1190,12 @@ func (c *AuRa) GenerateSeal(chain consensus.ChainHeaderReader, current, parent * // epochSet fetch correct validator set for epoch at header, taking into account // finality of previous transitions. -func (c *AuRa) epochSet(chain consensus.ChainHeaderReader, h *types.Header) (ValidatorSet, uint64, error) { +func (c *AuRa) epochSet(chain consensus.ChainHeaderReader, e consensus.EpochReader, h *types.Header, call consensus.SystemCall) (ValidatorSet, uint64, error) { if c.cfg.ImmediateTransitions { return c.cfg.Validators, h.Number.Uint64(), nil } - finalityChecker, epochTransitionNumber, ok := c.EpochManager.zoomToAfter(chain, c.cfg.Validators, h.ParentHash) + finalityChecker, epochTransitionNumber, ok := c.EpochManager.zoomToAfter(chain, e, c.cfg.Validators, h.ParentHash, call) if !ok { return nil, 0, fmt.Errorf("unable to zoomToAfter to epoch") } @@ -1423,3 +1539,179 @@ func EmptyStepRlp(step uint64, parentHash common.Hash) ([]byte, error) { } return rlp.EncodeToBytes(A{s: step, h: parentHash}) } + +//nolint +type unAssembledHeader struct { + hash common.Hash + number uint64 + signers []common.Address +} +type unAssembledHeaders struct { + l *list.List +} + +func (u unAssembledHeaders) PushBack(header *unAssembledHeader) { u.l.PushBack(header) } +func (u unAssembledHeaders) PushFront(header *unAssembledHeader) { u.l.PushFront(header) } +func (u unAssembledHeaders) Pop() *unAssembledHeader { + e := u.l.Front() + if e == nil { + return nil + } + u.l.Remove(e) + return e.Value.(*unAssembledHeader) +} +func (u unAssembledHeaders) Front() *unAssembledHeader { + e := u.l.Front() + if e == nil { + return nil + } + return e.Value.(*unAssembledHeader) +} + +// RollingFinality checker for authority round consensus. +// Stores a chain of unfinalized hashes that can be pushed onto. +//nolint +type RollingFinality struct { + headers unAssembledHeaders //nolint + signers *SimpleList + signCount map[common.Address]uint + lastPushed *common.Hash // Option, +} + +// NewRollingFinality creates a blank finality checker under the given validator set. +func NewRollingFinality(signers []common.Address) *RollingFinality { + return &RollingFinality{ + signers: NewSimpleList(signers), + headers: unAssembledHeaders{l: list.New()}, + signCount: map[common.Address]uint{}, + } +} + +// Clears the finality status, but keeps the validator set. +func (f *RollingFinality) print(num uint64) { + if num > DEBUG_LOG_FROM { + h := f.headers + fmt.Printf("finality_heads: %d\n", num) + i := 0 + for e := h.l.Front(); e != nil; e = e.Next() { + i++ + a := e.Value.(*unAssembledHeader) + fmt.Printf("\t%d,%x\n", a.number, a.signers[0]) + } + if i == 0 { + fmt.Printf("\tempty\n") + } + } +} + +func (f *RollingFinality) clear() { + f.headers = unAssembledHeaders{l: list.New()} + f.signCount = map[common.Address]uint{} + f.lastPushed = nil +} + +// Push a hash onto the rolling finality checker (implying `subchain_head` == head.parent) +// +// Fails if `signer` isn't a member of the active validator set. +// Returns a list of all newly finalized headers. +func (f *RollingFinality) push(head common.Hash, num uint64, signers []common.Address) (newlyFinalized []unAssembledHeader, err error) { + for i := range signers { + if !f.hasSigner(signers[i]) { + return nil, fmt.Errorf("unknown validator") + } + } + + f.addSigners(signers) + f.headers.PushBack(&unAssembledHeader{hash: head, number: num, signers: signers}) + + for f.isFinalized() { + e := f.headers.Pop() + if e == nil { + panic("headers length always greater than sign count length") + } + f.removeSigners(e.signers) + newlyFinalized = append(newlyFinalized, *e) + } + f.lastPushed = &head + return newlyFinalized, nil +} + +// isFinalized returns whether the first entry in `self.headers` is finalized. +func (f *RollingFinality) isFinalized() bool { + e := f.headers.Front() + if e == nil { + return false + } + return len(f.signCount)*2 > len(f.signers.validators) +} +func (f *RollingFinality) hasSigner(signer common.Address) bool { + for j := range f.signers.validators { + if f.signers.validators[j] == signer { + return true + + } + } + return false +} +func (f *RollingFinality) addSigners(signers []common.Address) bool { + for i := range signers { + count, ok := f.signCount[signers[i]] + if ok { + f.signCount[signers[i]] = count + 1 + } else { + f.signCount[signers[i]] = 1 + } + } + return false +} +func (f *RollingFinality) removeSigners(signers []common.Address) { + for i := range signers { + count, ok := f.signCount[signers[i]] + if !ok { + panic("all hashes in `header` should have entries in `sign_count` for their signers") + //continue + } + if count <= 1 { + delete(f.signCount, signers[i]) + } else { + f.signCount[signers[i]] = count - 1 + } + } +} +func (f *RollingFinality) buildAncestrySubChain(get func(hash common.Hash) ([]common.Address, common.Hash, common.Hash, uint64, bool), parentHash, epochTransitionHash common.Hash) error { // starts from chainHeadParentHash + f.clear() + + for { + signers, blockHash, newParentHash, blockNum, ok := get(parentHash) + if !ok { + return nil + } + if blockHash == epochTransitionHash { + return nil + } + for i := range signers { + if !f.hasSigner(signers[i]) { + return fmt.Errorf("unknown validator: blockNum=%d", blockNum) + } + } + if f.lastPushed == nil { + copyHash := parentHash + f.lastPushed = ©Hash + } + f.addSigners(signers) + f.headers.PushFront(&unAssembledHeader{hash: blockHash, number: blockNum, signers: signers}) + // break when we've got our first finalized block. + if f.isFinalized() { + e := f.headers.Pop() + if e == nil { + panic("we just pushed a block") + } + f.removeSigners(e.signers) + //log.Info("[aura] finality encountered already finalized block", "hash", e.hash.String(), "number", e.number) + break + } + + parentHash = newParentHash + } + return nil +} diff --git a/consensus/aura/config.go b/consensus/aura/config.go index 4a5d0f216..9015821c6 100644 --- a/consensus/aura/config.go +++ b/consensus/aura/config.go @@ -24,15 +24,16 @@ import ( "github.com/ledgerwatch/erigon/common" "github.com/ledgerwatch/erigon/common/hexutil" "github.com/ledgerwatch/erigon/common/u256" + "github.com/ledgerwatch/erigon/consensus" ) // 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) +func GetFromValidatorSet(set ValidatorSet, parent common.Hash, nonce uint, call consensus.Call) (common.Address, error) { + //d, err := set.defaultCaller(parent) + //if err != nil { + // return common.Address{}, err + //} + return set.getWithCaller(parent, nonce, call) } // Different ways of specifying validators. @@ -52,7 +53,7 @@ func newValidatorSetFromJson(j *ValidatorSetJson, posdaoTransition *uint64) Vali return &SimpleList{validators: j.List} } if j.SafeContract != nil { - return &ValidatorSafeContract{contractAddress: *j.SafeContract, posdaoTransition: posdaoTransition} + return NewValidatorSafeContract(*j.SafeContract, posdaoTransition, nil) } if j.Contract != nil { return &ValidatorContract{ diff --git a/consensus/aura/finality_test.go b/consensus/aura/finality_test.go new file mode 100644 index 000000000..f50a23e54 --- /dev/null +++ b/consensus/aura/finality_test.go @@ -0,0 +1,72 @@ +package aura + +import ( + "testing" + + "github.com/ledgerwatch/erigon/common" + "github.com/stretchr/testify/assert" +) + +func TestRollingFinality(t *testing.T) { + t.Run("RejectsUnknownSigners", func(t *testing.T) { + f := NewRollingFinality([]common.Address{{1}, {2}, {3}}) + _, err := f.push(common.Hash{}, 0, []common.Address{{0}, {4}}) + assert.Error(t, err) + _, err = f.push(common.Hash{}, 0, []common.Address{{0}, {1}, {4}}) + assert.Error(t, err) + }) + t.Run("FinalizeMultiple", func(t *testing.T) { + signers := []common.Address{{0}, {1}, {2}, {3}, {4}, {5}} + f := NewRollingFinality(signers) + // 3 / 6 signers is < 51% so no finality. + for i := 0; i < 6; i++ { + l, err := f.push(common.Hash{byte(i)}, uint64(i%3), []common.Address{signers[i%3]}) + assert.NoError(t, err) + assert.Equal(t, 0, len(l)) + } + // after pushing a block signed by a fourth validator, the first four + // blocks of the unverified chain become verified. + l, err := f.push(common.Hash{byte(6)}, 6, []common.Address{signers[4]}) + assert.NoError(t, err) + for i := uint64(0); i < 4; i++ { + assert.Equal(t, common.Hash{byte(i)}, l[i].hash) + } + assert.Equal(t, 4, len(l)) + }) + t.Run("FromAncestry", func(t *testing.T) { + signers := []common.Address{{0}, {1}, {2}, {3}, {4}, {5}} + f := NewRollingFinality(signers) + i := 12 + get := func(hash common.Hash) ([]common.Address, common.Hash, common.Hash, uint64, bool) { + i-- + if i == -1 { + return nil, common.Hash{}, common.Hash{}, 0, false + } + return []common.Address{signers[i%6]}, common.Hash{byte(i)}, common.Hash{byte(i - 1)}, uint64(i), true + } + err := f.buildAncestrySubChain(get, common.Hash{11}, common.Hash{99}) + assert.NoError(t, err) + assert.Equal(t, 3, f.headers.l.Len()) + assert.Equal(t, common.Hash{11}, *f.lastPushed) + }) + t.Run("FromAncestryMultipleSigners", func(t *testing.T) { + signers := []common.Address{{0}, {1}, {2}, {3}, {4}, {5}} + f := NewRollingFinality(signers) + i := 12 + get := func(hash common.Hash) ([]common.Address, common.Hash, common.Hash, uint64, bool) { + i-- + if i == -1 { + return nil, common.Hash{}, common.Hash{}, 0, false + } + return []common.Address{signers[i%6], signers[(i+1)%6], signers[(i+2)%6]}, common.Hash{byte(i)}, common.Hash{byte(i - 1)}, uint64(i), true + } + err := f.buildAncestrySubChain(get, common.Hash{11}, common.Hash{99}) + assert.NoError(t, err) + + // only the last hash has < 51% of authorities' signatures + assert.Equal(t, 1, f.headers.l.Len()) + assert.Equal(t, common.Hash{11}, f.headers.Front().hash) + assert.Equal(t, common.Hash{11}, *f.lastPushed) + }) + +} diff --git a/consensus/aura/validators.go b/consensus/aura/validators.go index c8e8c963e..8d91815af 100644 --- a/consensus/aura/validators.go +++ b/consensus/aura/validators.go @@ -54,7 +54,7 @@ type ValidatorSet interface { // 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, EthcoreError> + //func generateEngineTransactions(_firstInEpoch bool, _header *types.Header, _call SystemCall) -> Result, EthcoreError> // Signalling that a new epoch has begun. // @@ -63,15 +63,15 @@ type ValidatorSet interface { // The caller provided here may not generate proofs. // // `first` is true if this is the first block in the set. - onEpochBegin(first bool, header *types.Header, caller consensus.SystemCall) error + onEpochBegin(firstInEpoch bool, header *types.Header, caller consensus.SystemCall) error // 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) + getWithCaller(parentHash common.Hash, nonce uint, caller consensus.Call) (common.Address, error) // Returns the current number of validators. - countWithCaller(parentHash common.Hash, caller Call) (uint64, error) + countWithCaller(parentHash common.Hash, caller consensus.Call) (uint64, error) // Recover the validator set from the given proof, the block number, and // whether this header is first in its set. @@ -81,11 +81,10 @@ type ValidatorSet interface { // // Returns the set, along with a flag indicating whether finality of a specific // hash should be proven. - - epochSet(first bool, num uint64, proof []byte) (SimpleList, common.Hash, error) + epochSet(firstInEpoch bool, num uint64, setProof []byte, call consensus.SystemCall) (SimpleList, common.Hash, error) // Extract genesis epoch data from the genesis state and header. - genesisEpochData(header *types.Header, call Call) ([]byte, error) + genesisEpochData(header *types.Header, call consensus.Call) ([]byte, error) /* // Returns the current number of validators. @@ -121,7 +120,7 @@ type ValidatorSet interface { // that doesn't require finality. // // `first` is true if this is the first block in the set. - signalEpochEnd(first bool, header *types.Header, receipts types.Receipts) ([]byte, bool) + signalEpochEnd(firstInEpoch bool, header *types.Header, receipts types.Receipts) ([]byte, error) /* // Whether the given block signals the end of an epoch, but change won't take effect // until finality. @@ -178,19 +177,19 @@ type ValidatorSet interface { */ } -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) +func get(s ValidatorSet, h common.Hash, nonce uint, call consensus.Call) (common.Address, error) { + //d, err := s.defaultCaller(h) + //if err != nil { + // return common.Address{}, err + //} + return s.getWithCaller(h, nonce, call) } -func count(s ValidatorSet, h common.Hash) (uint64, error) { - d, err := s.defaultCaller(h) - if err != nil { - return 0, err - } - return s.countWithCaller(h, d) +func count(s ValidatorSet, h common.Hash, call consensus.Call) (uint64, error) { + //d, err := s.defaultCaller(h) + //if err != nil { + // return 0, err + //} + return s.countWithCaller(h, call) } //nolint @@ -232,10 +231,10 @@ func (s *Multi) defaultCaller(blockHash common.Hash) (Call, error) { return set.defaultCaller(blockHash) } -func (s *Multi) getWithCaller(parentHash common.Hash, nonce uint, caller Call) (common.Address, error) { +func (s *Multi) getWithCaller(parentHash common.Hash, nonce uint, caller consensus.Call) (common.Address, error) { panic("not implemented") } -func (s *Multi) countWithCaller(parentHash common.Hash, caller Call) (uint64, error) { +func (s *Multi) countWithCaller(parentHash common.Hash, caller consensus.Call) (uint64, error) { set, ok := s.correctSet(parentHash) if !ok { return math.MaxUint64, nil @@ -263,10 +262,10 @@ func (s *Multi) correctSetByNumber(parentNumber uint64) (uint64, ValidatorSet) { 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) { +func (s *Multi) get(num uint64) (firstInEpoch bool, set ValidatorSet) { block, set := s.correctSetByNumber(num) - first = block == num - return first, set + firstInEpoch = block == num + return firstInEpoch, set } func (s *Multi) onCloseBlock(header *types.Header, address common.Address) error { @@ -276,12 +275,12 @@ func (s *Multi) onCloseBlock(header *types.Header, address common.Address) error // TODO: do we need add `proof` argument? //nolint -func (s *Multi) epochSet(first bool, num uint64, proof []byte) (SimpleList, common.Hash, error) { +func (s *Multi) epochSet(firstInEpoch bool, num uint64, proof []byte, call consensus.SystemCall) (SimpleList, common.Hash, error) { setBlock, set := s.correctSetByNumber(num) - first = setBlock == num - return set.epochSet(first, num, proof) + firstInEpoch = setBlock == num + return set.epochSet(firstInEpoch, num, proof, call) } -func (s *Multi) genesisEpochData(header *types.Header, call Call) ([]byte, error) { +func (s *Multi) genesisEpochData(header *types.Header, call consensus.Call) ([]byte, error) { _, set := s.correctSetByNumber(0) return set.genesisEpochData(header, call) } @@ -290,7 +289,7 @@ func (s *Multi) onEpochBegin(_ bool, header *types.Header, caller consensus.Syst setTransition, set := s.correctSetByNumber(header.Number.Uint64()) return set.onEpochBegin(setTransition == header.Number.Uint64(), header, caller) } -func (s *Multi) signalEpochEnd(_ bool, header *types.Header, r types.Receipts) ([]byte, bool) { +func (s *Multi) signalEpochEnd(_ bool, header *types.Header, r types.Receipts) ([]byte, error) { num := header.Number.Uint64() setBlock, set := s.correctSetByNumber(num) first := setBlock == num @@ -301,31 +300,31 @@ type SimpleList struct { validators []common.Address } -func (s *SimpleList) epochSet(first bool, num uint64, proof []byte) (SimpleList, common.Hash, error) { +func (s *SimpleList) epochSet(firstInEpoch bool, num uint64, proof []byte, call consensus.SystemCall) (SimpleList, common.Hash, error) { return *s, common.Hash{}, nil } -func (s *SimpleList) onEpochBegin(first bool, header *types.Header, caller consensus.SystemCall) error { +func (s *SimpleList) onEpochBegin(firstInEpoch bool, header *types.Header, caller consensus.SystemCall) error { return nil } func (s *SimpleList) onCloseBlock(_header *types.Header, _address common.Address) error { return nil } func (s *SimpleList) defaultCaller(blockHash common.Hash) (Call, error) { return nil, nil //simple list doesn't require calls } -func (s *SimpleList) getWithCaller(parentHash common.Hash, nonce uint, caller Call) (common.Address, error) { +func (s *SimpleList) getWithCaller(parentHash common.Hash, nonce uint, caller consensus.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 } -func (s *SimpleList) countWithCaller(parentHash common.Hash, caller Call) (uint64, error) { +func (s *SimpleList) countWithCaller(parentHash common.Hash, caller consensus.Call) (uint64, error) { return uint64(len(s.validators)), nil } -func (s *SimpleList) genesisEpochData(header *types.Header, caller Call) ([]byte, error) { +func (s *SimpleList) genesisEpochData(header *types.Header, caller consensus.Call) ([]byte, error) { return []byte{}, nil } -func (s *SimpleList) signalEpochEnd(_ bool, header *types.Header, r types.Receipts) ([]byte, bool) { - return nil, false +func (s *SimpleList) signalEpochEnd(_ bool, header *types.Header, r types.Receipts) ([]byte, error) { + return nil, nil } // Draws an validator nonce modulo number of validators. @@ -340,6 +339,8 @@ type ReportQueueItem struct { blockNum uint64 data []byte } + +//nolint type ReportQueue struct { mu sync.RWMutex list *list.List @@ -353,7 +354,7 @@ func (q *ReportQueue) push(addr common.Address, blockNum uint64, data []byte) { } // Filters reports of validators that have already been reported or are banned. - +//nolint func (q *ReportQueue) filter(abi aurainterfaces.ValidatorSetABI, client client, ourAddr, contractAddr common.Address) error { q.mu.Lock() defer q.mu.Unlock() @@ -383,6 +384,7 @@ func (q *ReportQueue) filter(abi aurainterfaces.ValidatorSetABI, client client, } // Removes reports from the queue if it contains more than `MAX_QUEUED_REPORTS` entries. +//nolint func (q *ReportQueue) truncate() { // The maximum number of reports to keep queued. const MaxQueuedReports = 10 @@ -414,17 +416,22 @@ type ValidatorSafeContract struct { // with POSDAO modifications. posdaoTransition *uint64 - abi aurainterfaces.ValidatorSetABI + abi abi.ABI client client } -func NewValidatorSafeContract(contractAddress common.Address, posdaoTransition *uint64, abi aurainterfaces.ValidatorSetABI, client client) *ValidatorSafeContract { +func NewValidatorSafeContract(contractAddress common.Address, posdaoTransition *uint64, 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} + + parsed, err := abi.JSON(strings.NewReader(auraabi.ValidatorSetABI)) + if err != nil { + panic(err) + } + return &ValidatorSafeContract{contractAddress: contractAddress, posdaoTransition: posdaoTransition, validators: c, abi: parsed} } // Called for each new block this node is creating. If this block is @@ -432,31 +439,45 @@ func NewValidatorSafeContract(contractAddress common.Address, posdaoTransition * // 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, EthcoreError> +//func generateEngineTransactions(_firstInEpoch bool, _header *types.Header, _call SystemCall) -> Result, EthcoreError> +func (s *ValidatorSafeContract) epochSet(firstInEpoch bool, num uint64, setProof []byte, call consensus.SystemCall) (SimpleList, common.Hash, error) { + if firstInEpoch { + var proof FirstValidatorSetProof + if err := rlp.DecodeBytes(setProof, &proof); err != nil { + return SimpleList{}, common.Hash{}, fmt.Errorf("[ValidatorSafeContract.epochSet] %w", err) + } -func (s *ValidatorSafeContract) epochSet(first bool, num uint64, proofRlp []byte) (SimpleList, common.Hash, error) { - var proof EpochTransitionProof - if err := rlp.DecodeBytes(proofRlp, &proof); err != nil { + if num == 0 { + return *NewSimpleList([]common.Address{proof.Header.Coinbase}), proof.Header.ParentHash, nil + } + l, ok := s.getListSyscall(call) + if !ok { + panic(1) + } + + //addresses, err := checkFirstValidatorSetProof(s.contractAddress, oldHeader, state_items) + //if err != nil { + // panic(err) + // return SimpleList{}, common.Hash{}, fmt.Errorf("insufitient proof: block=%d,%x: %w", oldHeader.Number.Uint64(), oldHeader.Hash(), err) + //} + + //fmt.Printf("aaaa: %x,%x\n", common.HexToAddress("0xe8ddc5c7a2d2f0d7a9798459c0104fdf5e987aca"), params.SokolGenesisHash) + //fmt.Printf("bbbbb: %x,%x\n", proof.ContractAddress, proof.Header.Hash()) + return *l, proof.Header.ParentHash, nil + } + var proof ValidatorSetProof + if err := rlp.DecodeBytes(setProof, &proof); err != nil { return SimpleList{}, common.Hash{}, fmt.Errorf("[ValidatorSafeContract.epochSet] %w", err) } - if first { - oldHeader, state_items, err := decodeFirstValidatorSetProof(proof.SetProof) - if err != nil { - return SimpleList{}, common.Hash{}, err - } + if num > DEBUG_LOG_FROM { + fmt.Printf("epoch_set1: %d,%d,%d\n", proof.Header.Number.Uint64(), len(setProof), len(proof.Receipts)) + } + ll, ok := s.extractFromEvent(proof.Header, proof.Receipts) + if !ok { + panic(1) + } - addresses, err := checkFirstValidatorSetProof(s.contractAddress, oldHeader, state_items) - if err != nil { - return SimpleList{}, common.Hash{}, fmt.Errorf("insufitient proof: block=%d,%x: %w", oldHeader.Number.Uint64(), oldHeader.Hash(), err) - } - return *NewSimpleList(addresses), oldHeader.Hash(), nil - } - setProof, err := decodeValidatorSetProof(proof.SetProof) - if err != nil { - return SimpleList{}, common.Hash{}, err - } - _ = setProof // ensure receipts match header. // TODO: optimize? these were just decoded. /* @@ -479,34 +500,19 @@ func (s *ValidatorSafeContract) epochSet(first bool, num uint64, proofRlp []byte .into()), } */ + return *ll, common.Hash{}, nil + /* + setProof, err := decodeValidatorSetProof(proof.SetProof) + if err != nil { + return SimpleList{}, common.Hash{}, err + } + _ = setProof + */ - return SimpleList{}, common.Hash{}, fmt.Errorf("ValidatorSafeContract.epochSet not implemented") -} - -// first proof is just a state proof call of `getValidators` at header's state. -//nolint -func encodeFirstValidatorSetProof(h *types.Header, dbItems [][]byte) (proofRlp []byte, err error) { - type S struct { - h *types.Header - dbItems [][]byte - } - res := S{h: h, dbItems: dbItems} - return rlp.EncodeToBytes(res) -} - -func decodeFirstValidatorSetProof(proofRlp []byte) (h *types.Header, dbItems [][]byte, err error) { - type S struct { - h *types.Header - dbItems [][]byte - } - var res S - if err := rlp.DecodeBytes(proofRlp, &res); err != nil { - return nil, nil, err - } - return res.h, res.dbItems, nil } // check a first proof: fetch the validator set at the given block. +//nolint func checkFirstValidatorSetProof(contract_address common.Address, oldHeader *types.Header, dbItems [][]byte) ([]common.Address, error) { /* fn check_first_proof( @@ -574,26 +580,15 @@ func checkFirstValidatorSetProof(contract_address common.Address, oldHeader *typ // checking will involve ensuring that the receipts match the header and // extracting the validator set from the receipts. //nolint -func encodeValidatorSetProof(p ValidatorSetProof) (proofRlp []byte, err error) { - return rlp.EncodeToBytes(p) -} -func decodeValidatorSetProof(proofRlp []byte) (ValidatorSetProof, error) { - var res ValidatorSetProof - if err := rlp.DecodeBytes(proofRlp, &res); err != nil { - return res, err - } - return res, nil -} - 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) { +func (s *ValidatorSafeContract) getWithCaller(blockHash common.Hash, nonce uint, caller consensus.Call) (common.Address, error) { set, ok := s.validators.Get(blockHash) if ok { - return get(set.(ValidatorSet), blockHash, nonce) + return get(set.(ValidatorSet), blockHash, nonce, caller) } list, ok := s.getList(caller) @@ -601,46 +596,60 @@ func (s *ValidatorSafeContract) getWithCaller(blockHash common.Hash, nonce uint, return common.Address{}, nil } s.validators.Add(blockHash, list) - return get(list, blockHash, nonce) + return get(list, blockHash, nonce, caller) } -func (s *ValidatorSafeContract) countWithCaller(parentHash common.Hash, caller Call) (uint64, error) { +func (s *ValidatorSafeContract) countWithCaller(parentHash common.Hash, caller consensus.Call) (uint64, error) { set, ok := s.validators.Get(parentHash) if ok { - return count(set.(ValidatorSet), parentHash) + return count(set.(ValidatorSet), parentHash, caller) } list, ok := s.getList(caller) if !ok { return math.MaxUint64, nil } s.validators.Add(parentHash, list) - return count(list, parentHash) + return count(list, parentHash, caller) } -func (s *ValidatorSafeContract) getList(caller Call) (*SimpleList, bool) { - code, decoder := s.abi.GetValidators() - callResult, err := caller(s.contractAddress, code) +func (s *ValidatorSafeContract) getList(caller consensus.Call) (*SimpleList, bool) { + packed, err := s.abi.Pack("getValidators") if err != nil { - log.Debug("Set of validators could not be updated: ", "err", err) - return nil, false + panic(err) } - 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) + out, err := caller(s.contractAddress, packed) if err != nil { - log.Debug("Set of validators could not be updated: ", "err", err) - return nil, false + panic(err) } - return NewSimpleList(res), true + res, err := s.abi.Unpack("getValidators", out) + if err != nil { + panic(err) + } + out0 := *abi.ConvertType(res[0], new([]common.Address)).(*[]common.Address) + return NewSimpleList(out0), true } -func (s *ValidatorSafeContract) genesisEpochData(header *types.Header, call Call) ([]byte, error) { - return proveInitial(s.contractAddress, header, call) +func (s *ValidatorSafeContract) getListSyscall(caller consensus.SystemCall) (*SimpleList, bool) { + packed, err := s.abi.Pack("getValidators") + if err != nil { + panic(err) + } + out, err := caller(s.contractAddress, packed) + if err != nil { + panic(err) + } + res, err := s.abi.Unpack("getValidators", out) + if err != nil { + panic(err) + } + out0 := *abi.ConvertType(res[0], new([]common.Address)).(*[]common.Address) + return NewSimpleList(out0), true } -func (s *ValidatorSafeContract) onEpochBegin(first bool, header *types.Header, caller consensus.SystemCall) error { +func (s *ValidatorSafeContract) genesisEpochData(header *types.Header, call consensus.Call) ([]byte, error) { + return proveInitial(s, s.contractAddress, header, call) +} + +func (s *ValidatorSafeContract) onEpochBegin(firstInEpoch bool, header *types.Header, caller consensus.SystemCall) error { data := common.FromHex("75286211") _, err := caller(s.contractAddress, data) if err != nil { @@ -657,20 +666,20 @@ func (s *ValidatorSafeContract) onEpochBegin(first bool, header *types.Header, c return nil } -func (s *ValidatorSafeContract) signalEpochEnd(first bool, header *types.Header, r types.Receipts) ([]byte, bool) { - //TODO: return real proof - return nil now - +func (s *ValidatorSafeContract) signalEpochEnd(firstInEpoch bool, header *types.Header, r types.Receipts) ([]byte, error) { + if header.Number.Uint64() >= DEBUG_LOG_FROM { + fmt.Printf("signalEpochEnd: %d,%t\n", header.Number.Uint64(), firstInEpoch) + } // transition to the first block of a contract requires finality but has no log event. - if first { + if firstInEpoch { /* - debug!(target: "engine", "signalling transition to fresh contract."); - let state_proof = Arc::new(StateProof { - contract_address: self.contract_address, - header: header.clone(), - }); - return ::engines::EpochChange::Yes(::engines::Proof::WithState(state_proof as Arc<_>)); + let state_proof = Arc::new(FirstValidatorSetProof { + contract_address: self.contract_address, + header: header.clone(), + }); + return ::engines::EpochChange::Yes(::engines::Proof::WithState(state_proof as Arc<_>)); */ - return nil, true + return rlp.EncodeToBytes(FirstValidatorSetProof{Header: header, ContractAddress: s.contractAddress}) } // otherwise, we're checking for logs. @@ -680,85 +689,87 @@ func (s *ValidatorSafeContract) signalEpochEnd(first bool, header *types.Header, // return ::engines::EpochChange::No; //} - /* - match receipts { - None => ::engines::EpochChange::Unsure(AuxiliaryRequest::Receipts), - Some(receipts) => match self.extract_from_event(bloom, header, receipts) { - None => ::engines::EpochChange::No, - Some(list) => { - info!(target: "engine", "Signal for transition within contract. New list: {:?}", - &*list); - - let proof = encode_proof(&header, receipts); - ::engines::EpochChange::Yes(::engines::Proof::Known(proof)) - } - }, - } - */ _, ok := s.extractFromEvent(header, r) - //if header.Number.Uint64() == 205821 { - // panic(1) - //} - return nil, ok + if !ok { + if header.Number.Uint64() >= DEBUG_LOG_FROM { + fmt.Printf("signalEpochEnd: no-no-no %d,%d\n", header.Number.Uint64(), len(r)) + } + return nil, nil + } + proof, err := rlp.EncodeToBytes(ValidatorSetProof{Header: header, Receipts: r}) + if err != nil { + return nil, err + } + if header.Number.Uint64() >= DEBUG_LOG_FROM { + fmt.Printf("signalEpochEnd: %d,%d, proofLen=%d\n", header.Number.Uint64(), len(r), len(proof)) + } + return proof, nil } func (s *ValidatorSafeContract) extractFromEvent(header *types.Header, receipts types.Receipts) (*SimpleList, bool) { if len(receipts) == 0 { + if header.Number.Uint64() >= DEBUG_LOG_FROM { + fmt.Printf("extractFromEvent1: %d\n", header.Number.Uint64()) + } return nil, false } + if header.Number.Uint64() >= DEBUG_LOG_FROM { + fmt.Printf("extractFromEvent111: %d,%d\n", header.Number.Uint64(), len(receipts)) + } // iterate in reverse because only the _last_ change in a given // block actually has any effect. // the contract should only increment the nonce once. - lastReceipt := receipts[len(receipts)-1] - if len(lastReceipt.Logs) == 0 { - return nil, false - } - logs := lastReceipt.Logs - //for i := len(logs) - 1; i >= 0; i-- { - for i := 0; i < len(logs); i++ { - l := logs[i] - if len(l.Topics) != 2 { - continue - } - found := l.Address == s.contractAddress && l.Topics[0] == EVENT_NAME_HASH && l.Topics[1] == header.ParentHash - if !found { - continue - } - - parsed, err := abi.JSON(strings.NewReader(auraabi.ValidatorSetABI)) - if err != nil { - panic(err) - } - contract := bind.NewBoundContract(l.Address, parsed, nil, nil, nil) - if err != nil { - panic(err) - } - event := new(auraabi.ValidatorSetInitiateChange) - if err := contract.UnpackLog(event, "InitiateChange", *l); err != nil { - panic(err) - } - _ = event.NewSet - - //auraabi.NewValidatorSetFilterer() - fmt.Printf("#ddd: %x\n", event.NewSet) - fmt.Printf("#aa: %x,%x\n", receipts[0].Logs[0].Topics[0], EVENT_NAME_HASH) - fmt.Printf("#bb: %x,%x\n", receipts[0].Logs[1].Topics[0], EVENT_NAME_HASH) - //if header.Number.Uint64() == 672 { - if header.Number.Uint64() == 205821 { - panic(1) - } - + for j := len(receipts) - 1; j >= 0; j-- { + logs := receipts[j].Logs /* - validator_set::events::initiate_change::parse_log( - (log.topics.clone(), log.data.clone()).into(), - ) + TODO: skipped next bloom check (is it required?) + expectedBloom := expected_bloom(&self, header: &Header) -> Bloom { + let topics = vec![*EVENT_NAME_HASH, *header.parent_hash()]; + + debug!(target: "engine", "Expected topics for header {}: {:?}", + header.hash(), topics); + + LogEntry { + address: self.contract_address, + topics: topics, + data: Vec::new(), // irrelevant for bloom. + } + .bloom() + } + if !r.log_bloom.contains_bloom(&bloom){ + continue + } */ - if found { - return nil, true + for i := 0; i < len(logs); i++ { + l := logs[i] + if header.Number.Uint64() >= DEBUG_LOG_FROM { + fmt.Printf("extractFromEvent3: %d\n", header.Number.Uint64()) + } + if len(l.Topics) != 2 { + continue + } + found := l.Address == s.contractAddress && l.Topics[0] == EVENT_NAME_HASH && l.Topics[1] == header.ParentHash + if !found { + if header.Number.Uint64() >= DEBUG_LOG_FROM { + fmt.Printf("extractFromEvent4: %d\n", header.Number.Uint64()) + } + continue + } + + contract := bind.NewBoundContract(l.Address, s.abi, nil, nil, nil) + event := new(auraabi.ValidatorSetInitiateChange) + if err := contract.UnpackLog(event, "InitiateChange", *l); err != nil { + panic(err) + } + if header.Number.Uint64() >= DEBUG_LOG_FROM { + fmt.Printf("extractFromEvent5: %d\n", header.Number.Uint64()) + } + + // only one last log is taken into account + return NewSimpleList(event.NewSet), true } } - /* let check_log = |log: &LogEntry| { log.address == self.contract_address @@ -803,11 +814,14 @@ func (s *ValidatorSafeContract) onCloseBlock(header *types.Header, ourAddress co 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() + /* + 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(); @@ -850,33 +864,39 @@ type ValidatorContract struct { posdaoTransition *uint64 } -func (s *ValidatorContract) epochSet(first bool, num uint64, proof []byte) (SimpleList, common.Hash, error) { - return s.validators.epochSet(first, num, proof) +func (s *ValidatorContract) epochSet(firstInEpoch bool, num uint64, proof []byte, call consensus.SystemCall) (SimpleList, common.Hash, error) { + return s.validators.epochSet(firstInEpoch, num, proof, call) } 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) { +func (s *ValidatorContract) getWithCaller(parentHash common.Hash, nonce uint, caller consensus.Call) (common.Address, error) { return s.validators.getWithCaller(parentHash, nonce, caller) } -func (s *ValidatorContract) countWithCaller(parentHash common.Hash, caller Call) (uint64, error) { +func (s *ValidatorContract) countWithCaller(parentHash common.Hash, caller consensus.Call) (uint64, error) { return s.validators.countWithCaller(parentHash, caller) } -func (s *ValidatorContract) onEpochBegin(first bool, header *types.Header, caller consensus.SystemCall) error { - return s.validators.onEpochBegin(first, header, caller) +func (s *ValidatorContract) onEpochBegin(firstInEpoch bool, header *types.Header, caller consensus.SystemCall) error { + return s.validators.onEpochBegin(firstInEpoch, header, caller) } func (s *ValidatorContract) onCloseBlock(header *types.Header, address common.Address) error { return s.validators.onCloseBlock(header, address) } -func (s *ValidatorContract) genesisEpochData(header *types.Header, call Call) ([]byte, error) { +func (s *ValidatorContract) genesisEpochData(header *types.Header, call consensus.Call) ([]byte, error) { return s.validators.genesisEpochData(header, call) } -func (s *ValidatorContract) signalEpochEnd(first bool, header *types.Header, r types.Receipts) ([]byte, bool) { - return s.validators.signalEpochEnd(first, header, r) +func (s *ValidatorContract) signalEpochEnd(firstInEpoch bool, header *types.Header, r types.Receipts) ([]byte, error) { + return s.validators.signalEpochEnd(firstInEpoch, header, r) } -func proveInitial(contractAddr common.Address, header *types.Header, caller Call) ([]byte, error) { - return common.FromHex("0xf91a84f9020da00000000000000000000000000000000000000000000000000000000000000000a01dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347940000000000000000000000000000000000000000a0fad4af258fd11939fae0c6c6eec9d340b1caac0b0196fd9a1bc3f489c5bf00b3a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421b9010000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000830200008083663be080808080b8410000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000f91871b8d3f8d1a0dc277c93a9f9dcee99aac9b8ba3cfa4c51821998522469c37715644e8fbac0bfa0ab8cdb808c8303bb61fb48e276217be9770fa83ecf3f90f2234d558885f5abf1808080a0fe137c3a474fbde41d89a59dd76da4c55bf696b86d3af64a55632f76cf30786780808080a06301b39b2ea8a44df8b0356120db64b788e71f52e1d7a6309d0d2e5b86fee7cb80a0da5d8b08dea0c5a4799c0f44d8a24d7cdf209f9b7a5588c1ecafb5361f6b9f07a01b7779e149cadf24d4ffb77ca7e11314b8db7097e4d70b2a173493153ca2e5a0808080b8f3f8f1a08023c0d95fc2364e0bf7593f5ff32e1db8ef9f4b41c0bd474eae62d1af896e99808080a0b47b4f0b3e73b5edc8f9a9da1cbcfed562eb06bf54619b6aefeadebf5b3604c280a0da6ec08940a924cb08c947dd56cdb40076b29a6f0ea4dba4e2d02d9a9a72431b80a030cc4138c9e74b6cf79d624b4b5612c0fd888e91f55316cfee7d1694e1a90c0b80a0c5d54b915b56a888eee4e6eeb3141e778f9b674d1d322962eed900f02c29990aa017256b36ef47f907c6b1378a2636942ce894c17075e56fc054d4283f6846659e808080a03340bbaeafcda3a8672eb83099231dbbfab8dae02a1e8ec2f7180538fac207e080b838f7a03868bdfa8727775661e4ccf117824a175a33f8703d728c04488fbfffcafda9f99594e8ddc5c7a2d2f0d7a9798459c0104fdf5e987acaa3e2a02052222313e28459528d920b65115c16c04f3efc82aaedc97be59f3f377c0d3f01b853f851808080a0a87d9bb950836582673aa0eecc0ff64aac607870637a2dd2012b8b1b31981f698080a08da6d5c36a404670c553a2c9052df7cd604f04e3863c4c7b9e0027bfd54206d680808080808080808080b86bf869a033aa5d69545785694b808840be50c182dad2ec3636dfccbe6572fb69828742c0b846f8440101a0663ce0d171e545a26aa67e4ca66f72ba96bb48287dbcc03beea282867f80d44ba01f0e7726926cb43c03a0abf48197dba78522ec8ba1b158e2aa30da7d2a2c6f9eb86bf869a02080c7b7ae81a58eb98d9c78de4a1fd7fd9535fc953ed2be602daaa41767312ab846f8448080a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421a0c5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470b853f851808080a07bb75cabebdcbd1dbb4331054636d0c6d7a2b08483b9e04df057395a7434c9e080808080808080a0e61e567237b49c44d8f906ceea49027260b4010c10a547b38d8b131b9d3b6f848080808080b914c26060604052600436106100fc576000357c0100000000000000000000000000000000000000000000000000000000900463ffffffff16806303aca79214610101578063108552691461016457806340a141ff1461019d57806340c9cdeb146101d65780634110a489146101ff57806345199e0a1461025757806349285b58146102c15780634d238c8e14610316578063752862111461034f578063900eb5a8146103645780639a573786146103c7578063a26a47d21461041c578063ae4b1b5b14610449578063b3f05b971461049e578063b7ab4db5146104cb578063d3e848f114610535578063fa81b2001461058a578063facd743b146105df575b600080fd5b341561010c57600080fd5b6101226004808035906020019091905050610630565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b341561016f57600080fd5b61019b600480803573ffffffffffffffffffffffffffffffffffffffff1690602001909190505061066f565b005b34156101a857600080fd5b6101d4600480803573ffffffffffffffffffffffffffffffffffffffff16906020019091905050610807565b005b34156101e157600080fd5b6101e9610bb7565b6040518082815260200191505060405180910390f35b341561020a57600080fd5b610236600480803573ffffffffffffffffffffffffffffffffffffffff16906020019091905050610bbd565b60405180831515151581526020018281526020019250505060405180910390f35b341561026257600080fd5b61026a610bee565b6040518080602001828103825283818151815260200191508051906020019060200280838360005b838110156102ad578082015181840152602081019050610292565b505050509050019250505060405180910390f35b34156102cc57600080fd5b6102d4610c82565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b341561032157600080fd5b61034d600480803573ffffffffffffffffffffffffffffffffffffffff16906020019091905050610d32565b005b341561035a57600080fd5b610362610fcc565b005b341561036f57600080fd5b61038560048080359060200190919050506110fc565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b34156103d257600080fd5b6103da61113b565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b341561042757600080fd5b61042f6111eb565b604051808215151515815260200191505060405180910390f35b341561045457600080fd5b61045c6111fe565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b34156104a957600080fd5b6104b1611224565b604051808215151515815260200191505060405180910390f35b34156104d657600080fd5b6104de611237565b6040518080602001828103825283818151815260200191508051906020019060200280838360005b83811015610521578082015181840152602081019050610506565b505050509050019250505060405180910390f35b341561054057600080fd5b6105486112cb565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b341561059557600080fd5b61059d6112f1565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b34156105ea57600080fd5b610616600480803573ffffffffffffffffffffffffffffffffffffffff16906020019091905050611317565b604051808215151515815260200191505060405180910390f35b60078181548110151561063f57fe5b90600052602060002090016000915054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b600460029054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff161415156106cb57600080fd5b600460019054906101000a900460ff161515156106e757600080fd5b600073ffffffffffffffffffffffffffffffffffffffff168173ffffffffffffffffffffffffffffffffffffffff161415151561072357600080fd5b80600a60006101000a81548173ffffffffffffffffffffffffffffffffffffffff021916908373ffffffffffffffffffffffffffffffffffffffff1602179055506001600460016101000a81548160ff0219169083151502179055507f600bcf04a13e752d1e3670a5a9f1c21177ca2a93c6f5391d4f1298d098097c22600a60009054906101000a900473ffffffffffffffffffffffffffffffffffffffff16604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390a150565b600080600061081461113b565b73ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff1614151561084d57600080fd5b83600960008273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200190815260200160002060000160009054906101000a900460ff1615156108a957600080fd5b600960008673ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff168152602001908152602001600020600101549350600160078054905003925060078381548110151561090857fe5b906000526020600020900160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1691508160078581548110151561094657fe5b906000526020600020900160006101000a81548173ffffffffffffffffffffffffffffffffffffffff021916908373ffffffffffffffffffffffffffffffffffffffff16021790555083600960008473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff168152602001908152602001600020600101819055506007838154811015156109e557fe5b906000526020600020900160006101000a81549073ffffffffffffffffffffffffffffffffffffffff02191690556000600780549050111515610a2757600080fd5b6007805480919060019003610a3c9190611370565b506000600960008773ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff168152602001908152602001600020600101819055506000600960008773ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200190815260200160002060000160006101000a81548160ff0219169083151502179055506000600460006101000a81548160ff0219169083151502179055506001430340600019167f55252fa6eee4741b4e24a74a70e9c11fd2c2281df8d6ea13126ff845f7825c89600760405180806020018281038252838181548152602001915080548015610ba257602002820191906000526020600020905b8160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff1681526020019060010190808311610b58575b50509250505060405180910390a25050505050565b60085481565b60096020528060005260406000206000915090508060000160009054906101000a900460ff16908060010154905082565b610bf661139c565b6007805480602002602001604051908101604052809291908181526020018280548015610c7857602002820191906000526020600020905b8160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff1681526020019060010190808311610c2e575b5050505050905090565b6000600a60009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff166349285b586000604051602001526040518163ffffffff167c0100000000000000000000000000000000000000000000000000000000028152600401602060405180830381600087803b1515610d1257600080fd5b6102c65a03f11515610d2357600080fd5b50505060405180519050905090565b610d3a61113b565b73ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff16141515610d7357600080fd5b80600960008273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200190815260200160002060000160009054906101000a900460ff16151515610dd057600080fd5b600073ffffffffffffffffffffffffffffffffffffffff168273ffffffffffffffffffffffffffffffffffffffff1614151515610e0c57600080fd5b6040805190810160405280600115158152602001600780549050815250600960008473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200190815260200160002060008201518160000160006101000a81548160ff0219169083151502179055506020820151816001015590505060078054806001018281610ea991906113b0565b9160005260206000209001600084909190916101000a81548173ffffffffffffffffffffffffffffffffffffffff021916908373ffffffffffffffffffffffffffffffffffffffff160217905550506000600460006101000a81548160ff0219169083151502179055506001430340600019167f55252fa6eee4741b4e24a74a70e9c11fd2c2281df8d6ea13126ff845f7825c89600760405180806020018281038252838181548152602001915080548015610fba57602002820191906000526020600020905b8160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff1681526020019060010190808311610f70575b50509250505060405180910390a25050565b600560009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff161480156110365750600460009054906101000a900460ff16155b151561104157600080fd5b6001600460006101000a81548160ff0219169083151502179055506007600690805461106e9291906113dc565b506006805490506008819055507f8564cd629b15f47dc310d45bcbfc9bcf5420b0d51bf0659a16c67f91d27632536110a4611237565b6040518080602001828103825283818151815260200191508051906020019060200280838360005b838110156110e75780820151818401526020810190506110cc565b505050509050019250505060405180910390a1565b60068181548110151561110b57fe5b90600052602060002090016000915054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b6000600a60009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16639a5737866000604051602001526040518163ffffffff167c0100000000000000000000000000000000000000000000000000000000028152600401602060405180830381600087803b15156111cb57600080fd5b6102c65a03f115156111dc57600080fd5b50505060405180519050905090565b600460019054906101000a900460ff1681565b600a60009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b600460009054906101000a900460ff1681565b61123f61139c565b60068054806020026020016040519081016040528092919081815260200182805480156112c157602002820191906000526020600020905b8160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff1681526020019060010190808311611277575b5050505050905090565b600560009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b600460029054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b6000600960008373ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200190815260200160002060000160009054906101000a900460ff169050919050565b81548183558181151161139757818360005260206000209182019101611396919061142e565b5b505050565b602060405190810160405280600081525090565b8154818355818115116113d7578183600052602060002091820191016113d6919061142e565b5b505050565b82805482825590600052602060002090810192821561141d5760005260206000209182015b8281111561141c578254825591600101919060010190611401565b5b50905061142a9190611453565b5090565b61145091905b8082111561144c576000816000905550600101611434565b5090565b90565b61149391905b8082111561148f57600081816101000a81549073ffffffffffffffffffffffffffffffffffffffff021916905550600101611459565b5090565b905600a165627a7a7230582036ea35935c8246b68074adece2eab70c40e69a0193c08a6277ce06e5b25188510029"), nil +func proveInitial(s *ValidatorSafeContract, contractAddr common.Address, header *types.Header, caller consensus.Call) ([]byte, error) { + fmt.Printf("prove initial: %d, %x\n", header.Number.Uint64(), contractAddr) + return rlp.EncodeToBytes(FirstValidatorSetProof{Header: header, ContractAddress: s.contractAddress}) + //list, err := s.getList(caller) + //fmt.Printf("aaa: %x,%t\n", list, err) + + //return common.FromHex("0xf91a84f9020da00000000000000000000000000000000000000000000000000000000000000000a01dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347940000000000000000000000000000000000000000a0fad4af258fd11939fae0c6c6eec9d340b1caac0b0196fd9a1bc3f489c5bf00b3a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421b9010000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000830200008083663be080808080b8410000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000f91871b914c26060604052600436106100fc576000357c0100000000000000000000000000000000000000000000000000000000900463ffffffff16806303aca79214610101578063108552691461016457806340a141ff1461019d57806340c9cdeb146101d65780634110a489146101ff57806345199e0a1461025757806349285b58146102c15780634d238c8e14610316578063752862111461034f578063900eb5a8146103645780639a573786146103c7578063a26a47d21461041c578063ae4b1b5b14610449578063b3f05b971461049e578063b7ab4db5146104cb578063d3e848f114610535578063fa81b2001461058a578063facd743b146105df575b600080fd5b341561010c57600080fd5b6101226004808035906020019091905050610630565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b341561016f57600080fd5b61019b600480803573ffffffffffffffffffffffffffffffffffffffff1690602001909190505061066f565b005b34156101a857600080fd5b6101d4600480803573ffffffffffffffffffffffffffffffffffffffff16906020019091905050610807565b005b34156101e157600080fd5b6101e9610bb7565b6040518082815260200191505060405180910390f35b341561020a57600080fd5b610236600480803573ffffffffffffffffffffffffffffffffffffffff16906020019091905050610bbd565b60405180831515151581526020018281526020019250505060405180910390f35b341561026257600080fd5b61026a610bee565b6040518080602001828103825283818151815260200191508051906020019060200280838360005b838110156102ad578082015181840152602081019050610292565b505050509050019250505060405180910390f35b34156102cc57600080fd5b6102d4610c82565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b341561032157600080fd5b61034d600480803573ffffffffffffffffffffffffffffffffffffffff16906020019091905050610d32565b005b341561035a57600080fd5b610362610fcc565b005b341561036f57600080fd5b61038560048080359060200190919050506110fc565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b34156103d257600080fd5b6103da61113b565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b341561042757600080fd5b61042f6111eb565b604051808215151515815260200191505060405180910390f35b341561045457600080fd5b61045c6111fe565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b34156104a957600080fd5b6104b1611224565b604051808215151515815260200191505060405180910390f35b34156104d657600080fd5b6104de611237565b6040518080602001828103825283818151815260200191508051906020019060200280838360005b83811015610521578082015181840152602081019050610506565b505050509050019250505060405180910390f35b341561054057600080fd5b6105486112cb565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b341561059557600080fd5b61059d6112f1565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b34156105ea57600080fd5b610616600480803573ffffffffffffffffffffffffffffffffffffffff16906020019091905050611317565b604051808215151515815260200191505060405180910390f35b60078181548110151561063f57fe5b90600052602060002090016000915054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b600460029054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff161415156106cb57600080fd5b600460019054906101000a900460ff161515156106e757600080fd5b600073ffffffffffffffffffffffffffffffffffffffff168173ffffffffffffffffffffffffffffffffffffffff161415151561072357600080fd5b80600a60006101000a81548173ffffffffffffffffffffffffffffffffffffffff021916908373ffffffffffffffffffffffffffffffffffffffff1602179055506001600460016101000a81548160ff0219169083151502179055507f600bcf04a13e752d1e3670a5a9f1c21177ca2a93c6f5391d4f1298d098097c22600a60009054906101000a900473ffffffffffffffffffffffffffffffffffffffff16604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390a150565b600080600061081461113b565b73ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff1614151561084d57600080fd5b83600960008273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200190815260200160002060000160009054906101000a900460ff1615156108a957600080fd5b600960008673ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff168152602001908152602001600020600101549350600160078054905003925060078381548110151561090857fe5b906000526020600020900160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1691508160078581548110151561094657fe5b906000526020600020900160006101000a81548173ffffffffffffffffffffffffffffffffffffffff021916908373ffffffffffffffffffffffffffffffffffffffff16021790555083600960008473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff168152602001908152602001600020600101819055506007838154811015156109e557fe5b906000526020600020900160006101000a81549073ffffffffffffffffffffffffffffffffffffffff02191690556000600780549050111515610a2757600080fd5b6007805480919060019003610a3c9190611370565b506000600960008773ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff168152602001908152602001600020600101819055506000600960008773ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200190815260200160002060000160006101000a81548160ff0219169083151502179055506000600460006101000a81548160ff0219169083151502179055506001430340600019167f55252fa6eee4741b4e24a74a70e9c11fd2c2281df8d6ea13126ff845f7825c89600760405180806020018281038252838181548152602001915080548015610ba257602002820191906000526020600020905b8160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff1681526020019060010190808311610b58575b50509250505060405180910390a25050505050565b60085481565b60096020528060005260406000206000915090508060000160009054906101000a900460ff16908060010154905082565b610bf661139c565b6007805480602002602001604051908101604052809291908181526020018280548015610c7857602002820191906000526020600020905b8160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff1681526020019060010190808311610c2e575b5050505050905090565b6000600a60009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff166349285b586000604051602001526040518163ffffffff167c0100000000000000000000000000000000000000000000000000000000028152600401602060405180830381600087803b1515610d1257600080fd5b6102c65a03f11515610d2357600080fd5b50505060405180519050905090565b610d3a61113b565b73ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff16141515610d7357600080fd5b80600960008273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200190815260200160002060000160009054906101000a900460ff16151515610dd057600080fd5b600073ffffffffffffffffffffffffffffffffffffffff168273ffffffffffffffffffffffffffffffffffffffff1614151515610e0c57600080fd5b6040805190810160405280600115158152602001600780549050815250600960008473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200190815260200160002060008201518160000160006101000a81548160ff0219169083151502179055506020820151816001015590505060078054806001018281610ea991906113b0565b9160005260206000209001600084909190916101000a81548173ffffffffffffffffffffffffffffffffffffffff021916908373ffffffffffffffffffffffffffffffffffffffff160217905550506000600460006101000a81548160ff0219169083151502179055506001430340600019167f55252fa6eee4741b4e24a74a70e9c11fd2c2281df8d6ea13126ff845f7825c89600760405180806020018281038252838181548152602001915080548015610fba57602002820191906000526020600020905b8160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff1681526020019060010190808311610f70575b50509250505060405180910390a25050565b600560009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff161480156110365750600460009054906101000a900460ff16155b151561104157600080fd5b6001600460006101000a81548160ff0219169083151502179055506007600690805461106e9291906113dc565b506006805490506008819055507f8564cd629b15f47dc310d45bcbfc9bcf5420b0d51bf0659a16c67f91d27632536110a4611237565b6040518080602001828103825283818151815260200191508051906020019060200280838360005b838110156110e75780820151818401526020810190506110cc565b505050509050019250505060405180910390a1565b60068181548110151561110b57fe5b90600052602060002090016000915054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b6000600a60009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16639a5737866000604051602001526040518163ffffffff167c0100000000000000000000000000000000000000000000000000000000028152600401602060405180830381600087803b15156111cb57600080fd5b6102c65a03f115156111dc57600080fd5b50505060405180519050905090565b600460019054906101000a900460ff1681565b600a60009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b600460009054906101000a900460ff1681565b61123f61139c565b60068054806020026020016040519081016040528092919081815260200182805480156112c157602002820191906000526020600020905b8160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff1681526020019060010190808311611277575b5050505050905090565b600560009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b600460029054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b6000600960008373ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200190815260200160002060000160009054906101000a900460ff169050919050565b81548183558181151161139757818360005260206000209182019101611396919061142e565b5b505050565b602060405190810160405280600081525090565b8154818355818115116113d7578183600052602060002091820191016113d6919061142e565b5b505050565b82805482825590600052602060002090810192821561141d5760005260206000209182015b8281111561141c578254825591600101919060010190611401565b5b50905061142a9190611453565b5090565b61145091905b8082111561144c576000816000905550600101611434565b5090565b90565b61149391905b8082111561148f57600081816101000a81549073ffffffffffffffffffffffffffffffffffffffff021916905550600101611459565b5090565b905600a165627a7a7230582036ea35935c8246b68074adece2eab70c40e69a0193c08a6277ce06e5b25188510029b8f3f8f1a08023c0d95fc2364e0bf7593f5ff32e1db8ef9f4b41c0bd474eae62d1af896e99808080a0b47b4f0b3e73b5edc8f9a9da1cbcfed562eb06bf54619b6aefeadebf5b3604c280a0da6ec08940a924cb08c947dd56cdb40076b29a6f0ea4dba4e2d02d9a9a72431b80a030cc4138c9e74b6cf79d624b4b5612c0fd888e91f55316cfee7d1694e1a90c0b80a0c5d54b915b56a888eee4e6eeb3141e778f9b674d1d322962eed900f02c29990aa017256b36ef47f907c6b1378a2636942ce894c17075e56fc054d4283f6846659e808080a03340bbaeafcda3a8672eb83099231dbbfab8dae02a1e8ec2f7180538fac207e080b86bf869a033aa5d69545785694b808840be50c182dad2ec3636dfccbe6572fb69828742c0b846f8440101a0663ce0d171e545a26aa67e4ca66f72ba96bb48287dbcc03beea282867f80d44ba01f0e7726926cb43c03a0abf48197dba78522ec8ba1b158e2aa30da7d2a2c6f9eb838f7a03868bdfa8727775661e4ccf117824a175a33f8703d728c04488fbfffcafda9f99594e8ddc5c7a2d2f0d7a9798459c0104fdf5e987acaa3e2a02052222313e28459528d920b65115c16c04f3efc82aaedc97be59f3f377c0d3f01b853f851808080a07bb75cabebdcbd1dbb4331054636d0c6d7a2b08483b9e04df057395a7434c9e080808080808080a0e61e567237b49c44d8f906ceea49027260b4010c10a547b38d8b131b9d3b6f848080808080b8d3f8d1a0dc277c93a9f9dcee99aac9b8ba3cfa4c51821998522469c37715644e8fbac0bfa0ab8cdb808c8303bb61fb48e276217be9770fa83ecf3f90f2234d558885f5abf1808080a0fe137c3a474fbde41d89a59dd76da4c55bf696b86d3af64a55632f76cf30786780808080a06301b39b2ea8a44df8b0356120db64b788e71f52e1d7a6309d0d2e5b86fee7cb80a0da5d8b08dea0c5a4799c0f44d8a24d7cdf209f9b7a5588c1ecafb5361f6b9f07a01b7779e149cadf24d4ffb77ca7e11314b8db7097e4d70b2a173493153ca2e5a0808080b853f851808080a0a87d9bb950836582673aa0eecc0ff64aac607870637a2dd2012b8b1b31981f698080a08da6d5c36a404670c553a2c9052df7cd604f04e3863c4c7b9e0027bfd54206d680808080808080808080b86bf869a02080c7b7ae81a58eb98d9c78de4a1fd7fd9535fc953ed2be602daaa41767312ab846f8448080a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421a0c5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470"), nil + //return common.FromHex("0xf91a84f9020da00000000000000000000000000000000000000000000000000000000000000000a01dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347940000000000000000000000000000000000000000a0fad4af258fd11939fae0c6c6eec9d340b1caac0b0196fd9a1bc3f489c5bf00b3a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421b9010000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000830200008083663be080808080b8410000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000f91871b8d3f8d1a0dc277c93a9f9dcee99aac9b8ba3cfa4c51821998522469c37715644e8fbac0bfa0ab8cdb808c8303bb61fb48e276217be9770fa83ecf3f90f2234d558885f5abf1808080a0fe137c3a474fbde41d89a59dd76da4c55bf696b86d3af64a55632f76cf30786780808080a06301b39b2ea8a44df8b0356120db64b788e71f52e1d7a6309d0d2e5b86fee7cb80a0da5d8b08dea0c5a4799c0f44d8a24d7cdf209f9b7a5588c1ecafb5361f6b9f07a01b7779e149cadf24d4ffb77ca7e11314b8db7097e4d70b2a173493153ca2e5a0808080b8f3f8f1a08023c0d95fc2364e0bf7593f5ff32e1db8ef9f4b41c0bd474eae62d1af896e99808080a0b47b4f0b3e73b5edc8f9a9da1cbcfed562eb06bf54619b6aefeadebf5b3604c280a0da6ec08940a924cb08c947dd56cdb40076b29a6f0ea4dba4e2d02d9a9a72431b80a030cc4138c9e74b6cf79d624b4b5612c0fd888e91f55316cfee7d1694e1a90c0b80a0c5d54b915b56a888eee4e6eeb3141e778f9b674d1d322962eed900f02c29990aa017256b36ef47f907c6b1378a2636942ce894c17075e56fc054d4283f6846659e808080a03340bbaeafcda3a8672eb83099231dbbfab8dae02a1e8ec2f7180538fac207e080b838f7a03868bdfa8727775661e4ccf117824a175a33f8703d728c04488fbfffcafda9f99594e8ddc5c7a2d2f0d7a9798459c0104fdf5e987acaa3e2a02052222313e28459528d920b65115c16c04f3efc82aaedc97be59f3f377c0d3f01b853f851808080a0a87d9bb950836582673aa0eecc0ff64aac607870637a2dd2012b8b1b31981f698080a08da6d5c36a404670c553a2c9052df7cd604f04e3863c4c7b9e0027bfd54206d680808080808080808080b86bf869a033aa5d69545785694b808840be50c182dad2ec3636dfccbe6572fb69828742c0b846f8440101a0663ce0d171e545a26aa67e4ca66f72ba96bb48287dbcc03beea282867f80d44ba01f0e7726926cb43c03a0abf48197dba78522ec8ba1b158e2aa30da7d2a2c6f9eb86bf869a02080c7b7ae81a58eb98d9c78de4a1fd7fd9535fc953ed2be602daaa41767312ab846f8448080a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421a0c5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470b853f851808080a07bb75cabebdcbd1dbb4331054636d0c6d7a2b08483b9e04df057395a7434c9e080808080808080a0e61e567237b49c44d8f906ceea49027260b4010c10a547b38d8b131b9d3b6f848080808080b914c26060604052600436106100fc576000357c0100000000000000000000000000000000000000000000000000000000900463ffffffff16806303aca79214610101578063108552691461016457806340a141ff1461019d57806340c9cdeb146101d65780634110a489146101ff57806345199e0a1461025757806349285b58146102c15780634d238c8e14610316578063752862111461034f578063900eb5a8146103645780639a573786146103c7578063a26a47d21461041c578063ae4b1b5b14610449578063b3f05b971461049e578063b7ab4db5146104cb578063d3e848f114610535578063fa81b2001461058a578063facd743b146105df575b600080fd5b341561010c57600080fd5b6101226004808035906020019091905050610630565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b341561016f57600080fd5b61019b600480803573ffffffffffffffffffffffffffffffffffffffff1690602001909190505061066f565b005b34156101a857600080fd5b6101d4600480803573ffffffffffffffffffffffffffffffffffffffff16906020019091905050610807565b005b34156101e157600080fd5b6101e9610bb7565b6040518082815260200191505060405180910390f35b341561020a57600080fd5b610236600480803573ffffffffffffffffffffffffffffffffffffffff16906020019091905050610bbd565b60405180831515151581526020018281526020019250505060405180910390f35b341561026257600080fd5b61026a610bee565b6040518080602001828103825283818151815260200191508051906020019060200280838360005b838110156102ad578082015181840152602081019050610292565b505050509050019250505060405180910390f35b34156102cc57600080fd5b6102d4610c82565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b341561032157600080fd5b61034d600480803573ffffffffffffffffffffffffffffffffffffffff16906020019091905050610d32565b005b341561035a57600080fd5b610362610fcc565b005b341561036f57600080fd5b61038560048080359060200190919050506110fc565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b34156103d257600080fd5b6103da61113b565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b341561042757600080fd5b61042f6111eb565b604051808215151515815260200191505060405180910390f35b341561045457600080fd5b61045c6111fe565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b34156104a957600080fd5b6104b1611224565b604051808215151515815260200191505060405180910390f35b34156104d657600080fd5b6104de611237565b6040518080602001828103825283818151815260200191508051906020019060200280838360005b83811015610521578082015181840152602081019050610506565b505050509050019250505060405180910390f35b341561054057600080fd5b6105486112cb565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b341561059557600080fd5b61059d6112f1565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b34156105ea57600080fd5b610616600480803573ffffffffffffffffffffffffffffffffffffffff16906020019091905050611317565b604051808215151515815260200191505060405180910390f35b60078181548110151561063f57fe5b90600052602060002090016000915054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b600460029054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff161415156106cb57600080fd5b600460019054906101000a900460ff161515156106e757600080fd5b600073ffffffffffffffffffffffffffffffffffffffff168173ffffffffffffffffffffffffffffffffffffffff161415151561072357600080fd5b80600a60006101000a81548173ffffffffffffffffffffffffffffffffffffffff021916908373ffffffffffffffffffffffffffffffffffffffff1602179055506001600460016101000a81548160ff0219169083151502179055507f600bcf04a13e752d1e3670a5a9f1c21177ca2a93c6f5391d4f1298d098097c22600a60009054906101000a900473ffffffffffffffffffffffffffffffffffffffff16604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390a150565b600080600061081461113b565b73ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff1614151561084d57600080fd5b83600960008273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200190815260200160002060000160009054906101000a900460ff1615156108a957600080fd5b600960008673ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff168152602001908152602001600020600101549350600160078054905003925060078381548110151561090857fe5b906000526020600020900160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1691508160078581548110151561094657fe5b906000526020600020900160006101000a81548173ffffffffffffffffffffffffffffffffffffffff021916908373ffffffffffffffffffffffffffffffffffffffff16021790555083600960008473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff168152602001908152602001600020600101819055506007838154811015156109e557fe5b906000526020600020900160006101000a81549073ffffffffffffffffffffffffffffffffffffffff02191690556000600780549050111515610a2757600080fd5b6007805480919060019003610a3c9190611370565b506000600960008773ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff168152602001908152602001600020600101819055506000600960008773ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200190815260200160002060000160006101000a81548160ff0219169083151502179055506000600460006101000a81548160ff0219169083151502179055506001430340600019167f55252fa6eee4741b4e24a74a70e9c11fd2c2281df8d6ea13126ff845f7825c89600760405180806020018281038252838181548152602001915080548015610ba257602002820191906000526020600020905b8160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff1681526020019060010190808311610b58575b50509250505060405180910390a25050505050565b60085481565b60096020528060005260406000206000915090508060000160009054906101000a900460ff16908060010154905082565b610bf661139c565b6007805480602002602001604051908101604052809291908181526020018280548015610c7857602002820191906000526020600020905b8160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff1681526020019060010190808311610c2e575b5050505050905090565b6000600a60009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff166349285b586000604051602001526040518163ffffffff167c0100000000000000000000000000000000000000000000000000000000028152600401602060405180830381600087803b1515610d1257600080fd5b6102c65a03f11515610d2357600080fd5b50505060405180519050905090565b610d3a61113b565b73ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff16141515610d7357600080fd5b80600960008273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200190815260200160002060000160009054906101000a900460ff16151515610dd057600080fd5b600073ffffffffffffffffffffffffffffffffffffffff168273ffffffffffffffffffffffffffffffffffffffff1614151515610e0c57600080fd5b6040805190810160405280600115158152602001600780549050815250600960008473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200190815260200160002060008201518160000160006101000a81548160ff0219169083151502179055506020820151816001015590505060078054806001018281610ea991906113b0565b9160005260206000209001600084909190916101000a81548173ffffffffffffffffffffffffffffffffffffffff021916908373ffffffffffffffffffffffffffffffffffffffff160217905550506000600460006101000a81548160ff0219169083151502179055506001430340600019167f55252fa6eee4741b4e24a74a70e9c11fd2c2281df8d6ea13126ff845f7825c89600760405180806020018281038252838181548152602001915080548015610fba57602002820191906000526020600020905b8160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff1681526020019060010190808311610f70575b50509250505060405180910390a25050565b600560009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff161480156110365750600460009054906101000a900460ff16155b151561104157600080fd5b6001600460006101000a81548160ff0219169083151502179055506007600690805461106e9291906113dc565b506006805490506008819055507f8564cd629b15f47dc310d45bcbfc9bcf5420b0d51bf0659a16c67f91d27632536110a4611237565b6040518080602001828103825283818151815260200191508051906020019060200280838360005b838110156110e75780820151818401526020810190506110cc565b505050509050019250505060405180910390a1565b60068181548110151561110b57fe5b90600052602060002090016000915054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b6000600a60009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16639a5737866000604051602001526040518163ffffffff167c0100000000000000000000000000000000000000000000000000000000028152600401602060405180830381600087803b15156111cb57600080fd5b6102c65a03f115156111dc57600080fd5b50505060405180519050905090565b600460019054906101000a900460ff1681565b600a60009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b600460009054906101000a900460ff1681565b61123f61139c565b60068054806020026020016040519081016040528092919081815260200182805480156112c157602002820191906000526020600020905b8160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff1681526020019060010190808311611277575b5050505050905090565b600560009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b600460029054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b6000600960008373ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200190815260200160002060000160009054906101000a900460ff169050919050565b81548183558181151161139757818360005260206000209182019101611396919061142e565b5b505050565b602060405190810160405280600081525090565b8154818355818115116113d7578183600052602060002091820191016113d6919061142e565b5b505050565b82805482825590600052602060002090810192821561141d5760005260206000209182015b8281111561141c578254825591600101919060010190611401565b5b50905061142a9190611453565b5090565b61145091905b8082111561144c576000816000905550600101611434565b5090565b90565b61149391905b8082111561148f57600081816101000a81549073ffffffffffffffffffffffffffffffffffffffff021916905550600101611459565b5090565b905600a165627a7a7230582036ea35935c8246b68074adece2eab70c40e69a0193c08a6277ce06e5b25188510029"), nil /* // given a provider and caller, generate proof. this will just be a state proof diff --git a/consensus/clique/clique.go b/consensus/clique/clique.go index 39f96a214..137dac48c 100644 --- a/consensus/clique/clique.go +++ b/consensus/clique/clique.go @@ -363,19 +363,23 @@ func (c *Clique) Prepare(chain consensus.ChainHeaderReader, header *types.Header return nil } -func (c *Clique) Initialize(_ *params.ChainConfig, e consensus.EpochReader, header *types.Header, txs []types.Transaction, uncles []*types.Header, syscall consensus.SystemCall) { +func (c *Clique) Initialize(_ *params.ChainConfig, chain consensus.ChainHeaderReader, e consensus.EpochReader, header *types.Header, txs []types.Transaction, uncles []*types.Header, + syscall consensus.SystemCall, call consensus.Call) { } // 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, r types.Receipts, e consensus.EpochReader, syscall consensus.SystemCall) { +func (c *Clique) Finalize(_ *params.ChainConfig, header *types.Header, state *state.IntraBlockState, txs []types.Transaction, uncles []*types.Header, r types.Receipts, + e consensus.EpochReader, chain consensus.ChainHeaderReader, syscall consensus.SystemCall, call consensus.Call) error { // No block rewards in PoA, so the state remains as is and uncles are dropped header.UncleHash = types.CalcUncleHash(nil) + return 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.Receipts, e consensus.EpochReader, syscall consensus.SystemCall) (*types.Block, error) { +func (c *Clique) FinalizeAndAssemble(chainConfig *params.ChainConfig, header *types.Header, state *state.IntraBlockState, txs []types.Transaction, uncles []*types.Header, receipts types.Receipts, + e consensus.EpochReader, chain consensus.ChainHeaderReader, syscall consensus.SystemCall, call consensus.Call) (*types.Block, error) { // No block rewards in PoA, so the state remains as is and uncles are dropped header.UncleHash = types.CalcUncleHash(nil) @@ -467,7 +471,7 @@ func (c *Clique) Seal(chain consensus.ChainHeaderReader, block *types.Block, res return nil } -func (c *Clique) GenerateSeal(chain consensus.ChainHeaderReader, currnt, parent *types.Header) []rlp.RawValue { +func (c *Clique) GenerateSeal(chain consensus.ChainHeaderReader, currnt, parent *types.Header, call consensus.Call) []rlp.RawValue { return nil } diff --git a/consensus/consensus.go b/consensus/consensus.go index 59f6a81cb..e3c8edefc 100644 --- a/consensus/consensus.go +++ b/consensus/consensus.go @@ -62,6 +62,9 @@ type ChainReader interface { type EpochReader interface { GetEpoch(blockHash common.Hash, blockN uint64) (transitionProof []byte, err error) PutEpoch(blockHash common.Hash, blockN uint64, transitionProof []byte) (err error) + GetPendingEpoch(blockHash common.Hash, blockN uint64) (transitionProof []byte, err error) + PutPendingEpoch(blockHash common.Hash, blockN uint64, transitionProof []byte) (err error) + FindBeforeOrEqualNumber(number uint64) (blockNum uint64, blockHash common.Hash, transitionProof []byte, err error) } type SystemCall func(contract common.Address, data []byte) ([]byte, error) @@ -94,14 +97,14 @@ type Engine interface { Prepare(chain ChainHeaderReader, header *types.Header) error // Initialize runs any pre-transaction state modifications (e.g. epoch start) - Initialize(config *params.ChainConfig, e EpochReader, header *types.Header, txs []types.Transaction, uncles []*types.Header, syscall SystemCall) + Initialize(config *params.ChainConfig, chain ChainHeaderReader, e EpochReader, header *types.Header, txs []types.Transaction, uncles []*types.Header, syscall SystemCall, call Call) // Finalize runs any post-transaction state modifications (e.g. block rewards) // but does not assemble the block. // // 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, r types.Receipts, e EpochReader, syscall SystemCall) + Finalize(config *params.ChainConfig, header *types.Header, state *state.IntraBlockState, txs []types.Transaction, uncles []*types.Header, r types.Receipts, e EpochReader, chain ChainHeaderReader, syscall SystemCall, call Call) error // FinalizeAndAssemble runs any post-transaction state modifications (e.g. block // rewards) and assembles the final block. @@ -109,7 +112,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.Receipts, e EpochReader, syscall SystemCall) (*types.Block, error) + uncles []*types.Header, receipts types.Receipts, e EpochReader, chain ChainHeaderReader, syscall SystemCall, call Call) (*types.Block, error) // Seal generates a new sealing request for the given input block and pushes // the result into the given channel. @@ -125,7 +128,7 @@ type Engine interface { // that a new block should have. CalcDifficulty(chain ChainHeaderReader, time, parentTime uint64, parentDifficulty *big.Int, parentNumber uint64, parentHash, parentUncleHash common.Hash, parentSeal []rlp.RawValue) *big.Int - GenerateSeal(chain ChainHeaderReader, currnt, parent *types.Header) []rlp.RawValue + GenerateSeal(chain ChainHeaderReader, currnt, parent *types.Header, call Call) []rlp.RawValue // VerifyFamily only used by Aura now - later may be merged into VerifyHeaders VerifyFamily(chain ChainHeaderReader, header *types.Header) error diff --git a/consensus/ethash/consensus.go b/consensus/ethash/consensus.go index 9b73ce3af..6b0d543df 100644 --- a/consensus/ethash/consensus.go +++ b/consensus/ethash/consensus.go @@ -328,7 +328,7 @@ func (ethash *Ethash) verifyHeader(chain consensus.ChainHeaderReader, header, pa return nil } -func (ethash *Ethash) GenerateSeal(chain consensus.ChainHeaderReader, currnt, parent *types.Header) []rlp.RawValue { +func (ethash *Ethash) GenerateSeal(chain consensus.ChainHeaderReader, currnt, parent *types.Header, call consensus.Call) []rlp.RawValue { return nil } @@ -596,22 +596,26 @@ func (ethash *Ethash) Prepare(chain consensus.ChainHeaderReader, header *types.H return nil } -func (ethash *Ethash) Initialize(config *params.ChainConfig, e consensus.EpochReader, header *types.Header, txs []types.Transaction, uncles []*types.Header, _ consensus.SystemCall) { +func (ethash *Ethash) Initialize(config *params.ChainConfig, chain consensus.ChainHeaderReader, e consensus.EpochReader, header *types.Header, txs []types.Transaction, uncles []*types.Header, + _ consensus.SystemCall, _ consensus.Call) { } // 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, r types.Receipts, e consensus.EpochReader, _ consensus.SystemCall) { +func (ethash *Ethash) Finalize(config *params.ChainConfig, header *types.Header, state *state.IntraBlockState, txs []types.Transaction, uncles []*types.Header, r types.Receipts, + e consensus.EpochReader, chain consensus.ChainHeaderReader, _ consensus.SystemCall, call consensus.Call) error { // Accumulate any block and uncle rewards and commit the final state root accumulateRewards(config, state, header, uncles) + return nil } // 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, r types.Receipts, e consensus.EpochReader, syscall consensus.SystemCall) (*types.Block, error) { +func (ethash *Ethash) FinalizeAndAssemble(chainConfig *params.ChainConfig, header *types.Header, state *state.IntraBlockState, txs []types.Transaction, uncles []*types.Header, r types.Receipts, + e consensus.EpochReader, chain consensus.ChainHeaderReader, syscall consensus.SystemCall, call consensus.Call) (*types.Block, error) { // Finalize block - ethash.Finalize(chainConfig, header, state, txs, uncles, r, e, syscall) + ethash.Finalize(chainConfig, header, state, txs, uncles, r, e, chain, syscall, call) // Header seems complete, assemble into a block and return return types.NewBlock(header, txs, uncles, r), nil } diff --git a/core/blockchain.go b/core/blockchain.go index 5ac121fb1..6db76ff32 100644 --- a/core/blockchain.go +++ b/core/blockchain.go @@ -96,6 +96,7 @@ func ExecuteBlockEphemerally( stateReader state.StateReader, stateWriter state.WriterWithChangeSets, epochReader consensus.EpochReader, + chainReader consensus.ChainHeaderReader, checkTEVM func(codeHash common.Hash) (bool, error), ) (types.Receipts, error) { defer blockExecutionTimer.UpdateSince(time.Now()) @@ -108,7 +109,7 @@ func ExecuteBlockEphemerally( gp.AddGas(block.GasLimit()) if !vmConfig.ReadOnly { - if err := InitializeBlockExecution(engine, epochReader, block.Header(), block.Transactions(), block.Uncles(), chainConfig, ibs); err != nil { + if err := InitializeBlockExecution(engine, chainReader, epochReader, block.Header(), block.Transactions(), block.Uncles(), chainConfig, ibs); err != nil { return nil, err } } @@ -167,7 +168,7 @@ func ExecuteBlockEphemerally( } } if !vmConfig.ReadOnly { - if err := FinalizeBlockExecution(engine, block.Header(), block.Transactions(), block.Uncles(), stateWriter, chainConfig, ibs, receipts, epochReader); err != nil { + if err := FinalizeBlockExecution(engine, block.Header(), block.Transactions(), block.Uncles(), stateWriter, chainConfig, ibs, receipts, epochReader, chainReader); err != nil { return nil, err } } @@ -264,14 +265,19 @@ func CallContractTx(contract common.Address, data []byte, ibs *state.IntraBlockS func FinalizeBlockExecution(engine consensus.Engine, header *types.Header, txs types.Transactions, uncles []*types.Header, stateWriter state.WriterWithChangeSets, cc *params.ChainConfig, - ibs *state.IntraBlockState, receipts types.Receipts, e consensus.EpochReader, + ibs *state.IntraBlockState, receipts types.Receipts, e consensus.EpochReader, headerReader consensus.ChainHeaderReader, ) error { //ibs.Print(cc.Rules(header.Number.Uint64())) //fmt.Printf("====tx processing end====\n") - engine.Finalize(cc, header, ibs, txs, uncles, receipts, e, func(contract common.Address, data []byte) ([]byte, error) { + if err := engine.Finalize(cc, header, ibs, txs, uncles, receipts, e, headerReader, func(contract common.Address, data []byte) ([]byte, error) { + return SysCallContract(contract, data, *cc, ibs, header, engine) + }, func(contract common.Address, data []byte) ([]byte, error) { return CallContract(contract, data, *cc, ibs, header, engine) - }) + }); err != nil { + return err + } + //fmt.Printf("====finalize start====\n") //ibs.Print(cc.Rules(header.Number.Uint64())) //fmt.Printf("====finalize end====\n") @@ -293,10 +299,12 @@ func FinalizeBlockExecution(engine consensus.Engine, header *types.Header, return nil } -func InitializeBlockExecution(engine consensus.Engine, epochReader consensus.EpochReader, header *types.Header, txs types.Transactions, uncles []*types.Header, cc *params.ChainConfig, ibs *state.IntraBlockState) error { +func InitializeBlockExecution(engine consensus.Engine, chain consensus.ChainHeaderReader, epochReader consensus.EpochReader, header *types.Header, txs types.Transactions, uncles []*types.Header, cc *params.ChainConfig, ibs *state.IntraBlockState) error { // Finalize the block, applying any consensus engine specific extras (e.g. block rewards) - engine.Initialize(cc, epochReader, header, txs, uncles, func(contract common.Address, data []byte) ([]byte, error) { + engine.Initialize(cc, chain, epochReader, header, txs, uncles, func(contract common.Address, data []byte) ([]byte, error) { return SysCallContract(contract, data, *cc, ibs, header, engine) + }, func(contract common.Address, data []byte) ([]byte, error) { + return CallContract(contract, data, *cc, ibs, header, engine) }) //fmt.Printf("====InitializeBlockExecution start %d====\n", header.Number.Uint64()) //ibs.Print(cc.Rules(header.Number.Uint64())) diff --git a/core/chain_makers.go b/core/chain_makers.go index 36d988b04..0a5364752 100644 --- a/core/chain_makers.go +++ b/core/chain_makers.go @@ -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, nil, nil); err != nil { + if _, err := b.engine.FinalizeAndAssemble(config, b.header, ibs, b.txs, b.uncles, b.receipts, nil, nil, nil, nil); err != nil { return nil, nil, fmt.Errorf("call to FinaliseAndAssemble: %w", err) } // Write state changes to db @@ -408,7 +408,7 @@ func makeHeader(chain consensus.ChainReader, parent *types.Block, state *state.I Number: new(big.Int).Add(parent.Number(), common.Big1), Time: time, } - header.Seal = engine.GenerateSeal(chain, header, parent.Header()) + header.Seal = engine.GenerateSeal(chain, header, parent.Header(), nil) if chain.Config().IsLondon(header.Number.Uint64()) { header.BaseFee = misc.CalcBaseFee(chain.Config(), parent.Header()) diff --git a/core/rawdb/accessors_chain.go b/core/rawdb/accessors_chain.go index 79df3f1e8..5a790f968 100644 --- a/core/rawdb/accessors_chain.go +++ b/core/rawdb/accessors_chain.go @@ -877,15 +877,56 @@ func ReadAncestor(db ethdb.KVGetter, hash common.Hash, number, ancestor uint64, } func ReadEpoch(tx ethdb.Tx, blockNum uint64, blockHash common.Hash) (transitionProof []byte, err error) { - k := make([]byte, 8+32) + k := make([]byte, dbutils.NumberLength+common.HashLength) binary.BigEndian.PutUint64(k, blockNum) - copy(k[8:], blockHash[:]) + copy(k[dbutils.NumberLength:], blockHash[:]) return tx.GetOne(dbutils.Epoch, k) } +func FindEpochBeforeOrEqualNumber(tx ethdb.Tx, n uint64) (blockNum uint64, blockHash common.Hash, transitionProof []byte, err error) { + c, err := tx.Cursor(dbutils.Epoch) + if err != nil { + return 0, common.Hash{}, nil, err + } + defer c.Close() + k := make([]byte, dbutils.NumberLength) + binary.BigEndian.PutUint64(k, n) + k, v, err := c.Seek(k) + if err != nil { + return 0, common.Hash{}, nil, err + } + if k != nil { + num := binary.BigEndian.Uint64(k) + if num == n { + return n, common.BytesToHash(k[dbutils.NumberLength:]), v, nil + } + } + k, v, err = c.Prev() + if err != nil { + return 0, common.Hash{}, nil, err + } + if k == nil { + return 0, common.Hash{}, nil, nil + } + return binary.BigEndian.Uint64(k), common.BytesToHash(k[dbutils.NumberLength:]), v, nil +} func WriteEpoch(tx ethdb.RwTx, blockNum uint64, blockHash common.Hash, transitionProof []byte) (err error) { + k := make([]byte, dbutils.NumberLength+common.HashLength) + binary.BigEndian.PutUint64(k, blockNum) + copy(k[dbutils.NumberLength:], blockHash[:]) + return tx.Put(dbutils.Epoch, k, transitionProof) +} + +func ReadPendingEpoch(tx ethdb.Tx, blockNum uint64, blockHash common.Hash) (transitionProof []byte, err error) { k := make([]byte, 8+32) binary.BigEndian.PutUint64(k, blockNum) copy(k[8:], blockHash[:]) - return tx.Put(dbutils.Epoch, k, transitionProof) + return tx.GetOne(dbutils.PendingEpoch, k) +} + +func WritePendingEpoch(tx ethdb.RwTx, blockNum uint64, blockHash common.Hash, transitionProof []byte) (err error) { + k := make([]byte, 8+32) + binary.BigEndian.PutUint64(k, blockNum) + copy(k[8:], blockHash[:]) + return tx.Put(dbutils.PendingEpoch, k, transitionProof) } diff --git a/eth/stagedsync/stage_call_traces.go b/eth/stagedsync/stage_call_traces.go index 8dfd23b4c..c76ad8657 100644 --- a/eth/stagedsync/stage_call_traces.go +++ b/eth/stagedsync/stage_call_traces.go @@ -172,6 +172,7 @@ func promoteCallTraces(logPrefix string, tx ethdb.RwTx, startBlock, endBlock uin if err = flushBitmaps64(collectorTo, tos); err != nil { return err } + // Clean up before loading call traces to reclaim space var prunedMin uint64 = math.MaxUint64 var prunedMax uint64 = 0 diff --git a/eth/stagedsync/stage_execute.go b/eth/stagedsync/stage_execute.go index ab76158bb..f0f7ddb7a 100644 --- a/eth/stagedsync/stage_execute.go +++ b/eth/stagedsync/stage_execute.go @@ -112,7 +112,7 @@ func executeBlock( callTracer := NewCallTracer(checkTEVM) vmConfig.Debug = true vmConfig.Tracer = callTracer - receipts, err := core.ExecuteBlockEphemerally(cfg.chainConfig, &vmConfig, getHeader, cfg.engine, block, stateReader, stateWriter, epochReader{tx: tx}, checkTEVM) + receipts, err := core.ExecuteBlockEphemerally(cfg.chainConfig, &vmConfig, getHeader, cfg.engine, block, stateReader, stateWriter, epochReader{tx: tx}, chainReader{config: cfg.chainConfig, tx: tx}, checkTEVM) if err != nil { return err } diff --git a/eth/stagedsync/stage_headers.go b/eth/stagedsync/stage_headers.go index 02d539a26..ce892290f 100644 --- a/eth/stagedsync/stage_headers.go +++ b/eth/stagedsync/stage_headers.go @@ -364,6 +364,15 @@ func (cr epochReader) GetEpoch(hash common.Hash, number uint64) ([]byte, error) func (cr epochReader) PutEpoch(hash common.Hash, number uint64, proof []byte) error { return rawdb.WriteEpoch(cr.tx, number, hash, proof) } +func (cr epochReader) GetPendingEpoch(hash common.Hash, number uint64) ([]byte, error) { + return rawdb.ReadPendingEpoch(cr.tx, number, hash) +} +func (cr epochReader) PutPendingEpoch(hash common.Hash, number uint64, proof []byte) error { + return rawdb.WritePendingEpoch(cr.tx, number, hash, proof) +} +func (cr epochReader) FindBeforeOrEqualNumber(number uint64) (blockNum uint64, blockHash common.Hash, transitionProof []byte, err error) { + return rawdb.FindEpochBeforeOrEqualNumber(cr.tx, number) +} func HeadersPrune(p *PruneState, tx ethdb.RwTx, cfg HeadersCfg, ctx context.Context) (err error) { useExternalTx := tx != nil diff --git a/eth/stagedsync/stage_mining_exec.go b/eth/stagedsync/stage_mining_exec.go index 319ebe31e..8aaba342c 100644 --- a/eth/stagedsync/stage_mining_exec.go +++ b/eth/stagedsync/stage_mining_exec.go @@ -104,7 +104,7 @@ func SpawnMiningExecStage(s *StageState, tx ethdb.RwTx, cfg MiningExecCfg, quit } } - if err := core.FinalizeBlockExecution(cfg.engine, current.Header, current.Txs, current.Uncles, stateWriter, &cfg.chainConfig, ibs, nil, nil); err != nil { + if err := core.FinalizeBlockExecution(cfg.engine, current.Header, current.Txs, current.Uncles, stateWriter, &cfg.chainConfig, ibs, nil, nil, nil); err != nil { return err } diff --git a/eth/stagedsync/stage_txlookup.go b/eth/stagedsync/stage_txlookup.go index d7f3032c8..7cf3b7f85 100644 --- a/eth/stagedsync/stage_txlookup.go +++ b/eth/stagedsync/stage_txlookup.go @@ -85,8 +85,8 @@ func TxLookupTransform(logPrefix string, tx ethdb.RwTx, startKey, endKey []byte, return fmt.Errorf("%s: tx lookup generation, empty block body %d, hash %x", logPrefix, blocknum, v) } - for _, tx := range body.Transactions { - if err := next(k, tx.Hash().Bytes(), bigNum.SetUint64(blocknum).Bytes()); err != nil { + for _, txn := range body.Transactions { + if err := next(k, txn.Hash().Bytes(), bigNum.SetUint64(blocknum).Bytes()); err != nil { return err } } diff --git a/migrations/receipt_repair.go b/migrations/receipt_repair.go index d667ad449..36ca8a414 100644 --- a/migrations/receipt_repair.go +++ b/migrations/receipt_repair.go @@ -167,7 +167,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, nil, nil); err != nil { + if _, err := engine.FinalizeAndAssemble(chainConfig, header, ibs, block.Transactions(), block.Uncles(), receipts, nil, nil, nil, nil); err != nil { return nil, fmt.Errorf("finalize of block %d failed: %v", block.NumberU64(), err) } diff --git a/turbo/adapter/chain_context.go b/turbo/adapter/chain_context.go index f0291739d..f0fc6b3e2 100644 --- a/turbo/adapter/chain_context.go +++ b/turbo/adapter/chain_context.go @@ -46,14 +46,18 @@ 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) Initialize(chainConfig *params.ChainConfig, e consensus.EpochReader, header *types.Header, txs []types.Transaction, uncles []*types.Header, syscall consensus.SystemCall) { +func (c *powEngine) Initialize(chainConfig *params.ChainConfig, chain consensus.ChainHeaderReader, e consensus.EpochReader, header *types.Header, txs []types.Transaction, uncles []*types.Header, + syscall consensus.SystemCall, call consensus.Call) { panic("must not be called") } -func (c *powEngine) Finalize(chainConfig *params.ChainConfig, header *types.Header, state *state.IntraBlockState, txs []types.Transaction, uncles []*types.Header, r types.Receipts, e consensus.EpochReader, syscall consensus.SystemCall) { +func (c *powEngine) Finalize(chainConfig *params.ChainConfig, header *types.Header, state *state.IntraBlockState, + txs []types.Transaction, uncles []*types.Header, r types.Receipts, + e consensus.EpochReader, h consensus.ChainHeaderReader, syscall consensus.SystemCall, call consensus.Call) error { 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.Receipts, e consensus.EpochReader, syscall consensus.SystemCall) (*types.Block, error) { +func (c *powEngine) FinalizeAndAssemble(chainConfig *params.ChainConfig, header *types.Header, state *state.IntraBlockState, + txs []types.Transaction, uncles []*types.Header, receipts types.Receipts, + e consensus.EpochReader, h consensus.ChainHeaderReader, syscall consensus.SystemCall, call consensus.Call) (*types.Block, error) { panic("must not be called") } @@ -64,7 +68,7 @@ func (c *powEngine) SealHash(header *types.Header) common.Hash { panic("must not be called") } -func (c *powEngine) GenerateSeal(chain consensus.ChainHeaderReader, currnt, parent *types.Header) []rlp.RawValue { +func (c *powEngine) GenerateSeal(chain consensus.ChainHeaderReader, currnt, parent *types.Header, call consensus.Call) []rlp.RawValue { return nil }