From 389f994118e823e2b13cbf5799b4b1ff2e05f6a7 Mon Sep 17 00:00:00 2001 From: Tudor Malene Date: Thu, 5 Sep 2024 17:29:49 +0300 Subject: [PATCH 1/3] remove the body of the l1 blocks from the payload sent into the enclave. Only send the relevant transactions. --- go/common/enclave.go | 9 ++- go/common/gethutil/gethutil.go | 26 ++++---- go/common/types.go | 51 ++++++--------- go/enclave/components/batch_executor.go | 4 +- go/enclave/components/batch_registry.go | 8 +-- go/enclave/components/block_processor.go | 28 ++++---- go/enclave/components/interfaces.go | 8 +-- go/enclave/components/rollup_compression.go | 18 +++--- go/enclave/components/rollup_consumer.go | 10 +-- go/enclave/components/rollup_producer.go | 2 +- .../components/shared_secret_process.go | 6 +- go/enclave/core/batch.go | 2 +- go/enclave/core/rollup.go | 2 +- .../crosschain/block_message_extractor.go | 12 ++-- go/enclave/crosschain/common.go | 2 +- go/enclave/crosschain/interfaces.go | 8 ++- go/enclave/crosschain/message_bus_manager.go | 10 +-- go/enclave/enclave.go | 16 ++--- go/enclave/gas/oracle.go | 20 +++--- go/enclave/nodetype/common.go | 2 +- go/enclave/nodetype/interfaces.go | 4 +- go/enclave/nodetype/sequencer.go | 10 +-- go/enclave/nodetype/validator.go | 4 +- go/enclave/rpc/EstimateGas.go | 8 +-- go/enclave/rpc/GetBalance.go | 2 +- go/enclave/rpc/GetTransactionCount.go | 2 +- go/enclave/rpc_server.go | 8 +-- go/enclave/storage/cache_service.go | 8 +-- go/enclave/storage/enclavedb/block.go | 13 ++-- go/enclave/storage/interfaces.go | 22 +++---- go/enclave/storage/storage.go | 26 ++++---- go/ethadapter/geth_rpc_client.go | 2 +- go/ethadapter/interface.go | 10 +-- go/host/enclave/guardian.go | 11 +++- go/host/l1/blockrepository.go | 3 + go/host/rpc/enclaverpc/enclave_client.go | 6 +- integration/ethereummock/db.go | 6 +- integration/ethereummock/gethutil.go | 64 +++++++++++++++++++ integration/ethereummock/node.go | 18 ++---- integration/ethereummock/utils.go | 6 +- integration/simulation/simulation.go | 2 +- integration/simulation/validate_chain.go | 2 +- tools/walletextension/main/main.go | 2 +- 43 files changed, 276 insertions(+), 207 deletions(-) create mode 100644 integration/ethereummock/gethutil.go diff --git a/go/common/enclave.go b/go/common/enclave.go index 94bb74a03a..348360d30c 100644 --- a/go/common/enclave.go +++ b/go/common/enclave.go @@ -5,6 +5,8 @@ import ( "encoding/json" "math/big" + "github.com/ethereum/go-ethereum/core/types" + "github.com/ten-protocol/go-ten/go/common/errutil" "github.com/ten-protocol/go-ten/go/common/tracers" @@ -24,6 +26,11 @@ type Status struct { L2Head *big.Int } +type TxAndReceipt struct { + Tx *types.Transaction + Receipt *types.Receipt +} + const ( Running StatusCode = iota // the enclave is running, accepting L1 blocks AwaitingSecret // the enclave has not received the network secret and cannot process L1 blocks @@ -55,7 +62,7 @@ type Enclave interface { // It is the responsibility of the host to gossip the returned rollup // For good functioning the caller should always submit blocks ordered by height // submitting a block before receiving ancestors of it, will result in it being ignored - SubmitL1Block(ctx context.Context, block *L1Block, receipts L1Receipts, isLatest bool) (*BlockSubmissionResponse, SystemError) + SubmitL1Block(ctx context.Context, blockHeader *types.Header, receipts []*TxAndReceipt, isLatest bool) (*BlockSubmissionResponse, SystemError) // SubmitTx - user transactions SubmitTx(ctx context.Context, tx EncryptedTx) (*responses.RawTx, SystemError) diff --git a/go/common/gethutil/gethutil.go b/go/common/gethutil/gethutil.go index 8141db2222..514f313004 100644 --- a/go/common/gethutil/gethutil.go +++ b/go/common/gethutil/gethutil.go @@ -18,7 +18,7 @@ var EmptyHash = gethcommon.Hash{} // LCA - returns the latest common ancestor of the 2 blocks or an error if no common ancestor is found // it also returns the blocks that became canonical, and the once that are now the fork -func LCA(ctx context.Context, newCanonical *types.Block, oldCanonical *types.Block, resolver storage.BlockResolver) (*common.ChainFork, error) { +func LCA(ctx context.Context, newCanonical *types.Header, oldCanonical *types.Header, resolver storage.BlockResolver) (*common.ChainFork, error) { b, cp, ncp, err := internalLCA(ctx, newCanonical, oldCanonical, resolver, []common.L1BlockHash{}, []common.L1BlockHash{}) return &common.ChainFork{ NewCanonical: newCanonical, @@ -29,37 +29,37 @@ func LCA(ctx context.Context, newCanonical *types.Block, oldCanonical *types.Blo }, err } -func internalLCA(ctx context.Context, newCanonical *types.Block, oldCanonical *types.Block, resolver storage.BlockResolver, canonicalPath []common.L1BlockHash, nonCanonicalPath []common.L1BlockHash) (*types.Block, []common.L1BlockHash, []common.L1BlockHash, error) { - if newCanonical.NumberU64() == common.L1GenesisHeight || oldCanonical.NumberU64() == common.L1GenesisHeight { +func internalLCA(ctx context.Context, newCanonical *types.Header, oldCanonical *types.Header, resolver storage.BlockResolver, canonicalPath []common.L1BlockHash, nonCanonicalPath []common.L1BlockHash) (*types.Header, []common.L1BlockHash, []common.L1BlockHash, error) { + if newCanonical.Number.Uint64() == common.L1GenesisHeight || oldCanonical.Number.Uint64() == common.L1GenesisHeight { return newCanonical, canonicalPath, nonCanonicalPath, nil } if newCanonical.Hash() == oldCanonical.Hash() { // this is where we reach the common ancestor, which we add to the canonical path return newCanonical, append(canonicalPath, newCanonical.Hash()), nonCanonicalPath, nil } - if newCanonical.NumberU64() > oldCanonical.NumberU64() { - p, err := resolver.FetchBlock(ctx, newCanonical.ParentHash()) + if newCanonical.Number.Uint64() > oldCanonical.Number.Uint64() { + p, err := resolver.FetchBlock(ctx, newCanonical.ParentHash) if err != nil { - return nil, nil, nil, fmt.Errorf("could not retrieve parent block %s. Cause: %w", newCanonical.ParentHash(), err) + return nil, nil, nil, fmt.Errorf("could not retrieve parent block %s. Cause: %w", newCanonical.ParentHash, err) } return internalLCA(ctx, p, oldCanonical, resolver, append(canonicalPath, newCanonical.Hash()), nonCanonicalPath) } - if oldCanonical.NumberU64() > newCanonical.NumberU64() { - p, err := resolver.FetchBlock(ctx, oldCanonical.ParentHash()) + if oldCanonical.Number.Uint64() > newCanonical.Number.Uint64() { + p, err := resolver.FetchBlock(ctx, oldCanonical.ParentHash) if err != nil { - return nil, nil, nil, fmt.Errorf("could not retrieve parent block %s. Cause: %w", oldCanonical.ParentHash(), err) + return nil, nil, nil, fmt.Errorf("could not retrieve parent block %s. Cause: %w", oldCanonical.ParentHash, err) } return internalLCA(ctx, newCanonical, p, resolver, canonicalPath, append(nonCanonicalPath, oldCanonical.Hash())) } - parentBlockA, err := resolver.FetchBlock(ctx, newCanonical.ParentHash()) + parentBlockA, err := resolver.FetchBlock(ctx, newCanonical.ParentHash) if err != nil { - return nil, nil, nil, fmt.Errorf("could not retrieve parent block %s. Cause: %w", newCanonical.ParentHash(), err) + return nil, nil, nil, fmt.Errorf("could not retrieve parent block %s. Cause: %w", newCanonical.ParentHash, err) } - parentBlockB, err := resolver.FetchBlock(ctx, oldCanonical.ParentHash()) + parentBlockB, err := resolver.FetchBlock(ctx, oldCanonical.ParentHash) if err != nil { - return nil, nil, nil, fmt.Errorf("could not retrieve parent block %s. Cause: %w", oldCanonical.ParentHash(), err) + return nil, nil, nil, fmt.Errorf("could not retrieve parent block %s. Cause: %w", oldCanonical.ParentHash, err) } return internalLCA(ctx, parentBlockA, parentBlockB, resolver, append(canonicalPath, newCanonical.Hash()), append(nonCanonicalPath, oldCanonical.Hash())) diff --git a/go/common/types.go b/go/common/types.go index 12fcbc5c97..ff2d1c5572 100644 --- a/go/common/types.go +++ b/go/common/types.go @@ -125,61 +125,52 @@ type ( // To work properly, all of the receipts are required, due to rlp encoding pruning some of the information. // The receipts must also be in the correct order. type BlockAndReceipts struct { - Block *L1Block - ReceiptsMap map[int]*types.Receipt // sparse map with obscuro-relevant receipts in it - Receipts *types.Receipts + BlockHeader *types.Header + TxsWithReceipts []*TxAndReceipt successfulTransactions *types.Transactions } // ParseBlockAndReceipts - will create a container struct that has preprocessed the receipts // and verified if they indeed match the receipt root hash in the block. -func ParseBlockAndReceipts(block *L1Block, receipts *L1Receipts) (*BlockAndReceipts, error) { - if len(block.Transactions()) != len(*receipts) { - // the receipts list is currently a *sparse* list of relevant receipts, it needs to have the same length as the - // transactions list even though some of the entries may be nil - return nil, fmt.Errorf("transactions and receipts are not the same length") - } - +func ParseBlockAndReceipts(block *types.Header, receipts []*TxAndReceipt) (*BlockAndReceipts, error) { br := BlockAndReceipts{ - Block: block, - Receipts: receipts, - ReceiptsMap: make(map[int]*types.Receipt, len(block.Transactions())), - successfulTransactions: nil, - } - - for idx, receipt := range *receipts { - br.ReceiptsMap[idx] = receipt + BlockHeader: block, + TxsWithReceipts: receipts, } return &br, nil } -// SuccessfulTransactions - returns slice containing only the transactions that have receipts with successful status. -func (br *BlockAndReceipts) SuccessfulTransactions() *types.Transactions { +func (br *BlockAndReceipts) Receipts() L1Receipts { + rec := make(L1Receipts, 0) + for _, txsWithReceipt := range br.TxsWithReceipts { + rec = append(rec, txsWithReceipt.Receipt) + } + return rec +} + +// RelevantTransactions - returns slice containing only the transactions that have receipts with successful status. +func (br *BlockAndReceipts) RelevantTransactions() *types.Transactions { if br.successfulTransactions != nil { return br.successfulTransactions } - txs := br.Block.Transactions() st := make(types.Transactions, 0) - - for idx, tx := range txs { - receipt, ok := br.ReceiptsMap[idx] - if ok && receipt.Status == types.ReceiptStatusSuccessful { - st = append(st, tx) + for _, tx := range br.TxsWithReceipts { + if tx.Receipt.Status == types.ReceiptStatusSuccessful { + st = append(st, tx.Tx) } } - br.successfulTransactions = &st return br.successfulTransactions } // ChainFork - represents the result of walking the chain when processing a fork type ChainFork struct { - NewCanonical *types.Block - OldCanonical *types.Block + NewCanonical *types.Header + OldCanonical *types.Header - CommonAncestor *types.Block + CommonAncestor *types.Header CanonicalPath []L1BlockHash NonCanonicalPath []L1BlockHash } diff --git a/go/enclave/components/batch_executor.go b/go/enclave/components/batch_executor.go index 16ffdc108c..6af0a7b7d6 100644 --- a/go/enclave/components/batch_executor.go +++ b/go/enclave/components/batch_executor.go @@ -343,7 +343,7 @@ func (executor *batchExecutor) CreateGenesisState( return genesisBatch, deployTx, nil } -func (executor *batchExecutor) populateOutboundCrossChainData(ctx context.Context, batch *core.Batch, block *types.Block, receipts types.Receipts) error { +func (executor *batchExecutor) populateOutboundCrossChainData(ctx context.Context, batch *core.Batch, block *types.Header, receipts types.Receipts) error { crossChainMessages, err := executor.crossChainProcessors.Local.ExtractOutboundMessages(ctx, receipts) if err != nil { executor.logger.Error("Failed extracting L2->L1 messages", log.ErrKey, err, log.CmpKey, log.CrossChainCmp) @@ -395,7 +395,7 @@ func (executor *batchExecutor) populateOutboundCrossChainData(ctx context.Contex len(batch.Header.CrossChainMessages)), log.CmpKey, log.CrossChainCmp) batch.Header.LatestInboundCrossChainHash = block.Hash() - batch.Header.LatestInboundCrossChainHeight = block.Number() + batch.Header.LatestInboundCrossChainHeight = block.Number return nil } diff --git a/go/enclave/components/batch_registry.go b/go/enclave/components/batch_registry.go index 1d8d01d90b..d38a4a889e 100644 --- a/go/enclave/components/batch_registry.go +++ b/go/enclave/components/batch_registry.go @@ -112,7 +112,7 @@ func (br *batchRegistry) HasGenesisBatch() (bool, error) { return br.HeadBatchSeq() != nil, nil } -func (br *batchRegistry) BatchesAfter(ctx context.Context, batchSeqNo uint64, upToL1Height uint64, rollupLimiter limiters.RollupLimiter) ([]*core.Batch, []*types.Block, error) { +func (br *batchRegistry) BatchesAfter(ctx context.Context, batchSeqNo uint64, upToL1Height uint64, rollupLimiter limiters.RollupLimiter) ([]*core.Batch, []*types.Header, error) { // sanity check headBatch, err := br.storage.FetchBatchHeaderBySeqNo(ctx, br.HeadBatchSeq().Uint64()) if err != nil { @@ -124,10 +124,10 @@ func (br *batchRegistry) BatchesAfter(ctx context.Context, batchSeqNo uint64, up } resultBatches := make([]*core.Batch, 0) - resultBlocks := make([]*types.Block, 0) + resultBlocks := make([]*types.Header, 0) currentBatchSeq := batchSeqNo - var currentBlock *types.Block + var currentBlock *types.Header for currentBatchSeq <= headBatch.SequencerOrderNo.Uint64() { batch, err := br.storage.FetchBatchBySeqNo(ctx, currentBatchSeq) if err != nil { @@ -142,7 +142,7 @@ func (br *batchRegistry) BatchesAfter(ctx context.Context, batchSeqNo uint64, up return nil, nil, fmt.Errorf("could not retrieve block. Cause: %w", err) } currentBlock = block - if block.NumberU64() > upToL1Height { + if block.Number.Uint64() > upToL1Height { break } resultBlocks = append(resultBlocks, block) diff --git a/go/enclave/components/block_processor.go b/go/enclave/components/block_processor.go index 15cb1ebcc9..48f432ef9a 100644 --- a/go/enclave/components/block_processor.go +++ b/go/enclave/components/block_processor.go @@ -6,6 +6,8 @@ import ( "fmt" "time" + "github.com/ethereum/go-ethereum/core/types" + "github.com/ten-protocol/go-ten/go/common/async" "github.com/ten-protocol/go-ten/go/enclave/core" "github.com/ten-protocol/go-ten/go/enclave/gas" @@ -58,7 +60,7 @@ func NewBlockProcessor(storage storage.Storage, cc *crosschain.Processors, gasOr } func (bp *l1BlockProcessor) Process(ctx context.Context, br *common.BlockAndReceipts) (*BlockIngestionType, error) { - defer core.LogMethodDuration(bp.logger, measure.NewStopwatch(), "L1 block processed", log.BlockHashKey, br.Block.Hash()) + defer core.LogMethodDuration(bp.logger, measure.NewStopwatch(), "L1 block processed", log.BlockHashKey, br.BlockHeader.Hash()) ingestion, err := bp.tryAndInsertBlock(ctx, br) if err != nil { @@ -67,21 +69,21 @@ func (bp *l1BlockProcessor) Process(ctx context.Context, br *common.BlockAndRece if !ingestion.PreGenesis { // This requires block to be stored first ... but can permanently fail a block - err = bp.crossChainProcessors.Remote.StoreCrossChainMessages(ctx, br.Block, *br.Receipts) + err = bp.crossChainProcessors.Remote.StoreCrossChainMessages(ctx, br.BlockHeader, br.Receipts()) if err != nil { return nil, errors.New("failed to process cross chain messages") } - err = bp.crossChainProcessors.Remote.StoreCrossChainValueTransfers(ctx, br.Block, *br.Receipts) + err = bp.crossChainProcessors.Remote.StoreCrossChainValueTransfers(ctx, br.BlockHeader, br.Receipts()) if err != nil { return nil, fmt.Errorf("failed to process cross chain transfers. Cause: %w", err) } } // todo @siliev - not sure if this is the best way to update the price, will pick up random stale blocks from forks? - bp.gasOracle.ProcessL1Block(br.Block) + bp.gasOracle.ProcessL1Block(br.BlockHeader) - h := br.Block.Hash() + h := br.BlockHeader.Hash() bp.currentL1Head = &h bp.lastIngestedBlock.Mark() return ingestion, nil @@ -98,7 +100,7 @@ func (bp *l1BlockProcessor) HealthCheck() (bool, error) { } func (bp *l1BlockProcessor) tryAndInsertBlock(ctx context.Context, br *common.BlockAndReceipts) (*BlockIngestionType, error) { - block := br.Block + block := br.BlockHeader // We insert the block into the L1 chain and store it. // in case the block already exists in the database, this will be treated like a fork, because the head changes to @@ -118,13 +120,13 @@ func (bp *l1BlockProcessor) tryAndInsertBlock(ctx context.Context, br *common.Bl return nil, fmt.Errorf("1. could not store block. Cause: %w", err) } - bp.logger.Trace("Block inserted successfully", - log.BlockHeightKey, block.NumberU64(), log.BlockHashKey, block.Hash(), "ingestionType", ingestionType) + bp.logger.Trace("BlockHeader inserted successfully", + log.BlockHeightKey, block.Number, log.BlockHashKey, block.Hash(), "ingestionType", ingestionType) return ingestionType, nil } -func (bp *l1BlockProcessor) ingestBlock(ctx context.Context, block *common.L1Block) (*BlockIngestionType, error) { +func (bp *l1BlockProcessor) ingestBlock(ctx context.Context, block *types.Header) (*BlockIngestionType, error) { // todo (#1056) - this is minimal L1 tracking/validation, and should be removed when we are using geth's blockchain or lightchain structures for validation prevL1Head, err := bp.GetHead(ctx) if err != nil { @@ -135,7 +137,7 @@ func (bp *l1BlockProcessor) ingestBlock(ctx context.Context, block *common.L1Blo return nil, fmt.Errorf("could not retrieve head block. Cause: %w", err) } // we do a basic sanity check, comparing the received block to the head block on the chain - if block.ParentHash() != prevL1Head.Hash() { + if block.ParentHash != prevL1Head.Hash() { isCanon, err := bp.storage.IsBlockCanonical(ctx, block.Hash()) if err != nil { return nil, fmt.Errorf("could not check if block is canonical. Cause: %w", err) @@ -147,8 +149,8 @@ func (bp *l1BlockProcessor) ingestBlock(ctx context.Context, block *common.L1Blo chainFork, err := gethutil.LCA(ctx, block, prevL1Head, bp.storage) if err != nil { bp.logger.Trace("cannot calculate the fork for received block", - "blkHeight", block.NumberU64(), log.BlockHashKey, block.Hash(), - "l1HeadHeight", prevL1Head.NumberU64(), "l1HeadHash", prevL1Head.Hash(), + "blkHeight", block.Number.Uint64(), log.BlockHashKey, block.Hash(), + "l1HeadHeight", prevL1Head.Number.Uint64(), "l1HeadHash", prevL1Head.Hash(), log.ErrKey, err, ) return nil, errutil.ErrBlockAncestorNotFound @@ -163,7 +165,7 @@ func (bp *l1BlockProcessor) ingestBlock(ctx context.Context, block *common.L1Blo return &BlockIngestionType{ChainFork: nil, PreGenesis: false}, nil } -func (bp *l1BlockProcessor) GetHead(ctx context.Context) (*common.L1Block, error) { +func (bp *l1BlockProcessor) GetHead(ctx context.Context) (*types.Header, error) { if bp.currentL1Head == nil { return nil, errutil.ErrNotFound } diff --git a/go/enclave/components/interfaces.go b/go/enclave/components/interfaces.go index cad76c7292..d74924a0f0 100644 --- a/go/enclave/components/interfaces.go +++ b/go/enclave/components/interfaces.go @@ -26,7 +26,7 @@ type BlockIngestionType struct { // ChainFork contains information about the status of the new block in the chain ChainFork *common.ChainFork - // Block that is already on the canonical chain + // BlockHeader that is already on the canonical chain OldCanonicalBlock bool } @@ -39,14 +39,14 @@ func (bit *BlockIngestionType) IsFork() bool { type L1BlockProcessor interface { Process(ctx context.Context, br *common.BlockAndReceipts) (*BlockIngestionType, error) - GetHead(context.Context) (*common.L1Block, error) + GetHead(context.Context) (*types.Header, error) GetCrossChainContractAddress() *gethcommon.Address HealthCheck() (bool, error) } // BatchExecutionContext - Contains all of the data that each batch depends on type BatchExecutionContext struct { - BlockPtr common.L1BlockHash // Block is needed for the cross chain messages + BlockPtr common.L1BlockHash // BlockHeader is needed for the cross chain messages ParentPtr common.L2BatchHash Transactions common.L2Transactions AtTime uint64 @@ -88,7 +88,7 @@ type BatchExecutor interface { type BatchRegistry interface { // BatchesAfter - Given a hash, will return batches following it until the head batch and the l1 blocks referenced by those batches - BatchesAfter(ctx context.Context, batchSeqNo uint64, upToL1Height uint64, rollupLimiter limiters.RollupLimiter) ([]*core.Batch, []*types.Block, error) + BatchesAfter(ctx context.Context, batchSeqNo uint64, upToL1Height uint64, rollupLimiter limiters.RollupLimiter) ([]*core.Batch, []*types.Header, error) // GetBatchStateAtHeight - creates a stateDB for the block number GetBatchStateAtHeight(ctx context.Context, blockNumber *gethrpc.BlockNumber) (*state.StateDB, error) diff --git a/go/enclave/components/rollup_compression.go b/go/enclave/components/rollup_compression.go index 6b36d188da..1f9fbef73c 100644 --- a/go/enclave/components/rollup_compression.go +++ b/go/enclave/components/rollup_compression.go @@ -201,11 +201,11 @@ func (rc *RollupCompression) createRollupHeader(ctx context.Context, rollup *cor // the first element is the actual height if i == 0 { - l1HeightDeltas[i] = block.Number() + l1HeightDeltas[i] = block.Number } else { - l1HeightDeltas[i] = big.NewInt(block.Number().Int64() - prevL1Height.Int64()) + l1HeightDeltas[i] = big.NewInt(block.Number.Int64() - prevL1Height.Int64()) } - prevL1Height = block.Number() + prevL1Height = block.Number } l1DeltasBA := make([][]byte, len(l1HeightDeltas)) @@ -284,7 +284,7 @@ func (rc *RollupCompression) createIncompleteBatches(ctx context.Context, callda } // a cache of the l1 blocks used by the current rollup, indexed by their height - l1BlocksAtHeight := make(map[uint64]*types.Block) + l1BlocksAtHeight := make(map[uint64]*types.Header) err = rc.calcL1AncestorsOfHeight(ctx, big.NewInt(int64(slices.Min(l1Heights))), rollupL1Block, l1BlocksAtHeight) if err != nil { return nil, err @@ -353,7 +353,7 @@ func (rc *RollupCompression) createIncompleteBatches(ctx context.Context, callda baseFee: calldataRollupHeader.BaseFee, gasLimit: calldataRollupHeader.GasLimit, } - rc.logger.Info("Rollup decompressed batch", log.BatchSeqNoKey, currentSeqNo, log.BatchHeightKey, currentHeight, "rollup_idx", currentBatchIdx, "l1_height", block.Number(), "l1_hash", block.Hash()) + rc.logger.Info("Rollup decompressed batch", log.BatchSeqNoKey, currentSeqNo, log.BatchHeightKey, currentHeight, "rollup_idx", currentBatchIdx, "l1_height", block.Number, "l1_hash", block.Hash()) } return incompleteBatches, nil } @@ -388,12 +388,12 @@ func (rc *RollupCompression) calculateL1HeightsFromDeltas(calldataRollupHeader * return l1Heights, nil } -func (rc *RollupCompression) calcL1AncestorsOfHeight(ctx context.Context, fromHeight *big.Int, toBlock *types.Block, path map[uint64]*types.Block) error { - path[toBlock.NumberU64()] = toBlock - if toBlock.NumberU64() == fromHeight.Uint64() { +func (rc *RollupCompression) calcL1AncestorsOfHeight(ctx context.Context, fromHeight *big.Int, toBlock *types.Header, path map[uint64]*types.Header) error { + path[toBlock.Number.Uint64()] = toBlock + if toBlock.Number.Uint64() == fromHeight.Uint64() { return nil } - p, err := rc.storage.FetchBlock(ctx, toBlock.ParentHash()) + p, err := rc.storage.FetchBlock(ctx, toBlock.ParentHash) if err != nil { return err } diff --git a/go/enclave/components/rollup_consumer.go b/go/enclave/components/rollup_consumer.go index be97898e8e..0a72e9294d 100644 --- a/go/enclave/components/rollup_consumer.go +++ b/go/enclave/components/rollup_consumer.go @@ -48,7 +48,7 @@ func NewRollupConsumer( } func (rc *rollupConsumerImpl) ProcessRollupsInBlock(ctx context.Context, b *common.BlockAndReceipts) error { - defer core.LogMethodDuration(rc.logger, measure.NewStopwatch(), "Rollup consumer processed block", log.BlockHashKey, b.Block.Hash()) + defer core.LogMethodDuration(rc.logger, measure.NewStopwatch(), "Rollup consumer processed block", log.BlockHashKey, b.BlockHeader.Hash()) rollups := rc.extractRollups(b) if len(rollups) == 0 { @@ -62,7 +62,7 @@ func (rc *rollupConsumerImpl) ProcessRollupsInBlock(ctx context.Context, b *comm if len(rollups) > 1 { // todo - we need to sort this out - rc.logger.Warn(fmt.Sprintf("Multiple rollups %d in block %s", len(rollups), b.Block.Hash())) + rc.logger.Warn(fmt.Sprintf("Multiple rollups %d in block %s", len(rollups), b.BlockHeader.Hash())) } for _, rollup := range rollups { @@ -71,7 +71,7 @@ func (rc *rollupConsumerImpl) ProcessRollupsInBlock(ctx context.Context, b *comm rc.logger.Warn("Can't process rollup because the l1 block used for compression is not available", "block_hash", rollup.Header.CompressionL1Head, log.RollupHashKey, rollup.Hash(), log.ErrKey, err) continue } - canonicalBlockByHeight, err := rc.storage.FetchCanonicaBlockByHeight(ctx, l1CompressionBlock.Number()) + canonicalBlockByHeight, err := rc.storage.FetchCanonicaBlockByHeight(ctx, l1CompressionBlock.Number) if err != nil { return err } @@ -113,9 +113,9 @@ func (rc *rollupConsumerImpl) getSignedRollup(rollups []*common.ExtRollup) ([]*c // extractRollups - returns a list of the rollups published in this block func (rc *rollupConsumerImpl) extractRollups(br *common.BlockAndReceipts) []*common.ExtRollup { rollups := make([]*common.ExtRollup, 0) - b := br.Block + b := br.BlockHeader - for _, tx := range *br.SuccessfulTransactions() { + for _, tx := range *br.RelevantTransactions() { // go through all rollup transactions t := rc.MgmtContractLib.DecodeTx(tx) if t == nil { diff --git a/go/enclave/components/rollup_producer.go b/go/enclave/components/rollup_producer.go index 17bb301461..9158214d80 100644 --- a/go/enclave/components/rollup_producer.go +++ b/go/enclave/components/rollup_producer.go @@ -64,7 +64,7 @@ func (re *rollupProducerImpl) CreateInternalRollup(ctx context.Context, fromBatc lastBatch := batches[len(batches)-1] rh.LastBatchSeqNo = lastBatch.SeqNo().Uint64() - blockMap := map[common.L1BlockHash]*types.Block{} + blockMap := map[common.L1BlockHash]*types.Header{} for _, b := range blocks { blockMap[b.Hash()] = b } diff --git a/go/enclave/components/shared_secret_process.go b/go/enclave/components/shared_secret_process.go index 82daff355c..c3f9ad13c5 100644 --- a/go/enclave/components/shared_secret_process.go +++ b/go/enclave/components/shared_secret_process.go @@ -36,14 +36,14 @@ func NewSharedSecretProcessor(mgmtcontractlib mgmtcontractlib.MgmtContractLib, a // ProcessNetworkSecretMsgs we watch for all messages that are requesting or receiving the secret and we store the nodes attested keys func (ssp *SharedSecretProcessor) ProcessNetworkSecretMsgs(ctx context.Context, br *common.BlockAndReceipts) []*common.ProducedSecretResponse { var responses []*common.ProducedSecretResponse - transactions := br.SuccessfulTransactions() - block := br.Block + transactions := br.RelevantTransactions() + block := br.BlockHeader for _, tx := range *transactions { t := ssp.mgmtContractLib.DecodeTx(tx) // this transaction is for a node that has joined the network and needs to be sent the network secret if scrtReqTx, ok := t.(*ethadapter.L1RequestSecretTx); ok { - ssp.logger.Info("Process shared secret request.", log.BlockHeightKey, block.Number(), log.BlockHashKey, block.Hash(), log.TxKey, tx.Hash()) + ssp.logger.Info("Process shared secret request.", log.BlockHeightKey, block.Number, log.BlockHashKey, block.Hash(), log.TxKey, tx.Hash()) resp, err := ssp.processSecretRequest(ctx, scrtReqTx) if err != nil { ssp.logger.Error("Failed to process shared secret request.", log.ErrKey, err) diff --git a/go/enclave/core/batch.go b/go/enclave/core/batch.go index cdc15cf9e4..3d9e7298ef 100644 --- a/go/enclave/core/batch.go +++ b/go/enclave/core/batch.go @@ -96,7 +96,7 @@ func ToBatch(extBatch *common.ExtBatch, transactionBlobCrypto crypto.DataEncrypt func DeterministicEmptyBatch( parent *common.BatchHeader, - block *types.Block, + block *types.Header, time uint64, sequencerNo *big.Int, baseFee *big.Int, diff --git a/go/enclave/core/rollup.go b/go/enclave/core/rollup.go index 54e4bc4249..790efea46b 100644 --- a/go/enclave/core/rollup.go +++ b/go/enclave/core/rollup.go @@ -12,7 +12,7 @@ import ( type Rollup struct { Header *common.RollupHeader Batches []*Batch - Blocks map[common.L1BlockHash]*types.Block // these are the blocks required during compression. The key is the hash + Blocks map[common.L1BlockHash]*types.Header // these are the blocks required during compression. The key is the hash hash atomic.Value } diff --git a/go/enclave/crosschain/block_message_extractor.go b/go/enclave/crosschain/block_message_extractor.go index b6f5214123..a38d003603 100644 --- a/go/enclave/crosschain/block_message_extractor.go +++ b/go/enclave/crosschain/block_message_extractor.go @@ -4,6 +4,8 @@ import ( "context" "fmt" + "github.com/ethereum/go-ethereum/core/types" + "github.com/ten-protocol/go-ten/go/enclave/core" "github.com/ten-protocol/go-ten/go/enclave/storage" @@ -37,8 +39,8 @@ func (m *blockMessageExtractor) Enabled() bool { return m.GetBusAddress().Big().Cmp(gethcommon.Big0) != 0 } -func (m *blockMessageExtractor) StoreCrossChainValueTransfers(ctx context.Context, block *common.L1Block, receipts common.L1Receipts) error { - defer core.LogMethodDuration(m.logger, measure.NewStopwatch(), "Block value transfer messages processed", log.BlockHashKey, block.Hash()) +func (m *blockMessageExtractor) StoreCrossChainValueTransfers(ctx context.Context, block *types.Header, receipts common.L1Receipts) error { + defer core.LogMethodDuration(m.logger, measure.NewStopwatch(), "BlockHeader value transfer messages processed", log.BlockHashKey, block.Hash()) /*areReceiptsValid := common.VerifyReceiptHash(block, receipts) @@ -76,8 +78,8 @@ func (m *blockMessageExtractor) StoreCrossChainValueTransfers(ctx context.Contex // The messages will be stored in DB storage for later usage. // block - the L1 block for which events are extracted. // receipts - all of the receipts for the corresponding block. This is validated. -func (m *blockMessageExtractor) StoreCrossChainMessages(ctx context.Context, block *common.L1Block, receipts common.L1Receipts) error { - defer core.LogMethodDuration(m.logger, measure.NewStopwatch(), "Block cross chain messages processed", log.BlockHashKey, block.Hash()) +func (m *blockMessageExtractor) StoreCrossChainMessages(ctx context.Context, block *types.Header, receipts common.L1Receipts) error { + defer core.LogMethodDuration(m.logger, measure.NewStopwatch(), "BlockHeader cross chain messages processed", log.BlockHashKey, block.Hash()) if len(receipts) == 0 { return nil @@ -108,7 +110,7 @@ func (m *blockMessageExtractor) GetBusAddress() *common.L1Address { } // getCrossChainMessages - Converts the relevant logs from the appropriate message bus address to synthetic transactions and returns them -func (m *blockMessageExtractor) getCrossChainMessages(block *common.L1Block, receipts common.L1Receipts) (common.CrossChainMessages, error) { +func (m *blockMessageExtractor) getCrossChainMessages(block *types.Header, receipts common.L1Receipts) (common.CrossChainMessages, error) { if len(receipts) == 0 { return make(common.CrossChainMessages, 0), nil } diff --git a/go/enclave/crosschain/common.go b/go/enclave/crosschain/common.go index 0796db1709..68806301ed 100644 --- a/go/enclave/crosschain/common.go +++ b/go/enclave/crosschain/common.go @@ -30,7 +30,7 @@ var ( ValueTransferEventID = MessageBusABI.Events["ValueTransfer"].ID ) -func lazilyLogReceiptChecksum(block *common.L1Block, receipts types.Receipts, logger gethlog.Logger) { +func lazilyLogReceiptChecksum(block *types.Header, receipts types.Receipts, logger gethlog.Logger) { if logger.Enabled(context.Background(), gethlog.LevelTrace) { logger.Trace("Processing block", log.BlockHashKey, block.Hash(), "nr_rec", len(receipts), "Hash", receiptsHash(receipts)) } diff --git a/go/enclave/crosschain/interfaces.go b/go/enclave/crosschain/interfaces.go index e1130cc6f1..37607e1fe9 100644 --- a/go/enclave/crosschain/interfaces.go +++ b/go/enclave/crosschain/interfaces.go @@ -3,6 +3,8 @@ package crosschain import ( "context" + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/core" "github.com/ethereum/go-ethereum/core/state" "github.com/ten-protocol/go-ten/go/common" @@ -16,9 +18,9 @@ type ( type BlockMessageExtractor interface { // StoreCrossChainMessages - Verifies receipts belong to block and saves the relevant cross chain messages from the receipts - StoreCrossChainMessages(ctx context.Context, block *common.L1Block, receipts common.L1Receipts) error + StoreCrossChainMessages(ctx context.Context, block *types.Header, receipts common.L1Receipts) error - StoreCrossChainValueTransfers(ctx context.Context, block *common.L1Block, receipts common.L1Receipts) error + StoreCrossChainValueTransfers(ctx context.Context, block *types.Header, receipts common.L1Receipts) error // GetBusAddress - Returns the L1 message bus address. GetBusAddress() *common.L1Address @@ -53,5 +55,5 @@ type Manager interface { ExecuteValueTransfers(ctx context.Context, transfers common.ValueTransferEvents, rollupState *state.StateDB) - RetrieveInboundMessages(ctx context.Context, fromBlock *common.L1Block, toBlock *common.L1Block, rollupState *state.StateDB) (common.CrossChainMessages, common.ValueTransferEvents) + RetrieveInboundMessages(ctx context.Context, fromBlock *types.Header, toBlock *types.Header, rollupState *state.StateDB) (common.CrossChainMessages, common.ValueTransferEvents) } diff --git a/go/enclave/crosschain/message_bus_manager.go b/go/enclave/crosschain/message_bus_manager.go index cb59ab764b..01c05e8242 100644 --- a/go/enclave/crosschain/message_bus_manager.go +++ b/go/enclave/crosschain/message_bus_manager.go @@ -146,12 +146,12 @@ func (m *MessageBusManager) ExtractOutboundTransfers(_ context.Context, receipts // todo (@stefan) - fix ordering of messages, currently it is irrelevant. // todo (@stefan) - do not extract messages below their consistency level. Irrelevant security wise. // todo (@stefan) - surface errors -func (m *MessageBusManager) RetrieveInboundMessages(ctx context.Context, fromBlock *common.L1Block, toBlock *common.L1Block, _ *state.StateDB) (common.CrossChainMessages, common.ValueTransferEvents) { +func (m *MessageBusManager) RetrieveInboundMessages(ctx context.Context, fromBlock *types.Header, toBlock *types.Header, rollupState *state.StateDB) (common.CrossChainMessages, common.ValueTransferEvents) { messages := make(common.CrossChainMessages, 0) transfers := make(common.ValueTransferEvents, 0) from := fromBlock.Hash() - height := fromBlock.NumberU64() + height := fromBlock.Number.Uint64() if !m.storage.IsAncestor(ctx, toBlock, fromBlock) { m.logger.Crit("Synthetic transactions can't be processed because the rollups are not on the same Ethereum fork. This should not happen.") } @@ -178,10 +178,10 @@ func (m *MessageBusManager) RetrieveInboundMessages(ctx context.Context, fromBlo transfers = append(transfers, transfersForBlock...) // No deposits before genesis. - if b.NumberU64() < height { + if b.Number.Uint64() < height { m.logger.Crit("block height is less than genesis height") } - p, err := m.storage.FetchBlock(ctx, b.ParentHash()) + p, err := m.storage.FetchBlock(ctx, b.ParentHash) if err != nil { m.logger.Crit("Synthetic transactions can't be processed because the rollups are not on the same Ethereum fork") } @@ -192,7 +192,7 @@ func (m *MessageBusManager) RetrieveInboundMessages(ctx context.Context, fromBlo if len(messages)+len(transfers) == 0 { logf = m.logger.Debug } - logf(fmt.Sprintf("Extracted cross chain messages for block height %d ->%d", fromBlock.NumberU64(), toBlock.NumberU64()), "no_msgs", len(messages), "no_value_transfers", len(transfers)) + logf(fmt.Sprintf("Extracted cross chain messages for block height %d ->%d", fromBlock.Number.Uint64(), toBlock.Number.Uint64()), "no_msgs", len(messages), "no_value_transfers", len(transfers)) return messages, transfers } diff --git a/go/enclave/enclave.go b/go/enclave/enclave.go index ea2b930956..f31f7f6398 100644 --- a/go/enclave/enclave.go +++ b/go/enclave/enclave.go @@ -415,7 +415,7 @@ func (e *enclaveImpl) StreamL2Updates() (chan common.StreamL2UpdatesResponse, fu } // SubmitL1Block is used to update the enclave with an additional L1 block. -func (e *enclaveImpl) SubmitL1Block(ctx context.Context, block *common.L1Block, receipts common.L1Receipts, isLatest bool) (*common.BlockSubmissionResponse, common.SystemError) { +func (e *enclaveImpl) SubmitL1Block(ctx context.Context, blockHeader *types.Header, receipts []*common.TxAndReceipt, isLatest bool) (*common.BlockSubmissionResponse, common.SystemError) { if e.stopControl.IsStopping() { return nil, responses.ToInternalError(fmt.Errorf("requested SubmitL1Block with the enclave stopping")) } @@ -423,10 +423,10 @@ func (e *enclaveImpl) SubmitL1Block(ctx context.Context, block *common.L1Block, e.mainMutex.Lock() defer e.mainMutex.Unlock() - e.logger.Info("SubmitL1Block", log.BlockHeightKey, block.Number(), log.BlockHashKey, block.Hash()) + e.logger.Info("SubmitL1Block", log.BlockHeightKey, blockHeader.Number, log.BlockHashKey, blockHeader.Hash()) // If the block and receipts do not match, reject the block. - br, err := common.ParseBlockAndReceipts(block, &receipts) + br, err := common.ParseBlockAndReceipts(blockHeader, receipts) if err != nil { return nil, e.rejectBlockErr(ctx, fmt.Errorf("could not submit L1 block. Cause: %w", err)) } @@ -437,10 +437,10 @@ func (e *enclaveImpl) SubmitL1Block(ctx context.Context, block *common.L1Block, } if result.IsFork() { - e.logger.Info(fmt.Sprintf("Detected fork at block %s with height %d", block.Hash(), block.Number())) + e.logger.Info(fmt.Sprintf("Detected fork at block %s with height %d", blockHeader.Hash(), blockHeader.Number)) } - err = e.service.OnL1Block(ctx, block, result) + err = e.service.OnL1Block(ctx, blockHeader, result) if err != nil { return nil, e.rejectBlockErr(ctx, fmt.Errorf("could not submit L1 block. Cause: %w", err)) } @@ -450,14 +450,14 @@ func (e *enclaveImpl) SubmitL1Block(ctx context.Context, block *common.L1Block, } func (e *enclaveImpl) ingestL1Block(ctx context.Context, br *common.BlockAndReceipts) (*components.BlockIngestionType, error) { - e.logger.Info("Start ingesting block", log.BlockHashKey, br.Block.Hash()) + e.logger.Info("Start ingesting block", log.BlockHashKey, br.BlockHeader.Hash()) ingestion, err := e.l1BlockProcessor.Process(ctx, br) if err != nil { // only warn for unexpected errors if errors.Is(err, errutil.ErrBlockAncestorNotFound) || errors.Is(err, errutil.ErrBlockAlreadyProcessed) { - e.logger.Debug("Did not ingest block", log.ErrKey, err, log.BlockHashKey, br.Block.Hash()) + e.logger.Debug("Did not ingest block", log.ErrKey, err, log.BlockHashKey, br.BlockHeader.Hash()) } else { - e.logger.Warn("Failed ingesting block", log.ErrKey, err, log.BlockHashKey, br.Block.Hash()) + e.logger.Warn("Failed ingesting block", log.ErrKey, err, log.BlockHashKey, br.BlockHeader.Hash()) } return nil, err } diff --git a/go/enclave/gas/oracle.go b/go/enclave/gas/oracle.go index 8ab2cc8d24..8e60d552b6 100644 --- a/go/enclave/gas/oracle.go +++ b/go/enclave/gas/oracle.go @@ -12,9 +12,9 @@ import ( // Oracle - the interface for the future precompiled gas oracle contract // which will expose necessary l1 information. type Oracle interface { - ProcessL1Block(block *types.Block) - EstimateL1StorageGasCost(tx *types.Transaction, block *types.Block) (*big.Int, error) - EstimateL1CostForMsg(args *gethapi.TransactionArgs, block *types.Block) (*big.Int, error) + ProcessL1Block(block *types.Header) + EstimateL1StorageGasCost(tx *types.Transaction, block *types.Header) (*big.Int, error) + EstimateL1CostForMsg(args *gethapi.TransactionArgs, block *types.Header) (*big.Int, error) } type oracle struct { @@ -30,30 +30,30 @@ func NewGasOracle() Oracle { // ProcessL1Block - should be used to update the gas oracle. Currently does not really // fit into phase 1 gas mechanics as the information needs to be available per block. // would be fixed when this becomes a smart contract using the stateDB -func (o *oracle) ProcessL1Block(block *types.Block) { - blockBaseFee := block.BaseFee() +func (o *oracle) ProcessL1Block(block *types.Header) { + blockBaseFee := block.BaseFee if blockBaseFee != nil { o.baseFee = blockBaseFee } } // EstimateL1StorageGasCost - Returns the expected l1 gas cost for a transaction at a given l1 block. -func (o *oracle) EstimateL1StorageGasCost(tx *types.Transaction, block *types.Block) (*big.Int, error) { +func (o *oracle) EstimateL1StorageGasCost(tx *types.Transaction, block *types.Header) (*big.Int, error) { encodedTx, err := rlp.EncodeToBytes(tx) if err != nil { return nil, err } - blockBaseFee := block.BaseFee() + blockBaseFee := block.BaseFee if blockBaseFee == nil { return big.NewInt(0), nil } l1Gas := CalculateL1GasUsed(encodedTx, big.NewInt(0)) - return big.NewInt(0).Mul(l1Gas, block.BaseFee()), nil + return big.NewInt(0).Mul(l1Gas, block.BaseFee), nil } -func (o *oracle) EstimateL1CostForMsg(args *gethapi.TransactionArgs, block *types.Block) (*big.Int, error) { +func (o *oracle) EstimateL1CostForMsg(args *gethapi.TransactionArgs, block *types.Header) (*big.Int, error) { encoded := make([]byte, 0) if args.Data != nil { encoded = append(encoded, *args.Data...) @@ -65,5 +65,5 @@ func (o *oracle) EstimateL1CostForMsg(args *gethapi.TransactionArgs, block *type nonZeroGas := big.NewInt(int64(params.TxDataNonZeroGasEIP2028)) overhead := big.NewInt(0).Mul(big.NewInt(150), nonZeroGas) l1Gas := CalculateL1GasUsed(encoded, overhead) - return big.NewInt(0).Mul(l1Gas, block.BaseFee()), nil + return big.NewInt(0).Mul(l1Gas, block.BaseFee), nil } diff --git a/go/enclave/nodetype/common.go b/go/enclave/nodetype/common.go index c414f63c81..52d0802b77 100644 --- a/go/enclave/nodetype/common.go +++ b/go/enclave/nodetype/common.go @@ -44,7 +44,7 @@ func ExportCrossChainData(ctx context.Context, storage storage.Storage, fromSeqN bundle := &common.ExtCrossChainBundle{ LastBatchHash: batchHash, // unused for now. L1BlockHash: block.Hash(), - L1BlockNum: big.NewInt(0).Set(block.Header().Number), + L1BlockNum: big.NewInt(0).Set(block.Number), CrossChainRootHashes: crossChainHashes, } // todo: check fromSeqNo return bundle, nil diff --git a/go/enclave/nodetype/interfaces.go b/go/enclave/nodetype/interfaces.go index 8b8bc957e9..069c59b834 100644 --- a/go/enclave/nodetype/interfaces.go +++ b/go/enclave/nodetype/interfaces.go @@ -3,6 +3,8 @@ package nodetype import ( "context" + "github.com/ethereum/go-ethereum/core/types" + "github.com/ten-protocol/go-ten/go/common" "github.com/ten-protocol/go-ten/go/enclave/components" "github.com/ten-protocol/go-ten/go/enclave/core" @@ -20,7 +22,7 @@ type NodeType interface { OnL1Fork(ctx context.Context, fork *common.ChainFork) error // OnL1Block - performed after the block was processed - OnL1Block(ctx context.Context, block *common.L1Block, result *components.BlockIngestionType) error + OnL1Block(ctx context.Context, block *types.Header, result *components.BlockIngestionType) error ExportCrossChainData(context.Context, uint64, uint64) (*common.ExtCrossChainBundle, error) diff --git a/go/enclave/nodetype/sequencer.go b/go/enclave/nodetype/sequencer.go index 837ff48478..d9ee58c084 100644 --- a/go/enclave/nodetype/sequencer.go +++ b/go/enclave/nodetype/sequencer.go @@ -133,7 +133,7 @@ func (s *sequencer) CreateBatch(ctx context.Context, skipBatchIfEmpty bool) erro // should only create batches and stateDBs but not commit them to the database, // this is the responsibility of the sequencer. Refactor the code so genesis state // won't be committed by the producer. -func (s *sequencer) createGenesisBatch(ctx context.Context, block *common.L1Block) error { +func (s *sequencer) createGenesisBatch(ctx context.Context, block *types.Header) error { s.logger.Info("Initializing genesis state", log.BlockHashKey, block.Hash()) batch, msgBusTx, err := s.batchProducer.CreateGenesisState( ctx, @@ -200,7 +200,7 @@ func (s *sequencer) createGenesisBatch(ctx context.Context, block *common.L1Bloc return nil } -func (s *sequencer) createNewHeadBatch(ctx context.Context, l1HeadBlock *common.L1Block, skipBatchIfEmpty bool) error { +func (s *sequencer) createNewHeadBatch(ctx context.Context, l1HeadBlock *types.Header, skipBatchIfEmpty bool) error { headBatchSeq := s.batchRegistry.HeadBatchSeq() if headBatchSeq == nil { headBatchSeq = big.NewInt(int64(common.L2GenesisSeqNo)) @@ -353,7 +353,7 @@ func (s *sequencer) CreateRollup(ctx context.Context, lastBatchNo uint64) (*comm if err != nil { return nil, err } - upToL1Height := currentL1Head.NumberU64() - RollupDelay + upToL1Height := currentL1Head.Number.Uint64() - RollupDelay rollup, err := s.rollupProducer.CreateInternalRollup(ctx, lastBatchNo, upToL1Height, rollupLimiter) if err != nil { return nil, err @@ -372,7 +372,7 @@ func (s *sequencer) CreateRollup(ctx context.Context, lastBatchNo uint64) (*comm return extRollup, nil } -func (s *sequencer) duplicateBatches(ctx context.Context, l1Head *types.Block, nonCanonicalL1Path []common.L1BlockHash, canonicalL1Path []common.L1BlockHash) error { +func (s *sequencer) duplicateBatches(ctx context.Context, l1Head *types.Header, nonCanonicalL1Path []common.L1BlockHash, canonicalL1Path []common.L1BlockHash) error { batchesToDuplicate := make([]*common.BatchHeader, 0) batchesToExclude := make(map[uint64]*common.BatchHeader, 0) @@ -513,7 +513,7 @@ func (s *sequencer) signCrossChainBundle(bundle *common.ExtCrossChainBundle) err return nil } -func (s *sequencer) OnL1Block(ctx context.Context, _ *common.L1Block, result *components.BlockIngestionType) error { +func (s *sequencer) OnL1Block(ctx context.Context, block *types.Header, result *components.BlockIngestionType) error { // nothing to do return nil } diff --git a/go/enclave/nodetype/validator.go b/go/enclave/nodetype/validator.go index 0ce7cef746..f833e0e40c 100644 --- a/go/enclave/nodetype/validator.go +++ b/go/enclave/nodetype/validator.go @@ -6,6 +6,8 @@ import ( "fmt" "math/big" + "github.com/ethereum/go-ethereum/core/types" + "github.com/ten-protocol/go-ten/go/enclave/crypto" "github.com/ten-protocol/go-ten/go/enclave/txpool" @@ -185,7 +187,7 @@ func (val *obsValidator) handleGenesis(ctx context.Context, batch *common.BatchH return nil } -func (val *obsValidator) OnL1Block(ctx context.Context, _ *common.L1Block, result *components.BlockIngestionType) error { +func (val *obsValidator) OnL1Block(ctx context.Context, block *types.Header, result *components.BlockIngestionType) error { return val.ExecuteStoredBatches(ctx) } diff --git a/go/enclave/rpc/EstimateGas.go b/go/enclave/rpc/EstimateGas.go index 0873ec35ee..b2197332be 100644 --- a/go/enclave/rpc/EstimateGas.go +++ b/go/enclave/rpc/EstimateGas.go @@ -23,7 +23,7 @@ import ( ) func EstimateGasValidate(reqParams []any, builder *CallBuilder[CallParamsWithBlock, hexutil.Uint64], _ *EncryptionManager) error { - // Parameters are [callMsg, Block number (optional)] + // Parameters are [callMsg, BlockHeader number (optional)] if len(reqParams) < 1 { builder.Err = fmt.Errorf("unexpected number of parameters") return nil @@ -40,10 +40,10 @@ func EstimateGasValidate(reqParams []any, builder *CallBuilder[CallParamsWithBlo return nil } - // extract optional Block number - defaults to the latest Block if not avail + // extract optional BlockHeader number - defaults to the latest BlockHeader if not avail blockNumber, err := gethencoding.ExtractOptionalBlockNumber(reqParams, 1) if err != nil { - builder.Err = fmt.Errorf("unable to extract requested Block number - %w", err) + builder.Err = fmt.Errorf("unable to extract requested BlockHeader number - %w", err) return nil } @@ -68,7 +68,7 @@ func EstimateGasExecute(builder *CallBuilder[CallParamsWithBlock, hexutil.Uint64 } // The message is run through the l1 publishing cost estimation for the current - // known head Block. + // known head BlockHeader. l1Cost, err := rpc.gasOracle.EstimateL1CostForMsg(txArgs, block) if err != nil { return err diff --git a/go/enclave/rpc/GetBalance.go b/go/enclave/rpc/GetBalance.go index 18446a1d2a..5ccfc35e6c 100644 --- a/go/enclave/rpc/GetBalance.go +++ b/go/enclave/rpc/GetBalance.go @@ -32,7 +32,7 @@ func GetBalanceValidate(reqParams []any, builder *CallBuilder[BalanceReq, hexuti blockNumber, err := gethencoding.ExtractBlockNumber(reqParams[1]) if err != nil { - builder.Err = fmt.Errorf("unable to extract requested Block number - %w", err) + builder.Err = fmt.Errorf("unable to extract requested BlockHeader number - %w", err) return nil } builder.Param = &BalanceReq{ diff --git a/go/enclave/rpc/GetTransactionCount.go b/go/enclave/rpc/GetTransactionCount.go index 5282f733fe..8ffb17ee67 100644 --- a/go/enclave/rpc/GetTransactionCount.go +++ b/go/enclave/rpc/GetTransactionCount.go @@ -9,7 +9,7 @@ import ( ) func GetTransactionCountValidate(reqParams []any, builder *CallBuilder[uint64, string], rpc *EncryptionManager) error { - // Parameters are [Address, Block?] + // Parameters are [Address, BlockHeader?] if len(reqParams) < 1 { builder.Err = fmt.Errorf("unexpected number of parameters") return nil diff --git a/go/enclave/rpc_server.go b/go/enclave/rpc_server.go index f2f0ad6a50..0ae63f8965 100644 --- a/go/enclave/rpc_server.go +++ b/go/enclave/rpc_server.go @@ -478,8 +478,8 @@ func (s *RPCServer) EnclavePublicConfig(ctx context.Context, _ *generated.Enclav return &generated.EnclavePublicConfigResponse{L2MessageBusAddress: enclaveCfg.L2MessageBusAddress.Bytes()}, nil } -func (s *RPCServer) decodeBlock(encodedBlock []byte) (*types.Block, error) { - block := types.Block{} +func (s *RPCServer) decodeBlock(encodedBlock []byte) (*types.Header, error) { + block := types.Header{} err := rlp.DecodeBytes(encodedBlock, &block) if err != nil { return nil, fmt.Errorf("unable to decode block, bytes=%x, err=%w", encodedBlock, err) @@ -488,8 +488,8 @@ func (s *RPCServer) decodeBlock(encodedBlock []byte) (*types.Block, error) { } // decodeReceipts - converts the rlp encoded bytes to receipts if possible. -func (s *RPCServer) decodeReceipts(encodedReceipts []byte) (types.Receipts, error) { - receipts := make(types.Receipts, 0) +func (s *RPCServer) decodeReceipts(encodedReceipts []byte) ([]*common.TxAndReceipt, error) { + receipts := make([]*common.TxAndReceipt, 0) err := rlp.DecodeBytes(encodedReceipts, &receipts) if err != nil { diff --git a/go/enclave/storage/cache_service.go b/go/enclave/storage/cache_service.go index 8982434aa6..6085980d1c 100644 --- a/go/enclave/storage/cache_service.go +++ b/go/enclave/storage/cache_service.go @@ -30,7 +30,7 @@ const ( type CacheService struct { // cache for the immutable blocks and batches. // this avoids a trip to the database. - blockCache *cache.Cache[*types.Block] + blockCache *cache.Cache[*types.Header] // stores batches using the sequence number as key batchCacheBySeqNo *cache.Cache[*common.BatchHeader] @@ -71,7 +71,7 @@ func NewCacheService(logger gethlog.Logger) *CacheService { } ristrettoStore := ristretto_store.NewRistretto(ristrettoCache) return &CacheService{ - blockCache: cache.New[*types.Block](ristrettoStore), + blockCache: cache.New[*types.Header](ristrettoStore), batchCacheBySeqNo: cache.New[*common.BatchHeader](ristrettoStore), seqCacheByHash: cache.New[*big.Int](ristrettoStore), seqCacheByHeight: cache.New[*big.Int](ristrettoStore), @@ -84,7 +84,7 @@ func NewCacheService(logger gethlog.Logger) *CacheService { } } -func (cs *CacheService) CacheBlock(ctx context.Context, b *types.Block) { +func (cs *CacheService) CacheBlock(ctx context.Context, b *types.Header) { cacheValue(ctx, cs.blockCache, cs.logger, b.Hash(), b, blockCost) } @@ -96,7 +96,7 @@ func (cs *CacheService) CacheBatch(ctx context.Context, batch *core.Batch) { cacheValue(ctx, cs.seqCacheByHeight, cs.logger, batch.NumberU64()+1, batch.SeqNo(), idCost) } -func (cs *CacheService) ReadBlock(ctx context.Context, key gethcommon.Hash, onCacheMiss func(any) (*types.Block, error)) (*types.Block, error) { +func (cs *CacheService) ReadBlock(ctx context.Context, key gethcommon.Hash, onCacheMiss func(any) (*types.Header, error)) (*types.Header, error) { return getCachedValue(ctx, cs.blockCache, cs.logger, key, blockCost, onCacheMiss) } diff --git a/go/enclave/storage/enclavedb/block.go b/go/enclave/storage/enclavedb/block.go index 2e9cd6e35e..e2b2267fa9 100644 --- a/go/enclave/storage/enclavedb/block.go +++ b/go/enclave/storage/enclavedb/block.go @@ -83,12 +83,11 @@ func HandleBlockArrivedAfterBatches(ctx context.Context, dbtx *sql.Tx, blockId i return err } -// todo - remove this. For now creates a "block" but without a body. -func FetchBlock(ctx context.Context, db *sql.DB, hash common.L1BlockHash) (*types.Block, error) { +func FetchBlockHeader(ctx context.Context, db *sql.DB, hash common.L1BlockHash) (*types.Header, error) { return fetchBlock(ctx, db, " where hash=?", hash.Bytes()) } -func FetchHeadBlock(ctx context.Context, db *sql.DB) (*types.Block, error) { +func FetchHeadBlock(ctx context.Context, db *sql.DB) (*types.Header, error) { return fetchBlock(ctx, db, "order by id desc limit 1") } @@ -242,10 +241,6 @@ func fetchBlockHeader(ctx context.Context, db *sql.DB, whereQuery string, args . return h, nil } -func fetchBlock(ctx context.Context, db *sql.DB, whereQuery string, args ...any) (*types.Block, error) { - h, err := fetchBlockHeader(ctx, db, whereQuery, args...) - if err != nil { - return nil, err - } - return types.NewBlockWithHeader(h), nil +func fetchBlock(ctx context.Context, db *sql.DB, whereQuery string, args ...any) (*types.Header, error) { + return fetchBlockHeader(ctx, db, whereQuery, args...) } diff --git a/go/enclave/storage/interfaces.go b/go/enclave/storage/interfaces.go index fb7ac7b73a..e71efaef26 100644 --- a/go/enclave/storage/interfaces.go +++ b/go/enclave/storage/interfaces.go @@ -20,21 +20,21 @@ import ( // BlockResolver stores new blocks and returns information on existing blocks type BlockResolver interface { - // FetchBlock returns the L1 Block with the given hash. - FetchBlock(ctx context.Context, blockHash common.L1BlockHash) (*types.Block, error) + // FetchBlock returns the L1 BlockHeader with the given hash. + FetchBlock(ctx context.Context, blockHash common.L1BlockHash) (*types.Header, error) IsBlockCanonical(ctx context.Context, blockHash common.L1BlockHash) (bool, error) // FetchCanonicaBlockByHeight - self explanatory - FetchCanonicaBlockByHeight(ctx context.Context, height *big.Int) (*types.Block, error) + FetchCanonicaBlockByHeight(ctx context.Context, height *big.Int) (*types.Header, error) // FetchHeadBlock - returns the head of the current chain. - FetchHeadBlock(ctx context.Context) (*types.Block, error) - // StoreBlock persists the L1 Block and updates the canonical ancestors if there was a fork - StoreBlock(ctx context.Context, block *types.Block, fork *common.ChainFork) error - // IsAncestor returns true if maybeAncestor is an ancestor of the L1 Block, and false otherwise - IsAncestor(ctx context.Context, block *types.Block, maybeAncestor *types.Block) bool - // IsBlockAncestor returns true if maybeAncestor is an ancestor of the L1 Block, and false otherwise - // Takes into consideration that the Block to verify might be on a branch we haven't received yet + FetchHeadBlock(ctx context.Context) (*types.Header, error) + // StoreBlock persists the L1 BlockHeader and updates the canonical ancestors if there was a fork + StoreBlock(ctx context.Context, block *types.Header, fork *common.ChainFork) error + // IsAncestor returns true if maybeAncestor is an ancestor of the L1 BlockHeader, and false otherwise + IsAncestor(ctx context.Context, block *types.Header, maybeAncestor *types.Header) bool + // IsBlockAncestor returns true if maybeAncestor is an ancestor of the L1 BlockHeader, and false otherwise + // Takes into consideration that the BlockHeader to verify might be on a branch we haven't received yet // todo (low priority) - this is super confusing, analyze the usage - IsBlockAncestor(ctx context.Context, block *types.Block, maybeAncestor common.L1BlockHash) bool + IsBlockAncestor(ctx context.Context, block *types.Header, maybeAncestor common.L1BlockHash) bool } type BatchResolver interface { diff --git a/go/enclave/storage/storage.go b/go/enclave/storage/storage.go index 244ffd7c5d..424f318997 100644 --- a/go/enclave/storage/storage.go +++ b/go/enclave/storage/storage.go @@ -219,7 +219,7 @@ func (s *storageImpl) IsBatchCanonical(ctx context.Context, seq uint64) (bool, e return enclavedb.IsCanonicalBatchSeq(ctx, s.db.GetSQLDB(), seq) } -func (s *storageImpl) StoreBlock(ctx context.Context, block *types.Block, chainFork *common.ChainFork) error { +func (s *storageImpl) StoreBlock(ctx context.Context, block *types.Header, chainFork *common.ChainFork) error { defer s.logDuration("StoreBlock", measure.NewStopwatch()) dbTx, err := s.db.NewDBTransaction(ctx) if err != nil { @@ -230,7 +230,7 @@ func (s *storageImpl) StoreBlock(ctx context.Context, block *types.Block, chainF // only insert the block if it doesn't exist already blockId, err := enclavedb.GetBlockId(ctx, dbTx, block.Hash()) if errors.Is(err, sql.ErrNoRows) { - if err := enclavedb.WriteBlock(ctx, dbTx, block.Header()); err != nil { + if err := enclavedb.WriteBlock(ctx, dbTx, block); err != nil { return fmt.Errorf("2. could not store block %s. Cause: %w", block.Hash(), err) } @@ -282,10 +282,10 @@ func (s *storageImpl) StoreBlock(ctx context.Context, block *types.Block, chainF return nil } -func (s *storageImpl) FetchBlock(ctx context.Context, blockHash common.L1BlockHash) (*types.Block, error) { - defer s.logDuration("FetchBlock", measure.NewStopwatch()) - return s.cachingService.ReadBlock(ctx, blockHash, func(hash any) (*types.Block, error) { - return enclavedb.FetchBlock(ctx, s.db.GetSQLDB(), hash.(common.L1BlockHash)) +func (s *storageImpl) FetchBlock(ctx context.Context, blockHash common.L1BlockHash) (*types.Header, error) { + defer s.logDuration("FetchBlockHeader", measure.NewStopwatch()) + return s.cachingService.ReadBlock(ctx, blockHash, func(hash any) (*types.Header, error) { + return enclavedb.FetchBlockHeader(ctx, s.db.GetSQLDB(), hash.(common.L1BlockHash)) }) } @@ -299,7 +299,7 @@ func (s *storageImpl) IsBlockCanonical(ctx context.Context, blockHash common.L1B return enclavedb.IsCanonicalBlock(ctx, dbtx, &blockHash) } -func (s *storageImpl) FetchCanonicaBlockByHeight(ctx context.Context, height *big.Int) (*types.Block, error) { +func (s *storageImpl) FetchCanonicaBlockByHeight(ctx context.Context, height *big.Int) (*types.Header, error) { defer s.logDuration("FetchCanonicaBlockByHeight", measure.NewStopwatch()) header, err := enclavedb.FetchBlockHeaderByHeight(ctx, s.db.GetSQLDB(), height) if err != nil { @@ -308,7 +308,7 @@ func (s *storageImpl) FetchCanonicaBlockByHeight(ctx context.Context, height *bi return s.FetchBlock(ctx, header.Hash()) } -func (s *storageImpl) FetchHeadBlock(ctx context.Context) (*types.Block, error) { +func (s *storageImpl) FetchHeadBlock(ctx context.Context) (*types.Header, error) { defer s.logDuration("FetchHeadBlock", measure.NewStopwatch()) return enclavedb.FetchHeadBlock(ctx, s.db.GetSQLDB()) } @@ -356,26 +356,26 @@ func (s *storageImpl) FetchSecret(ctx context.Context) (*crypto.SharedEnclaveSec return s.cachedSharedSecret, nil } -func (s *storageImpl) IsAncestor(ctx context.Context, block *types.Block, maybeAncestor *types.Block) bool { +func (s *storageImpl) IsAncestor(ctx context.Context, block *types.Header, maybeAncestor *types.Header) bool { defer s.logDuration("IsAncestor", measure.NewStopwatch()) if bytes.Equal(maybeAncestor.Hash().Bytes(), block.Hash().Bytes()) { return true } - if maybeAncestor.NumberU64() >= block.NumberU64() { + if maybeAncestor.Number.Uint64() >= block.Number.Uint64() { return false } - p, err := s.FetchBlock(ctx, block.ParentHash()) + p, err := s.FetchBlock(ctx, block.ParentHash) if err != nil { - s.logger.Debug("Could not find block with hash", log.BlockHashKey, block.ParentHash(), log.ErrKey, err) + s.logger.Debug("Could not find block with hash", log.BlockHashKey, block.ParentHash, log.ErrKey, err) return false } return s.IsAncestor(ctx, p, maybeAncestor) } -func (s *storageImpl) IsBlockAncestor(ctx context.Context, block *types.Block, maybeAncestor common.L1BlockHash) bool { +func (s *storageImpl) IsBlockAncestor(ctx context.Context, block *types.Header, maybeAncestor common.L1BlockHash) bool { defer s.logDuration("IsBlockAncestor", measure.NewStopwatch()) resolvedBlock, err := s.FetchBlock(ctx, maybeAncestor) if err != nil { diff --git a/go/ethadapter/geth_rpc_client.go b/go/ethadapter/geth_rpc_client.go index 983ab08877..e9b51a9640 100644 --- a/go/ethadapter/geth_rpc_client.go +++ b/go/ethadapter/geth_rpc_client.go @@ -84,7 +84,7 @@ func (e *gethRPCClient) Info() Info { } } -func (e *gethRPCClient) BlocksBetween(startingBlock *types.Block, lastBlock *types.Block) []*types.Block { +func (e *gethRPCClient) BlocksBetween(startingBlock *types.Header, lastBlock *types.Block) []*types.Block { var blocksBetween []*types.Block var err error diff --git a/go/ethadapter/interface.go b/go/ethadapter/interface.go index 31a1071869..1245da18d4 100644 --- a/go/ethadapter/interface.go +++ b/go/ethadapter/interface.go @@ -25,11 +25,11 @@ type EthClient interface { BalanceAt(account gethcommon.Address, blockNumber *big.Int) (*big.Int, error) // fetches the balance of the account GetLogs(q ethereum.FilterQuery) ([]types.Log, error) // fetches the logs for a given query - Info() Info // retrieves the node Info - FetchHeadBlock() (*types.Block, error) // retrieves the block at head height - BlocksBetween(block *types.Block, head *types.Block) []*types.Block // returns the blocks between two blocks - IsBlockAncestor(block *types.Block, proof common.L1BlockHash) bool // returns if the node considers a block the ancestor - BlockListener() (chan *types.Header, ethereum.Subscription) // subscribes to new blocks and returns a listener with the blocks heads and the subscription handler + Info() Info // retrieves the node Info + FetchHeadBlock() (*types.Block, error) // retrieves the block at head height + BlocksBetween(block *types.Header, head *types.Block) []*types.Block // returns the blocks between two blocks + IsBlockAncestor(block *types.Block, proof common.L1BlockHash) bool // returns if the node considers a block the ancestor + BlockListener() (chan *types.Header, ethereum.Subscription) // subscribes to new blocks and returns a listener with the blocks heads and the subscription handler CallContract(msg ethereum.CallMsg) ([]byte, error) // Runs the provided call message on the latest block. diff --git a/go/host/enclave/guardian.go b/go/host/enclave/guardian.go index a6ccb8bbed..bd6137ad94 100644 --- a/go/host/enclave/guardian.go +++ b/go/host/enclave/guardian.go @@ -425,16 +425,21 @@ func (g *Guardian) submitL1Block(block *common.L1Block, isLatest bool) (bool, er g.submitDataLock.Unlock() // lock must be released before returning return false, fmt.Errorf("could not fetch obscuro receipts for block=%s - %w", block.Hash(), err) } + txWithReceipts := make([]*common.TxAndReceipt, 0) // only submit the relevant transactions to the enclave // nullify all non-relevant transactions txs := block.Transactions() for i, rec := range receipts { - if rec == nil { - txs[i] = nil + // the FetchObscuroReceipts method returns dummy receipts on non-relevant positions. + if rec.BlockNumber != nil { + txWithReceipts = append(txWithReceipts, &common.TxAndReceipt{ + Tx: txs[i], + Receipt: rec, + }) } } - resp, err := g.enclaveClient.SubmitL1Block(context.Background(), block, receipts, isLatest) + resp, err := g.enclaveClient.SubmitL1Block(context.Background(), block.Header(), txWithReceipts, isLatest) g.submitDataLock.Unlock() // lock is only guarding the enclave call, so we can release it now if err != nil { if strings.Contains(err.Error(), errutil.ErrBlockAlreadyProcessed.Error()) { diff --git a/go/host/l1/blockrepository.go b/go/host/l1/blockrepository.go index bd0a2ad4fc..fdae951245 100644 --- a/go/host/l1/blockrepository.go +++ b/go/host/l1/blockrepository.go @@ -131,6 +131,9 @@ func (r *Repository) latestCanonAncestor(blkHash gethcommon.Hash) (*types.Block, // FetchObscuroReceipts returns all obscuro-relevant receipts for an L1 block func (r *Repository) FetchObscuroReceipts(block *common.L1Block) (types.Receipts, error) { receipts := make([]*types.Receipt, len(block.Transactions())) + if len(block.Transactions()) == 0 { + return receipts, nil + } blkHash := block.Hash() // we want to send receipts for any transactions that produced obscuro-relevant log events diff --git a/go/host/rpc/enclaverpc/enclave_client.go b/go/host/rpc/enclaverpc/enclave_client.go index cef24fb274..1eaa67d1ee 100644 --- a/go/host/rpc/enclaverpc/enclave_client.go +++ b/go/host/rpc/enclaverpc/enclave_client.go @@ -8,6 +8,8 @@ import ( "math/big" "time" + "github.com/ethereum/go-ethereum/core/types" + "github.com/ten-protocol/go-ten/go/enclave/core" "github.com/ethereum/go-ethereum/rlp" @@ -195,9 +197,9 @@ func (c *Client) EnclaveID(ctx context.Context) (common.EnclaveID, common.System return common.EnclaveID(response.EnclaveID), nil } -func (c *Client) SubmitL1Block(ctx context.Context, block *common.L1Block, receipts common.L1Receipts, isLatest bool) (*common.BlockSubmissionResponse, common.SystemError) { +func (c *Client) SubmitL1Block(ctx context.Context, blockHeader *types.Header, receipts []*common.TxAndReceipt, isLatest bool) (*common.BlockSubmissionResponse, common.SystemError) { var buffer bytes.Buffer - if err := block.EncodeRLP(&buffer); err != nil { + if err := blockHeader.EncodeRLP(&buffer); err != nil { return nil, fmt.Errorf("could not encode block. Cause: %w", err) } diff --git a/integration/ethereummock/db.go b/integration/ethereummock/db.go index 76db7ff012..d3491e79b9 100644 --- a/integration/ethereummock/db.go +++ b/integration/ethereummock/db.go @@ -6,8 +6,6 @@ import ( "math/big" "sync" - "github.com/ten-protocol/go-ten/go/enclave/storage" - "github.com/ten-protocol/go-ten/go/common/log" "github.com/ten-protocol/go-ten/go/common/errutil" @@ -37,7 +35,7 @@ func (n *blockResolverInMem) Proof(_ context.Context, _ *core.Rollup) (*types.Bl panic("implement me") } -func NewResolver() storage.BlockResolver { +func NewResolver() *blockResolverInMem { return &blockResolverInMem{ blockCache: map[common.L1BlockHash]*types.Block{}, m: sync.RWMutex{}, @@ -161,7 +159,7 @@ func (m *Node) removeCommittedTransactions( ctx context.Context, cb *types.Block, mempool []*types.Transaction, - resolver storage.BlockResolver, + resolver *blockResolverInMem, db TxDB, ) []*types.Transaction { if cb.NumberU64() <= common.HeightCommittedBlocks { diff --git a/integration/ethereummock/gethutil.go b/integration/ethereummock/gethutil.go new file mode 100644 index 0000000000..18a8d6068d --- /dev/null +++ b/integration/ethereummock/gethutil.go @@ -0,0 +1,64 @@ +package ethereummock + +import ( + "context" + "fmt" + + gethcommon "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/core/types" + "github.com/ten-protocol/go-ten/go/common" +) + +// Utilities for working with geth structures + +// EmptyHash is useful for comparisons to check if hash has been set +var EmptyHash = gethcommon.Hash{} + +// LCA - returns the latest common ancestor of the 2 blocks or an error if no common ancestor is found +// it also returns the blocks that became canonical, and the once that are now the fork +func LCA(ctx context.Context, newCanonical *types.Block, oldCanonical *types.Block, resolver *blockResolverInMem) (*common.ChainFork, error) { + b, cp, ncp, err := internalLCA(ctx, newCanonical, oldCanonical, resolver, []common.L1BlockHash{}, []common.L1BlockHash{}) + return &common.ChainFork{ + NewCanonical: newCanonical.Header(), + OldCanonical: oldCanonical.Header(), + CommonAncestor: b.Header(), + CanonicalPath: cp, + NonCanonicalPath: ncp, + }, err +} + +func internalLCA(ctx context.Context, newCanonical *types.Block, oldCanonical *types.Block, resolver *blockResolverInMem, canonicalPath []common.L1BlockHash, nonCanonicalPath []common.L1BlockHash) (*types.Block, []common.L1BlockHash, []common.L1BlockHash, error) { + if newCanonical.NumberU64() == common.L1GenesisHeight || oldCanonical.NumberU64() == common.L1GenesisHeight { + return newCanonical, canonicalPath, nonCanonicalPath, nil + } + if newCanonical.Hash() == oldCanonical.Hash() { + // this is where we reach the common ancestor, which we add to the canonical path + return newCanonical, append(canonicalPath, newCanonical.Hash()), nonCanonicalPath, nil + } + if newCanonical.NumberU64() > oldCanonical.NumberU64() { + p, err := resolver.FetchBlock(ctx, newCanonical.ParentHash()) + if err != nil { + return nil, nil, nil, fmt.Errorf("could not retrieve parent block %s. Cause: %w", newCanonical.ParentHash, err) + } + + return internalLCA(ctx, p, oldCanonical, resolver, append(canonicalPath, newCanonical.Hash()), nonCanonicalPath) + } + if oldCanonical.NumberU64() > newCanonical.NumberU64() { + p, err := resolver.FetchBlock(ctx, oldCanonical.ParentHash()) + if err != nil { + return nil, nil, nil, fmt.Errorf("could not retrieve parent block %s. Cause: %w", oldCanonical.ParentHash, err) + } + + return internalLCA(ctx, newCanonical, p, resolver, canonicalPath, append(nonCanonicalPath, oldCanonical.Hash())) + } + parentBlockA, err := resolver.FetchBlock(ctx, newCanonical.ParentHash()) + if err != nil { + return nil, nil, nil, fmt.Errorf("could not retrieve parent block %s. Cause: %w", newCanonical.ParentHash, err) + } + parentBlockB, err := resolver.FetchBlock(ctx, oldCanonical.ParentHash()) + if err != nil { + return nil, nil, nil, fmt.Errorf("could not retrieve parent block %s. Cause: %w", oldCanonical.ParentHash, err) + } + + return internalLCA(ctx, parentBlockA, parentBlockB, resolver, append(canonicalPath, newCanonical.Hash()), append(nonCanonicalPath, oldCanonical.Hash())) +} diff --git a/integration/ethereummock/node.go b/integration/ethereummock/node.go index 669c2de5c1..457258fb68 100644 --- a/integration/ethereummock/node.go +++ b/integration/ethereummock/node.go @@ -10,16 +10,12 @@ import ( "sync/atomic" "time" - "github.com/ten-protocol/go-ten/go/enclave/storage" - "github.com/ten-protocol/go-ten/go/common/async" "github.com/google/uuid" "github.com/ten-protocol/go-ten/go/common/errutil" - "github.com/ten-protocol/go-ten/go/common/gethutil" - gethlog "github.com/ethereum/go-ethereum/log" "github.com/ten-protocol/go-ten/go/common/log" @@ -63,7 +59,7 @@ type Node struct { Network L1Network mining bool stats StatsCollector - Resolver storage.BlockResolver + Resolver *blockResolverInMem db TxDB subs map[uuid.UUID]*mockSubscription // active subscription for mock blocks subMu sync.Mutex @@ -111,7 +107,8 @@ func (m *Node) SendTransaction(tx *types.Transaction) error { func (m *Node) TransactionReceipt(_ gethcommon.Hash) (*types.Receipt, error) { // all transactions are immediately processed return &types.Receipt{ - Status: types.ReceiptStatusSuccessful, + BlockNumber: big.NewInt(1), + Status: types.ReceiptStatusSuccessful, }, nil } @@ -327,15 +324,14 @@ func (m *Node) processBlock(b *types.Block, head *types.Block) *types.Block { // Check for Reorgs if !m.Resolver.IsAncestor(context.Background(), b, head) { m.stats.L1Reorg(m.l2ID) - fork, err := gethutil.LCA(context.Background(), head, b, m.Resolver) + fork, err := LCA(context.Background(), head, b, m.Resolver) if err != nil { panic(err) } m.logger.Info( - fmt.Sprintf("L1Reorg new=b_%d(%d), old=b_%d(%d), fork=b_%d(%d)", common.ShortHash(b.Hash()), b.NumberU64(), common.ShortHash(head.Hash()), head.NumberU64(), common.ShortHash(fork.CommonAncestor.Hash()), fork.CommonAncestor.NumberU64())) + fmt.Sprintf("L1Reorg new=b_%d(%d), old=b_%d(%d), fork=b_%d(%d)", common.ShortHash(b.Hash()), b.NumberU64(), common.ShortHash(head.Hash()), head.NumberU64(), common.ShortHash(fork.CommonAncestor.Hash()), fork.CommonAncestor.Number.Uint64())) return m.setFork(m.BlocksBetween(fork.CommonAncestor, b)) } - if b.NumberU64() > (head.NumberU64() + 1) { m.logger.Crit("Should not happen") } @@ -462,9 +458,9 @@ func (m *Node) Stop() { m.exitCh <- true } -func (m *Node) BlocksBetween(blockA *types.Block, blockB *types.Block) []*types.Block { +func (m *Node) BlocksBetween(blockA *types.Header, blockB *types.Block) []*types.Block { if bytes.Equal(blockA.Hash().Bytes(), blockB.Hash().Bytes()) { - return []*types.Block{blockA} + return []*types.Block{blockB} } blocks := make([]*types.Block, 0) tempBlock := blockB diff --git a/integration/ethereummock/utils.go b/integration/ethereummock/utils.go index 326f478155..3bad980648 100644 --- a/integration/ethereummock/utils.go +++ b/integration/ethereummock/utils.go @@ -4,8 +4,6 @@ import ( "context" "fmt" - "github.com/ten-protocol/go-ten/go/enclave/storage" - "github.com/ten-protocol/go-ten/go/common" "github.com/ethereum/go-ethereum/core/types" @@ -13,12 +11,12 @@ import ( // findNotIncludedTxs - given a list of transactions, it keeps only the ones that were not included in the block // todo (#1491) - inefficient -func findNotIncludedTxs(head *types.Block, txs []*types.Transaction, r storage.BlockResolver, db TxDB) []*types.Transaction { +func findNotIncludedTxs(head *types.Block, txs []*types.Transaction, r *blockResolverInMem, db TxDB) []*types.Transaction { included := allIncludedTransactions(head, r, db) return removeExisting(txs, included) } -func allIncludedTransactions(b *types.Block, r storage.BlockResolver, db TxDB) map[common.TxHash]*types.Transaction { +func allIncludedTransactions(b *types.Block, r *blockResolverInMem, db TxDB) map[common.TxHash]*types.Transaction { val, found := db.Txs(b) if found { return val diff --git a/integration/simulation/simulation.go b/integration/simulation/simulation.go index 6bacf215c5..d95a3d16aa 100644 --- a/integration/simulation/simulation.go +++ b/integration/simulation/simulation.go @@ -102,7 +102,7 @@ func (s *Simulation) waitForTenGenesisOnL1() { panic(fmt.Errorf("could not fetch head block. Cause: %w", err)) } if err == nil { - for _, b := range client.BlocksBetween(ethereummock.MockGenesisBlock, head) { + for _, b := range client.BlocksBetween(ethereummock.MockGenesisBlock.Header(), head) { for _, tx := range b.Transactions() { t := s.Params.MgmtContractLib.DecodeTx(tx) if t == nil { diff --git a/integration/simulation/validate_chain.go b/integration/simulation/validate_chain.go index a4ebc91c5f..24d42254b7 100644 --- a/integration/simulation/validate_chain.go +++ b/integration/simulation/validate_chain.go @@ -255,7 +255,7 @@ func ExtractDataFromEthereumChain( rollupReceipts := make(types.Receipts, 0) totalDeposited := big.NewInt(0) - blockchain := node.BlocksBetween(startBlock, endBlock) + blockchain := node.BlocksBetween(startBlock.Header(), endBlock) successfulDeposits := uint64(0) for _, block := range blockchain { for _, tx := range block.Transactions() { diff --git a/tools/walletextension/main/main.go b/tools/walletextension/main/main.go index 871ee4e5cb..217a6c08f5 100644 --- a/tools/walletextension/main/main.go +++ b/tools/walletextension/main/main.go @@ -17,7 +17,7 @@ const ( tcp = "tcp" // @fixme - // this is a temporary fix as out forked version of log.go does not map with gethlog.Level - //and should be fixed as part of logging refactoring in the future + // and should be fixed as part of logging refactoring in the future legacyLevelDebug = 4 legacyLevelError = 1 ) From ce7fd993d78fea345c0d7253a691f8b2faea3f92 Mon Sep 17 00:00:00 2001 From: Tudor Malene Date: Thu, 5 Sep 2024 17:36:44 +0300 Subject: [PATCH 2/3] fix --- integration/ethereummock/gethutil.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/integration/ethereummock/gethutil.go b/integration/ethereummock/gethutil.go index 18a8d6068d..433737e7ea 100644 --- a/integration/ethereummock/gethutil.go +++ b/integration/ethereummock/gethutil.go @@ -38,7 +38,7 @@ func internalLCA(ctx context.Context, newCanonical *types.Block, oldCanonical *t if newCanonical.NumberU64() > oldCanonical.NumberU64() { p, err := resolver.FetchBlock(ctx, newCanonical.ParentHash()) if err != nil { - return nil, nil, nil, fmt.Errorf("could not retrieve parent block %s. Cause: %w", newCanonical.ParentHash, err) + return nil, nil, nil, fmt.Errorf("could not retrieve parent block %s. Cause: %w", newCanonical.ParentHash(), err) } return internalLCA(ctx, p, oldCanonical, resolver, append(canonicalPath, newCanonical.Hash()), nonCanonicalPath) @@ -46,18 +46,18 @@ func internalLCA(ctx context.Context, newCanonical *types.Block, oldCanonical *t if oldCanonical.NumberU64() > newCanonical.NumberU64() { p, err := resolver.FetchBlock(ctx, oldCanonical.ParentHash()) if err != nil { - return nil, nil, nil, fmt.Errorf("could not retrieve parent block %s. Cause: %w", oldCanonical.ParentHash, err) + return nil, nil, nil, fmt.Errorf("could not retrieve parent block %s. Cause: %w", oldCanonical.ParentHash(), err) } return internalLCA(ctx, newCanonical, p, resolver, canonicalPath, append(nonCanonicalPath, oldCanonical.Hash())) } parentBlockA, err := resolver.FetchBlock(ctx, newCanonical.ParentHash()) if err != nil { - return nil, nil, nil, fmt.Errorf("could not retrieve parent block %s. Cause: %w", newCanonical.ParentHash, err) + return nil, nil, nil, fmt.Errorf("could not retrieve parent block %s. Cause: %w", newCanonical.ParentHash(), err) } parentBlockB, err := resolver.FetchBlock(ctx, oldCanonical.ParentHash()) if err != nil { - return nil, nil, nil, fmt.Errorf("could not retrieve parent block %s. Cause: %w", oldCanonical.ParentHash, err) + return nil, nil, nil, fmt.Errorf("could not retrieve parent block %s. Cause: %w", oldCanonical.ParentHash(), err) } return internalLCA(ctx, parentBlockA, parentBlockB, resolver, append(canonicalPath, newCanonical.Hash()), append(nonCanonicalPath, oldCanonical.Hash())) From f7b586bbcd363017819777d1ae7e9f580a8f714d Mon Sep 17 00:00:00 2001 From: Tudor Malene Date: Thu, 5 Sep 2024 17:38:33 +0300 Subject: [PATCH 3/3] fix --- go/enclave/components/consumer_test.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/go/enclave/components/consumer_test.go b/go/enclave/components/consumer_test.go index 34b3032705..b6cac1ac96 100644 --- a/go/enclave/components/consumer_test.go +++ b/go/enclave/components/consumer_test.go @@ -8,7 +8,6 @@ import ( "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/core" "github.com/ethereum/go-ethereum/core/types" - "github.com/ethereum/go-ethereum/trie" ) func TestInvalidBlocksAreRejected(t *testing.T) { @@ -25,7 +24,7 @@ func TestInvalidBlocksAreRejected(t *testing.T) { for _, header := range invalidHeaders { loopHeader := header - _, err := blockConsumer.ingestBlock(context.Background(), types.NewBlock(&loopHeader, nil, nil, &trie.StackTrie{})) + _, err := blockConsumer.ingestBlock(context.Background(), &loopHeader) if err == nil { t.Errorf("expected block with invalid header to be rejected but was accepted") }