Browse Source

Process withdrawals in HistoryDB, test withdrawals and vars in sync

feature/sql-semaphore1
Eduard S 3 years ago
parent
commit
9f5af0aa07
6 changed files with 332 additions and 36 deletions
  1. +66
    -1
      db/historydb/historydb.go
  2. +94
    -0
      db/historydb/historydb_test.go
  3. +4
    -3
      eth/rollup.go
  4. +70
    -4
      synchronizer/synchronizer_test.go
  5. +71
    -12
      test/ethclient.go
  6. +27
    -16
      test/til/txs.go

+ 66
- 1
db/historydb/historydb.go

@ -511,6 +511,51 @@ func (hdb *HistoryDB) addExitTree(d meddler.DB, exitTree []common.ExitInfo) erro
)
}
type exitID struct {
batchNum int64
idx int64
}
func (hdb *HistoryDB) updateExitTree(d meddler.DB, blockNum int64,
instantWithdrawn []exitID, delayedWithdrawRequest []exitID) error {
// helperQueryExitIDTuples is a helper function to build the query with
// an array of tuples in the arguments side built from []exitID
helperQueryExitIDTuples := func(queryTmpl string, blockNum int64, exits []exitID) (string, []interface{}) {
args := make([]interface{}, len(exits)*2+1)
holder := ""
args[0] = blockNum
for i, v := range exits {
args[1+i*2+0] = v.batchNum
args[1+i*2+1] = v.idx
holder += "(?, ?),"
}
query := fmt.Sprintf(queryTmpl, holder[:len(holder)-1])
return hdb.db.Rebind(query), args
}
if len(instantWithdrawn) > 0 {
query, args := helperQueryExitIDTuples(
`UPDATE exit_tree SET instant_withdrawn = ? WHERE (batch_num, account_idx) IN (%s);`,
blockNum,
instantWithdrawn,
)
if _, err := hdb.db.DB.Exec(query, args...); err != nil {
return err
}
}
if len(delayedWithdrawRequest) > 0 {
query, args := helperQueryExitIDTuples(
`UPDATE exit_tree SET delayed_withdraw_request = ? WHERE (batch_num, account_idx) IN (%s);`,
blockNum,
delayedWithdrawRequest,
)
if _, err := hdb.db.DB.Exec(query, args...); err != nil {
return err
}
}
return nil
}
// AddToken insert a token into the DB
func (hdb *HistoryDB) AddToken(token *common.Token) error {
return meddler.Insert(hdb.db, "token", token)
@ -1324,7 +1369,27 @@ func (hdb *HistoryDB) AddBlockSCData(blockData *common.BlockData) (err error) {
}
}
// TODO: Process withdrawals
if len(blockData.Rollup.Withdrawals) > 0 {
instantWithdrawn := []exitID{}
delayedWithdrawRequest := []exitID{}
for _, withdraw := range blockData.Rollup.Withdrawals {
exitID := exitID{
batchNum: int64(withdraw.NumExitRoot),
idx: int64(withdraw.Idx),
}
if withdraw.InstantWithdraw {
instantWithdrawn = append(instantWithdrawn, exitID)
} else {
delayedWithdrawRequest = append(delayedWithdrawRequest, exitID)
}
}
if err := hdb.updateExitTree(txn, blockData.Block.EthBlockNum,
instantWithdrawn, delayedWithdrawRequest); err != nil {
return err
}
}
// TODO: Process WDelayer withdrawals
return txn.Commit()
}

+ 94
- 0
db/historydb/historydb_test.go

