From 6a990376b406f00a6b23565279bb23b0668d084c Mon Sep 17 00:00:00 2001 From: Eduard S Date: Tue, 22 Dec 2020 16:50:58 +0100 Subject: [PATCH 1/3] Implement Pipeline.prepareForgeBatchArgs() - Implement Pipeline.prepareForgeBatchArgs() - Add a minimal stress test for the coordinator (that also runs the synchronizer) - Update txprocessor.ProcessTxs() to return valid results for batches without transactions - Add the boilerplate for the corresponding test, leaving as TODO the zkInput values - Update prover client to use the same point format as proof server (projective) - Update interface of TxSelector.GetCoordIdxs to also return the authorizations to create accounts that go with the l1CoordinatorTxs. --- coordinator/batch.go | 21 ++--- coordinator/coordinator.go | 50 ++++++++++-- coordinator/coordinator_test.go | 77 +++++++++++++++++- db/statedb/txprocessors.go | 16 ++-- db/statedb/zkinputsgen_test.go | 137 ++++++++++++++++++++++++++++++++ prover/prover.go | 23 ++++-- prover/prover_test.go | 7 +- test/proofserver/proofserver.go | 52 +++++++++++- txselector/txselector.go | 23 +++--- txselector/txselector_test.go | 4 +- 10 files changed, 361 insertions(+), 49 deletions(-) diff --git a/coordinator/batch.go b/coordinator/batch.go index ce94b4f..2bba88a 100644 --- a/coordinator/batch.go +++ b/coordinator/batch.go @@ -26,15 +26,18 @@ const ( // BatchInfo contans the Batch information type BatchInfo struct { - BatchNum common.BatchNum - ServerProof prover.Client - ZKInputs *common.ZKInputs - Proof *prover.Proof - PublicInputs []*big.Int - L1UserTxsExtra []common.L1Tx - L1CoordTxs []common.L1Tx - L2Txs []common.PoolL2Tx - ForgeBatchArgs *eth.RollupForgeBatchArgs + BatchNum common.BatchNum + ServerProof prover.Client + ZKInputs *common.ZKInputs + Proof *prover.Proof + PublicInputs []*big.Int + L1Batch bool + L1UserTxsExtra []common.L1Tx + L1CoordTxs []common.L1Tx + L1CoordinatorTxsAuths [][]byte + L2Txs []common.L2Tx + CoordIdxs []common.Idx + ForgeBatchArgs *eth.RollupForgeBatchArgs // FeesInfo TxStatus TxStatus EthTx *types.Transaction diff --git a/coordinator/coordinator.go b/coordinator/coordinator.go index 52852fd..897ab72 100644 --- a/coordinator/coordinator.go +++ b/coordinator/coordinator.go @@ -3,6 +3,7 @@ package coordinator import ( "context" "fmt" + "math/big" "strings" "sync" "time" @@ -772,7 +773,15 @@ func (p *Pipeline) Stop(ctx context.Context) { } } -func l2TxsIDs(txs []common.PoolL2Tx) []common.TxID { +func poolL2TxsIDs(txs []common.PoolL2Tx) []common.TxID { + txIDs := make([]common.TxID, len(txs)) + for i, tx := range txs { + txIDs[i] = tx.TxID + } + return txIDs +} + +func l2TxsIDs(txs []common.L2Tx) []common.TxID { txIDs := make([]common.TxID, len(txs)) for i, tx := range txs { txIDs[i] = tx.TxID @@ -810,9 +819,11 @@ func (p *Pipeline) forgeBatch(ctx context.Context, batchNum common.BatchNum, sel var poolL2Txs []common.PoolL2Tx // var feesInfo var l1UserTxsExtra, l1CoordTxs []common.L1Tx + var auths [][]byte var coordIdxs []common.Idx // 1. Decide if we forge L2Tx or L1+L2Tx if p.shouldL1L2Batch() { + batchInfo.L1Batch = true p.lastScheduledL1BatchBlockNum = p.stats.Eth.LastBlock.Num // 2a: L1+L2 txs p.lastForgeL1TxsNum++ @@ -821,14 +832,14 @@ func (p *Pipeline) forgeBatch(ctx context.Context, batchNum common.BatchNum, sel return nil, tracerr.Wrap(err) } // TODO once feesInfo is added to method return, add the var - coordIdxs, l1UserTxsExtra, l1CoordTxs, poolL2Txs, err = + coordIdxs, auths, l1UserTxsExtra, l1CoordTxs, poolL2Txs, err = p.txSelector.GetL1L2TxSelection(selectionConfig, batchNum, l1UserTxs) if err != nil { return nil, tracerr.Wrap(err) } } else { // 2b: only L2 txs - coordIdxs, l1CoordTxs, poolL2Txs, err = + coordIdxs, auths, l1CoordTxs, poolL2Txs, err = p.txSelector.GetL2TxSelection(selectionConfig, batchNum) if err != nil { return nil, tracerr.Wrap(err) @@ -840,9 +851,10 @@ func (p *Pipeline) forgeBatch(ctx context.Context, batchNum common.BatchNum, sel // TODO feesInfo batchInfo.L1UserTxsExtra = l1UserTxsExtra batchInfo.L1CoordTxs = l1CoordTxs - batchInfo.L2Txs = poolL2Txs + batchInfo.L1CoordinatorTxsAuths = auths + batchInfo.CoordIdxs = coordIdxs - if err := p.l2DB.StartForging(l2TxsIDs(batchInfo.L2Txs), batchInfo.BatchNum); err != nil { + if err := p.l2DB.StartForging(poolL2TxsIDs(poolL2Txs), batchInfo.BatchNum); err != nil { return nil, tracerr.Wrap(err) } @@ -864,6 +876,11 @@ func (p *Pipeline) forgeBatch(ctx context.Context, batchNum common.BatchNum, sel if err != nil { return nil, tracerr.Wrap(err) } + l2Txs, err := common.PoolL2TxsToL2Txs(poolL2Txs) // NOTE: This is a big uggly, find a better way + if err != nil { + return nil, tracerr.Wrap(err) + } + batchInfo.L2Txs = l2Txs // 5. Save metadata from BatchBuilder output for BatchNum batchInfo.ZKInputs = zkInputs @@ -903,6 +920,25 @@ func (p *Pipeline) shouldL1L2Batch() bool { } func (p *Pipeline) prepareForgeBatchArgs(batchInfo *BatchInfo) *eth.RollupForgeBatchArgs { - // TODO - return ð.RollupForgeBatchArgs{} + proof := batchInfo.Proof + zki := batchInfo.ZKInputs + return ð.RollupForgeBatchArgs{ + NewLastIdx: int64(zki.Metadata.NewLastIdxRaw), + NewStRoot: zki.Metadata.NewStateRootRaw.BigInt(), + NewExitRoot: zki.Metadata.NewExitRootRaw.BigInt(), + L1UserTxs: batchInfo.L1UserTxsExtra, + L1CoordinatorTxs: batchInfo.L1CoordTxs, + L1CoordinatorTxsAuths: batchInfo.L1CoordinatorTxsAuths, + L2TxsData: batchInfo.L2Txs, + FeeIdxCoordinator: batchInfo.CoordIdxs, + // Circuit selector + VerifierIdx: 0, // TODO + L1Batch: batchInfo.L1Batch, + ProofA: [2]*big.Int{proof.PiA[0], proof.PiA[1]}, + ProofB: [2][2]*big.Int{ + {proof.PiB[0][0], proof.PiB[0][1]}, + {proof.PiB[1][0], proof.PiB[1][1]}, + }, + ProofC: [2]*big.Int{proof.PiC[0], proof.PiC[1]}, + } } diff --git a/coordinator/coordinator_test.go b/coordinator/coordinator_test.go index 3207ae0..1060236 100644 --- a/coordinator/coordinator_test.go +++ b/coordinator/coordinator_test.go @@ -6,6 +6,7 @@ import ( "io/ioutil" "math/big" "os" + "sync" "testing" "time" @@ -157,11 +158,21 @@ func newTestCoordinator(t *testing.T, forgerAddr ethCommon.Address, ethClient *t ConfirmBlocks: 5, L1BatchTimeoutPerc: 0.5, EthClientAttempts: 5, + SyncRetryInterval: 400 * time.Microsecond, EthClientAttemptsDelay: 100 * time.Millisecond, TxManagerCheckInterval: 300 * time.Millisecond, DebugBatchPath: debugBatchPath, + Purger: PurgerCfg{ + PurgeBatchDelay: 10, + PurgeBlockDelay: 10, + InvalidateBatchDelay: 4, + InvalidateBlockDelay: 4, + }, + } + serverProofs := []prover.Client{ + &prover.MockClient{Delay: 300 * time.Millisecond}, + &prover.MockClient{Delay: 400 * time.Millisecond}, } - serverProofs := []prover.Client{&prover.MockClient{}, &prover.MockClient{}} scConsts := &synchronizer.SCConsts{ Rollup: *ethClientSetup.RollupConstants, @@ -628,6 +639,70 @@ PoolTransfer(0) User2-User3: 300 (126) assert.Equal(t, 0, len(batchInfo.L2Txs)) } +func TestCoordinatorStress(t *testing.T) { + if os.Getenv("TEST_COORD_STRESS") == "" { + return + } + log.Info("Begin Test Coord Stress") + ethClientSetup := test.NewClientSetupExample() + var timer timer + ethClient := test.NewClient(true, &timer, &bidder, ethClientSetup) + modules := newTestModules(t) + coord := newTestCoordinator(t, forger, ethClient, ethClientSetup, modules) + syn := newTestSynchronizer(t, ethClient, ethClientSetup, modules) + + coord.Start() + ctx, cancel := context.WithCancel(context.Background()) + var wg sync.WaitGroup + + // Synchronizer loop + wg.Add(1) + go func() { + for { + blockData, _, err := syn.Sync2(ctx, nil) + if ctx.Err() != nil { + wg.Done() + return + } + require.NoError(t, err) + if blockData != nil { + stats := syn.Stats() + coord.SendMsg(MsgSyncBlock{ + Stats: *stats, + Batches: blockData.Rollup.Batches, + Vars: synchronizer.SCVariablesPtr{ + Rollup: blockData.Rollup.Vars, + Auction: blockData.Auction.Vars, + WDelayer: blockData.WDelayer.Vars, + }, + }) + } else { + time.Sleep(100 * time.Millisecond) + } + } + }() + + // Blockchain mining loop + wg.Add(1) + go func() { + for { + select { + case <-ctx.Done(): + wg.Done() + return + case <-time.After(100 * time.Millisecond): + ethClient.CtlMineBlock() + } + } + }() + + time.Sleep(600 * time.Second) + + cancel() + wg.Wait() + coord.Stop() +} + // TODO: Test Reorg // TODO: Test Pipeline // TODO: Test TxMonitor diff --git a/db/statedb/txprocessors.go b/db/statedb/txprocessors.go index 431142b..0547fe0 100644 --- a/db/statedb/txprocessors.go +++ b/db/statedb/txprocessors.go @@ -82,16 +82,6 @@ func (s *StateDB) ProcessTxs(ptc ProcessTxsConfig, coordIdxs []common.Idx, l1use s.accumulatedFees = make(map[common.Idx]*big.Int) nTx := len(l1usertxs) + len(l1coordinatortxs) + len(l2txs) - if nTx == 0 { - // TODO return ZKInputs of batch without txs - return &ProcessTxOutput{ - ZKInputs: nil, - ExitInfos: nil, - CreatedAccounts: nil, - CoordinatorIdxsMap: nil, - CollectedFees: nil, - }, nil - } if nTx > int(ptc.MaxTx) { return nil, tracerr.Wrap(fmt.Errorf("L1UserTx + L1CoordinatorTx + L2Tx (%d) can not be bigger than MaxTx (%d)", nTx, ptc.MaxTx)) @@ -272,7 +262,11 @@ func (s *StateDB) ProcessTxs(ptc ProcessTxsConfig, coordIdxs []common.Idx, l1use } if s.zki != nil { - for i := s.i - 1; i < int(ptc.MaxTx); i++ { + last := s.i - 1 + if s.i == 0 { + last = 0 + } + for i := last; i < int(ptc.MaxTx); i++ { if i < int(ptc.MaxTx)-1 { s.zki.ISOutIdx[i] = s.idx.BigInt() s.zki.ISStateRoot[i] = s.mt.Root().BigInt() diff --git a/db/statedb/zkinputsgen_test.go b/db/statedb/zkinputsgen_test.go index 3fccd89..48fa7b8 100644 --- a/db/statedb/zkinputsgen_test.go +++ b/db/statedb/zkinputsgen_test.go @@ -240,6 +240,143 @@ func TestZKInputsHashTestVector1(t *testing.T) { assert.Equal(t, "20293112365009290386650039345314592436395562810005523677125576447132206192598", h.String()) } +// TestZKInputsEmpty: +// tests: +// - L1: empty +// - L2: empty +func TestZKInputsEmpty(t *testing.T) { + dir, err := ioutil.TempDir("", "tmpdb") + require.Nil(t, err) + defer assert.Nil(t, os.RemoveAll(dir)) + + nLevels := 16 + + sdb, err := NewStateDB(dir, TypeBatchBuilder, nLevels) + assert.Nil(t, err) + + ptc := ProcessTxsConfig{ + NLevels: uint32(nLevels), + MaxTx: 10, + MaxL1Tx: 5, + MaxFeeTx: 2, + } + + // 0. Generate a batch from the empty state with no transactions + + coordIdxs := []common.Idx{} + l1UserTxs := []common.L1Tx{} + l1CoordTxs := []common.L1Tx{} + l2Txs := []common.PoolL2Tx{} + + ptOut, err := sdb.ProcessTxs(ptc, coordIdxs, l1UserTxs, l1CoordTxs, l2Txs) + require.Nil(t, err) + + assert.Equal(t, "0", sdb.mt.Root().BigInt().String()) + assert.Equal(t, "0", ptOut.ZKInputs.Metadata.NewExitRootRaw.BigInt().String()) + + // check that there are no accounts + accs, err := sdb.GetAccounts() + require.NoError(t, err) + assert.Equal(t, 0, len(accs)) + + /* // TODO + h, err := ptOut.ZKInputs.HashGlobalData() + require.Nil(t, err) + assert.Equal(t, "TODO", h.String()) + + toHash, err := ptOut.ZKInputs.ToHashGlobalData() + require.Nil(t, err) + assert.Equal(t, "TODO", hex.EncodeToString(toHash)) + + s, err := json.Marshal(ptOut.ZKInputs) + require.Nil(t, err) + // the 'expected' data has been checked with the circom circuits + expected := `TODO` + assert.Equal(t, expected, string(s)) + */ + + // 1. Generate a batch with two transactions that create one account + // so that the state tree is not empty (same transactions as + // TestZKInputs0) + + // same values than in the js test + users := generateJsUsers(t) + + l1UserTxs = []common.L1Tx{ + { + FromIdx: 0, + DepositAmount: big.NewInt(16000000), + Amount: big.NewInt(0), + TokenID: 1, + FromBJJ: users[0].BJJ.Public().Compress(), + FromEthAddr: users[0].Addr, + ToIdx: 0, + Type: common.TxTypeCreateAccountDeposit, + UserOrigin: true, + }, + } + l2Txs = []common.PoolL2Tx{ + { + FromIdx: 256, + ToIdx: 256, + TokenID: 1, + Amount: big.NewInt(1000), + Nonce: 0, + Fee: 0, + Type: common.TxTypeTransfer, + }, + } + + toSign, err := l2Txs[0].HashToSign() + require.Nil(t, err) + sig := users[0].BJJ.SignPoseidon(toSign) + l2Txs[0].Signature = sig.Compress() + + _, err = sdb.ProcessTxs(ptc, nil, l1UserTxs, nil, l2Txs) + require.Nil(t, err) + + rootNonZero := sdb.mt.Root() + + // check that there is 1 account + accs, err = sdb.GetAccounts() + require.NoError(t, err) + assert.Equal(t, 1, len(accs)) + + // 2. Generate a batch from a non-empty state with no transactions + + coordIdxs = []common.Idx{} + l1UserTxs = []common.L1Tx{} + l1CoordTxs = []common.L1Tx{} + l2Txs = []common.PoolL2Tx{} + + ptOut, err = sdb.ProcessTxs(ptc, coordIdxs, l1UserTxs, l1CoordTxs, l2Txs) + require.Nil(t, err) + + assert.Equal(t, rootNonZero, sdb.mt.Root()) + assert.Equal(t, "0", ptOut.ZKInputs.Metadata.NewExitRootRaw.BigInt().String()) + + // check that there is still 1 account + accs, err = sdb.GetAccounts() + require.NoError(t, err) + assert.Equal(t, 1, len(accs)) + + /* // TODO + h, err := ptOut.ZKInputs.HashGlobalData() + require.Nil(t, err) + assert.Equal(t, "TODO", h.String()) + + toHash, err := ptOut.ZKInputs.ToHashGlobalData() + require.Nil(t, err) + assert.Equal(t, "TODO", hex.EncodeToString(toHash)) + + s, err := json.Marshal(ptOut.ZKInputs) + require.Nil(t, err) + // the 'expected' data has been checked with the circom circuits + expected := `TODO` + assert.Equal(t, expected, string(s)) + */ +} + // TestZKInputs0: // tests: // - L1: CreateAccountDeposit diff --git a/prover/prover.go b/prover/prover.go index a20f0af..effa429 100644 --- a/prover/prover.go +++ b/prover/prover.go @@ -16,9 +16,9 @@ import ( // Proof TBD this type will be received from the proof server type Proof struct { - PiA [2]*big.Int `json:"pi_a"` + PiA [3]*big.Int `json:"pi_a"` PiB [3][2]*big.Int `json:"pi_b"` - PiC [2]*big.Int `json:"pi_c"` + PiC [3]*big.Int `json:"pi_c"` Protocol string `json:"protocol"` } @@ -36,9 +36,9 @@ func (b *bigInt) UnmarshalText(text []byte) error { // ints as strings func (p *Proof) UnmarshalJSON(data []byte) error { proof := struct { - PiA [2]*bigInt `json:"pi_a"` + PiA [3]*bigInt `json:"pi_a"` PiB [3][2]*bigInt `json:"pi_b"` - PiC [2]*bigInt `json:"pi_c"` + PiC [3]*bigInt `json:"pi_c"` Protocol string `json:"protocol"` }{} if err := json.Unmarshal(data, &proof); err != nil { @@ -46,14 +46,26 @@ func (p *Proof) UnmarshalJSON(data []byte) error { } p.PiA[0] = (*big.Int)(proof.PiA[0]) p.PiA[1] = (*big.Int)(proof.PiA[1]) + p.PiA[2] = (*big.Int)(proof.PiA[2]) + if p.PiA[2].Int64() != 1 { + return fmt.Errorf("Expected PiA[2] == 1, but got %v", p.PiA[2]) + } p.PiB[0][0] = (*big.Int)(proof.PiB[0][0]) p.PiB[0][1] = (*big.Int)(proof.PiB[0][1]) p.PiB[1][0] = (*big.Int)(proof.PiB[1][0]) p.PiB[1][1] = (*big.Int)(proof.PiB[1][1]) p.PiB[2][0] = (*big.Int)(proof.PiB[2][0]) p.PiB[2][1] = (*big.Int)(proof.PiB[2][1]) + if p.PiB[2][0].Int64() != 1 || p.PiB[2][1].Int64() != 0 { + return fmt.Errorf("Expected PiB[2] == [1, 0], but got %v", p.PiB[2]) + } p.PiC[0] = (*big.Int)(proof.PiC[0]) p.PiC[1] = (*big.Int)(proof.PiC[1]) + p.PiC[2] = (*big.Int)(proof.PiC[2]) + if p.PiC[2].Int64() != 1 { + return fmt.Errorf("Expected PiC[2] == 1, but got %v", p.PiC[2]) + } + // TODO: Assert ones and zeroes p.Protocol = proof.Protocol return nil } @@ -276,6 +288,7 @@ func (p *ProofServerClient) WaitReady(ctx context.Context) error { // MockClient is a mock ServerProof to be used in tests. It doesn't calculate anything type MockClient struct { + Delay time.Duration } // CalculateProof sends the *common.ZKInputs to the ServerProof to compute the @@ -288,7 +301,7 @@ func (p *MockClient) CalculateProof(ctx context.Context, zkInputs *common.ZKInpu func (p *MockClient) GetProof(ctx context.Context) (*Proof, []*big.Int, error) { // Simulate a delay select { - case <-time.After(500 * time.Millisecond): //nolint:gomnd + case <-time.After(p.Delay): //nolint:gomnd return &Proof{}, []*big.Int{big.NewInt(1234)}, nil //nolint:gomnd case <-ctx.Done(): return nil, nil, tracerr.Wrap(common.ErrDone) diff --git a/prover/prover_test.go b/prover/prover_test.go index 99e7ba9..e36ba85 100644 --- a/prover/prover_test.go +++ b/prover/prover_test.go @@ -12,7 +12,8 @@ import ( "github.com/stretchr/testify/require" ) -const apiURL = "http://localhost:3000/api" +var apiURL = "http://localhost:3000/api" + const pollInterval = 1 * time.Second var proofServerClient *ProofServerClient @@ -20,6 +21,10 @@ var proofServerClient *ProofServerClient func TestMain(m *testing.M) { exitVal := 0 if os.Getenv("INTEGRATION") != "" { + _apiURL := os.Getenv("PROOF_SERVER_URL") + if _apiURL != "" { + apiURL = _apiURL + } proofServerClient = NewProofServerClient(apiURL, pollInterval) err := proofServerClient.WaitReady(context.Background()) if err != nil { diff --git a/test/proofserver/proofserver.go b/test/proofserver/proofserver.go index 2586049..0e81439 100644 --- a/test/proofserver/proofserver.go +++ b/test/proofserver/proofserver.go @@ -67,6 +67,50 @@ func (s *Mock) handleCancel(c *gin.Context) { c.JSON(http.StatusOK, "OK") } +/* Status example from the real server proof: + +Status: +{ + "proof": "{\n \"pi_a\": [\n \"1368015179489954701390400359078579693043519447331113978918064868415326638035\",\n \"9918110051302171585080402603319702774565515993150576347155970296011118125764\",\n \"1\"\n ],\n \"pi_b\": [\n [\n \"10857046999023057135944570762232829481370756359578518086990519993285655852781\",\n \"11559732032986387107991004021392285783925812861821192530917403151452391805634\"\n ],\n [\n \"8495653923123431417604973247489272438418190587263600148770280649306958101930\",\n \"4082367875863433681332203403145435568316851327593401208105741076214120093531\"\n ],\n [\n \"1\",\n \"0\"\n ]\n ],\n \"pi_c\": [\n \"1368015179489954701390400359078579693043519447331113978918064868415326638035\",\n \"9918110051302171585080402603319702774565515993150576347155970296011118125764\",\n \"1\"\n ],\n \"protocol\": \"groth\"\n}\n", + "pubData": "[\n \"8863150934551775031093873719629424744398133643983814385850330952980893030086\"\n]\n", + "status": "success" +} + +proof: +{ + "pi_a": [ + "1368015179489954701390400359078579693043519447331113978918064868415326638035", + "9918110051302171585080402603319702774565515993150576347155970296011118125764", + "1" + ], + "pi_b": [ + [ + "10857046999023057135944570762232829481370756359578518086990519993285655852781", + "11559732032986387107991004021392285783925812861821192530917403151452391805634" + ], + [ + "8495653923123431417604973247489272438418190587263600148770280649306958101930", + "4082367875863433681332203403145435568316851327593401208105741076214120093531" + ], + [ + "1", + "0" + ] + ], + "pi_c": [ + "1368015179489954701390400359078579693043519447331113978918064868415326638035", + "9918110051302171585080402603319702774565515993150576347155970296011118125764", + "1" + ], + "protocol": "groth" +} + +pubData: +[ + "8863150934551775031093873719629424744398133643983814385850330952980893030086" +] +*/ + func (s *Mock) handleStatus(c *gin.Context) { s.RLock() c.JSON(http.StatusOK, prover.Status{ @@ -133,11 +177,11 @@ func (s *Mock) runProver(ctx context.Context) { s.counter++ // Mock data s.proof = fmt.Sprintf(`{ - "pi_a": ["%v", "%v"], - "pi_b": [["%v", "%v"],["%v", "%v"],["%v", "%v"]], - "pi_c": ["%v", "%v"], + "pi_a": ["%v", "%v", "1"], + "pi_b": [["%v", "%v"],["%v", "%v"],["1", "0"]], + "pi_c": ["%v", "%v", "1"], "protocol": "groth16" - }`, i, i+1, i+2, i+3, i+4, i+5, i+6, i+7, i+8, i+9) //nolint:gomnd + }`, i, i+1, i+2, i+3, i+4, i+5, i+6, i+7) //nolint:gomnd s.pubData = fmt.Sprintf(`[ "%v" ]`, i+42) //nolint:gomnd diff --git a/txselector/txselector.go b/txselector/txselector.go index bf59337..bf75653 100644 --- a/txselector/txselector.go +++ b/txselector/txselector.go @@ -151,15 +151,15 @@ func (txsel *TxSelector) GetCoordIdxs() (map[common.TokenID]common.Idx, error) { // GetL2TxSelection returns the L1CoordinatorTxs and a selection of the L2Txs // for the next batch, from the L2DB pool func (txsel *TxSelector) GetL2TxSelection(selectionConfig *SelectionConfig, - batchNum common.BatchNum) ([]common.Idx, []common.L1Tx, []common.PoolL2Tx, error) { - coordIdxs, _, l1CoordinatorTxs, l2Txs, err := txsel.GetL1L2TxSelection(selectionConfig, batchNum, + batchNum common.BatchNum) ([]common.Idx, [][]byte, []common.L1Tx, []common.PoolL2Tx, error) { + coordIdxs, auths, _, l1CoordinatorTxs, l2Txs, err := txsel.GetL1L2TxSelection(selectionConfig, batchNum, []common.L1Tx{}) - return coordIdxs, l1CoordinatorTxs, l2Txs, tracerr.Wrap(err) + return coordIdxs, auths, l1CoordinatorTxs, l2Txs, tracerr.Wrap(err) } // GetL1L2TxSelection returns the selection of L1 + L2 txs func (txsel *TxSelector) GetL1L2TxSelection(selectionConfig *SelectionConfig, - batchNum common.BatchNum, l1Txs []common.L1Tx) ([]common.Idx, []common.L1Tx, []common.L1Tx, + batchNum common.BatchNum, l1Txs []common.L1Tx) ([]common.Idx, [][]byte, []common.L1Tx, []common.L1Tx, []common.PoolL2Tx, error) { // apply l1-user-tx to localAccountDB // create new leaves @@ -169,7 +169,7 @@ func (txsel *TxSelector) GetL1L2TxSelection(selectionConfig *SelectionConfig, // get existing CoordIdxs coordIdxsMap, err := txsel.GetCoordIdxs() if err != nil { - return nil, nil, nil, nil, tracerr.Wrap(err) + return nil, nil, nil, nil, nil, tracerr.Wrap(err) } var coordIdxs []common.Idx for tokenID := range coordIdxsMap { @@ -179,7 +179,7 @@ func (txsel *TxSelector) GetL1L2TxSelection(selectionConfig *SelectionConfig, // get pending l2-tx from tx-pool l2TxsRaw, err := txsel.l2db.GetPendingTxs() // (batchID) if err != nil { - return nil, nil, nil, nil, tracerr.Wrap(err) + return nil, nil, nil, nil, nil, tracerr.Wrap(err) } var validTxs txs @@ -235,14 +235,19 @@ func (txsel *TxSelector) GetL1L2TxSelection(selectionConfig *SelectionConfig, // process the txs in the local AccountsDB _, err = txsel.localAccountsDB.ProcessTxs(ptc, coordIdxs, l1Txs, l1CoordinatorTxs, l2Txs) if err != nil { - return nil, nil, nil, nil, tracerr.Wrap(err) + return nil, nil, nil, nil, nil, tracerr.Wrap(err) } err = txsel.localAccountsDB.MakeCheckpoint() if err != nil { - return nil, nil, nil, nil, tracerr.Wrap(err) + return nil, nil, nil, nil, nil, tracerr.Wrap(err) } - return nil, l1Txs, l1CoordinatorTxs, l2Txs, nil + // TODO + auths := make([][]byte, len(l1CoordinatorTxs)) + for i := range auths { + auths[i] = make([]byte, 65) + } + return nil, auths, l1Txs, l1CoordinatorTxs, l2Txs, nil } // processTxsToEthAddrBJJ process the common.PoolL2Tx in the case where diff --git a/txselector/txselector_test.go b/txselector/txselector_test.go index 3e02ce8..a9226cd 100644 --- a/txselector/txselector_test.go +++ b/txselector/txselector_test.go @@ -133,12 +133,12 @@ func TestGetL2TxSelection(t *testing.T) { // add the 1st batch of transactions to the TxSelector addL2Txs(t, txsel, common.L2TxsToPoolL2Txs(blocks[0].Rollup.Batches[0].L2Txs)) - _, l1CoordTxs, l2Txs, err := txsel.GetL2TxSelection(selectionConfig, 0) + _, _, l1CoordTxs, l2Txs, err := txsel.GetL2TxSelection(selectionConfig, 0) assert.NoError(t, err) assert.Equal(t, 0, len(l2Txs)) assert.Equal(t, 0, len(l1CoordTxs)) - _, _, _, _, err = txsel.GetL1L2TxSelection(selectionConfig, 0, blocks[0].Rollup.L1UserTxs) + _, _, _, _, _, err = txsel.GetL1L2TxSelection(selectionConfig, 0, blocks[0].Rollup.L1UserTxs) assert.NoError(t, err) // TODO once L2DB is updated to return error in case that AddTxTest From a50bda3f5932a84dd2099d51c00a77c98f58bb5e Mon Sep 17 00:00:00 2001 From: Eduard S Date: Tue, 22 Dec 2020 16:58:07 +0100 Subject: [PATCH 2/3] Update packr dependency Update packr to 2.8.1 which fixes this issue: https://github.com/gobuffalo/packr/issues/261 which I encountered while running a test with `-race` --- go.mod | 2 +- go.sum | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/go.mod b/go.mod index 336a167..8dcebe2 100644 --- a/go.mod +++ b/go.mod @@ -12,7 +12,7 @@ require ( github.com/gin-contrib/cors v1.3.1 github.com/gin-gonic/gin v1.5.0 github.com/go-sql-driver/mysql v1.5.0 // indirect - github.com/gobuffalo/packr/v2 v2.8.0 + github.com/gobuffalo/packr/v2 v2.8.1 github.com/hermeznetwork/tracerr v0.3.1-0.20201126162137-de9930d0cf29 github.com/iden3/go-iden3-crypto v0.0.6-0.20201221160344-58e589b6eb4c github.com/iden3/go-merkletree v0.0.0-20201215142017-730707e5659a diff --git a/go.sum b/go.sum index a032a30..ed07b22 100644 --- a/go.sum +++ b/go.sum @@ -237,6 +237,8 @@ github.com/gobuffalo/packd v1.0.0/go.mod h1:6VTc4htmJRFB7u1m/4LeMTWjFoYrUiBkU9Fd github.com/gobuffalo/packr/v2 v2.7.1/go.mod h1:qYEvAazPaVxy7Y7KR0W8qYEE+RymX74kETFqjFoFlOc= github.com/gobuffalo/packr/v2 v2.8.0 h1:IULGd15bQL59ijXLxEvA5wlMxsmx/ZkQv9T282zNVIY= github.com/gobuffalo/packr/v2 v2.8.0/go.mod h1:PDk2k3vGevNE3SwVyVRgQCCXETC9SaONCNSXT1Q8M1g= +github.com/gobuffalo/packr/v2 v2.8.1 h1:tkQpju6i3EtMXJ9uoF5GT6kB+LMTimDWD8Xvbz6zDVA= +github.com/gobuffalo/packr/v2 v2.8.1/go.mod h1:c/PLlOuTU+p3SybaJATW3H6lX/iK7xEz5OeMf+NnJpg= github.com/godror/godror v0.13.3/go.mod h1:2ouUT4kdhUBk7TAkHWD4SN0CdI0pgEQbo8FVHhbSKWg= github.com/gofrs/flock v0.7.1/go.mod h1:F1TvTiK9OcQqauNUHlbJvyl9Qa1QvF/gOUDKA14jxHU= github.com/gogo/googleapis v1.1.0/go.mod h1:gf4bu3Q80BeJ6H1S1vYPm8/ELATdvryBaNFGgqEef3s= @@ -392,6 +394,8 @@ github.com/karalabe/usb v0.0.0-20190919080040-51dc0efba356 h1:I/yrLt2WilKxlQKCM5 github.com/karalabe/usb v0.0.0-20190919080040-51dc0efba356/go.mod h1:Od972xHfMJowv7NGVDiWVxk2zxnWgjLlJzE+F4F7AGU= github.com/karrick/godirwalk v1.15.3 h1:0a2pXOgtB16CqIqXTiT7+K9L73f74n/aNQUnH6Ortew= github.com/karrick/godirwalk v1.15.3/go.mod h1:j4mkqPuvaLI8mp1DroR3P6ad7cyYd4c1qeJ3RV7ULlk= +github.com/karrick/godirwalk v1.15.8 h1:7+rWAZPn9zuRxaIqqT8Ohs2Q2Ac0msBqwRdxNCr2VVs= +github.com/karrick/godirwalk v1.15.8/go.mod h1:j4mkqPuvaLI8mp1DroR3P6ad7cyYd4c1qeJ3RV7ULlk= github.com/kevinburke/ssh_config v0.0.0-20190725054713-01f96b0aa0cd/go.mod h1:CT57kijsi8u/K/BOFA39wgDQJ9CxiF4nAY/ojJ6r6mM= github.com/kisielk/errcheck v1.1.0/go.mod h1:EZBBE59ingxPouuu3KfxchcWSUPOHkagtvWXihfKN4Q= github.com/kisielk/errcheck v1.2.0/go.mod h1:/BMXB+zMLi60iA8Vv6Ksmxu/1UDYcXs4uQLJ+jE2L00= From 2583d35adf57586f6534f4ef3405c6241dcf0c7d Mon Sep 17 00:00:00 2001 From: arnaucube Date: Wed, 23 Dec 2020 11:01:20 +0100 Subject: [PATCH 3/3] Update ZKInputs for empty batches - Add TestZKInputsEmpty circom test vectors - Fix ZKInputsHash OldLastIdx bytearray --- common/zk.go | 5 ++-- db/statedb/txprocessors.go | 1 + db/statedb/zkinputsgen_test.go | 45 +++++++++++++++++++++------------- 3 files changed, 32 insertions(+), 19 deletions(-) diff --git a/common/zk.go b/common/zk.go index 53c3d6f..6977f0a 100644 --- a/common/zk.go +++ b/common/zk.go @@ -431,8 +431,9 @@ func (z ZKInputs) ToHashGlobalData() ([]byte, error) { // [MAX_NLEVELS bits] oldLastIdx oldLastIdx := make([]byte, bytesMaxLevels) - copy(oldLastIdx, z.OldLastIdx.Bytes()) - b = append(b, SwapEndianness(oldLastIdx)...) + oldLastIdxBytes := z.OldLastIdx.Bytes() + copy(oldLastIdx[len(oldLastIdx)-len(oldLastIdxBytes):], oldLastIdxBytes) + b = append(b, oldLastIdx...) // [MAX_NLEVELS bits] newLastIdx newLastIdx := make([]byte, bytesMaxLevels) diff --git a/db/statedb/txprocessors.go b/db/statedb/txprocessors.go index 0547fe0..1e9876f 100644 --- a/db/statedb/txprocessors.go +++ b/db/statedb/txprocessors.go @@ -96,6 +96,7 @@ func (s *StateDB) ProcessTxs(ptc ProcessTxsConfig, coordIdxs []common.Idx, l1use s.zki = common.NewZKInputs(ptc.MaxTx, ptc.MaxL1Tx, ptc.MaxTx, ptc.MaxFeeTx, ptc.NLevels, s.currentBatch.BigInt()) s.zki.OldLastIdx = s.idx.BigInt() s.zki.OldStateRoot = s.mt.Root().BigInt() + s.zki.Metadata.NewLastIdxRaw = s.idx } // TBD if ExitTree is only in memory or stored in disk, for the moment diff --git a/db/statedb/zkinputsgen_test.go b/db/statedb/zkinputsgen_test.go index 48fa7b8..529f7f4 100644 --- a/db/statedb/zkinputsgen_test.go +++ b/db/statedb/zkinputsgen_test.go @@ -256,10 +256,13 @@ func TestZKInputsEmpty(t *testing.T) { ptc := ProcessTxsConfig{ NLevels: uint32(nLevels), - MaxTx: 10, - MaxL1Tx: 5, + MaxTx: 3, + MaxL1Tx: 2, MaxFeeTx: 2, } + // skip first batch to do the test with BatchNum=1 + _, err = sdb.ProcessTxs(ptc, nil, nil, nil, nil) + require.NoError(t, err) // 0. Generate a batch from the empty state with no transactions @@ -279,21 +282,21 @@ func TestZKInputsEmpty(t *testing.T) { require.NoError(t, err) assert.Equal(t, 0, len(accs)) - /* // TODO h, err := ptOut.ZKInputs.HashGlobalData() require.Nil(t, err) - assert.Equal(t, "TODO", h.String()) + assert.Equal(t, "3188598899537193557996756147565812826329089180539076403761918575055837892421", h.String()) toHash, err := ptOut.ZKInputs.ToHashGlobalData() require.Nil(t, err) - assert.Equal(t, "TODO", hex.EncodeToString(toHash)) + assert.Equal(t, "0000000000ff0000000000ff00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000001", hex.EncodeToString(toHash)) + + // printZKInputs(t, ptOut.ZKInputs) s, err := json.Marshal(ptOut.ZKInputs) require.Nil(t, err) // the 'expected' data has been checked with the circom circuits - expected := `TODO` + expected := `{"auxFromIdx":["0","0","0"],"auxToIdx":["0","0","0"],"ay1":["0","0","0"],"ay2":["0","0","0"],"ay3":["0","0"],"balance1":["0","0","0"],"balance2":["0","0","0"],"balance3":["0","0"],"currentNumBatch":"1","ethAddr1":["0","0","0"],"ethAddr2":["0","0","0"],"ethAddr3":["0","0"],"feeIdxs":["0","0"],"feePlanTokens":["0","0"],"fromBjjCompressed":[["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"]],"fromEthAddr":["0","0","0"],"fromIdx":["0","0","0"],"globalChainID":"0","imAccFeeOut":[["0","0"],["0","0"]],"imExitRoot":["0","0"],"imFinalAccFee":["0","0"],"imInitStateRootFee":"0","imOnChain":["0","0"],"imOutIdx":["255","255"],"imStateRoot":["0","0"],"imStateRootFee":["0"],"isOld0_1":["0","0","0"],"isOld0_2":["0","0","0"],"loadAmountF":["0","0","0"],"maxNumBatch":["0","0","0"],"newAccount":["0","0","0"],"newExit":["0","0","0"],"nonce1":["0","0","0"],"nonce2":["0","0","0"],"nonce3":["0","0"],"oldKey1":["0","0","0"],"oldKey2":["0","0","0"],"oldLastIdx":"255","oldStateRoot":"0","oldValue1":["0","0","0"],"oldValue2":["0","0","0"],"onChain":["0","0","0"],"r8x":["0","0","0"],"r8y":["0","0","0"],"rqOffset":["0","0","0"],"rqToBjjAy":["0","0","0"],"rqToEthAddr":["0","0","0"],"rqTxCompressedDataV2":["0","0","0"],"s":["0","0","0"],"siblings1":[["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"]],"siblings2":[["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"]],"siblings3":[["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"]],"sign1":["0","0","0"],"sign2":["0","0","0"],"sign3":["0","0"],"toBjjAy":["0","0","0"],"toEthAddr":["0","0","0"],"toIdx":["0","0","0"],"tokenID1":["0","0","0"],"tokenID2":["0","0","0"],"tokenID3":["0","0"],"txCompressedData":["3322668559","3322668559","3322668559"],"txCompressedDataV2":["0","0","0"]}` assert.Equal(t, expected, string(s)) - */ // 1. Generate a batch with two transactions that create one account // so that the state tree is not empty (same transactions as @@ -332,7 +335,7 @@ func TestZKInputsEmpty(t *testing.T) { sig := users[0].BJJ.SignPoseidon(toSign) l2Txs[0].Signature = sig.Compress() - _, err = sdb.ProcessTxs(ptc, nil, l1UserTxs, nil, l2Txs) + ptOut, err = sdb.ProcessTxs(ptc, nil, l1UserTxs, nil, l2Txs) require.Nil(t, err) rootNonZero := sdb.mt.Root() @@ -342,6 +345,14 @@ func TestZKInputsEmpty(t *testing.T) { require.NoError(t, err) assert.Equal(t, 1, len(accs)) + h, err = ptOut.ZKInputs.HashGlobalData() + require.Nil(t, err) + assert.Equal(t, "3310178391322068804285093727572639741003903261273589212922632857075167683076", h.String()) + + toHash, err = ptOut.ZKInputs.ToHashGlobalData() + require.Nil(t, err) + assert.Equal(t, "0000000000ff0000000001000000000000000000000000000000000000000000000000000000000000000000071a61ed5a1ac052b0d1086a330c540b55318a07f6b7989573b9bbbb5380d1a900000000000000000000000000000000000000000000000000000000000000007e5f4552091a69125d5dfcb7b8c2659029395bdf21b0a1688b37f77b1d1d5539ec3b826db5ac78b2513f574a04c50a7d4f8246d700000000000028a0000000000001000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000100010003e8000000000000000000000000000000000002", hex.EncodeToString(toHash)) + // 2. Generate a batch from a non-empty state with no transactions coordIdxs = []common.Idx{} @@ -360,21 +371,21 @@ func TestZKInputsEmpty(t *testing.T) { require.NoError(t, err) assert.Equal(t, 1, len(accs)) - /* // TODO - h, err := ptOut.ZKInputs.HashGlobalData() + // printZKInputs(t, ptOut.ZKInputs) + + h, err = ptOut.ZKInputs.HashGlobalData() require.Nil(t, err) - assert.Equal(t, "TODO", h.String()) + assert.Equal(t, "19357487518263066850851765738275153101092738034836476310547111817256740018046", h.String()) - toHash, err := ptOut.ZKInputs.ToHashGlobalData() + toHash, err = ptOut.ZKInputs.ToHashGlobalData() require.Nil(t, err) - assert.Equal(t, "TODO", hex.EncodeToString(toHash)) + assert.Equal(t, "000000000100000000000100071a61ed5a1ac052b0d1086a330c540b55318a07f6b7989573b9bbbb5380d1a9071a61ed5a1ac052b0d1086a330c540b55318a07f6b7989573b9bbbb5380d1a9000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000003", hex.EncodeToString(toHash)) - s, err := json.Marshal(ptOut.ZKInputs) + s, err = json.Marshal(ptOut.ZKInputs) require.Nil(t, err) // the 'expected' data has been checked with the circom circuits - expected := `TODO` + expected = `{"auxFromIdx":["0","0","0"],"auxToIdx":["0","0","0"],"ay1":["0","0","0"],"ay2":["0","0","0"],"ay3":["0","0"],"balance1":["0","0","0"],"balance2":["0","0","0"],"balance3":["0","0"],"currentNumBatch":"3","ethAddr1":["0","0","0"],"ethAddr2":["0","0","0"],"ethAddr3":["0","0"],"feeIdxs":["0","0"],"feePlanTokens":["0","0"],"fromBjjCompressed":[["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"]],"fromEthAddr":["0","0","0"],"fromIdx":["0","0","0"],"globalChainID":"0","imAccFeeOut":[["0","0"],["0","0"]],"imExitRoot":["0","0"],"imFinalAccFee":["0","0"],"imInitStateRootFee":"3212803832159212591526550848126062808026208063555125878245901046146545013161","imOnChain":["0","0"],"imOutIdx":["256","256"],"imStateRoot":["3212803832159212591526550848126062808026208063555125878245901046146545013161","3212803832159212591526550848126062808026208063555125878245901046146545013161"],"imStateRootFee":["3212803832159212591526550848126062808026208063555125878245901046146545013161"],"isOld0_1":["0","0","0"],"isOld0_2":["0","0","0"],"loadAmountF":["0","0","0"],"maxNumBatch":["0","0","0"],"newAccount":["0","0","0"],"newExit":["0","0","0"],"nonce1":["0","0","0"],"nonce2":["0","0","0"],"nonce3":["0","0"],"oldKey1":["0","0","0"],"oldKey2":["0","0","0"],"oldLastIdx":"256","oldStateRoot":"3212803832159212591526550848126062808026208063555125878245901046146545013161","oldValue1":["0","0","0"],"oldValue2":["0","0","0"],"onChain":["0","0","0"],"r8x":["0","0","0"],"r8y":["0","0","0"],"rqOffset":["0","0","0"],"rqToBjjAy":["0","0","0"],"rqToEthAddr":["0","0","0"],"rqTxCompressedDataV2":["0","0","0"],"s":["0","0","0"],"siblings1":[["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"]],"siblings2":[["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"]],"siblings3":[["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"],["0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0","0"]],"sign1":["0","0","0"],"sign2":["0","0","0"],"sign3":["0","0"],"toBjjAy":["0","0","0"],"toEthAddr":["0","0","0"],"toIdx":["0","0","0"],"tokenID1":["0","0","0"],"tokenID2":["0","0","0"],"tokenID3":["0","0"],"txCompressedData":["3322668559","3322668559","3322668559"],"txCompressedDataV2":["0","0","0"]}` assert.Equal(t, expected, string(s)) - */ } // TestZKInputs0: @@ -1201,7 +1212,7 @@ func TestZKInputs6(t *testing.T) { assert.Equal(t, "4506051426679555819811005692198685182747763336038770877076710632305611650930", sdb.mt.Root().BigInt().String()) h, err = ptOut.ZKInputs.HashGlobalData() require.NoError(t, err) - assert.Equal(t, "212658139389894676556504584698627643826403691212271531527535357720589015657", h.String()) + assert.Equal(t, "4701632846207201125105176884973241543664109364248244712634276477520091620527", h.String()) // printZKInputs(t, ptOut.ZKInputs) }