Skip to content

Commit

Permalink
etrog: batch timestamp is set when is virtualized (0xPolygonHermez#2904)
Browse files Browse the repository at this point in the history
* + add field virtual.timestamp_batch_etrog into DB, rpc returns this timestamp of the batch of null if not defined yet
  • Loading branch information
joanestebanr authored Dec 14, 2023
1 parent b5dd453 commit c9c72e1
Show file tree
Hide file tree
Showing 21 changed files with 299 additions and 73 deletions.
6 changes: 6 additions & 0 deletions db/migrations/state/0013.sql
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,9 @@ CREATE INDEX IF NOT EXISTS idx_transaction_l2_hash ON state.transaction (l2_hash
ALTER TABLE state.batch
ADD COLUMN wip BOOLEAN NOT NULL;

ALTER TABLE state.virtual_batch
ADD COLUMN timestamp_batch_etrog TIMESTAMP WITH TIME ZONE NULL;

-- +migrate Down
ALTER TABLE state.exit_root
DROP COLUMN prev_block_hash,
Expand All @@ -30,3 +33,6 @@ DROP INDEX IF EXISTS state.idx_transaction_l2_hash;
ALTER TABLE state.batch
DROP COLUMN wip;

ALTER TABLE state.virtual_batch
DROP COLUMN timestamp_batch_etrog;

10 changes: 10 additions & 0 deletions jsonrpc/endpoints_zkevm.go
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,7 @@ import (
"errors"
"fmt"
"math/big"
"time"

"github.com/0xPolygonHermez/zkevm-node/hex"
"github.com/0xPolygonHermez/zkevm-node/jsonrpc/types"
Expand Down Expand Up @@ -139,6 +140,15 @@ func (z *ZKEVMEndpoints) GetBatchByNumber(batchNumber types.BatchNumber, fullTx
} else if err != nil {
return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't load batch from state by number %v", batchNumber), err, true)
}
batchTimestamp, err := z.state.GetBatchTimestamp(ctx, batchNumber, nil, dbTx)
if err != nil {
return RPCErrorResponse(types.DefaultErrorCode, fmt.Sprintf("couldn't load batch timestamp from state by number %v", batchNumber), err, true)
}
if batchTimestamp == nil {
batch.Timestamp = time.Time{}
} else {
batch.Timestamp = *batchTimestamp
}

txs, _, err := z.state.GetTransactionsByBatchNumber(ctx, batchNumber, dbTx)
if !errors.Is(err, state.ErrNotFound) && err != nil {
Expand Down
16 changes: 16 additions & 0 deletions jsonrpc/endpoints_zkevm_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/trie"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/require"
)

Expand Down Expand Up @@ -732,6 +733,11 @@ func TestGetBatchByNumber(t *testing.T) {
Return(batch, nil).
Once()

m.State.
On("GetBatchTimestamp", mock.Anything, mock.Anything, (*uint64)(nil), m.DbTx).
Return(&batch.Timestamp, nil).
Once()

virtualBatch := &state.VirtualBatch{
TxHash: common.HexToHash("0x10"),
}
Expand Down Expand Up @@ -851,6 +857,11 @@ func TestGetBatchByNumber(t *testing.T) {
Return(batch, nil).
Once()

m.State.
On("GetBatchTimestamp", mock.Anything, mock.Anything, (*uint64)(nil), m.DbTx).
Return(&batch.Timestamp, nil).
Once()

virtualBatch := &state.VirtualBatch{
TxHash: common.HexToHash("0x10"),
}
Expand Down Expand Up @@ -1001,6 +1012,11 @@ func TestGetBatchByNumber(t *testing.T) {
Return(batch, nil).
Once()

m.State.
On("GetBatchTimestamp", mock.Anything, mock.Anything, (*uint64)(nil), m.DbTx).
Return(&batch.Timestamp, nil).
Once()

virtualBatch := &state.VirtualBatch{
TxHash: common.HexToHash("0x10"),
}
Expand Down
26 changes: 26 additions & 0 deletions jsonrpc/mocks/mock_state.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

1 change: 1 addition & 0 deletions jsonrpc/types/interfaces.go
Original file line number Diff line number Diff line change
Expand Up @@ -68,6 +68,7 @@ type StateInterface interface {
GetLastClosedBatchNumber(ctx context.Context, dbTx pgx.Tx) (uint64, error)
GetLastVerifiedL2BlockNumberUntilL1Block(ctx context.Context, l1FinalizedBlockNumber uint64, dbTx pgx.Tx) (uint64, error)
GetLastVerifiedBatchNumberUntilL1Block(ctx context.Context, l1BlockNumber uint64, dbTx pgx.Tx) (uint64, error)
GetBatchTimestamp(ctx context.Context, batchNumber uint64, forcedForkId *uint64, dbTx pgx.Tx) (*time.Time, error)
}

// EthermanInterface provides integration with L1
Expand Down
1 change: 1 addition & 0 deletions jsonrpc/types/types.go
Original file line number Diff line number Diff line change
Expand Up @@ -371,6 +371,7 @@ func NewBatch(batch *state.Batch, virtualBatch *state.VirtualBatch, verifiedBatc
BatchL2Data: ArgBytes(batchL2Data),
Closed: closed,
}

if batch.ForcedBatchNum != nil {
fb := ArgUint64(*batch.ForcedBatchNum)
res.ForcedBatchNumber = &fb
Expand Down
3 changes: 2 additions & 1 deletion sequencer/batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -139,7 +139,8 @@ func (f *finalizer) setWIPBatch(ctx context.Context, wipStateBatch *state.Batch)

//TODO: We execute the batch to get the used counter. To avoid this We can update the counters in the state.batch table for the wip batch
if wipStateBatchCountOfTxs > 0 {
batchResponse, err := f.state.ExecuteBatchV2(ctx, *wipStateBatch, false, dbTx)
//TODO: Change wipStateBatch.GlobalExitRoot for L1InfoRoot and wipStateBatch.Timestamp for the TimeLimit
batchResponse, err := f.state.ExecuteBatchV2(ctx, *wipStateBatch, wipStateBatch.GlobalExitRoot, wipStateBatch.Timestamp, false, dbTx)
if err != nil {
return nil, err
}
Expand Down
2 changes: 1 addition & 1 deletion sequencer/interfaces.go
Original file line number Diff line number Diff line change
Expand Up @@ -61,7 +61,7 @@ type stateInterface interface {
CloseBatch(ctx context.Context, receipt state.ProcessingReceipt, dbTx pgx.Tx) error
CloseWIPBatch(ctx context.Context, receipt state.ProcessingReceipt, dbTx pgx.Tx) error
ExecuteBatch(ctx context.Context, batch state.Batch, updateMerkleTree bool, dbTx pgx.Tx) (*executor.ProcessBatchResponse, error)
ExecuteBatchV2(ctx context.Context, batch state.Batch, updateMerkleTree bool, dbTx pgx.Tx) (*executor.ProcessBatchResponseV2, error)
ExecuteBatchV2(ctx context.Context, batch state.Batch, l1InfoRoot common.Hash, timestampLimit time.Time, updateMerkleTree bool, dbTx pgx.Tx) (*executor.ProcessBatchResponseV2, error)
GetForcedBatch(ctx context.Context, forcedBatchNumber uint64, dbTx pgx.Tx) (*state.ForcedBatch, error)
GetLastBatch(ctx context.Context, dbTx pgx.Tx) (*state.Batch, error)
GetLastBatchNumber(ctx context.Context, dbTx pgx.Tx) (uint64, error)
Expand Down
18 changes: 9 additions & 9 deletions sequencer/mock_state.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

38 changes: 32 additions & 6 deletions state/batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,12 +25,14 @@ const (

// Batch struct
type Batch struct {
BatchNumber uint64
Coinbase common.Address
BatchL2Data []byte
StateRoot common.Hash
LocalExitRoot common.Hash
AccInputHash common.Hash
BatchNumber uint64
Coinbase common.Address
BatchL2Data []byte
StateRoot common.Hash
LocalExitRoot common.Hash
AccInputHash common.Hash
// Timestamp (<=incaberry) -> batch time
// (>incaberry) -> minTimestamp used in batch creation, real timestamp is in virtual_batch.batch_timestamp
Timestamp time.Time
Transactions []types.Transaction
GlobalExitRoot common.Hash
Expand Down Expand Up @@ -97,6 +99,9 @@ type VirtualBatch struct {
Coinbase common.Address
SequencerAddr common.Address
BlockNumber uint64
// TimestampBatchEtrog etrog: Batch timestamp comes from L1 block timestamp
// for previous batches is NULL because the batch timestamp is in batch table
TimestampBatchEtrog *time.Time
}

// Sequence represents the sequence interval
Expand Down Expand Up @@ -525,3 +530,24 @@ func (s *State) GetLastBatch(ctx context.Context, dbTx pgx.Tx) (*Batch, error) {
}
return batches[0], nil
}

// GetBatchTimestamp returns the batch timestamp.
//
// for >= etrog is stored on virtual_batch.batch_timestamp
// previous batches is stored on batch.timestamp
func (s *State) GetBatchTimestamp(ctx context.Context, batchNumber uint64, forcedForkId *uint64, dbTx pgx.Tx) (*time.Time, error) {
var forkid uint64
if forcedForkId != nil {
forkid = *forcedForkId
} else {
forkid = s.GetForkIDByBatchNumber(batchNumber)
}
batchTimestamp, virtualTimestamp, err := s.GetRawBatchTimestamps(ctx, batchNumber, dbTx)
if err != nil {
return nil, err
}
if forkid >= FORKID_ETROG {
return virtualTimestamp, nil
}
return batchTimestamp, nil
}
7 changes: 4 additions & 3 deletions state/batchV2.go
Original file line number Diff line number Diff line change
Expand Up @@ -99,7 +99,7 @@ func (s *State) ProcessBatchV2(ctx context.Context, request ProcessRequest, upda

// ExecuteBatchV2 is used by the synchronizer to reprocess batches to compare generated state root vs stored one
// It is also used by the sequencer in order to calculate used zkCounter of a WIPBatch
func (s *State) ExecuteBatchV2(ctx context.Context, batch Batch, updateMerkleTree bool, dbTx pgx.Tx) (*executor.ProcessBatchResponseV2, error) {
func (s *State) ExecuteBatchV2(ctx context.Context, batch Batch, l1InfoRoot common.Hash, timestampLimit time.Time, updateMerkleTree bool, dbTx pgx.Tx) (*executor.ProcessBatchResponseV2, error) {
if dbTx == nil {
return nil, ErrDBTxNil
}
Expand All @@ -124,10 +124,10 @@ func (s *State) ExecuteBatchV2(ctx context.Context, batch Batch, updateMerkleTre
BatchL2Data: batch.BatchL2Data,
OldStateRoot: previousBatch.StateRoot.Bytes(),
// TODO: Change this to L1InfoRoot
L1InfoRoot: batch.GlobalExitRoot.Bytes(),
L1InfoRoot: l1InfoRoot.Bytes(),
OldAccInputHash: previousBatch.AccInputHash.Bytes(),
// TODO: Change this to TimestampLimit
TimestampLimit: uint64(batch.Timestamp.Unix()),
TimestampLimit: uint64(timestampLimit.Unix()),
// Changed for new sequencer strategy
UpdateMerkleTree: updateMT,
ChainId: s.cfg.ChainID,
Expand Down Expand Up @@ -264,6 +264,7 @@ func (s *State) sendBatchRequestToExecutorV2(ctx context.Context, processBatchRe
err = executor.ExecutorErr(res.Error)
s.eventLog.LogExecutorErrorV2(ctx, res.Error, processBatchRequest)
}
//workarroundDuplicatedBlock(res)
elapsed := time.Since(now)
if caller != metrics.DiscardCallerLabel {
metrics.ExecutorProcessingTime(string(caller), elapsed)
Expand Down
3 changes: 2 additions & 1 deletion state/interfaces.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@ type storage interface {
GetLastBatchTime(ctx context.Context, dbTx pgx.Tx) (time.Time, error)
GetLastVirtualBatchNum(ctx context.Context, dbTx pgx.Tx) (uint64, error)
GetLatestVirtualBatchTimestamp(ctx context.Context, dbTx pgx.Tx) (time.Time, error)
GetVirtualBatch(ctx context.Context, batchNumber uint64, dbTx pgx.Tx) (*VirtualBatch, error)
SetLastBatchInfoSeenOnEthereum(ctx context.Context, lastBatchNumberSeen, lastBatchNumberVerified uint64, dbTx pgx.Tx) error
SetInitSyncBatch(ctx context.Context, batchNumber uint64, dbTx pgx.Tx) error
GetBatchByNumber(ctx context.Context, batchNumber uint64, dbTx pgx.Tx) (*Batch, error)
Expand All @@ -53,7 +54,6 @@ type storage interface {
GetTransactionsByBatchNumber(ctx context.Context, batchNumber uint64, dbTx pgx.Tx) (txs []types.Transaction, effectivePercentages []uint8, err error)
GetTxsHashesByBatchNumber(ctx context.Context, batchNumber uint64, dbTx pgx.Tx) (encoded []common.Hash, err error)
AddVirtualBatch(ctx context.Context, virtualBatch *VirtualBatch, dbTx pgx.Tx) error
GetVirtualBatch(ctx context.Context, batchNumber uint64, dbTx pgx.Tx) (*VirtualBatch, error)
UpdateGERInOpenBatch(ctx context.Context, ger common.Hash, dbTx pgx.Tx) error
IsBatchClosed(ctx context.Context, batchNum uint64, dbTx pgx.Tx) (bool, error)
GetNextForcedBatches(ctx context.Context, nextForcedBatches int, dbTx pgx.Tx) ([]ForcedBatch, error)
Expand Down Expand Up @@ -140,4 +140,5 @@ type storage interface {
GetForkIDByBatchNumber(batchNumber uint64) uint64
GetLatestIndex(ctx context.Context, dbTx pgx.Tx) (uint32, error)
BuildChangeL2Block(deltaTimestamp uint32, l1InfoTreeIndex uint32) []byte
GetRawBatchTimestamps(ctx context.Context, batchNumber uint64, dbTx pgx.Tx) (*time.Time, *time.Time, error)
}
38 changes: 32 additions & 6 deletions state/pgstatestorage/batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -483,10 +483,18 @@ func scanForcedBatch(row pgx.Row) (state.ForcedBatch, error) {

// AddVirtualBatch adds a new virtual batch to the storage.
func (p *PostgresStorage) AddVirtualBatch(ctx context.Context, virtualBatch *state.VirtualBatch, dbTx pgx.Tx) error {
const addVirtualBatchSQL = "INSERT INTO state.virtual_batch (batch_num, tx_hash, coinbase, block_num, sequencer_addr) VALUES ($1, $2, $3, $4, $5)"
e := p.getExecQuerier(dbTx)
_, err := e.Exec(ctx, addVirtualBatchSQL, virtualBatch.BatchNumber, virtualBatch.TxHash.String(), virtualBatch.Coinbase.String(), virtualBatch.BlockNumber, virtualBatch.SequencerAddr.String())
return err
if virtualBatch.TimestampBatchEtrog == nil {
const addVirtualBatchSQL = "INSERT INTO state.virtual_batch (batch_num, tx_hash, coinbase, block_num, sequencer_addr) VALUES ($1, $2, $3, $4, $5)"
e := p.getExecQuerier(dbTx)
_, err := e.Exec(ctx, addVirtualBatchSQL, virtualBatch.BatchNumber, virtualBatch.TxHash.String(), virtualBatch.Coinbase.String(), virtualBatch.BlockNumber, virtualBatch.SequencerAddr.String())
return err
} else {
const addVirtualBatchSQL = "INSERT INTO state.virtual_batch (batch_num, tx_hash, coinbase, block_num, sequencer_addr, timestamp_batch_etrog) VALUES ($1, $2, $3, $4, $5, $6)"
e := p.getExecQuerier(dbTx)
_, err := e.Exec(ctx, addVirtualBatchSQL, virtualBatch.BatchNumber, virtualBatch.TxHash.String(), virtualBatch.Coinbase.String(), virtualBatch.BlockNumber, virtualBatch.SequencerAddr.String(),
virtualBatch.TimestampBatchEtrog.UTC())
return err
}
}

// GetVirtualBatch get an L1 virtualBatch.
Expand All @@ -499,12 +507,12 @@ func (p *PostgresStorage) GetVirtualBatch(ctx context.Context, batchNumber uint6
)

const getVirtualBatchSQL = `
SELECT block_num, batch_num, tx_hash, coinbase, sequencer_addr
SELECT block_num, batch_num, tx_hash, coinbase, sequencer_addr, timestamp_batch_etrog
FROM state.virtual_batch
WHERE batch_num = $1`

e := p.getExecQuerier(dbTx)
err := e.QueryRow(ctx, getVirtualBatchSQL, batchNumber).Scan(&virtualBatch.BlockNumber, &virtualBatch.BatchNumber, &txHash, &coinbase, &sequencerAddr)
err := e.QueryRow(ctx, getVirtualBatchSQL, batchNumber).Scan(&virtualBatch.BlockNumber, &virtualBatch.BatchNumber, &txHash, &coinbase, &sequencerAddr, &virtualBatch.TimestampBatchEtrog)
if errors.Is(err, pgx.ErrNoRows) {
return nil, state.ErrNotFound
} else if err != nil {
Expand Down Expand Up @@ -899,3 +907,21 @@ func (p *PostgresStorage) BuildChangeL2Block(deltaTimestamp uint32, l1InfoTreeIn

return changeL2Block
}

// GetRawBatchTimestamps returns the timestamp of the batch with the given number.
// it returns batch_num.tstamp and virtual_batch.batch_timestamp
func (p *PostgresStorage) GetRawBatchTimestamps(ctx context.Context, batchNumber uint64, dbTx pgx.Tx) (*time.Time, *time.Time, error) {
const sql = `
SELECT b.timestamp AS batch_timestamp, v.timestamp_batch_etrog AS virtual_batch_timestamp
FROM state.batch AS b
LEFT JOIN state.virtual_batch AS v ON b.batch_num = v.batch_num
WHERE b.batch_num = $1;
`
var batchTimestamp, virtualBatchTimestamp *time.Time
e := p.getExecQuerier(dbTx)
err := e.QueryRow(ctx, sql, batchNumber).Scan(&batchTimestamp, &virtualBatchTimestamp)
if errors.Is(err, pgx.ErrNoRows) {
return nil, nil, nil
}
return batchTimestamp, virtualBatchTimestamp, err
}
Loading

0 comments on commit c9c72e1

Please sign in to comment.