Browse Source

Merge pull request #489 from hermeznetwork/feature/test-flow-txsel-batchbuild-zkinputs

Add test connecting TxSelector, BatchBuilder, ZKInputs, ProofServer
feature/sql-semaphore1
Eduard S 3 years ago
committed by GitHub
parent
commit
9ce4dd9b6d
No known key found for this signature in database GPG Key ID: 4AEE18F83AFDEB23
10 changed files with 347 additions and 95 deletions
  1. +4
    -9
      batchbuilder/batchbuilder.go
  2. +1
    -1
      batchbuilder/batchbuilder_test.go
  3. +1
    -1
      coordinator/coordinator_test.go
  4. +1
    -2
      coordinator/pipeline.go
  5. +2
    -1
      db/l2db/l2db_test.go
  6. +1
    -1
      node/node.go
  7. +247
    -0
      test/zkproof/flows_test.go
  8. +49
    -45
      test/zkproof/zkproof_test.go
  9. +1
    -1
      txprocessor/zkinputsgen_test.go
  10. +40
    -34
      txselector/txselector_test.go

+ 4
- 9
batchbuilder/batchbuilder.go

@ -1,7 +1,6 @@
package batchbuilder package batchbuilder
import ( import (
ethCommon "github.com/ethereum/go-ethereum/common"
"github.com/hermeznetwork/hermez-node/common" "github.com/hermeznetwork/hermez-node/common"
"github.com/hermeznetwork/hermez-node/db/statedb" "github.com/hermeznetwork/hermez-node/db/statedb"
"github.com/hermeznetwork/hermez-node/txprocessor" "github.com/hermeznetwork/hermez-node/txprocessor"
@ -18,19 +17,17 @@ type ConfigCircuit struct {
// BatchBuilder implements the batch builder type, which contains the // BatchBuilder implements the batch builder type, which contains the
// functionalities // functionalities
type BatchBuilder struct { type BatchBuilder struct {
localStateDB *statedb.LocalStateDB
configCircuits []ConfigCircuit
localStateDB *statedb.LocalStateDB
} }
// ConfigBatch contains the batch configuration // ConfigBatch contains the batch configuration
type ConfigBatch struct { type ConfigBatch struct {
ForgerAddress ethCommon.Address
TxProcessorConfig txprocessor.Config TxProcessorConfig txprocessor.Config
} }
// NewBatchBuilder constructs a new BatchBuilder, and executes the bb.Reset // NewBatchBuilder constructs a new BatchBuilder, and executes the bb.Reset
// method // method
func NewBatchBuilder(dbpath string, synchronizerStateDB *statedb.StateDB, configCircuits []ConfigCircuit, batchNum common.BatchNum, nLevels uint64) (*BatchBuilder, error) {
func NewBatchBuilder(dbpath string, synchronizerStateDB *statedb.StateDB, batchNum common.BatchNum, nLevels uint64) (*BatchBuilder, error) {
localStateDB, err := statedb.NewLocalStateDB(dbpath, 128, synchronizerStateDB, localStateDB, err := statedb.NewLocalStateDB(dbpath, 128, synchronizerStateDB,
statedb.TypeBatchBuilder, int(nLevels)) statedb.TypeBatchBuilder, int(nLevels))
if err != nil { if err != nil {
@ -38,8 +35,7 @@ func NewBatchBuilder(dbpath string, synchronizerStateDB *statedb.StateDB, config
} }
bb := BatchBuilder{ bb := BatchBuilder{
localStateDB: localStateDB,
configCircuits: configCircuits,
localStateDB: localStateDB,
} }
err = bb.Reset(batchNum, true) err = bb.Reset(batchNum, true)
@ -56,8 +52,7 @@ func (bb *BatchBuilder) Reset(batchNum common.BatchNum, fromSynchronizer bool) e
// BuildBatch takes the transactions and returns the common.ZKInputs of the next batch // BuildBatch takes the transactions and returns the common.ZKInputs of the next batch
func (bb *BatchBuilder) BuildBatch(coordIdxs []common.Idx, configBatch *ConfigBatch, l1usertxs, func (bb *BatchBuilder) BuildBatch(coordIdxs []common.Idx, configBatch *ConfigBatch, l1usertxs,
l1coordinatortxs []common.L1Tx, pooll2txs []common.PoolL2Tx,
tokenIDs []common.TokenID) (*common.ZKInputs, error) {
l1coordinatortxs []common.L1Tx, pooll2txs []common.PoolL2Tx) (*common.ZKInputs, error) {
bbStateDB := bb.localStateDB.StateDB bbStateDB := bb.localStateDB.StateDB
tp := txprocessor.NewTxProcessor(bbStateDB, configBatch.TxProcessorConfig) tp := txprocessor.NewTxProcessor(bbStateDB, configBatch.TxProcessorConfig)

+ 1
- 1
batchbuilder/batchbuilder_test.go

@ -21,6 +21,6 @@ func TestBatchBuilder(t *testing.T) {
bbDir, err := ioutil.TempDir("", "tmpBatchBuilderDB") bbDir, err := ioutil.TempDir("", "tmpBatchBuilderDB")
require.Nil(t, err) require.Nil(t, err)
defer assert.Nil(t, os.RemoveAll(bbDir)) defer assert.Nil(t, os.RemoveAll(bbDir))
_, err = NewBatchBuilder(bbDir, synchDB, nil, 0, 32)
_, err = NewBatchBuilder(bbDir, synchDB, 0, 32)
assert.Nil(t, err) assert.Nil(t, err)
} }

+ 1
- 1
coordinator/coordinator_test.go

@ -125,7 +125,7 @@ func newTestModules(t *testing.T) modules {
batchBuilderDBPath, err = ioutil.TempDir("", "tmpBatchBuilderDB") batchBuilderDBPath, err = ioutil.TempDir("", "tmpBatchBuilderDB")
require.NoError(t, err) require.NoError(t, err)
deleteme = append(deleteme, batchBuilderDBPath) deleteme = append(deleteme, batchBuilderDBPath)
batchBuilder, err := batchbuilder.NewBatchBuilder(batchBuilderDBPath, syncStateDB, nil, 0, uint64(nLevels))
batchBuilder, err := batchbuilder.NewBatchBuilder(batchBuilderDBPath, syncStateDB, 0, uint64(nLevels))
assert.NoError(t, err) assert.NoError(t, err)
return modules{ return modules{

+ 1
- 2
coordinator/pipeline.go

@ -348,11 +348,10 @@ func (p *Pipeline) forgeBatch(batchNum common.BatchNum) (batchInfo *BatchInfo, e
// 4. Call BatchBuilder with TxSelector output // 4. Call BatchBuilder with TxSelector output
configBatch := &batchbuilder.ConfigBatch{ configBatch := &batchbuilder.ConfigBatch{
ForgerAddress: p.cfg.ForgerAddress,
TxProcessorConfig: p.cfg.TxProcessorConfig, TxProcessorConfig: p.cfg.TxProcessorConfig,
} }
zkInputs, err := p.batchBuilder.BuildBatch(coordIdxs, configBatch, l1UserTxsExtra, zkInputs, err := p.batchBuilder.BuildBatch(coordIdxs, configBatch, l1UserTxsExtra,
l1CoordTxs, poolL2Txs, nil)
l1CoordTxs, poolL2Txs)
if err != nil { if err != nil {
return nil, tracerr.Wrap(err) return nil, tracerr.Wrap(err)
} }

+ 2
- 1
db/l2db/l2db_test.go

@ -131,7 +131,8 @@ func generatePoolL2Txs() ([]common.PoolL2Tx, error) {
PoolTransfer(1) B-A: 8 (2) PoolTransfer(1) B-A: 8 (2)
PoolTransfer(2) B-A: 1 (1) PoolTransfer(2) B-A: 1 (1)
PoolTransfer(1) A-B: 3 (1) PoolTransfer(1) A-B: 3 (1)
PoolTransfer(2) B-A: 5 (2)
PoolTransferToEthAddr(2) B-A: 5 (2)
PoolTransferToBJJ(2) B-A: 5 (2)
PoolExit(1) A: 5 (2) PoolExit(1) A: 5 (2)
PoolExit(2) B: 3 (1) PoolExit(2) B: 3 (1)

+ 1
- 1
node/node.go

@ -228,7 +228,7 @@ func NewNode(mode Mode, cfg *config.Node) (*Node, error) {
return nil, tracerr.Wrap(err) return nil, tracerr.Wrap(err)
} }
batchBuilder, err := batchbuilder.NewBatchBuilder(cfg.Coordinator.BatchBuilder.Path, batchBuilder, err := batchbuilder.NewBatchBuilder(cfg.Coordinator.BatchBuilder.Path,
stateDB, nil, 0, uint64(cfg.Coordinator.Circuit.NLevels))
stateDB, 0, uint64(cfg.Coordinator.Circuit.NLevels))
if err != nil { if err != nil {
return nil, tracerr.Wrap(err) return nil, tracerr.Wrap(err)
} }

+ 247
- 0
test/zkproof/flows_test.go

@ -0,0 +1,247 @@
package zkproof
import (
"io/ioutil"
"os"
"strconv"
"testing"
"time"
ethCommon "github.com/ethereum/go-ethereum/common"
ethCrypto "github.com/ethereum/go-ethereum/crypto"
"github.com/hermeznetwork/hermez-node/batchbuilder"
"github.com/hermeznetwork/hermez-node/common"
dbUtils "github.com/hermeznetwork/hermez-node/db"
"github.com/hermeznetwork/hermez-node/db/historydb"
"github.com/hermeznetwork/hermez-node/db/l2db"
"github.com/hermeznetwork/hermez-node/db/statedb"
"github.com/hermeznetwork/hermez-node/log"
"github.com/hermeznetwork/hermez-node/test"
"github.com/hermeznetwork/hermez-node/test/til"
"github.com/hermeznetwork/hermez-node/test/txsets"
"github.com/hermeznetwork/hermez-node/txselector"
"github.com/jmoiron/sqlx"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
)
func addTokens(t *testing.T, tc *til.Context, db *sqlx.DB) {
var tokens []common.Token
for i := 0; i < int(tc.LastRegisteredTokenID); i++ {
tokens = append(tokens, common.Token{
TokenID: common.TokenID(i + 1),
EthBlockNum: 1,
EthAddr: ethCommon.BytesToAddress([]byte{byte(i + 1)}),
Name: strconv.Itoa(i),
Symbol: strconv.Itoa(i),
Decimals: 18,
})
}
hdb := historydb.NewHistoryDB(db)
assert.NoError(t, hdb.AddBlock(&common.Block{
Num: 1,
}))
assert.NoError(t, hdb.AddTokens(tokens))
}
func addL2Txs(t *testing.T, l2DB *l2db.L2DB, poolL2Txs []common.PoolL2Tx) {
for i := 0; i < len(poolL2Txs); i++ {
err := l2DB.AddTxTest(&poolL2Txs[i])
if err != nil {
log.Error(err)
}
require.NoError(t, err)
}
}
func addAccCreationAuth(t *testing.T, tc *til.Context, l2DB *l2db.L2DB, chainID uint16, hermezContractAddr ethCommon.Address, username string) []byte {
user := tc.Users[username]
auth := &common.AccountCreationAuth{
EthAddr: user.Addr,
BJJ: user.BJJ.Public().Compress(),
}
err := auth.Sign(func(hash []byte) ([]byte, error) {
return ethCrypto.Sign(hash, user.EthSk)
}, chainID, hermezContractAddr)
assert.NoError(t, err)
err = l2DB.AddAccountCreationAuth(auth)
assert.NoError(t, err)
return auth.Signature
}
func initTxSelector(t *testing.T, chainID uint16, hermezContractAddr ethCommon.Address, coordUser *til.User) (*txselector.TxSelector, *l2db.L2DB, *statedb.StateDB) {
pass := os.Getenv("POSTGRES_PASS")
db, err := dbUtils.InitSQLDB(5432, "localhost", "hermez", pass, "hermez")
require.NoError(t, err)
l2DB := l2db.NewL2DB(db, 10, 100, 24*time.Hour)
dir, err := ioutil.TempDir("", "tmpSyncDB")
require.NoError(t, err)
defer assert.NoError(t, os.RemoveAll(dir))
syncStateDB, err := statedb.NewStateDB(dir, 128, statedb.TypeSynchronizer, 0)
require.NoError(t, err)
txselDir, err := ioutil.TempDir("", "tmpTxSelDB")
require.NoError(t, err)
defer assert.NoError(t, os.RemoveAll(dir))
// use Til Coord keys for tests compatibility
coordAccount := &txselector.CoordAccount{
Addr: coordUser.Addr,
BJJ: coordUser.BJJ.Public().Compress(),
AccountCreationAuth: nil,
}
auth := common.AccountCreationAuth{
EthAddr: coordUser.Addr,
BJJ: coordUser.BJJ.Public().Compress(),
}
err = auth.Sign(func(hash []byte) ([]byte, error) {
return ethCrypto.Sign(hash, coordUser.EthSk)
}, chainID, hermezContractAddr)
assert.NoError(t, err)
coordAccount.AccountCreationAuth = auth.Signature
txsel, err := txselector.NewTxSelector(coordAccount, txselDir, syncStateDB, l2DB)
require.NoError(t, err)
test.WipeDB(l2DB.DB())
return txsel, l2DB, syncStateDB
}
func TestTxSelectorBatchBuilderZKInputs(t *testing.T) {
tc := til.NewContext(ChainID, common.RollupConstMaxL1UserTx)
// generate test transactions, the L1CoordinatorTxs generated by Til
// will be ignored at this test, as will be the TxSelector who
// generates them when needed
blocks, err := tc.GenerateBlocks(txsets.SetBlockchainMinimumFlow0)
require.NoError(t, err)
hermezContractAddr := ethCommon.HexToAddress("0xc344E203a046Da13b0B4467EB7B3629D0C99F6E6")
txsel, l2DBTxSel, syncStateDB := initTxSelector(t, ChainID, hermezContractAddr, tc.Users["Coord"])
bbDir, err := ioutil.TempDir("", "tmpBatchBuilderDB")
require.NoError(t, err)
bb, err := batchbuilder.NewBatchBuilder(bbDir, syncStateDB, 0, NLevels)
require.NoError(t, err)
// restart nonces of TilContext, as will be set by generating directly
// the PoolL2Txs for each specific batch with tc.GeneratePoolL2Txs
tc.RestartNonces()
// add tokens to HistoryDB to avoid breaking FK constrains
addTokens(t, tc, l2DBTxSel.DB())
selectionConfig := &txselector.SelectionConfig{
MaxL1UserTxs: 100, // TODO
TxProcessorConfig: txprocConfig,
}
configBatch := &batchbuilder.ConfigBatch{
// ForgerAddress:
TxProcessorConfig: txprocConfig,
}
// loop over the first 6 batches
expectedRoots := []string{"0", "0", "13644148972047617726265275926674266298636745191961029124811988256139761111521", "12433441613247342495680642890662773367605896324555599297255745922589338651261", "12433441613247342495680642890662773367605896324555599297255745922589338651261", "4191361650490017591061467288209836928064232431729236465872209988325272262963"}
for i := 0; i < 6; i++ {
log.Debugf("block:0 batch:%d", i+1)
var l1UserTxs []common.L1Tx
if blocks[0].Rollup.Batches[i].Batch.ForgeL1TxsNum != nil {
l1UserTxs = til.L1TxsToCommonL1Txs(tc.Queues[*blocks[0].Rollup.Batches[i].Batch.ForgeL1TxsNum])
}
// TxSelector select the transactions for the next Batch
coordIdxs, _, oL1UserTxs, oL1CoordTxs, oL2Txs, err := txsel.GetL1L2TxSelection(selectionConfig, l1UserTxs)
require.NoError(t, err)
// BatchBuilder build Batch
zki, err := bb.BuildBatch(coordIdxs, configBatch, oL1UserTxs, oL1CoordTxs, oL2Txs)
require.NoError(t, err)
assert.Equal(t, expectedRoots[i], bb.LocalStateDB().MT.Root().BigInt().String())
sendProofAndCheckResp(t, zki)
}
log.Debug("block:0 batch:7")
// simulate the PoolL2Txs of the batch6
batchPoolL2 := `
Type: PoolL2
PoolTransferToEthAddr(1) A-B: 200 (126)
PoolTransferToEthAddr(0) B-C: 100 (126)`
l2Txs, err := tc.GeneratePoolL2Txs(batchPoolL2)
require.NoError(t, err)
// add AccountCreationAuths that will be used at the next batch
_ = addAccCreationAuth(t, tc, l2DBTxSel, ChainID, hermezContractAddr, "B")
_ = addAccCreationAuth(t, tc, l2DBTxSel, ChainID, hermezContractAddr, "C")
addL2Txs(t, l2DBTxSel, l2Txs) // Add L2s to TxSelector.L2DB
l1UserTxs := til.L1TxsToCommonL1Txs(tc.Queues[*blocks[0].Rollup.Batches[6].Batch.ForgeL1TxsNum])
// TxSelector select the transactions for the next Batch
coordIdxs, _, oL1UserTxs, oL1CoordTxs, oL2Txs, err := txsel.GetL1L2TxSelection(selectionConfig, l1UserTxs)
require.NoError(t, err)
// BatchBuilder build Batch
zki, err := bb.BuildBatch(coordIdxs, configBatch, oL1UserTxs, oL1CoordTxs, oL2Txs)
require.NoError(t, err)
assert.Equal(t, "7614010373759339299470010949167613050707822522530721724565424494781010548240", bb.LocalStateDB().MT.Root().BigInt().String())
sendProofAndCheckResp(t, zki)
err = l2DBTxSel.StartForging(common.TxIDsFromPoolL2Txs(oL2Txs), txsel.LocalAccountsDB().CurrentBatch())
require.NoError(t, err)
log.Debug("block:0 batch:8")
// simulate the PoolL2Txs of the batch8
batchPoolL2 = `
Type: PoolL2
PoolTransfer(0) A-B: 100 (126)
PoolTransfer(0) C-A: 50 (126)
PoolTransfer(1) B-C: 100 (126)
PoolExit(0) A: 100 (126)`
l2Txs, err = tc.GeneratePoolL2Txs(batchPoolL2)
require.NoError(t, err)
addL2Txs(t, l2DBTxSel, l2Txs) // Add L2s to TxSelector.L2DB
l1UserTxs = til.L1TxsToCommonL1Txs(tc.Queues[*blocks[0].Rollup.Batches[7].Batch.ForgeL1TxsNum])
// TxSelector select the transactions for the next Batch
coordIdxs, _, oL1UserTxs, oL1CoordTxs, oL2Txs, err = txsel.GetL1L2TxSelection(selectionConfig, l1UserTxs)
require.NoError(t, err)
// BatchBuilder build Batch
zki, err = bb.BuildBatch(coordIdxs, configBatch, oL1UserTxs, oL1CoordTxs, oL2Txs)
require.NoError(t, err)
assert.Equal(t, "21231789250434471575486264439945776732824482207853465397552873521865656677689", bb.LocalStateDB().MT.Root().BigInt().String())
sendProofAndCheckResp(t, zki)
err = l2DBTxSel.StartForging(common.TxIDsFromPoolL2Txs(l2Txs), txsel.LocalAccountsDB().CurrentBatch())
require.NoError(t, err)
log.Debug("(batch9) block:1 batch:1")
// simulate the PoolL2Txs of the batch9
batchPoolL2 = `
Type: PoolL2
PoolTransfer(0) D-A: 300 (126)
PoolTransfer(0) B-D: 100 (126)`
l2Txs, err = tc.GeneratePoolL2Txs(batchPoolL2)
require.NoError(t, err)
addL2Txs(t, l2DBTxSel, l2Txs) // Add L2s to TxSelector.L2DB
l1UserTxs = til.L1TxsToCommonL1Txs(tc.Queues[*blocks[1].Rollup.Batches[0].Batch.ForgeL1TxsNum])
// TxSelector select the transactions for the next Batch
coordIdxs, _, oL1UserTxs, oL1CoordTxs, oL2Txs, err = txsel.GetL1L2TxSelection(selectionConfig, l1UserTxs)
require.NoError(t, err)
// BatchBuilder build Batch
zki, err = bb.BuildBatch(coordIdxs, configBatch, oL1UserTxs, oL1CoordTxs, oL2Txs)
require.NoError(t, err)
assert.Equal(t, "11289313644810782435120113035387729451095637380468777086895109386127538554246", bb.LocalStateDB().MT.Root().BigInt().String())
sendProofAndCheckResp(t, zki)
err = l2DBTxSel.StartForging(common.TxIDsFromPoolL2Txs(l2Txs), txsel.LocalAccountsDB().CurrentBatch())
require.NoError(t, err)
log.Debug("(batch10) block:1 batch:2")
l2Txs = []common.PoolL2Tx{}
l1UserTxs = til.L1TxsToCommonL1Txs(tc.Queues[*blocks[1].Rollup.Batches[1].Batch.ForgeL1TxsNum])
// TxSelector select the transactions for the next Batch
coordIdxs, _, oL1UserTxs, oL1CoordTxs, oL2Txs, err = txsel.GetL1L2TxSelection(selectionConfig, l1UserTxs)
require.NoError(t, err)
// BatchBuilder build Batch
zki, err = bb.BuildBatch(coordIdxs, configBatch, oL1UserTxs, oL1CoordTxs, oL2Txs)
require.NoError(t, err)
// same root as previous batch, as the L1CoordinatorTxs created by the
// Til set is not created by the TxSelector in this test
assert.Equal(t, "11289313644810782435120113035387729451095637380468777086895109386127538554246", bb.LocalStateDB().MT.Root().BigInt().String())
sendProofAndCheckResp(t, zki)
err = l2DBTxSel.StartForging(common.TxIDsFromPoolL2Txs(l2Txs), txsel.LocalAccountsDB().CurrentBatch())
require.NoError(t, err)
}

+ 49
- 45
test/zkproof/zkproof_test.go

@ -11,6 +11,7 @@ import (
"github.com/hermeznetwork/hermez-node/common" "github.com/hermeznetwork/hermez-node/common"
"github.com/hermeznetwork/hermez-node/db/statedb" "github.com/hermeznetwork/hermez-node/db/statedb"
"github.com/hermeznetwork/hermez-node/log"
"github.com/hermeznetwork/hermez-node/prover" "github.com/hermeznetwork/hermez-node/prover"
"github.com/hermeznetwork/hermez-node/test/til" "github.com/hermeznetwork/hermez-node/test/til"
"github.com/hermeznetwork/hermez-node/test/txsets" "github.com/hermeznetwork/hermez-node/test/txsets"
@ -26,9 +27,7 @@ const pollInterval = 200 * time.Millisecond
func TestMain(m *testing.M) { func TestMain(m *testing.M) {
exitVal := 0 exitVal := 0
proofServerURL = os.Getenv("PROOF_SERVER_URL") proofServerURL = os.Getenv("PROOF_SERVER_URL")
if proofServerURL != "" {
exitVal = m.Run()
}
exitVal = m.Run()
os.Exit(exitVal) os.Exit(exitVal)
} }
@ -38,7 +37,7 @@ const MaxL1Tx = 256
const MaxFeeTx = 64 const MaxFeeTx = 64
const ChainID uint16 = 1 const ChainID uint16 = 1
var config = txprocessor.Config{
var txprocConfig = txprocessor.Config{
NLevels: uint32(NLevels), NLevels: uint32(NLevels),
MaxTx: MaxTx, MaxTx: MaxTx,
MaxL1Tx: MaxL1Tx, MaxL1Tx: MaxL1Tx,
@ -46,19 +45,24 @@ var config = txprocessor.Config{
ChainID: ChainID, ChainID: ChainID,
} }
func initStateDB(t *testing.T) *statedb.StateDB {
func initStateDB(t *testing.T, typ statedb.TypeStateDB) *statedb.StateDB {
dir, err := ioutil.TempDir("", "tmpdb") dir, err := ioutil.TempDir("", "tmpdb")
require.NoError(t, err) require.NoError(t, err)
defer assert.Nil(t, os.RemoveAll(dir)) defer assert.Nil(t, os.RemoveAll(dir))
sdb, err := statedb.NewStateDB(dir, 128, statedb.TypeBatchBuilder, NLevels)
sdb, err := statedb.NewStateDB(dir, 128, typ, NLevels)
require.NoError(t, err) require.NoError(t, err)
return sdb return sdb
} }
func sendProofAndCheckResp(t *testing.T, ptOut *txprocessor.ProcessTxOutput) {
func sendProofAndCheckResp(t *testing.T, zki *common.ZKInputs) {
if proofServerURL == "" {
log.Debug("No PROOF_SERVER_URL defined, not using ProofServer")
return
}
// Store zkinputs json for debugging purposes // Store zkinputs json for debugging purposes
zkInputsJSON, err := json.Marshal(ptOut.ZKInputs)
zkInputsJSON, err := json.Marshal(zki)
require.NoError(t, err) require.NoError(t, err)
err = ioutil.WriteFile("/tmp/dbgZKInputs.json", zkInputsJSON, 0640) //nolint:gosec err = ioutil.WriteFile("/tmp/dbgZKInputs.json", zkInputsJSON, 0640) //nolint:gosec
require.NoError(t, err) require.NoError(t, err)
@ -66,7 +70,7 @@ func sendProofAndCheckResp(t *testing.T, ptOut *txprocessor.ProcessTxOutput) {
proofServerClient := prover.NewProofServerClient(proofServerURL, pollInterval) proofServerClient := prover.NewProofServerClient(proofServerURL, pollInterval)
err = proofServerClient.WaitReady(context.Background()) err = proofServerClient.WaitReady(context.Background())
require.NoError(t, err) require.NoError(t, err)
err = proofServerClient.CalculateProof(context.Background(), ptOut.ZKInputs)
err = proofServerClient.CalculateProof(context.Background(), zki)
require.NoError(t, err) require.NoError(t, err)
proof, pubInputs, err := proofServerClient.GetProof(context.Background()) proof, pubInputs, err := proofServerClient.GetProof(context.Background())
require.NoError(t, err) require.NoError(t, err)
@ -75,9 +79,9 @@ func sendProofAndCheckResp(t *testing.T, ptOut *txprocessor.ProcessTxOutput) {
} }
func TestZKInputsEmpty(t *testing.T) { func TestZKInputsEmpty(t *testing.T) {
sdb := initStateDB(t)
sdb := initStateDB(t, statedb.TypeBatchBuilder)
tp := txprocessor.NewTxProcessor(sdb, config)
tp := txprocessor.NewTxProcessor(sdb, txprocConfig)
coordIdxs := []common.Idx{} coordIdxs := []common.Idx{}
l1UserTxs := []common.L1Tx{} l1UserTxs := []common.L1Tx{}
@ -86,7 +90,7 @@ func TestZKInputsEmpty(t *testing.T) {
ptOut, err := tp.ProcessTxs(coordIdxs, l1UserTxs, l1CoordTxs, l2Txs) ptOut, err := tp.ProcessTxs(coordIdxs, l1UserTxs, l1CoordTxs, l2Txs)
require.NoError(t, err) require.NoError(t, err)
sendProofAndCheckResp(t, ptOut) // test empty batch ZKInputs
sendProofAndCheckResp(t, ptOut.ZKInputs) // test empty batch ZKInputs
_, coordIdxs, l1UserTxs, l1CoordTxs, l2Txs = txsets.GenerateTxsZKInputs0(t, ChainID) _, coordIdxs, l1UserTxs, l1CoordTxs, l2Txs = txsets.GenerateTxsZKInputs0(t, ChainID)
@ -99,79 +103,79 @@ func TestZKInputsEmpty(t *testing.T) {
l2Txs = []common.PoolL2Tx{} l2Txs = []common.PoolL2Tx{}
ptOut, err = tp.ProcessTxs(coordIdxs, l1UserTxs, l1CoordTxs, l2Txs) ptOut, err = tp.ProcessTxs(coordIdxs, l1UserTxs, l1CoordTxs, l2Txs)
require.NoError(t, err) require.NoError(t, err)
sendProofAndCheckResp(t, ptOut) // test empty batch ZKInputs after a non-empty batch
sendProofAndCheckResp(t, ptOut.ZKInputs) // test empty batch ZKInputs after a non-empty batch
} }
func TestZKInputs0(t *testing.T) { func TestZKInputs0(t *testing.T) {
sdb := initStateDB(t)
sdb := initStateDB(t, statedb.TypeBatchBuilder)
_, coordIdxs, l1UserTxs, l1CoordTxs, l2Txs := txsets.GenerateTxsZKInputs0(t, ChainID) _, coordIdxs, l1UserTxs, l1CoordTxs, l2Txs := txsets.GenerateTxsZKInputs0(t, ChainID)
tp := txprocessor.NewTxProcessor(sdb, config)
tp := txprocessor.NewTxProcessor(sdb, txprocConfig)
ptOut, err := tp.ProcessTxs(coordIdxs, l1UserTxs, l1CoordTxs, l2Txs) ptOut, err := tp.ProcessTxs(coordIdxs, l1UserTxs, l1CoordTxs, l2Txs)
require.NoError(t, err) require.NoError(t, err)
sendProofAndCheckResp(t, ptOut)
sendProofAndCheckResp(t, ptOut.ZKInputs)
} }
func TestZKInputs1(t *testing.T) { func TestZKInputs1(t *testing.T) {
sdb := initStateDB(t)
sdb := initStateDB(t, statedb.TypeBatchBuilder)
_, coordIdxs, l1UserTxs, l1CoordTxs, l2Txs := txsets.GenerateTxsZKInputs1(t, ChainID) _, coordIdxs, l1UserTxs, l1CoordTxs, l2Txs := txsets.GenerateTxsZKInputs1(t, ChainID)
tp := txprocessor.NewTxProcessor(sdb, config)
tp := txprocessor.NewTxProcessor(sdb, txprocConfig)
ptOut, err := tp.ProcessTxs(coordIdxs, l1UserTxs, l1CoordTxs, l2Txs) ptOut, err := tp.ProcessTxs(coordIdxs, l1UserTxs, l1CoordTxs, l2Txs)
require.NoError(t, err) require.NoError(t, err)
sendProofAndCheckResp(t, ptOut)
sendProofAndCheckResp(t, ptOut.ZKInputs)
} }
func TestZKInputs2(t *testing.T) { func TestZKInputs2(t *testing.T) {
sdb := initStateDB(t)
sdb := initStateDB(t, statedb.TypeBatchBuilder)
_, coordIdxs, l1UserTxs, l1CoordTxs, l2Txs := txsets.GenerateTxsZKInputs2(t, ChainID) _, coordIdxs, l1UserTxs, l1CoordTxs, l2Txs := txsets.GenerateTxsZKInputs2(t, ChainID)
tp := txprocessor.NewTxProcessor(sdb, config)
tp := txprocessor.NewTxProcessor(sdb, txprocConfig)
ptOut, err := tp.ProcessTxs(coordIdxs, l1UserTxs, l1CoordTxs, l2Txs) ptOut, err := tp.ProcessTxs(coordIdxs, l1UserTxs, l1CoordTxs, l2Txs)
require.NoError(t, err) require.NoError(t, err)
sendProofAndCheckResp(t, ptOut)
sendProofAndCheckResp(t, ptOut.ZKInputs)
} }
func TestZKInputs3(t *testing.T) { func TestZKInputs3(t *testing.T) {
sdb := initStateDB(t)
sdb := initStateDB(t, statedb.TypeBatchBuilder)
_, coordIdxs, l1UserTxs, l1CoordTxs, l2Txs := txsets.GenerateTxsZKInputs3(t, ChainID) _, coordIdxs, l1UserTxs, l1CoordTxs, l2Txs := txsets.GenerateTxsZKInputs3(t, ChainID)
tp := txprocessor.NewTxProcessor(sdb, config)
tp := txprocessor.NewTxProcessor(sdb, txprocConfig)
ptOut, err := tp.ProcessTxs(coordIdxs, l1UserTxs, l1CoordTxs, l2Txs) ptOut, err := tp.ProcessTxs(coordIdxs, l1UserTxs, l1CoordTxs, l2Txs)
require.NoError(t, err) require.NoError(t, err)
sendProofAndCheckResp(t, ptOut)
sendProofAndCheckResp(t, ptOut.ZKInputs)
} }
func TestZKInputs4(t *testing.T) { func TestZKInputs4(t *testing.T) {
sdb := initStateDB(t)
sdb := initStateDB(t, statedb.TypeBatchBuilder)
_, coordIdxs, l1UserTxs, l1CoordTxs, l2Txs := txsets.GenerateTxsZKInputs4(t, ChainID) _, coordIdxs, l1UserTxs, l1CoordTxs, l2Txs := txsets.GenerateTxsZKInputs4(t, ChainID)
tp := txprocessor.NewTxProcessor(sdb, config)
tp := txprocessor.NewTxProcessor(sdb, txprocConfig)
ptOut, err := tp.ProcessTxs(coordIdxs, l1UserTxs, l1CoordTxs, l2Txs) ptOut, err := tp.ProcessTxs(coordIdxs, l1UserTxs, l1CoordTxs, l2Txs)
require.NoError(t, err) require.NoError(t, err)
sendProofAndCheckResp(t, ptOut)
sendProofAndCheckResp(t, ptOut.ZKInputs)
} }
func TestZKInputs5(t *testing.T) { func TestZKInputs5(t *testing.T) {
sdb := initStateDB(t)
sdb := initStateDB(t, statedb.TypeBatchBuilder)
_, coordIdxs, l1UserTxs, l1CoordTxs, l2Txs := txsets.GenerateTxsZKInputs5(t, ChainID) _, coordIdxs, l1UserTxs, l1CoordTxs, l2Txs := txsets.GenerateTxsZKInputs5(t, ChainID)
tp := txprocessor.NewTxProcessor(sdb, config)
tp := txprocessor.NewTxProcessor(sdb, txprocConfig)
ptOut, err := tp.ProcessTxs(coordIdxs, l1UserTxs, l1CoordTxs, l2Txs) ptOut, err := tp.ProcessTxs(coordIdxs, l1UserTxs, l1CoordTxs, l2Txs)
require.NoError(t, err) require.NoError(t, err)
sendProofAndCheckResp(t, ptOut)
sendProofAndCheckResp(t, ptOut.ZKInputs)
} }
func TestZKInputs6(t *testing.T) { func TestZKInputs6(t *testing.T) {
sdb := initStateDB(t)
sdb := initStateDB(t, statedb.TypeBatchBuilder)
tc := til.NewContext(ChainID, common.RollupConstMaxL1UserTx) tc := til.NewContext(ChainID, common.RollupConstMaxL1UserTx)
blocks, err := tc.GenerateBlocks(txsets.SetBlockchainMinimumFlow0) blocks, err := tc.GenerateBlocks(txsets.SetBlockchainMinimumFlow0)
@ -181,12 +185,12 @@ func TestZKInputs6(t *testing.T) {
// the PoolL2Txs for each specific batch with tc.GeneratePoolL2Txs // the PoolL2Txs for each specific batch with tc.GeneratePoolL2Txs
tc.RestartNonces() tc.RestartNonces()
tp := txprocessor.NewTxProcessor(sdb, config)
tp := txprocessor.NewTxProcessor(sdb, txprocConfig)
// batch1 // batch1
ptOut, err := tp.ProcessTxs(nil, nil, blocks[0].Rollup.Batches[0].L1CoordinatorTxs, nil) ptOut, err := tp.ProcessTxs(nil, nil, blocks[0].Rollup.Batches[0].L1CoordinatorTxs, nil)
require.NoError(t, err) require.NoError(t, err)
sendProofAndCheckResp(t, ptOut)
sendProofAndCheckResp(t, ptOut.ZKInputs)
// batch2 // batch2
l1UserTxs := []common.L1Tx{} l1UserTxs := []common.L1Tx{}
@ -194,7 +198,7 @@ func TestZKInputs6(t *testing.T) {
ptOut, err = tp.ProcessTxs(nil, l1UserTxs, blocks[0].Rollup.Batches[1].L1CoordinatorTxs, l2Txs) ptOut, err = tp.ProcessTxs(nil, l1UserTxs, blocks[0].Rollup.Batches[1].L1CoordinatorTxs, l2Txs)
require.NoError(t, err) require.NoError(t, err)
sendProofAndCheckResp(t, ptOut)
sendProofAndCheckResp(t, ptOut.ZKInputs)
// batch3 // batch3
l1UserTxs = til.L1TxsToCommonL1Txs(tc.Queues[*blocks[0].Rollup.Batches[2].Batch.ForgeL1TxsNum]) l1UserTxs = til.L1TxsToCommonL1Txs(tc.Queues[*blocks[0].Rollup.Batches[2].Batch.ForgeL1TxsNum])
@ -202,7 +206,7 @@ func TestZKInputs6(t *testing.T) {
ptOut, err = tp.ProcessTxs(nil, l1UserTxs, blocks[0].Rollup.Batches[2].L1CoordinatorTxs, l2Txs) ptOut, err = tp.ProcessTxs(nil, l1UserTxs, blocks[0].Rollup.Batches[2].L1CoordinatorTxs, l2Txs)
require.NoError(t, err) require.NoError(t, err)
sendProofAndCheckResp(t, ptOut)
sendProofAndCheckResp(t, ptOut.ZKInputs)
// batch4 // batch4
l1UserTxs = til.L1TxsToCommonL1Txs(tc.Queues[*blocks[0].Rollup.Batches[3].Batch.ForgeL1TxsNum]) l1UserTxs = til.L1TxsToCommonL1Txs(tc.Queues[*blocks[0].Rollup.Batches[3].Batch.ForgeL1TxsNum])
@ -210,7 +214,7 @@ func TestZKInputs6(t *testing.T) {
ptOut, err = tp.ProcessTxs(nil, l1UserTxs, blocks[0].Rollup.Batches[3].L1CoordinatorTxs, l2Txs) ptOut, err = tp.ProcessTxs(nil, l1UserTxs, blocks[0].Rollup.Batches[3].L1CoordinatorTxs, l2Txs)
require.NoError(t, err) require.NoError(t, err)
sendProofAndCheckResp(t, ptOut)
sendProofAndCheckResp(t, ptOut.ZKInputs)
// batch5 // batch5
l1UserTxs = til.L1TxsToCommonL1Txs(tc.Queues[*blocks[0].Rollup.Batches[4].Batch.ForgeL1TxsNum]) l1UserTxs = til.L1TxsToCommonL1Txs(tc.Queues[*blocks[0].Rollup.Batches[4].Batch.ForgeL1TxsNum])
@ -218,7 +222,7 @@ func TestZKInputs6(t *testing.T) {
ptOut, err = tp.ProcessTxs(nil, l1UserTxs, blocks[0].Rollup.Batches[4].L1CoordinatorTxs, l2Txs) ptOut, err = tp.ProcessTxs(nil, l1UserTxs, blocks[0].Rollup.Batches[4].L1CoordinatorTxs, l2Txs)
require.NoError(t, err) require.NoError(t, err)
sendProofAndCheckResp(t, ptOut)
sendProofAndCheckResp(t, ptOut.ZKInputs)
// batch6 // batch6
l1UserTxs = til.L1TxsToCommonL1Txs(tc.Queues[*blocks[0].Rollup.Batches[5].Batch.ForgeL1TxsNum]) l1UserTxs = til.L1TxsToCommonL1Txs(tc.Queues[*blocks[0].Rollup.Batches[5].Batch.ForgeL1TxsNum])
@ -226,10 +230,10 @@ func TestZKInputs6(t *testing.T) {
ptOut, err = tp.ProcessTxs(nil, l1UserTxs, blocks[0].Rollup.Batches[5].L1CoordinatorTxs, l2Txs) ptOut, err = tp.ProcessTxs(nil, l1UserTxs, blocks[0].Rollup.Batches[5].L1CoordinatorTxs, l2Txs)
require.NoError(t, err) require.NoError(t, err)
sendProofAndCheckResp(t, ptOut)
sendProofAndCheckResp(t, ptOut.ZKInputs)
// batch7 // batch7
// simulate the PoolL2Txs of the batch6
// simulate the PoolL2Txs of the batch7
batchPoolL2 := ` batchPoolL2 := `
Type: PoolL2 Type: PoolL2
PoolTransferToEthAddr(1) A-B: 200 (126) PoolTransferToEthAddr(1) A-B: 200 (126)
@ -244,10 +248,10 @@ func TestZKInputs6(t *testing.T) {
ptOut, err = tp.ProcessTxs(coordIdxs, l1UserTxs, blocks[0].Rollup.Batches[6].L1CoordinatorTxs, l2Txs) ptOut, err = tp.ProcessTxs(coordIdxs, l1UserTxs, blocks[0].Rollup.Batches[6].L1CoordinatorTxs, l2Txs)
require.NoError(t, err) require.NoError(t, err)
sendProofAndCheckResp(t, ptOut)
sendProofAndCheckResp(t, ptOut.ZKInputs)
// batch8 // batch8
// simulate the PoolL2Txs of the batch7
// simulate the PoolL2Txs of the batch8
batchPoolL2 = ` batchPoolL2 = `
Type: PoolL2 Type: PoolL2
PoolTransfer(0) A-B: 100 (126) PoolTransfer(0) A-B: 100 (126)
@ -262,7 +266,7 @@ func TestZKInputs6(t *testing.T) {
ptOut, err = tp.ProcessTxs(coordIdxs, l1UserTxs, blocks[0].Rollup.Batches[7].L1CoordinatorTxs, l2Txs) ptOut, err = tp.ProcessTxs(coordIdxs, l1UserTxs, blocks[0].Rollup.Batches[7].L1CoordinatorTxs, l2Txs)
require.NoError(t, err) require.NoError(t, err)
sendProofAndCheckResp(t, ptOut)
sendProofAndCheckResp(t, ptOut.ZKInputs)
// batch9 // batch9
// simulate the PoolL2Txs of the batch9 // simulate the PoolL2Txs of the batch9
@ -279,7 +283,7 @@ func TestZKInputs6(t *testing.T) {
ptOut, err = tp.ProcessTxs(coordIdxs, l1UserTxs, blocks[1].Rollup.Batches[0].L1CoordinatorTxs, l2Txs) ptOut, err = tp.ProcessTxs(coordIdxs, l1UserTxs, blocks[1].Rollup.Batches[0].L1CoordinatorTxs, l2Txs)
require.NoError(t, err) require.NoError(t, err)
sendProofAndCheckResp(t, ptOut)
sendProofAndCheckResp(t, ptOut.ZKInputs)
// batch10 // batch10
l1UserTxs = til.L1TxsToCommonL1Txs(tc.Queues[*blocks[1].Rollup.Batches[1].Batch.ForgeL1TxsNum]) l1UserTxs = til.L1TxsToCommonL1Txs(tc.Queues[*blocks[1].Rollup.Batches[1].Batch.ForgeL1TxsNum])
@ -288,5 +292,5 @@ func TestZKInputs6(t *testing.T) {
ptOut, err = tp.ProcessTxs(coordIdxs, l1UserTxs, blocks[1].Rollup.Batches[1].L1CoordinatorTxs, l2Txs) ptOut, err = tp.ProcessTxs(coordIdxs, l1UserTxs, blocks[1].Rollup.Batches[1].L1CoordinatorTxs, l2Txs)
require.NoError(t, err) require.NoError(t, err)
sendProofAndCheckResp(t, ptOut)
sendProofAndCheckResp(t, ptOut.ZKInputs)
} }

+ 1
- 1
txprocessor/zkinputsgen_test.go

@ -674,7 +674,7 @@ func TestZKInputs6(t *testing.T) {
} }
tp := NewTxProcessor(sdb, config) tp := NewTxProcessor(sdb, config)
tc := til.NewContext(0, common.RollupConstMaxL1UserTx)
tc := til.NewContext(chainID, common.RollupConstMaxL1UserTx)
blocks, err := tc.GenerateBlocks(txsets.SetBlockchainMinimumFlow0) blocks, err := tc.GenerateBlocks(txsets.SetBlockchainMinimumFlow0)
require.NoError(t, err) require.NoError(t, err)

+ 40
- 34
txselector/txselector_test.go

@ -1,10 +1,8 @@
package txselector package txselector
import ( import (
"crypto/ecdsa"
"fmt" "fmt"
"io/ioutil" "io/ioutil"
"math/big"
"os" "os"
"strconv" "strconv"
"testing" "testing"
@ -22,13 +20,12 @@ import (
"github.com/hermeznetwork/hermez-node/test/til" "github.com/hermeznetwork/hermez-node/test/til"
"github.com/hermeznetwork/hermez-node/test/txsets" "github.com/hermeznetwork/hermez-node/test/txsets"
"github.com/hermeznetwork/hermez-node/txprocessor" "github.com/hermeznetwork/hermez-node/txprocessor"
"github.com/iden3/go-iden3-crypto/babyjub"
"github.com/jmoiron/sqlx" "github.com/jmoiron/sqlx"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
) )
func initTest(t *testing.T, chainID uint16, hermezContractAddr ethCommon.Address, testSet string) (*TxSelector, *til.Context) {
func initTest(t *testing.T, chainID uint16, hermezContractAddr ethCommon.Address, coordUser *til.User) *TxSelector {
pass := os.Getenv("POSTGRES_PASS") pass := os.Getenv("POSTGRES_PASS")
db, err := dbUtils.InitSQLDB(5432, "localhost", "hermez", pass, "hermez") db, err := dbUtils.InitSQLDB(5432, "localhost", "hermez", pass, "hermez")
require.NoError(t, err) require.NoError(t, err)
@ -37,45 +34,36 @@ func initTest(t *testing.T, chainID uint16, hermezContractAddr ethCommon.Address
dir, err := ioutil.TempDir("", "tmpdb") dir, err := ioutil.TempDir("", "tmpdb")
require.NoError(t, err) require.NoError(t, err)
defer assert.NoError(t, os.RemoveAll(dir)) defer assert.NoError(t, os.RemoveAll(dir))
sdb, err := statedb.NewStateDB(dir, 128, statedb.TypeTxSelector, 0)
syncStateDB, err := statedb.NewStateDB(dir, 128, statedb.TypeTxSelector, 0)
require.NoError(t, err) require.NoError(t, err)
txselDir, err := ioutil.TempDir("", "tmpTxSelDB") txselDir, err := ioutil.TempDir("", "tmpTxSelDB")
require.NoError(t, err) require.NoError(t, err)
defer assert.NoError(t, os.RemoveAll(dir)) defer assert.NoError(t, os.RemoveAll(dir))
// coordinator keys
var ethSk ecdsa.PrivateKey
ethSk.D = big.NewInt(int64(1)) // only for testing
ethSk.PublicKey.X, ethSk.PublicKey.Y = ethCrypto.S256().ScalarBaseMult(ethSk.D.Bytes())
ethSk.Curve = ethCrypto.S256()
addr := ethCrypto.PubkeyToAddress(ethSk.PublicKey)
var bjj babyjub.PublicKeyComp
err = bjj.UnmarshalText([]byte("c433f7a696b7aa3a5224efb3993baf0ccd9e92eecee0c29a3f6c8208a9e81d9e"))
require.NoError(t, err)
// use Til Coord keys for tests compatibility
coordAccount := &CoordAccount{ coordAccount := &CoordAccount{
Addr: addr,
BJJ: bjj,
Addr: coordUser.Addr,
BJJ: coordUser.BJJ.Public().Compress(),
AccountCreationAuth: nil, AccountCreationAuth: nil,
} }
fmt.Printf("%v", coordAccount)
auth := common.AccountCreationAuth{ auth := common.AccountCreationAuth{
EthAddr: addr,
BJJ: bjj,
EthAddr: coordUser.Addr,
BJJ: coordUser.BJJ.Public().Compress(),
} }
err = auth.Sign(func(hash []byte) ([]byte, error) { err = auth.Sign(func(hash []byte) ([]byte, error) {
return ethCrypto.Sign(hash, &ethSk)
return ethCrypto.Sign(hash, coordUser.EthSk)
}, chainID, hermezContractAddr) }, chainID, hermezContractAddr)
assert.NoError(t, err) assert.NoError(t, err)
coordAccount.AccountCreationAuth = auth.Signature coordAccount.AccountCreationAuth = auth.Signature
txsel, err := NewTxSelector(coordAccount, txselDir, sdb, l2DB)
txsel, err := NewTxSelector(coordAccount, txselDir, syncStateDB, l2DB)
require.NoError(t, err) require.NoError(t, err)
test.WipeDB(txsel.l2db.DB()) test.WipeDB(txsel.l2db.DB())
tc := til.NewContext(chainID, common.RollupConstMaxL1UserTx)
return txsel, tc
return txsel
} }
func addAccCreationAuth(t *testing.T, tc *til.Context, txsel *TxSelector, chainID uint16, hermezContractAddr ethCommon.Address, username string) []byte { func addAccCreationAuth(t *testing.T, tc *til.Context, txsel *TxSelector, chainID uint16, hermezContractAddr ethCommon.Address, username string) []byte {
@ -155,15 +143,16 @@ func checkSortedByNonce(t *testing.T, testAccNonces map[common.Idx]common.Nonce,
func TestGetL2TxSelectionMinimumFlow0(t *testing.T) { func TestGetL2TxSelectionMinimumFlow0(t *testing.T) {
chainID := uint16(0) chainID := uint16(0)
hermezContractAddr := ethCommon.HexToAddress("0xc344E203a046Da13b0B4467EB7B3629D0C99F6E6")
txsel, tc := initTest(t, chainID, hermezContractAddr, txsets.SetPool0)
tc := til.NewContext(chainID, common.RollupConstMaxL1UserTx)
// generate test transactions, the L1CoordinatorTxs generated by Til // generate test transactions, the L1CoordinatorTxs generated by Til
// will be ignored at this test, as will be the TxSelector who // will be ignored at this test, as will be the TxSelector who
// generates them when needed // generates them when needed
blocks, err := tc.GenerateBlocks(txsets.SetBlockchainMinimumFlow0) blocks, err := tc.GenerateBlocks(txsets.SetBlockchainMinimumFlow0)
assert.NoError(t, err) assert.NoError(t, err)
hermezContractAddr := ethCommon.HexToAddress("0xc344E203a046Da13b0B4467EB7B3629D0C99F6E6")
txsel := initTest(t, chainID, hermezContractAddr, tc.Users["Coord"])
// restart nonces of TilContext, as will be set by generating directly // restart nonces of TilContext, as will be set by generating directly
// the PoolL2Txs for each specific batch with tc.GeneratePoolL2Txs // the PoolL2Txs for each specific batch with tc.GeneratePoolL2Txs
tc.RestartNonces() tc.RestartNonces()
@ -262,21 +251,22 @@ func TestGetL2TxSelectionMinimumFlow0(t *testing.T) {
// simulate the PoolL2Txs of the batch7 // simulate the PoolL2Txs of the batch7
batchPoolL2 := ` batchPoolL2 := `
Type: PoolL2 Type: PoolL2
PoolTransfer(1) A-B: 200 (126)
PoolTransfer(0) B-C: 100 (126)`
PoolTransferToEthAddr(1) A-B: 200 (126)
PoolTransferToEthAddr(0) B-C: 100 (126)`
poolL2Txs, err := tc.GeneratePoolL2Txs(batchPoolL2) poolL2Txs, err := tc.GeneratePoolL2Txs(batchPoolL2)
require.NoError(t, err) require.NoError(t, err)
// add AccountCreationAuths that will be used at the next batch // add AccountCreationAuths that will be used at the next batch
accAuthSig0 := addAccCreationAuth(t, tc, txsel, chainID, hermezContractAddr, "B") accAuthSig0 := addAccCreationAuth(t, tc, txsel, chainID, hermezContractAddr, "B")
accAuthSig1 := addAccCreationAuth(t, tc, txsel, chainID, hermezContractAddr, "C") accAuthSig1 := addAccCreationAuth(t, tc, txsel, chainID, hermezContractAddr, "C")
// add ToEthAddr for the corresponent ToIdx, and remove ToIdx for Batches[6].L2Tx
poolL2Txs[0].ToEthAddr = tc.Users["B"].Addr
poolL2Txs[0].ToIdx = common.Idx(0)
poolL2Txs[1].ToEthAddr = tc.Users["C"].Addr
poolL2Txs[1].ToIdx = common.Idx(0)
// add the PoolL2Txs to the l2DB // add the PoolL2Txs to the l2DB
addL2Txs(t, txsel, poolL2Txs) addL2Txs(t, txsel, poolL2Txs)
// check signatures of L2Txs from the L2DB (to check that the
// parameters of the PoolL2Tx match the original parameters signed
// before inserting it to the L2DB)
l2TxsFromDB, err := txsel.l2db.GetPendingTxs()
require.NoError(t, err)
assert.True(t, l2TxsFromDB[0].VerifySignature(chainID, tc.Users["A"].BJJ.Public().Compress()))
assert.True(t, l2TxsFromDB[1].VerifySignature(chainID, tc.Users["B"].BJJ.Public().Compress()))
l1UserTxs = til.L1TxsToCommonL1Txs(tc.Queues[*blocks[0].Rollup.Batches[6].Batch.ForgeL1TxsNum]) l1UserTxs = til.L1TxsToCommonL1Txs(tc.Queues[*blocks[0].Rollup.Batches[6].Batch.ForgeL1TxsNum])
coordIdxs, accAuths, oL1UserTxs, oL1CoordTxs, oL2Txs, err := txsel.GetL1L2TxSelection(selectionConfig, l1UserTxs) coordIdxs, accAuths, oL1UserTxs, oL1CoordTxs, oL2Txs, err := txsel.GetL1L2TxSelection(selectionConfig, l1UserTxs)
require.NoError(t, err) require.NoError(t, err)
@ -314,6 +304,15 @@ func TestGetL2TxSelectionMinimumFlow0(t *testing.T) {
poolL2Txs, err = tc.GeneratePoolL2Txs(batchPoolL2) poolL2Txs, err = tc.GeneratePoolL2Txs(batchPoolL2)
require.NoError(t, err) require.NoError(t, err)
addL2Txs(t, txsel, poolL2Txs) addL2Txs(t, txsel, poolL2Txs)
// check signatures of L2Txs from the L2DB (to check that the
// parameters of the PoolL2Tx match the original parameters signed
// before inserting it to the L2DB)
l2TxsFromDB, err = txsel.l2db.GetPendingTxs()
require.NoError(t, err)
assert.True(t, l2TxsFromDB[0].VerifySignature(chainID, tc.Users["A"].BJJ.Public().Compress()))
assert.True(t, l2TxsFromDB[1].VerifySignature(chainID, tc.Users["C"].BJJ.Public().Compress()))
assert.True(t, l2TxsFromDB[2].VerifySignature(chainID, tc.Users["B"].BJJ.Public().Compress()))
assert.True(t, l2TxsFromDB[3].VerifySignature(chainID, tc.Users["A"].BJJ.Public().Compress()))
l1UserTxs = til.L1TxsToCommonL1Txs(tc.Queues[*blocks[0].Rollup.Batches[7].Batch.ForgeL1TxsNum]) l1UserTxs = til.L1TxsToCommonL1Txs(tc.Queues[*blocks[0].Rollup.Batches[7].Batch.ForgeL1TxsNum])
coordIdxs, accAuths, oL1UserTxs, oL1CoordTxs, oL2Txs, err = txsel.GetL1L2TxSelection(selectionConfig, l1UserTxs) coordIdxs, accAuths, oL1UserTxs, oL1CoordTxs, oL2Txs, err = txsel.GetL1L2TxSelection(selectionConfig, l1UserTxs)
require.NoError(t, err) require.NoError(t, err)
@ -348,6 +347,13 @@ func TestGetL2TxSelectionMinimumFlow0(t *testing.T) {
poolL2Txs, err = tc.GeneratePoolL2Txs(batchPoolL2) poolL2Txs, err = tc.GeneratePoolL2Txs(batchPoolL2)
require.NoError(t, err) require.NoError(t, err)
addL2Txs(t, txsel, poolL2Txs) addL2Txs(t, txsel, poolL2Txs)
// check signatures of L2Txs from the L2DB (to check that the
// parameters of the PoolL2Tx match the original parameters signed
// before inserting it to the L2DB)
l2TxsFromDB, err = txsel.l2db.GetPendingTxs()
require.NoError(t, err)
assert.True(t, l2TxsFromDB[0].VerifySignature(chainID, tc.Users["D"].BJJ.Public().Compress()))
assert.True(t, l2TxsFromDB[1].VerifySignature(chainID, tc.Users["B"].BJJ.Public().Compress()))
l1UserTxs = til.L1TxsToCommonL1Txs(tc.Queues[*blocks[1].Rollup.Batches[0].Batch.ForgeL1TxsNum]) l1UserTxs = til.L1TxsToCommonL1Txs(tc.Queues[*blocks[1].Rollup.Batches[0].Batch.ForgeL1TxsNum])
coordIdxs, accAuths, oL1UserTxs, oL1CoordTxs, oL2Txs, err = txsel.GetL1L2TxSelection(selectionConfig, l1UserTxs) coordIdxs, accAuths, oL1UserTxs, oL1CoordTxs, oL2Txs, err = txsel.GetL1L2TxSelection(selectionConfig, l1UserTxs)
require.NoError(t, err) require.NoError(t, err)

Loading…
Cancel
Save