mirror of
https://gitlab.com/pulsechaincom/erigon-pulse.git
synced 2024-12-22 03:30:37 +00:00
use BlockReader.CanonicalHash method instead of rawdb (#7604)
This commit is contained in:
parent
af83845f7e
commit
3cba3908dd
@ -275,19 +275,12 @@ func (b *SimulatedBackend) TransactionReceipt(ctx context.Context, txHash libcom
|
||||
if blockNumber == nil {
|
||||
return nil, nil
|
||||
}
|
||||
blockHash, err := rawdb.ReadCanonicalHash(tx, *blockNumber)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if blockHash == (libcommon.Hash{}) {
|
||||
return nil, nil
|
||||
}
|
||||
block, senders, err := b.BlockReader().BlockWithSenders(b.m.Ctx, tx, blockHash, *blockNumber)
|
||||
block, err := b.BlockReader().BlockByNumber(b.m.Ctx, tx, *blockNumber)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
// Read all the receipts from the block and return the one with the matching hash
|
||||
receipts := rawdb.ReadReceipts(tx, block, senders)
|
||||
receipts := rawdb.ReadReceipts(tx, block, nil)
|
||||
for _, receipt := range receipts {
|
||||
if receipt.TxHash == txHash {
|
||||
return receipt, nil
|
||||
@ -304,7 +297,7 @@ func (b *SimulatedBackend) TransactionByHash(ctx context.Context, txHash libcomm
|
||||
b.mu.Lock()
|
||||
defer b.mu.Unlock()
|
||||
|
||||
tx, err := b.m.DB.BeginRo(context.Background())
|
||||
tx, err := b.m.DB.BeginRo(ctx)
|
||||
if err != nil {
|
||||
return nil, false, err
|
||||
}
|
||||
@ -314,18 +307,18 @@ func (b *SimulatedBackend) TransactionByHash(ctx context.Context, txHash libcomm
|
||||
if txn != nil {
|
||||
return txn, true, nil
|
||||
}
|
||||
blockNumber, err := rawdb.ReadTxLookupEntry(tx, txHash)
|
||||
blockNumber, ok, err := b.BlockReader().TxnLookup(ctx, tx, txHash)
|
||||
if err != nil {
|
||||
return nil, false, err
|
||||
}
|
||||
if blockNumber == nil {
|
||||
if !ok {
|
||||
return nil, false, ethereum.NotFound
|
||||
}
|
||||
blockHash, err := rawdb.ReadCanonicalHash(tx, *blockNumber)
|
||||
blockHash, err := b.BlockReader().CanonicalHash(ctx, tx, blockNumber)
|
||||
if err != nil {
|
||||
return nil, false, err
|
||||
}
|
||||
body, err := b.BlockReader().BodyWithTransactions(ctx, tx, blockHash, *blockNumber)
|
||||
body, err := b.BlockReader().BodyWithTransactions(ctx, tx, blockHash, blockNumber)
|
||||
if err != nil {
|
||||
return nil, false, err
|
||||
}
|
||||
@ -348,7 +341,7 @@ func (b *SimulatedBackend) BlockByHash(ctx context.Context, hash libcommon.Hash)
|
||||
if hash == b.pendingBlock.Hash() {
|
||||
return b.pendingBlock, nil
|
||||
}
|
||||
tx, err := b.m.DB.BeginRo(context.Background())
|
||||
tx, err := b.m.DB.BeginRo(ctx)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@ -392,11 +385,7 @@ func (b *SimulatedBackend) blockByNumberNoLock(ctx context.Context, number *big.
|
||||
}
|
||||
defer tx.Rollback()
|
||||
|
||||
hash, err := rawdb.ReadCanonicalHash(tx, number.Uint64())
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
block, _, err := b.BlockReader().BlockWithSenders(ctx, tx, hash, number.Uint64())
|
||||
block, err := b.BlockReader().BlockByNumber(ctx, tx, number.Uint64())
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@ -450,11 +439,7 @@ func (b *SimulatedBackend) HeaderByNumber(ctx context.Context, number *big.Int)
|
||||
if number == nil || number.Cmp(b.prependBlock.Number()) == 0 {
|
||||
return b.prependBlock.Header(), nil
|
||||
}
|
||||
hash, err := rawdb.ReadCanonicalHash(tx, number.Uint64())
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
header, err := b.BlockReader().Header(ctx, tx, hash, number.Uint64())
|
||||
header, err := b.BlockReader().HeaderByNumber(ctx, tx, number.Uint64())
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@ -867,102 +852,3 @@ func (m callMsg) IsFree() bool { return false }
|
||||
func (m callMsg) DataGas() uint64 { return params.DataGasPerBlob * uint64(len(m.CallMsg.DataHashes)) }
|
||||
func (m callMsg) MaxFeePerDataGas() *uint256.Int { return m.CallMsg.MaxFeePerDataGas }
|
||||
func (m callMsg) DataHashes() []libcommon.Hash { return m.CallMsg.DataHashes }
|
||||
|
||||
/*
|
||||
// filterBackend implements filters.Backend to support filtering for logs without
|
||||
// taking bloom-bits acceleration structures into account.
|
||||
type filterBackend struct {
|
||||
db kv.RwDB
|
||||
b *SimulatedBackend
|
||||
}
|
||||
|
||||
func (fb *filterBackend) HeaderByNumber(ctx context.Context, block rpc.BlockNumber) (*types.Header, error) {
|
||||
if block == rpc.LatestBlockNumber {
|
||||
return fb.b.HeaderByNumber(ctx, nil)
|
||||
}
|
||||
return fb.b.HeaderByNumber(ctx, big.NewInt(block.Int64()))
|
||||
}
|
||||
|
||||
func (fb *filterBackend) HeaderByHash(ctx context.Context, hash libcommon.Hash) (*types.Header, error) {
|
||||
return fb.b.HeaderByHash(ctx, hash)
|
||||
}
|
||||
|
||||
func (fb *filterBackend) GetReceipts(ctx context.Context, hash libcommon.Hash) (types.Receipts, error) {
|
||||
tx, err := fb.db.BeginRo(context.Background())
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
defer tx.Rollback()
|
||||
number := rawdb.ReadHeaderNumber(tx, hash)
|
||||
if number == nil {
|
||||
return nil, err
|
||||
}
|
||||
canonicalHash, err := rawdb.ReadCanonicalHash(tx, *number)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("requested non-canonical hash %x. canonical=%x", hash, canonicalHash)
|
||||
}
|
||||
b, senders, err := rawdb.ReadBlockWithSenders(tx, hash, *number)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return rawdb.ReadReceipts(tx, b, senders), nil
|
||||
}
|
||||
|
||||
func (fb *filterBackend) GetLogs(ctx context.Context, hash libcommon.Hash) ([][]*types.Log, error) {
|
||||
tx, err := fb.db.BeginRo(context.Background())
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
defer tx.Rollback()
|
||||
number := rawdb.ReadHeaderNumber(tx, hash)
|
||||
if number == nil {
|
||||
return nil, err
|
||||
}
|
||||
canonicalHash, err := rawdb.ReadCanonicalHash(tx, *number)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("requested non-canonical hash %x. canonical=%x", hash, canonicalHash)
|
||||
}
|
||||
b, senders, err := rawdb.ReadBlockWithSenders(tx, hash, *number)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
receipts := rawdb.ReadReceipts(tx, b, senders)
|
||||
if receipts == nil {
|
||||
return nil, nil
|
||||
}
|
||||
logs := make([][]*types.Log, len(receipts))
|
||||
for i, receipt := range receipts {
|
||||
logs[i] = receipt.Logs
|
||||
}
|
||||
return logs, nil
|
||||
}
|
||||
|
||||
func (fb *filterBackend) SubscribeNewTxsEvent(ch chan<- core.NewTxsEvent) event.Subscription {
|
||||
return nullSubscription()
|
||||
}
|
||||
|
||||
func (fb *filterBackend) SubscribeChainEvent(ch chan<- core.ChainEvent) event.Subscription {
|
||||
return fb.b.chainFeed.Subscribe(ch)
|
||||
}
|
||||
|
||||
func (fb *filterBackend) SubscribeRemovedLogsEvent(ch chan<- core.RemovedLogsEvent) event.Subscription {
|
||||
return fb.b.rmLogsFeed.Subscribe(ch)
|
||||
}
|
||||
|
||||
func (fb *filterBackend) SubscribeLogsEvent(ch chan<- []*types.Log) event.Subscription {
|
||||
return fb.b.logsFeed.Subscribe(ch)
|
||||
}
|
||||
|
||||
func (fb *filterBackend) SubscribePendingLogsEvent(ch chan<- []*types.Log) event.Subscription {
|
||||
return nullSubscription()
|
||||
}
|
||||
|
||||
func (fb *filterBackend) BloomStatus() (uint64, uint64) { return 4096, 0 }
|
||||
|
||||
func nullSubscription() event.Subscription {
|
||||
return event.NewSubscription(func(quit <-chan struct{}) error {
|
||||
<-quit
|
||||
return nil
|
||||
})
|
||||
}
|
||||
*/
|
||||
|
@ -94,14 +94,14 @@ func dbSlice(chaindata string, bucket string, prefix []byte) {
|
||||
func testBlockHashes(chaindata string, block int, stateRoot libcommon.Hash) {
|
||||
ethDb := mdbx.MustOpen(chaindata)
|
||||
defer ethDb.Close()
|
||||
br, _ := blocksIO(ethDb)
|
||||
tool.Check(ethDb.View(context.Background(), func(tx kv.Tx) error {
|
||||
blocksToSearch := 10000000
|
||||
for i := uint64(block); i < uint64(block+blocksToSearch); i++ {
|
||||
hash, err := rawdb.ReadCanonicalHash(tx, i)
|
||||
header, err := br.HeaderByNumber(context.Background(), tx, i)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
header := rawdb.ReadHeader(tx, hash, i)
|
||||
if header.Root == stateRoot || stateRoot == (libcommon.Hash{}) {
|
||||
fmt.Printf("\n===============\nCanonical hash for %d: %x\n", i, hash)
|
||||
fmt.Printf("Header.Root: %x\n", header.Root)
|
||||
@ -153,11 +153,7 @@ func printTxHashes(chaindata string, block uint64) error {
|
||||
br, _ := blocksIO(db)
|
||||
if err := db.View(context.Background(), func(tx kv.Tx) error {
|
||||
for b := block; b < block+1; b++ {
|
||||
hash, e := rawdb.ReadCanonicalHash(tx, b)
|
||||
if e != nil {
|
||||
return e
|
||||
}
|
||||
block, _, _ := br.BlockWithSenders(context.Background(), tx, hash, b)
|
||||
block, _ := br.BlockByNumber(context.Background(), tx, b)
|
||||
if block == nil {
|
||||
break
|
||||
}
|
||||
@ -469,6 +465,7 @@ func getBlockTotal(tx kv.Tx, blockFrom uint64, blockTotalOrOffset int64) uint64
|
||||
func extractHashes(chaindata string, blockStep uint64, blockTotalOrOffset int64, name string) error {
|
||||
db := mdbx.MustOpen(chaindata)
|
||||
defer db.Close()
|
||||
br, _ := blocksIO(db)
|
||||
|
||||
f, err := os.Create(fmt.Sprintf("preverified_hashes_%s.go", name))
|
||||
if err != nil {
|
||||
@ -487,7 +484,7 @@ func extractHashes(chaindata string, blockStep uint64, blockTotalOrOffset int64,
|
||||
blockTotal := getBlockTotal(tx, b, blockTotalOrOffset)
|
||||
// Note: blockTotal used here as block number rather than block count
|
||||
for b <= blockTotal {
|
||||
hash, err := rawdb.ReadCanonicalHash(tx, b)
|
||||
hash, err := br.CanonicalHash(context.Background(), tx, b)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -582,15 +579,15 @@ func extractBodies(datadir string) error {
|
||||
return nil
|
||||
})
|
||||
*/
|
||||
br := snapshotsync.NewBlockReader(snaps, false)
|
||||
lastTxnID, _, err := br.LastTxNumInSnapshot(snaps.BlocksAvailable())
|
||||
db := mdbx.MustOpen(filepath.Join(datadir, "chaindata"))
|
||||
defer db.Close()
|
||||
br, _ := blocksIO(db)
|
||||
lastTxnID, _, err := br.(*snapshotsync.BlockReader).LastTxNumInSnapshot(snaps.BlocksAvailable())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
fmt.Printf("txTxnID = %d\n", lastTxnID)
|
||||
|
||||
db := mdbx.MustOpen(filepath.Join(datadir, "chaindata"))
|
||||
defer db.Close()
|
||||
tx, err := db.BeginRo(context.Background())
|
||||
if err != nil {
|
||||
return err
|
||||
@ -610,7 +607,7 @@ func extractBodies(datadir string) error {
|
||||
blockNumber := binary.BigEndian.Uint64(k[:8])
|
||||
blockHash := libcommon.BytesToHash(k[8:])
|
||||
var hash libcommon.Hash
|
||||
if hash, err = rawdb.ReadCanonicalHash(tx, blockNumber); err != nil {
|
||||
if hash, err = br.CanonicalHash(context.Background(), tx, blockNumber); err != nil {
|
||||
return err
|
||||
}
|
||||
_, baseTxId, txAmount := rawdb.ReadBody(tx, blockHash, blockNumber)
|
||||
@ -1049,6 +1046,8 @@ func scanReceipts2(chaindata string) error {
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
br, _ := blocksIO(dbdb)
|
||||
|
||||
defer tx.Rollback()
|
||||
blockNum, err := historyv2.AvailableFrom(tx)
|
||||
if err != nil {
|
||||
@ -1066,7 +1065,7 @@ func scanReceipts2(chaindata string) error {
|
||||
log.Info("Scanned", "block", blockNum, "fixed", fixedCount)
|
||||
}
|
||||
var hash libcommon.Hash
|
||||
if hash, err = rawdb.ReadCanonicalHash(tx, blockNum); err != nil {
|
||||
if hash, err = br.CanonicalHash(context.Background(), tx, blockNum); err != nil {
|
||||
return err
|
||||
}
|
||||
if hash == (libcommon.Hash{}) {
|
||||
|
@ -751,7 +751,7 @@ func stageHeaders(db kv.RwDB, ctx context.Context, logger log.Logger) error {
|
||||
if err = bw.TruncateTd(tx, progress+1); err != nil {
|
||||
return err
|
||||
}
|
||||
hash, err := rawdb.ReadCanonicalHash(tx, progress-1)
|
||||
hash, err := br.CanonicalHash(ctx, tx, progress-1)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -27,7 +27,6 @@ import (
|
||||
"github.com/ledgerwatch/erigon/common/dbutils"
|
||||
"github.com/ledgerwatch/erigon/common/debugprint"
|
||||
"github.com/ledgerwatch/erigon/core"
|
||||
"github.com/ledgerwatch/erigon/core/rawdb"
|
||||
"github.com/ledgerwatch/erigon/core/state"
|
||||
"github.com/ledgerwatch/erigon/core/types"
|
||||
"github.com/ledgerwatch/erigon/eth/ethconfig"
|
||||
@ -351,11 +350,7 @@ func syncBySmallSteps(db kv.RwDB, miningConfig params.MiningConfig, ctx context.
|
||||
break
|
||||
}
|
||||
|
||||
hash, err := rawdb.ReadCanonicalHash(tx, execAtBlock+1)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed ReadCanonicalHash: %w", err)
|
||||
}
|
||||
nextBlock, _, err := br.BlockWithSenders(context.Background(), tx, hash, execAtBlock+1)
|
||||
nextBlock, err := br.BlockByNumber(context.Background(), tx, execAtBlock+1)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
@ -144,15 +144,12 @@ func (api *PrivateDebugAPIImpl) AccountRange(ctx context.Context, blockNrOrHash
|
||||
return state.IteratorDump{}, err
|
||||
}
|
||||
|
||||
hash, err := rawdb.ReadCanonicalHash(tx, blockNumber)
|
||||
header, err := api._blockReader.HeaderByNumber(ctx, tx, blockNumber)
|
||||
if err != nil {
|
||||
return state.IteratorDump{}, err
|
||||
}
|
||||
if hash != (common.Hash{}) {
|
||||
header := rawdb.ReadHeader(tx, hash, blockNumber)
|
||||
if header != nil {
|
||||
res.Root = header.Root.String()
|
||||
}
|
||||
if header != nil {
|
||||
res.Root = header.Root.String()
|
||||
}
|
||||
|
||||
return res, nil
|
||||
@ -299,7 +296,7 @@ func (api *PrivateDebugAPIImpl) AccountAt(ctx context.Context, blockHash common.
|
||||
if number == nil {
|
||||
return nil, nil
|
||||
}
|
||||
canonicalHash, _ := rawdb.ReadCanonicalHash(tx, *number)
|
||||
canonicalHash, _ := api._blockReader.CanonicalHash(ctx, tx, *number)
|
||||
isCanonical := canonicalHash == blockHash
|
||||
if !isCanonical {
|
||||
return nil, fmt.Errorf("block hash is not canonical")
|
||||
|
@ -15,7 +15,6 @@ import (
|
||||
"github.com/ledgerwatch/erigon-lib/kv/order"
|
||||
"github.com/ledgerwatch/erigon/cmd/rpcdaemon/rpcdaemontest"
|
||||
common2 "github.com/ledgerwatch/erigon/common"
|
||||
"github.com/ledgerwatch/erigon/core/rawdb"
|
||||
"github.com/ledgerwatch/erigon/core/state/temporal"
|
||||
"github.com/ledgerwatch/erigon/core/types"
|
||||
"github.com/ledgerwatch/erigon/eth/tracers"
|
||||
@ -479,11 +478,11 @@ func TestAccountAt(t *testing.T) {
|
||||
|
||||
var blockHash0, blockHash1, blockHash3, blockHash10, blockHash12 common.Hash
|
||||
_ = m.DB.View(m.Ctx, func(tx kv.Tx) error {
|
||||
blockHash0, _ = rawdb.ReadCanonicalHash(tx, 0)
|
||||
blockHash1, _ = rawdb.ReadCanonicalHash(tx, 1)
|
||||
blockHash3, _ = rawdb.ReadCanonicalHash(tx, 3)
|
||||
blockHash10, _ = rawdb.ReadCanonicalHash(tx, 10)
|
||||
blockHash12, _ = rawdb.ReadCanonicalHash(tx, 12)
|
||||
blockHash0, _ = br.CanonicalHash(m.Ctx, tx, 0)
|
||||
blockHash1, _ = br.CanonicalHash(m.Ctx, tx, 1)
|
||||
blockHash3, _ = br.CanonicalHash(m.Ctx, tx, 3)
|
||||
blockHash10, _ = br.CanonicalHash(m.Ctx, tx, 10)
|
||||
blockHash12, _ = br.CanonicalHash(m.Ctx, tx, 12)
|
||||
_, _, _, _, _ = blockHash0, blockHash1, blockHash3, blockHash10, blockHash12
|
||||
return nil
|
||||
})
|
||||
|
@ -158,7 +158,7 @@ func (api *BaseAPI) txnLookup(ctx context.Context, tx kv.Tx, txnHash common.Hash
|
||||
}
|
||||
|
||||
func (api *BaseAPI) blockByNumberWithSenders(ctx context.Context, tx kv.Tx, number uint64) (*types.Block, error) {
|
||||
hash, hashErr := rawdb.ReadCanonicalHash(tx, number)
|
||||
hash, hashErr := api._blockReader.CanonicalHash(ctx, tx, number)
|
||||
if hashErr != nil {
|
||||
return nil, hashErr
|
||||
}
|
||||
|
@ -14,7 +14,6 @@ import (
|
||||
"github.com/ledgerwatch/erigon/common/hexutil"
|
||||
"github.com/ledgerwatch/erigon/common/math"
|
||||
"github.com/ledgerwatch/erigon/core"
|
||||
"github.com/ledgerwatch/erigon/core/rawdb"
|
||||
"github.com/ledgerwatch/erigon/core/state"
|
||||
"github.com/ledgerwatch/erigon/core/types"
|
||||
"github.com/ledgerwatch/erigon/core/vm"
|
||||
@ -149,7 +148,7 @@ func (api *APIImpl) CallMany(ctx context.Context, bundles []Bundle, simulateCont
|
||||
if hash, ok := overrideBlockHash[i]; ok {
|
||||
return hash
|
||||
}
|
||||
hash, err := rawdb.ReadCanonicalHash(tx, i)
|
||||
hash, err := api._blockReader.CanonicalHash(ctx, tx, i)
|
||||
if err != nil {
|
||||
log.Debug("Can't get block hash by number", "number", i, "only-canonical", true)
|
||||
}
|
||||
|
@ -197,7 +197,7 @@ func (api *APIImpl) GetLogs(ctx context.Context, crit filters.FilterCriteria) (t
|
||||
continue
|
||||
}
|
||||
|
||||
blockHash, err := rawdb.ReadCanonicalHash(tx, blockNumber)
|
||||
blockHash, err := api._blockReader.CanonicalHash(ctx, tx, blockNumber)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -44,7 +44,7 @@ func (api *OtterscanAPIImpl) traceBlock(dbtx kv.Tx, ctx context.Context, blockNu
|
||||
receipts := make([]map[string]interface{}, 0)
|
||||
|
||||
// Retrieve the transaction and assemble its EVM context
|
||||
blockHash, err := rawdb.ReadCanonicalHash(dbtx, blockNum)
|
||||
blockHash, err := api._blockReader.CanonicalHash(ctx, dbtx, blockNum)
|
||||
if err != nil {
|
||||
return false, nil, err
|
||||
}
|
||||
|
@ -16,7 +16,6 @@ import (
|
||||
"github.com/ledgerwatch/erigon/core/state/temporal"
|
||||
"github.com/ledgerwatch/log/v3"
|
||||
|
||||
"github.com/ledgerwatch/erigon/core/rawdb"
|
||||
"github.com/ledgerwatch/erigon/core/types/accounts"
|
||||
)
|
||||
|
||||
@ -267,7 +266,7 @@ func (api *OtterscanAPIImpl) GetTransactionBySenderAndNonce(ctx context.Context,
|
||||
}
|
||||
|
||||
func (api *OtterscanAPIImpl) findNonce(ctx context.Context, tx kv.Tx, addr common.Address, nonce uint64, blockNum uint64) (bool, common.Hash, error) {
|
||||
hash, err := rawdb.ReadCanonicalHash(tx, blockNum)
|
||||
hash, err := api._blockReader.CanonicalHash(ctx, tx, blockNum)
|
||||
if err != nil {
|
||||
return false, common.Hash{}, err
|
||||
}
|
||||
|
@ -387,20 +387,7 @@ func (api *TraceAPIImpl) Filter(ctx context.Context, req TraceFilterRequest, gas
|
||||
for it.HasNext() {
|
||||
b := it.Next()
|
||||
// Extract transactions from block
|
||||
hash, hashErr := rawdb.ReadCanonicalHash(dbtx, b)
|
||||
if hashErr != nil {
|
||||
if first {
|
||||
first = false
|
||||
} else {
|
||||
stream.WriteMore()
|
||||
}
|
||||
stream.WriteObjectStart()
|
||||
rpc.HandleError(hashErr, stream)
|
||||
stream.WriteObjectEnd()
|
||||
continue
|
||||
}
|
||||
|
||||
block, bErr := api.blockWithSenders(ctx, dbtx, hash, b)
|
||||
block, bErr := api.blockByNumberWithSenders(ctx, dbtx, b)
|
||||
if bErr != nil {
|
||||
if first {
|
||||
first = false
|
||||
@ -419,7 +406,7 @@ func (api *TraceAPIImpl) Filter(ctx context.Context, req TraceFilterRequest, gas
|
||||
stream.WriteMore()
|
||||
}
|
||||
stream.WriteObjectStart()
|
||||
rpc.HandleError(fmt.Errorf("could not find block %x %d", hash, b), stream)
|
||||
rpc.HandleError(fmt.Errorf("could not find block %d", b), stream)
|
||||
stream.WriteObjectEnd()
|
||||
continue
|
||||
}
|
||||
|
@ -419,7 +419,7 @@ func (api *PrivateDebugAPIImpl) TraceCallMany(ctx context.Context, bundles []Bun
|
||||
if hash, ok := overrideBlockHash[i]; ok {
|
||||
return hash
|
||||
}
|
||||
hash, err := rawdb.ReadCanonicalHash(tx, i)
|
||||
hash, err := api._blockReader.CanonicalHash(ctx, tx, i)
|
||||
if err != nil {
|
||||
log.Debug("Can't get block hash by number", "number", i, "only-canonical", true)
|
||||
}
|
||||
|
@ -52,7 +52,7 @@ func (back *RemoteBackend) RawTransactions(ctx context.Context, tx kv.Getter, fr
|
||||
panic("not implemented")
|
||||
}
|
||||
func (back *RemoteBackend) BlockByNumber(ctx context.Context, db kv.Tx, number uint64) (*types.Block, error) {
|
||||
hash, err := rawdb.ReadCanonicalHash(db, number)
|
||||
hash, err := back.CanonicalHash(ctx, db, number)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed ReadCanonicalHash: %w", err)
|
||||
}
|
||||
|
@ -265,7 +265,7 @@ type MultiClient struct {
|
||||
networkId uint64
|
||||
db kv.RwDB
|
||||
Engine consensus.Engine
|
||||
blockReader services.HeaderAndCanonicalReader
|
||||
blockReader services.FullBlockReader
|
||||
logPeerInfo bool
|
||||
sendHeaderRequestsToMultiplePeers bool
|
||||
|
||||
@ -636,7 +636,7 @@ func (cs *MultiClient) getBlockBodies66(ctx context.Context, inreq *proto_sentry
|
||||
return err
|
||||
}
|
||||
defer tx.Rollback()
|
||||
response := eth.AnswerGetBlockBodiesQuery(tx, query.GetBlockBodiesPacket)
|
||||
response := eth.AnswerGetBlockBodiesQuery(tx, query.GetBlockBodiesPacket, cs.blockReader)
|
||||
tx.Rollback()
|
||||
b, err := rlp.EncodeToBytes(ð.BlockBodiesRLPPacket66{
|
||||
RequestId: query.RequestId,
|
||||
|
@ -24,7 +24,6 @@ import (
|
||||
"github.com/spf13/cobra"
|
||||
|
||||
"github.com/ledgerwatch/erigon/consensus/ethash"
|
||||
"github.com/ledgerwatch/erigon/core/rawdb"
|
||||
"github.com/ledgerwatch/erigon/core/state"
|
||||
"github.com/ledgerwatch/erigon/core/types"
|
||||
"github.com/ledgerwatch/erigon/core/vm"
|
||||
@ -137,12 +136,8 @@ func StateRoot(genesis *types.Genesis, logger log.Logger, blockNum uint64, datad
|
||||
if block >= blockNum {
|
||||
break
|
||||
}
|
||||
blockHash, err := rawdb.ReadCanonicalHash(historyTx, block)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
var b *types.Block
|
||||
b, _, err = blockReader.BlockWithSenders(ctx, historyTx, blockHash, block)
|
||||
b, err = blockReader.BlockByNumber(ctx, historyTx, block)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -11,13 +11,31 @@ import (
|
||||
|
||||
libcommon "github.com/ledgerwatch/erigon-lib/common"
|
||||
"github.com/ledgerwatch/erigon-lib/kv"
|
||||
"github.com/ledgerwatch/erigon-lib/kv/kvcfg"
|
||||
"github.com/ledgerwatch/erigon-lib/kv/mdbx"
|
||||
"github.com/ledgerwatch/erigon/core/rawdb"
|
||||
"github.com/ledgerwatch/erigon/core/rawdb/blockio"
|
||||
"github.com/ledgerwatch/erigon/eth/ethconfig"
|
||||
"github.com/ledgerwatch/erigon/turbo/services"
|
||||
"github.com/ledgerwatch/erigon/turbo/snapshotsync"
|
||||
"github.com/ledgerwatch/log/v3"
|
||||
)
|
||||
|
||||
func blocksIO(db kv.RoDB) (services.FullBlockReader, *blockio.BlockWriter) {
|
||||
var transactionsV3 bool
|
||||
if err := db.View(context.Background(), func(tx kv.Tx) error {
|
||||
transactionsV3, _ = kvcfg.TransactionsV3.Enabled(tx)
|
||||
return nil
|
||||
}); err != nil {
|
||||
panic(err)
|
||||
}
|
||||
br := snapshotsync.NewBlockReader(snapshotsync.NewRoSnapshots(ethconfig.Snapshot{Enabled: false}, "", log.New()), transactionsV3)
|
||||
bw := blockio.NewBlockWriter(transactionsV3)
|
||||
return br, bw
|
||||
}
|
||||
|
||||
func ValidateTxLookups(chaindata string, logger log.Logger) error {
|
||||
db := mdbx.MustOpen(chaindata)
|
||||
br, _ := blocksIO(db)
|
||||
tx, err := db.BeginRo(context.Background())
|
||||
if err != nil {
|
||||
return err
|
||||
@ -40,15 +58,19 @@ func ValidateTxLookups(chaindata string, logger log.Logger) error {
|
||||
var interrupt bool
|
||||
// Validation Process
|
||||
blockBytes := big.NewInt(0)
|
||||
ctx := context.Background()
|
||||
for !interrupt {
|
||||
if err := libcommon.Stopped(quitCh); err != nil {
|
||||
return err
|
||||
}
|
||||
blockHash, err := rawdb.ReadCanonicalHash(tx, blockNum)
|
||||
blockHash, err := br.CanonicalHash(ctx, tx, blockNum)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
body, err := br.BodyWithTransactions(ctx, tx, blockHash, blockNum)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
body := rawdb.ReadCanonicalBodyWithTransactions(tx, blockHash, blockNum)
|
||||
|
||||
if body == nil {
|
||||
logger.Error("Empty body", "blocknum", blockNum)
|
||||
|
@ -42,12 +42,7 @@ func (cr ChainReaderImpl) GetHeader(hash libcommon.Hash, number uint64) *types.H
|
||||
|
||||
// GetHeaderByNumber retrieves a block header from the database by number.
|
||||
func (cr ChainReaderImpl) GetHeaderByNumber(number uint64) *types.Header {
|
||||
hash, err := rawdb.ReadCanonicalHash(cr.Db, number)
|
||||
if err != nil {
|
||||
log.Error("ReadCanonicalHash failed", "err", err)
|
||||
return nil
|
||||
}
|
||||
h, _ := cr.BlockReader.Header(context.Background(), cr.Db, hash, number)
|
||||
h, _ := cr.BlockReader.HeaderByNumber(context.Background(), cr.Db, number)
|
||||
return h
|
||||
}
|
||||
|
||||
|
@ -1052,6 +1052,8 @@ func ReadReceipts(db kv.Tx, block *types.Block, senders []libcommon.Address) typ
|
||||
}
|
||||
if len(senders) > 0 {
|
||||
block.SendersToTxs(senders)
|
||||
} else {
|
||||
senders = block.Body().SendersFromTxs()
|
||||
}
|
||||
if err := receipts.DeriveFields(block.Hash(), block.NumberU64(), block.Transactions(), senders); err != nil {
|
||||
log.Error("Failed to derive block receipts fields", "hash", block.Hash(), "number", block.NumberU64(), "err", err, "stack", dbg.Stack())
|
||||
|
@ -318,11 +318,11 @@ func TestCanonicalMappingStorage(t *testing.T) {
|
||||
tx, err := m.DB.BeginRw(m.Ctx)
|
||||
require.NoError(t, err)
|
||||
defer tx.Rollback()
|
||||
_, bw := m.NewBlocksIO()
|
||||
br, bw := m.NewBlocksIO()
|
||||
|
||||
// Create a test canonical number and assinged hash to move around
|
||||
hash, number := libcommon.Hash{0: 0xff}, uint64(314)
|
||||
entry, err := rawdb.ReadCanonicalHash(tx, number)
|
||||
entry, err := br.CanonicalHash(m.Ctx, tx, number)
|
||||
if err != nil {
|
||||
t.Fatalf("ReadCanonicalHash failed: %v", err)
|
||||
}
|
||||
@ -334,7 +334,7 @@ func TestCanonicalMappingStorage(t *testing.T) {
|
||||
if err != nil {
|
||||
t.Fatalf("WriteCanoncalHash failed: %v", err)
|
||||
}
|
||||
entry, err = rawdb.ReadCanonicalHash(tx, number)
|
||||
entry, err = br.CanonicalHash(m.Ctx, tx, number)
|
||||
if err != nil {
|
||||
t.Fatalf("ReadCanonicalHash failed: %v", err)
|
||||
}
|
||||
@ -348,7 +348,7 @@ func TestCanonicalMappingStorage(t *testing.T) {
|
||||
if err != nil {
|
||||
t.Fatalf("DeleteCanonicalHash failed: %v", err)
|
||||
}
|
||||
entry, err = rawdb.ReadCanonicalHash(tx, number)
|
||||
entry, err = br.CanonicalHash(m.Ctx, tx, number)
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
|
@ -18,7 +18,6 @@ package gasprice_test
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"math"
|
||||
"math/big"
|
||||
"testing"
|
||||
@ -74,14 +73,7 @@ func (b *testBackend) HeaderByNumber(ctx context.Context, number rpc.BlockNumber
|
||||
if number == rpc.LatestBlockNumber {
|
||||
return rawdb.ReadCurrentHeader(tx), nil
|
||||
}
|
||||
hash, err := rawdb.ReadCanonicalHash(tx, uint64(number))
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed ReadCanonicalHash: %w", err)
|
||||
}
|
||||
if hash == (libcommon.Hash{}) {
|
||||
return nil, nil
|
||||
}
|
||||
return b.blockReader.Header(ctx, tx, hash, uint64(number))
|
||||
return b.blockReader.HeaderByNumber(ctx, tx, uint64(number))
|
||||
}
|
||||
|
||||
func (b *testBackend) BlockByNumber(ctx context.Context, number rpc.BlockNumber) (*types.Block, error) {
|
||||
@ -94,15 +86,7 @@ func (b *testBackend) BlockByNumber(ctx context.Context, number rpc.BlockNumber)
|
||||
if number == rpc.LatestBlockNumber {
|
||||
return b.blockReader.CurrentBlock(tx)
|
||||
}
|
||||
hash, err := rawdb.ReadCanonicalHash(tx, uint64(number))
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed ReadCanonicalHash: %w", err)
|
||||
}
|
||||
if hash == (libcommon.Hash{}) {
|
||||
return nil, nil
|
||||
}
|
||||
block, _, err := b.blockReader.BlockWithSenders(ctx, tx, hash, uint64(number))
|
||||
return block, err
|
||||
return b.blockReader.BlockByNumber(ctx, tx, uint64(number))
|
||||
}
|
||||
|
||||
func (b *testBackend) ChainConfig() *chain.Config {
|
||||
@ -157,11 +141,7 @@ func (b *testBackend) GetBlockByNumber(number uint64) *types.Block {
|
||||
}
|
||||
defer tx.Rollback()
|
||||
|
||||
hash, err := rawdb.ReadCanonicalHash(tx, number)
|
||||
if err != nil {
|
||||
return nil
|
||||
}
|
||||
block, _, _ := b.blockReader.BlockWithSenders(context.Background(), tx, hash, number)
|
||||
block, _ := b.blockReader.BlockByNumber(context.Background(), tx, number)
|
||||
return block
|
||||
}
|
||||
|
||||
|
@ -90,17 +90,16 @@ func TestGetBlockReceipts(t *testing.T) {
|
||||
hashes []libcommon.Hash
|
||||
receipts []rlp.RawValue
|
||||
)
|
||||
br, _ := m.NewBlocksIO()
|
||||
|
||||
err := m.DB.View(m.Ctx, func(tx kv.Tx) error {
|
||||
for i := uint64(0); i <= rawdb.ReadCurrentHeader(tx).Number.Uint64(); i++ {
|
||||
hash, err := rawdb.ReadCanonicalHash(tx, i)
|
||||
require.NoError(t, err)
|
||||
block, senders, err := rawdb.ReadBlockWithSenders(tx, hash, i)
|
||||
block, err := br.BlockByNumber(m.Ctx, tx, i)
|
||||
require.NoError(t, err)
|
||||
|
||||
hashes = append(hashes, block.Hash())
|
||||
// If known, encode and queue for response packet
|
||||
r := rawdb.ReadReceipts(tx, block, senders)
|
||||
r := rawdb.ReadReceipts(tx, block, nil)
|
||||
encoded, err := rlp.EncodeToBytes(r)
|
||||
require.NoError(t, err)
|
||||
receipts = append(receipts, encoded)
|
||||
|
@ -133,7 +133,7 @@ func AnswerGetBlockHeadersQuery(db kv.Tx, query *GetBlockHeadersPacket, blockRea
|
||||
return headers, nil
|
||||
}
|
||||
|
||||
func AnswerGetBlockBodiesQuery(db kv.Tx, query GetBlockBodiesPacket) []rlp.RawValue { //nolint:unparam
|
||||
func AnswerGetBlockBodiesQuery(db kv.Tx, query GetBlockBodiesPacket, blockReader services.FullBlockReader) []rlp.RawValue { //nolint:unparam
|
||||
// Gather blocks until the fetch or network limits is reached
|
||||
var bytes int
|
||||
bodies := make([]rlp.RawValue, 0, len(query))
|
||||
@ -147,21 +147,12 @@ func AnswerGetBlockBodiesQuery(db kv.Tx, query GetBlockBodiesPacket) []rlp.RawVa
|
||||
if number == nil {
|
||||
continue
|
||||
}
|
||||
canonicalHash, err := rawdb.ReadCanonicalHash(db, *number)
|
||||
if err != nil {
|
||||
break
|
||||
}
|
||||
var bodyRlP []byte
|
||||
if canonicalHash == hash {
|
||||
bodyRlP = rawdb.ReadBodyRLP(db, hash, *number)
|
||||
} else {
|
||||
bodyRlP = rawdb.NonCanonicalBodyRLP(db, hash, *number)
|
||||
}
|
||||
if len(bodyRlP) == 0 {
|
||||
bodyRLP, _ := blockReader.BodyRlp(context.Background(), db, hash, *number)
|
||||
if len(bodyRLP) == 0 {
|
||||
continue
|
||||
}
|
||||
bodies = append(bodies, bodyRlP)
|
||||
bytes += len(bodyRlP)
|
||||
bodies = append(bodies, bodyRLP)
|
||||
bytes += len(bodyRLP)
|
||||
}
|
||||
return bodies
|
||||
}
|
||||
|
@ -43,12 +43,7 @@ func (cr ChainReader) GetHeader(hash libcommon.Hash, number uint64) *types.Heade
|
||||
|
||||
// GetHeaderByNumber retrieves a block header from the database by number.
|
||||
func (cr ChainReader) GetHeaderByNumber(number uint64) *types.Header {
|
||||
hash, err := rawdb.ReadCanonicalHash(cr.Db, number)
|
||||
if err != nil {
|
||||
log.Error("ReadCanonicalHash failed", "err", err)
|
||||
return nil
|
||||
}
|
||||
h, _ := cr.BlockReader.Header(context.Background(), cr.Db, hash, number)
|
||||
h, _ := cr.BlockReader.HeaderByNumber(context.Background(), cr.Db, number)
|
||||
return h
|
||||
}
|
||||
|
||||
|
@ -36,7 +36,6 @@ import (
|
||||
"github.com/ledgerwatch/erigon/cmd/state/exec3"
|
||||
"github.com/ledgerwatch/erigon/consensus"
|
||||
"github.com/ledgerwatch/erigon/core"
|
||||
"github.com/ledgerwatch/erigon/core/rawdb"
|
||||
"github.com/ledgerwatch/erigon/core/rawdb/rawdbhelpers"
|
||||
"github.com/ledgerwatch/erigon/core/state"
|
||||
"github.com/ledgerwatch/erigon/core/types"
|
||||
@ -746,15 +745,7 @@ func blockWithSenders(db kv.RoDB, tx kv.Tx, blockReader services.BlockReader, bl
|
||||
}
|
||||
defer tx.Rollback()
|
||||
}
|
||||
blockHash, err := rawdb.ReadCanonicalHash(tx, blockNum)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
b, _, err = blockReader.BlockWithSenders(context.Background(), tx, blockHash, blockNum)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return b, nil
|
||||
return blockReader.BlockByNumber(context.Background(), tx, blockNum)
|
||||
}
|
||||
|
||||
func processResultQueue(in *exec22.QueueWithRetry, rws *exec22.ResultsQueue, outputTxNumIn uint64, rs *state.StateV3, agg *state2.AggregatorV3, applyTx kv.Tx, backPressure chan struct{}, applyWorker *exec3.Worker, canRetry, forceStopAtBlockEnd bool) (outputTxNum uint64, conflicts, triggers int, processedBlockNum uint64, stopedAtBlockEnd bool, err error) {
|
||||
|
@ -437,7 +437,7 @@ Loop:
|
||||
}
|
||||
}
|
||||
|
||||
blockHash, err := rawdb.ReadCanonicalHash(tx, blockNum)
|
||||
blockHash, err := cfg.blockReader.CanonicalHash(ctx, tx, blockNum)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -576,17 +576,14 @@ func blocksReadAhead(ctx context.Context, cfg *ExecuteBlockCfg, workers int) (ch
|
||||
}
|
||||
}
|
||||
func blocksReadAheadFunc(ctx context.Context, tx kv.Tx, cfg *ExecuteBlockCfg, blockNum uint64) error {
|
||||
blockHash, err := rawdb.ReadCanonicalHash(tx, blockNum)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
block, senders, err := cfg.blockReader.BlockWithSenders(ctx, tx, blockHash, blockNum)
|
||||
block, err := cfg.blockReader.BlockByNumber(ctx, tx, blockNum)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if block == nil {
|
||||
return nil
|
||||
}
|
||||
senders := block.Body().SendersFromTxs() //TODO: BlockByNumber can return senders
|
||||
stateReader := state.NewPlainStateReader(tx) //TODO: can do on batch! if make batch thread-safe
|
||||
for _, sender := range senders {
|
||||
a, _ := stateReader.ReadAccountData(sender)
|
||||
@ -681,7 +678,7 @@ func unwindExecutionStage(u *UnwindState, s *StageState, tx kv.RwTx, ctx context
|
||||
if !initialCycle && cfg.stateStream && s.BlockNumber-u.UnwindPoint < stateStreamLimit {
|
||||
accumulator = cfg.accumulator
|
||||
|
||||
hash, err := rawdb.ReadCanonicalHash(tx, u.UnwindPoint)
|
||||
hash, err := cfg.blockReader.CanonicalHash(ctx, tx, u.UnwindPoint)
|
||||
if err != nil {
|
||||
return fmt.Errorf("read canonical hash of unwind point: %w", err)
|
||||
}
|
||||
|
@ -774,8 +774,9 @@ func HeadersPOW(
|
||||
defer cfg.hd.SetFetchingNew(false)
|
||||
headerProgress = cfg.hd.Progress()
|
||||
logPrefix := s.LogPrefix()
|
||||
|
||||
// Check if this is called straight after the unwinds, which means we need to create new canonical markings
|
||||
hash, err := rawdb.ReadCanonicalHash(tx, headerProgress)
|
||||
hash, err := cfg.blockReader.CanonicalHash(ctx, tx, headerProgress)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -197,7 +197,7 @@ func (bt *BlockTest) insertBlocks(m *stages.MockSentry) ([]btBlock, error) {
|
||||
}
|
||||
} else if b.BlockHeader == nil {
|
||||
if err := m.DB.View(context.Background(), func(tx kv.Tx) error {
|
||||
canonical, cErr := rawdb.ReadCanonicalHash(tx, cb.NumberU64())
|
||||
canonical, cErr := bt.br.CanonicalHash(context.Background(), tx, cb.NumberU64())
|
||||
if cErr != nil {
|
||||
return cErr
|
||||
}
|
||||
|
@ -84,7 +84,7 @@ func (fv *ForkValidator) ExtendingForkHeadHash() libcommon.Hash {
|
||||
}
|
||||
|
||||
func (fv *ForkValidator) notifyTxPool(to uint64, accumulator *shards.Accumulator, c shards.StateChangeConsumer) error {
|
||||
hash, err := rawdb.ReadCanonicalHash(fv.extendingFork, to)
|
||||
hash, err := fv.blockReader.CanonicalHash(context.Background(), fv.extendingFork, to)
|
||||
if err != nil {
|
||||
return fmt.Errorf("read canonical hash of unwind point: %w", err)
|
||||
}
|
||||
|
@ -36,7 +36,7 @@ func (r *RemoteBlockReader) RawTransactions(ctx context.Context, tx kv.Getter, f
|
||||
panic("not implemented")
|
||||
}
|
||||
func (r *RemoteBlockReader) BlockByNumber(ctx context.Context, db kv.Tx, number uint64) (*types.Block, error) {
|
||||
hash, err := rawdb.ReadCanonicalHash(db, number)
|
||||
hash, err := r.CanonicalHash(ctx, db, number)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed ReadCanonicalHash: %w", err)
|
||||
}
|
||||
@ -103,7 +103,7 @@ func (r *RemoteBlockReader) TxnLookup(ctx context.Context, tx kv.Getter, txnHash
|
||||
}
|
||||
|
||||
func (r *RemoteBlockReader) TxnByIdxInBlock(ctx context.Context, tx kv.Getter, blockNum uint64, i int) (txn types.Transaction, err error) {
|
||||
canonicalHash, err := rawdb.ReadCanonicalHash(tx, blockNum)
|
||||
canonicalHash, err := r.CanonicalHash(ctx, tx, blockNum)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@ -200,7 +200,14 @@ func NewBlockReader(snapshots services.BlockSnapshots, transactionsV3 bool) *Blo
|
||||
func (r *BlockReader) Snapshots() services.BlockSnapshots { return r.sn }
|
||||
|
||||
func (r *BlockReader) HeaderByNumber(ctx context.Context, tx kv.Getter, blockHeight uint64) (h *types.Header, err error) {
|
||||
h = rawdb.ReadHeaderByNumber(tx, blockHeight)
|
||||
blockHash, err := rawdb.ReadCanonicalHash(tx, blockHeight)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if blockHash == (libcommon.Hash{}) {
|
||||
return nil, nil
|
||||
}
|
||||
h = rawdb.ReadHeader(tx, blockHash, blockHeight)
|
||||
if h != nil {
|
||||
return h, nil
|
||||
}
|
||||
|
@ -95,8 +95,8 @@ func testFork(t *testing.T, m *stages.MockSentry, i, n int, comparator func(td1,
|
||||
|
||||
// Assert the chains have the same header/block at #i
|
||||
var hash1, hash2 libcommon.Hash
|
||||
err = m.DB.View(context.Background(), func(tx kv.Tx) error {
|
||||
if hash1, err = rawdb.ReadCanonicalHash(tx, uint64(i)); err != nil {
|
||||
err = m.DB.View(m.Ctx, func(tx kv.Tx) error {
|
||||
if hash1, err = blockReader.CanonicalHash(m.Ctx, tx, uint64(i)); err != nil {
|
||||
t.Fatalf("Failed to read canonical hash: %v", err)
|
||||
}
|
||||
if block1, _, _ := blockReader.BlockWithSenders(ctx, tx, hash1, uint64(i)); block1 == nil {
|
||||
@ -107,8 +107,8 @@ func testFork(t *testing.T, m *stages.MockSentry, i, n int, comparator func(td1,
|
||||
require.NoError(t, err)
|
||||
|
||||
canonicalMock.DB.View(ctx, func(tx kv.Tx) error {
|
||||
if hash2, err = rawdb.ReadCanonicalHash(tx, uint64(i)); err != nil {
|
||||
t.Fatalf("Failed to read canonical hash 2: %v", err)
|
||||
if hash2, err = blockReader.CanonicalHash(m.Ctx, tx, uint64(i)); err != nil {
|
||||
t.Fatalf("Failed to read canonical hash: %v", err)
|
||||
}
|
||||
if block2, _, _ := blockReader.BlockWithSenders(ctx, tx, hash2, uint64(i)); block2 == nil {
|
||||
t.Fatalf("Did not find canonical block 2")
|
||||
@ -564,14 +564,14 @@ func TestCanonicalBlockRetrieval(t *testing.T) {
|
||||
err := m.InsertChain(chain)
|
||||
require.NoError(t, err)
|
||||
|
||||
tx, err := m.DB.BeginRo(context.Background())
|
||||
tx, err := m.DB.BeginRo(m.Ctx)
|
||||
require.NoError(t, err)
|
||||
defer tx.Rollback()
|
||||
|
||||
br, _ := m.NewBlocksIO()
|
||||
for _, block := range chain.Blocks {
|
||||
// try to retrieve a block by its canonical hash and see if the block data can be retrieved.
|
||||
ch, err := rawdb.ReadCanonicalHash(tx, block.NumberU64())
|
||||
ch, err := br.CanonicalHash(m.Ctx, tx, block.NumberU64())
|
||||
require.NoError(t, err)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
|
@ -47,7 +47,7 @@ func (bd *BodyDownload) UpdateFromDb(db kv.Tx) (headHeight, headTime uint64, hea
|
||||
maps.Clear(bd.peerMap)
|
||||
bd.ClearBodyCache()
|
||||
headHeight = bodyProgress
|
||||
headHash, err = rawdb.ReadCanonicalHash(db, headHeight)
|
||||
headHash, err = bd.br.CanonicalHash(context.Background(), db, headHeight)
|
||||
if err != nil {
|
||||
return 0, 0, libcommon.Hash{}, nil, err
|
||||
}
|
||||
@ -65,7 +65,10 @@ func (bd *BodyDownload) UpdateFromDb(db kv.Tx) (headHeight, headTime uint64, hea
|
||||
return 0, 0, libcommon.Hash{}, nil, fmt.Errorf("headTd higher than 2^256-1")
|
||||
}
|
||||
headTime = 0
|
||||
headHeader := rawdb.ReadHeader(db, headHash, headHeight)
|
||||
headHeader, err := bd.br.Header(context.Background(), db, headHash, headHeight)
|
||||
if err != nil {
|
||||
return 0, 0, libcommon.Hash{}, nil, fmt.Errorf("reading total difficulty for head height %d and hash %x: %d, %w", headHeight, headHash, headTd, err)
|
||||
}
|
||||
if headHeader != nil {
|
||||
headTime = headHeader.Time
|
||||
}
|
||||
@ -120,7 +123,7 @@ func (bd *BodyDownload) RequestMoreBodies(tx kv.RwTx, blockReader services.FullB
|
||||
request = false
|
||||
}
|
||||
} else {
|
||||
hash, err = rawdb.ReadCanonicalHash(tx, blockNum)
|
||||
hash, err = blockReader.CanonicalHash(context.Background(), tx, blockNum)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("could not find canonical header: %w, blockNum=%d, trace=%s", err, blockNum, dbg.Stack())
|
||||
}
|
||||
@ -402,16 +405,13 @@ func (bd *BodyDownload) GetHeader(blockNum uint64, blockReader services.FullBloc
|
||||
if bd.deliveriesH[blockNum] != nil {
|
||||
header = bd.deliveriesH[blockNum]
|
||||
} else {
|
||||
hash, err := rawdb.ReadCanonicalHash(tx, blockNum)
|
||||
if err != nil {
|
||||
return nil, libcommon.Hash{}, err
|
||||
}
|
||||
header, err = blockReader.Header(context.Background(), tx, hash, blockNum)
|
||||
var err error
|
||||
header, err = blockReader.HeaderByNumber(context.Background(), tx, blockNum)
|
||||
if err != nil {
|
||||
return nil, libcommon.Hash{}, err
|
||||
}
|
||||
if header == nil {
|
||||
return nil, libcommon.Hash{}, fmt.Errorf("header not found: blockNum=%d, hash=%x, trace=%s", blockNum, hash, dbg.Stack())
|
||||
return nil, libcommon.Hash{}, fmt.Errorf("header not found: blockNum=%d, trace=%s", blockNum, dbg.Stack())
|
||||
}
|
||||
}
|
||||
return header, header.Hash(), nil
|
||||
|
Loading…
Reference in New Issue
Block a user