Skip to content

Commit

Permalink
feat: Use InitL1InfoRootMap (#87)
Browse files Browse the repository at this point in the history
- Fix l1infotreesync. Error UNIQUE constraint failed: rollup_exit_root.hash on VerifyBatches Event
- l1infotreesync: Add verification for contract address. The problem is that if you set bad address cdk run normally but you don't get any information about L1InfoTree.
- l1infotreesync: Add support to `InitL1InfoRootMap`
- Allow to use `InitL1InfoRootMap` if there are no leaves on L1InfoTree

Internal:
- Fix local config file generation for debug on vscode (`./scripts/local_config`)
    - Add support to `contractVersions`
    - Remove param `-custom-network-file` that is no longer used
- Refactor `l1infotreesync/processor.go` in multiples files
- Change some tree functions to use a `tx db.Querier` instead of `ctx context.Context`: context was not used to do DB query was using `db` directly. In some test I need to query over current tx
  • Loading branch information
joanestebanr authored Sep 30, 2024
1 parent 6c148e9 commit c4c790f
Show file tree
Hide file tree
Showing 36 changed files with 2,050 additions and 209 deletions.
1 change: 1 addition & 0 deletions .github/workflows/test-e2e.yml
Original file line number Diff line number Diff line change
Expand Up @@ -65,6 +65,7 @@ jobs:
uses: actions/checkout@v4
with:
repository: 0xPolygon/kurtosis-cdk
ref: fix/missing_cdk_config_rollupmanager
path: "kurtosis-cdk"
ref: "v0.2.11"

Expand Down
24 changes: 23 additions & 1 deletion bridgesync/mock_l2_test.go

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

3 changes: 2 additions & 1 deletion cmd/run.go
Original file line number Diff line number Diff line change
Expand Up @@ -492,6 +492,7 @@ func runL1InfoTreeSyncerIfNeeded(
cfg.L1InfoTreeSync.InitialBlock,
cfg.L1InfoTreeSync.RetryAfterErrorPeriod.Duration,
cfg.L1InfoTreeSync.MaxRetryAttemptsAfterError,
l1infotreesync.FlagNone,
)
if err != nil {
log.Fatal(err)
Expand All @@ -511,7 +512,7 @@ func runL1ClientIfNeeded(components []string, urlRPCL1 string) *ethclient.Client
log.Debugf("dialing L1 client at: %s", urlRPCL1)
l1CLient, err := ethclient.Dial(urlRPCL1)
if err != nil {
log.Fatal(err)
log.Fatalf("failed to create client for L1 using URL: %s. Err:%v", urlRPCL1, err)
}

return l1CLient
Expand Down
22 changes: 21 additions & 1 deletion config/default.go
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,18 @@ const DefaultValues = `
ForkUpgradeBatchNumber = 0
ForkUpgradeNewForkId = 0
[Etherman]
URL="http://localhost:8545"
ForkIDChunkSize=100
[Etherman.EthermanConfig]
URL="http://localhost:8545"
MultiGasProvider=false
L1ChainID=1337
HTTPHeaders=[]
[Etherman.EthermanConfig.Etherscan]
ApiKey=""
Url="https://api.etherscan.io/api?module=gastracker&action=gasoracle&apikey="
[Common]
NetworkID = 1
IsValidiumMode = false
Expand Down Expand Up @@ -141,7 +153,7 @@ DBPath = "/tmp/reorgdetectorl1"
DBPath = "/tmp/reorgdetectorl2"
[L1InfoTreeSync]
DBPath = "/tmp/L1InfoTreeSync"
DBPath = "/tmp/L1InfoTreeSync.sqlite"
GlobalExitRootAddr="0x8464135c8F25Da09e49BC8782676a84730C318bC"
SyncBlockChunkSize=10
BlockFinality="LatestBlock"
Expand Down Expand Up @@ -250,4 +262,12 @@ RetryAfterErrorPeriod = "1s"
MaxRetryAttemptsAfterError = -1
WaitForNewBlocksPeriod = "1s"
DownloadBufferSize = 100
[NetworkConfig.L1]
L1ChainID = 0
PolAddr = "0x0000000000000000000000000000000000000000"
ZkEVMAddr = "0x0000000000000000000000000000000000000000"
RollupManagerAddr = "0x0000000000000000000000000000000000000000"
GlobalExitRootManagerAddr = "0x0000000000000000000000000000000000000000"
`
62 changes: 58 additions & 4 deletions l1infotreesync/downloader.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,15 +33,68 @@ type EthClienter interface {
bind.ContractBackend
}

func buildAppender(client EthClienter, globalExitRoot, rollupManager common.Address) (sync.LogAppenderMap, error) {
ger, err := polygonzkevmglobalexitrootv2.NewPolygonzkevmglobalexitrootv2(globalExitRoot, client)
func checkSMCIsRollupManager(rollupManagerAddr common.Address,
rollupManagerContract *polygonrollupmanager.Polygonrollupmanager) error {
bridgeAddr, err := rollupManagerContract.BridgeAddress(nil)
if err != nil {
return nil, err
return fmt.Errorf("fail sanity check RollupManager(%s) Contract. Err: %w", rollupManagerAddr.String(), err)
}
log.Infof("sanity check rollupManager(%s) OK. bridgeAddr: %s", rollupManagerAddr.String(), bridgeAddr.String())
return nil
}

func checkSMCIsGlobalExitRoot(globalExitRootAddr common.Address,
gerContract *polygonzkevmglobalexitrootv2.Polygonzkevmglobalexitrootv2) error {
depositCount, err := gerContract.DepositCount(nil)
if err != nil {
return fmt.Errorf("fail sanity check GlobalExitRoot(%s) Contract. Err: %w", globalExitRootAddr.String(), err)
}
rm, err := polygonrollupmanager.NewPolygonrollupmanager(rollupManager, client)
log.Infof("sanity check GlobalExitRoot(%s) OK. DepositCount: %v", globalExitRootAddr.String(), depositCount)
return nil
}

func sanityCheckContracts(globalExitRoot, rollupManager common.Address,
gerContract *polygonzkevmglobalexitrootv2.Polygonzkevmglobalexitrootv2,
rollupManagerContract *polygonrollupmanager.Polygonrollupmanager) error {
errGER := checkSMCIsGlobalExitRoot(globalExitRoot, gerContract)
errRollup := checkSMCIsRollupManager(rollupManager, rollupManagerContract)
if errGER != nil || errRollup != nil {
err := fmt.Errorf("sanityCheckContracts: fails sanity check contracts. ErrGER: %w, ErrRollup: %w", errGER, errRollup)
log.Error(err)
return err
}
return nil
}

func createContracts(client EthClienter, globalExitRoot, rollupManager common.Address) (
*polygonzkevmglobalexitrootv2.Polygonzkevmglobalexitrootv2,
*polygonrollupmanager.Polygonrollupmanager,
error) {
gerContract, err := polygonzkevmglobalexitrootv2.NewPolygonzkevmglobalexitrootv2(globalExitRoot, client)
if err != nil {
return nil, nil, err
}

rollupManagerContract, err := polygonrollupmanager.NewPolygonrollupmanager(rollupManager, client)
if err != nil {
return nil, nil, err
}
return gerContract, rollupManagerContract, nil
}

func buildAppender(client EthClienter, globalExitRoot,
rollupManager common.Address, flags CreationFlags) (sync.LogAppenderMap, error) {
ger, rm, err := createContracts(client, globalExitRoot, rollupManager)
if err != nil {
err := fmt.Errorf("buildAppender: fails contracts creation. Err:%w", err)
log.Error(err)
return nil, err
}
err = sanityCheckContracts(globalExitRoot, rollupManager, ger, rm)
if err != nil && flags&FlagAllowWrongContractsAddrs == 0 {
return nil, fmt.Errorf("buildAppender: fails sanity check contracts. Err:%w", err)
}

appender := make(sync.LogAppenderMap)
appender[initL1InfoRootMapSignature] = func(b *sync.EVMBlock, l types.Log) error {
init, err := ger.ParseInitL1InfoRootMap(l)
Expand Down Expand Up @@ -91,6 +144,7 @@ func buildAppender(client EthClienter, globalExitRoot, rollupManager common.Addr

return nil
}
// This event is coming from RollupManager
appender[verifyBatchesSignature] = func(b *sync.EVMBlock, l types.Log) error {
verifyBatches, err := rm.ParseVerifyBatches(l)
if err != nil {
Expand Down
55 changes: 55 additions & 0 deletions l1infotreesync/downloader_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,55 @@
package l1infotreesync

import (
"fmt"
"math/big"
"strings"
"testing"

"github.com/0xPolygon/cdk-contracts-tooling/contracts/banana/polygonzkevmglobalexitrootv2"
mocks_l1infotreesync "github.com/0xPolygon/cdk/l1infotreesync/mocks"
"github.com/ethereum/go-ethereum/accounts/abi"
"github.com/ethereum/go-ethereum/common"
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/require"
)

func TestBuildAppenderErrorOnBadContractAddr(t *testing.T) {
l1Client := mocks_l1infotreesync.NewEthClienter(t)
globalExitRoot := common.HexToAddress("0x1")
rollupManager := common.HexToAddress("0x2")
l1Client.EXPECT().CallContract(mock.Anything, mock.Anything, mock.Anything).Return(nil, fmt.Errorf("test-error"))
flags := FlagNone
_, err := buildAppender(l1Client, globalExitRoot, rollupManager, flags)
require.Error(t, err)
}

func TestBuildAppenderBypassBadContractAddr(t *testing.T) {
l1Client := mocks_l1infotreesync.NewEthClienter(t)
globalExitRoot := common.HexToAddress("0x1")
rollupManager := common.HexToAddress("0x2")
l1Client.EXPECT().CallContract(mock.Anything, mock.Anything, mock.Anything).Return(nil, fmt.Errorf("test-error"))
flags := FlagAllowWrongContractsAddrs
_, err := buildAppender(l1Client, globalExitRoot, rollupManager, flags)
require.NoError(t, err)
}

func TestBuildAppenderVerifiedContractAddr(t *testing.T) {
l1Client := mocks_l1infotreesync.NewEthClienter(t)
globalExitRoot := common.HexToAddress("0x1")
rollupManager := common.HexToAddress("0x2")

smcAbi, err := abi.JSON(strings.NewReader(polygonzkevmglobalexitrootv2.Polygonzkevmglobalexitrootv2ABI))
require.NoError(t, err)
bigInt := big.NewInt(1)
returnGER, err := smcAbi.Methods["depositCount"].Outputs.Pack(bigInt)
require.NoError(t, err)
l1Client.EXPECT().CallContract(mock.Anything, mock.Anything, mock.Anything).Return(returnGER, nil).Once()
v := common.HexToAddress("0x1234")
returnRM, err := smcAbi.Methods["bridgeAddress"].Outputs.Pack(v)
require.NoError(t, err)
l1Client.EXPECT().CallContract(mock.Anything, mock.Anything, mock.Anything).Return(returnRM, nil).Once()
flags := FlagNone
_, err = buildAppender(l1Client, globalExitRoot, rollupManager, flags)
require.NoError(t, err)
}
11 changes: 7 additions & 4 deletions l1infotreesync/e2e_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -80,7 +80,8 @@ func TestE2E(t *testing.T) {
rdm.On("AddBlockToTrack", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil)
client, gerAddr, verifyAddr, gerSc, verifySC, err := newSimulatedClient(auth)
require.NoError(t, err)
syncer, err := l1infotreesync.New(ctx, dbPath, gerAddr, verifyAddr, 10, etherman.LatestBlock, rdm, client.Client(), time.Millisecond, 0, 100*time.Millisecond, 3)
syncer, err := l1infotreesync.New(ctx, dbPath, gerAddr, verifyAddr, 10, etherman.LatestBlock, rdm, client.Client(), time.Millisecond, 0, 100*time.Millisecond, 3,
l1infotreesync.FlagAllowWrongContractsAddrs)
require.NoError(t, err)
go syncer.Start(ctx)

Expand Down Expand Up @@ -173,7 +174,8 @@ func TestWithReorgs(t *testing.T) {
rd, err := reorgdetector.New(client.Client(), reorgdetector.Config{DBPath: dbPathReorg, CheckReorgsInterval: cdktypes.NewDuration(time.Millisecond * 30)})
require.NoError(t, err)
require.NoError(t, rd.Start(ctx))
syncer, err := l1infotreesync.New(ctx, dbPathSyncer, gerAddr, verifyAddr, 10, etherman.LatestBlock, rd, client.Client(), time.Millisecond, 0, time.Second, 25)
syncer, err := l1infotreesync.New(ctx, dbPathSyncer, gerAddr, verifyAddr, 10, etherman.LatestBlock, rd, client.Client(), time.Millisecond, 0, time.Second, 25,
l1infotreesync.FlagAllowWrongContractsAddrs)
require.NoError(t, err)
go syncer.Start(ctx)

Expand Down Expand Up @@ -281,7 +283,7 @@ func TestStressAndReorgs(t *testing.T) {
)

ctx := context.Background()
dbPathSyncer := path.Join(t.TempDir(), "file::memory:?cache=shared")
dbPathSyncer := path.Join(t.TempDir(), "file:TestStressAndReorgs:memory:?cache=shared")
dbPathReorg := t.TempDir()
privateKey, err := crypto.GenerateKey()
require.NoError(t, err)
Expand All @@ -292,7 +294,8 @@ func TestStressAndReorgs(t *testing.T) {
rd, err := reorgdetector.New(client.Client(), reorgdetector.Config{DBPath: dbPathReorg, CheckReorgsInterval: cdktypes.NewDuration(time.Millisecond * 100)})
require.NoError(t, err)
require.NoError(t, rd.Start(ctx))
syncer, err := l1infotreesync.New(ctx, dbPathSyncer, gerAddr, verifyAddr, 10, etherman.LatestBlock, rd, client.Client(), time.Millisecond, 0, time.Second, 100)
syncer, err := l1infotreesync.New(ctx, dbPathSyncer, gerAddr, verifyAddr, 10, etherman.LatestBlock, rd, client.Client(), time.Millisecond, 0, time.Second, 100,
l1infotreesync.FlagAllowWrongContractsAddrs)
require.NoError(t, err)
go syncer.Start(ctx)

Expand Down
37 changes: 32 additions & 5 deletions l1infotreesync/l1infotreesync.go
Original file line number Diff line number Diff line change
Expand Up @@ -5,16 +5,26 @@ import (
"errors"
"time"

"github.com/0xPolygon/cdk/db"
"github.com/0xPolygon/cdk/etherman"
"github.com/0xPolygon/cdk/sync"
"github.com/0xPolygon/cdk/tree"
"github.com/0xPolygon/cdk/tree/types"
"github.com/ethereum/go-ethereum/common"
)

type CreationFlags uint64

const (
reorgDetectorID = "l1infotreesync"
downloadBufferSize = 1000
// CreationFlags defitinion
FlagNone CreationFlags = 0
FlagAllowWrongContractsAddrs CreationFlags = 1 << iota // Allow to set wrong contracts addresses
)

var (
ErrNotFound = errors.New("l1infotreesync: not found")
)

type L1InfoTreeSync struct {
Expand All @@ -36,6 +46,7 @@ func New(
initialBlock uint64,
retryAfterErrorPeriod time.Duration,
maxRetryAttemptsAfterError int,
flags CreationFlags,
) (*L1InfoTreeSync, error) {
processor, err := newProcessor(dbPath)
if err != nil {
Expand All @@ -59,7 +70,7 @@ func New(
MaxRetryAttemptsAfterError: maxRetryAttemptsAfterError,
}

appender, err := buildAppender(l1Client, globalExitRoot, rollupManager)
appender, err := buildAppender(l1Client, globalExitRoot, rollupManager, flags)
if err != nil {
return nil, err
}
Expand Down Expand Up @@ -111,10 +122,21 @@ func (s *L1InfoTreeSync) GetRollupExitTreeMerkleProof(
return s.processor.rollupExitTree.GetProof(ctx, networkID-1, root)
}

func translateError(err error) error {
if errors.Is(err, db.ErrNotFound) {
return ErrNotFound
}
return err
}

// GetLatestInfoUntilBlock returns the most recent L1InfoTreeLeaf that occurred before or at blockNum.
// If the blockNum has not been processed yet the error ErrBlockNotProcessed will be returned
// It can returns next errors:
// - ErrBlockNotProcessed,
// - ErrNotFound
func (s *L1InfoTreeSync) GetLatestInfoUntilBlock(ctx context.Context, blockNum uint64) (*L1InfoTreeLeaf, error) {
return s.processor.GetLatestInfoUntilBlock(ctx, blockNum)
leaf, err := s.processor.GetLatestInfoUntilBlock(ctx, blockNum)
return leaf, translateError(err)
}

// GetInfoByIndex returns the value of a leaf (not the hash) of the L1 info tree
Expand All @@ -129,12 +151,12 @@ func (s *L1InfoTreeSync) GetL1InfoTreeRootByIndex(ctx context.Context, index uin

// GetLastRollupExitRoot return the last rollup exit root processed
func (s *L1InfoTreeSync) GetLastRollupExitRoot(ctx context.Context) (types.Root, error) {
return s.processor.rollupExitTree.GetLastRoot(ctx)
return s.processor.rollupExitTree.GetLastRoot(nil)
}

// GetLastL1InfoTreeRoot return the last root and index processed from the L1 Info tree
func (s *L1InfoTreeSync) GetLastL1InfoTreeRoot(ctx context.Context) (types.Root, error) {
return s.processor.l1InfoTree.GetLastRoot(ctx)
return s.processor.l1InfoTree.GetLastRoot(nil)
}

// GetLastProcessedBlock return the last processed block
Expand All @@ -149,7 +171,7 @@ func (s *L1InfoTreeSync) GetLocalExitRoot(
return common.Hash{}, errors.New("network 0 is not a rollup, and it's not part of the rollup exit tree")
}

return s.processor.rollupExitTree.GetLeaf(ctx, networkID-1, rollupExitRoot)
return s.processor.rollupExitTree.GetLeaf(nil, networkID-1, rollupExitRoot)
}

func (s *L1InfoTreeSync) GetLastVerifiedBatches(rollupID uint32) (*VerifyBatches, error) {
Expand Down Expand Up @@ -190,3 +212,8 @@ func (s *L1InfoTreeSync) GetL1InfoTreeMerkleProofFromIndexToRoot(
) (types.Proof, error) {
return s.processor.l1InfoTree.GetProof(ctx, index, root)
}

// GetInitL1InfoRootMap returns the initial L1 info root map, nil if no root map has been set
func (s *L1InfoTreeSync) GetInitL1InfoRootMap(ctx context.Context) (*L1InfoTreeInitial, error) {
return s.processor.GetInitL1InfoRootMap(nil)
}
14 changes: 14 additions & 0 deletions l1infotreesync/migrations/l1infotreesync0002.sql
Original file line number Diff line number Diff line change
@@ -0,0 +1,14 @@
-- +migrate Down
DROP TABLE IF EXISTS l1info_initial;

-- +migrate Up

CREATE TABLE l1info_initial (
-- single_row_id prevent to have more than 1 row in this table
single_row_id INTEGER check(single_row_id=1) NOT NULL DEFAULT 1,
block_num INTEGER NOT NULL REFERENCES block(num) ON DELETE CASCADE,
leaf_count INTEGER NOT NULL,
l1_info_root VARCHAR NOT NULL,
PRIMARY KEY (single_row_id)
);

Loading

0 comments on commit c4c790f

Please sign in to comment.