diff --git a/go/common/headers.go b/go/common/headers.go index 8c06b9c05e..bea9c8ed91 100644 --- a/go/common/headers.go +++ b/go/common/headers.go @@ -47,6 +47,12 @@ type BatchHeader struct { CrossChainTree SerializedCrossChainTree `json:"crossChainTree"` // Those are the leafs of the merkle tree hashed for privacy. Necessary for clients to be able to build proofs as they have no access to all transactions in a batch or their receipts. } +// IsGenesis indicates whether the batch is the genesis batch. +// todo (#718) - Change this to a check against a hardcoded genesis hash. +func (b *BatchHeader) IsGenesis() bool { + return b.Number.Cmp(big.NewInt(int64(L2GenesisHeight))) == 0 +} + type batchHeaderEncoding struct { Hash common.Hash `json:"hash"` ParentHash L2BatchHash `json:"parentHash"` diff --git a/go/enclave/components/batch_executor.go b/go/enclave/components/batch_executor.go index ea5b47516f..a49555ea59 100644 --- a/go/enclave/components/batch_executor.go +++ b/go/enclave/components/batch_executor.go @@ -144,7 +144,7 @@ func (executor *batchExecutor) ComputeBatch(ctx context.Context, context *BatchE } // These variables will be used to create the new batch - parentBatch, err := executor.storage.FetchBatch(ctx, context.ParentPtr) + parentBatch, err := executor.storage.FetchBatchHeader(ctx, context.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 @@ -154,9 +154,9 @@ func (executor *batchExecutor) ComputeBatch(ctx context.Context, context *BatchE } parentBlock := block - if parentBatch.Header.L1Proof != block.Hash() { + if parentBatch.L1Proof != block.Hash() { var err error - parentBlock, err = executor.storage.FetchBlock(ctx, parentBatch.Header.L1Proof) + parentBlock, err = executor.storage.FetchBlock(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 @@ -164,7 +164,7 @@ func (executor *batchExecutor) ComputeBatch(ctx context.Context, context *BatchE } // Create a new batch based on the fromBlock of inclusion of the previous, including all new transactions - batch := core.DeterministicEmptyBatch(parentBatch.Header, block, context.AtTime, context.SequencerNo, context.BaseFee, context.Creator) + batch := core.DeterministicEmptyBatch(parentBatch, block, context.AtTime, context.SequencerNo, context.BaseFee, context.Creator) stateDB, err := executor.batchRegistry.GetBatchState(ctx, &batch.Header.ParentHash) if err != nil { @@ -286,7 +286,10 @@ func (executor *batchExecutor) ExecuteBatch(ctx context.Context, batch *core.Bat if cb.Batch.Hash() != batch.Hash() { // todo @stefan - generate a validator challenge here and return it - executor.logger.Error(fmt.Sprintf("Error validating batch. Calculated: %+v Incoming: %+v\n", cb.Batch.Header, batch.Header)) + executor.logger.Error(fmt.Sprintf("Error validating batch. Calculated: %+v Incoming: %+v", cb.Batch.Header, batch.Header)) + for i, transaction := range batch.Transactions { + executor.logger.Error(fmt.Sprintf("Tx %d. Hash %s. Len %d", i, transaction.Hash(), len(transaction.Data()))) + } return nil, fmt.Errorf("batch is in invalid state. Incoming hash: %s Computed hash: %s", batch.Hash(), cb.Batch.Hash()) } diff --git a/go/enclave/components/batch_registry.go b/go/enclave/components/batch_registry.go index c76b967f95..9fdbf06d69 100644 --- a/go/enclave/components/batch_registry.go +++ b/go/enclave/components/batch_registry.go @@ -8,6 +8,8 @@ import ( "sync" "time" + "github.com/ten-protocol/go-ten/go/common/measure" + "github.com/ten-protocol/go-ten/go/common" "github.com/ethereum/go-ethereum/core/types" @@ -18,7 +20,6 @@ import ( "github.com/ten-protocol/go-ten/go/common/async" "github.com/ten-protocol/go-ten/go/common/errutil" "github.com/ten-protocol/go-ten/go/common/log" - "github.com/ten-protocol/go-ten/go/common/measure" "github.com/ten-protocol/go-ten/go/enclave/core" "github.com/ten-protocol/go-ten/go/enclave/limiters" gethrpc "github.com/ten-protocol/go-ten/lib/gethfork/rpc" @@ -37,7 +38,7 @@ type batchRegistry struct { func NewBatchRegistry(storage storage.Storage, logger gethlog.Logger) BatchRegistry { var headBatchSeq *big.Int - headBatch, err := storage.FetchHeadBatch(context.Background()) + headBatch, err := storage.FetchHeadBatchHeader(context.Background()) if err != nil { if errors.Is(err, errutil.ErrNotFound) { headBatchSeq = nil @@ -46,7 +47,7 @@ func NewBatchRegistry(storage storage.Storage, logger gethlog.Logger) BatchRegis return nil } } else { - headBatchSeq = headBatch.SeqNo() + headBatchSeq = headBatch.SequencerOrderNo } return &batchRegistry{ @@ -77,22 +78,29 @@ func (br *batchRegistry) UnsubscribeFromBatches() { func (br *batchRegistry) OnL1Reorg(_ *BlockIngestionType) { // refresh the cached head batch from the database because there was an L1 reorg - headBatch, err := br.storage.FetchHeadBatch(context.Background()) + headBatch, err := br.storage.FetchHeadBatchHeader(context.Background()) if err != nil { br.logger.Error("Could not fetch head batch", log.ErrKey, err) return } - br.headBatchSeq = headBatch.SeqNo() + br.headBatchSeq = headBatch.SequencerOrderNo } -func (br *batchRegistry) OnBatchExecuted(batch *core.Batch, receipts types.Receipts) { +func (br *batchRegistry) OnBatchExecuted(batchHeader *common.BatchHeader, receipts types.Receipts) { + defer core.LogMethodDuration(br.logger, measure.NewStopwatch(), "OnBatchExecuted", log.BatchHashKey, batchHeader.Hash()) br.callbackMutex.RLock() defer br.callbackMutex.RUnlock() - defer core.LogMethodDuration(br.logger, measure.NewStopwatch(), "Sending batch and events", log.BatchHashKey, batch.Hash()) - - br.headBatchSeq = batch.SeqNo() + txs, err := br.storage.FetchBatchTransactionsBySeq(context.Background(), batchHeader.SequencerOrderNo.Uint64()) + if err != nil { + br.logger.Crit("cannot get transactions. ", log.ErrKey, err) + } + br.headBatchSeq = batchHeader.SequencerOrderNo if br.batchesCallback != nil { + batch := &core.Batch{ + Header: batchHeader, + Transactions: txs, + } br.batchesCallback(batch, receipts) } @@ -105,13 +113,13 @@ func (br *batchRegistry) HasGenesisBatch() (bool, error) { func (br *batchRegistry) BatchesAfter(ctx context.Context, batchSeqNo uint64, upToL1Height uint64, rollupLimiter limiters.RollupLimiter) ([]*core.Batch, []*types.Block, error) { // sanity check - headBatch, err := br.storage.FetchBatchBySeqNo(ctx, br.HeadBatchSeq().Uint64()) + headBatch, err := br.storage.FetchBatchHeaderBySeqNo(ctx, br.HeadBatchSeq().Uint64()) if err != nil { return nil, nil, err } - if headBatch.SeqNo().Uint64() < batchSeqNo { - return nil, nil, fmt.Errorf("head batch height %d is in the past compared to requested batch %d", headBatch.SeqNo().Uint64(), batchSeqNo) + if headBatch.SequencerOrderNo.Uint64() < batchSeqNo { + return nil, nil, fmt.Errorf("head batch height %d is in the past compared to requested batch %d", headBatch.SequencerOrderNo.Uint64(), batchSeqNo) } resultBatches := make([]*core.Batch, 0) @@ -119,7 +127,7 @@ func (br *batchRegistry) BatchesAfter(ctx context.Context, batchSeqNo uint64, up currentBatchSeq := batchSeqNo var currentBlock *types.Block - for currentBatchSeq <= headBatch.SeqNo().Uint64() { + for currentBatchSeq <= headBatch.SequencerOrderNo.Uint64() { batch, err := br.storage.FetchBatchBySeqNo(ctx, currentBatchSeq) if err != nil { return nil, nil, fmt.Errorf("could not retrieve batch by sequence number %d. Cause: %w", currentBatchSeq, err) @@ -168,11 +176,7 @@ func (br *batchRegistry) BatchesAfter(ctx context.Context, batchSeqNo uint64, up } func (br *batchRegistry) GetBatchState(ctx context.Context, hash *common.L2BatchHash) (*state.StateDB, error) { - batch, err := br.storage.FetchBatch(ctx, *hash) - if err != nil { - return nil, err - } - return getBatchState(ctx, br.storage, batch) + return getBatchState(ctx, br.storage, *hash) } func (br *batchRegistry) GetBatchStateAtHeight(ctx context.Context, blockNumber *gethrpc.BlockNumber) (*state.StateDB, error) { @@ -182,17 +186,17 @@ func (br *batchRegistry) GetBatchStateAtHeight(ctx context.Context, blockNumber return nil, err } - return getBatchState(ctx, br.storage, batch) + return getBatchState(ctx, br.storage, batch.Hash()) } -func getBatchState(ctx context.Context, storage storage.Storage, batch *core.Batch) (*state.StateDB, error) { - blockchainState, err := storage.CreateStateDB(ctx, batch.Hash()) +func getBatchState(ctx context.Context, storage storage.Storage, batchHash common.L2BatchHash) (*state.StateDB, error) { + blockchainState, err := storage.CreateStateDB(ctx, batchHash) if err != nil { return nil, fmt.Errorf("could not create stateDB. Cause: %w", err) } if blockchainState == nil { - return nil, fmt.Errorf("unable to fetch chain state for batch %s", batch.Hash().Hex()) + return nil, fmt.Errorf("unable to fetch chain state for batch %s", batchHash.Hex()) } return blockchainState, err diff --git a/go/enclave/components/interfaces.go b/go/enclave/components/interfaces.go index cb74f0bf4b..b4f43fa8fb 100644 --- a/go/enclave/components/interfaces.go +++ b/go/enclave/components/interfaces.go @@ -102,7 +102,7 @@ type BatchRegistry interface { SubscribeForExecutedBatches(func(*core.Batch, types.Receipts)) UnsubscribeFromBatches() - OnBatchExecuted(batch *core.Batch, receipts types.Receipts) + OnBatchExecuted(batch *common.BatchHeader, receipts types.Receipts) OnL1Reorg(*BlockIngestionType) // HasGenesisBatch - returns if genesis batch is available yet or not, or error in case diff --git a/go/enclave/components/rollup_compression.go b/go/enclave/components/rollup_compression.go index ca4536e0ca..9b66ff040f 100644 --- a/go/enclave/components/rollup_compression.go +++ b/go/enclave/components/rollup_compression.go @@ -175,8 +175,8 @@ func (rc *RollupCompression) createRollupHeader(ctx context.Context, rollup *cor } reorgMap := make(map[uint64]bool) for _, batch := range reorgedBatches { - rc.logger.Info("Reorg batch", log.BatchSeqNoKey, batch.SeqNo().Uint64()) - reorgMap[batch.SeqNo().Uint64()] = true + rc.logger.Info("Reorg batch", log.BatchSeqNoKey, batch.SequencerOrderNo.Uint64()) + reorgMap[batch.SequencerOrderNo.Uint64()] = true } for i, batch := range batches { @@ -404,7 +404,7 @@ func (rc *RollupCompression) executeAndSaveIncompleteBatches(ctx context.Context parentHash := calldataRollupHeader.FirstCanonParentHash if calldataRollupHeader.FirstBatchSequence.Uint64() != common.L2GenesisSeqNo { - _, err := rc.storage.FetchBatch(ctx, parentHash) + _, err := rc.storage.FetchBatchHeader(ctx, parentHash) if err != nil { rc.logger.Error("Could not find batch mentioned in the rollup. This should not happen.", log.ErrKey, err) return err @@ -470,11 +470,11 @@ func (rc *RollupCompression) executeAndSaveIncompleteBatches(ctx context.Context if err != nil { return err } - err = rc.storage.StoreExecutedBatch(ctx, genBatch, nil) + err = rc.storage.StoreExecutedBatch(ctx, genBatch.Header, nil) if err != nil { return err } - rc.batchRegistry.OnBatchExecuted(genBatch, nil) + rc.batchRegistry.OnBatchExecuted(genBatch.Header, nil) rc.logger.Info("Stored genesis", log.BatchHashKey, genBatch.Hash()) parentHash = genBatch.Hash() @@ -514,11 +514,11 @@ func (rc *RollupCompression) executeAndSaveIncompleteBatches(ctx context.Context if err != nil { return err } - err = rc.storage.StoreExecutedBatch(ctx, computedBatch.Batch, computedBatch.Receipts) + err = rc.storage.StoreExecutedBatch(ctx, computedBatch.Batch.Header, computedBatch.Receipts) if err != nil { return err } - rc.batchRegistry.OnBatchExecuted(computedBatch.Batch, nil) + rc.batchRegistry.OnBatchExecuted(computedBatch.Batch.Header, nil) parentHash = computedBatch.Batch.Hash() } diff --git a/go/enclave/core/batch.go b/go/enclave/core/batch.go index bc97ff22e1..cdc15cf9e4 100644 --- a/go/enclave/core/batch.go +++ b/go/enclave/core/batch.go @@ -49,12 +49,6 @@ func (b *Batch) NumberU64() uint64 { return b.Header.Number.Uint64() } func (b *Batch) Number() *big.Int { return new(big.Int).Set(b.Header.Number) } func (b *Batch) SeqNo() *big.Int { return new(big.Int).Set(b.Header.SequencerOrderNo) } -// IsGenesis indicates whether the batch is the genesis batch. -// todo (#718) - Change this to a check against a hardcoded genesis hash. -func (b *Batch) IsGenesis() bool { - return b.Header.Number.Cmp(big.NewInt(int64(common.L2GenesisHeight))) == 0 -} - func (b *Batch) ToExtBatch(transactionBlobCrypto crypto.DataEncryptionService, compression compression.DataCompressionService) (*common.ExtBatch, error) { txHashes := make([]gethcommon.Hash, len(b.Transactions)) for idx, tx := range b.Transactions { diff --git a/go/enclave/enclave.go b/go/enclave/enclave.go index 7db4324e61..98e2e160a3 100644 --- a/go/enclave/enclave.go +++ b/go/enclave/enclave.go @@ -510,7 +510,7 @@ func (e *enclaveImpl) SubmitBatch(ctx context.Context, extBatch *common.ExtBatch e.logger.Info("Received new p2p batch", log.BatchHeightKey, extBatch.Header.Number, log.BatchHashKey, extBatch.Hash(), "l1", extBatch.Header.L1Proof) seqNo := extBatch.Header.SequencerOrderNo.Uint64() if seqNo > common.L2GenesisSeqNo+1 { - _, err := e.storage.FetchBatchBySeqNo(ctx, seqNo-1) + _, err := e.storage.FetchBatchHeaderBySeqNo(ctx, seqNo-1) if err != nil { return responses.ToInternalError(fmt.Errorf("could not find previous batch with seq: %d", seqNo-1)) } diff --git a/go/enclave/evm/chain_context.go b/go/enclave/evm/chain_context.go index 75e4b942e6..a69ef9b89e 100644 --- a/go/enclave/evm/chain_context.go +++ b/go/enclave/evm/chain_context.go @@ -42,7 +42,7 @@ func (occ *ObscuroChainContext) GetHeader(hash common.Hash, _ uint64) *types.Hea ctx, cancelCtx := context.WithTimeout(context.Background(), occ.config.RPCTimeout) defer cancelCtx() - batch, err := occ.storage.FetchBatch(ctx, hash) + batch, err := occ.storage.FetchBatchHeader(ctx, hash) if err != nil { if errors.Is(err, errutil.ErrNotFound) { return nil @@ -50,7 +50,7 @@ func (occ *ObscuroChainContext) GetHeader(hash common.Hash, _ uint64) *types.Hea occ.logger.Crit("Could not retrieve rollup", log.ErrKey, err) } - h, err := occ.gethEncodingService.CreateEthHeaderForBatch(ctx, batch.Header) + h, err := occ.gethEncodingService.CreateEthHeaderForBatch(ctx, batch) if err != nil { occ.logger.Crit("Could not convert to eth header", log.ErrKey, err) return nil diff --git a/go/enclave/evm/ethchainadapter/eth_chainadapter.go b/go/enclave/evm/ethchainadapter/eth_chainadapter.go index c64529be46..1dbb927724 100644 --- a/go/enclave/evm/ethchainadapter/eth_chainadapter.go +++ b/go/enclave/evm/ethchainadapter/eth_chainadapter.go @@ -59,12 +59,12 @@ func (e *EthChainAdapter) CurrentBlock() *gethtypes.Header { ctx, cancelCtx := context.WithTimeout(context.Background(), e.config.RPCTimeout) defer cancelCtx() - currentBatch, err := e.storage.FetchBatchBySeqNo(ctx, currentBatchSeqNo.Uint64()) + currentBatch, err := e.storage.FetchBatchHeaderBySeqNo(ctx, currentBatchSeqNo.Uint64()) if err != nil { e.logger.Warn("unable to retrieve batch seq no", "currentBatchSeqNo", currentBatchSeqNo, log.ErrKey, err) return nil } - batch, err := e.gethEncoding.CreateEthHeaderForBatch(ctx, currentBatch.Header) + batch, err := e.gethEncoding.CreateEthHeaderForBatch(ctx, currentBatch) if err != nil { e.logger.Warn("unable to convert batch to eth header ", "currentBatchSeqNo", currentBatchSeqNo, log.ErrKey, err) return nil diff --git a/go/enclave/nodetype/common.go b/go/enclave/nodetype/common.go index a99dc0f690..a0c083ac4d 100644 --- a/go/enclave/nodetype/common.go +++ b/go/enclave/nodetype/common.go @@ -20,8 +20,8 @@ func ExportCrossChainData(ctx context.Context, storage storage.Storage, fromSeqN return nil, errutil.ErrCrossChainBundleNoBatches } - blockHash := canonicalBatches[len(canonicalBatches)-1].Header.L1Proof - batchHash := canonicalBatches[len(canonicalBatches)-1].Header.Hash() + blockHash := canonicalBatches[len(canonicalBatches)-1].L1Proof + batchHash := canonicalBatches[len(canonicalBatches)-1].Hash() block, err := storage.FetchBlock(ctx, blockHash) if err != nil { @@ -30,8 +30,8 @@ func ExportCrossChainData(ctx context.Context, storage storage.Storage, fromSeqN crossChainHashes := make([][]byte, 0) for _, batch := range canonicalBatches { - if batch.Header.CrossChainRoot != gethcommon.BigToHash(gethcommon.Big0) { - crossChainHashes = append(crossChainHashes, batch.Header.CrossChainRoot.Bytes()) + if batch.CrossChainRoot != gethcommon.BigToHash(gethcommon.Big0) { + crossChainHashes = append(crossChainHashes, batch.CrossChainRoot.Bytes()) } } diff --git a/go/enclave/nodetype/sequencer.go b/go/enclave/nodetype/sequencer.go index ff7ed22dc4..b5f017df3f 100644 --- a/go/enclave/nodetype/sequencer.go +++ b/go/enclave/nodetype/sequencer.go @@ -205,7 +205,7 @@ func (s *sequencer) createNewHeadBatch(ctx context.Context, l1HeadBlock *common. if headBatchSeq == nil { headBatchSeq = big.NewInt(int64(common.L2GenesisSeqNo)) } - headBatch, err := s.storage.FetchBatchBySeqNo(ctx, headBatchSeq.Uint64()) + headBatch, err := s.storage.FetchBatchHeaderBySeqNo(ctx, headBatchSeq.Uint64()) if err != nil { return err } @@ -220,7 +220,7 @@ func (s *sequencer) createNewHeadBatch(ctx context.Context, l1HeadBlock *common. } // sanity check that the headBatch.Header.L1Proof is an ancestor of the l1HeadBlock - b, err := s.storage.FetchBlock(ctx, headBatch.Header.L1Proof) + b, err := s.storage.FetchBlock(ctx, headBatch.L1Proof) if err != nil { return err } @@ -337,11 +337,11 @@ func (s *sequencer) StoreExecutedBatch(ctx context.Context, batch *core.Batch, r return fmt.Errorf("failed to store batch. Cause: %w", err) } - if err := s.storage.StoreExecutedBatch(ctx, batch, receipts); err != nil { + if err := s.storage.StoreExecutedBatch(ctx, batch.Header, receipts); err != nil { return fmt.Errorf("failed to store batch. Cause: %w", err) } - s.batchRegistry.OnBatchExecuted(batch, receipts) + s.batchRegistry.OnBatchExecuted(batch.Header, receipts) return nil } @@ -373,8 +373,8 @@ func (s *sequencer) CreateRollup(ctx context.Context, lastBatchNo uint64) (*comm } func (s *sequencer) duplicateBatches(ctx context.Context, l1Head *types.Block, nonCanonicalL1Path []common.L1BlockHash, canonicalL1Path []common.L1BlockHash) error { - batchesToDuplicate := make([]*core.Batch, 0) - batchesToExclude := make(map[uint64]*core.Batch, 0) + batchesToDuplicate := make([]*common.BatchHeader, 0) + batchesToExclude := make(map[uint64]*common.BatchHeader, 0) // read the batches attached to these blocks for _, l1BlockHash := range nonCanonicalL1Path { @@ -399,7 +399,7 @@ func (s *sequencer) duplicateBatches(ctx context.Context, l1Head *types.Block, n return fmt.Errorf("could not FetchBatchesByBlock %s. Cause %w", l1BlockHash, err) } for _, batch := range batches { - batchesToExclude[batch.NumberU64()] = batch + batchesToExclude[batch.Number.Uint64()] = batch } } @@ -409,20 +409,20 @@ func (s *sequencer) duplicateBatches(ctx context.Context, l1Head *types.Block, n // sort by height sort.Slice(batchesToDuplicate, func(i, j int) bool { - return batchesToDuplicate[i].Number().Cmp(batchesToDuplicate[j].Number()) == -1 + return batchesToDuplicate[i].Number.Cmp(batchesToDuplicate[j].Number) == -1 }) - currentHead := batchesToDuplicate[0].Header.ParentHash + currentHead := batchesToDuplicate[0].ParentHash // find all batches for that path for i, orphanBatch := range batchesToDuplicate { // sanity check that all these batches are consecutive - if i > 0 && batchesToDuplicate[i].Header.ParentHash != batchesToDuplicate[i-1].Hash() { + if i > 0 && batchesToDuplicate[i].ParentHash != batchesToDuplicate[i-1].Hash() { s.logger.Crit("the batches that must be duplicated are invalid") } - if batchesToExclude[orphanBatch.NumberU64()] != nil { - s.logger.Info("Not duplicating batch because there is already a canonical batch on that height", log.BatchSeqNoKey, orphanBatch.SeqNo()) - currentHead = batchesToExclude[orphanBatch.NumberU64()].Hash() + if batchesToExclude[orphanBatch.Number.Uint64()] != nil { + s.logger.Info("Not duplicating batch because there is already a canonical batch on that height", log.BatchSeqNoKey, orphanBatch.SequencerOrderNo) + currentHead = batchesToExclude[orphanBatch.Number.Uint64()].Hash() continue } sequencerNo, err := s.storage.FetchCurrentSequencerNo(ctx) @@ -430,8 +430,11 @@ func (s *sequencer) duplicateBatches(ctx context.Context, l1Head *types.Block, n return fmt.Errorf("could not fetch sequencer no. Cause %w", err) } sequencerNo = sequencerNo.Add(sequencerNo, big.NewInt(1)) + // todo + var transactions common.L2Transactions + // transactions:=orphanBatch.Transactions // create the duplicate and store/broadcast it, recreate batch even if it was empty - cb, err := s.produceBatch(ctx, sequencerNo, l1Head.Hash(), currentHead, orphanBatch.Transactions, orphanBatch.Header.Time, false) + cb, err := s.produceBatch(ctx, sequencerNo, l1Head.Hash(), currentHead, transactions, orphanBatch.Time, false) if err != nil { return fmt.Errorf("could not produce batch. Cause %w", err) } diff --git a/go/enclave/nodetype/validator.go b/go/enclave/nodetype/validator.go index 1ccd9ce81b..8f7783aabf 100644 --- a/go/enclave/nodetype/validator.go +++ b/go/enclave/nodetype/validator.go @@ -105,62 +105,73 @@ func (val *obsValidator) ExecuteStoredBatches(ctx context.Context) error { startMempool(val.batchRegistry, val.mempool) - for _, batch := range batches { - if batch.IsGenesis() { - if err = val.handleGenesis(ctx, batch); err != nil { + for _, batchHeader := range batches { + if batchHeader.IsGenesis() { + if err = val.handleGenesis(ctx, batchHeader); err != nil { return err } } - val.logger.Trace("Executing stored batch", log.BatchSeqNoKey, batch.SeqNo()) + val.logger.Trace("Executing stored batchHeader", log.BatchSeqNoKey, batchHeader.SequencerOrderNo) - // check batch execution prerequisites - canExecute, err := val.executionPrerequisites(ctx, batch) + // check batchHeader execution prerequisites + canExecute, err := val.executionPrerequisites(ctx, batchHeader) if err != nil { - return fmt.Errorf("could not determine the execution prerequisites for batch %s. Cause: %w", batch.Hash(), err) + return fmt.Errorf("could not determine the execution prerequisites for batchHeader %s. Cause: %w", batchHeader.Hash(), err) } - val.logger.Trace("Can execute stored batch", log.BatchSeqNoKey, batch.SeqNo(), "can", canExecute) + val.logger.Trace("Can execute stored batchHeader", log.BatchSeqNoKey, batchHeader.SequencerOrderNo, "can", canExecute) if canExecute { + txs, err := val.storage.FetchBatchTransactionsBySeq(ctx, batchHeader.SequencerOrderNo.Uint64()) + if err != nil { + return fmt.Errorf("could not get txs for batch %s. Cause: %w", batchHeader.Hash(), err) + } + val.logger.Debug("Read txs ***", log.BatchHeightKey, batchHeader.Number, "nr", len(txs)) + + batch := &core.Batch{ + Header: batchHeader, + Transactions: txs, + } + receipts, err := val.batchExecutor.ExecuteBatch(ctx, batch) if err != nil { - return fmt.Errorf("could not execute batch %s. Cause: %w", batch.Hash(), err) + return fmt.Errorf("could not execute batchHeader %s. Cause: %w", batchHeader.Hash(), err) } - err = val.storage.StoreExecutedBatch(ctx, batch, receipts) + err = val.storage.StoreExecutedBatch(ctx, batchHeader, receipts) if err != nil { - return fmt.Errorf("could not store executed batch %s. Cause: %w", batch.Hash(), err) + return fmt.Errorf("could not store executed batchHeader %s. Cause: %w", batchHeader.Hash(), err) } err = val.mempool.Chain.IngestNewBlock(batch) if err != nil { - return fmt.Errorf("failed to feed batch into the virtual eth chain- %w", err) + return fmt.Errorf("failed to feed batchHeader into the virtual eth chain- %w", err) } - val.batchRegistry.OnBatchExecuted(batch, receipts) + val.batchRegistry.OnBatchExecuted(batchHeader, receipts) } } return nil } -func (val *obsValidator) executionPrerequisites(ctx context.Context, batch *core.Batch) (bool, error) { +func (val *obsValidator) executionPrerequisites(ctx context.Context, batch *common.BatchHeader) (bool, error) { // 1.l1 block exists - block, err := val.storage.FetchBlock(ctx, batch.Header.L1Proof) + block, err := val.storage.FetchBlock(ctx, batch.L1Proof) if err != nil && errors.Is(err, errutil.ErrNotFound) { - val.logger.Warn("Error fetching block", log.BlockHashKey, batch.Header.L1Proof, log.ErrKey, err) + val.logger.Warn("Error fetching block", log.BlockHashKey, batch.L1Proof, log.ErrKey, err) return false, err } - val.logger.Trace("l1 block exists", log.BatchSeqNoKey, batch.SeqNo()) + val.logger.Trace("l1 block exists", log.BatchSeqNoKey, batch.SequencerOrderNo) // 2. parent was executed - parentExecuted, err := val.storage.BatchWasExecuted(ctx, batch.Header.ParentHash) + parentExecuted, err := val.storage.BatchWasExecuted(ctx, batch.ParentHash) if err != nil { - val.logger.Info("Error reading execution status of batch", log.BatchHashKey, batch.Header.ParentHash, log.ErrKey, err) + val.logger.Info("Error reading execution status of batch", log.BatchHashKey, batch.ParentHash, log.ErrKey, err) return false, err } - val.logger.Trace("parentExecuted", log.BatchSeqNoKey, batch.SeqNo(), "val", parentExecuted) + val.logger.Trace("parentExecuted", log.BatchSeqNoKey, batch.SequencerOrderNo, "val", parentExecuted) return block != nil && parentExecuted, nil } -func (val *obsValidator) handleGenesis(ctx context.Context, batch *core.Batch) error { - genBatch, _, err := val.batchExecutor.CreateGenesisState(ctx, batch.Header.L1Proof, batch.Header.Time, batch.Header.Coinbase, batch.Header.BaseFee) +func (val *obsValidator) handleGenesis(ctx context.Context, batch *common.BatchHeader) error { + genBatch, _, err := val.batchExecutor.CreateGenesisState(ctx, batch.L1Proof, batch.Time, batch.Coinbase, batch.BaseFee) if err != nil { return err } @@ -169,7 +180,7 @@ func (val *obsValidator) handleGenesis(ctx context.Context, batch *core.Batch) e return fmt.Errorf("received invalid genesis batch") } - err = val.storage.StoreExecutedBatch(ctx, genBatch, nil) + err = val.storage.StoreExecutedBatch(ctx, genBatch.Header, nil) if err != nil { return err } diff --git a/go/enclave/rpc/EstimateGas.go b/go/enclave/rpc/EstimateGas.go index 0c583da43e..f09a3d4b96 100644 --- a/go/enclave/rpc/EstimateGas.go +++ b/go/enclave/rpc/EstimateGas.go @@ -75,7 +75,7 @@ func EstimateGasExecute(builder *CallBuilder[CallParamsWithBlock, hexutil.Uint64 } headBatchSeq := rpc.registry.HeadBatchSeq() - batch, err := rpc.storage.FetchBatchBySeqNo(builder.ctx, headBatchSeq.Uint64()) + batch, err := rpc.storage.FetchBatchHeaderBySeqNo(builder.ctx, headBatchSeq.Uint64()) if err != nil { return err } @@ -83,7 +83,7 @@ func EstimateGasExecute(builder *CallBuilder[CallParamsWithBlock, hexutil.Uint64 // We divide the total estimated l1 cost by the l2 fee per gas in order to convert // the expected cost into l2 gas based on current pricing. // todo @siliev - add overhead when the base fee becomes dynamic. - publishingGas := big.NewInt(0).Div(l1Cost, batch.Header.BaseFee) + publishingGas := big.NewInt(0).Div(l1Cost, batch.BaseFee) // The one additional gas captures the modulo leftover in some edge cases // where BaseFee is bigger than the l1cost. diff --git a/go/enclave/rpc/GetLogs.go b/go/enclave/rpc/GetLogs.go index 3d938883d0..8aac4ee974 100644 --- a/go/enclave/rpc/GetLogs.go +++ b/go/enclave/rpc/GetLogs.go @@ -51,12 +51,12 @@ func GetLogsExecute(builder *CallBuilder[filters.FilterCriteria, []*types.Log], from := filter.FromBlock if from != nil && from.Int64() < 0 { - batch, err := rpc.storage.FetchBatchBySeqNo(builder.ctx, rpc.registry.HeadBatchSeq().Uint64()) + batch, err := rpc.storage.FetchBatchHeaderBySeqNo(builder.ctx, rpc.registry.HeadBatchSeq().Uint64()) if err != nil { // system error return fmt.Errorf("could not retrieve head batch. Cause: %w", err) } - from = batch.Number() + from = batch.Number } // Set from to the height of the block hash diff --git a/go/enclave/rpc/GetTransactionCount.go b/go/enclave/rpc/GetTransactionCount.go index 99989b32af..5282f733fe 100644 --- a/go/enclave/rpc/GetTransactionCount.go +++ b/go/enclave/rpc/GetTransactionCount.go @@ -52,7 +52,7 @@ func GetTransactionCountExecute(builder *CallBuilder[uint64, string], rpc *Encry } var nonce uint64 - l2Head, err := rpc.storage.FetchBatchBySeqNo(builder.ctx, *builder.Param) + l2Head, err := rpc.storage.FetchBatchHeaderBySeqNo(builder.ctx, *builder.Param) if err == nil { // todo - we should return an error when head state is not available, but for current test situations with race // conditions we allow it to return zero while head state is uninitialized diff --git a/go/enclave/storage/enclavedb/batch.go b/go/enclave/storage/enclavedb/batch.go index de06dba08a..4c142b03ce 100644 --- a/go/enclave/storage/enclavedb/batch.go +++ b/go/enclave/storage/enclavedb/batch.go @@ -19,54 +19,21 @@ import ( ) const ( - selectBatch = "select b.header, bb.content from batch b join batch_body bb on b.body=bb.id" - queryReceipts = "select exec_tx.receipt, tx.content, batch.hash, batch.height from exec_tx join tx on tx.id=exec_tx.tx join batch on batch.sequence=exec_tx.batch " ) -// WriteBatchAndTransactions - persists the batch and the transactions -// todo split into simple functions, and let the higher level handle the logic -func WriteBatchAndTransactions(ctx context.Context, dbtx *sql.Tx, batch *core.Batch, convertedHash gethcommon.Hash, blockId int64) error { - // todo - optimize for reorgs - batchBodyID := batch.SeqNo().Uint64() - - body, err := rlp.EncodeToBytes(batch.Transactions) - if err != nil { - return fmt.Errorf("could not encode L2 transactions. Cause: %w", err) - } +func WriteBatchHeader(ctx context.Context, dbtx *sql.Tx, batch *core.Batch, convertedHash gethcommon.Hash, blockId int64, isCanonical bool) error { header, err := rlp.EncodeToBytes(batch.Header) if err != nil { return fmt.Errorf("could not encode batch header. Cause: %w", err) } - - _, err = dbtx.ExecContext(ctx, "replace into batch_body values (?,?)", batchBodyID, body) - if err != nil { - return err - } - - isL1ProofCanonical, err := IsCanonicalBlock(ctx, dbtx, &batch.Header.L1Proof) - if err != nil { - return err - } - parentIsCanon, err := IsCanonicalBatch(ctx, dbtx, &batch.Header.ParentHash) - if err != nil { - return err - } - parentIsCanon = parentIsCanon || batch.SeqNo().Uint64() <= common.L2GenesisSeqNo+2 - - // sanity check that the parent is canonical - if isL1ProofCanonical && !parentIsCanon { - panic(fmt.Errorf("invalid chaining. Batch %s is canonical. Parent %s is not", batch.Hash(), batch.Header.ParentHash)) - } - args := []any{ batch.Header.SequencerOrderNo.Uint64(), // sequence convertedHash, // converted_hash batch.Hash(), // hash batch.Header.Number.Uint64(), // height - isL1ProofCanonical, // is_canonical + isCanonical, // is_canonical header, // header blob - batchBodyID, // reference to the batch body batch.Header.L1Proof.Bytes(), // l1 proof hash } if blockId == 0 { @@ -75,14 +42,24 @@ func WriteBatchAndTransactions(ctx context.Context, dbtx *sql.Tx, batch *core.Ba args = append(args, blockId) } args = append(args, false) // executed - _, err = dbtx.ExecContext(ctx, "insert into batch values (?,?,?,?,?,?,?,?,?,?)", args...) + _, err = dbtx.ExecContext(ctx, "insert into batch values (?,?,?,?,?,?,?,?,?)", args...) + return err +} + +func ExistsBatchAtHeight(ctx context.Context, dbTx *sql.Tx, height *big.Int) (bool, error) { + var count int + err := dbTx.QueryRowContext(ctx, "select count(*) from batch where height=?", height.Uint64()).Scan(&count) if err != nil { - return err + return false, err } + return count > 0, nil +} - // creates a big insert statement for all transactions +// WriteTransactions - persists the batch and the transactions +func WriteTransactions(ctx context.Context, dbtx *sql.Tx, batch *core.Batch) error { + // creates a batch insert statement for all entries if len(batch.Transactions) > 0 { - insert := "replace into tx (hash, content, sender_address, nonce, idx, body) values " + repeat("(?,?,?,?,?,?)", ",", len(batch.Transactions)) + insert := "insert into tx (hash, content, sender_address, idx, batch_height) values " + repeat("(?,?,?,?,?)", ",", len(batch.Transactions)) args := make([]any, 0) for i, transaction := range batch.Transactions { @@ -96,23 +73,21 @@ func WriteBatchAndTransactions(ctx context.Context, dbtx *sql.Tx, batch *core.Ba return fmt.Errorf("unable to convert tx to message - %w", err) } - args = append(args, transaction.Hash()) // tx_hash - args = append(args, txBytes) // content - args = append(args, from.Bytes()) // sender_address - args = append(args, transaction.Nonce()) // nonce - args = append(args, i) // idx - args = append(args, batchBodyID) // the batch body which contained it + args = append(args, transaction.Hash()) // tx_hash + args = append(args, txBytes) // content + args = append(args, from.Bytes()) // sender_address + args = append(args, i) // idx + args = append(args, batch.Header.Number.Uint64()) // the batch height which contained it } - _, err = dbtx.ExecContext(ctx, insert, args...) + _, err := dbtx.ExecContext(ctx, insert, args...) if err != nil { return err } } - return nil } -func IsCanonicalBatch(ctx context.Context, dbtx *sql.Tx, hash *gethcommon.Hash) (bool, error) { +func IsCanonicalBatchHash(ctx context.Context, dbtx *sql.Tx, hash *gethcommon.Hash) (bool, error) { var isCanon bool err := dbtx.QueryRowContext(ctx, "select is_canonical from batch where hash=? ", hash.Bytes()).Scan(&isCanon) if err != nil { @@ -160,32 +135,32 @@ func GetTxId(ctx context.Context, dbtx *sql.Tx, txHash gethcommon.Hash) (uint64, return txId, err } -func ReadBatchBySeqNo(ctx context.Context, db *sql.DB, seqNo uint64) (*core.Batch, error) { - return fetchBatch(ctx, db, " where sequence=?", seqNo) +func ReadBatchHeaderBySeqNo(ctx context.Context, db *sql.DB, seqNo uint64) (*common.BatchHeader, error) { + return fetchBatchHeader(ctx, db, " where sequence=?", seqNo) } -func ReadBatchByHash(ctx context.Context, db *sql.DB, hash common.L2BatchHash) (*core.Batch, error) { - return fetchBatch(ctx, db, " where b.hash=? ", hash.Bytes()) +func ReadBatchHeaderByHash(ctx context.Context, db *sql.DB, hash common.L2BatchHash) (*common.BatchHeader, error) { + return fetchBatchHeader(ctx, db, " where b.hash=? ", hash.Bytes()) } -func ReadCanonicalBatchByHeight(ctx context.Context, db *sql.DB, height uint64) (*core.Batch, error) { - return fetchBatch(ctx, db, " where b.height=? and is_canonical=true", height) +func ReadCanonicalBatchHeaderByHeight(ctx context.Context, db *sql.DB, height uint64) (*common.BatchHeader, error) { + return fetchBatchHeader(ctx, db, " where b.height=? and is_canonical=true", height) } -func ReadNonCanonicalBatches(ctx context.Context, db *sql.DB, startAtSeq uint64, endSeq uint64) ([]*core.Batch, error) { +func ReadNonCanonicalBatches(ctx context.Context, db *sql.DB, startAtSeq uint64, endSeq uint64) ([]*common.BatchHeader, error) { return fetchBatches(ctx, db, " where b.sequence>=? and b.sequence <=? and b.is_canonical=false order by b.sequence", startAtSeq, endSeq) } -func ReadCanonicalBatches(ctx context.Context, db *sql.DB, startAtSeq uint64, endSeq uint64) ([]*core.Batch, error) { +func ReadCanonicalBatches(ctx context.Context, db *sql.DB, startAtSeq uint64, endSeq uint64) ([]*common.BatchHeader, error) { return fetchBatches(ctx, db, " where b.sequence>=? and b.sequence <=? and b.is_canonical=true order by b.sequence", startAtSeq, endSeq) } // todo - is there a better way to write this query? -func ReadCurrentHeadBatch(ctx context.Context, db *sql.DB) (*core.Batch, error) { - return fetchBatch(ctx, db, " where b.is_canonical=true and b.is_executed=true and b.height=(select max(b1.height) from batch b1 where b1.is_canonical=true and b1.is_executed=true)") +func ReadCurrentHeadBatchHeader(ctx context.Context, db *sql.DB) (*common.BatchHeader, error) { + return fetchBatchHeader(ctx, db, " where b.is_canonical=true and b.is_executed=true and b.height=(select max(b1.height) from batch b1 where b1.is_canonical=true and b1.is_executed=true)") } -func ReadBatchesByBlock(ctx context.Context, db *sql.DB, hash common.L1BlockHash) ([]*core.Batch, error) { +func ReadBatchesByBlock(ctx context.Context, db *sql.DB, hash common.L1BlockHash) ([]*common.BatchHeader, error) { return fetchBatches(ctx, db, " where l1_proof_hash=? order by b.sequence", hash.Bytes()) } @@ -206,15 +181,14 @@ func ReadCurrentSequencerNo(ctx context.Context, db *sql.DB) (*big.Int, error) { return big.NewInt(seq.Int64), nil } -func fetchBatch(ctx context.Context, db *sql.DB, whereQuery string, args ...any) (*core.Batch, error) { +func fetchBatchHeader(ctx context.Context, db *sql.DB, whereQuery string, args ...any) (*common.BatchHeader, error) { var header string - var body []byte - query := selectBatch + " " + whereQuery + query := "select b.header from batch b " + whereQuery var err error if len(args) > 0 { - err = db.QueryRowContext(ctx, query, args...).Scan(&header, &body) + err = db.QueryRowContext(ctx, query, args...).Scan(&header) } else { - err = db.QueryRowContext(ctx, query).Scan(&header, &body) + err = db.QueryRowContext(ctx, query).Scan(&header) } if err != nil { if errors.Is(err, sql.ErrNoRows) { @@ -227,23 +201,14 @@ func fetchBatch(ctx context.Context, db *sql.DB, whereQuery string, args ...any) if err := rlp.DecodeBytes([]byte(header), h); err != nil { return nil, fmt.Errorf("could not decode batch header. Cause: %w", err) } - txs := new([]*common.L2Tx) - if err := rlp.DecodeBytes(body, txs); err != nil { - return nil, fmt.Errorf("could not decode L2 transactions %v. Cause: %w", body, err) - } - - b := core.Batch{ - Header: h, - Transactions: *txs, - } - return &b, nil + return h, nil } -func fetchBatches(ctx context.Context, db *sql.DB, whereQuery string, args ...any) ([]*core.Batch, error) { - result := make([]*core.Batch, 0) +func fetchBatches(ctx context.Context, db *sql.DB, whereQuery string, args ...any) ([]*common.BatchHeader, error) { + result := make([]*common.BatchHeader, 0) - rows, err := db.QueryContext(ctx, selectBatch+" "+whereQuery, args...) + rows, err := db.QueryContext(ctx, "select b.header from batch b "+whereQuery, args...) if err != nil { if errors.Is(err, sql.ErrNoRows) { // make sure the error is converted to obscuro-wide not found error @@ -257,8 +222,7 @@ func fetchBatches(ctx context.Context, db *sql.DB, whereQuery string, args ...an } for rows.Next() { var header string - var body []byte - err := rows.Scan(&header, &body) + err := rows.Scan(&header) if err != nil { return nil, err } @@ -266,16 +230,8 @@ func fetchBatches(ctx context.Context, db *sql.DB, whereQuery string, args ...an if err := rlp.DecodeBytes([]byte(header), h); err != nil { return nil, fmt.Errorf("could not decode batch header. Cause: %w", err) } - txs := new([]*common.L2Tx) - if err := rlp.DecodeBytes(body, txs); err != nil { - return nil, fmt.Errorf("could not decode L2 transactions %v. Cause: %w", body, err) - } - result = append(result, - &core.Batch{ - Header: h, - Transactions: *txs, - }) + result = append(result, h) } return result, nil } @@ -329,17 +285,6 @@ func selectReceipts(ctx context.Context, db *sql.DB, config *params.ChainConfig, return allReceipts, nil } -// ReadReceiptsByBatchHash retrieves all the transaction receipts belonging to a block, including -// its corresponding metadata fields. If it is unable to populate these metadata -// fields then nil is returned. -// -// The current implementation populates these metadata fields by reading the receipts' -// corresponding block body, so if the block body is not found it will return nil even -// if the receipt itself is stored. -func ReadReceiptsByBatchHash(ctx context.Context, db *sql.DB, hash common.L2BatchHash, config *params.ChainConfig) (types.Receipts, error) { - return selectReceipts(ctx, db, config, "where batch.hash=? ", hash.Bytes()) -} - func ReadReceipt(ctx context.Context, db *sql.DB, txHash common.L2TxHash, config *params.ChainConfig) (*types.Receipt, error) { // todo - canonical? row := db.QueryRowContext(ctx, queryReceipts+" where tx.hash=? ", txHash.Bytes()) @@ -404,6 +349,38 @@ func ReadTransaction(ctx context.Context, db *sql.DB, txHash gethcommon.Hash) (* return tx, batch, height, idx, nil } +func ReadBatchTransactions(ctx context.Context, db *sql.DB, height uint64) ([]*common.L2Tx, error) { + var txs []*common.L2Tx + + rows, err := db.QueryContext(ctx, "select content from tx where batch_height=? order by idx", height) + if err != nil { + if errors.Is(err, sql.ErrNoRows) { + // make sure the error is converted to obscuro-wide not found error + return nil, errutil.ErrNotFound + } + return nil, err + } + defer rows.Close() + for rows.Next() { + // receipt, tx, batch, height + var txContent []byte + err := rows.Scan(&txContent) + if err != nil { + return nil, err + } + tx := new(common.L2Tx) + if err := rlp.DecodeBytes(txContent, tx); err != nil { + return nil, fmt.Errorf("could not decode L2 transaction. Cause: %w", err) + } + txs = append(txs, tx) + } + if rows.Err() != nil { + return nil, rows.Err() + } + + return txs, nil +} + func GetContractCreationTx(ctx context.Context, db *sql.DB, address gethcommon.Address) (*gethcommon.Hash, error) { row := db.QueryRowContext(ctx, "select tx.hash from exec_tx join tx on tx.id=exec_tx.tx where created_contract_address=? ", address.Bytes()) @@ -433,7 +410,7 @@ func ReadContractCreationCount(ctx context.Context, db *sql.DB) (*big.Int, error return big.NewInt(count + 1), nil } -func ReadUnexecutedBatches(ctx context.Context, db *sql.DB, from *big.Int) ([]*core.Batch, error) { +func ReadUnexecutedBatches(ctx context.Context, db *sql.DB, from *big.Int) ([]*common.BatchHeader, error) { return fetchBatches(ctx, db, "where is_executed=false and is_canonical=true and sequence >= ? order by b.sequence", from.Uint64()) } @@ -483,90 +460,3 @@ func FetchConvertedBatchHash(ctx context.Context, db *sql.DB, seqNo uint64) (get } return gethcommon.BytesToHash(hash), nil } - -func WriteEoa(ctx context.Context, dbTX *sql.Tx, sender *gethcommon.Address) (uint64, error) { - insert := "insert into externally_owned_account (address) values (?)" - res, err := dbTX.ExecContext(ctx, insert, sender.Bytes()) - if err != nil { - return 0, err - } - id, err := res.LastInsertId() - if err != nil { - return 0, err - } - return uint64(id), nil -} - -func ReadEoa(ctx context.Context, dbTx *sql.Tx, addr *gethcommon.Address) (uint64, error) { - row := dbTx.QueryRowContext(ctx, "select id from externally_owned_account where address = ?", addr.Bytes()) - - var id uint64 - err := row.Scan(&id) - if err != nil { - if errors.Is(err, sql.ErrNoRows) { - // make sure the error is converted to obscuro-wide not found error - return 0, errutil.ErrNotFound - } - return 0, err - } - - return id, nil -} - -func WriteContractAddress(ctx context.Context, dbTX *sql.Tx, contractAddress *gethcommon.Address) (uint64, error) { - insert := "insert into contract (address) values (?)" - res, err := dbTX.ExecContext(ctx, insert, contractAddress.Bytes()) - if err != nil { - return 0, err - } - id, err := res.LastInsertId() - if err != nil { - return 0, err - } - return uint64(id), nil -} - -func ReadContractAddress(ctx context.Context, dbTx *sql.Tx, addr gethcommon.Address) (uint64, error) { - row := dbTx.QueryRowContext(ctx, "select id from contract where address = ?", addr.Bytes()) - - var id uint64 - err := row.Scan(&id) - if err != nil { - if errors.Is(err, sql.ErrNoRows) { - // make sure the error is converted to obscuro-wide not found error - return 0, errutil.ErrNotFound - } - return 0, err - } - - return id, nil -} - -//func GetExecTxIds(ctx context.Context, dbTx *sql.Tx, batchSeq uint64, txHashes []*gethcommon.Hash) (map[gethcommon.Hash]uint64, error) { -// query := "select et.id, tx.hash from exec_tx et join tx on et.tx=tx.id where et.batch=? AND tx.hash in (" + repeat("?", ",", len(txHashes)) + ")" -// result := map[gethcommon.Hash]uint64{} -// args := make([]any, 0) -// args = append(args, batchSeq) -// for _, hash := range txHashes { -// args = append(args, hash.Bytes()) -// } -// rows, err := dbTx.QueryContext(ctx, query, args...) -// if err != nil { -// if errors.Is(err, sql.ErrNoRows) { -// // make sure the error is converted to obscuro-wide not found error -// return nil, errutil.ErrNotFound -// } -// return nil, err -// } -// defer rows.Close() -// for rows.Next() { -// var execTxId uint64 -// var txHash gethcommon.Hash -// err := rows.Scan(&execTxId, &txHash) -// if err != nil { -// return nil, err -// } -// result[txHash] = execTxId -// } -// return result, nil -//} diff --git a/go/enclave/storage/enclavedb/events.go b/go/enclave/storage/enclavedb/events.go index 44229143af..17dc749bbb 100644 --- a/go/enclave/storage/enclavedb/events.go +++ b/go/enclave/storage/enclavedb/events.go @@ -271,6 +271,64 @@ func loadLogs(ctx context.Context, db *sql.DB, requestingAccount *gethcommon.Add return result, nil } +func WriteEoa(ctx context.Context, dbTX *sql.Tx, sender *gethcommon.Address) (uint64, error) { + insert := "insert into externally_owned_account (address) values (?)" + res, err := dbTX.ExecContext(ctx, insert, sender.Bytes()) + if err != nil { + return 0, err + } + id, err := res.LastInsertId() + if err != nil { + return 0, err + } + return uint64(id), nil +} + +func ReadEoa(ctx context.Context, dbTx *sql.Tx, addr *gethcommon.Address) (uint64, error) { + row := dbTx.QueryRowContext(ctx, "select id from externally_owned_account where address = ?", addr.Bytes()) + + var id uint64 + err := row.Scan(&id) + if err != nil { + if errors.Is(err, sql.ErrNoRows) { + // make sure the error is converted to obscuro-wide not found error + return 0, errutil.ErrNotFound + } + return 0, err + } + + return id, nil +} + +func WriteContractAddress(ctx context.Context, dbTX *sql.Tx, contractAddress *gethcommon.Address) (uint64, error) { + insert := "insert into contract (address) values (?)" + res, err := dbTX.ExecContext(ctx, insert, contractAddress.Bytes()) + if err != nil { + return 0, err + } + id, err := res.LastInsertId() + if err != nil { + return 0, err + } + return uint64(id), nil +} + +func ReadContractAddress(ctx context.Context, dbTx *sql.Tx, addr gethcommon.Address) (uint64, error) { + row := dbTx.QueryRowContext(ctx, "select id from contract where address = ?", addr.Bytes()) + + var id uint64 + err := row.Scan(&id) + if err != nil { + if errors.Is(err, sql.ErrNoRows) { + // make sure the error is converted to obscuro-wide not found error + return 0, errutil.ErrNotFound + } + return 0, err + } + + return id, nil +} + func stringToHash(ns sql.NullString) gethcommon.Hash { value, err := ns.Value() if err != nil { diff --git a/go/enclave/storage/init/edgelessdb/001_init.sql b/go/enclave/storage/init/edgelessdb/001_init.sql index 6a85180aff..39a37fbe39 100644 --- a/go/enclave/storage/init/edgelessdb/001_init.sql +++ b/go/enclave/storage/init/edgelessdb/001_init.sql @@ -7,7 +7,6 @@ create table if not exists obsdb.keyvalue ky varbinary(64) NOT NULL, val mediumblob NOT NULL, primary key (id), - UNIQUE (ky), INDEX USING HASH (ky) ); GRANT ALL ON obsdb.keyvalue TO obscuro; @@ -39,7 +38,7 @@ create table if not exists obsdb.block height int NOT NULL, primary key (id), INDEX (height), - INDEX USING HASH (hash(8)) + INDEX USING HASH (hash) ); GRANT ALL ON obsdb.block TO obscuro; @@ -64,19 +63,11 @@ create table if not exists obsdb.rollup header blob NOT NULL, compression_block INTEGER NOT NULL, INDEX (compression_block), - INDEX USING HASH (hash(8)), + INDEX USING HASH (hash), primary key (id) ); GRANT ALL ON obsdb.rollup TO obscuro; -create table if not exists obsdb.batch_body -( - id INTEGER, - content mediumblob NOT NULL, - primary key (id) -); -GRANT ALL ON obsdb.batch_body TO obscuro; - create table if not exists obsdb.batch ( sequence INTEGER, @@ -85,14 +76,12 @@ create table if not exists obsdb.batch height int NOT NULL, is_canonical boolean NOT NULL, header blob NOT NULL, - body int NOT NULL, l1_proof_hash binary(32) NOT NULL, l1_proof INTEGER, is_executed boolean NOT NULL, primary key (sequence), - INDEX USING HASH (hash(8)), - INDEX USING HASH (l1_proof_hash(8)), - INDEX (body), + INDEX USING HASH (hash), + INDEX USING HASH (l1_proof_hash), INDEX (l1_proof), INDEX (height) ); @@ -104,11 +93,11 @@ create table if not exists obsdb.tx hash binary(32) NOT NULL, content mediumblob NOT NULL, sender_address binary(20) NOT NULL, - nonce int NOT NULL, idx int NOT NULL, - body int NOT NULL, - INDEX USING HASH (hash(8)), + batch_height int NOT NULL, + INDEX USING HASH (hash), INDEX USING HASH (sender_address), + INDEX (batch_height, idx), primary key (id) ); GRANT ALL ON obsdb.tx TO obscuro; @@ -131,7 +120,7 @@ create table if not exists obsdb.contract id INTEGER AUTO_INCREMENT, address binary(20) NOT NULL, primary key (id), - INDEX USING HASH (address(8)) + INDEX USING HASH (address) ); GRANT ALL ON obsdb.contract TO obscuro; @@ -140,7 +129,7 @@ create table if not exists obsdb.externally_owned_account id INTEGER AUTO_INCREMENT, address binary(20) NOT NULL, primary key (id), - INDEX USING HASH (address(8)) + INDEX USING HASH (address) ); GRANT ALL ON obsdb.externally_owned_account TO obscuro; @@ -151,7 +140,7 @@ create table if not exists obsdb.event_type event_sig binary(32) NOT NULL, lifecycle_event boolean NOT NULL, primary key (id), - INDEX USING HASH (contract, event_sig(8)) + INDEX USING HASH (contract, event_sig) ); GRANT ALL ON obsdb.event_type TO obscuro; diff --git a/go/enclave/storage/init/sqlite/001_init.sql b/go/enclave/storage/init/sqlite/001_init.sql index ac8dcecfdc..d924e88ff6 100644 --- a/go/enclave/storage/init/sqlite/001_init.sql +++ b/go/enclave/storage/init/sqlite/001_init.sql @@ -57,12 +57,6 @@ create table if not exists rollup create index ROLLUP_COMPRESSION_BLOCK_IDX on rollup (compression_block); create index ROLLUP_COMPRESSION_HASH_IDX on rollup (hash); -create table if not exists batch_body -( - id int NOT NULL primary key, - content mediumblob NOT NULL -); - create table if not exists batch ( sequence int primary key, @@ -71,7 +65,6 @@ create table if not exists batch height int NOT NULL, is_canonical boolean NOT NULL, header blob NOT NULL, - body int NOT NULL REFERENCES batch_body, l1_proof_hash binary(32) NOT NULL, l1_proof INTEGER, -- normally this would be a FK, but there is a weird edge case where an L2 node might not have the block used to create this batch is_executed boolean NOT NULL @@ -80,7 +73,6 @@ create table if not exists batch ); create index IDX_BATCH_HASH on batch (hash); create index IDX_BATCH_BLOCK on batch (l1_proof_hash); -create index IDX_BATCH_BODY on batch (body); create index IDX_BATCH_L1 on batch (l1_proof); create index IDX_BATCH_HEIGHT on batch (height); @@ -90,12 +82,12 @@ create table if not exists tx hash binary(32) NOT NULL, content mediumblob NOT NULL, sender_address binary(20) NOT NULL, - nonce int NOT NULL, idx int NOT NULL, - body int NOT NULL REFERENCES batch_body + batch_height int NOT NULL ); create index IDX_TX_HASH on tx (hash); create index IDX_TX_SENDER_ADDRESS on tx (sender_address); +create index IDX_TX_BATCH_HEIGHT on tx (batch_height, idx); create table if not exists exec_tx ( diff --git a/go/enclave/storage/interfaces.go b/go/enclave/storage/interfaces.go index 88d2340be4..26fab330fa 100644 --- a/go/enclave/storage/interfaces.go +++ b/go/enclave/storage/interfaces.go @@ -42,24 +42,26 @@ type BatchResolver interface { FetchBatch(ctx context.Context, hash common.L2BatchHash) (*core.Batch, error) // FetchBatchHeader returns the batch header with the given hash. FetchBatchHeader(ctx context.Context, hash common.L2BatchHash) (*common.BatchHeader, error) + FetchBatchTransactionsBySeq(ctx context.Context, seqNo uint64) ([]*common.L2Tx, error) // FetchBatchByHeight returns the batch on the canonical chain with the given height. FetchBatchByHeight(ctx context.Context, height uint64) (*core.Batch, error) // FetchBatchBySeqNo returns the batch with the given seq number. FetchBatchBySeqNo(ctx context.Context, seqNum uint64) (*core.Batch, error) - // FetchHeadBatch returns the current head batch of the canonical chain. - FetchHeadBatch(ctx context.Context) (*core.Batch, error) + // FetchBatchHeaderBySeqNo returns the batch header with the given seq number. + FetchBatchHeaderBySeqNo(ctx context.Context, seqNum uint64) (*common.BatchHeader, error) + FetchHeadBatchHeader(ctx context.Context) (*common.BatchHeader, error) // FetchCurrentSequencerNo returns the sequencer number FetchCurrentSequencerNo(ctx context.Context) (*big.Int, error) // FetchBatchesByBlock returns all batches with the block hash as the L1 proof - FetchBatchesByBlock(ctx context.Context, hash common.L1BlockHash) ([]*core.Batch, error) + FetchBatchesByBlock(ctx context.Context, hash common.L1BlockHash) ([]*common.BatchHeader, error) // FetchNonCanonicalBatchesBetween - returns all reorged batches between the sequences - FetchNonCanonicalBatchesBetween(ctx context.Context, startSeq uint64, endSeq uint64) ([]*core.Batch, error) + FetchNonCanonicalBatchesBetween(ctx context.Context, startSeq uint64, endSeq uint64) ([]*common.BatchHeader, error) // FetchCanonicalBatchesBetween - returns all canon batches between the sequences - FetchCanonicalBatchesBetween(ctx context.Context, startSeq uint64, endSeq uint64) ([]*core.Batch, error) + FetchCanonicalBatchesBetween(ctx context.Context, startSeq uint64, endSeq uint64) ([]*common.BatchHeader, error) // IsBatchCanonical - true if the batch is canonical IsBatchCanonical(ctx context.Context, seq uint64) (bool, error) // FetchCanonicalUnexecutedBatches - return the list of the unexecuted batches that are canonical - FetchCanonicalUnexecutedBatches(context.Context, *big.Int) ([]*core.Batch, error) + FetchCanonicalUnexecutedBatches(context.Context, *big.Int) ([]*common.BatchHeader, error) FetchConvertedHash(ctx context.Context, hash common.L2BatchHash) (gethcommon.Hash, error) @@ -69,7 +71,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 *core.Batch, receipts []*types.Receipt) error + StoreExecutedBatch(ctx context.Context, batch *common.BatchHeader, receipts []*types.Receipt) error // StoreRollup StoreRollup(ctx context.Context, rollup *common.ExtRollup, header *common.CalldataRollupHeader) error @@ -96,8 +98,6 @@ type TransactionStorage interface { GetTransaction(ctx context.Context, txHash common.L2TxHash) (*types.Transaction, common.L2BatchHash, uint64, uint64, error) // GetTransactionReceipt - returns the receipt of a tx by tx hash GetTransactionReceipt(ctx context.Context, txHash common.L2TxHash) (*types.Receipt, error) - // GetReceiptsByBatchHash retrieves the receipts for all transactions in a given rollup. - GetReceiptsByBatchHash(ctx context.Context, hash common.L2BatchHash) (types.Receipts, error) // GetContractCreationTx returns the hash of the tx that created a contract GetContractCreationTx(ctx context.Context, address gethcommon.Address) (*gethcommon.Hash, error) } diff --git a/go/enclave/storage/storage.go b/go/enclave/storage/storage.go index 7e67e63632..dbc209232a 100644 --- a/go/enclave/storage/storage.go +++ b/go/enclave/storage/storage.go @@ -63,7 +63,7 @@ type storageImpl struct { blockCache *cache.Cache[*types.Block] // stores batches using the sequence number as key - batchCacheBySeqNo *cache.Cache[*core.Batch] + batchCacheBySeqNo *cache.Cache[*common.BatchHeader] // mapping between the hash and the sequence number // note: to fetch a batch by hash will require 2 cache hits @@ -136,7 +136,7 @@ func NewStorage(backingDB enclavedb.EnclaveDB, chainConfig *params.ChainConfig, stateCache: stateDB, chainConfig: chainConfig, blockCache: cache.New[*types.Block](ristrettoStore), - batchCacheBySeqNo: cache.New[*core.Batch](ristrettoStore), + batchCacheBySeqNo: cache.New[*common.BatchHeader](ristrettoStore), seqCacheByHash: cache.New[*big.Int](ristrettoStore), seqCacheByHeight: cache.New[*big.Int](ristrettoStore), convertedHashCache: cache.New[*gethcommon.Hash](ristrettoStore), @@ -159,9 +159,13 @@ func (s *storageImpl) Close() error { return s.db.GetSQLDB().Close() } -func (s *storageImpl) FetchHeadBatch(ctx context.Context) (*core.Batch, error) { - defer s.logDuration("FetchHeadBatch", measure.NewStopwatch()) - return enclavedb.ReadCurrentHeadBatch(ctx, s.db.GetSQLDB()) +func (s *storageImpl) FetchHeadBatchHeader(ctx context.Context) (*common.BatchHeader, error) { + defer s.logDuration("FetchHeadBatchHeader", measure.NewStopwatch()) + b, err := enclavedb.ReadCurrentHeadBatchHeader(ctx, s.db.GetSQLDB()) + if err != nil { + return nil, err + } + return b, nil } func (s *storageImpl) FetchCurrentSequencerNo(ctx context.Context) (*big.Int, error) { @@ -171,32 +175,37 @@ func (s *storageImpl) FetchCurrentSequencerNo(ctx context.Context) (*big.Int, er func (s *storageImpl) FetchBatch(ctx context.Context, hash common.L2BatchHash) (*core.Batch, error) { defer s.logDuration("FetchBatch", measure.NewStopwatch()) + seqNo, err := s.fetchSeqNoByHash(ctx, hash) + if err != nil { + return nil, err + } + return s.FetchBatchBySeqNo(ctx, seqNo.Uint64()) +} + +func (s *storageImpl) fetchSeqNoByHash(ctx context.Context, hash common.L2BatchHash) (*big.Int, error) { seqNo, err := common.GetCachedValue(ctx, s.seqCacheByHash, s.logger, hash, func(v any) (*big.Int, error) { - batch, err := enclavedb.ReadBatchByHash(ctx, s.db.GetSQLDB(), v.(common.L2BatchHash)) + batch, err := enclavedb.ReadBatchHeaderByHash(ctx, s.db.GetSQLDB(), v.(common.L2BatchHash)) if err != nil { return nil, err } - return batch.SeqNo(), nil + return batch.SequencerOrderNo, nil }) - if err != nil { - return nil, err - } - return s.FetchBatchBySeqNo(ctx, seqNo.Uint64()) + return seqNo, err } func (s *storageImpl) FetchConvertedHash(ctx context.Context, hash common.L2BatchHash) (gethcommon.Hash, error) { defer s.logDuration("FetchConvertedHash", measure.NewStopwatch()) - batch, err := s.FetchBatch(ctx, hash) + batch, err := s.FetchBatchHeader(ctx, hash) if err != nil { return gethcommon.Hash{}, err } convertedHash, err := common.GetCachedValue(ctx, s.convertedHashCache, s.logger, hash, func(v any) (*gethcommon.Hash, error) { - convertedHash, err := enclavedb.FetchConvertedBatchHash(ctx, s.db.GetSQLDB(), batch.Header.SequencerOrderNo.Uint64()) + ch, err := enclavedb.FetchConvertedBatchHash(ctx, s.db.GetSQLDB(), batch.SequencerOrderNo.Uint64()) if err != nil { return nil, err } - return &convertedHash, nil + return &ch, nil }) if err != nil { return gethcommon.Hash{}, err @@ -206,22 +215,32 @@ func (s *storageImpl) FetchConvertedHash(ctx context.Context, hash common.L2Batc func (s *storageImpl) FetchBatchHeader(ctx context.Context, hash common.L2BatchHash) (*common.BatchHeader, error) { defer s.logDuration("FetchBatchHeader", measure.NewStopwatch()) - b, err := s.FetchBatch(ctx, hash) + seqNo, err := s.fetchSeqNoByHash(ctx, hash) if err != nil { return nil, err } - return b.Header, nil + + return s.FetchBatchHeaderBySeqNo(ctx, seqNo.Uint64()) +} + +func (s *storageImpl) FetchBatchTransactionsBySeq(ctx context.Context, seqNo uint64) ([]*common.L2Tx, error) { + defer s.logDuration("FetchBatchTransactionsBySeq", measure.NewStopwatch()) + batch, err := s.FetchBatchHeaderBySeqNo(ctx, seqNo) + if err != nil { + return nil, err + } + return enclavedb.ReadBatchTransactions(ctx, s.db.GetSQLDB(), batch.Number.Uint64()) } func (s *storageImpl) FetchBatchByHeight(ctx context.Context, height uint64) (*core.Batch, error) { defer s.logDuration("FetchBatchByHeight", measure.NewStopwatch()) // the key is (height+1), because for some reason it doesn't like a key of 0 seqNo, err := common.GetCachedValue(ctx, s.seqCacheByHeight, s.logger, height+1, func(h any) (*big.Int, error) { - batch, err := enclavedb.ReadCanonicalBatchByHeight(ctx, s.db.GetSQLDB(), height) + batch, err := enclavedb.ReadCanonicalBatchHeaderByHeight(ctx, s.db.GetSQLDB(), height) if err != nil { return nil, err } - return batch.SeqNo(), nil + return batch.SequencerOrderNo, nil }) if err != nil { return nil, err @@ -229,12 +248,12 @@ func (s *storageImpl) FetchBatchByHeight(ctx context.Context, height uint64) (*c return s.FetchBatchBySeqNo(ctx, seqNo.Uint64()) } -func (s *storageImpl) FetchNonCanonicalBatchesBetween(ctx context.Context, startSeq uint64, endSeq uint64) ([]*core.Batch, error) { +func (s *storageImpl) FetchNonCanonicalBatchesBetween(ctx context.Context, startSeq uint64, endSeq uint64) ([]*common.BatchHeader, error) { defer s.logDuration("FetchNonCanonicalBatchesBetween", measure.NewStopwatch()) return enclavedb.ReadNonCanonicalBatches(ctx, s.db.GetSQLDB(), startSeq, endSeq) } -func (s *storageImpl) FetchCanonicalBatchesBetween(ctx context.Context, startSeq uint64, endSeq uint64) ([]*core.Batch, error) { +func (s *storageImpl) FetchCanonicalBatchesBetween(ctx context.Context, startSeq uint64, endSeq uint64) ([]*common.BatchHeader, error) { defer s.logDuration("FetchCanonicalBatchesBetween", measure.NewStopwatch()) return enclavedb.ReadCanonicalBatches(ctx, s.db.GetSQLDB(), startSeq, endSeq) } @@ -417,14 +436,14 @@ func (s *storageImpl) HealthCheck(ctx context.Context) (bool, error) { func (s *storageImpl) CreateStateDB(ctx context.Context, batchHash common.L2BatchHash) (*state.StateDB, error) { defer s.logDuration("CreateStateDB", measure.NewStopwatch()) - batch, err := s.FetchBatch(ctx, batchHash) + batch, err := s.FetchBatchHeader(ctx, batchHash) if err != nil { return nil, err } - statedb, err := state.New(batch.Header.Root, s.stateCache, nil) + statedb, err := state.New(batch.Root, s.stateCache, nil) if err != nil { - return nil, fmt.Errorf("could not create state DB for %s. Cause: %w", batch.Header.Root, err) + return nil, fmt.Errorf("could not create state DB for %s. Cause: %w", batch.Root, err) } return statedb, nil } @@ -438,12 +457,6 @@ func (s *storageImpl) EmptyStateDB() (*state.StateDB, error) { return statedb, nil } -// GetReceiptsByBatchHash retrieves the receipts for all transactions in a given batch. -func (s *storageImpl) GetReceiptsByBatchHash(ctx context.Context, hash gethcommon.Hash) (types.Receipts, error) { - defer s.logDuration("GetReceiptsByBatchHash", measure.NewStopwatch()) - return enclavedb.ReadReceiptsByBatchHash(ctx, s.db.GetSQLDB(), hash, s.chainConfig) -} - func (s *storageImpl) GetTransaction(ctx context.Context, txHash gethcommon.Hash) (*types.Transaction, common.L2BatchHash, uint64, uint64, error) { defer s.logDuration("GetTransaction", measure.NewStopwatch()) return enclavedb.ReadTransaction(ctx, s.db.GetSQLDB(), txHash) @@ -494,16 +507,30 @@ func (s *storageImpl) StoreAttestedKey(ctx context.Context, aggregator gethcommo func (s *storageImpl) FetchBatchBySeqNo(ctx context.Context, seqNum uint64) (*core.Batch, error) { defer s.logDuration("FetchBatchBySeqNo", measure.NewStopwatch()) - b, err := common.GetCachedValue(ctx, s.batchCacheBySeqNo, s.logger, seqNum, func(seq any) (*core.Batch, error) { - return enclavedb.ReadBatchBySeqNo(ctx, s.db.GetSQLDB(), seqNum) + h, err := common.GetCachedValue(ctx, s.batchCacheBySeqNo, s.logger, seqNum, func(seq any) (*common.BatchHeader, error) { + return enclavedb.ReadBatchHeaderBySeqNo(ctx, s.db.GetSQLDB(), seqNum) }) - if err == nil && b == nil { - return nil, fmt.Errorf("not found") + if err != nil { + return nil, err + } + txs, err := s.FetchBatchTransactionsBySeq(ctx, seqNum) + if err != nil { + return nil, err } - return b, err + return &core.Batch{ + Header: h, + Transactions: txs, + }, err +} + +func (s *storageImpl) FetchBatchHeaderBySeqNo(ctx context.Context, seqNum uint64) (*common.BatchHeader, error) { + defer s.logDuration("FetchBatchHeaderBySeqNo", measure.NewStopwatch()) + return common.GetCachedValue(ctx, s.batchCacheBySeqNo, s.logger, seqNum, func(seq any) (*common.BatchHeader, error) { + return enclavedb.ReadBatchHeaderBySeqNo(ctx, s.db.GetSQLDB(), seqNum) + }) } -func (s *storageImpl) FetchBatchesByBlock(ctx context.Context, block common.L1BlockHash) ([]*core.Batch, error) { +func (s *storageImpl) FetchBatchesByBlock(ctx context.Context, block common.L1BlockHash) ([]*common.BatchHeader, error) { defer s.logDuration("FetchBatchesByBlock", measure.NewStopwatch()) return enclavedb.ReadBatchesByBlock(ctx, s.db.GetSQLDB(), block) } @@ -536,8 +563,41 @@ func (s *storageImpl) StoreBatch(ctx context.Context, batch *core.Batch, convert } s.logger.Trace("write batch", log.BatchHashKey, batch.Hash(), "l1Proof", batch.Header.L1Proof, log.BatchSeqNoKey, batch.SeqNo(), "block_id", blockId) - if err := enclavedb.WriteBatchAndTransactions(ctx, dbTx, batch, convertedHash, blockId); err != nil { - return fmt.Errorf("could not write batch. Cause: %w", err) + // the batch is canonical only if the l1 proof is canonical + isL1ProofCanonical, err := enclavedb.IsCanonicalBlock(ctx, dbTx, &batch.Header.L1Proof) + if err != nil { + return err + } + // sanity check because a batch can't be canonical if its parent is not + parentIsCanon, err := enclavedb.IsCanonicalBatchHash(ctx, dbTx, &batch.Header.ParentHash) + if err != nil { + return err + } + parentIsCanon = parentIsCanon || batch.SeqNo().Uint64() <= common.L2GenesisSeqNo+2 + + // sanity check that the parent is canonical + if isL1ProofCanonical && !parentIsCanon { + s.logger.Crit("invalid chaining. Batch is canonical. Parent is not", log.BatchHashKey, batch.Hash(), "parentHash", batch.Header.ParentHash) + } + + existsHeight, err := enclavedb.ExistsBatchAtHeight(ctx, dbTx, batch.Header.Number) + if err != nil { + return fmt.Errorf("could not read ExistsBatchAtHeight. Cause: %w", err) + } + + if err := enclavedb.WriteBatchHeader(ctx, dbTx, batch, convertedHash, blockId, isL1ProofCanonical); err != nil { + return fmt.Errorf("could not write batch header. Cause: %w", err) + } + + // only insert transactions if this is the first time a batch of this height is created + if !existsHeight { + s.logger.Debug("Write txs ***", log.BatchHeightKey, batch.Header.Number, "nr", len(batch.Transactions)) + for i, transaction := range batch.Transactions { + s.logger.Debug(fmt.Sprintf("Write tx %d. Hash %s ", i, transaction.Hash())) + } + if err := enclavedb.WriteTransactions(ctx, dbTx, batch); err != nil { + return fmt.Errorf("could not write transactions. Cause: %w", err) + } } // insert the tx signers as externally owned accounts @@ -549,17 +609,18 @@ func (s *storageImpl) StoreBatch(ctx context.Context, batch *core.Batch, convert _, err = s.findEOA(ctx, dbTx, sender) if err != nil { if errors.Is(err, errutil.ErrNotFound) { - id, err := enclavedb.WriteEoa(ctx, dbTx, sender) + _, err := enclavedb.WriteEoa(ctx, dbTx, sender) if err != nil { return fmt.Errorf("could not write the eoa. Cause: %w", err) } - etId, _, err := s.findEventTopic(ctx, dbTx, sender.Bytes()) - if err == nil { - err = enclavedb.UpdateEventTopic(ctx, dbTx, etId, id) - if err != nil { - return fmt.Errorf("could not update the event topic. Cause: %w", err) - } - } + //tood + //etId, _, err := s.findEventTopic(ctx, dbTx, sender.Bytes()) + //if err == nil { + // err = enclavedb.UpdateEventTopic(ctx, dbTx, etId, id) + // if err != nil { + // return fmt.Errorf("could not update the event topic. Cause: %w", err) + // } + //} } } } @@ -568,7 +629,7 @@ func (s *storageImpl) StoreBatch(ctx context.Context, batch *core.Batch, convert return fmt.Errorf("could not commit batch %w", err) } - common.CacheValue(ctx, s.batchCacheBySeqNo, s.logger, batch.SeqNo().Uint64(), batch) + common.CacheValue(ctx, s.batchCacheBySeqNo, s.logger, batch.SeqNo().Uint64(), batch.Header) common.CacheValue(ctx, s.seqCacheByHash, s.logger, batch.Hash(), batch.SeqNo()) // note: the key is (height+1), because for some reason it doesn't like a key of 0 // should always contain the canonical batch because the cache is overwritten by each new batch after a reorg @@ -576,7 +637,7 @@ func (s *storageImpl) StoreBatch(ctx context.Context, batch *core.Batch, convert return nil } -func (s *storageImpl) StoreExecutedBatch(ctx context.Context, batch *core.Batch, receipts []*types.Receipt) error { +func (s *storageImpl) StoreExecutedBatch(ctx context.Context, batch *common.BatchHeader, receipts []*types.Receipt) error { defer s.logDuration("StoreExecutedBatch", measure.NewStopwatch()) executed, err := enclavedb.BatchWasExecuted(ctx, s.db.GetSQLDB(), batch.Hash()) if err != nil { @@ -587,7 +648,7 @@ func (s *storageImpl) StoreExecutedBatch(ctx context.Context, batch *core.Batch, return nil } - s.logger.Trace("storing executed batch", log.BatchHashKey, batch.Hash(), log.BatchSeqNoKey, batch.SeqNo(), "receipts", len(receipts)) + s.logger.Trace("storing executed batch", log.BatchHashKey, batch.Hash(), log.BatchSeqNoKey, batch.SequencerOrderNo, "receipts", len(receipts)) dbTx, err := s.db.NewDBTransaction(ctx) if err != nil { @@ -595,7 +656,7 @@ func (s *storageImpl) StoreExecutedBatch(ctx context.Context, batch *core.Batch, } defer dbTx.Rollback() - if err := enclavedb.MarkBatchExecuted(ctx, dbTx, batch.SeqNo()); err != nil { + if err := enclavedb.MarkBatchExecuted(ctx, dbTx, batch.SequencerOrderNo); err != nil { return fmt.Errorf("could not set the executed flag. Cause: %w", err) } @@ -613,7 +674,7 @@ func (s *storageImpl) StoreExecutedBatch(ctx context.Context, batch *core.Batch, } // todo - move this to a separate service -func (s *storageImpl) storeReceiptAndEventLogs(ctx context.Context, dbTX *sql.Tx, batch *core.Batch, receipt *types.Receipt) error { +func (s *storageImpl) storeReceiptAndEventLogs(ctx context.Context, dbTX *sql.Tx, batch *common.BatchHeader, receipt *types.Receipt) error { // store the contract.address var createdContract *uint64 var nilAddr gethcommon.Address @@ -637,9 +698,12 @@ func (s *storageImpl) storeReceiptAndEventLogs(ctx context.Context, dbTX *sql.Tx return fmt.Errorf("failed to encode block receipts. Cause: %w", err) } - txId, _ := enclavedb.GetTxId(ctx, dbTX, receipt.TxHash) + txId, err := enclavedb.GetTxId(ctx, dbTX, receipt.TxHash) + if err != nil { + return fmt.Errorf("could not get transaction id. Cause: %w", err) + } - execTxId, err := enclavedb.WriteExecutedTransaction(ctx, dbTX, batch.SeqNo().Uint64(), txId, createdContract, receiptBytes) + execTxId, err := enclavedb.WriteExecutedTransaction(ctx, dbTX, batch.SequencerOrderNo.Uint64(), txId, createdContract, receiptBytes) if err != nil { return fmt.Errorf("could not write receipt. Cause: %w", err) } @@ -908,7 +972,7 @@ func (s *storageImpl) GetContractCount(ctx context.Context) (*big.Int, error) { return enclavedb.ReadContractCreationCount(ctx, s.db.GetSQLDB()) } -func (s *storageImpl) FetchCanonicalUnexecutedBatches(ctx context.Context, from *big.Int) ([]*core.Batch, error) { +func (s *storageImpl) FetchCanonicalUnexecutedBatches(ctx context.Context, from *big.Int) ([]*common.BatchHeader, error) { defer s.logDuration("FetchCanonicalUnexecutedBatches", measure.NewStopwatch()) return enclavedb.ReadUnexecutedBatches(ctx, s.db.GetSQLDB(), from) } diff --git a/integration/common/testlog/testlog.go b/integration/common/testlog/testlog.go index 67493fa90d..8e3c0ec57b 100644 --- a/integration/common/testlog/testlog.go +++ b/integration/common/testlog/testlog.go @@ -8,8 +8,6 @@ import ( "github.com/ten-protocol/go-ten/lib/gethfork/debug" - "github.com/ten-protocol/go-ten/go/common/log" - gethlog "github.com/ethereum/go-ethereum/log" ) @@ -53,6 +51,6 @@ func Setup(cfg *Cfg) *os.File { panic(err) } - testlog = gethlog.New(log.CmpKey, log.TestLogCmp) + testlog = gethlog.New() return f }