diff --git a/go/common/types.go b/go/common/types.go index dea6606040..30ecd729ac 100644 --- a/go/common/types.go +++ b/go/common/types.go @@ -90,11 +90,11 @@ func (txs L2PricedTransactions) ToTransactions() types.Transactions { } const ( - L2GenesisHeight = uint64(0) L1GenesisHeight = uint64(0) - L2GenesisSeqNo = uint64(1) - // HeightCommittedBlocks is the number of blocks deep a transaction must be to be considered safe from reorganisations. - HeightCommittedBlocks = 15 + + L2GenesisHeight = uint64(0) + L2GenesisSeqNo = uint64(1) + L2SysContractGenesisSeqNo = uint64(2) ) var GethGenesisParentHash = common.Hash{} diff --git a/go/enclave/components/batch_executor.go b/go/enclave/components/batch_executor.go index b894fc4636..82f771439a 100644 --- a/go/enclave/components/batch_executor.go +++ b/go/enclave/components/batch_executor.go @@ -9,6 +9,8 @@ import ( "sort" "sync" + "github.com/ethereum/go-ethereum/trie" + "github.com/ten-protocol/go-ten/go/enclave/crypto" "github.com/ten-protocol/go-ten/lib/gethfork/rpc" @@ -24,11 +26,9 @@ import ( gethcommon "github.com/ethereum/go-ethereum/common" smt "github.com/FantasyJony/openzeppelin-merkle-tree-go/standard_merkle_tree" - "github.com/ethereum/go-ethereum/core/state" "github.com/ethereum/go-ethereum/core/types" gethlog "github.com/ethereum/go-ethereum/log" "github.com/ethereum/go-ethereum/params" - "github.com/ethereum/go-ethereum/trie" "github.com/ten-protocol/go-ten/go/common" "github.com/ten-protocol/go-ten/go/common/errutil" "github.com/ten-protocol/go-ten/go/common/log" @@ -91,117 +91,197 @@ func NewBatchExecutor( } } -// filterTransactionsWithSufficientFunds - this function estimates hte l1 fees for the transaction in a given batch execution context. It does so by taking the price of the -// pinned L1 block and using it as the cost per gas for the estimated gas of the calldata encoding of a transaction. It filters out any transactions that cannot afford to pay for their L1 -// publishing cost. -func (executor *batchExecutor) filterTransactionsWithSufficientFunds(ctx context.Context, stateDB *state.StateDB, context *BatchExecutionContext) (common.L2PricedTransactions, common.L2PricedTransactions) { - transactions := make(common.L2PricedTransactions, 0) - freeTransactions := make(common.L2PricedTransactions, 0) - block, _ := executor.storage.FetchBlock(ctx, context.BlockPtr) - - for _, tx := range context.Transactions { - // Transactions that are created inside the enclave can have no GasPrice set. - // External transactions are always required to have a gas price set. Thus we filter - // those transactions for separate processing than the normal ones and we run them through the EVM - // with a flag that disables the baseFee logic and wont fail them for having price lower than the base fee. - isFreeTransaction := tx.GasFeeCap().Cmp(gethcommon.Big0) == 0 - isFreeTransaction = isFreeTransaction && tx.GasPrice().Cmp(gethcommon.Big0) == 0 - - if isFreeTransaction { - freeTransactions = append(freeTransactions, common.L2PricedTransaction{ - Tx: tx, - PublishingCost: big.NewInt(0), - FromSelf: true, - }) - continue - } +// ComputeBatch where the batch execution conventions are +func (executor *batchExecutor) ComputeBatch(ctx context.Context, ec *BatchExecutionContext, failForEmptyBatch bool) (*ComputedBatch, error) { + defer core.LogMethodDuration(executor.logger, measure.NewStopwatch(), "Batch context processed") - sender, err := core.GetAuthenticatedSender(context.ChainConfig.ChainID.Int64(), tx) - if err != nil { - executor.logger.Error("Unable to extract sender for tx. Should not happen at this point.", log.TxKey, tx.Hash(), log.ErrKey, err) - continue - } - accBalance := stateDB.GetBalance(*sender) + ec.ctx = ctx + if err := executor.verifyContext(ec); err != nil { + return nil, err + } - cost, err := executor.gasOracle.EstimateL1StorageGasCost(tx, block) - if err != nil { - executor.logger.Error("Unable to get gas cost for tx. Should not happen at this point.", log.TxKey, tx.Hash(), log.ErrKey, err) - continue - } + if err := executor.prepareState(ec); err != nil { + return nil, err + } - if accBalance.Cmp(uint256.MustFromBig(cost)) == -1 { - executor.logger.Info(fmt.Sprintf("insufficient account balance for tx - want: %d have: %d", cost, accBalance), log.TxKey, tx.Hash(), "addr", sender.Hex()) - continue + // the batch with seqNo==2 is by convention the batch where we deploy the system contracts + if ec.SequencerNo.Uint64() == common.L2SysContractGenesisSeqNo { + if err := executor.handleSysContractGenesis(ec); err != nil { + return nil, err } + // the sys genesis batch will not contain anything else + return executor.execResult(ec) + } - transactions = append(transactions, common.L2PricedTransaction{ - Tx: tx, - PublishingCost: big.NewInt(0).Set(cost), - }) + // Step 1: execute the transactions included in the batch + if err := executor.execBatchTransactions(ec); err != nil { + return nil, err } - return transactions, freeTransactions -} -func (executor *batchExecutor) ComputeBatch(ctx context.Context, context *BatchExecutionContext, failForEmptyBatch bool) (*ComputedBatch, error) { //nolint:gocognit - defer core.LogMethodDuration(executor.logger, measure.NewStopwatch(), "Batch context processed") + // Step 2: execute the xChain messages + if err := executor.execXChainMessages(ec); err != nil { + return nil, err + } + // Step 3: execute the registered Callbacks + if err := executor.execRegisteredCallbacks(ec); err != nil { + return nil, err + } + + // Step 4: execute the system contract registered at the end of the block + if err := executor.execOnBlockEndTx(ec); err != nil { + return nil, err + } + + // When the `failForEmptyBatch` flag is true, we skip if there is no transaction or xChain tx + if failForEmptyBatch && len(ec.batchTxResults) == 0 && len(ec.xChainResults) == 0 { + if ec.beforeProcessingSnap > 0 { + //// revert any unexpected mutation to the statedb + ec.stateDB.RevertToSnapshot(ec.beforeProcessingSnap) + } + return nil, ErrNoTransactionsToProcess + } + + return executor.execResult(ec) +} + +func (executor *batchExecutor) verifyContext(ec *BatchExecutionContext) error { // sanity check that the l1 block exists. We don't have to execute batches of forks. - block, err := executor.storage.FetchBlock(ctx, context.BlockPtr) + block, err := executor.storage.FetchBlock(ec.ctx, ec.BlockPtr) if errors.Is(err, errutil.ErrNotFound) { - return nil, errutil.ErrBlockForBatchNotFound + return errutil.ErrBlockForBatchNotFound } else if err != nil { - return nil, fmt.Errorf("failed to retrieve block %s for batch. Cause: %w", context.BlockPtr, err) + return fmt.Errorf("failed to retrieve block %s for batch. Cause: %w", ec.BlockPtr, err) } + ec.l1block = block + // These variables will be used to create the new batch - parentBatch, err := executor.storage.FetchBatchHeader(ctx, context.ParentPtr) + parentBatch, err := executor.storage.FetchBatchHeader(ec.ctx, ec.ParentPtr) if errors.Is(err, errutil.ErrNotFound) { - executor.logger.Error(fmt.Sprintf("can't find parent batch %s. Seq %d", context.ParentPtr, context.SequencerNo)) - return nil, errutil.ErrAncestorBatchNotFound + executor.logger.Error(fmt.Sprintf("can't find parent batch %s. Seq %d", ec.ParentPtr, ec.SequencerNo)) + return errutil.ErrAncestorBatchNotFound } if err != nil { - return nil, fmt.Errorf("failed to retrieve parent batch %s. Cause: %w", context.ParentPtr, err) + return fmt.Errorf("failed to retrieve parent batch %s. Cause: %w", ec.ParentPtr, err) } + ec.parentBatch = parentBatch parentBlock := block if parentBatch.L1Proof != block.Hash() { var err error - parentBlock, err = executor.storage.FetchBlock(ctx, parentBatch.L1Proof) + parentBlock, err = executor.storage.FetchBlock(ec.ctx, parentBatch.L1Proof) if err != nil { executor.logger.Error(fmt.Sprintf("Could not retrieve a proof for batch %s", parentBatch.Hash()), log.ErrKey, err) - return nil, err + return err } } + ec.parentL1Block = parentBlock - // Create a new batch based on the fromBlock of inclusion of the previous, including all new transactions - batch := core.DeterministicEmptyBatch(parentBatch, block, context.AtTime, context.SequencerNo, context.BaseFee, context.Creator) + return nil +} - stateDB, err := executor.batchRegistry.GetBatchState(ctx, rpc.BlockNumberOrHash{BlockHash: &batch.Header.ParentHash}) +func (executor *batchExecutor) prepareState(ec *BatchExecutionContext) error { + var err error + // Create a new batch based on the provided context + ec.currentBatch = core.DeterministicEmptyBatch(ec.parentBatch, ec.l1block, ec.AtTime, ec.SequencerNo, ec.BaseFee, ec.Creator) + ec.stateDB, err = executor.batchRegistry.GetBatchState(ec.ctx, rpc.BlockNumberOrHash{BlockHash: &ec.currentBatch.Header.ParentHash}) if err != nil { - return nil, fmt.Errorf("could not create stateDB. Cause: %w", err) + return fmt.Errorf("could not create stateDB. Cause: %w", err) } - snap := stateDB.Snapshot() + ec.beforeProcessingSnap = ec.stateDB.Snapshot() + return nil +} - syntheticTxResults := make(core.TxExecResults, 0) +func (executor *batchExecutor) handleSysContractGenesis(ec *BatchExecutionContext) error { + systemDeployerTx, err := system.SystemDeployerInitTransaction(executor.logger, *executor.systemContracts.SystemContractsUpgrader()) + if err != nil { + executor.logger.Error("[SystemContracts] Failed to create system deployer contract", log.ErrKey, err) + return err + } + + transactions := common.L2PricedTransactions{ + common.L2PricedTransaction{ + Tx: systemDeployerTx, + PublishingCost: big.NewInt(0), + SystemDeployer: true, + }, + } - var messages common.CrossChainMessages - var transfers common.ValueTransferEvents - if context.SequencerNo.Int64() > int64(common.L2GenesisSeqNo+1) { - messages, transfers = executor.crossChainProcessors.Local.RetrieveInboundMessages(ctx, parentBlock, block, stateDB) + sysCtrGenesisResult, err := executor.executeTxs(ec, 0, transactions, true) + if err != nil { + return fmt.Errorf("could not process system deployer transaction. Cause: %w", err) } - crossChainTransactions := executor.crossChainProcessors.Local.CreateSyntheticTransactions(ctx, messages, transfers, stateDB) - executor.crossChainProcessors.Local.ExecuteValueTransfers(ctx, transfers, stateDB) + if err = executor.verifySyntheticTransactionsSuccess(transactions, sysCtrGenesisResult); err != nil { + return fmt.Errorf("batch computation failed due to system deployer reverting. Cause: %w", err) + } + + ec.genesisSysCtrResult = sysCtrGenesisResult + ec.genesisSysCtrResult.MarkSynthetic(true) + return nil +} + +// filterTransactionsWithSufficientFunds - this function estimates hte l1 fees for the transaction in a given batch execution context. It does so by taking the price of the +// pinned L1 block and using it as the cost per gas for the estimated gas of the calldata encoding of a transaction. It filters out any transactions that cannot afford to pay for their L1 +// publishing cost. +func (executor *batchExecutor) filterTransactionsWithSufficientFunds(ec *BatchExecutionContext) common.L2PricedTransactions { + transactions := make(common.L2PricedTransactions, 0) + block, _ := executor.storage.FetchBlock(ec.ctx, ec.BlockPtr) + + for _, tx := range ec.Transactions { + sender, err := core.GetAuthenticatedSender(ec.ChainConfig.ChainID.Int64(), tx) + if err != nil { + executor.logger.Error("Unable to extract sender for tx. Should not happen at this point.", log.TxKey, tx.Hash(), log.ErrKey, err) + continue + } + accBalance := ec.stateDB.GetBalance(*sender) + + cost, err := executor.gasOracle.EstimateL1StorageGasCost(tx, block) + if err != nil { + executor.logger.Error("Unable to get gas cost for tx. Should not happen at this point.", log.TxKey, tx.Hash(), log.ErrKey, err) + continue + } - transactionsToProcess, freeTransactions := executor.filterTransactionsWithSufficientFunds(ctx, stateDB, context) + if accBalance.Cmp(uint256.MustFromBig(cost)) == -1 { + executor.logger.Info(fmt.Sprintf("insufficient account balance for tx - want: %d have: %d", cost, accBalance), log.TxKey, tx.Hash(), "addr", sender.Hex()) + continue + } - systemDeployerOffset, systemContractCreationResult, err := executor.processSystemDeployer(ctx, stateDB, batch, context) + transactions = append(transactions, common.L2PricedTransaction{ + Tx: tx, + PublishingCost: big.NewInt(0).Set(cost), + }) + } + return transactions +} + +func (executor *batchExecutor) execBatchTransactions(ec *BatchExecutionContext) error { + transactionsToProcess := executor.filterTransactionsWithSufficientFunds(ec) + + txResults, err := executor.executeTxs(ec, 0, transactionsToProcess, false) if err != nil { - return nil, fmt.Errorf("could not deploy system contracts. Cause: %w", err) + return fmt.Errorf("could not process transactions. Cause: %w", err) } - syntheticTxResults.Add(systemContractCreationResult...) + ec.batchTxResults = txResults + return nil +} +func (executor *batchExecutor) readXChainMessages(ec *BatchExecutionContext) error { + if ec.SequencerNo.Int64() > int64(common.L2SysContractGenesisSeqNo) { + ec.xChainMsgs, ec.xChainValueMsgs = executor.crossChainProcessors.Local.RetrieveInboundMessages(ec.ctx, ec.parentL1Block, ec.l1block) + } + return nil +} + +func (executor *batchExecutor) execXChainMessages(ec *BatchExecutionContext) error { + if err := executor.readXChainMessages(ec); err != nil { + return err + } + + crossChainTransactions := executor.crossChainProcessors.Local.CreateSyntheticTransactions(ec.ctx, ec.xChainMsgs, ec.xChainValueMsgs, ec.stateDB) + executor.crossChainProcessors.Local.ExecuteValueTransfers(ec.ctx, ec.xChainValueMsgs, ec.stateDB) xchainTxs := make(common.L2PricedTransactions, 0) for _, xTx := range crossChainTransactions { xchainTxs = append(xchainTxs, common.L2PricedTransaction{ @@ -210,127 +290,92 @@ func (executor *batchExecutor) ComputeBatch(ctx context.Context, context *BatchE FromSelf: true, }) } - - syntheticTransactions := append(xchainTxs, freeTransactions...) - - // fromTxIndex - Here we start from the 0 index. This will be the same for a validator. - successfulTxs, excludedTxs, txResults, err := executor.processTransactions(ctx, batch, systemDeployerOffset, transactionsToProcess, stateDB, context.ChainConfig, false) - if err != nil { - return nil, fmt.Errorf("could not process transactions. Cause: %w", err) - } - txReceipts := make(types.Receipts, 0) - for _, txResult := range txResults { - txReceipts = append(txReceipts, txResult.Receipt) - } - // populate the derived fields in the receipt - err = txReceipts.DeriveFields(executor.chainConfig, batch.Hash(), batch.NumberU64(), batch.Header.Time, batch.Header.BaseFee, nil, successfulTxs) + xChainResults, err := executor.executeTxs(ec, len(ec.batchTxResults), xchainTxs, true) if err != nil { - return nil, fmt.Errorf("could not derive receipts. Cause: %w", err) - } - - onBlockTx, err := executor.systemContracts.CreateOnBatchEndTransaction(ctx, stateDB, successfulTxs, txReceipts) - if err != nil && !errors.Is(err, system.ErrNoTransactions) { - return nil, fmt.Errorf("could not create on block end transaction. Cause: %w", err) - } - if onBlockTx != nil { - onBlockPricedTxes := common.L2PricedTransactions{ - common.L2PricedTransaction{ - Tx: onBlockTx, - PublishingCost: big.NewInt(0), - FromSelf: true, - }, - } - onBlockSuccessfulTx, _, onBlockTxResult, err := executor.processTransactions(ctx, batch, len(successfulTxs), onBlockPricedTxes, stateDB, context.ChainConfig, true) - if err != nil { - return nil, fmt.Errorf("could not process on block end transaction hook. Cause: %w", err) - } - // Ensure the onBlock callback transaction is successful. It should NEVER fail. - if err = executor.verifySyntheticTransactionsSuccess(onBlockPricedTxes, onBlockSuccessfulTx, onBlockTxResult); err != nil { - return nil, fmt.Errorf("batch computation failed due to onBlock hook reverting. Cause: %w", err) - } - result := onBlockTxResult[0] - if ok, err := executor.systemContracts.VerifyOnBlockReceipt(successfulTxs, result.Receipt); !ok || err != nil { - executor.logger.Error("VerifyOnBlockReceipt failed", "error", err, "ok", ok) - return nil, fmt.Errorf("VerifyOnBlockReceipt failed") - } - - syntheticTxResults.Add(onBlockTxResult...) - } else if err == nil && batch.Header.SequencerOrderNo.Uint64() > 2 { - executor.logger.Crit("Bootstrapping of network failed! System contract hooks have not been initialised after genesis.") + return fmt.Errorf("could not process cross chain messages. Cause: %w", err) } - // fromTxIndex - Here we start from the len of the successful transactions; As long as we have the exact same successful transactions in a batch, - // we will start from the same place. - onBatchTxOffset := 0 - if onBlockTx != nil { - onBatchTxOffset = 1 + if len(xchainTxs) != len(xChainResults) { + return fmt.Errorf("could not process cross chain messages. Some were excluded. Cause: %w", err) } + ec.xChainResults = xChainResults + ec.xChainResults.MarkSynthetic(true) + return nil +} +func (executor *batchExecutor) execRegisteredCallbacks(ec *BatchExecutionContext) error { // Create and process public callback transaction if needed - var publicCallbackTxResult core.TxExecResults - onBatchTxOffset, publicCallbackTxResult, err = executor.executePublicCallbacks(ctx, stateDB, context, batch, len(successfulTxs)+onBatchTxOffset) + publicCallbackTx, err := executor.systemContracts.CreatePublicCallbackHandlerTransaction(ec.ctx, ec.stateDB) if err != nil { - return nil, fmt.Errorf("could not execute public callbacks. Cause: %w", err) + return fmt.Errorf("could not create public callback transaction. Cause: %w", err) } - syntheticTxResults.Add(publicCallbackTxResult...) - ccSuccessfulTxs, _, ccTxResults, err := executor.processTransactions(ctx, batch, onBatchTxOffset, syntheticTransactions, stateDB, context.ChainConfig, true) + if publicCallbackTx == nil { + return nil + } + + publicCallbackPricedTxes := common.L2PricedTransactions{ + common.L2PricedTransaction{ + Tx: publicCallbackTx, + PublishingCost: big.NewInt(0), + FromSelf: true, + }, + } + offset := len(ec.batchTxResults) + len(ec.xChainResults) + publicCallbackTxResult, err := executor.executeTxs(ec, offset, publicCallbackPricedTxes, true) if err != nil { - return nil, err + return fmt.Errorf("could not process public callback transaction. Cause: %w", err) } - if len(ccSuccessfulTxs) != len(syntheticTransactions) { - return nil, fmt.Errorf("failed cross chain transactions") + // Ensure the public callback transaction is successful. It should NEVER fail. + if err = executor.verifySyntheticTransactionsSuccess(publicCallbackPricedTxes, publicCallbackTxResult); err != nil { + return fmt.Errorf("batch computation failed due to public callback reverting. Cause: %w", err) } + ec.callbackTxResults = publicCallbackTxResult + ec.callbackTxResults.MarkSynthetic(true) + return nil +} - ccReceipts := make(types.Receipts, 0) - for _, txResult := range ccTxResults { - ccReceipts = append(ccReceipts, txResult.Receipt) +func (executor *batchExecutor) execOnBlockEndTx(ec *BatchExecutionContext) error { + onBlockTx, err := executor.systemContracts.CreateOnBatchEndTransaction(ec.ctx, ec.stateDB, ec.batchTxResults) + if err != nil && !errors.Is(err, system.ErrNoTransactions) { + return fmt.Errorf("could not create on block end transaction. Cause: %w", err) } - if err = executor.verifySyntheticTransactionsSuccess(syntheticTransactions, ccSuccessfulTxs, ccTxResults); err != nil { - return nil, fmt.Errorf("batch computation failed due to cross chain messages. Cause: %w", err) + if onBlockTx == nil { + return nil } - - if failForEmptyBatch && - len(txResults) == 0 && - len(ccTxResults) == 0 && - len(transactionsToProcess)-len(excludedTxs) == 0 && - len(crossChainTransactions) == 0 && - len(messages) == 0 && - len(transfers) == 0 { - if snap > 0 { - //// revert any unexpected mutation to the statedb - stateDB.RevertToSnapshot(snap) - } - return nil, ErrNoTransactionsToProcess + onBlockPricedTx := common.L2PricedTransactions{ + common.L2PricedTransaction{ + Tx: onBlockTx, + PublishingCost: big.NewInt(0), + FromSelf: true, + }, } - - // we need to copy the batch to reset the internal hash cache - copyBatch := *batch - copyBatch.Header.Root = stateDB.IntermediateRoot(false) - copyBatch.Transactions = append(successfulTxs, freeTransactions.ToTransactions()...) - copyBatch.ResetHash() - - if err = executor.populateOutboundCrossChainData(ctx, ©Batch, block, txReceipts); err != nil { - return nil, fmt.Errorf("failed adding cross chain data to batch. Cause: %w", err) + offset := len(ec.callbackTxResults) + len(ec.batchTxResults) + len(ec.xChainResults) + onBlockTxResult, err := executor.executeTxs(ec, offset, onBlockPricedTx, true) + if err != nil { + return fmt.Errorf("could not process on block end transaction hook. Cause: %w", err) } + // Ensure the onBlock callback transaction is successful. It should NEVER fail. + if err = executor.verifySyntheticTransactionsSuccess(onBlockPricedTx, onBlockTxResult); err != nil { + return fmt.Errorf("batch computation failed due to onBlock hook reverting. Cause: %w", err) + } + ec.blockEndResult = onBlockTxResult + ec.blockEndResult.MarkSynthetic(true) + return nil +} - allReceipts := append(txReceipts, ccReceipts...) - executor.populateHeader(©Batch, allReceipts) - - // the logs and receipts produced by the EVM have the wrong hash which must be adjusted - for _, receipt := range allReceipts { - receipt.BlockHash = copyBatch.Hash() - for _, l := range receipt.Logs { - l.BlockHash = copyBatch.Hash() - } +func (executor *batchExecutor) execResult(ec *BatchExecutionContext) (*ComputedBatch, error) { + batch, allResults, err := executor.createBatch(ec) + if err != nil { + return nil, fmt.Errorf("failed creating batch. Cause: %w", err) } commitFunc := func(deleteEmptyObjects bool) (gethcommon.Hash, error) { executor.stateDBMutex.Lock() defer executor.stateDBMutex.Unlock() - h, err := stateDB.Commit(copyBatch.Number().Uint64(), deleteEmptyObjects) + h, err := ec.stateDB.Commit(batch.Number().Uint64(), deleteEmptyObjects) if err != nil { - return gethcommon.Hash{}, fmt.Errorf("commit failure for batch %d. Cause: %w", batch.SeqNo(), err) + return gethcommon.Hash{}, fmt.Errorf("commit failure for batch %d. Cause: %w", ec.currentBatch.SeqNo(), err) } trieDB := executor.storage.TrieDB() err = trieDB.Commit(h, false) @@ -338,89 +383,57 @@ func (executor *batchExecutor) ComputeBatch(ctx context.Context, context *BatchE // When system contract deployment genesis batch is committed, initialize executor's addresses for the hooks. // Further restarts will call into Load() which will take the receipts for batch number 2 (which should never be deleted) // and reinitialize them. - if err == nil && batch.Header.SequencerOrderNo.Uint64() == 2 { - if len(systemContractCreationResult) == 0 { + if err == nil && ec.currentBatch.Header.SequencerOrderNo.Uint64() == common.L2SysContractGenesisSeqNo { + if len(ec.genesisSysCtrResult) == 0 { return h, fmt.Errorf("failed to instantiate system contracts: expected receipt for system deployer transaction, but no receipts found in batch") } - return h, executor.initializeSystemContracts(ctx, batch, systemContractCreationResult[0].Receipt) + return h, executor.systemContracts.Initialize(batch, *ec.genesisSysCtrResult.Receipts()[0], executor.crossChainProcessors.Local) } - return h, err } - syntheticTxResults.Add(ccTxResults...) - syntheticTxResults.MarkSynthetic(true) - return &ComputedBatch{ - Batch: ©Batch, - TxExecResults: append(txResults, syntheticTxResults...), + Batch: batch, + TxExecResults: allResults, Commit: commitFunc, }, nil } -func (executor *batchExecutor) processSystemDeployer(ctx context.Context, stateDB *state.StateDB, batch *core.Batch, context *BatchExecutionContext) (int, []*core.TxExecResult, error) { - if context.SequencerNo.Uint64() != 2 { - return 0, nil, nil - } - - systemDeployerTx, err := system.SystemDeployerInitTransaction(executor.logger, *executor.systemContracts.SystemContractsUpgrader()) - if err != nil { - executor.logger.Crit("[SystemContracts] Failed to create system deployer contract", log.ErrKey, err) - return 0, nil, err - } - - transactions := common.L2PricedTransactions{ - common.L2PricedTransaction{ - Tx: systemDeployerTx, - PublishingCost: big.NewInt(0), - SystemDeployer: true, - }, - } +func (executor *batchExecutor) createBatch(ec *BatchExecutionContext) (*core.Batch, core.TxExecResults, error) { + // we need to copy the batch to reset the internal hash cache + batch := *ec.currentBatch + batch.Header.Root = ec.stateDB.IntermediateRoot(false) + batch.Transactions = ec.batchTxResults.BatchTransactions() + batch.ResetHash() - successfulTxs, _, result, err := executor.processTransactions(ctx, batch, 0, transactions, stateDB, context.ChainConfig, true) - if err != nil { - return 0, nil, fmt.Errorf("could not process system deployer transaction. Cause: %w", err) + txReceipts := ec.batchTxResults.Receipts() + if err := executor.populateOutboundCrossChainData(ec.ctx, &batch, ec.l1block, txReceipts); err != nil { + return nil, nil, fmt.Errorf("failed adding cross chain data to batch. Cause: %w", err) } - if err = executor.verifySyntheticTransactionsSuccess(transactions, successfulTxs, result); err != nil { - return 0, nil, fmt.Errorf("batch computation failed due to system deployer reverting. Cause: %w", err) + allResults := append(append(append(append(ec.batchTxResults, ec.xChainResults...), ec.callbackTxResults...), ec.blockEndResult...), ec.genesisSysCtrResult...) + receipts := allResults.Receipts() + if len(receipts) == 0 { + batch.Header.ReceiptHash = types.EmptyRootHash + } else { + batch.Header.ReceiptHash = types.DeriveSha(receipts, trie.NewStackTrie(nil)) } - return 1, result, nil -} - -func (executor *batchExecutor) executePublicCallbacks(ctx context.Context, stateDB *state.StateDB, context *BatchExecutionContext, batch *core.Batch, txOffset int) (int, core.TxExecResults, error) { - // Create and process public callback transaction if needed - publicCallbackTx, err := executor.systemContracts.CreatePublicCallbackHandlerTransaction(ctx, stateDB) - if err != nil { - return txOffset, nil, fmt.Errorf("could not create public callback transaction. Cause: %w", err) + if len(batch.Transactions) == 0 { + batch.Header.TxHash = types.EmptyRootHash + } else { + batch.Header.TxHash = types.DeriveSha(types.Transactions(batch.Transactions), trie.NewStackTrie(nil)) } - if publicCallbackTx != nil { - publicCallbackPricedTxes := common.L2PricedTransactions{ - common.L2PricedTransaction{ - Tx: publicCallbackTx, - PublishingCost: big.NewInt(0), - FromSelf: true, - }, - } - publicCallbackSuccessfulTx, _, publicCallbackTxResult, err := executor.processTransactions(ctx, batch, txOffset, publicCallbackPricedTxes, stateDB, context.ChainConfig, true) - if err != nil { - return txOffset, nil, fmt.Errorf("could not process public callback transaction. Cause: %w", err) - } - // Ensure the public callback transaction is successful. It should NEVER fail. - if err = executor.verifySyntheticTransactionsSuccess(publicCallbackPricedTxes, publicCallbackSuccessfulTx, publicCallbackTxResult); err != nil { - return txOffset, nil, fmt.Errorf("batch computation failed due to public callback reverting. Cause: %w", err) + // the logs and receipts produced by the EVM have the wrong hash which must be adjusted + for _, receipt := range receipts { + receipt.BlockHash = batch.Hash() + for _, l := range receipt.Logs { + l.BlockHash = batch.Hash() } - - return len(publicCallbackSuccessfulTx) + txOffset, publicCallbackTxResult, nil } - return txOffset, nil, nil -} - -func (executor *batchExecutor) initializeSystemContracts(_ context.Context, batch *core.Batch, receipts *types.Receipt) error { - return executor.systemContracts.Initialize(batch, *receipts, executor.crossChainProcessors.Local) + return &batch, allResults, nil } func (executor *batchExecutor) ExecuteBatch(ctx context.Context, batch *core.Batch) ([]*core.TxExecResult, error) { @@ -522,7 +535,7 @@ func (executor *batchExecutor) populateOutboundCrossChainData(ctx context.Contex hasMessages = true } - var xchainHash gethcommon.Hash = gethcommon.BigToHash(gethcommon.Big0) + xchainHash := gethcommon.BigToHash(gethcommon.Big0) if hasMessages { tree, err := smt.Of(xchainTree, crosschain.CrossChainEncodings) if err != nil { @@ -551,26 +564,12 @@ func (executor *batchExecutor) populateOutboundCrossChainData(ctx context.Contex return nil } -func (executor *batchExecutor) populateHeader(batch *core.Batch, receipts types.Receipts) { - if len(receipts) == 0 { - batch.Header.ReceiptHash = types.EmptyRootHash - } else { - batch.Header.ReceiptHash = types.DeriveSha(receipts, trie.NewStackTrie(nil)) - } - - if len(batch.Transactions) == 0 { - batch.Header.TxHash = types.EmptyRootHash - } else { - batch.Header.TxHash = types.DeriveSha(types.Transactions(batch.Transactions), trie.NewStackTrie(nil)) - } -} - -func (executor *batchExecutor) verifySyntheticTransactionsSuccess(transactions common.L2PricedTransactions, executedTxs types.Transactions, receipts core.TxExecResults) error { - if len(transactions) != executedTxs.Len() { +func (executor *batchExecutor) verifySyntheticTransactionsSuccess(transactions common.L2PricedTransactions, results core.TxExecResults) error { + if len(transactions) != len(results) { return fmt.Errorf("some synthetic transactions have not been executed") } - for _, rec := range receipts { + for _, rec := range results { if rec.Receipt.Status == 1 { continue } @@ -579,53 +578,56 @@ func (executor *batchExecutor) verifySyntheticTransactionsSuccess(transactions c return nil } -func (executor *batchExecutor) processTransactions( - ctx context.Context, - batch *core.Batch, - tCount int, - txs common.L2PricedTransactions, - stateDB *state.StateDB, - cc *params.ChainConfig, - noBaseFee bool, -) ([]*common.L2Tx, []*common.L2Tx, []*core.TxExecResult, error) { - var executedTransactions []*common.L2Tx - var excludedTransactions []*common.L2Tx +func (executor *batchExecutor) executeTxs(ec *BatchExecutionContext, offset int, txs common.L2PricedTransactions, noBaseFee bool) (core.TxExecResults, error) { txResultsMap, err := evm.ExecuteTransactions( - ctx, + ec.ctx, executor.entropyService, txs, - stateDB, - batch.Header, + ec.stateDB, + ec.currentBatch.Header, executor.storage, executor.gethEncodingService, - cc, + ec.ChainConfig, executor.config, - tCount, + offset, noBaseFee, executor.batchGasLimit, executor.logger, ) if err != nil { - return nil, nil, nil, err + return nil, err } - txResults := make([]*core.TxExecResult, 0) + txResults := make(core.TxExecResults, 0) for _, tx := range txs { result, f := txResultsMap[tx.Tx.Hash()] if !f { - return nil, nil, nil, fmt.Errorf("there should be an entry for each transaction") + return nil, fmt.Errorf("there should be an entry for each transaction") } if result.Receipt != nil { - executedTransactions = append(executedTransactions, tx.Tx) txResults = append(txResults, result) } else { // Exclude failed transactions - excludedTransactions = append(excludedTransactions, tx.Tx) - executor.logger.Debug("Excluding transaction from batch", log.TxKey, tx.Tx.Hash(), log.BatchHashKey, batch.Hash(), "cause", result.Err) + executor.logger.Debug("Excluding transaction from batch", log.TxKey, tx.Tx.Hash(), log.BatchHashKey, ec.currentBatch.Hash(), "cause", result.Err) } } - sort.Sort(sortByTxIndex(txResults)) - return executedTransactions, excludedTransactions, txResults, nil + // populate the derived fields in the receipt + batch := ec.currentBatch + txReceipts := txResults.Receipts() + err = txReceipts.DeriveFields(executor.chainConfig, batch.Hash(), batch.NumberU64(), batch.Header.Time, batch.Header.BaseFee, nil, txResults.BatchTransactions()) + if err != nil { + return nil, fmt.Errorf("could not derive receipts. Cause: %w", err) + } + for i, txResult := range txResults { + // sanity check + if txResult.Receipt.TxHash != txReceipts[i].TxHash { + panic("Should not happen. Tx receipts and tx results do not match") + } + txResult.Receipt = txReceipts[i] + } + + sort.Sort(sortByTxIndex(txResults)) + return txResults, nil } type sortByTxIndex []*core.TxExecResult diff --git a/go/enclave/components/interfaces.go b/go/enclave/components/interfaces.go index da96aa9527..996f7b6f33 100644 --- a/go/enclave/components/interfaces.go +++ b/go/enclave/components/interfaces.go @@ -54,6 +54,26 @@ type BatchExecutionContext struct { ChainConfig *params.ChainConfig SequencerNo *big.Int BaseFee *big.Int + + // these properties are calculated during execution + ctx context.Context + l1block *types.Header + parentL1Block *types.Header + parentBatch *common.BatchHeader + + xChainMsgs common.CrossChainMessages + xChainValueMsgs common.ValueTransferEvents + + currentBatch *core.Batch + stateDB *state.StateDB + beforeProcessingSnap int + + genesisSysCtrResult core.TxExecResults + + xChainResults core.TxExecResults + batchTxResults core.TxExecResults + callbackTxResults core.TxExecResults + blockEndResult core.TxExecResults } // ComputedBatch - a structure representing the result of a batch diff --git a/go/enclave/components/rollup_compression.go b/go/enclave/components/rollup_compression.go index 5e630a99df..03bdea89c2 100644 --- a/go/enclave/components/rollup_compression.go +++ b/go/enclave/components/rollup_compression.go @@ -470,7 +470,7 @@ func (rc *RollupCompression) executeAndSaveIncompleteBatches(ctx context.Context if err != nil { return err } - err = rc.storage.StoreExecutedBatch(ctx, genBatch.Header, nil) + err = rc.storage.StoreExecutedBatch(ctx, genBatch, nil) if err != nil { return err } @@ -518,7 +518,7 @@ func (rc *RollupCompression) executeAndSaveIncompleteBatches(ctx context.Context if err != nil { return err } - err = rc.storage.StoreExecutedBatch(ctx, computedBatch.Batch.Header, computedBatch.TxExecResults) + err = rc.storage.StoreExecutedBatch(ctx, computedBatch.Batch, computedBatch.TxExecResults) if err != nil { return err } diff --git a/go/enclave/core/event_types.go b/go/enclave/core/event_types.go index 25df4cc9ef..4bf5cad028 100644 --- a/go/enclave/core/event_types.go +++ b/go/enclave/core/event_types.go @@ -149,7 +149,7 @@ func (txResults *TxExecResults) MarkSynthetic(isSynthetic bool) { } } -func (txResults *TxExecResults) GetSynthetic() *TxExecResults { +func (txResults *TxExecResults) SyntheticTransactions() *TxExecResults { syntheticTxs := make(TxExecResults, 0) for _, txResult := range *txResults { if txResult.TxWithSender.IsSynthetic { @@ -159,6 +159,16 @@ func (txResults *TxExecResults) GetSynthetic() *TxExecResults { return &syntheticTxs } +func (txResults *TxExecResults) BatchTransactions() []*common.L2Tx { + txs := make([]*common.L2Tx, 0) + for _, txResult := range *txResults { + if !txResult.TxWithSender.IsSynthetic { + txs = append(txs, txResult.TxWithSender.Tx) + } + } + return txs +} + func (txResults *TxExecResults) GetReal() *TxExecResults { realTxs := make(TxExecResults, 0) for _, txResult := range *txResults { @@ -176,3 +186,11 @@ func (txResults *TxExecResults) ToTransactionsWithSenders() TransactionsWithSend } return transactionsWithSenders } + +func (txResults *TxExecResults) Receipts() types.Receipts { + receipts := make(types.Receipts, len(*txResults)) + for i, txResult := range *txResults { + receipts[i] = txResult.Receipt + } + return receipts +} diff --git a/go/enclave/crosschain/interfaces.go b/go/enclave/crosschain/interfaces.go index 4c54322049..2f3a9ecd65 100644 --- a/go/enclave/crosschain/interfaces.go +++ b/go/enclave/crosschain/interfaces.go @@ -51,11 +51,11 @@ type Manager interface { ExtractOutboundTransfers(ctx context.Context, receipts common.L2Receipts) (common.ValueTransferEvents, error) - CreateSyntheticTransactions(ctx context.Context, messages common.CrossChainMessages, transfers common.ValueTransferEvents, rollupState *state.StateDB) common.L2Transactions + CreateSyntheticTransactions(ctx context.Context, messages common.CrossChainMessages, transfers common.ValueTransferEvents, stateDB *state.StateDB) common.L2Transactions - ExecuteValueTransfers(ctx context.Context, transfers common.ValueTransferEvents, rollupState *state.StateDB) + ExecuteValueTransfers(ctx context.Context, transfers common.ValueTransferEvents, stateDB *state.StateDB) - RetrieveInboundMessages(ctx context.Context, fromBlock *types.Header, toBlock *types.Header, rollupState *state.StateDB) (common.CrossChainMessages, common.ValueTransferEvents) + RetrieveInboundMessages(ctx context.Context, fromBlock *types.Header, toBlock *types.Header) (common.CrossChainMessages, common.ValueTransferEvents) system.SystemContractsInitializable } diff --git a/go/enclave/crosschain/message_bus_manager.go b/go/enclave/crosschain/message_bus_manager.go index 6639e9024a..b4f2548bc8 100644 --- a/go/enclave/crosschain/message_bus_manager.go +++ b/go/enclave/crosschain/message_bus_manager.go @@ -149,7 +149,7 @@ 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 *types.Header, toBlock *types.Header, rollupState *state.StateDB) (common.CrossChainMessages, common.ValueTransferEvents) { +func (m *MessageBusManager) RetrieveInboundMessages(ctx context.Context, fromBlock *types.Header, toBlock *types.Header) (common.CrossChainMessages, common.ValueTransferEvents) { messages := make(common.CrossChainMessages, 0) transfers := make(common.ValueTransferEvents, 0) diff --git a/go/enclave/nodetype/sequencer.go b/go/enclave/nodetype/sequencer.go index aef315a6d5..02d1fe64a5 100644 --- a/go/enclave/nodetype/sequencer.go +++ b/go/enclave/nodetype/sequencer.go @@ -146,7 +146,7 @@ func (s *sequencer) createGenesisBatch(ctx context.Context, block *types.Header) // produce batch #2 which has the message bus and any other system contracts _, err = s.produceBatch( ctx, - big.NewInt(0).Add(batch.Header.SequencerOrderNo, big.NewInt(1)), + big.NewInt(0).SetUint64(common.L2SysContractGenesisSeqNo), block.Hash(), batch.Hash(), common.L2Transactions{}, @@ -304,7 +304,7 @@ func (s *sequencer) StoreExecutedBatch(ctx context.Context, batch *core.Batch, t return fmt.Errorf("failed to store batch. Cause: %w", err) } - if err := s.storage.StoreExecutedBatch(ctx, batch.Header, txResults); err != nil { + if err := s.storage.StoreExecutedBatch(ctx, batch, txResults); err != nil { return fmt.Errorf("failed to store batch. Cause: %w", err) } diff --git a/go/enclave/nodetype/validator.go b/go/enclave/nodetype/validator.go index 6edea60b21..e6dff3ebca 100644 --- a/go/enclave/nodetype/validator.go +++ b/go/enclave/nodetype/validator.go @@ -111,7 +111,7 @@ func (val *validator) ExecuteStoredBatches(ctx context.Context) error { if err != nil { return fmt.Errorf("could not execute batch %s. Cause: %w", batchHeader.Hash(), err) } - err = val.storage.StoreExecutedBatch(ctx, batchHeader, txResults) + err = val.storage.StoreExecutedBatch(ctx, batch, txResults) if err != nil { return fmt.Errorf("could not store executed batch %s. Cause: %w", batchHeader.Hash(), err) } @@ -153,7 +153,7 @@ func (val *validator) handleGenesis(ctx context.Context, batch *common.BatchHead return fmt.Errorf("received invalid genesis batch") } - err = val.storage.StoreExecutedBatch(ctx, genBatch.Header, nil) + err = val.storage.StoreExecutedBatch(ctx, genBatch, nil) if err != nil { return err } diff --git a/go/enclave/storage/enclavedb/batch.go b/go/enclave/storage/enclavedb/batch.go index 30943e5760..9fe5f20d42 100644 --- a/go/enclave/storage/enclavedb/batch.go +++ b/go/enclave/storage/enclavedb/batch.go @@ -65,7 +65,7 @@ func ExistsBatchAtHeight(ctx context.Context, dbTx *sql.Tx, height *big.Int) (bo } // WriteTransactions - persists the batch and the transactions -func WriteTransactions(ctx context.Context, dbtx *sql.Tx, transactions []*core.TxWithSender, height uint64, isSynthetic bool, senders []uint64, toContracts []*uint64) error { +func WriteTransactions(ctx context.Context, dbtx *sql.Tx, transactions []*core.TxWithSender, height uint64, isSynthetic bool, senderIds []uint64, toContractIds []*uint64, fromIdx int) error { // creates a batch insert statement for all entries if len(transactions) > 0 { insert := "insert into tx (hash, content, to_address, type, sender_address, idx, batch_height, is_synthetic) values " + repeat("(?,?,?,?,?,?,?,?)", ",", len(transactions)) @@ -79,10 +79,10 @@ func WriteTransactions(ctx context.Context, dbtx *sql.Tx, transactions []*core.T args = append(args, transaction.Tx.Hash()) // tx_hash args = append(args, txBytes) // content - args = append(args, toContracts[i]) // To + args = append(args, toContractIds[i]) // To args = append(args, transaction.Tx.Type()) // Type - args = append(args, senders[i]) // sender_address - args = append(args, i) // idx + args = append(args, senderIds[i]) // sender_address + args = append(args, fromIdx+i) // idx args = append(args, height) // the batch height which contained it args = append(args, isSynthetic) // is_synthetic if the transaction is a synthetic (internally derived transaction) } diff --git a/go/enclave/storage/enclavedb/events.go b/go/enclave/storage/enclavedb/events.go index bdb4d84292..ae665703dd 100644 --- a/go/enclave/storage/enclavedb/events.go +++ b/go/enclave/storage/enclavedb/events.go @@ -51,7 +51,7 @@ func WriteEventType(ctx context.Context, dbTX *sql.Tx, et *EventType) (uint64, e } func ReadEventType(ctx context.Context, dbTX *sql.Tx, contract *Contract, eventSignature gethcommon.Hash) (*EventType, error) { - var et EventType = EventType{Contract: contract} + et := EventType{Contract: contract} err := dbTX.QueryRowContext(ctx, "select id, event_sig, auto_visibility, auto_public, config_public, topic1_can_view, topic2_can_view, topic3_can_view, sender_can_view from event_type where contract=? and event_sig=?", contract.Id, eventSignature.Bytes(), diff --git a/go/enclave/storage/events_storage.go b/go/enclave/storage/events_storage.go index 68245f8644..8d778bb484 100644 --- a/go/enclave/storage/events_storage.go +++ b/go/enclave/storage/events_storage.go @@ -29,7 +29,7 @@ func newEventsStorage(cachingService *CacheService, db enclavedb.EnclaveDB, logg func (es *eventsStorage) storeReceiptAndEventLogs(ctx context.Context, dbTX *sql.Tx, batch *common.BatchHeader, txExecResult *core.TxExecResult) error { txId, senderId, err := enclavedb.ReadTransactionIdAndSender(ctx, dbTX, txExecResult.Receipt.TxHash) - if err != nil && !errors.Is(err, errutil.ErrNotFound) { + if err != nil { return fmt.Errorf("could not get transaction id. Cause: %w", err) } @@ -99,7 +99,7 @@ func (es *eventsStorage) storeReceipt(ctx context.Context, dbTX *sql.Tx, batch * return execTxId, nil } -func (es *eventsStorage) storeEventLog(ctx context.Context, dbTX *sql.Tx, execTxId uint64, l *types.Log) error { +func (es *eventsStorage) storeEventLog(ctx context.Context, dbTX *sql.Tx, receiptId uint64, l *types.Log) error { eventSig := l.Topics[0] contract, err := es.readContract(ctx, dbTX, l.Address) @@ -130,7 +130,7 @@ func (es *eventsStorage) storeEventLog(ctx context.Context, dbTX *sql.Tx, execTx if len(data) == 0 { data = nil } - err = enclavedb.WriteEventLog(ctx, dbTX, eventType.Id, topicIds, data, l.Index, execTxId) + err = enclavedb.WriteEventLog(ctx, dbTX, eventType.Id, topicIds, data, l.Index, receiptId) if err != nil { return fmt.Errorf("could not write event log. Cause: %w", err) } diff --git a/go/enclave/storage/init/edgelessdb/001_init.sql b/go/enclave/storage/init/edgelessdb/001_init.sql index 52b2ede728..52f04d077e 100644 --- a/go/enclave/storage/init/edgelessdb/001_init.sql +++ b/go/enclave/storage/init/edgelessdb/001_init.sql @@ -110,7 +110,7 @@ create table if not exists tendb.receipt cumulative_gas_used BIGINT not null, effective_gas_price BIGINT, created_contract_address binary(20), - tx int, + tx int NOT NULL, batch int NOT NULL, INDEX (batch), INDEX (tx, batch), diff --git a/go/enclave/storage/init/sqlite/001_init.sql b/go/enclave/storage/init/sqlite/001_init.sql index 653fb5e7ba..13d59f0e53 100644 --- a/go/enclave/storage/init/sqlite/001_init.sql +++ b/go/enclave/storage/init/sqlite/001_init.sql @@ -102,8 +102,7 @@ create table if not exists receipt cumulative_gas_used int not null, effective_gas_price int, created_contract_address binary(20), - -- commenting out the fk until synthetic transactions are also stored - tx INTEGER, + tx INTEGER NOT NULL REFERENCES tx, batch INTEGER NOT NULL REFERENCES batch ); create index IDX_EX_TX_BATCH on receipt (batch); diff --git a/go/enclave/storage/interfaces.go b/go/enclave/storage/interfaces.go index fa0bb6194c..dea8d84815 100644 --- a/go/enclave/storage/interfaces.go +++ b/go/enclave/storage/interfaces.go @@ -72,7 +72,7 @@ type BatchResolver interface { // StoreBatch stores an un-executed batch. StoreBatch(ctx context.Context, batch *core.Batch, convertedHash gethcommon.Hash) error // StoreExecutedBatch - store the batch after it was executed - StoreExecutedBatch(ctx context.Context, batch *common.BatchHeader, results core.TxExecResults) error + StoreExecutedBatch(ctx context.Context, batch *core.Batch, results core.TxExecResults) error // StoreRollup StoreRollup(ctx context.Context, rollup *common.ExtRollup, header *common.CalldataRollupHeader) error diff --git a/go/enclave/storage/storage.go b/go/enclave/storage/storage.go index 77248a5df0..22402aeebf 100644 --- a/go/enclave/storage/storage.go +++ b/go/enclave/storage/storage.go @@ -589,12 +589,12 @@ func (s *storageImpl) StoreBatch(ctx context.Context, batch *core.Batch, convert transactionsWithSenders[i] = &core.TxWithSender{Tx: tx, Sender: &sender} } - senders, toContracts, err := s.handleTxSendersAndReceivers(ctx, transactionsWithSenders, dbTx) + senderIds, toContractIds, err := s.handleTxSendersAndReceivers(ctx, transactionsWithSenders, dbTx) if err != nil { return err } - if err := enclavedb.WriteTransactions(ctx, dbTx, transactionsWithSenders, batch.Header.Number.Uint64(), false, senders, toContracts); err != nil { + if err := enclavedb.WriteTransactions(ctx, dbTx, transactionsWithSenders, batch.Header.Number.Uint64(), false, senderIds, toContractIds, 0); err != nil { return fmt.Errorf("could not write transactions. Cause: %w", err) } } @@ -633,7 +633,7 @@ func (s *storageImpl) handleTxSendersAndReceivers(ctx context.Context, transacti return senders, toContracts, nil } -func (s *storageImpl) StoreExecutedBatch(ctx context.Context, batch *common.BatchHeader, results core.TxExecResults) error { +func (s *storageImpl) StoreExecutedBatch(ctx context.Context, batch *core.Batch, results core.TxExecResults) error { defer s.logDuration("StoreExecutedBatch", measure.NewStopwatch()) executed, err := enclavedb.BatchWasExecuted(ctx, s.db.GetSQLDB(), batch.Hash()) if err != nil { @@ -644,7 +644,7 @@ func (s *storageImpl) StoreExecutedBatch(ctx context.Context, batch *common.Batc return nil } - s.logger.Trace("storing executed batch", log.BatchHashKey, batch.Hash(), log.BatchSeqNoKey, batch.SequencerOrderNo, "receipts", len(results)) + s.logger.Trace("storing executed batch", log.BatchHashKey, batch.Hash(), log.BatchSeqNoKey, batch.Header.SequencerOrderNo, "receipts", len(results)) dbTx, err := s.db.NewDBTransaction(ctx) if err != nil { @@ -652,23 +652,24 @@ func (s *storageImpl) StoreExecutedBatch(ctx context.Context, batch *common.Batc } defer dbTx.Rollback() - if err := enclavedb.MarkBatchExecuted(ctx, dbTx, batch.SequencerOrderNo); err != nil { + if err := enclavedb.MarkBatchExecuted(ctx, dbTx, batch.Header.SequencerOrderNo); err != nil { return fmt.Errorf("could not set the executed flag. Cause: %w", err) } - transactionsWithSenders := results.GetSynthetic().ToTransactionsWithSenders() + // store the synthetic transactions + transactionsWithSenders := results.SyntheticTransactions().ToTransactionsWithSenders() senders, toContracts, err := s.handleTxSendersAndReceivers(ctx, transactionsWithSenders, dbTx) if err != nil { return fmt.Errorf("could not handle synthetic txs senders and receivers. Cause: %w", err) } - if err := enclavedb.WriteTransactions(ctx, dbTx, transactionsWithSenders, batch.Number.Uint64(), true, senders, toContracts); err != nil { + if err := enclavedb.WriteTransactions(ctx, dbTx, transactionsWithSenders, batch.Header.Number.Uint64(), true, senders, toContracts, len(batch.Transactions)); err != nil { return fmt.Errorf("could not write synthetic txs. Cause: %w", err) } for _, txExecResult := range results { - err = s.eventsStorage.storeReceiptAndEventLogs(ctx, dbTx, batch, txExecResult) + err = s.eventsStorage.storeReceiptAndEventLogs(ctx, dbTx, batch.Header, txExecResult) if err != nil { return fmt.Errorf("could not store receipt. Cause: %w", err) } @@ -802,7 +803,7 @@ func (s *storageImpl) FilterLogs( return nil, err } // the database returns an unsorted list of event logs. - // we have to perform the sorting programatically + // we have to perform the sorting programmatically sort.Slice(logs, func(i, j int) bool { if logs[i].BlockNumber == logs[j].BlockNumber { return logs[i].Index < logs[j].Index diff --git a/go/enclave/system/hooks.go b/go/enclave/system/hooks.go index d24ca6d885..3156d7a0db 100644 --- a/go/enclave/system/hooks.go +++ b/go/enclave/system/hooks.go @@ -6,6 +6,8 @@ import ( "math/big" "strings" + "github.com/ten-protocol/go-ten/go/common/log" + "github.com/ethereum/go-ethereum/accounts/abi" gethcommon "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/core/state" @@ -37,8 +39,10 @@ type SystemContractCallbacks interface { Load() error // Usage - CreateOnBatchEndTransaction(ctx context.Context, stateDB *state.StateDB, transactions common.L2Transactions, receipts types.Receipts) (*types.Transaction, error) + CreateOnBatchEndTransaction(ctx context.Context, stateDB *state.StateDB, results core.TxExecResults) (*types.Transaction, error) CreatePublicCallbackHandlerTransaction(ctx context.Context, stateDB *state.StateDB) (*types.Transaction, error) + + // VerifyOnBlockReceipt - used for debugging VerifyOnBlockReceipt(transactions common.L2Transactions, receipt *types.Receipt) (bool, error) } @@ -127,7 +131,7 @@ func (s *systemContractCallbacks) initializeRequiredAddresses(addresses SystemCo func (s *systemContractCallbacks) Initialize(batch *core.Batch, receipt types.Receipt, msgBusManager SystemContractsInitializable) error { s.logger.Info("Initialize: Starting initialization of system contracts", "batchSeqNo", batch.SeqNo()) - if batch.SeqNo().Uint64() != 2 { + if batch.SeqNo().Uint64() != common.L2SysContractGenesisSeqNo { s.logger.Error("Initialize: Batch is not genesis", "batchSeqNo", batch.SeqNo) return fmt.Errorf("batch is not genesis") } @@ -176,13 +180,13 @@ func (s *systemContractCallbacks) CreatePublicCallbackHandlerTransaction(ctx con return formedTx, nil } -func (s *systemContractCallbacks) CreateOnBatchEndTransaction(ctx context.Context, l2State *state.StateDB, transactions common.L2Transactions, receipts types.Receipts) (*types.Transaction, error) { +func (s *systemContractCallbacks) CreateOnBatchEndTransaction(_ context.Context, l2State *state.StateDB, results core.TxExecResults) (*types.Transaction, error) { if s.transactionsPostProcessorAddress == nil { s.logger.Debug("CreateOnBatchEndTransaction: TransactionsPostProcessorAddress is nil, skipping transaction creation") return nil, nil } - if len(transactions) == 0 { + if len(results) == 0 { s.logger.Debug("CreateOnBatchEndTransaction: Batch has no transactions, skipping transaction creation") return nil, ErrNoTransactions } @@ -190,39 +194,24 @@ func (s *systemContractCallbacks) CreateOnBatchEndTransaction(ctx context.Contex nonceForSyntheticTx := l2State.GetNonce(common.MaskedSender(*s.transactionsPostProcessorAddress)) s.logger.Debug("CreateOnBatchEndTransaction: Retrieved nonce for synthetic transaction", "nonce", nonceForSyntheticTx) - solidityTransactions := make([]TransactionPostProcessor.StructsTransaction, 0) - - type statusWithGasUsed struct { - status bool - gasUsed uint64 - } - - txSuccessMap := map[gethcommon.Hash]statusWithGasUsed{} - for _, receipt := range receipts { - txSuccessMap[receipt.TxHash] = statusWithGasUsed{ - status: receipt.Status == types.ReceiptStatusSuccessful, - gasUsed: receipt.GasUsed, - } - } - - for _, tx := range transactions { - // Start of Selection - - txMetadata := txSuccessMap[tx.Hash()] - - transaction := TransactionPostProcessor.StructsTransaction{ - Nonce: big.NewInt(int64(tx.Nonce())), + // the data that is passed when the block ends + synTxs := make([]TransactionPostProcessor.StructsTransaction, 0) + for _, txExecResult := range results { + tx := txExecResult.TxWithSender.Tx + receipt := txExecResult.Receipt + synTx := TransactionPostProcessor.StructsTransaction{ + Nonce: big.NewInt(int64(txExecResult.TxWithSender.Tx.Nonce())), GasPrice: tx.GasPrice(), GasLimit: big.NewInt(int64(tx.Gas())), Value: tx.Value(), Data: tx.Data(), - Successful: txMetadata.status, - GasUsed: txMetadata.gasUsed, + Successful: receipt.Status == types.ReceiptStatusSuccessful, + GasUsed: receipt.GasUsed, } if tx.To() != nil { - transaction.To = *tx.To() + synTx.To = *tx.To() } else { - transaction.To = gethcommon.Address{} // Zero address - contract deployment + synTx.To = gethcommon.Address{} // Zero address - contract deployment } sender, err := core.GetExternalTxSigner(tx) @@ -230,13 +219,13 @@ func (s *systemContractCallbacks) CreateOnBatchEndTransaction(ctx context.Contex s.logger.Error("CreateOnBatchEndTransaction: Failed to recover sender address", "error", err, "transactionHash", tx.Hash().Hex()) return nil, fmt.Errorf("failed to recover sender address: %w", err) } - transaction.From = sender + synTx.From = sender - solidityTransactions = append(solidityTransactions, transaction) - s.logger.Debug("CreateOnBatchEndTransaction: Encoded transaction", "transactionHash", tx.Hash().Hex(), "sender", sender.Hex()) + synTxs = append(synTxs, synTx) + s.logger.Debug("CreateOnBatchEndTransaction: Encoded transaction", log.TxKey, tx.Hash(), "sender", sender.Hex()) } - data, err := transactionPostProcessorABI.Pack("onBlock", solidityTransactions) + data, err := transactionPostProcessorABI.Pack("onBlock", synTxs) if err != nil { s.logger.Error("CreateOnBatchEndTransaction: Failed packing onBlock data", "error", err) return nil, fmt.Errorf("failed packing onBlock() %w", err) @@ -252,7 +241,7 @@ func (s *systemContractCallbacks) CreateOnBatchEndTransaction(ctx context.Contex } formedTx := types.NewTx(tx) - s.logger.Info("CreateOnBatchEndTransaction: Successfully created signed transaction", "transactionHash", formedTx.Hash().Hex()) + s.logger.Info("CreateOnBatchEndTransaction: Successfully created synthetic transaction", log.TxKey, formedTx.Hash()) return formedTx, nil } diff --git a/integration/ethereummock/db.go b/integration/ethereummock/db.go index d3491e79b9..4a034fa387 100644 --- a/integration/ethereummock/db.go +++ b/integration/ethereummock/db.go @@ -16,6 +16,9 @@ import ( "github.com/ten-protocol/go-ten/go/enclave/core" ) +// HeightCommittedBlocks is the number of blocks deep a transaction must be to be considered safe from reorganisations. +const HeightCommittedBlocks = 15 + // Received blocks ar stored here type blockResolverInMem struct { blockCache map[common.L1BlockHash]*types.Block @@ -162,7 +165,7 @@ func (m *Node) removeCommittedTransactions( resolver *blockResolverInMem, db TxDB, ) []*types.Transaction { - if cb.NumberU64() <= common.HeightCommittedBlocks { + if cb.NumberU64() <= HeightCommittedBlocks { return mempool } @@ -170,7 +173,7 @@ func (m *Node) removeCommittedTransactions( i := 0 for { - if i == common.HeightCommittedBlocks { + if i == HeightCommittedBlocks { break } diff --git a/tools/walletextension/services/conn_utils.go b/tools/walletextension/services/conn_utils.go index 4b7813fc52..bc4876bf14 100644 --- a/tools/walletextension/services/conn_utils.go +++ b/tools/walletextension/services/conn_utils.go @@ -3,6 +3,7 @@ package services import ( "context" "fmt" + "time" gethlog "github.com/ethereum/go-ethereum/log" pool "github.com/jolestar/go-commons-pool/v2" @@ -69,13 +70,24 @@ func readEncKey(hostAddrHTTP string, logger gethlog.Logger) []byte { rpcClient, err := gethrpc.Dial(hostAddrHTTP) if err != nil { logger.Crit("failed to connect to the node", "err", err) + return nil } defer rpcClient.Close() - k, err := tenrpc.ReadEnclaveKey(rpcClient) - if err != nil { - logger.Crit("failed to read enc key", "err", err) + n := 0 + for { + n++ + k, err := tenrpc.ReadEnclaveKey(rpcClient) + if err != nil { + logger.Warn("failed to read enc key", "err", err) + if n > 10 { // wait for ~1m for the backend node to spin up and respond + logger.Crit("failed to read enc key", "err", err) + return nil + } + time.Sleep(time.Duration(n) * time.Second) + } else { + return k + } } - return k } func (rpc *BackendRPC) ConnectWS(ctx context.Context, account *wecommon.GWAccount) (*tenrpc.EncRPCClient, error) {