Skip to content

Commit

Permalink
Remove pending and empty pending from DB
Browse files Browse the repository at this point in the history
Pending Block is now only managed in memory this is to make sure that
pending block in the DB and in memory do not become out of sync. Before
the pending block was managed in memory as a cache, however, since there
is only one pending block at a given time it doesn't make sense to keep
track of pending block in both memory and DB.

To reduce the number of block not found errors while simulating
transactions it was decided to store empty pending block, using the
latest header to fill in fields such as block number, parent block hash,
etc. This meant that any time we didn't have a pending block this
empty pending block would be served along with empty state diff and
classes. Every time a new block was added to the blockchain a new empty
pending block was also added to the DB.

The unforeseen side effect of this change was when the
--poll-pending-interval flag was disabled the rpc would still serve a
pending block. This is incorrect behaviour.

As the blocks changed per new versions of starknet the empty block also
needed to be changed and a storage diff with a special contract "0x1"
needed to be updated in the state diff. This overhead is unnecessary and
incorrectly informs the user that there is a pending block when
there isn't one.
  • Loading branch information
IronGauntlets committed Dec 12, 2024
1 parent 7687360 commit 7634b9f
Show file tree
Hide file tree
Showing 10 changed files with 104 additions and 248 deletions.
157 changes: 39 additions & 118 deletions blockchain/blockchain.go
Original file line number Diff line number Diff line change
Expand Up @@ -6,7 +6,6 @@ import (
"errors"
"fmt"
"sync/atomic"
"time"

"github.com/Masterminds/semver/v3"
"github.com/NethermindEth/juno/core"
Expand Down Expand Up @@ -53,6 +52,7 @@ type Reader interface {

var (
ErrParentDoesNotMatchHead = errors.New("block's parent hash does not match head block hash")
ErrPendingBlockNotFound = errors.New("pending block not found")
SupportedStarknetVersion = semver.MustParse("0.13.3")
)

Expand Down Expand Up @@ -81,14 +81,14 @@ func copyWithoutPatch(v *semver.Version) *semver.Version {

var _ Reader = (*Blockchain)(nil)

// Todo: Remove after cachedPending is moved to sychcroniser
var cachedPending atomic.Pointer[Pending]

// Blockchain is responsible for keeping track of all things related to the Starknet blockchain
type Blockchain struct {
network *utils.Network
database db.DB

listener EventListener

cachedPending atomic.Pointer[Pending]
}

func New(database db.DB, network *utils.Network) *Blockchain {
Expand Down Expand Up @@ -270,7 +270,7 @@ func (b *Blockchain) TransactionByHash(hash *felt.Felt) (core.Transaction, error
// not found in the canonical blocks, try pending
if errors.Is(err, db.ErrKeyNotFound) {
var pending Pending
pending, err = b.pendingBlock(txn)
pending, err = pendingBlock(txn)
if err != nil {
return err
}
Expand Down Expand Up @@ -303,9 +303,12 @@ func (b *Blockchain) Receipt(hash *felt.Felt) (*core.TransactionReceipt, *felt.F
// not found in the canonical blocks, try pending
if errors.Is(err, db.ErrKeyNotFound) {
var pending Pending
pending, err = b.pendingBlock(txn)
pending, err = pendingBlock(txn)
if err != nil {
return err
if !errors.Is(err, ErrPendingBlockNotFound) {
return err
}
return db.ErrKeyNotFound
}

for i, t := range pending.Block.Transactions {
Expand Down Expand Up @@ -389,10 +392,6 @@ func (b *Blockchain) Store(block *core.Block, blockCommitments *core.BlockCommit
return err
}

if err := b.storeEmptyPending(txn, block.Header); err != nil {
return err
}

// Head of the blockchain is maintained as follows:
// [db.ChainHeight]() -> (BlockNumber)
heightBin := core.MarshalBlockNumber(block.Number)
Expand Down Expand Up @@ -866,7 +865,7 @@ func (b *Blockchain) EventFilter(from *felt.Felt, keys [][]felt.Felt) (EventFilt
return nil, err
}

return newEventFilter(txn, from, keys, 0, latest), nil
return newEventFilter(txn, from, keys, 0, latest, &cachedPending), nil
}

// RevertHead reverts the head block
Expand Down Expand Up @@ -936,23 +935,11 @@ func (b *Blockchain) revertHead(txn db.Transaction) error {
return err
}

// Revert chain height and pending.
// Revert chain height.
if genesisBlock {
if err = txn.Delete(db.Pending.Key()); err != nil {
return err
}
return txn.Delete(db.ChainHeight.Key())
}

var newHeader *core.Header
newHeader, err = blockHeaderByNumber(txn, blockNumber-1)
if err != nil {
return err
}
if err := b.storeEmptyPending(txn, newHeader); err != nil {
return err
}

heightBin := core.MarshalBlockNumber(blockNumber - 1)
return txn.Set(db.ChainHeight.Key(), heightBin)
}
Expand Down Expand Up @@ -989,42 +976,9 @@ func removeTxsAndReceipts(txn db.Transaction, blockNumber, numTxs uint64) error
return nil
}

func (b *Blockchain) storeEmptyPending(txn db.Transaction, latestHeader *core.Header) error {
receipts := make([]*core.TransactionReceipt, 0)
pendingBlock := &core.Block{
Header: &core.Header{
ParentHash: latestHeader.Hash,
SequencerAddress: latestHeader.SequencerAddress,
Number: latestHeader.Number + 1,
Timestamp: uint64(time.Now().Unix()),
ProtocolVersion: latestHeader.ProtocolVersion,
EventsBloom: core.EventsBloom(receipts),
GasPrice: latestHeader.GasPrice,
GasPriceSTRK: latestHeader.GasPriceSTRK,
},
Transactions: make([]core.Transaction, 0),
Receipts: receipts,
}

stateDiff, err := MakeStateDiffForEmptyBlock(b, latestHeader.Number+1)
if err != nil {
return err
}

emptyPending := &Pending{
Block: pendingBlock,
StateUpdate: &core.StateUpdate{
OldRoot: latestHeader.GlobalStateRoot,
StateDiff: stateDiff,
},
NewClasses: make(map[felt.Felt]core.Class, 0),
}
return b.storePending(txn, emptyPending)
}

// StorePending stores a pending block given that it is for the next height
func (b *Blockchain) StorePending(pending *Pending) error {
return b.database.Update(func(txn db.Transaction) error {
return b.database.View(func(txn db.Transaction) error {
expectedParentHash := new(felt.Felt)
h, err := headsHeader(txn)
if err != nil && !errors.Is(err, db.ErrKeyNotFound) {
Expand All @@ -1037,57 +991,44 @@ func (b *Blockchain) StorePending(pending *Pending) error {
return ErrParentDoesNotMatchHead
}

if existingPending, err := b.pendingBlock(txn); err == nil {
if existingPending, err := pendingBlock(txn); err == nil {
if existingPending.Block.TransactionCount >= pending.Block.TransactionCount {
return nil // ignore the incoming pending if it has fewer transactions than the one we already have
// ignore the incoming pending if it has fewer transactions than the one we already have
return nil
}
pending.Block.Number = existingPending.Block.Number // Just in case the number is not set.
} else if !errors.Is(err, db.ErrKeyNotFound) { // Allow StorePending before block zero.
} else if !errors.Is(err, ErrPendingBlockNotFound) {
return err
}

return b.storePending(txn, pending)
if h != nil {
pending.Block.Number = h.Number + 1
}
cachedPending.Store(pending)

return nil
})
}

func (b *Blockchain) storePending(txn db.Transaction, pending *Pending) error {
if err := storePending(txn, pending); err != nil {
return err
// Todo:
// 2. Consider returning reference to Pending instead of Pending
func pendingBlock(txn db.Transaction) (Pending, error) {
pending := cachedPending.Load()
if pending == nil {
return Pending{}, ErrPendingBlockNotFound
}
b.cachedPending.Store(pending)
return nil
}

func storePending(txn db.Transaction, pending *Pending) error {
pendingBytes, err := encoder.Marshal(pending)
if err != nil {
return err
expectedParentHash := &felt.Zero
if head, err := headsHeader(txn); err == nil {
expectedParentHash = head.Hash
}
return txn.Set(db.Pending.Key(), pendingBytes)
}

func (b *Blockchain) pendingBlock(txn db.Transaction) (Pending, error) {
if cachedPending := b.cachedPending.Load(); cachedPending != nil {
expectedParentHash := &felt.Zero
if head, err := headsHeader(txn); err == nil {
expectedParentHash = head.Hash
}
if cachedPending.Block.ParentHash.Equal(expectedParentHash) {
return *cachedPending, nil
}
if pending.Block.ParentHash.Equal(expectedParentHash) {
return *pending, nil
}

// Either cachedPending was nil or wasn't consistent with the HEAD we have
// in the database, so read it directly from the database
return pendingBlock(txn)
}
// Since the pending block in the cache is outdated remove it
cachedPending.Store(nil)

func pendingBlock(txn db.Transaction) (Pending, error) {
var pending Pending
err := txn.Get(db.Pending.Key(), func(bytes []byte) error {
return encoder.Unmarshal(bytes, &pending)
})
return pending, err
return Pending{}, ErrPendingBlockNotFound
}

// Pending returns the pending block from the database
Expand All @@ -1096,7 +1037,7 @@ func (b *Blockchain) Pending() (Pending, error) {
var pending Pending
return pending, b.database.View(func(txn db.Transaction) error {
var err error
pending, err = b.pendingBlock(txn)
pending, err = pendingBlock(txn)
return err
})
}
Expand All @@ -1109,7 +1050,7 @@ func (b *Blockchain) PendingState() (core.StateReader, StateCloser, error) {
return nil, nil, err
}

pending, err := b.pendingBlock(txn)
pending, err := pendingBlock(txn)
if err != nil {
return nil, nil, utils.RunAndWrapOnError(txn.Discard, err)
}
Expand All @@ -1120,23 +1061,3 @@ func (b *Blockchain) PendingState() (core.StateReader, StateCloser, error) {
core.NewState(txn),
), txn.Discard, nil
}

func MakeStateDiffForEmptyBlock(bc Reader, blockNumber uint64) (*core.StateDiff, error) {
stateDiff := core.EmptyStateDiff()

const blockHashLag = 10
if blockNumber < blockHashLag {
return stateDiff, nil
}

header, err := bc.BlockHeaderByNumber(blockNumber - blockHashLag)
if err != nil {
return nil, err
}

blockHashStorageContract := new(felt.Felt).SetUint64(1)
stateDiff.StorageDiffs[*blockHashStorageContract] = map[felt.Felt]*felt.Felt{
*new(felt.Felt).SetUint64(header.Number): header.Hash,
}
return stateDiff, nil
}
89 changes: 1 addition & 88 deletions blockchain/blockchain_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -4,21 +4,18 @@ import (
"context"
"fmt"
"testing"
"time"

"github.com/NethermindEth/juno/blockchain"
"github.com/NethermindEth/juno/clients/feeder"
"github.com/NethermindEth/juno/core"
"github.com/NethermindEth/juno/core/felt"
"github.com/NethermindEth/juno/db"
"github.com/NethermindEth/juno/db/pebble"
"github.com/NethermindEth/juno/mocks"
adaptfeeder "github.com/NethermindEth/juno/starknetdata/feeder"
"github.com/NethermindEth/juno/utils"
"github.com/ethereum/go-ethereum/common"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"go.uber.org/mock/gomock"
)

var emptyCommitments = core.BlockCommitments{}
Expand Down Expand Up @@ -356,7 +353,7 @@ func TestTransactionAndReceipt(t *testing.T) {
t.Run("GetTransactionByHash returns error if transaction does not exist", func(t *testing.T) {
tx, err := chain.TransactionByHash(new(felt.Felt).SetUint64(345))
assert.Nil(t, tx)
assert.EqualError(t, err, db.ErrKeyNotFound.Error())
assert.EqualError(t, err, blockchain.ErrPendingBlockNotFound.Error())
})

t.Run("GetTransactionReceipt returns error if receipt does not exist", func(t *testing.T) {
Expand Down Expand Up @@ -723,63 +720,6 @@ func TestPending(t *testing.T) {
b.GasPriceSTRK = utils.HexToFelt(t, "0xDEADBEEF")
require.NoError(t, chain.Store(b, &emptyCommitments, su, nil))

t.Run("no pending block means pending state matches head state", func(t *testing.T) {
pending, pErr := chain.Pending()
require.NoError(t, pErr)

require.LessOrEqual(t, pending.Block.Timestamp, uint64(time.Now().Unix()))
require.GreaterOrEqual(t, pending.Block.Timestamp, b.Timestamp)
receipts := make([]*core.TransactionReceipt, 0)
require.Equal(t, blockchain.Pending{
Block: &core.Block{
Header: &core.Header{
ParentHash: b.Hash,
SequencerAddress: b.SequencerAddress,
Number: b.Number + 1,
Timestamp: pending.Block.Timestamp, // Tested above.
ProtocolVersion: b.ProtocolVersion,
EventsBloom: core.EventsBloom(receipts),
GasPrice: b.GasPrice,
GasPriceSTRK: b.GasPriceSTRK,
},
Transactions: make([]core.Transaction, 0),
Receipts: receipts,
},
StateUpdate: &core.StateUpdate{
OldRoot: su.NewRoot,
StateDiff: core.EmptyStateDiff(),
},
NewClasses: make(map[felt.Felt]core.Class, 0),
}, pending)

// PendingState matches head state.
require.NoError(t, pErr)
reader, closer, pErr := chain.PendingState()
require.NoError(t, pErr)
t.Cleanup(func() {
require.NoError(t, closer())
})

for addr, diff := range su.StateDiff.StorageDiffs {
for key, diffVal := range diff {
value, csErr := reader.ContractStorage(&addr, &key)
require.NoError(t, csErr)
require.Equal(t, diffVal, value)
}
}

for address, nonce := range su.StateDiff.Nonces {
got, cnErr := reader.ContractNonce(&address)
require.NoError(t, cnErr)
require.Equal(t, nonce, got)
}

for _, hash := range su.StateDiff.DeclaredV0Classes {
_, err = reader.Class(hash)
require.NoError(t, err)
}
})

t.Run("storing a pending too far into the future should fail", func(t *testing.T) {
b, err = gw.BlockByNumber(context.Background(), 2)
require.NoError(t, err)
Expand Down Expand Up @@ -871,30 +811,3 @@ func TestStorePendingIncludesNumber(t *testing.T) {
require.NoError(t, err)
require.Equal(t, uint64(1), pending.Block.Number)
}

func TestMakeStateDiffForEmptyBlock(t *testing.T) {
mockCtrl := gomock.NewController(t)
t.Cleanup(mockCtrl.Finish)

mockReader := mocks.NewMockReader(mockCtrl)
t.Run("earlier blocks shouldnt have block hash in state", func(t *testing.T) {
for i := uint64(0); i < 10; i++ {
sd, err := blockchain.MakeStateDiffForEmptyBlock(mockReader, i)
require.NoError(t, err)
assert.Equal(t, core.EmptyStateDiff(), sd)
}
})

t.Run("should have block hash in state", func(t *testing.T) {
blockHash := utils.HexToFelt(t, "0xDEADBEEF")
storageContractAddr := utils.HexToFelt(t, "0x1")

mockReader.EXPECT().BlockHeaderByNumber(uint64(0)).Return(&core.Header{
Number: 0,
Hash: blockHash,
}, nil)
sd, err := blockchain.MakeStateDiffForEmptyBlock(mockReader, 10)
require.NoError(t, err)
assert.Equal(t, blockHash, sd.StorageDiffs[*storageContractAddr][felt.Zero])
})
}
Loading

0 comments on commit 7634b9f

Please sign in to comment.