@ -504,6 +504,100 @@ func TestSetInitialSCVars(t *testing.T) {
require.Equal(t, wDelayer, dbWDelayer)
}
func TestUpdateExitTree(t *testing.T) {
test.WipeDB(historyDB.DB())
set := `
Type: Blockchain
AddToken(1)
CreateAccountDeposit(1) C: 2000 // Idx=256+2=258
CreateAccountDeposit(1) D: 500 // Idx=256+3=259
CreateAccountCoordinator(1) A // Idx=256+0=256
CreateAccountCoordinator(1) B // Idx=256+1=257
> batchL1 // forge L1UserTxs{nil}, freeze defined L1UserTxs{5}
> batchL1 // forge defined L1UserTxs{5}, freeze L1UserTxs{nil}
> block // blockNum=2
ForceExit(1) A: 100
ForceExit(1) B: 80
Exit(1) C: 50 (200)
Exit(1) D: 30 (200)
> batchL1 // forge L1UserTxs{nil}, freeze defined L1UserTxs{3}
> batchL1 // forge L1UserTxs{3}, freeze defined L1UserTxs{nil}
> block // blockNum=3
> block // blockNum=4 (empty block)
`
tc := til.NewContext(common.RollupConstMaxL1UserTx)
tilCfgExtra := til.ConfigExtra{
BootCoordAddr: ethCommon.HexToAddress("0xE39fEc6224708f0772D2A74fd3f9055A90E0A9f2"),
CoordUser: "A",
}
blocks, err := tc.GenerateBlocks(set)
require.Nil(t, err)
err = tc.FillBlocksExtra(blocks, &tilCfgExtra)
assert.Nil(t, err)
// Add all blocks except for the last one
for i := range blocks[:len(blocks)-1] {
err = historyDB.AddBlockSCData(&blocks[i])
require.Nil(t, err)
}
// Add withdraws to the last block, and insert block into the DB
block := &blocks[len(blocks)-1]
require.Equal(t, int64(4), block.Block.EthBlockNum)
block.Rollup.Withdrawals = append(block.Rollup.Withdrawals,
common.WithdrawInfo{Idx: 256, NumExitRoot: 4, InstantWithdraw: true},
common.WithdrawInfo{Idx: 257, NumExitRoot: 4, InstantWithdraw: false},
common.WithdrawInfo{Idx: 258, NumExitRoot: 3, InstantWithdraw: true},
common.WithdrawInfo{Idx: 259, NumExitRoot: 3, InstantWithdraw: false},
)
err = historyDB.addBlock(historyDB.db, &block.Block)
require.Nil(t, err)
// update exit trees in DB
instantWithdrawn := []exitID{}
delayedWithdrawRequest := []exitID{}
for _, withdraw := range block.Rollup.Withdrawals {
exitID := exitID{
batchNum: int64(withdraw.NumExitRoot),
idx: int64(withdraw.Idx),
}
if withdraw.InstantWithdraw {
instantWithdrawn = append(instantWithdrawn, exitID)
} else {
delayedWithdrawRequest = append(delayedWithdrawRequest, exitID)
}
}
err = historyDB.updateExitTree(historyDB.db, block.Block.EthBlockNum, instantWithdrawn, delayedWithdrawRequest)
require.Nil(t, err)
// Check that exits in DB match with the expected values
dbExits, err := historyDB.GetAllExits()
require.Nil(t, err)
assert.Equal(t, 4, len(dbExits))
dbExitsByIdx := make(map[common.Idx]common.ExitInfo)
for _, dbExit := range dbExits {
dbExitsByIdx[dbExit.AccountIdx] = dbExit
}
for _, withdraw := range block.Rollup.Withdrawals {
assert.Equal(t, withdraw.NumExitRoot, dbExitsByIdx[withdraw.Idx].BatchNum)
if withdraw.InstantWithdraw {
assert.Equal(t, &block.Block.EthBlockNum, dbExitsByIdx[withdraw.Idx].InstantWithdrawn)
} else {
assert.Equal(t, &block.Block.EthBlockNum, dbExitsByIdx[withdraw.Idx].DelayedWithdrawRequest)
}
}
}
// setTestBlocks WARNING: this will delete the blocks and recreate them
func setTestBlocks(from, to int64) []common.Block {
test.WipeDB(historyDB.DB())

+ 4
- 3
eth/rollup.go

@ -36,9 +36,10 @@ func NewQueueStruct() *QueueStruct {
// RollupState represents the state of the Rollup in the Smart Contract
type RollupState struct {
StateRoot *big.Int
ExitRoots []*big.Int
ExitNullifierMap map[[256 / 8]byte]bool
StateRoot *big.Int
ExitRoots []*big.Int
// ExitNullifierMap map[[256 / 8]byte]bool
ExitNullifierMap map[int64]map[int64]bool // batchNum -> idx -> bool
TokenList []ethCommon.Address
TokenMap map[ethCommon.Address]bool
MapL1TxQueue map[int64]*QueueStruct

+ 70
- 4
synchronizer/synchronizer_test.go

@ -287,13 +287,15 @@ func TestSync(t *testing.T) {
CreateAccountDepositTransfer(1) E-A: 1000, 200 // Idx=256+7=263
ForceExit(1) A: 100
ForceExit(1) B: 80
ForceTransfer(1) A-D: 100
Transfer(1) C-A: 100 (200)
Exit(1) C: 50 (200)
Exit(1) D: 30 (200)
> batchL1 // forge L1UserTxs{nil}, freeze defined L1UserTxs{2}
> batchL1 // forge L1UserTxs{2}, freeze defined L1UserTxs{nil}
> batchL1 // forge L1UserTxs{nil}, freeze defined L1UserTxs{3}
> batchL1 // forge L1UserTxs{3}, freeze defined L1UserTxs{nil}
> block // blockNum=3
`
tc := til.NewContext(common.RollupConstMaxL1UserTx)
@ -315,9 +317,9 @@ func TestSync(t *testing.T) {
// blocks 1 (blockNum=3)
i = 1
require.Equal(t, 3, int(blocks[i].Block.EthBlockNum))
require.Equal(t, 3, len(blocks[i].Rollup.L1UserTxs))
require.Equal(t, 4, len(blocks[i].Rollup.L1UserTxs))
require.Equal(t, 2, len(blocks[i].Rollup.Batches))
require.Equal(t, 2, len(blocks[i].Rollup.Batches[0].L2Txs))
require.Equal(t, 3, len(blocks[i].Rollup.Batches[0].L2Txs))
// Generate extra required data
for _, block := range blocks {
@ -334,6 +336,7 @@ func TestSync(t *testing.T) {
err = tc.FillBlocksExtra(blocks, &tilCfgExtra)
assert.Nil(t, err)
tc.FillBlocksL1UserTxsBatchNum(blocks)
// Add block data to the smart contracts
for _, block := range blocks {
@ -401,6 +404,69 @@ func TestSync(t *testing.T) {
checkSyncBlock(t, s, 3, &blocks[1], syncBlock)
// Block 4
// Generate 2 withdraws manually
_, err = client.RollupWithdrawMerkleProof(tc.Users["A"].BJJ.Public(), 1, 4, 256, big.NewInt(100), []*big.Int{}, true)
require.Nil(t, err)
_, err = client.RollupWithdrawMerkleProof(tc.Users["C"].BJJ.Public(), 1, 3, 258, big.NewInt(50), []*big.Int{}, false)
require.Nil(t, err)
client.CtlMineBlock()
syncBlock, discards, err = s.Sync2(ctx, nil)
require.Nil(t, err)
require.Nil(t, discards)
require.NotNil(t, syncBlock)
assert.Equal(t, int64(4), syncBlock.Block.EthBlockNum)
dbExits, err := s.historyDB.GetAllExits()
require.Nil(t, err)
foundA1, foundC1 := false, false
for _, exit := range dbExits {
if exit.AccountIdx == 256 && exit.BatchNum == 4 {
foundA1 = true
assert.Equal(t, int64(4), *exit.InstantWithdrawn)
}
if exit.AccountIdx == 258 && exit.BatchNum == 3 {
foundC1 = true
assert.Equal(t, int64(4), *exit.DelayedWithdrawRequest)
}
}
assert.True(t, foundA1)
assert.True(t, foundC1)
// Block 5
// Update variables manually
rollupVars, auctionVars, wDelayerVars, err := s.historyDB.GetSCVars()
require.Nil(t, err)
rollupVars.ForgeL1L2BatchTimeout = 42
_, err = client.RollupUpdateForgeL1L2BatchTimeout(rollupVars.ForgeL1L2BatchTimeout)
require.Nil(t, err)
auctionVars.OpenAuctionSlots = 17
_, err = client.AuctionSetOpenAuctionSlots(auctionVars.OpenAuctionSlots)
require.Nil(t, err)
wDelayerVars.WithdrawalDelay = 99
_, err = client.WDelayerChangeWithdrawalDelay(wDelayerVars.WithdrawalDelay)
require.Nil(t, err)
client.CtlMineBlock()
syncBlock, discards, err = s.Sync2(ctx, nil)
require.Nil(t, err)
require.Nil(t, discards)
require.NotNil(t, syncBlock)
assert.Equal(t, int64(5), syncBlock.Block.EthBlockNum)
dbRollupVars, dbAuctionVars, dbWDelayerVars, err := s.historyDB.GetSCVars()
require.Nil(t, err)
// Set EthBlockNum for Vars to the blockNum in which they were updated (should be 5)
rollupVars.EthBlockNum = syncBlock.Block.EthBlockNum
auctionVars.EthBlockNum = syncBlock.Block.EthBlockNum
wDelayerVars.EthBlockNum = syncBlock.Block.EthBlockNum
assert.Equal(t, rollupVars, dbRollupVars)
assert.Equal(t, auctionVars, dbAuctionVars)
assert.Equal(t, wDelayerVars, dbWDelayerVars)
// TODO: Reorg will be properly tested once we have the mock ethClient implemented
/*
// Force a Reorg

+ 71
- 12
test/ethclient.go

@ -39,6 +39,12 @@ type WDelayerBlock struct {
Eth *EthereumBlock
}
func (w *WDelayerBlock) addTransaction(tx *types.Transaction) *types.Transaction {
txHash := tx.Hash()
w.Txs[txHash] = tx
return tx
}
// RollupBlock stores all the data related to the Rollup SC from an ethereum block
type RollupBlock struct {
State eth.RollupState
@ -369,8 +375,8 @@ func NewClient(l bool, timer Timer, addr *ethCommon.Address, setup *ClientSetup)
Rollup: &RollupBlock{
State: eth.RollupState{
StateRoot: big.NewInt(0),
ExitRoots: make([]*big.Int, 0),
ExitNullifierMap: make(map[[256 / 8]byte]bool),
ExitRoots: make([]*big.Int, 1),
ExitNullifierMap: make(map[int64]map[int64]bool),
// TokenID = 0 is ETH. Set first entry in TokenList with 0x0 address for ETH.
TokenList: []ethCommon.Address{{}},
TokenMap: make(map[ethCommon.Address]bool),
@ -746,9 +752,46 @@ func (c *Client) RollupWithdrawCircuit(proofA, proofC [2]*big.Int, proofB [2][2]
}
// RollupWithdrawMerkleProof is the interface to call the smart contract function
func (c *Client) RollupWithdrawMerkleProof(babyPubKey *babyjub.PublicKey, tokenID uint32, numExitRoot, idx int64, amount *big.Int, siblings []*big.Int, instantWithdraw bool) (*types.Transaction, error) {
log.Error("TODO")
return nil, errTODO
func (c *Client) RollupWithdrawMerkleProof(babyPubKey *babyjub.PublicKey, tokenID uint32, numExitRoot, idx int64, amount *big.Int, siblings []*big.Int, instantWithdraw bool) (tx *types.Transaction, err error) {
c.rw.Lock()
defer c.rw.Unlock()
cpy := c.nextBlock().copy()
defer func() { c.revertIfErr(err, cpy) }()
nextBlock := c.nextBlock()
r := nextBlock.Rollup
if int(numExitRoot) >= len(r.State.ExitRoots) {
return nil, fmt.Errorf("numExitRoot >= len(r.State.ExitRoots)")
}
if _, ok := r.State.ExitNullifierMap[numExitRoot][idx]; ok {
return nil, fmt.Errorf("exit already withdrawn")
}
r.State.ExitNullifierMap[numExitRoot][idx] = true
r.Events.Withdraw = append(r.Events.Withdraw, eth.RollupEventWithdraw{
Idx: uint64(idx),
NumExitRoot: uint64(numExitRoot),
InstantWithdraw: instantWithdraw,
})
type data struct {
babyPubKey *babyjub.PublicKey
tokenID uint32
numExitRoot int64
idx int64
amount *big.Int
siblings []*big.Int
instantWithdraw bool
}
return r.addTransaction(newTransaction("withdrawMerkleProof", data{
babyPubKey: babyPubKey,
tokenID: tokenID,
numExitRoot: numExitRoot,
idx: idx,
amount: amount,
siblings: siblings,
instantWithdraw: instantWithdraw,
})), nil
}
type transactionData struct {
@ -817,6 +860,7 @@ func (c *Client) addBatch(args *eth.RollupForgeBatchArgs) (*types.Transaction, e
return nil, fmt.Errorf("args.NewLastIdx < r.State.CurrentIdx")
}
r.State.CurrentIdx = args.NewLastIdx
r.State.ExitNullifierMap[int64(len(r.State.ExitRoots))] = make(map[int64]bool)
r.State.ExitRoots = append(r.State.ExitRoots, args.NewExitRoot)
if args.L1Batch {
r.State.CurrentToForgeL1TxsNum++
@ -828,7 +872,7 @@ func (c *Client) addBatch(args *eth.RollupForgeBatchArgs) (*types.Transaction, e
ethTx := r.addTransaction(newTransaction("forgebatch", args))
c.forgeBatchArgsPending[ethTx.Hash()] = &batch{*args, *c.addr}
r.Events.ForgeBatch = append(r.Events.ForgeBatch, eth.RollupEventForgeBatch{
BatchNum: int64(len(r.State.ExitRoots)),
BatchNum: int64(len(r.State.ExitRoots)) - 1,
EthTxHash: ethTx.Hash(),
})
@ -887,8 +931,13 @@ func (c *Client) RollupUpdateForgeL1L2BatchTimeout(newForgeL1Timeout int64) (tx
return nil, eth.ErrAccountNil
}
log.Error("TODO")
return nil, errTODO
nextBlock := c.nextBlock()
r := nextBlock.Rollup
r.Vars.ForgeL1L2BatchTimeout = newForgeL1Timeout
r.Events.UpdateForgeL1L2BatchTimeout = append(r.Events.UpdateForgeL1L2BatchTimeout,
eth.RollupEventUpdateForgeL1L2BatchTimeout{NewForgeL1L2BatchTimeout: newForgeL1Timeout})
return r.addTransaction(newTransaction("updateForgeL1L2BatchTimeout", newForgeL1Timeout)), nil
}
// RollupUpdateFeeAddToken is the interface to call the smart contract function
@ -990,8 +1039,13 @@ func (c *Client) AuctionSetOpenAuctionSlots(newOpenAuctionSlots uint16) (tx *typ
return nil, eth.ErrAccountNil
}
log.Error("TODO")
return nil, errTODO
nextBlock := c.nextBlock()
a := nextBlock.Auction
a.Vars.OpenAuctionSlots = newOpenAuctionSlots
a.Events.NewOpenAuctionSlots = append(a.Events.NewOpenAuctionSlots,
eth.AuctionEventNewOpenAuctionSlots{NewOpenAuctionSlots: newOpenAuctionSlots})
return a.addTransaction(newTransaction("setOpenAuctionSlots", newOpenAuctionSlots)), nil
}
// AuctionGetOpenAuctionSlots is the interface to call the smart contract function
@ -1507,8 +1561,13 @@ func (c *Client) WDelayerChangeWithdrawalDelay(newWithdrawalDelay uint64) (tx *t
return nil, eth.ErrAccountNil
}
log.Error("TODO")
return nil, errTODO
nextBlock := c.nextBlock()
w := nextBlock.WDelayer
w.Vars.WithdrawalDelay = newWithdrawalDelay
w.Events.NewWithdrawalDelay = append(w.Events.NewWithdrawalDelay,
eth.WDelayerEventNewWithdrawalDelay{WithdrawalDelay: newWithdrawalDelay})
return w.addTransaction(newTransaction("changeWithdrawalDelay", newWithdrawalDelay)), nil
}
// WDelayerDepositInfo is the interface to call the smart contract function

+ 27
- 16
test/til/txs.go

@ -663,9 +663,33 @@ type ConfigExtra struct {
CoordUser string
}
// FillBlocksL1UserTxsBatchNum fills the BatchNum of forged L1UserTxs:
// - blocks[].Rollup.L1UserTxs[].BatchNum
func (tc *Context) FillBlocksL1UserTxsBatchNum(blocks []common.BlockData) {
for i := range blocks {
block := &blocks[i]
for j := range block.Rollup.Batches {
batch := &block.Rollup.Batches[j]
if batch.L1Batch {
// Set BatchNum for forged L1UserTxs to til blocks
bn := batch.Batch.BatchNum
for k := range blocks {
block := &blocks[k]
for l := range block.Rollup.L1UserTxs {
tx := &block.Rollup.L1UserTxs[l]
if *tx.ToForgeL1TxsNum == tc.extra.openToForge {
tx.BatchNum = &bn
}
}
}
tc.extra.openToForge++
}
}
}
}
// FillBlocksExtra fills extra fields not generated by til in each block, so
// that the blockData is closer to what the HistoryDB stores. The filled fields are:
// - blocks[].Rollup.L1UserTxs[].BatchNum
// - blocks[].Rollup.Batch.EthBlockNum
// - blocks[].Rollup.Batch.ForgerAddr
// - blocks[].Rollup.Batch.ForgeL1TxsNum
@ -686,21 +710,6 @@ func (tc *Context) FillBlocksExtra(blocks []common.BlockData, cfg *ConfigExtra)
block := &blocks[i]
for j := range block.Rollup.Batches {
batch := &block.Rollup.Batches[j]
if batch.L1Batch {
// Set BatchNum for forged L1UserTxs to til blocks
bn := batch.Batch.BatchNum
for k := range blocks {
block := &blocks[k]
for l := range block.Rollup.L1UserTxs {
tx := &block.Rollup.L1UserTxs[l]
if *tx.ToForgeL1TxsNum == tc.extra.openToForge {
tx.BatchNum = &bn
}
}
}
tc.extra.openToForge++
}
batch.Batch.EthBlockNum = block.Block.EthBlockNum
// til doesn't fill the batch forger addr
batch.Batch.ForgerAddr = cfg.BootCoordAddr
@ -802,6 +811,7 @@ func (tc *Context) FillBlocksExtra(blocks []common.BlockData, cfg *ConfigExtra)
batch.ExitTree =
append(batch.ExitTree,
common.ExitInfo{
BatchNum: batch.Batch.BatchNum,
AccountIdx: tx.FromIdx,
Balance: tx.Amount,
})
@ -812,6 +822,7 @@ func (tc *Context) FillBlocksExtra(blocks []common.BlockData, cfg *ConfigExtra)
tx := &batch.L2Txs[k]
if tx.Type == common.TxTypeExit {
batch.ExitTree = append(batch.ExitTree, common.ExitInfo{
BatchNum: batch.Batch.BatchNum,
AccountIdx: tx.FromIdx,
Balance: tx.Amount,
})

Loading…
Cancel
Save