all: change chain head markers from block to header (#26777)

This commit is contained in:
Péter Szilágyi 2023-03-02 08:29:15 +02:00 committed by GitHub
parent e1b98f49a5
commit cd31f2dee2
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
41 changed files with 462 additions and 412 deletions

View File

@ -95,7 +95,10 @@ func NewSimulatedBackendWithDatabase(database ethdb.Database, alloc core.Genesis
backend.filterSystem = filters.NewFilterSystem(filterBackend, filters.Config{})
backend.events = filters.NewEventSystem(backend.filterSystem, false)
backend.rollback(blockchain.CurrentBlock())
header := backend.blockchain.CurrentBlock()
block := backend.blockchain.GetBlock(header.Hash(), header.Number.Uint64())
backend.rollback(block)
return backend
}
@ -135,7 +138,10 @@ func (b *SimulatedBackend) Rollback() {
b.mu.Lock()
defer b.mu.Unlock()
b.rollback(b.blockchain.CurrentBlock())
header := b.blockchain.CurrentBlock()
block := b.blockchain.GetBlock(header.Hash(), header.Number.Uint64())
b.rollback(block)
}
func (b *SimulatedBackend) rollback(parent *types.Block) {
@ -174,7 +180,7 @@ func (b *SimulatedBackend) Fork(ctx context.Context, parent common.Hash) error {
// stateByBlockNumber retrieves a state by a given blocknumber.
func (b *SimulatedBackend) stateByBlockNumber(ctx context.Context, blockNumber *big.Int) (*state.StateDB, error) {
if blockNumber == nil || blockNumber.Cmp(b.blockchain.CurrentBlock().Number()) == 0 {
if blockNumber == nil || blockNumber.Cmp(b.blockchain.CurrentBlock().Number) == 0 {
return b.blockchain.State()
}
block, err := b.blockByNumber(ctx, blockNumber)
@ -303,7 +309,7 @@ func (b *SimulatedBackend) BlockByNumber(ctx context.Context, number *big.Int) (
// (associated with its hash) if found without Lock.
func (b *SimulatedBackend) blockByNumber(ctx context.Context, number *big.Int) (*types.Block, error) {
if number == nil || number.Cmp(b.pendingBlock.Number()) == 0 {
return b.blockchain.CurrentBlock(), nil
return b.blockByHash(ctx, b.blockchain.CurrentBlock().Hash())
}
block := b.blockchain.GetBlockByNumber(uint64(number.Int64()))
@ -431,7 +437,7 @@ func (b *SimulatedBackend) CallContract(ctx context.Context, call ethereum.CallM
b.mu.Lock()
defer b.mu.Unlock()
if blockNumber != nil && blockNumber.Cmp(b.blockchain.CurrentBlock().Number()) != 0 {
if blockNumber != nil && blockNumber.Cmp(b.blockchain.CurrentBlock().Number) != 0 {
return nil, errBlockNumberUnsupported
}
stateDB, err := b.blockchain.State()
@ -455,7 +461,7 @@ func (b *SimulatedBackend) PendingCallContract(ctx context.Context, call ethereu
defer b.mu.Unlock()
defer b.pendingState.RevertToSnapshot(b.pendingState.Snapshot())
res, err := b.callContract(ctx, call, b.pendingBlock, b.pendingState)
res, err := b.callContract(ctx, call, b.pendingBlock.Header(), b.pendingState)
if err != nil {
return nil, err
}
@ -549,7 +555,7 @@ func (b *SimulatedBackend) EstimateGas(ctx context.Context, call ethereum.CallMs
call.Gas = gas
snapshot := b.pendingState.Snapshot()
res, err := b.callContract(ctx, call, b.pendingBlock, b.pendingState)
res, err := b.callContract(ctx, call, b.pendingBlock.Header(), b.pendingState)
b.pendingState.RevertToSnapshot(snapshot)
if err != nil {
@ -599,7 +605,7 @@ func (b *SimulatedBackend) EstimateGas(ctx context.Context, call ethereum.CallMs
// callContract implements common code between normal and pending contract calls.
// state is modified during execution, make sure to copy it if necessary.
func (b *SimulatedBackend) callContract(ctx context.Context, call ethereum.CallMsg, block *types.Block, stateDB *state.StateDB) (*core.ExecutionResult, error) {
func (b *SimulatedBackend) callContract(ctx context.Context, call ethereum.CallMsg, header *types.Header, stateDB *state.StateDB) (*core.ExecutionResult, error) {
// Gas prices post 1559 need to be initialized
if call.GasPrice != nil && (call.GasFeeCap != nil || call.GasTipCap != nil) {
return nil, errors.New("both gasPrice and (maxFeePerGas or maxPriorityFeePerGas) specified")
@ -645,7 +651,7 @@ func (b *SimulatedBackend) callContract(ctx context.Context, call ethereum.CallM
msg := callMsg{call}
txContext := core.NewEVMTxContext(msg)
evmContext := core.NewEVMBlockContext(block.Header(), b.blockchain, nil)
evmContext := core.NewEVMBlockContext(header, b.blockchain, nil)
// Create a new environment which holds all relevant information
// about the transaction and calling mechanisms.
vmEnv := vm.NewEVM(evmContext, txContext, stateDB, b.config, vm.Config{NoBaseFee: true})
@ -854,15 +860,9 @@ func (fb *filterBackend) HeaderByNumber(ctx context.Context, number rpc.BlockNum
case rpc.LatestBlockNumber:
return fb.bc.CurrentHeader(), nil
case rpc.FinalizedBlockNumber:
if block := fb.bc.CurrentFinalizedBlock(); block != nil {
return block.Header(), nil
}
return nil, errors.New("finalized block not found")
return fb.bc.CurrentFinalBlock(), nil
case rpc.SafeBlockNumber:
if block := fb.bc.CurrentSafeBlock(); block != nil {
return block.Header(), nil
}
return nil, errors.New("safe block not found")
return fb.bc.CurrentSafeBlock(), nil
default:
return fb.bc.GetHeaderByNumber(uint64(number.Int64())), nil
}

View File

@ -1189,7 +1189,7 @@ func TestFork(t *testing.T) {
sim.Commit()
}
// 3.
if sim.blockchain.CurrentBlock().NumberU64() != uint64(n) {
if sim.blockchain.CurrentBlock().Number.Uint64() != uint64(n) {
t.Error("wrong chain length")
}
// 4.
@ -1199,7 +1199,7 @@ func TestFork(t *testing.T) {
sim.Commit()
}
// 6.
if sim.blockchain.CurrentBlock().NumberU64() != uint64(n+1) {
if sim.blockchain.CurrentBlock().Number.Uint64() != uint64(n+1) {
t.Error("wrong chain length")
}
}
@ -1344,7 +1344,7 @@ func TestCommitReturnValue(t *testing.T) {
sim := simTestBackend(testAddr)
defer sim.Close()
startBlockHeight := sim.blockchain.CurrentBlock().NumberU64()
startBlockHeight := sim.blockchain.CurrentBlock().Number.Uint64()
// Test if Commit returns the correct block hash
h1 := sim.Commit()

View File

@ -349,8 +349,8 @@ func exportChain(ctx *cli.Context) error {
if first < 0 || last < 0 {
utils.Fatalf("Export error: block number must be greater than 0\n")
}
if head := chain.CurrentFastBlock(); uint64(last) > head.NumberU64() {
utils.Fatalf("Export error: block number %d larger than head block %d\n", uint64(last), head.NumberU64())
if head := chain.CurrentSnapBlock(); uint64(last) > head.Number.Uint64() {
utils.Fatalf("Export error: block number %d larger than head block %d\n", uint64(last), head.Number.Uint64())
}
err = utils.ExportAppendChain(chain, fp, uint64(first), uint64(last))
}

View File

@ -222,7 +222,7 @@ func missingBlocks(chain *core.BlockChain, blocks []*types.Block) []*types.Block
head := chain.CurrentBlock()
for i, block := range blocks {
// If we're behind the chain head, only check block, state is available at head
if head.NumberU64() > block.NumberU64() {
if head.Number.Uint64() > block.NumberU64() {
if !chain.HasBlock(block.Hash(), block.NumberU64()) {
return blocks[i:]
}

View File

@ -93,7 +93,7 @@ func TestReimportMirroredState(t *testing.T) {
if _, err := chain.InsertChain(blocks[:2]); err != nil {
t.Fatalf("failed to insert initial blocks: %v", err)
}
if head := chain.CurrentBlock().NumberU64(); head != 2 {
if head := chain.CurrentBlock().Number.Uint64(); head != 2 {
t.Fatalf("chain head mismatch: have %d, want %d", head, 2)
}
@ -106,7 +106,7 @@ func TestReimportMirroredState(t *testing.T) {
if _, err := chain.InsertChain(blocks[2:]); err != nil {
t.Fatalf("failed to insert final block: %v", err)
}
if head := chain.CurrentBlock().NumberU64(); head != 3 {
if head := chain.CurrentBlock().Number.Uint64(); head != 3 {
t.Fatalf("chain head mismatch: have %d, want %d", head, 3)
}
}

View File

@ -199,10 +199,10 @@ type BlockChain struct {
// Readers don't need to take it, they can just read the database.
chainmu *syncx.ClosableMutex
currentBlock atomic.Value // Current head of the block chain
currentFastBlock atomic.Value // Current head of the fast-sync chain (may be above the block chain!)
currentFinalizedBlock atomic.Value // Current finalized head
currentSafeBlock atomic.Value // Current safe head
currentBlock atomic.Pointer[types.Header] // Current head of the chain
currentSnapBlock atomic.Pointer[types.Header] // Current head of snap-sync
currentFinalBlock atomic.Pointer[types.Header] // Latest (consensus) finalized block
currentSafeBlock atomic.Pointer[types.Header] // Latest (consensus) safe block
bodyCache *lru.Cache[common.Hash, *types.Body]
bodyRLPCache *lru.Cache[common.Hash, rlp.RawValue]
@ -289,11 +289,10 @@ func NewBlockChain(db ethdb.Database, cacheConfig *CacheConfig, genesis *Genesis
return nil, ErrNoGenesis
}
var nilBlock *types.Block
bc.currentBlock.Store(nilBlock)
bc.currentFastBlock.Store(nilBlock)
bc.currentFinalizedBlock.Store(nilBlock)
bc.currentSafeBlock.Store(nilBlock)
bc.currentBlock.Store(nil)
bc.currentSnapBlock.Store(nil)
bc.currentFinalBlock.Store(nil)
bc.currentSafeBlock.Store(nil)
// If Geth is initialized with an external ancient store, re-initialize the
// missing chain indexes and chain flags. This procedure can survive crash
@ -307,7 +306,7 @@ func NewBlockChain(db ethdb.Database, cacheConfig *CacheConfig, genesis *Genesis
}
// Make sure the state associated with the block is available
head := bc.CurrentBlock()
if !bc.HasState(head.Root()) {
if !bc.HasState(head.Root) {
// Head state is missing, before the state recovery, find out the
// disk layer point of snapshot(if it's enabled). Make sure the
// rewound point is lower than disk layer.
@ -316,9 +315,9 @@ func NewBlockChain(db ethdb.Database, cacheConfig *CacheConfig, genesis *Genesis
diskRoot = rawdb.ReadSnapshotRoot(bc.db)
}
if diskRoot != (common.Hash{}) {
log.Warn("Head state missing, repairing", "number", head.Number(), "hash", head.Hash(), "snaproot", diskRoot)
log.Warn("Head state missing, repairing", "number", head.Number, "hash", head.Hash(), "snaproot", diskRoot)
snapDisk, err := bc.setHeadBeyondRoot(head.NumberU64(), 0, diskRoot, true)
snapDisk, err := bc.setHeadBeyondRoot(head.Number.Uint64(), 0, diskRoot, true)
if err != nil {
return nil, err
}
@ -327,13 +326,12 @@ func NewBlockChain(db ethdb.Database, cacheConfig *CacheConfig, genesis *Genesis
rawdb.WriteSnapshotRecoveryNumber(bc.db, snapDisk)
}
} else {
log.Warn("Head state missing, repairing", "number", head.Number(), "hash", head.Hash())
if _, err := bc.setHeadBeyondRoot(head.NumberU64(), 0, common.Hash{}, true); err != nil {
log.Warn("Head state missing, repairing", "number", head.Number, "hash", head.Hash())
if _, err := bc.setHeadBeyondRoot(head.Number.Uint64(), 0, common.Hash{}, true); err != nil {
return nil, err
}
}
}
// Ensure that a previous crash in SetHead doesn't leave extra ancients
if frozen, err := bc.db.Ancients(); err == nil && frozen > 0 {
var (
@ -344,18 +342,18 @@ func NewBlockChain(db ethdb.Database, cacheConfig *CacheConfig, genesis *Genesis
// blockchain repair. If the head full block is even lower than the ancient
// chain, truncate the ancient store.
fullBlock := bc.CurrentBlock()
if fullBlock != nil && fullBlock.Hash() != bc.genesisBlock.Hash() && fullBlock.NumberU64() < frozen-1 {
if fullBlock != nil && fullBlock.Hash() != bc.genesisBlock.Hash() && fullBlock.Number.Uint64() < frozen-1 {
needRewind = true
low = fullBlock.NumberU64()
low = fullBlock.Number.Uint64()
}
// In fast sync, it may happen that ancient data has been written to the
// ancient store, but the LastFastBlock has not been updated, truncate the
// extra data here.
fastBlock := bc.CurrentFastBlock()
if fastBlock != nil && fastBlock.NumberU64() < frozen-1 {
snapBlock := bc.CurrentSnapBlock()
if snapBlock != nil && snapBlock.Number.Uint64() < frozen-1 {
needRewind = true
if fastBlock.NumberU64() < low || low == 0 {
low = fastBlock.NumberU64()
if snapBlock.Number.Uint64() < low || low == 0 {
low = snapBlock.Number.Uint64()
}
}
if needRewind {
@ -395,8 +393,8 @@ func NewBlockChain(db ethdb.Database, cacheConfig *CacheConfig, genesis *Genesis
var recover bool
head := bc.CurrentBlock()
if layer := rawdb.ReadSnapshotRecoveryNumber(bc.db); layer != nil && *layer >= head.NumberU64() {
log.Warn("Enabling snapshot recovery", "chainhead", head.NumberU64(), "diskbase", *layer)
if layer := rawdb.ReadSnapshotRecoveryNumber(bc.db); layer != nil && *layer >= head.Number.Uint64() {
log.Warn("Enabling snapshot recovery", "chainhead", head.Number, "diskbase", *layer)
recover = true
}
snapconfig := snapshot.Config{
@ -405,7 +403,7 @@ func NewBlockChain(db ethdb.Database, cacheConfig *CacheConfig, genesis *Genesis
NoBuild: bc.cacheConfig.SnapshotNoBuild,
AsyncBuild: !bc.cacheConfig.SnapshotWait,
}
bc.snaps, _ = snapshot.New(snapconfig, bc.db, bc.triedb, head.Root())
bc.snaps, _ = snapshot.New(snapconfig, bc.db, bc.triedb, head.Root)
}
// Start future block processor.
@ -469,32 +467,32 @@ func (bc *BlockChain) loadLastState() error {
return bc.Reset()
}
// Make sure the entire head block is available
currentBlock := bc.GetBlockByHash(head)
if currentBlock == nil {
headBlock := bc.GetBlockByHash(head)
if headBlock == nil {
// Corrupt or empty database, init from scratch
log.Warn("Head block missing, resetting chain", "hash", head)
return bc.Reset()
}
// Everything seems to be fine, set as the head block
bc.currentBlock.Store(currentBlock)
headBlockGauge.Update(int64(currentBlock.NumberU64()))
bc.currentBlock.Store(headBlock.Header())
headBlockGauge.Update(int64(headBlock.NumberU64()))
// Restore the last known head header
currentHeader := currentBlock.Header()
headHeader := headBlock.Header()
if head := rawdb.ReadHeadHeaderHash(bc.db); head != (common.Hash{}) {
if header := bc.GetHeaderByHash(head); header != nil {
currentHeader = header
headHeader = header
}
}
bc.hc.SetCurrentHeader(currentHeader)
bc.hc.SetCurrentHeader(headHeader)
// Restore the last known head fast block
bc.currentFastBlock.Store(currentBlock)
headFastBlockGauge.Update(int64(currentBlock.NumberU64()))
bc.currentSnapBlock.Store(headBlock.Header())
headFastBlockGauge.Update(int64(headBlock.NumberU64()))
if head := rawdb.ReadHeadFastBlockHash(bc.db); head != (common.Hash{}) {
if block := bc.GetBlockByHash(head); block != nil {
bc.currentFastBlock.Store(block)
bc.currentSnapBlock.Store(block.Header())
headFastBlockGauge.Update(int64(block.NumberU64()))
}
}
@ -504,27 +502,31 @@ func (bc *BlockChain) loadLastState() error {
// known finalized block on startup
if head := rawdb.ReadFinalizedBlockHash(bc.db); head != (common.Hash{}) {
if block := bc.GetBlockByHash(head); block != nil {
bc.currentFinalizedBlock.Store(block)
bc.currentFinalBlock.Store(block.Header())
headFinalizedBlockGauge.Update(int64(block.NumberU64()))
bc.currentSafeBlock.Store(block)
bc.currentSafeBlock.Store(block.Header())
headSafeBlockGauge.Update(int64(block.NumberU64()))
}
}
// Issue a status log for the user
currentFastBlock := bc.CurrentFastBlock()
currentFinalizedBlock := bc.CurrentFinalizedBlock()
var (
currentSnapBlock = bc.CurrentSnapBlock()
currentFinalBlock = bc.CurrentFinalBlock()
headerTd := bc.GetTd(currentHeader.Hash(), currentHeader.Number.Uint64())
blockTd := bc.GetTd(currentBlock.Hash(), currentBlock.NumberU64())
fastTd := bc.GetTd(currentFastBlock.Hash(), currentFastBlock.NumberU64())
log.Info("Loaded most recent local header", "number", currentHeader.Number, "hash", currentHeader.Hash(), "td", headerTd, "age", common.PrettyAge(time.Unix(int64(currentHeader.Time), 0)))
log.Info("Loaded most recent local full block", "number", currentBlock.Number(), "hash", currentBlock.Hash(), "td", blockTd, "age", common.PrettyAge(time.Unix(int64(currentBlock.Time()), 0)))
log.Info("Loaded most recent local fast block", "number", currentFastBlock.Number(), "hash", currentFastBlock.Hash(), "td", fastTd, "age", common.PrettyAge(time.Unix(int64(currentFastBlock.Time()), 0)))
if currentFinalizedBlock != nil {
finalTd := bc.GetTd(currentFinalizedBlock.Hash(), currentFinalizedBlock.NumberU64())
log.Info("Loaded most recent local finalized block", "number", currentFinalizedBlock.Number(), "hash", currentFinalizedBlock.Hash(), "td", finalTd, "age", common.PrettyAge(time.Unix(int64(currentFinalizedBlock.Time()), 0)))
headerTd = bc.GetTd(headHeader.Hash(), headHeader.Number.Uint64())
blockTd = bc.GetTd(headBlock.Hash(), headBlock.NumberU64())
)
if headHeader.Hash() != headBlock.Hash() {
log.Info("Loaded most recent local header", "number", headHeader.Number, "hash", headHeader.Hash(), "td", headerTd, "age", common.PrettyAge(time.Unix(int64(headHeader.Time), 0)))
}
log.Info("Loaded most recent local block", "number", headBlock.Number(), "hash", headBlock.Hash(), "td", blockTd, "age", common.PrettyAge(time.Unix(int64(headBlock.Time()), 0)))
if headBlock.Hash() != currentSnapBlock.Hash() {
fastTd := bc.GetTd(currentSnapBlock.Hash(), currentSnapBlock.Number.Uint64())
log.Info("Loaded most recent local snap block", "number", currentSnapBlock.Number, "hash", currentSnapBlock.Hash(), "td", fastTd, "age", common.PrettyAge(time.Unix(int64(currentSnapBlock.Time), 0)))
}
if currentFinalBlock != nil {
finalTd := bc.GetTd(currentFinalBlock.Hash(), currentFinalBlock.Number.Uint64())
log.Info("Loaded most recent local finalized block", "number", currentFinalBlock.Number, "hash", currentFinalBlock.Hash(), "td", finalTd, "age", common.PrettyAge(time.Unix(int64(currentFinalBlock.Time), 0)))
}
if pivot := rawdb.ReadLastPivotNumber(bc.db); pivot != nil {
log.Info("Loaded last fast-sync pivot marker", "number", *pivot)
@ -540,7 +542,16 @@ func (bc *BlockChain) SetHead(head uint64) error {
return err
}
// Send chain head event to update the transaction pool
bc.chainHeadFeed.Send(ChainHeadEvent{Block: bc.CurrentBlock()})
header := bc.CurrentBlock()
block := bc.GetBlock(header.Hash(), header.Number.Uint64())
if block == nil {
// This should never happen. In practice, previsouly currentBlock
// contained the entire block whereas now only a "marker", so there
// is an ever so slight chance for a race we should handle.
log.Error("Current block not found in database", "block", header.Number, "hash", header.Hash())
return fmt.Errorf("current block missing: #%d [%x..]", header.Number, header.Hash().Bytes()[:4])
}
bc.chainHeadFeed.Send(ChainHeadEvent{Block: block})
return nil
}
@ -553,16 +564,25 @@ func (bc *BlockChain) SetHeadWithTimestamp(timestamp uint64) error {
return err
}
// Send chain head event to update the transaction pool
bc.chainHeadFeed.Send(ChainHeadEvent{Block: bc.CurrentBlock()})
header := bc.CurrentBlock()
block := bc.GetBlock(header.Hash(), header.Number.Uint64())
if block == nil {
// This should never happen. In practice, previsouly currentBlock
// contained the entire block whereas now only a "marker", so there
// is an ever so slight chance for a race we should handle.
log.Error("Current block not found in database", "block", header.Number, "hash", header.Hash())
return fmt.Errorf("current block missing: #%d [%x..]", header.Number, header.Hash().Bytes()[:4])
}
bc.chainHeadFeed.Send(ChainHeadEvent{Block: block})
return nil
}
// SetFinalized sets the finalized block.
func (bc *BlockChain) SetFinalized(block *types.Block) {
bc.currentFinalizedBlock.Store(block)
if block != nil {
rawdb.WriteFinalizedBlockHash(bc.db, block.Hash())
headFinalizedBlockGauge.Update(int64(block.NumberU64()))
func (bc *BlockChain) SetFinalized(header *types.Header) {
bc.currentFinalBlock.Store(header)
if header != nil {
rawdb.WriteFinalizedBlockHash(bc.db, header.Hash())
headFinalizedBlockGauge.Update(int64(header.Number.Uint64()))
} else {
rawdb.WriteFinalizedBlockHash(bc.db, common.Hash{})
headFinalizedBlockGauge.Update(0)
@ -570,10 +590,10 @@ func (bc *BlockChain) SetFinalized(block *types.Block) {
}
// SetSafe sets the safe block.
func (bc *BlockChain) SetSafe(block *types.Block) {
bc.currentSafeBlock.Store(block)
if block != nil {
headSafeBlockGauge.Update(int64(block.NumberU64()))
func (bc *BlockChain) SetSafe(header *types.Header) {
bc.currentSafeBlock.Store(header)
if header != nil {
headSafeBlockGauge.Update(int64(header.Number.Uint64()))
} else {
headSafeBlockGauge.Update(0)
}
@ -609,7 +629,7 @@ func (bc *BlockChain) setHeadBeyondRoot(head uint64, time uint64, root common.Ha
// Rewind the blockchain, ensuring we don't end up with a stateless head
// block. Note, depth equality is permitted to allow using SetHead as a
// chain reparation mechanism without deleting any data!
if currentBlock := bc.CurrentBlock(); currentBlock != nil && header.Number.Uint64() <= currentBlock.NumberU64() {
if currentBlock := bc.CurrentBlock(); currentBlock != nil && header.Number.Uint64() <= currentBlock.Number.Uint64() {
newHeadBlock := bc.GetBlock(header.Hash(), header.Number.Uint64())
if newHeadBlock == nil {
log.Error("Gap in the chain, rewinding to genesis", "number", header.Number, "hash", header.Hash())
@ -667,27 +687,27 @@ func (bc *BlockChain) setHeadBeyondRoot(head uint64, time uint64, root common.Ha
// In theory we should update all in-memory markers in the
// last step, however the direction of SetHead is from high
// to low, so it's safe to update in-memory markers directly.
bc.currentBlock.Store(newHeadBlock)
bc.currentBlock.Store(newHeadBlock.Header())
headBlockGauge.Update(int64(newHeadBlock.NumberU64()))
}
// Rewind the fast block in a simpleton way to the target head
if currentFastBlock := bc.CurrentFastBlock(); currentFastBlock != nil && header.Number.Uint64() < currentFastBlock.NumberU64() {
newHeadFastBlock := bc.GetBlock(header.Hash(), header.Number.Uint64())
if currentSnapBlock := bc.CurrentSnapBlock(); currentSnapBlock != nil && header.Number.Uint64() < currentSnapBlock.Number.Uint64() {
newHeadSnapBlock := bc.GetBlock(header.Hash(), header.Number.Uint64())
// If either blocks reached nil, reset to the genesis state
if newHeadFastBlock == nil {
newHeadFastBlock = bc.genesisBlock
if newHeadSnapBlock == nil {
newHeadSnapBlock = bc.genesisBlock
}
rawdb.WriteHeadFastBlockHash(db, newHeadFastBlock.Hash())
rawdb.WriteHeadFastBlockHash(db, newHeadSnapBlock.Hash())
// Degrade the chain markers if they are explicitly reverted.
// In theory we should update all in-memory markers in the
// last step, however the direction of SetHead is from high
// to low, so it's safe the update in-memory markers directly.
bc.currentFastBlock.Store(newHeadFastBlock)
headFastBlockGauge.Update(int64(newHeadFastBlock.NumberU64()))
bc.currentSnapBlock.Store(newHeadSnapBlock.Header())
headFastBlockGauge.Update(int64(newHeadSnapBlock.NumberU64()))
}
var (
headHeader = bc.CurrentBlock().Header()
headHeader = bc.CurrentBlock()
headNumber = headHeader.Number.Uint64()
)
// If setHead underflown the freezer threshold and the block processing
@ -723,7 +743,7 @@ func (bc *BlockChain) setHeadBeyondRoot(head uint64, time uint64, root common.Ha
// If SetHead was only called as a chain reparation method, try to skip
// touching the header chain altogether, unless the freezer is broken
if repair {
if target, force := updateFn(bc.db, bc.CurrentBlock().Header()); force {
if target, force := updateFn(bc.db, bc.CurrentBlock()); force {
bc.hc.SetHead(target.Number.Uint64(), updateFn, delFn)
}
} else {
@ -746,15 +766,14 @@ func (bc *BlockChain) setHeadBeyondRoot(head uint64, time uint64, root common.Ha
bc.futureBlocks.Purge()
// Clear safe block, finalized block if needed
if safe := bc.CurrentSafeBlock(); safe != nil && head < safe.NumberU64() {
if safe := bc.CurrentSafeBlock(); safe != nil && head < safe.Number.Uint64() {
log.Warn("SetHead invalidated safe block")
bc.SetSafe(nil)
}
if finalized := bc.CurrentFinalizedBlock(); finalized != nil && head < finalized.NumberU64() {
if finalized := bc.CurrentFinalBlock(); finalized != nil && head < finalized.Number.Uint64() {
log.Error("SetHead invalidated finalized block")
bc.SetFinalized(nil)
}
return rootNumber, bc.loadLastState()
}
@ -774,7 +793,7 @@ func (bc *BlockChain) SnapSyncCommitHead(hash common.Hash) error {
if !bc.chainmu.TryLock() {
return errChainStopped
}
bc.currentBlock.Store(block)
bc.currentBlock.Store(block.Header())
headBlockGauge.Update(int64(block.NumberU64()))
bc.chainmu.Unlock()
@ -815,18 +834,18 @@ func (bc *BlockChain) ResetWithGenesisBlock(genesis *types.Block) error {
// Last update all in-memory chain markers
bc.genesisBlock = genesis
bc.currentBlock.Store(bc.genesisBlock)
bc.currentBlock.Store(bc.genesisBlock.Header())
headBlockGauge.Update(int64(bc.genesisBlock.NumberU64()))
bc.hc.SetGenesis(bc.genesisBlock.Header())
bc.hc.SetCurrentHeader(bc.genesisBlock.Header())
bc.currentFastBlock.Store(bc.genesisBlock)
bc.currentSnapBlock.Store(bc.genesisBlock.Header())
headFastBlockGauge.Update(int64(bc.genesisBlock.NumberU64()))
return nil
}
// Export writes the active chain to the given writer.
func (bc *BlockChain) Export(w io.Writer) error {
return bc.ExportN(w, uint64(0), bc.CurrentBlock().NumberU64())
return bc.ExportN(w, uint64(0), bc.CurrentBlock().Number.Uint64())
}
// ExportN writes a subset of the active chain to the given writer.
@ -883,10 +902,10 @@ func (bc *BlockChain) writeHeadBlock(block *types.Block) {
// Update all in-memory chain markers in the last step
bc.hc.SetCurrentHeader(block.Header())
bc.currentFastBlock.Store(block)
bc.currentSnapBlock.Store(block.Header())
headFastBlockGauge.Update(int64(block.NumberU64()))
bc.currentBlock.Store(block)
bc.currentBlock.Store(block.Header())
headBlockGauge.Update(int64(block.NumberU64()))
}
@ -927,7 +946,7 @@ func (bc *BlockChain) Stop() {
var snapBase common.Hash
if bc.snaps != nil {
var err error
if snapBase, err = bc.snaps.Journal(bc.CurrentBlock().Root()); err != nil {
if snapBase, err = bc.snaps.Journal(bc.CurrentBlock().Root); err != nil {
log.Error("Failed to journal state snapshot", "err", err)
}
}
@ -941,7 +960,7 @@ func (bc *BlockChain) Stop() {
triedb := bc.triedb
for _, offset := range []uint64{0, 1, TriesInMemory - 1} {
if number := bc.CurrentBlock().NumberU64(); number > offset {
if number := bc.CurrentBlock().Number.Uint64(); number > offset {
recent := bc.GetBlockByNumber(number - offset)
log.Info("Writing cached state to disk", "block", recent.Number(), "hash", recent.Hash(), "root", recent.Root())
@ -1059,7 +1078,7 @@ func (bc *BlockChain) InsertReceiptChain(blockChain types.Blocks, receiptChain [
// Rewind may have occurred, skip in that case.
if bc.CurrentHeader().Number.Cmp(head.Number()) >= 0 {
reorg, err := bc.forker.ReorgNeeded(bc.CurrentFastBlock().Header(), head.Header())
reorg, err := bc.forker.ReorgNeeded(bc.CurrentSnapBlock(), head.Header())
if err != nil {
log.Warn("Reorg failed", "err", err)
return false
@ -1067,13 +1086,12 @@ func (bc *BlockChain) InsertReceiptChain(blockChain types.Blocks, receiptChain [
return false
}
rawdb.WriteHeadFastBlockHash(bc.db, head.Hash())
bc.currentFastBlock.Store(head)
bc.currentSnapBlock.Store(head.Header())
headFastBlockGauge.Update(int64(head.NumberU64()))
return true
}
return false
}
// writeAncient writes blockchain and corresponding receipt chain into ancient store.
//
// this function only accepts canonical chain data. All side chain will be reverted
@ -1135,8 +1153,8 @@ func (bc *BlockChain) InsertReceiptChain(blockChain types.Blocks, receiptChain [
if batch.ValueSize() > ethdb.IdealBatchSize || i == len(blockChain)-1 {
size += int64(batch.ValueSize())
if err = batch.Write(); err != nil {
fastBlock := bc.CurrentFastBlock().NumberU64()
if err := bc.db.TruncateHead(fastBlock + 1); err != nil {
snapBlock := bc.CurrentSnapBlock().Number.Uint64()
if err := bc.db.TruncateHead(snapBlock + 1); err != nil {
log.Error("Can't truncate ancient store after failed insert", "err", err)
}
return 0, err
@ -1150,11 +1168,11 @@ func (bc *BlockChain) InsertReceiptChain(blockChain types.Blocks, receiptChain [
return 0, err
}
// Update the current fast block because all block data is now present in DB.
previousFastBlock := bc.CurrentFastBlock().NumberU64()
previousSnapBlock := bc.CurrentSnapBlock().Number.Uint64()
if !updateHead(blockChain[len(blockChain)-1]) {
// We end up here if the header chain has reorg'ed, and the blocks/receipts
// don't match the canonical chain.
if err := bc.db.TruncateHead(previousFastBlock + 1); err != nil {
if err := bc.db.TruncateHead(previousSnapBlock + 1); err != nil {
log.Error("Can't truncate ancient store after failed insert", "err", err)
}
return 0, errSideChainReceipts
@ -1414,7 +1432,7 @@ func (bc *BlockChain) writeBlockAndSetHead(block *types.Block, receipts []*types
return NonStatTy, err
}
currentBlock := bc.CurrentBlock()
reorg, err := bc.forker.ReorgNeeded(currentBlock.Header(), block.Header())
reorg, err := bc.forker.ReorgNeeded(currentBlock, block.Header())
if err != nil {
return NonStatTy, err
}
@ -1562,7 +1580,7 @@ func (bc *BlockChain) insertChain(chain types.Blocks, verifySeals, setHead bool)
current = bc.CurrentBlock()
)
for block != nil && bc.skipBlock(err, it) {
reorg, err = bc.forker.ReorgNeeded(current.Header(), block.Header())
reorg, err = bc.forker.ReorgNeeded(current, block.Header())
if err != nil {
return it.index, err
}
@ -1572,7 +1590,7 @@ func (bc *BlockChain) insertChain(chain types.Blocks, verifySeals, setHead bool)
// In eth2 the forker always returns true for reorg decision (blindly trusting
// the external consensus engine), but in order to prevent the unnecessary
// reorgs when importing known blocks, the special case is handled here.
if block.NumberU64() > current.NumberU64() || bc.GetCanonicalHash(block.NumberU64()) != block.Hash() {
if block.NumberU64() > current.Number.Uint64() || bc.GetCanonicalHash(block.NumberU64()) != block.Hash() {
break
}
}
@ -1872,7 +1890,7 @@ func (bc *BlockChain) insertSideChain(block *types.Block, it *insertIterator) (i
err := consensus.ErrPrunedAncestor
for ; block != nil && errors.Is(err, consensus.ErrPrunedAncestor); block, err = it.next() {
// Check the canonical state root for that number
if number := block.NumberU64(); current.NumberU64() >= number {
if number := block.NumberU64(); current.Number.Uint64() >= number {
canonical := bc.GetBlockByNumber(number)
if canonical != nil && canonical.Hash() == block.Hash() {
// Not a sidechain block, this is a re-import of a canon block which has it's state pruned
@ -1922,12 +1940,12 @@ func (bc *BlockChain) insertSideChain(block *types.Block, it *insertIterator) (i
//
// If the externTd was larger than our local TD, we now need to reimport the previous
// blocks to regenerate the required state
reorg, err := bc.forker.ReorgNeeded(current.Header(), lastBlock.Header())
reorg, err := bc.forker.ReorgNeeded(current, lastBlock.Header())
if err != nil {
return it.index, err
}
if !reorg {
localTd := bc.GetTd(current.Hash(), current.NumberU64())
localTd := bc.GetTd(current.Hash(), current.Number.Uint64())
log.Info("Sidechain written to disk", "start", it.first().NumberU64(), "end", it.previous().Number, "sidetd", externTd, "localtd", localTd)
return it.index, err
}
@ -2051,7 +2069,7 @@ func (bc *BlockChain) collectLogs(b *types.Block, removed bool) []*types.Log {
// potential missing transactions and post an event about them.
// Note the new head block won't be processed here, callers need to handle it
// externally.
func (bc *BlockChain) reorg(oldBlock, newBlock *types.Block) error {
func (bc *BlockChain) reorg(oldHead *types.Header, newHead *types.Block) error {
var (
newChain types.Blocks
oldChain types.Blocks
@ -2060,6 +2078,12 @@ func (bc *BlockChain) reorg(oldBlock, newBlock *types.Block) error {
deletedTxs []common.Hash
addedTxs []common.Hash
)
oldBlock := bc.GetBlock(oldHead.Hash(), oldHead.Number.Uint64())
if oldBlock == nil {
return errors.New("current head block missing")
}
newBlock := newHead
// Reduce the longer chain to the same number as the shorter one
if oldBlock.NumberU64() > newBlock.NumberU64() {
// Old chain is longer, gather all transactions and logs as deleted ones
@ -2076,10 +2100,10 @@ func (bc *BlockChain) reorg(oldBlock, newBlock *types.Block) error {
}
}
if oldBlock == nil {
return fmt.Errorf("invalid old chain")
return errors.New("invalid old chain")
}
if newBlock == nil {
return fmt.Errorf("invalid new chain")
return errors.New("invalid new chain")
}
// Both sides of the reorg are at the same number, reduce both until the common
// ancestor is found

View File

@ -40,26 +40,26 @@ func (bc *BlockChain) CurrentHeader() *types.Header {
// CurrentBlock retrieves the current head block of the canonical chain. The
// block is retrieved from the blockchain's internal cache.
func (bc *BlockChain) CurrentBlock() *types.Block {
return bc.currentBlock.Load().(*types.Block)
func (bc *BlockChain) CurrentBlock() *types.Header {
return bc.currentBlock.Load()
}
// CurrentFastBlock retrieves the current fast-sync head block of the canonical
// CurrentSnapBlock retrieves the current snap-sync head block of the canonical
// chain. The block is retrieved from the blockchain's internal cache.
func (bc *BlockChain) CurrentFastBlock() *types.Block {
return bc.currentFastBlock.Load().(*types.Block)
func (bc *BlockChain) CurrentSnapBlock() *types.Header {
return bc.currentSnapBlock.Load()
}
// CurrentFinalizedBlock retrieves the current finalized block of the canonical
// CurrentFinalBlock retrieves the current finalized block of the canonical
// chain. The block is retrieved from the blockchain's internal cache.
func (bc *BlockChain) CurrentFinalizedBlock() *types.Block {
return bc.currentFinalizedBlock.Load().(*types.Block)
func (bc *BlockChain) CurrentFinalBlock() *types.Header {
return bc.currentFinalBlock.Load()
}
// CurrentSafeBlock retrieves the current safe block of the canonical
// chain. The block is retrieved from the blockchain's internal cache.
func (bc *BlockChain) CurrentSafeBlock() *types.Block {
return bc.currentSafeBlock.Load().(*types.Block)
func (bc *BlockChain) CurrentSafeBlock() *types.Header {
return bc.currentSafeBlock.Load()
}
// HasHeader checks if a block header is present in the database or not, caching
@ -315,7 +315,7 @@ func (bc *BlockChain) ContractCodeWithPrefix(hash common.Hash) ([]byte, error) {
// State returns a new mutable state based on the current HEAD block.
func (bc *BlockChain) State() (*state.StateDB, error) {
return bc.StateAt(bc.CurrentBlock().Root())
return bc.StateAt(bc.CurrentBlock().Root)
}
// StateAt returns a new mutable state based on a particular point in time.
@ -351,7 +351,7 @@ func (bc *BlockChain) StateCache() state.Database {
// GasLimit returns the gas limit of the current HEAD block.
func (bc *BlockChain) GasLimit() uint64 {
return bc.CurrentBlock().GasLimit()
return bc.CurrentBlock().GasLimit
}
// Genesis retrieves the chain's genesis block.

View File

@ -1857,11 +1857,11 @@ func testRepair(t *testing.T, tt *rewindTest, snapshots bool) {
if head := newChain.CurrentHeader(); head.Number.Uint64() != tt.expHeadHeader {
t.Errorf("Head header mismatch: have %d, want %d", head.Number, tt.expHeadHeader)
}
if head := newChain.CurrentFastBlock(); head.NumberU64() != tt.expHeadFastBlock {
t.Errorf("Head fast block mismatch: have %d, want %d", head.NumberU64(), tt.expHeadFastBlock)
if head := newChain.CurrentSnapBlock(); head.Number.Uint64() != tt.expHeadFastBlock {
t.Errorf("Head fast block mismatch: have %d, want %d", head.Number, tt.expHeadFastBlock)
}
if head := newChain.CurrentBlock(); head.NumberU64() != tt.expHeadBlock {
t.Errorf("Head block mismatch: have %d, want %d", head.NumberU64(), tt.expHeadBlock)
if head := newChain.CurrentBlock(); head.Number.Uint64() != tt.expHeadBlock {
t.Errorf("Head block mismatch: have %d, want %d", head.Number, tt.expHeadBlock)
}
if frozen, err := db.(freezer).Ancients(); err != nil {
t.Errorf("Failed to retrieve ancient count: %v\n", err)
@ -1973,11 +1973,11 @@ func TestIssue23496(t *testing.T) {
if head := chain.CurrentHeader(); head.Number.Uint64() != uint64(4) {
t.Errorf("Head header mismatch: have %d, want %d", head.Number, 4)
}
if head := chain.CurrentFastBlock(); head.NumberU64() != uint64(4) {
t.Errorf("Head fast block mismatch: have %d, want %d", head.NumberU64(), uint64(4))
if head := chain.CurrentSnapBlock(); head.Number.Uint64() != uint64(4) {
t.Errorf("Head fast block mismatch: have %d, want %d", head.Number, uint64(4))
}
if head := chain.CurrentBlock(); head.NumberU64() != uint64(1) {
t.Errorf("Head block mismatch: have %d, want %d", head.NumberU64(), uint64(1))
if head := chain.CurrentBlock(); head.Number.Uint64() != uint64(1) {
t.Errorf("Head block mismatch: have %d, want %d", head.Number, uint64(1))
}
// Reinsert B2-B4
@ -1987,11 +1987,11 @@ func TestIssue23496(t *testing.T) {
if head := chain.CurrentHeader(); head.Number.Uint64() != uint64(4) {
t.Errorf("Head header mismatch: have %d, want %d", head.Number, 4)
}
if head := chain.CurrentFastBlock(); head.NumberU64() != uint64(4) {
t.Errorf("Head fast block mismatch: have %d, want %d", head.NumberU64(), uint64(4))
if head := chain.CurrentSnapBlock(); head.Number.Uint64() != uint64(4) {
t.Errorf("Head fast block mismatch: have %d, want %d", head.Number, uint64(4))
}
if head := chain.CurrentBlock(); head.NumberU64() != uint64(4) {
t.Errorf("Head block mismatch: have %d, want %d", head.NumberU64(), uint64(4))
if head := chain.CurrentBlock(); head.Number.Uint64() != uint64(4) {
t.Errorf("Head block mismatch: have %d, want %d", head.Number, uint64(4))
}
if layer := chain.Snapshots().Snapshot(blocks[2].Root()); layer == nil {
t.Error("Failed to regenerate the snapshot of known state")

View File

@ -2047,11 +2047,11 @@ func testSetHead(t *testing.T, tt *rewindTest, snapshots bool) {
if head := chain.CurrentHeader(); head.Number.Uint64() != tt.expHeadHeader {
t.Errorf("Head header mismatch: have %d, want %d", head.Number, tt.expHeadHeader)
}
if head := chain.CurrentFastBlock(); head.NumberU64() != tt.expHeadFastBlock {
t.Errorf("Head fast block mismatch: have %d, want %d", head.NumberU64(), tt.expHeadFastBlock)
if head := chain.CurrentSnapBlock(); head.Number.Uint64() != tt.expHeadFastBlock {
t.Errorf("Head fast block mismatch: have %d, want %d", head.Number, tt.expHeadFastBlock)
}
if head := chain.CurrentBlock(); head.NumberU64() != tt.expHeadBlock {
t.Errorf("Head block mismatch: have %d, want %d", head.NumberU64(), tt.expHeadBlock)
if head := chain.CurrentBlock(); head.Number.Uint64() != tt.expHeadBlock {
t.Errorf("Head block mismatch: have %d, want %d", head.Number, tt.expHeadBlock)
}
if frozen, err := db.(freezer).Ancients(); err != nil {
t.Errorf("Failed to retrieve ancient count: %v\n", err)

View File

@ -136,11 +136,11 @@ func (basic *snapshotTestBasic) verify(t *testing.T, chain *BlockChain, blocks [
if head := chain.CurrentHeader(); head.Number.Uint64() != basic.expHeadHeader {
t.Errorf("Head header mismatch: have %d, want %d", head.Number, basic.expHeadHeader)
}
if head := chain.CurrentFastBlock(); head.NumberU64() != basic.expHeadFastBlock {
t.Errorf("Head fast block mismatch: have %d, want %d", head.NumberU64(), basic.expHeadFastBlock)
if head := chain.CurrentSnapBlock(); head.Number.Uint64() != basic.expHeadFastBlock {
t.Errorf("Head fast block mismatch: have %d, want %d", head.Number, basic.expHeadFastBlock)
}
if head := chain.CurrentBlock(); head.NumberU64() != basic.expHeadBlock {
t.Errorf("Head block mismatch: have %d, want %d", head.NumberU64(), basic.expHeadBlock)
if head := chain.CurrentBlock(); head.Number.Uint64() != basic.expHeadBlock {
t.Errorf("Head block mismatch: have %d, want %d", head.Number, basic.expHeadBlock)
}
// Check the disk layer, ensure they are matched

View File

@ -109,7 +109,7 @@ func testFork(t *testing.T, blockchain *BlockChain, i, n int, full bool, compara
headerChainB []*types.Header
)
if full {
blockChainB = makeBlockChain(blockchain2.chainConfig, blockchain2.CurrentBlock(), n, ethash.NewFaker(), genDb, forkSeed)
blockChainB = makeBlockChain(blockchain2.chainConfig, blockchain2.GetBlockByHash(blockchain2.CurrentBlock().Hash()), n, ethash.NewFaker(), genDb, forkSeed)
if _, err := blockchain2.InsertChain(blockChainB); err != nil {
t.Fatalf("failed to insert forking chain: %v", err)
}
@ -124,7 +124,7 @@ func testFork(t *testing.T, blockchain *BlockChain, i, n int, full bool, compara
if full {
cur := blockchain.CurrentBlock()
tdPre = blockchain.GetTd(cur.Hash(), cur.NumberU64())
tdPre = blockchain.GetTd(cur.Hash(), cur.Number.Uint64())
if err := testBlockChainImport(blockChainB, blockchain); err != nil {
t.Fatalf("failed to import forked block chain: %v", err)
}
@ -206,7 +206,7 @@ func TestLastBlock(t *testing.T) {
}
defer blockchain.Stop()
blocks := makeBlockChain(blockchain.chainConfig, blockchain.CurrentBlock(), 1, ethash.NewFullFaker(), genDb, 0)
blocks := makeBlockChain(blockchain.chainConfig, blockchain.GetBlockByHash(blockchain.CurrentBlock().Hash()), 1, ethash.NewFullFaker(), genDb, 0)
if _, err := blockchain.InsertChain(blocks); err != nil {
t.Fatalf("Failed to insert block: %v", err)
}
@ -240,11 +240,11 @@ func testInsertAfterMerge(t *testing.T, blockchain *BlockChain, i, n int, full b
// Extend the newly created chain
if full {
blockChainB := makeBlockChain(blockchain2.chainConfig, blockchain2.CurrentBlock(), n, ethash.NewFaker(), genDb, forkSeed)
blockChainB := makeBlockChain(blockchain2.chainConfig, blockchain2.GetBlockByHash(blockchain2.CurrentBlock().Hash()), n, ethash.NewFaker(), genDb, forkSeed)
if _, err := blockchain2.InsertChain(blockChainB); err != nil {
t.Fatalf("failed to insert forking chain: %v", err)
}
if blockchain2.CurrentBlock().NumberU64() != blockChainB[len(blockChainB)-1].NumberU64() {
if blockchain2.CurrentBlock().Number.Uint64() != blockChainB[len(blockChainB)-1].NumberU64() {
t.Fatalf("failed to reorg to the given chain")
}
if blockchain2.CurrentBlock().Hash() != blockChainB[len(blockChainB)-1].Hash() {
@ -477,7 +477,7 @@ func testBrokenChain(t *testing.T, full bool) {
// Create a forked chain, and try to insert with a missing link
if full {
chain := makeBlockChain(blockchain.chainConfig, blockchain.CurrentBlock(), 5, ethash.NewFaker(), genDb, forkSeed)[1:]
chain := makeBlockChain(blockchain.chainConfig, blockchain.GetBlockByHash(blockchain.CurrentBlock().Hash()), 5, ethash.NewFaker(), genDb, forkSeed)[1:]
if err := testBlockChainImport(chain, blockchain); err == nil {
t.Errorf("broken block chain not reported")
}
@ -527,10 +527,10 @@ func testReorg(t *testing.T, first, second []int64, td int64, full bool) {
defer blockchain.Stop()
// Insert an easy and a difficult chain afterwards
easyBlocks, _ := GenerateChain(params.TestChainConfig, blockchain.CurrentBlock(), ethash.NewFaker(), genDb, len(first), func(i int, b *BlockGen) {
easyBlocks, _ := GenerateChain(params.TestChainConfig, blockchain.GetBlockByHash(blockchain.CurrentBlock().Hash()), ethash.NewFaker(), genDb, len(first), func(i int, b *BlockGen) {
b.OffsetTime(first[i])
})
diffBlocks, _ := GenerateChain(params.TestChainConfig, blockchain.CurrentBlock(), ethash.NewFaker(), genDb, len(second), func(i int, b *BlockGen) {
diffBlocks, _ := GenerateChain(params.TestChainConfig, blockchain.GetBlockByHash(blockchain.CurrentBlock().Hash()), ethash.NewFaker(), genDb, len(second), func(i int, b *BlockGen) {
b.OffsetTime(second[i])
})
if full {
@ -559,9 +559,9 @@ func testReorg(t *testing.T, first, second []int64, td int64, full bool) {
// Check that the chain is valid number and link wise
if full {
prev := blockchain.CurrentBlock()
for block := blockchain.GetBlockByNumber(blockchain.CurrentBlock().NumberU64() - 1); block.NumberU64() != 0; prev, block = block, blockchain.GetBlockByNumber(block.NumberU64()-1) {
if prev.ParentHash() != block.Hash() {
t.Errorf("parent block hash mismatch: have %x, want %x", prev.ParentHash(), block.Hash())
for block := blockchain.GetBlockByNumber(blockchain.CurrentBlock().Number.Uint64() - 1); block.NumberU64() != 0; prev, block = block.Header(), blockchain.GetBlockByNumber(block.NumberU64()-1) {
if prev.ParentHash != block.Hash() {
t.Errorf("parent block hash mismatch: have %x, want %x", prev.ParentHash, block.Hash())
}
}
} else {
@ -576,7 +576,7 @@ func testReorg(t *testing.T, first, second []int64, td int64, full bool) {
want := new(big.Int).Add(blockchain.genesisBlock.Difficulty(), big.NewInt(td))
if full {
cur := blockchain.CurrentBlock()
if have := blockchain.GetTd(cur.Hash(), cur.NumberU64()); have.Cmp(want) != 0 {
if have := blockchain.GetTd(cur.Hash(), cur.Number.Uint64()); have.Cmp(want) != 0 {
t.Errorf("total difficulty mismatch: have %v, want %v", have, want)
}
} else {
@ -601,7 +601,7 @@ func testBadHashes(t *testing.T, full bool) {
// Create a chain, ban a hash and try to import
if full {
blocks := makeBlockChain(blockchain.chainConfig, blockchain.CurrentBlock(), 3, ethash.NewFaker(), genDb, 10)
blocks := makeBlockChain(blockchain.chainConfig, blockchain.GetBlockByHash(blockchain.CurrentBlock().Hash()), 3, ethash.NewFaker(), genDb, 10)
BadHashes[blocks[2].Header().Hash()] = true
defer func() { delete(BadHashes, blocks[2].Header().Hash()) }()
@ -633,7 +633,7 @@ func testReorgBadHashes(t *testing.T, full bool) {
}
// Create a chain, import and ban afterwards
headers := makeHeaderChain(blockchain.chainConfig, blockchain.CurrentHeader(), 4, ethash.NewFaker(), genDb, 10)
blocks := makeBlockChain(blockchain.chainConfig, blockchain.CurrentBlock(), 4, ethash.NewFaker(), genDb, 10)
blocks := makeBlockChain(blockchain.chainConfig, blockchain.GetBlockByHash(blockchain.CurrentBlock().Hash()), 4, ethash.NewFaker(), genDb, 10)
if full {
if _, err = blockchain.InsertChain(blocks); err != nil {
@ -696,7 +696,7 @@ func testInsertNonceError(t *testing.T, full bool) {
failNum uint64
)
if full {
blocks := makeBlockChain(blockchain.chainConfig, blockchain.CurrentBlock(), i, ethash.NewFaker(), genDb, 0)
blocks := makeBlockChain(blockchain.chainConfig, blockchain.GetBlockByHash(blockchain.CurrentBlock().Hash()), i, ethash.NewFaker(), genDb, 0)
failAt = rand.Int() % len(blocks)
failNum = blocks[failAt].NumberU64()
@ -894,11 +894,11 @@ func TestLightVsFastVsFullChainHeads(t *testing.T) {
assert := func(t *testing.T, kind string, chain *BlockChain, header uint64, fast uint64, block uint64) {
t.Helper()
if num := chain.CurrentBlock().NumberU64(); num != block {
if num := chain.CurrentBlock().Number.Uint64(); num != block {
t.Errorf("%s head block mismatch: have #%v, want #%v", kind, num, block)
}
if num := chain.CurrentFastBlock().NumberU64(); num != fast {
t.Errorf("%s head fast-block mismatch: have #%v, want #%v", kind, num, fast)
if num := chain.CurrentSnapBlock().Number.Uint64(); num != fast {
t.Errorf("%s head snap-block mismatch: have #%v, want #%v", kind, num, fast)
}
if num := chain.CurrentHeader().Number.Uint64(); num != header {
t.Errorf("%s head header mismatch: have #%v, want #%v", kind, num, header)
@ -1649,13 +1649,13 @@ func TestBlockchainHeaderchainReorgConsistency(t *testing.T) {
t.Fatalf("block %d: failed to insert into chain: %v", i, err)
}
if chain.CurrentBlock().Hash() != chain.CurrentHeader().Hash() {
t.Errorf("block %d: current block/header mismatch: block #%d [%x..], header #%d [%x..]", i, chain.CurrentBlock().Number(), chain.CurrentBlock().Hash().Bytes()[:4], chain.CurrentHeader().Number, chain.CurrentHeader().Hash().Bytes()[:4])
t.Errorf("block %d: current block/header mismatch: block #%d [%x..], header #%d [%x..]", i, chain.CurrentBlock().Number, chain.CurrentBlock().Hash().Bytes()[:4], chain.CurrentHeader().Number, chain.CurrentHeader().Hash().Bytes()[:4])
}
if _, err := chain.InsertChain(forks[i : i+1]); err != nil {
t.Fatalf(" fork %d: failed to insert into chain: %v", i, err)
}
if chain.CurrentBlock().Hash() != chain.CurrentHeader().Hash() {
t.Errorf(" fork %d: current block/header mismatch: block #%d [%x..], header #%d [%x..]", i, chain.CurrentBlock().Number(), chain.CurrentBlock().Hash().Bytes()[:4], chain.CurrentHeader().Number, chain.CurrentHeader().Hash().Bytes()[:4])
t.Errorf(" fork %d: current block/header mismatch: block #%d [%x..], header #%d [%x..]", i, chain.CurrentBlock().Number, chain.CurrentBlock().Hash().Bytes()[:4], chain.CurrentHeader().Number, chain.CurrentHeader().Hash().Bytes()[:4])
}
}
}
@ -1797,11 +1797,11 @@ func TestBlockchainRecovery(t *testing.T) {
// Reopen broken blockchain again
ancient, _ = NewBlockChain(ancientDb, nil, gspec, nil, ethash.NewFaker(), vm.Config{}, nil, nil)
defer ancient.Stop()
if num := ancient.CurrentBlock().NumberU64(); num != 0 {
if num := ancient.CurrentBlock().Number.Uint64(); num != 0 {
t.Errorf("head block mismatch: have #%v, want #%v", num, 0)
}
if num := ancient.CurrentFastBlock().NumberU64(); num != midBlock.NumberU64() {
t.Errorf("head fast-block mismatch: have #%v, want #%v", num, midBlock.NumberU64())
if num := ancient.CurrentSnapBlock().Number.Uint64(); num != midBlock.NumberU64() {
t.Errorf("head snap-block mismatch: have #%v, want #%v", num, midBlock.NumberU64())
}
if num := ancient.CurrentHeader().Number.Uint64(); num != midBlock.NumberU64() {
t.Errorf("head header mismatch: have #%v, want #%v", num, midBlock.NumberU64())
@ -1820,7 +1820,7 @@ func TestInsertReceiptChainRollback(t *testing.T) {
if _, err := tmpChain.InsertChain(sideblocks); err != nil {
t.Fatal("processing side chain failed:", err)
}
t.Log("sidechain head:", tmpChain.CurrentBlock().Number(), tmpChain.CurrentBlock().Hash())
t.Log("sidechain head:", tmpChain.CurrentBlock().Number, tmpChain.CurrentBlock().Hash())
sidechainReceipts := make([]types.Receipts, len(sideblocks))
for i, block := range sideblocks {
sidechainReceipts[i] = tmpChain.GetReceiptsByHash(block.Hash())
@ -1829,7 +1829,7 @@ func TestInsertReceiptChainRollback(t *testing.T) {
if _, err := tmpChain.InsertChain(canonblocks); err != nil {
t.Fatal("processing canon chain failed:", err)
}
t.Log("canon head:", tmpChain.CurrentBlock().Number(), tmpChain.CurrentBlock().Hash())
t.Log("canon head:", tmpChain.CurrentBlock().Number, tmpChain.CurrentBlock().Hash())
canonReceipts := make([]types.Receipts, len(canonblocks))
for i, block := range canonblocks {
canonReceipts[i] = tmpChain.GetReceiptsByHash(block.Hash())
@ -1859,8 +1859,8 @@ func TestInsertReceiptChainRollback(t *testing.T) {
if err == nil {
t.Fatal("expected error from InsertReceiptChain.")
}
if ancientChain.CurrentFastBlock().NumberU64() != 0 {
t.Fatalf("failed to rollback ancient data, want %d, have %d", 0, ancientChain.CurrentFastBlock().NumberU64())
if ancientChain.CurrentSnapBlock().Number.Uint64() != 0 {
t.Fatalf("failed to rollback ancient data, want %d, have %d", 0, ancientChain.CurrentSnapBlock().Number)
}
if frozen, err := ancientChain.db.Ancients(); err != nil || frozen != 1 {
t.Fatalf("failed to truncate ancient data, frozen index is %d", frozen)
@ -1871,7 +1871,7 @@ func TestInsertReceiptChainRollback(t *testing.T) {
if err != nil {
t.Fatalf("can't import canon chain receipts: %v", err)
}
if ancientChain.CurrentFastBlock().NumberU64() != canonblocks[len(canonblocks)-1].NumberU64() {
if ancientChain.CurrentSnapBlock().Number.Uint64() != canonblocks[len(canonblocks)-1].NumberU64() {
t.Fatalf("failed to insert ancient recept chain after rollback")
}
if frozen, _ := ancientChain.db.Ancients(); frozen != uint64(len(canonblocks))+1 {
@ -1926,7 +1926,7 @@ func TestLowDiffLongChain(t *testing.T) {
}
// Sanity check that all the canonical numbers are present
header := chain.CurrentHeader()
for number := head.NumberU64(); number > 0; number-- {
for number := head.Number.Uint64(); number > 0; number-- {
if hash := chain.GetHeaderByNumber(number).Hash(); hash != header.Hash() {
t.Fatalf("header %d: canonical hash mismatch: have %x, want %x", number, hash, header.Hash())
}
@ -2150,8 +2150,8 @@ func testInsertKnownChainData(t *testing.T, typ string) {
return err
}
asserter = func(t *testing.T, block *types.Block) {
if chain.CurrentFastBlock().Hash() != block.Hash() {
t.Fatalf("current head fast block mismatch, have %v, want %v", chain.CurrentFastBlock().Hash().Hex(), block.Hash().Hex())
if chain.CurrentSnapBlock().Hash() != block.Hash() {
t.Fatalf("current head fast block mismatch, have %v, want %v", chain.CurrentSnapBlock().Hash().Hex(), block.Hash().Hex())
}
}
} else {
@ -2324,8 +2324,8 @@ func testInsertKnownChainDataWithMerging(t *testing.T, typ string, mergeHeight i
return err
}
asserter = func(t *testing.T, block *types.Block) {
if chain.CurrentFastBlock().Hash() != block.Hash() {
t.Fatalf("current head fast block mismatch, have %v, want %v", chain.CurrentFastBlock().Hash().Hex(), block.Hash().Hex())
if chain.CurrentSnapBlock().Hash() != block.Hash() {
t.Fatalf("current head fast block mismatch, have %v, want %v", chain.CurrentSnapBlock().Hash().Hex(), block.Hash().Hex())
}
}
} else {
@ -2452,7 +2452,7 @@ func TestReorgToShorterRemovesCanonMapping(t *testing.T) {
if n, err := chain.InsertChain(canonblocks); err != nil {
t.Fatalf("block %d: failed to insert into chain: %v", n, err)
}
canonNum := chain.CurrentBlock().NumberU64()
canonNum := chain.CurrentBlock().Number.Uint64()
canonHash := chain.CurrentBlock().Hash()
_, err = chain.InsertChain(sideblocks)
if err != nil {
@ -2467,7 +2467,7 @@ func TestReorgToShorterRemovesCanonMapping(t *testing.T) {
t.Errorf("expected block to be gone: %v", blockByNum.NumberU64())
}
if headerByNum := chain.GetHeaderByNumber(canonNum); headerByNum != nil {
t.Errorf("expected header to be gone: %v", headerByNum.Number.Uint64())
t.Errorf("expected header to be gone: %v", headerByNum.Number)
}
if blockByHash := chain.GetBlockByHash(canonHash); blockByHash == nil {
t.Errorf("expected block to be present: %x", blockByHash.Hash())
@ -2553,7 +2553,7 @@ func TestTransactionIndices(t *testing.T) {
t.Fatalf("Oldest indexded block mismatch, want %d, have %d", *tail, *stored)
}
if tail != nil {
for i := *tail; i <= chain.CurrentBlock().NumberU64(); i++ {
for i := *tail; i <= chain.CurrentBlock().Number.Uint64(); i++ {
block := rawdb.ReadBlock(chain.db, rawdb.ReadCanonicalHash(chain.db, i), i)
if block.Transactions().Len() == 0 {
continue
@ -2649,7 +2649,7 @@ func TestSkipStaleTxIndicesInSnapSync(t *testing.T) {
t.Fatalf("Oldest indexded block mismatch, want %d, have %d", *tail, *stored)
}
if tail != nil {
for i := *tail; i <= chain.CurrentBlock().NumberU64(); i++ {
for i := *tail; i <= chain.CurrentBlock().Number.Uint64(); i++ {
block := rawdb.ReadBlock(chain.db, rawdb.ReadCanonicalHash(chain.db, i), i)
if block.Transactions().Len() == 0 {
continue
@ -2752,7 +2752,8 @@ func benchmarkLargeNumberOfValueToNonexisting(b *testing.B, numTxs, numBlocks in
b.Fatalf("failed to insert shared chain: %v", err)
}
b.StopTimer()
if got := chain.CurrentBlock().Transactions().Len(); got != numTxs*numBlocks {
block := chain.GetBlockByHash(chain.CurrentBlock().Hash())
if got := block.Transactions().Len(); got != numTxs*numBlocks {
b.Fatalf("Transactions were not included, expected %d, got %d", numTxs*numBlocks, got)
}
}
@ -3715,8 +3716,8 @@ func TestSetCanonical(t *testing.T) {
if chain.CurrentBlock().Hash() != head.Hash() {
t.Fatalf("Unexpected block hash, want %x, got %x", head.Hash(), chain.CurrentBlock().Hash())
}
if chain.CurrentFastBlock().Hash() != head.Hash() {
t.Fatalf("Unexpected fast block hash, want %x, got %x", head.Hash(), chain.CurrentFastBlock().Hash())
if chain.CurrentSnapBlock().Hash() != head.Hash() {
t.Fatalf("Unexpected fast block hash, want %x, got %x", head.Hash(), chain.CurrentSnapBlock().Hash())
}
if chain.CurrentHeader().Hash() != head.Hash() {
t.Fatalf("Unexpected head header, want %x, got %x", head.Hash(), chain.CurrentHeader().Hash())
@ -3799,8 +3800,8 @@ func TestCanonicalHashMarker(t *testing.T) {
if chain.CurrentBlock().Hash() != head.Hash() {
t.Fatalf("Unexpected block hash, want %x, got %x", head.Hash(), chain.CurrentBlock().Hash())
}
if chain.CurrentFastBlock().Hash() != head.Hash() {
t.Fatalf("Unexpected fast block hash, want %x, got %x", head.Hash(), chain.CurrentFastBlock().Hash())
if chain.CurrentSnapBlock().Hash() != head.Hash() {
t.Fatalf("Unexpected fast block hash, want %x, got %x", head.Hash(), chain.CurrentSnapBlock().Hash())
}
if chain.CurrentHeader().Hash() != head.Hash() {
t.Fatalf("Unexpected head header, want %x, got %x", head.Hash(), chain.CurrentHeader().Hash())

View File

@ -197,7 +197,7 @@ func ExampleGenerateChain() {
}
state, _ := blockchain.State()
fmt.Printf("last block: #%d\n", blockchain.CurrentBlock().Number())
fmt.Printf("last block: #%d\n", blockchain.CurrentBlock().Number)
fmt.Println("balance of addr1:", state.GetBalance(addr1))
fmt.Println("balance of addr2:", state.GetBalance(addr2))
fmt.Println("balance of addr3:", state.GetBalance(addr3))

View File

@ -76,7 +76,7 @@ func TestDAOForkRangeExtradata(t *testing.T) {
// Create a pro-fork block, and try to feed into the no-fork chain
bc, _ := NewBlockChain(rawdb.NewMemoryDatabase(), nil, congspec, nil, ethash.NewFaker(), vm.Config{}, nil, nil)
blocks := conBc.GetBlocksFromHash(conBc.CurrentBlock().Hash(), int(conBc.CurrentBlock().NumberU64()))
blocks := conBc.GetBlocksFromHash(conBc.CurrentBlock().Hash(), int(conBc.CurrentBlock().Number.Uint64()))
for j := 0; j < len(blocks)/2; j++ {
blocks[j], blocks[len(blocks)-1-j] = blocks[len(blocks)-1-j], blocks[j]
}
@ -87,19 +87,19 @@ func TestDAOForkRangeExtradata(t *testing.T) {
t.Fatalf("failed to commit contra-fork head for expansion: %v", err)
}
bc.Stop()
blocks, _ = GenerateChain(&proConf, conBc.CurrentBlock(), ethash.NewFaker(), genDb, 1, func(i int, gen *BlockGen) {})
blocks, _ = GenerateChain(&proConf, conBc.GetBlockByHash(conBc.CurrentBlock().Hash()), ethash.NewFaker(), genDb, 1, func(i int, gen *BlockGen) {})
if _, err := conBc.InsertChain(blocks); err == nil {
t.Fatalf("contra-fork chain accepted pro-fork block: %v", blocks[0])
}
// Create a proper no-fork block for the contra-forker
blocks, _ = GenerateChain(&conConf, conBc.CurrentBlock(), ethash.NewFaker(), genDb, 1, func(i int, gen *BlockGen) {})
blocks, _ = GenerateChain(&conConf, conBc.GetBlockByHash(conBc.CurrentBlock().Hash()), ethash.NewFaker(), genDb, 1, func(i int, gen *BlockGen) {})
if _, err := conBc.InsertChain(blocks); err != nil {
t.Fatalf("contra-fork chain didn't accepted no-fork block: %v", err)
}
// Create a no-fork block, and try to feed into the pro-fork chain
bc, _ = NewBlockChain(rawdb.NewMemoryDatabase(), nil, progspec, nil, ethash.NewFaker(), vm.Config{}, nil, nil)
blocks = proBc.GetBlocksFromHash(proBc.CurrentBlock().Hash(), int(proBc.CurrentBlock().NumberU64()))
blocks = proBc.GetBlocksFromHash(proBc.CurrentBlock().Hash(), int(proBc.CurrentBlock().Number.Uint64()))
for j := 0; j < len(blocks)/2; j++ {
blocks[j], blocks[len(blocks)-1-j] = blocks[len(blocks)-1-j], blocks[j]
}
@ -110,12 +110,12 @@ func TestDAOForkRangeExtradata(t *testing.T) {
t.Fatalf("failed to commit pro-fork head for expansion: %v", err)
}
bc.Stop()
blocks, _ = GenerateChain(&conConf, proBc.CurrentBlock(), ethash.NewFaker(), genDb, 1, func(i int, gen *BlockGen) {})
blocks, _ = GenerateChain(&conConf, proBc.GetBlockByHash(proBc.CurrentBlock().Hash()), ethash.NewFaker(), genDb, 1, func(i int, gen *BlockGen) {})
if _, err := proBc.InsertChain(blocks); err == nil {
t.Fatalf("pro-fork chain accepted contra-fork block: %v", blocks[0])
}
// Create a proper pro-fork block for the pro-forker
blocks, _ = GenerateChain(&proConf, proBc.CurrentBlock(), ethash.NewFaker(), genDb, 1, func(i int, gen *BlockGen) {})
blocks, _ = GenerateChain(&proConf, proBc.GetBlockByHash(proBc.CurrentBlock().Hash()), ethash.NewFaker(), genDb, 1, func(i int, gen *BlockGen) {})
if _, err := proBc.InsertChain(blocks); err != nil {
t.Fatalf("pro-fork chain didn't accepted pro-fork block: %v", err)
}
@ -124,7 +124,7 @@ func TestDAOForkRangeExtradata(t *testing.T) {
bc, _ := NewBlockChain(rawdb.NewMemoryDatabase(), nil, congspec, nil, ethash.NewFaker(), vm.Config{}, nil, nil)
defer bc.Stop()
blocks := conBc.GetBlocksFromHash(conBc.CurrentBlock().Hash(), int(conBc.CurrentBlock().NumberU64()))
blocks := conBc.GetBlocksFromHash(conBc.CurrentBlock().Hash(), int(conBc.CurrentBlock().Number.Uint64()))
for j := 0; j < len(blocks)/2; j++ {
blocks[j], blocks[len(blocks)-1-j] = blocks[len(blocks)-1-j], blocks[j]
}
@ -134,7 +134,7 @@ func TestDAOForkRangeExtradata(t *testing.T) {
if err := bc.stateCache.TrieDB().Commit(bc.CurrentHeader().Root, false); err != nil {
t.Fatalf("failed to commit contra-fork head for expansion: %v", err)
}
blocks, _ = GenerateChain(&proConf, conBc.CurrentBlock(), ethash.NewFaker(), genDb, 1, func(i int, gen *BlockGen) {})
blocks, _ = GenerateChain(&proConf, conBc.GetBlockByHash(conBc.CurrentBlock().Hash()), ethash.NewFaker(), genDb, 1, func(i int, gen *BlockGen) {})
if _, err := conBc.InsertChain(blocks); err != nil {
t.Fatalf("contra-fork chain didn't accept pro-fork block post-fork: %v", err)
}
@ -142,7 +142,7 @@ func TestDAOForkRangeExtradata(t *testing.T) {
bc, _ = NewBlockChain(rawdb.NewMemoryDatabase(), nil, progspec, nil, ethash.NewFaker(), vm.Config{}, nil, nil)
defer bc.Stop()
blocks = proBc.GetBlocksFromHash(proBc.CurrentBlock().Hash(), int(proBc.CurrentBlock().NumberU64()))
blocks = proBc.GetBlocksFromHash(proBc.CurrentBlock().Hash(), int(proBc.CurrentBlock().Number.Uint64()))
for j := 0; j < len(blocks)/2; j++ {
blocks[j], blocks[len(blocks)-1-j] = blocks[len(blocks)-1-j], blocks[j]
}
@ -152,7 +152,7 @@ func TestDAOForkRangeExtradata(t *testing.T) {
if err := bc.stateCache.TrieDB().Commit(bc.CurrentHeader().Root, false); err != nil {
t.Fatalf("failed to commit pro-fork head for expansion: %v", err)
}
blocks, _ = GenerateChain(&conConf, proBc.CurrentBlock(), ethash.NewFaker(), genDb, 1, func(i int, gen *BlockGen) {})
blocks, _ = GenerateChain(&conConf, proBc.GetBlockByHash(proBc.CurrentBlock().Hash()), ethash.NewFaker(), genDb, 1, func(i int, gen *BlockGen) {})
if _, err := proBc.InsertChain(blocks); err != nil {
t.Fatalf("pro-fork chain didn't accept contra-fork block post-fork: %v", err)
}

View File

@ -145,7 +145,7 @@ const (
// blockChain provides the state of blockchain and current gas limit to do
// some pre checks in tx pool and event subscribers.
type blockChain interface {
CurrentBlock() *types.Block
CurrentBlock() *types.Header
GetBlock(hash common.Hash, number uint64) *types.Block
StateAt(root common.Hash) (*state.StateDB, error)
@ -309,7 +309,7 @@ func NewTxPool(config Config, chainconfig *params.ChainConfig, chain blockChain)
pool.locals.add(addr)
}
pool.priced = newPricedList(pool.all)
pool.reset(nil, chain.CurrentBlock().Header())
pool.reset(nil, chain.CurrentBlock())
// Start the reorg loop early so it can handle requests generated during journal loading.
pool.wg.Add(1)
@ -361,8 +361,8 @@ func (pool *TxPool) loop() {
// Handle ChainHeadEvent
case ev := <-pool.chainHeadCh:
if ev.Block != nil {
pool.requestReset(head.Header(), ev.Block.Header())
head = ev.Block
pool.requestReset(head, ev.Block.Header())
head = ev.Block.Header()
}
// System shutdown.
@ -1291,7 +1291,7 @@ func (pool *TxPool) reset(oldHead, newHead *types.Header) {
}
// Initialize the internal state to the current head
if newHead == nil {
newHead = pool.chain.CurrentBlock().Header() // Special case during testing
newHead = pool.chain.CurrentBlock() // Special case during testing
}
statedb, err := pool.chain.StateAt(newHead.Root)
if err != nil {

View File

@ -64,14 +64,15 @@ type testBlockChain struct {
chainHeadFeed *event.Feed
}
func (bc *testBlockChain) CurrentBlock() *types.Block {
return types.NewBlock(&types.Header{
func (bc *testBlockChain) CurrentBlock() *types.Header {
return &types.Header{
Number: new(big.Int),
GasLimit: atomic.LoadUint64(&bc.gasLimit),
}, nil, nil, nil, trie.NewStackTrie(nil))
}
}
func (bc *testBlockChain) GetBlock(hash common.Hash, number uint64) *types.Block {
return bc.CurrentBlock()
return types.NewBlock(bc.CurrentBlock(), nil, nil, nil, trie.NewStackTrie(nil))
}
func (bc *testBlockChain) StateAt(common.Hash) (*state.StateDB, error) {

View File

@ -267,20 +267,24 @@ func (api *DebugAPI) DumpBlock(blockNr rpc.BlockNumber) (state.Dump, error) {
_, stateDb := api.eth.miner.Pending()
return stateDb.RawDump(opts), nil
}
var block *types.Block
var header *types.Header
if blockNr == rpc.LatestBlockNumber {
block = api.eth.blockchain.CurrentBlock()
header = api.eth.blockchain.CurrentBlock()
} else if blockNr == rpc.FinalizedBlockNumber {
block = api.eth.blockchain.CurrentFinalizedBlock()
header = api.eth.blockchain.CurrentFinalBlock()
} else if blockNr == rpc.SafeBlockNumber {
block = api.eth.blockchain.CurrentSafeBlock()
header = api.eth.blockchain.CurrentSafeBlock()
} else {
block = api.eth.blockchain.GetBlockByNumber(uint64(blockNr))
block := api.eth.blockchain.GetBlockByNumber(uint64(blockNr))
if block == nil {
return state.Dump{}, fmt.Errorf("block #%d not found", blockNr)
}
header = block.Header()
}
if block == nil {
if header == nil {
return state.Dump{}, fmt.Errorf("block #%d not found", blockNr)
}
stateDb, err := api.eth.BlockChain().StateAt(block.Root())
stateDb, err := api.eth.BlockChain().StateAt(header.Root)
if err != nil {
return state.Dump{}, err
}
@ -347,20 +351,24 @@ func (api *DebugAPI) AccountRange(blockNrOrHash rpc.BlockNumberOrHash, start hex
// the miner and operate on those
_, stateDb = api.eth.miner.Pending()
} else {
var block *types.Block
var header *types.Header
if number == rpc.LatestBlockNumber {
block = api.eth.blockchain.CurrentBlock()
header = api.eth.blockchain.CurrentBlock()
} else if number == rpc.FinalizedBlockNumber {
block = api.eth.blockchain.CurrentFinalizedBlock()
header = api.eth.blockchain.CurrentFinalBlock()
} else if number == rpc.SafeBlockNumber {
block = api.eth.blockchain.CurrentSafeBlock()
header = api.eth.blockchain.CurrentSafeBlock()
} else {
block = api.eth.blockchain.GetBlockByNumber(uint64(number))
block := api.eth.blockchain.GetBlockByNumber(uint64(number))
if block == nil {
return state.IteratorDump{}, fmt.Errorf("block #%d not found", number)
}
header = block.Header()
}
if block == nil {
if header == nil {
return state.IteratorDump{}, fmt.Errorf("block #%d not found", number)
}
stateDb, err = api.eth.BlockChain().StateAt(block.Root())
stateDb, err = api.eth.BlockChain().StateAt(header.Root)
if err != nil {
return state.IteratorDump{}, err
}
@ -552,7 +560,7 @@ func (api *DebugAPI) GetAccessibleState(from, to rpc.BlockNumber) (uint64, error
if block == nil {
return 0, fmt.Errorf("current block missing")
}
return block.NumberU64(), nil
return block.Number.Uint64(), nil
}
return uint64(num.Int64()), nil
}

View File

@ -55,7 +55,7 @@ func (b *EthAPIBackend) ChainConfig() *params.ChainConfig {
return b.eth.blockchain.Config()
}
func (b *EthAPIBackend) CurrentBlock() *types.Block {
func (b *EthAPIBackend) CurrentBlock() *types.Header {
return b.eth.blockchain.CurrentBlock()
}
@ -72,19 +72,19 @@ func (b *EthAPIBackend) HeaderByNumber(ctx context.Context, number rpc.BlockNumb
}
// Otherwise resolve and return the block
if number == rpc.LatestBlockNumber {
return b.eth.blockchain.CurrentBlock().Header(), nil
return b.eth.blockchain.CurrentBlock(), nil
}
if number == rpc.FinalizedBlockNumber {
block := b.eth.blockchain.CurrentFinalizedBlock()
block := b.eth.blockchain.CurrentFinalBlock()
if block != nil {
return block.Header(), nil
return block, nil
}
return nil, errors.New("finalized block not found")
}
if number == rpc.SafeBlockNumber {
block := b.eth.blockchain.CurrentSafeBlock()
if block != nil {
return block.Header(), nil
return block, nil
}
return nil, errors.New("safe block not found")
}
@ -120,13 +120,16 @@ func (b *EthAPIBackend) BlockByNumber(ctx context.Context, number rpc.BlockNumbe
}
// Otherwise resolve and return the block
if number == rpc.LatestBlockNumber {
return b.eth.blockchain.CurrentBlock(), nil
header := b.eth.blockchain.CurrentBlock()
return b.eth.blockchain.GetBlock(header.Hash(), header.Number.Uint64()), nil
}
if number == rpc.FinalizedBlockNumber {
return b.eth.blockchain.CurrentFinalizedBlock(), nil
header := b.eth.blockchain.CurrentFinalBlock()
return b.eth.blockchain.GetBlock(header.Hash(), header.Number.Uint64()), nil
}
if number == rpc.SafeBlockNumber {
return b.eth.blockchain.CurrentSafeBlock(), nil
header := b.eth.blockchain.CurrentSafeBlock()
return b.eth.blockchain.GetBlock(header.Hash(), header.Number.Uint64()), nil
}
return b.eth.blockchain.GetBlockByNumber(uint64(number)), nil
}

View File

@ -301,7 +301,7 @@ func (api *ConsensusAPI) forkchoiceUpdated(update engine.ForkchoiceStateV1, payl
} else {
// If the head block is already in our canonical chain, the beacon client is
// probably resyncing. Ignore the update.
log.Info("Ignoring beacon update to old head", "number", block.NumberU64(), "hash", update.HeadBlockHash, "age", common.PrettyAge(time.Unix(int64(block.Time()), 0)), "have", api.eth.BlockChain().CurrentBlock().NumberU64())
log.Info("Ignoring beacon update to old head", "number", block.NumberU64(), "hash", update.HeadBlockHash, "age", common.PrettyAge(time.Unix(int64(block.Time()), 0)), "have", api.eth.BlockChain().CurrentBlock().Number)
return valid(nil), nil
}
api.eth.SetSynced()
@ -322,7 +322,7 @@ func (api *ConsensusAPI) forkchoiceUpdated(update engine.ForkchoiceStateV1, payl
return engine.STATUS_INVALID, engine.InvalidForkChoiceState.With(errors.New("final block not in canonical chain"))
}
// Set the finalized block
api.eth.BlockChain().SetFinalized(finalBlock)
api.eth.BlockChain().SetFinalized(finalBlock.Header())
}
// Check if the safe block hash is in our canonical tree, if not somethings wrong
if update.SafeBlockHash != (common.Hash{}) {
@ -336,7 +336,7 @@ func (api *ConsensusAPI) forkchoiceUpdated(update engine.ForkchoiceStateV1, payl
return engine.STATUS_INVALID, engine.InvalidForkChoiceState.With(errors.New("safe block not in canonical chain"))
}
// Set the safe block
api.eth.BlockChain().SetSafe(safeBlock)
api.eth.BlockChain().SetSafe(safeBlock.Header())
}
// If payload generation was requested, create a new block to be potentially
// sealed by the beacon client. The payload will be requested later, and we
@ -804,7 +804,7 @@ func (api *ConsensusAPI) GetPayloadBodiesByRangeV1(start, count hexutil.Uint64)
return nil, engine.TooLargeRequest.With(fmt.Errorf("requested count too large: %v", count))
}
// limit count up until current
current := api.eth.BlockChain().CurrentBlock().NumberU64()
current := api.eth.BlockChain().CurrentBlock().Number.Uint64()
last := uint64(start) + uint64(count) - 1
if last > current {
last = current

View File

@ -248,8 +248,8 @@ func TestInvalidPayloadTimestamp(t *testing.T) {
shouldErr bool
}{
{0, true},
{parent.Time(), true},
{parent.Time() - 1, true},
{parent.Time, true},
{parent.Time - 1, true},
// TODO (MariusVanDerWijden) following tests are currently broken,
// fixed in upcoming merge-kiln-v2 pr
@ -262,7 +262,7 @@ func TestInvalidPayloadTimestamp(t *testing.T) {
params := engine.PayloadAttributes{
Timestamp: test.time,
Random: crypto.Keccak256Hash([]byte{byte(123)}),
SuggestedFeeRecipient: parent.Coinbase(),
SuggestedFeeRecipient: parent.Coinbase,
}
fcState := engine.ForkchoiceStateV1{
HeadBlockHash: parent.Hash(),
@ -319,7 +319,7 @@ func TestEth2NewBlock(t *testing.T) {
t.Fatalf("Failed to insert block: %v", err)
case newResp.Status != "VALID":
t.Fatalf("Failed to insert block: %v", newResp.Status)
case ethservice.BlockChain().CurrentBlock().NumberU64() != block.NumberU64()-1:
case ethservice.BlockChain().CurrentBlock().Number.Uint64() != block.NumberU64()-1:
t.Fatalf("Chain head shouldn't be updated")
}
checkLogEvents(t, newLogCh, rmLogsCh, 0, 0)
@ -331,7 +331,7 @@ func TestEth2NewBlock(t *testing.T) {
if _, err := api.ForkchoiceUpdatedV1(fcState, nil); err != nil {
t.Fatalf("Failed to insert block: %v", err)
}
if have, want := ethservice.BlockChain().CurrentBlock().NumberU64(), block.NumberU64(); have != want {
if have, want := ethservice.BlockChain().CurrentBlock().Number.Uint64(), block.NumberU64(); have != want {
t.Fatalf("Chain head should be updated, have %d want %d", have, want)
}
checkLogEvents(t, newLogCh, rmLogsCh, 1, 0)
@ -341,7 +341,7 @@ func TestEth2NewBlock(t *testing.T) {
// Introduce fork chain
var (
head = ethservice.BlockChain().CurrentBlock().NumberU64()
head = ethservice.BlockChain().CurrentBlock().Number.Uint64()
)
parent = preMergeBlocks[len(preMergeBlocks)-1]
for i := 0; i < 10; i++ {
@ -359,7 +359,7 @@ func TestEth2NewBlock(t *testing.T) {
if err != nil || newResp.Status != "VALID" {
t.Fatalf("Failed to insert block: %v", err)
}
if ethservice.BlockChain().CurrentBlock().NumberU64() != head {
if ethservice.BlockChain().CurrentBlock().Number.Uint64() != head {
t.Fatalf("Chain head shouldn't be updated")
}
@ -371,7 +371,7 @@ func TestEth2NewBlock(t *testing.T) {
if _, err := api.ForkchoiceUpdatedV1(fcState, nil); err != nil {
t.Fatalf("Failed to insert block: %v", err)
}
if ethservice.BlockChain().CurrentBlock().NumberU64() != block.NumberU64() {
if ethservice.BlockChain().CurrentBlock().Number.Uint64() != block.NumberU64() {
t.Fatalf("Chain head should be updated")
}
parent, head = block, block.NumberU64()
@ -389,7 +389,7 @@ func TestEth2DeepReorg(t *testing.T) {
var (
api = NewConsensusAPI(ethservice, nil)
parent = preMergeBlocks[len(preMergeBlocks)-core.TriesInMemory-1]
head = ethservice.BlockChain().CurrentBlock().NumberU64()
head = ethservice.BlockChain().CurrentBlock().Number.Uint64()()
)
if ethservice.BlockChain().HasBlockAndState(parent.Hash(), parent.NumberU64()) {
t.Errorf("Block %d not pruned", parent.NumberU64())
@ -410,13 +410,13 @@ func TestEth2DeepReorg(t *testing.T) {
if err != nil || newResp.Status != "VALID" {
t.Fatalf("Failed to insert block: %v", err)
}
if ethservice.BlockChain().CurrentBlock().NumberU64() != head {
if ethservice.BlockChain().CurrentBlock().Number.Uint64()() != head {
t.Fatalf("Chain head shouldn't be updated")
}
if err := api.setHead(block.Hash()); err != nil {
t.Fatalf("Failed to set head: %v", err)
}
if ethservice.BlockChain().CurrentBlock().NumberU64() != block.NumberU64() {
if ethservice.BlockChain().CurrentBlock().Number.Uint64()() != block.NumberU64() {
t.Fatalf("Chain head should be updated")
}
parent, head = block, block.NumberU64()
@ -466,8 +466,8 @@ func TestFullAPI(t *testing.T) {
logCode = common.Hex2Bytes("60606040525b7f24ec1d3ff24c2f6ff210738839dbc339cd45a5294d85c79361016243157aae7b60405180905060405180910390a15b600a8060416000396000f360606040526008565b00")
)
callback := func(parent *types.Block) {
statedb, _ := ethservice.BlockChain().StateAt(parent.Root())
callback := func(parent *types.Header) {
statedb, _ := ethservice.BlockChain().StateAt(parent.Root)
nonce := statedb.GetNonce(testAddr)
tx, _ := types.SignTx(types.NewContractCreation(nonce, new(big.Int), 1000000, big.NewInt(2*params.InitialBaseFee), logCode), types.LatestSigner(ethservice.BlockChain().Config()), testKey)
ethservice.TxPool().AddLocal(tx)
@ -476,9 +476,9 @@ func TestFullAPI(t *testing.T) {
setupBlocks(t, ethservice, 10, parent, callback)
}
func setupBlocks(t *testing.T, ethservice *eth.Ethereum, n int, parent *types.Block, callback func(parent *types.Block)) []*types.Block {
func setupBlocks(t *testing.T, ethservice *eth.Ethereum, n int, parent *types.Header, callback func(parent *types.Header)) []*types.Header {
api := NewConsensusAPI(ethservice)
var blocks []*types.Block
var blocks []*types.Header
for i := 0; i < n; i++ {
callback(parent)
@ -499,10 +499,10 @@ func setupBlocks(t *testing.T, ethservice *eth.Ethereum, n int, parent *types.Bl
if _, err := api.ForkchoiceUpdatedV1(fcState, nil); err != nil {
t.Fatalf("Failed to insert block: %v", err)
}
if ethservice.BlockChain().CurrentBlock().NumberU64() != payload.Number {
if ethservice.BlockChain().CurrentBlock().Number.Uint64() != payload.Number {
t.Fatal("Chain head should be updated")
}
if ethservice.BlockChain().CurrentFinalizedBlock().NumberU64() != payload.Number-1 {
if ethservice.BlockChain().CurrentFinalBlock().Number.Uint64() != payload.Number-1 {
t.Fatal("Finalized block should be updated")
}
parent = ethservice.BlockChain().CurrentBlock()
@ -585,7 +585,7 @@ func TestNewPayloadOnInvalidChain(t *testing.T) {
logCode = common.Hex2Bytes("60606040525b7f24ec1d3ff24c2f6ff210738839dbc339cd45a5294d85c79361016243157aae7b60405180905060405180910390a15b600a8060416000396000f360606040526008565b00")
)
for i := 0; i < 10; i++ {
statedb, _ := ethservice.BlockChain().StateAt(parent.Root())
statedb, _ := ethservice.BlockChain().StateAt(parent.Root)
tx := types.MustSignNewTx(testKey, signer, &types.LegacyTx{
Nonce: statedb.GetNonce(testAddr),
Value: new(big.Int),
@ -596,9 +596,9 @@ func TestNewPayloadOnInvalidChain(t *testing.T) {
ethservice.TxPool().AddRemotesSync([]*types.Transaction{tx})
var (
params = engine.PayloadAttributes{
Timestamp: parent.Time() + 1,
Timestamp: parent.Time + 1,
Random: crypto.Keccak256Hash([]byte{byte(i)}),
SuggestedFeeRecipient: parent.Coinbase(),
SuggestedFeeRecipient: parent.Coinbase,
}
fcState = engine.ForkchoiceStateV1{
HeadBlockHash: parent.Hash(),
@ -645,7 +645,7 @@ func TestNewPayloadOnInvalidChain(t *testing.T) {
if _, err := api.ForkchoiceUpdatedV1(fcState, nil); err != nil {
t.Fatalf("Failed to insert block: %v", err)
}
if ethservice.BlockChain().CurrentBlock().NumberU64() != payload.Number {
if ethservice.BlockChain().CurrentBlock().Number.Uint64() != payload.Number {
t.Fatalf("Chain head should be updated")
}
parent = ethservice.BlockChain().CurrentBlock()
@ -676,7 +676,7 @@ func TestEmptyBlocks(t *testing.T) {
api := NewConsensusAPI(ethservice)
// Setup 10 blocks on the canonical chain
setupBlocks(t, ethservice, 10, commonAncestor, func(parent *types.Block) {})
setupBlocks(t, ethservice, 10, commonAncestor, func(parent *types.Header) {})
// (1) check LatestValidHash by sending a normal payload (P1'')
payload := getNewPayload(t, api, commonAncestor)
@ -727,11 +727,11 @@ func TestEmptyBlocks(t *testing.T) {
}
}
func getNewPayload(t *testing.T, api *ConsensusAPI, parent *types.Block) *engine.ExecutableData {
func getNewPayload(t *testing.T, api *ConsensusAPI, parent *types.Header) *engine.ExecutableData {
params := engine.PayloadAttributes{
Timestamp: parent.Time() + 1,
Timestamp: parent.Time + 1,
Random: crypto.Keccak256Hash([]byte{byte(1)}),
SuggestedFeeRecipient: parent.Coinbase(),
SuggestedFeeRecipient: parent.Coinbase,
}
payload, err := assembleBlock(api, parent.Hash(), &params)
@ -799,7 +799,7 @@ func TestTrickRemoteBlockCache(t *testing.T) {
commonAncestor := ethserviceA.BlockChain().CurrentBlock()
// Setup 10 blocks on the canonical chain
setupBlocks(t, ethserviceA, 10, commonAncestor, func(parent *types.Block) {})
setupBlocks(t, ethserviceA, 10, commonAncestor, func(parent *types.Header) {})
commonAncestor = ethserviceA.BlockChain().CurrentBlock()
var invalidChain []*engine.ExecutableData
@ -855,7 +855,7 @@ func TestInvalidBloom(t *testing.T) {
api := NewConsensusAPI(ethservice)
// Setup 10 blocks on the canonical chain
setupBlocks(t, ethservice, 10, commonAncestor, func(parent *types.Block) {})
setupBlocks(t, ethservice, 10, commonAncestor, func(parent *types.Header) {})
// (1) check LatestValidHash by sending a normal payload (P1'')
payload := getNewPayload(t, api, commonAncestor)
@ -966,7 +966,7 @@ func TestSimultaneousNewBlock(t *testing.T) {
if err != nil {
t.Fatalf("Failed to convert executable data to block %v", err)
}
if ethservice.BlockChain().CurrentBlock().NumberU64() != block.NumberU64()-1 {
if ethservice.BlockChain().CurrentBlock().Number.Uint64() != block.NumberU64()-1 {
t.Fatalf("Chain head shouldn't be updated")
}
fcState := engine.ForkchoiceStateV1{
@ -997,7 +997,7 @@ func TestSimultaneousNewBlock(t *testing.T) {
t.Fatal(testErr)
}
}
if have, want := ethservice.BlockChain().CurrentBlock().NumberU64(), block.NumberU64(); have != want {
if have, want := ethservice.BlockChain().CurrentBlock().Number.Uint64(), block.NumberU64(); have != want {
t.Fatalf("Chain head should be updated, have %d want %d", have, want)
}
parent = block
@ -1242,14 +1242,18 @@ func setupBodies(t *testing.T) (*node.Node, *eth.Ethereum, []*types.Block) {
logCode = common.Hex2Bytes("60606040525b7f24ec1d3ff24c2f6ff210738839dbc339cd45a5294d85c79361016243157aae7b60405180905060405180910390a15b600a8060416000396000f360606040526008565b00")
)
callback := func(parent *types.Block) {
statedb, _ := ethservice.BlockChain().StateAt(parent.Root())
callback := func(parent *types.Header) {
statedb, _ := ethservice.BlockChain().StateAt(parent.Root)
nonce := statedb.GetNonce(testAddr)
tx, _ := types.SignTx(types.NewContractCreation(nonce, new(big.Int), 1000000, big.NewInt(2*params.InitialBaseFee), logCode), types.LatestSigner(ethservice.BlockChain().Config()), testKey)
ethservice.TxPool().AddLocal(tx)
}
postMergeBlocks := setupBlocks(t, ethservice, 10, parent, callback)
postMergeHeaders := setupBlocks(t, ethservice, 10, parent, callback)
postMergeBlocks := make([]*types.Block, len(postMergeHeaders))
for i, header := range postMergeHeaders {
postMergeBlocks[i] = ethservice.BlockChain().GetBlock(header.Hash(), header.Number.Uint64())
}
return n, ethservice, append(preMergeBlocks, postMergeBlocks...)
}

View File

@ -76,7 +76,7 @@ func (b *beaconBackfiller) suspend() *types.Header {
// Sync cycle was just terminated, retrieve and return the last filled header.
// Can't use `filled` as that contains a stale value from before cancellation.
return b.downloader.blockchain.CurrentFastBlock().Header()
return b.downloader.blockchain.CurrentSnapBlock()
}
// resume starts the downloader threads for backfilling state and chain data.
@ -101,7 +101,7 @@ func (b *beaconBackfiller) resume() {
defer func() {
b.lock.Lock()
b.filling = false
b.filled = b.downloader.blockchain.CurrentFastBlock().Header()
b.filled = b.downloader.blockchain.CurrentSnapBlock()
b.lock.Unlock()
}()
// If the downloader fails, report an error as in beacon chain mode there
@ -198,9 +198,9 @@ func (d *Downloader) findBeaconAncestor() (uint64, error) {
switch d.getMode() {
case FullSync:
chainHead = d.blockchain.CurrentBlock().Header()
chainHead = d.blockchain.CurrentBlock()
case SnapSync:
chainHead = d.blockchain.CurrentFastBlock().Header()
chainHead = d.blockchain.CurrentSnapBlock()
default:
chainHead = d.lightchain.CurrentHeader()
}

View File

@ -196,10 +196,10 @@ type BlockChain interface {
GetBlockByHash(common.Hash) *types.Block
// CurrentBlock retrieves the head block from the local chain.
CurrentBlock() *types.Block
CurrentBlock() *types.Header
// CurrentFastBlock retrieves the head snap block from the local chain.
CurrentFastBlock() *types.Block
// CurrentSnapBlock retrieves the head snap block from the local chain.
CurrentSnapBlock() *types.Header
// SnapSyncCommitHead directly commits the head block to a certain entity.
SnapSyncCommitHead(common.Hash) error
@ -236,7 +236,7 @@ func New(checkpoint uint64, stateDb ethdb.Database, mux *event.TypeMux, chain Bl
quitCh: make(chan struct{}),
SnapSyncer: snap.NewSyncer(stateDb, chain.TrieDB().Scheme()),
stateSyncStart: make(chan *stateSync),
syncStartBlock: chain.CurrentFastBlock().NumberU64(),
syncStartBlock: chain.CurrentSnapBlock().Number.Uint64(),
}
// Create the post-merge skeleton syncer and start the process
dl.skeleton = newSkeleton(stateDb, dl.peers, dropPeer, newBeaconBackfiller(dl, success))
@ -261,9 +261,9 @@ func (d *Downloader) Progress() ethereum.SyncProgress {
mode := d.getMode()
switch {
case d.blockchain != nil && mode == FullSync:
current = d.blockchain.CurrentBlock().NumberU64()
current = d.blockchain.CurrentBlock().Number.Uint64()
case d.blockchain != nil && mode == SnapSync:
current = d.blockchain.CurrentFastBlock().NumberU64()
current = d.blockchain.CurrentSnapBlock().Number.Uint64()
case d.lightchain != nil:
current = d.lightchain.CurrentHeader().Number.Uint64()
default:
@ -523,7 +523,7 @@ func (d *Downloader) syncWithPeer(p *peerConnection, hash common.Hash, td, ttd *
// anyway, but still need a valid pivot block to avoid some code hitting
// nil panics on access.
if mode == SnapSync && pivot == nil {
pivot = d.blockchain.CurrentBlock().Header()
pivot = d.blockchain.CurrentBlock()
}
height := latest.Number.Uint64()
@ -834,9 +834,9 @@ func (d *Downloader) findAncestor(p *peerConnection, remoteHeader *types.Header)
mode := d.getMode()
switch mode {
case FullSync:
localHeight = d.blockchain.CurrentBlock().NumberU64()
localHeight = d.blockchain.CurrentBlock().Number.Uint64()
case SnapSync:
localHeight = d.blockchain.CurrentFastBlock().NumberU64()
localHeight = d.blockchain.CurrentSnapBlock().Number.Uint64()
default:
localHeight = d.lightchain.CurrentHeader().Number.Uint64()
}
@ -1174,8 +1174,8 @@ func (d *Downloader) fetchHeaders(p *peerConnection, from uint64, head uint64) e
if mode == LightSync {
head = d.lightchain.CurrentHeader().Number.Uint64()
} else {
head = d.blockchain.CurrentFastBlock().NumberU64()
if full := d.blockchain.CurrentBlock().NumberU64(); head < full {
head = d.blockchain.CurrentSnapBlock().Number.Uint64()
if full := d.blockchain.CurrentBlock().Number.Uint64(); head < full {
head = full
}
}
@ -1289,8 +1289,8 @@ func (d *Downloader) processHeaders(origin uint64, td, ttd *big.Int, beaconMode
if rollback > 0 {
lastHeader, lastFastBlock, lastBlock := d.lightchain.CurrentHeader().Number, common.Big0, common.Big0
if mode != LightSync {
lastFastBlock = d.blockchain.CurrentFastBlock().Number()
lastBlock = d.blockchain.CurrentBlock().Number()
lastFastBlock = d.blockchain.CurrentSnapBlock().Number
lastBlock = d.blockchain.CurrentBlock().Number
}
if err := d.lightchain.SetHead(rollback - 1); err != nil { // -1 to target the parent of the first uncertain block
// We're already unwinding the stack, only print the error to make it more visible
@ -1298,8 +1298,8 @@ func (d *Downloader) processHeaders(origin uint64, td, ttd *big.Int, beaconMode
}
curFastBlock, curBlock := common.Big0, common.Big0
if mode != LightSync {
curFastBlock = d.blockchain.CurrentFastBlock().Number()
curBlock = d.blockchain.CurrentBlock().Number()
curFastBlock = d.blockchain.CurrentSnapBlock().Number
curBlock = d.blockchain.CurrentBlock().Number
}
log.Warn("Rolled back chain segment",
"header", fmt.Sprintf("%d->%d", lastHeader, d.lightchain.CurrentHeader().Number),
@ -1344,7 +1344,7 @@ func (d *Downloader) processHeaders(origin uint64, td, ttd *big.Int, beaconMode
// R: Nothing to give
if mode != LightSync {
head := d.blockchain.CurrentBlock()
if !gotHeaders && td.Cmp(d.blockchain.GetTd(head.Hash(), head.NumberU64())) > 0 {
if !gotHeaders && td.Cmp(d.blockchain.GetTd(head.Hash(), head.Number.Uint64())) > 0 {
return errStallingPeer
}
}
@ -1868,9 +1868,9 @@ func (d *Downloader) reportSnapSyncProgress(force bool) {
}
var (
header = d.blockchain.CurrentHeader()
block = d.blockchain.CurrentFastBlock()
block = d.blockchain.CurrentSnapBlock()
)
syncedBlocks := block.NumberU64() - d.syncStartBlock
syncedBlocks := block.Number.Uint64() - d.syncStartBlock
if syncedBlocks == 0 {
return
}
@ -1887,13 +1887,13 @@ func (d *Downloader) reportSnapSyncProgress(force bool) {
return
}
var (
left = latest.Number.Uint64() - block.NumberU64()
left = latest.Number.Uint64() - block.Number.Uint64()
eta = time.Since(d.syncStartTime) / time.Duration(syncedBlocks) * time.Duration(left)
progress = fmt.Sprintf("%.2f%%", float64(block.NumberU64())*100/float64(latest.Number.Uint64()))
progress = fmt.Sprintf("%.2f%%", float64(block.Number.Uint64())*100/float64(latest.Number.Uint64()))
headers = fmt.Sprintf("%v@%v", log.FormatLogfmtUint64(header.Number.Uint64()), common.StorageSize(headerBytes).TerminalString())
bodies = fmt.Sprintf("%v@%v", log.FormatLogfmtUint64(block.NumberU64()), common.StorageSize(bodyBytes).TerminalString())
receipts = fmt.Sprintf("%v@%v", log.FormatLogfmtUint64(block.NumberU64()), common.StorageSize(receiptBytes).TerminalString())
bodies = fmt.Sprintf("%v@%v", log.FormatLogfmtUint64(block.Number.Uint64()), common.StorageSize(bodyBytes).TerminalString())
receipts = fmt.Sprintf("%v@%v", log.FormatLogfmtUint64(block.Number.Uint64()), common.StorageSize(receiptBytes).TerminalString())
)
log.Info("Syncing: chain download in progress", "synced", progress, "chain", syncedBytes, "headers", headers, "bodies", bodies, "receipts", receipts, "eta", common.PrettyDuration(eta))
d.syncLogTime = time.Now()

View File

@ -100,7 +100,7 @@ func (dl *downloadTester) sync(id string, td *big.Int, mode SyncMode) error {
head := dl.peers[id].chain.CurrentBlock()
if td == nil {
// If no particular TD was requested, load from the peer's blockchain
td = dl.peers[id].chain.GetTd(head.Hash(), head.NumberU64())
td = dl.peers[id].chain.GetTd(head.Hash(), head.Number.Uint64())
}
// Synchronise with the chosen peer and ensure proper cleanup afterwards
err := dl.downloader.synchronise(id, head.Hash(), td, nil, mode, false, nil)
@ -158,7 +158,7 @@ type downloadTesterPeer struct {
// and total difficulty.
func (dlp *downloadTesterPeer) Head() (common.Hash, *big.Int) {
head := dlp.chain.CurrentBlock()
return head.Hash(), dlp.chain.GetTd(head.Hash(), head.NumberU64())
return head.Hash(), dlp.chain.GetTd(head.Hash(), head.Number.Uint64())
}
func unmarshalRlpHeaders(rlpdata []rlp.RawValue) []*types.Header {
@ -430,10 +430,10 @@ func assertOwnChain(t *testing.T, tester *downloadTester, length int) {
if hs := int(tester.chain.CurrentHeader().Number.Uint64()) + 1; hs != headers {
t.Fatalf("synchronised headers mismatch: have %v, want %v", hs, headers)
}
if bs := int(tester.chain.CurrentBlock().NumberU64()) + 1; bs != blocks {
if bs := int(tester.chain.CurrentBlock().Number.Uint64()) + 1; bs != blocks {
t.Fatalf("synchronised blocks mismatch: have %v, want %v", bs, blocks)
}
if rs := int(tester.chain.CurrentFastBlock().NumberU64()) + 1; rs != receipts {
if rs := int(tester.chain.CurrentSnapBlock().Number.Uint64()) + 1; rs != receipts {
t.Fatalf("synchronised receipts mismatch: have %v, want %v", rs, receipts)
}
}
@ -490,7 +490,7 @@ func testThrottling(t *testing.T, protocol uint, mode SyncMode) {
for {
// Check the retrieval count synchronously (! reason for this ugly block)
tester.lock.RLock()
retrieved := int(tester.chain.CurrentFastBlock().Number().Uint64()) + 1
retrieved := int(tester.chain.CurrentSnapBlock().Number.Uint64()) + 1
tester.lock.RUnlock()
if retrieved >= targetBlocks+1 {
break
@ -506,7 +506,7 @@ func testThrottling(t *testing.T, protocol uint, mode SyncMode) {
{
cached = tester.downloader.queue.resultCache.countCompleted()
frozen = int(atomic.LoadUint32(&blocked))
retrieved = int(tester.chain.CurrentFastBlock().Number().Uint64()) + 1
retrieved = int(tester.chain.CurrentSnapBlock().Number.Uint64()) + 1
}
tester.downloader.queue.resultCache.lock.Unlock()
tester.downloader.queue.lock.Unlock()
@ -522,7 +522,7 @@ func testThrottling(t *testing.T, protocol uint, mode SyncMode) {
// Make sure we filled up the cache, then exhaust it
time.Sleep(25 * time.Millisecond) // give it a chance to screw up
tester.lock.RLock()
retrieved = int(tester.chain.CurrentFastBlock().Number().Uint64()) + 1
retrieved = int(tester.chain.CurrentSnapBlock().Number.Uint64()) + 1
tester.lock.RUnlock()
if cached != blockCacheMaxItems && cached != blockCacheMaxItems-reorgProtHeaderDelay && retrieved+cached+frozen != targetBlocks+1 && retrieved+cached+frozen != targetBlocks+1-reorgProtHeaderDelay {
t.Fatalf("block count mismatch: have %v, want %v (owned %v, blocked %v, target %v)", cached, blockCacheMaxItems, retrieved, frozen, targetBlocks+1)
@ -921,7 +921,7 @@ func testInvalidHeaderRollback(t *testing.T, protocol uint, mode SyncMode) {
t.Errorf("rollback head mismatch: have %v, want at most %v", head, 2*fsHeaderSafetyNet+MaxHeaderFetch)
}
if mode == SnapSync {
if head := tester.chain.CurrentBlock().NumberU64(); head != 0 {
if head := tester.chain.CurrentBlock().Number.Uint64(); head != 0 {
t.Errorf("fast sync pivot block #%d not rolled back", head)
}
}
@ -943,7 +943,7 @@ func testInvalidHeaderRollback(t *testing.T, protocol uint, mode SyncMode) {
t.Errorf("rollback head mismatch: have %v, want at most %v", head, 2*fsHeaderSafetyNet+MaxHeaderFetch)
}
if mode == SnapSync {
if head := tester.chain.CurrentBlock().NumberU64(); head != 0 {
if head := tester.chain.CurrentBlock().Number.Uint64(); head != 0 {
t.Errorf("fast sync pivot block #%d not rolled back", head)
}
}
@ -1484,7 +1484,7 @@ func testBeaconSync(t *testing.T, protocol uint, mode SyncMode) {
select {
case <-success:
// Ok, downloader fully cancelled after sync cycle
if bs := int(tester.chain.CurrentBlock().NumberU64()) + 1; bs != len(chain.blocks) {
if bs := int(tester.chain.CurrentBlock().Number.Uint64()) + 1; bs != len(chain.blocks) {
t.Fatalf("synchronised blocks mismatch: have %v, want %v", bs, len(chain.blocks))
}
case <-time.NewTimer(time.Second * 3).C:

View File

@ -49,10 +49,10 @@ func (b *testBackend) HeaderByNumber(ctx context.Context, number rpc.BlockNumber
number = 0
}
if number == rpc.FinalizedBlockNumber {
return b.chain.CurrentFinalizedBlock().Header(), nil
return b.chain.CurrentFinalBlock(), nil
}
if number == rpc.SafeBlockNumber {
return b.chain.CurrentSafeBlock().Header(), nil
return b.chain.CurrentSafeBlock(), nil
}
if number == rpc.LatestBlockNumber {
number = testHead
@ -75,10 +75,10 @@ func (b *testBackend) BlockByNumber(ctx context.Context, number rpc.BlockNumber)
number = 0
}
if number == rpc.FinalizedBlockNumber {
return b.chain.CurrentFinalizedBlock(), nil
number = rpc.BlockNumber(b.chain.CurrentFinalBlock().Number.Uint64())
}
if number == rpc.SafeBlockNumber {
return b.chain.CurrentSafeBlock(), nil
number = rpc.BlockNumber(b.chain.CurrentSafeBlock().Number.Uint64())
}
if number == rpc.LatestBlockNumber {
number = testHead
@ -169,8 +169,8 @@ func newTestBackend(t *testing.T, londonBlock *big.Int, pending bool) *testBacke
t.Fatalf("Failed to create local chain, %v", err)
}
chain.InsertChain(blocks)
chain.SetFinalized(chain.GetBlockByNumber(25))
chain.SetSafe(chain.GetBlockByNumber(25))
chain.SetFinalized(chain.GetBlockByNumber(25).Header())
chain.SetSafe(chain.GetBlockByNumber(25).Header())
return &testBackend{chain: chain, pending: pending}
}

View File

@ -152,13 +152,13 @@ func newHandler(config *handlerConfig) (*handler, error) {
// * the last snap sync is not finished while user specifies a full sync this
// time. But we don't have any recent state for full sync.
// In these cases however it's safe to reenable snap sync.
fullBlock, fastBlock := h.chain.CurrentBlock(), h.chain.CurrentFastBlock()
if fullBlock.NumberU64() == 0 && fastBlock.NumberU64() > 0 {
fullBlock, snapBlock := h.chain.CurrentBlock(), h.chain.CurrentSnapBlock()
if fullBlock.Number.Uint64() == 0 && snapBlock.Number.Uint64() > 0 {
h.snapSync = uint32(1)
log.Warn("Switch sync mode from full sync to snap sync")
}
} else {
if h.chain.CurrentBlock().NumberU64() > 0 {
if h.chain.CurrentBlock().Number.Uint64() > 0 {
// Print warning log if database is not empty to run snap sync.
log.Warn("Switch sync mode from snap sync to full sync")
} else {
@ -183,10 +183,10 @@ func newHandler(config *handlerConfig) (*handler, error) {
// If we've successfully finished a sync cycle and passed any required
// checkpoint, enable accepting transactions from the network
head := h.chain.CurrentBlock()
if head.NumberU64() >= h.checkpointNumber {
if head.Number.Uint64() >= h.checkpointNumber {
// Checkpoint passed, sanity check the timestamp to have a fallback mechanism
// for non-checkpointed (number = 0) private networks.
if head.Time() >= uint64(time.Now().AddDate(0, -1, 0).Unix()) {
if head.Time >= uint64(time.Now().AddDate(0, -1, 0).Unix()) {
atomic.StoreUint32(&h.acceptTxs, 1)
}
}
@ -198,7 +198,7 @@ func newHandler(config *handlerConfig) (*handler, error) {
log.Info("Chain post-merge, sync via beacon client")
} else {
head := h.chain.CurrentBlock()
if td := h.chain.GetTd(head.Hash(), head.NumberU64()); td.Cmp(ttd) >= 0 {
if td := h.chain.GetTd(head.Hash(), head.Number.Uint64()); td.Cmp(ttd) >= 0 {
log.Info("Chain post-TTD, sync via beacon client")
} else {
log.Warn("Chain pre-merge, sync via PoW (ensure beacon client is ready)")
@ -227,7 +227,7 @@ func newHandler(config *handlerConfig) (*handler, error) {
return h.chain.Engine().VerifyHeader(h.chain, header, true)
}
heighter := func() uint64 {
return h.chain.CurrentBlock().NumberU64()
return h.chain.CurrentBlock().Number.Uint64()
}
inserter := func(blocks types.Blocks) (int, error) {
// All the block fetcher activities should be disabled
@ -250,7 +250,7 @@ func newHandler(config *handlerConfig) (*handler, error) {
// the propagated block if the head is too old. Unfortunately there is a corner
// case when starting new networks, where the genesis might be ancient (0 unix)
// which would prevent full nodes from accepting it.
if h.chain.CurrentBlock().NumberU64() < h.checkpointNumber {
if h.chain.CurrentBlock().Number.Uint64() < h.checkpointNumber {
log.Warn("Unsynced yet, discarded propagated block", "number", blocks[0].Number(), "hash", blocks[0].Hash())
return 0, nil
}

View File

@ -274,7 +274,7 @@ func testRecvTransactions(t *testing.T, protocol uint) {
var (
genesis = handler.chain.Genesis()
head = handler.chain.CurrentBlock()
td = handler.chain.GetTd(head.Hash(), head.NumberU64())
td = handler.chain.GetTd(head.Hash(), head.Number.Uint64())
)
if err := src.Handshake(1, td, head.Hash(), genesis.Hash(), forkid.NewIDWithChain(handler.chain), forkid.NewFilter(handler.chain)); err != nil {
t.Fatalf("failed to run protocol handshake")
@ -337,7 +337,7 @@ func testSendTransactions(t *testing.T, protocol uint) {
var (
genesis = handler.chain.Genesis()
head = handler.chain.CurrentBlock()
td = handler.chain.GetTd(head.Hash(), head.NumberU64())
td = handler.chain.GetTd(head.Hash(), head.Number.Uint64())
)
if err := sink.Handshake(1, td, head.Hash(), genesis.Hash(), forkid.NewIDWithChain(handler.chain), forkid.NewFilter(handler.chain)); err != nil {
t.Fatalf("failed to run protocol handshake")
@ -545,7 +545,7 @@ func testCheckpointChallenge(t *testing.T, syncmode downloader.SyncMode, checkpo
var (
genesis = handler.chain.Genesis()
head = handler.chain.CurrentBlock()
td = handler.chain.GetTd(head.Hash(), head.NumberU64())
td = handler.chain.GetTd(head.Hash(), head.Number.Uint64())
)
if err := remote.Handshake(1, td, head.Hash(), genesis.Hash(), forkid.NewIDWithChain(handler.chain), forkid.NewFilter(handler.chain)); err != nil {
t.Fatalf("failed to run protocol handshake")
@ -661,7 +661,8 @@ func testBroadcastBlock(t *testing.T, peers, bcasts int) {
}
// Initiate a block propagation across the peers
time.Sleep(100 * time.Millisecond)
source.handler.BroadcastBlock(source.chain.CurrentBlock(), true)
header := source.chain.CurrentBlock()
source.handler.BroadcastBlock(source.chain.GetBlock(header.Hash(), header.Number.Uint64()), true)
// Iterate through all the sinks and ensure the correct number got the block
done := make(chan struct{}, peers)
@ -734,18 +735,19 @@ func testBroadcastMalformedBlock(t *testing.T, protocol uint) {
// Create various combinations of malformed blocks
head := source.chain.CurrentBlock()
block := source.chain.GetBlock(head.Hash(), head.Number.Uint64())
malformedUncles := head.Header()
malformedUncles := head
malformedUncles.UncleHash[0]++
malformedTransactions := head.Header()
malformedTransactions := head
malformedTransactions.TxHash[0]++
malformedEverything := head.Header()
malformedEverything := head
malformedEverything.UncleHash[0]++
malformedEverything.TxHash[0]++
// Try to broadcast all malformations and ensure they all get discarded
for _, header := range []*types.Header{malformedUncles, malformedTransactions, malformedEverything} {
block := types.NewBlockWithHeader(header).WithBody(head.Transactions(), head.Uncles())
block := types.NewBlockWithHeader(header).WithBody(block.Transactions(), block.Uncles())
if err := src.SendNewBlock(block, big.NewInt(131136)); err != nil {
t.Fatalf("failed to broadcast block: %v", err)
}

View File

@ -137,12 +137,14 @@ type NodeInfo struct {
// nodeInfo retrieves some `eth` protocol metadata about the running host node.
func nodeInfo(chain *core.BlockChain, network uint64) *NodeInfo {
head := chain.CurrentBlock()
hash := head.Hash()
return &NodeInfo{
Network: network,
Difficulty: chain.GetTd(head.Hash(), head.NumberU64()),
Difficulty: chain.GetTd(hash, head.Number.Uint64()),
Genesis: chain.Genesis().Hash(),
Config: chain.Config(),
Head: head.Hash(),
Head: hash,
}
}

View File

@ -225,24 +225,24 @@ func testGetBlockHeaders(t *testing.T, protocol uint) {
[]common.Hash{backend.chain.GetBlockByNumber(0).Hash()},
},
{
&GetBlockHeadersPacket{Origin: HashOrNumber{Number: backend.chain.CurrentBlock().NumberU64()}, Amount: 1},
&GetBlockHeadersPacket{Origin: HashOrNumber{Number: backend.chain.CurrentBlock().Number.Uint64()}, Amount: 1},
[]common.Hash{backend.chain.CurrentBlock().Hash()},
},
{ // If the peer requests a bit into the future, we deliver what we have
&GetBlockHeadersPacket{Origin: HashOrNumber{Number: backend.chain.CurrentBlock().NumberU64()}, Amount: 10},
&GetBlockHeadersPacket{Origin: HashOrNumber{Number: backend.chain.CurrentBlock().Number.Uint64()}, Amount: 10},
[]common.Hash{backend.chain.CurrentBlock().Hash()},
},
// Ensure protocol limits are honored
{
&GetBlockHeadersPacket{Origin: HashOrNumber{Number: backend.chain.CurrentBlock().NumberU64() - 1}, Amount: limit + 10, Reverse: true},
getHashes(backend.chain.CurrentBlock().NumberU64(), limit),
&GetBlockHeadersPacket{Origin: HashOrNumber{Number: backend.chain.CurrentBlock().Number.Uint64() - 1}, Amount: limit + 10, Reverse: true},
getHashes(backend.chain.CurrentBlock().Number.Uint64(), limit),
},
// Check that requesting more than available is handled gracefully
{
&GetBlockHeadersPacket{Origin: HashOrNumber{Number: backend.chain.CurrentBlock().NumberU64() - 4}, Skip: 3, Amount: 3},
&GetBlockHeadersPacket{Origin: HashOrNumber{Number: backend.chain.CurrentBlock().Number.Uint64() - 4}, Skip: 3, Amount: 3},
[]common.Hash{
backend.chain.GetBlockByNumber(backend.chain.CurrentBlock().NumberU64() - 4).Hash(),
backend.chain.GetBlockByNumber(backend.chain.CurrentBlock().NumberU64()).Hash(),
backend.chain.GetBlockByNumber(backend.chain.CurrentBlock().Number.Uint64() - 4).Hash(),
backend.chain.GetBlockByNumber(backend.chain.CurrentBlock().Number.Uint64()).Hash(),
},
}, {
&GetBlockHeadersPacket{Origin: HashOrNumber{Number: 4}, Skip: 3, Amount: 3, Reverse: true},
@ -253,10 +253,10 @@ func testGetBlockHeaders(t *testing.T, protocol uint) {
},
// Check that requesting more than available is handled gracefully, even if mid skip
{
&GetBlockHeadersPacket{Origin: HashOrNumber{Number: backend.chain.CurrentBlock().NumberU64() - 4}, Skip: 2, Amount: 3},
&GetBlockHeadersPacket{Origin: HashOrNumber{Number: backend.chain.CurrentBlock().Number.Uint64() - 4}, Skip: 2, Amount: 3},
[]common.Hash{
backend.chain.GetBlockByNumber(backend.chain.CurrentBlock().NumberU64() - 4).Hash(),
backend.chain.GetBlockByNumber(backend.chain.CurrentBlock().NumberU64() - 1).Hash(),
backend.chain.GetBlockByNumber(backend.chain.CurrentBlock().Number.Uint64() - 4).Hash(),
backend.chain.GetBlockByNumber(backend.chain.CurrentBlock().Number.Uint64() - 1).Hash(),
},
}, {
&GetBlockHeadersPacket{Origin: HashOrNumber{Number: 4}, Skip: 2, Amount: 3, Reverse: true},
@ -293,7 +293,7 @@ func testGetBlockHeaders(t *testing.T, protocol uint) {
&GetBlockHeadersPacket{Origin: HashOrNumber{Hash: unknown}, Amount: 1},
[]common.Hash{},
}, {
&GetBlockHeadersPacket{Origin: HashOrNumber{Number: backend.chain.CurrentBlock().NumberU64() + 1}, Amount: 1},
&GetBlockHeadersPacket{Origin: HashOrNumber{Number: backend.chain.CurrentBlock().Number.Uint64() + 1}, Amount: 1},
[]common.Hash{},
},
}
@ -394,7 +394,7 @@ func testGetBlockBodies(t *testing.T, protocol uint) {
)
for j := 0; j < tt.random; j++ {
for {
num := rand.Int63n(int64(backend.chain.CurrentBlock().NumberU64()))
num := rand.Int63n(int64(backend.chain.CurrentBlock().Number.Uint64()))
if !seen[num] {
seen[num] = true
@ -529,7 +529,7 @@ func testGetNodeData(t *testing.T, protocol uint, drop bool) {
// Sanity check whether all state matches.
accounts := []common.Address{testAddr, acc1Addr, acc2Addr}
for i := uint64(0); i <= backend.chain.CurrentBlock().NumberU64(); i++ {
for i := uint64(0); i <= backend.chain.CurrentBlock().Number.Uint64(); i++ {
root := backend.chain.GetBlockByNumber(i).Root()
reconstructed, _ := state.New(root, state.NewDatabase(reconstructDB), nil)
for j, acc := range accounts {
@ -602,7 +602,7 @@ func testGetBlockReceipts(t *testing.T, protocol uint) {
hashes []common.Hash
receipts [][]*types.Receipt
)
for i := uint64(0); i <= backend.chain.CurrentBlock().NumberU64(); i++ {
for i := uint64(0); i <= backend.chain.CurrentBlock().Number.Uint64(); i++ {
block := backend.chain.GetBlockByNumber(i)
hashes = append(hashes, block.Hash())

View File

@ -39,7 +39,7 @@ func testHandshake(t *testing.T, protocol uint) {
var (
genesis = backend.chain.Genesis()
head = backend.chain.CurrentBlock()
td = backend.chain.GetTd(head.Hash(), head.NumberU64())
td = backend.chain.GetTd(head.Hash(), head.Number.Uint64())
forkID = forkid.NewID(backend.chain.Config(), backend.chain.Genesis().Hash(), backend.chain.CurrentHeader().Number.Uint64(), backend.chain.CurrentHeader().Time)
)
tests := []struct {

View File

@ -206,22 +206,22 @@ func peerToSyncOp(mode downloader.SyncMode, p *eth.Peer) *chainSyncOp {
func (cs *chainSyncer) modeAndLocalHead() (downloader.SyncMode, *big.Int) {
// If we're in snap sync mode, return that directly
if atomic.LoadUint32(&cs.handler.snapSync) == 1 {
block := cs.handler.chain.CurrentFastBlock()
td := cs.handler.chain.GetTd(block.Hash(), block.NumberU64())
block := cs.handler.chain.CurrentSnapBlock()
td := cs.handler.chain.GetTd(block.Hash(), block.Number.Uint64())
return downloader.SnapSync, td
}
// We are probably in full sync, but we might have rewound to before the
// snap sync pivot, check if we should reenable
if pivot := rawdb.ReadLastPivotNumber(cs.handler.database); pivot != nil {
if head := cs.handler.chain.CurrentBlock(); head.NumberU64() < *pivot {
block := cs.handler.chain.CurrentFastBlock()
td := cs.handler.chain.GetTd(block.Hash(), block.NumberU64())
if head := cs.handler.chain.CurrentBlock(); head.Number.Uint64() < *pivot {
block := cs.handler.chain.CurrentSnapBlock()
td := cs.handler.chain.GetTd(block.Hash(), block.Number.Uint64())
return downloader.SnapSync, td
}
}
// Nope, we're really full syncing
head := cs.handler.chain.CurrentBlock()
td := cs.handler.chain.GetTd(head.Hash(), head.NumberU64())
td := cs.handler.chain.GetTd(head.Hash(), head.Number.Uint64())
return downloader.FullSync, td
}
@ -263,21 +263,23 @@ func (h *handler) doSync(op *chainSyncOp) error {
// If we've successfully finished a sync cycle and passed any required checkpoint,
// enable accepting transactions from the network.
head := h.chain.CurrentBlock()
if head.NumberU64() >= h.checkpointNumber {
if head.Number.Uint64() >= h.checkpointNumber {
// Checkpoint passed, sanity check the timestamp to have a fallback mechanism
// for non-checkpointed (number = 0) private networks.
if head.Time() >= uint64(time.Now().AddDate(0, -1, 0).Unix()) {
if head.Time >= uint64(time.Now().AddDate(0, -1, 0).Unix()) {
atomic.StoreUint32(&h.acceptTxs, 1)
}
}
if head.NumberU64() > 0 {
if head.Number.Uint64() > 0 {
// We've completed a sync cycle, notify all peers of new state. This path is
// essential in star-topology networks where a gateway node needs to notify
// all its out-of-date peers of the availability of a new block. This failure
// scenario will most often crop up in private and hackathon networks with
// degenerate connectivity, but it should be healthy for the mainnet too to
// more reliably update peers or the local TD state.
h.BroadcastBlock(head, false)
if block := h.chain.GetBlock(head.Hash(), head.Number.Uint64()); block != nil {
h.BroadcastBlock(block, false)
}
}
return nil
}

View File

@ -109,7 +109,7 @@ func (b *testBackend) BlockByHash(ctx context.Context, hash common.Hash) (*types
func (b *testBackend) BlockByNumber(ctx context.Context, number rpc.BlockNumber) (*types.Block, error) {
if number == rpc.PendingBlockNumber || number == rpc.LatestBlockNumber {
return b.chain.CurrentBlock(), nil
return b.chain.GetBlockByNumber(b.chain.CurrentBlock().Number.Uint64()), nil
}
return b.chain.GetBlockByNumber(uint64(number)), nil
}

View File

@ -1219,7 +1219,7 @@ func (r *Resolver) Blocks(ctx context.Context, args struct {
if args.To != nil {
to = rpc.BlockNumber(*args.To)
} else {
to = rpc.BlockNumber(r.backend.CurrentBlock().Number().Int64())
to = rpc.BlockNumber(r.backend.CurrentBlock().Number.Int64())
}
if to < from {
return []*Block{}, nil

View File

@ -1704,7 +1704,7 @@ func SubmitTransaction(ctx context.Context, b Backend, tx *types.Transaction) (c
return common.Hash{}, err
}
// Print a log with full tx details for manual investigations and interventions
signer := types.MakeSigner(b.ChainConfig(), b.CurrentBlock().Number())
signer := types.MakeSigner(b.ChainConfig(), b.CurrentBlock().Number)
from, err := types.Sender(signer, tx)
if err != nil {
return common.Hash{}, err

View File

@ -59,7 +59,7 @@ type Backend interface {
HeaderByHash(ctx context.Context, hash common.Hash) (*types.Header, error)
HeaderByNumberOrHash(ctx context.Context, blockNrOrHash rpc.BlockNumberOrHash) (*types.Header, error)
CurrentHeader() *types.Header
CurrentBlock() *types.Block
CurrentBlock() *types.Header
BlockByNumber(ctx context.Context, number rpc.BlockNumber) (*types.Block, error)
BlockByHash(ctx context.Context, hash common.Hash) (*types.Block, error)
BlockByNumberOrHash(ctx context.Context, blockNrOrHash rpc.BlockNumberOrHash) (*types.Block, error)

View File

@ -278,7 +278,7 @@ func (b *backendMock) HeaderByHash(ctx context.Context, hash common.Hash) (*type
func (b *backendMock) HeaderByNumberOrHash(ctx context.Context, blockNrOrHash rpc.BlockNumberOrHash) (*types.Header, error) {
return nil, nil
}
func (b *backendMock) CurrentBlock() *types.Block { return nil }
func (b *backendMock) CurrentBlock() *types.Header { return nil }
func (b *backendMock) BlockByNumber(ctx context.Context, number rpc.BlockNumber) (*types.Block, error) {
return nil, nil
}

View File

@ -52,8 +52,8 @@ func (b *LesApiBackend) ChainConfig() *params.ChainConfig {
return b.eth.chainConfig
}
func (b *LesApiBackend) CurrentBlock() *types.Block {
return types.NewBlockWithHeader(b.eth.BlockChain().CurrentHeader())
func (b *LesApiBackend) CurrentBlock() *types.Header {
return b.eth.BlockChain().CurrentHeader()
}
func (b *LesApiBackend) SetHead(number uint64) {

View File

@ -121,20 +121,20 @@ func testGetBlockHeaders(t *testing.T, protocol int) {
&GetBlockHeadersData{Origin: hashOrNumber{Number: 0}, Amount: 1},
[]common.Hash{bc.GetBlockByNumber(0).Hash()},
}, {
&GetBlockHeadersData{Origin: hashOrNumber{Number: bc.CurrentBlock().NumberU64()}, Amount: 1},
&GetBlockHeadersData{Origin: hashOrNumber{Number: bc.CurrentBlock().Number.Uint64()}, Amount: 1},
[]common.Hash{bc.CurrentBlock().Hash()},
},
// Ensure protocol limits are honored
//{
// &GetBlockHeadersData{Origin: hashOrNumber{Number: bc.CurrentBlock().NumberU64() - 1}, Amount: limit + 10, Reverse: true},
// &GetBlockHeadersData{Origin: hashOrNumber{Number: bc.CurrentBlock().Number.Uint64()() - 1}, Amount: limit + 10, Reverse: true},
// []common.Hash{},
//},
// Check that requesting more than available is handled gracefully
{
&GetBlockHeadersData{Origin: hashOrNumber{Number: bc.CurrentBlock().NumberU64() - 4}, Skip: 3, Amount: 3},
&GetBlockHeadersData{Origin: hashOrNumber{Number: bc.CurrentBlock().Number.Uint64() - 4}, Skip: 3, Amount: 3},
[]common.Hash{
bc.GetBlockByNumber(bc.CurrentBlock().NumberU64() - 4).Hash(),
bc.GetBlockByNumber(bc.CurrentBlock().NumberU64()).Hash(),
bc.GetBlockByNumber(bc.CurrentBlock().Number.Uint64() - 4).Hash(),
bc.GetBlockByNumber(bc.CurrentBlock().Number.Uint64()).Hash(),
},
}, {
&GetBlockHeadersData{Origin: hashOrNumber{Number: 4}, Skip: 3, Amount: 3, Reverse: true},
@ -145,10 +145,10 @@ func testGetBlockHeaders(t *testing.T, protocol int) {
},
// Check that requesting more than available is handled gracefully, even if mid skip
{
&GetBlockHeadersData{Origin: hashOrNumber{Number: bc.CurrentBlock().NumberU64() - 4}, Skip: 2, Amount: 3},
&GetBlockHeadersData{Origin: hashOrNumber{Number: bc.CurrentBlock().Number.Uint64() - 4}, Skip: 2, Amount: 3},
[]common.Hash{
bc.GetBlockByNumber(bc.CurrentBlock().NumberU64() - 4).Hash(),
bc.GetBlockByNumber(bc.CurrentBlock().NumberU64() - 1).Hash(),
bc.GetBlockByNumber(bc.CurrentBlock().Number.Uint64() - 4).Hash(),
bc.GetBlockByNumber(bc.CurrentBlock().Number.Uint64() - 1).Hash(),
},
}, {
&GetBlockHeadersData{Origin: hashOrNumber{Number: 4}, Skip: 2, Amount: 3, Reverse: true},
@ -162,7 +162,7 @@ func testGetBlockHeaders(t *testing.T, protocol int) {
&GetBlockHeadersData{Origin: hashOrNumber{Hash: unknown}, Amount: 1},
[]common.Hash{},
}, {
&GetBlockHeadersData{Origin: hashOrNumber{Number: bc.CurrentBlock().NumberU64() + 1}, Amount: 1},
&GetBlockHeadersData{Origin: hashOrNumber{Number: bc.CurrentBlock().Number.Uint64() + 1}, Amount: 1},
[]common.Hash{},
},
}
@ -240,7 +240,7 @@ func testGetBlockBodies(t *testing.T, protocol int) {
for j := 0; j < tt.random; j++ {
for {
num := rand.Int63n(int64(bc.CurrentBlock().NumberU64()))
num := rand.Int63n(int64(bc.CurrentBlock().Number.Uint64()))
if !seen[num] {
seen[num] = true
@ -292,7 +292,7 @@ func testGetCode(t *testing.T, protocol int) {
var codereqs []*CodeReq
var codes [][]byte
for i := uint64(0); i <= bc.CurrentBlock().NumberU64(); i++ {
for i := uint64(0); i <= bc.CurrentBlock().Number.Uint64(); i++ {
header := bc.GetHeaderByNumber(i)
req := &CodeReq{
BHash: header.Hash(),
@ -367,7 +367,7 @@ func testGetReceipt(t *testing.T, protocol int) {
// Collect the hashes to request, and the response to expect
var receipts []types.Receipts
var hashes []common.Hash
for i := uint64(0); i <= bc.CurrentBlock().NumberU64(); i++ {
for i := uint64(0); i <= bc.CurrentBlock().Number.Uint64(); i++ {
block := bc.GetBlockByNumber(i)
hashes = append(hashes, block.Hash())
@ -404,7 +404,7 @@ func testGetProofs(t *testing.T, protocol int) {
proofsV2 := light.NewNodeSet()
accounts := []common.Address{bankAddr, userAddr1, userAddr2, signerAddr, {}}
for i := uint64(0); i <= bc.CurrentBlock().NumberU64(); i++ {
for i := uint64(0); i <= bc.CurrentBlock().Number.Uint64(); i++ {
header := bc.GetHeaderByNumber(i)
trie, _ := trie.New(trie.StateTrieID(header.Root), trie.NewDatabase(server.db))

View File

@ -281,7 +281,7 @@ func testGetTxStatusFromUnindexedPeers(t *testing.T, protocol int) {
blockHashes = make(map[common.Hash]common.Hash) // Transaction hash to block hash mappings
intraIndex = make(map[common.Hash]uint64) // Transaction intra-index in block
)
for number := uint64(1); number < server.backend.Blockchain().CurrentBlock().NumberU64(); number++ {
for number := uint64(1); number < server.backend.Blockchain().CurrentBlock().Number.Uint64(); number++ {
block := server.backend.Blockchain().GetBlockByNumber(number)
if block == nil {
t.Fatalf("Failed to retrieve block %d", number)

View File

@ -19,6 +19,7 @@ package miner
import (
"errors"
"math/big"
"testing"
"time"
@ -65,14 +66,15 @@ type testBlockChain struct {
chainHeadFeed *event.Feed
}
func (bc *testBlockChain) CurrentBlock() *types.Block {
return types.NewBlock(&types.Header{
func (bc *testBlockChain) CurrentBlock() *types.Header {
return &types.Header{
Number: new(big.Int),
GasLimit: bc.gasLimit,
}, nil, nil, nil, trie.NewStackTrie(nil))
}
}
func (bc *testBlockChain) GetBlock(hash common.Hash, number uint64) *types.Block {
return bc.CurrentBlock()
return types.NewBlock(bc.CurrentBlock(), nil, nil, nil, trie.NewStackTrie(nil))
}
func (bc *testBlockChain) StateAt(common.Hash) (*state.StateDB, error) {

View File

@ -494,7 +494,7 @@ func (w *worker) newWorkLoop(recommit time.Duration) {
for {
select {
case <-w.startCh:
clearPending(w.chain.CurrentBlock().NumberU64())
clearPending(w.chain.CurrentBlock().Number.Uint64())
timestamp = time.Now().Unix()
commit(false, commitInterruptNewHead)
@ -607,12 +607,12 @@ func (w *worker) mainLoop() {
case <-cleanTicker.C:
chainHead := w.chain.CurrentBlock()
for hash, uncle := range w.localUncles {
if uncle.NumberU64()+staleThreshold <= chainHead.NumberU64() {
if uncle.NumberU64()+staleThreshold <= chainHead.Number.Uint64() {
delete(w.localUncles, hash)
}
}
for hash, uncle := range w.remoteUncles {
if uncle.NumberU64()+staleThreshold <= chainHead.NumberU64() {
if uncle.NumberU64()+staleThreshold <= chainHead.Number.Uint64() {
delete(w.remoteUncles, hash)
}
}
@ -790,10 +790,10 @@ func (w *worker) resultLoop() {
}
// makeEnv creates a new environment for the sealing block.
func (w *worker) makeEnv(parent *types.Block, header *types.Header, coinbase common.Address) (*environment, error) {
func (w *worker) makeEnv(parent *types.Header, header *types.Header, coinbase common.Address) (*environment, error) {
// Retrieve the parent state to execute on top and start a prefetcher for
// the miner to speed block sealing up a bit.
state, err := w.chain.StateAt(parent.Root())
state, err := w.chain.StateAt(parent.Root)
if err != nil {
return nil, err
}
@ -988,25 +988,26 @@ func (w *worker) prepareWork(genParams *generateParams) (*environment, error) {
// Find the parent block for sealing task
parent := w.chain.CurrentBlock()
if genParams.parentHash != (common.Hash{}) {
parent = w.chain.GetBlockByHash(genParams.parentHash)
}
if parent == nil {
return nil, fmt.Errorf("missing parent")
block := w.chain.GetBlockByHash(genParams.parentHash)
if block == nil {
return nil, fmt.Errorf("missing parent")
}
parent = block.Header()
}
// Sanity check the timestamp correctness, recap the timestamp
// to parent+1 if the mutation is allowed.
timestamp := genParams.timestamp
if parent.Time() >= timestamp {
if parent.Time >= timestamp {
if genParams.forceTime {
return nil, fmt.Errorf("invalid timestamp, parent %d given %d", parent.Time(), timestamp)
return nil, fmt.Errorf("invalid timestamp, parent %d given %d", parent.Time, timestamp)
}
timestamp = parent.Time() + 1
timestamp = parent.Time + 1
}
// Construct the sealing block header.
header := &types.Header{
ParentHash: parent.Hash(),
Number: new(big.Int).Add(parent.Number(), common.Big1),
GasLimit: core.CalcGasLimit(parent.GasLimit(), w.config.GasCeil),
Number: new(big.Int).Add(parent.Number, common.Big1),
GasLimit: core.CalcGasLimit(parent.GasLimit, w.config.GasCeil),
Time: timestamp,
Coinbase: genParams.coinbase,
}
@ -1020,9 +1021,9 @@ func (w *worker) prepareWork(genParams *generateParams) (*environment, error) {
}
// Set baseFee and GasLimit if we are on an EIP-1559 chain
if w.chainConfig.IsLondon(header.Number) {
header.BaseFee = misc.CalcBaseFee(w.chainConfig, parent.Header())
if !w.chainConfig.IsLondon(parent.Number()) {
parentGasLimit := parent.GasLimit() * w.chainConfig.ElasticityMultiplier()
header.BaseFee = misc.CalcBaseFee(w.chainConfig, parent)
if !w.chainConfig.IsLondon(parent.Number) {
parentGasLimit := parent.GasLimit * w.chainConfig.ElasticityMultiplier()
header.GasLimit = core.CalcGasLimit(parentGasLimit, w.config.GasCeil)
}
}

View File

@ -147,7 +147,7 @@ func newTestWorkerBackend(t *testing.T, chainConfig *params.ChainConfig, engine
if _, err := chain.InsertChain(blocks); err != nil {
t.Fatalf("failed to insert origin chain: %v", err)
}
parent := chain.GetBlockByHash(chain.CurrentBlock().ParentHash())
parent := chain.GetBlockByHash(chain.CurrentBlock().ParentHash)
blocks, _ = core.GenerateChain(chainConfig, parent, engine, genDb, 1, func(i int, gen *core.BlockGen) {
gen.SetCoinbase(testUserAddress)
})
@ -176,10 +176,10 @@ func (b *testWorkerBackend) StateAtBlock(block *types.Block, reexec uint64, base
func (b *testWorkerBackend) newRandomUncle() *types.Block {
var parent *types.Block
cur := b.chain.CurrentBlock()
if cur.NumberU64() == 0 {
if cur.Number.Uint64() == 0 {
parent = b.chain.Genesis()
} else {
parent = b.chain.GetBlockByHash(b.chain.CurrentBlock().ParentHash())
parent = b.chain.GetBlockByHash(b.chain.CurrentBlock().ParentHash)
}
blocks, _ := core.GenerateChain(b.chain.Config(), parent, b.chain.Engine(), b.db, 1, func(i int, gen *core.BlockGen) {
var addr = make([]byte, common.AddressLength)
@ -607,21 +607,21 @@ func testGetSealingWork(t *testing.T, chainConfig *params.ChainConfig, engine co
b.chain.CurrentBlock().Hash(),
common.HexToAddress("0xdeadbeef"),
common.HexToHash("0xcafebabe"),
b.chain.CurrentBlock().NumberU64() + 1,
b.chain.CurrentBlock().Number.Uint64() + 1,
false,
},
{
b.chain.CurrentBlock().Hash(),
common.Address{},
common.HexToHash("0xcafebabe"),
b.chain.CurrentBlock().NumberU64() + 1,
b.chain.CurrentBlock().Number.Uint64() + 1,
false,
},
{
b.chain.CurrentBlock().Hash(),
common.Address{},
common.Hash{},
b.chain.CurrentBlock().NumberU64() + 1,
b.chain.CurrentBlock().Number.Uint64() + 1,
false,
},
{

View File

@ -154,7 +154,7 @@ func (t *BlockTest) Run(snapshotter bool) error {
}
// Cross-check the snapshot-to-hash against the trie hash
if snapshotter {
if err := chain.Snapshots().Verify(chain.CurrentBlock().Root()); err != nil {
if err := chain.Snapshots().Verify(chain.CurrentBlock().Root); err != nil {
return err
}
}
@ -317,8 +317,8 @@ func (t *BlockTest) validateImportedHeaders(cm *core.BlockChain, validBlocks []b
// block-by-block, so we can only validate imported headers after
// all blocks have been processed by BlockChain, as they may not
// be part of the longest chain until last block is imported.
for b := cm.CurrentBlock(); b != nil && b.NumberU64() != 0; b = cm.GetBlockByHash(b.Header().ParentHash) {
if err := validateHeader(bmap[b.Hash()].BlockHeader, b.Header()); err != nil {
for b := cm.CurrentBlock(); b != nil && b.Number.Uint64() != 0; b = cm.GetBlockByHash(b.ParentHash).Header() {
if err := validateHeader(bmap[b.Hash()].BlockHeader, b); err != nil {
return fmt.Errorf("imported block header validation failed: %v", err)
}
}