summaryrefslogtreecommitdiff
path: root/ledger/eval_test.go
diff options
context:
space:
mode:
Diffstat (limited to 'ledger/eval_test.go')
-rw-r--r--ledger/eval_test.go757
1 files changed, 592 insertions, 165 deletions
diff --git a/ledger/eval_test.go b/ledger/eval_test.go
index 0f2aea2b3..9d3598250 100644
--- a/ledger/eval_test.go
+++ b/ledger/eval_test.go
@@ -20,11 +20,13 @@ import (
"context"
"errors"
"fmt"
+ "math/rand"
"os"
"path/filepath"
"reflect"
"runtime/pprof"
"strings"
+ "sync"
"testing"
"time"
@@ -72,6 +74,7 @@ func TestBlockEvaluator(t *testing.T) {
newBlock := bookkeeping.MakeBlock(genesisInitState.Block.BlockHeader)
eval, err := l.StartEvaluator(newBlock.BlockHeader, 0)
+ require.Equal(t, eval.specials.FeeSink, testSinkAddr)
require.NoError(t, err)
genHash := genesisInitState.Block.BlockHeader.GenesisHash
@@ -347,40 +350,24 @@ func TestPrepareEvalParams(t *testing.T) {
}
params := []config.ConsensusParams{
- config.ConsensusParams{Application: true, MaxAppProgramCost: 700},
+ {Application: true, MaxAppProgramCost: 700},
config.Consensus[protocol.ConsensusV29],
config.Consensus[protocol.ConsensusFuture],
}
// Create some sample transactions
- payment := transactions.SignedTxnWithAD{
- SignedTxn: transactions.SignedTxn{
- Txn: transactions.Transaction{
- Type: protocol.PaymentTx,
- Header: transactions.Header{
- Sender: basics.Address{1, 2, 3, 4},
- },
- PaymentTxnFields: transactions.PaymentTxnFields{
- Receiver: basics.Address{4, 3, 2, 1},
- Amount: basics.MicroAlgos{Raw: 100},
- },
- },
- },
- }
-
- appcall1 := transactions.SignedTxnWithAD{
- SignedTxn: transactions.SignedTxn{
- Txn: transactions.Transaction{
- Type: protocol.ApplicationCallTx,
- Header: transactions.Header{
- Sender: basics.Address{1, 2, 3, 4},
- },
- ApplicationCallTxnFields: transactions.ApplicationCallTxnFields{
- ApplicationID: basics.AppIndex(1),
- },
- },
- },
- }
+ payment := txntest.Txn{
+ Type: protocol.PaymentTx,
+ Sender: basics.Address{1, 2, 3, 4},
+ Receiver: basics.Address{4, 3, 2, 1},
+ Amount: 100,
+ }.SignedTxnWithAD()
+
+ appcall1 := txntest.Txn{
+ Type: protocol.ApplicationCallTx,
+ Sender: basics.Address{1, 2, 3, 4},
+ ApplicationID: basics.AppIndex(1),
+ }.SignedTxnWithAD()
appcall2 := appcall1
appcall2.SignedTxn.Txn.ApplicationCallTxnFields.ApplicationID = basics.AppIndex(2)
@@ -431,7 +418,7 @@ func TestPrepareEvalParams(t *testing.T) {
if present {
require.NotNil(t, res[k])
require.NotNil(t, res[k].PastSideEffects)
- require.Equal(t, res[k].GroupIndex, k)
+ require.Equal(t, res[k].GroupIndex, uint64(k))
require.Equal(t, res[k].TxnGroup, expGroupNoAD)
require.Equal(t, *res[k].Proto, eval.proto)
require.Equal(t, *res[k].Txn, testCase.group[k].SignedTxn)
@@ -536,14 +523,16 @@ ok:
{
SignedTxn: stxn1,
ApplyData: transactions.ApplyData{
- EvalDelta: basics.EvalDelta{GlobalDelta: map[string]basics.ValueDelta{
+ EvalDelta: transactions.EvalDelta{GlobalDelta: map[string]basics.ValueDelta{
"creator": {Action: basics.SetBytesAction, Bytes: string(addrs[0][:])}},
- }},
+ },
+ ApplicationID: 1,
+ },
},
{
SignedTxn: stxn2,
ApplyData: transactions.ApplyData{
- EvalDelta: basics.EvalDelta{GlobalDelta: map[string]basics.ValueDelta{
+ EvalDelta: transactions.EvalDelta{GlobalDelta: map[string]basics.ValueDelta{
"caller": {Action: basics.SetBytesAction, Bytes: string(addrs[0][:])}},
}},
},
@@ -583,86 +572,34 @@ func TestEvalAppAllocStateWithTxnGroup(t *testing.T) {
require.Equal(t, basics.TealValue{Type: basics.TealBytesType, Bytes: string(addr[:])}, state["creator"])
}
-func testEvalAppPoolingGroup(t *testing.T, schema basics.StateSchema, approvalProgram string, consensusVersion protocol.ConsensusVersion) (*BlockEvaluator, error) {
- genesisInitState, addrs, keys := genesis(10)
-
- dbName := fmt.Sprintf("%s.%d", t.Name(), crypto.RandUint64())
- const inMem = true
- cfg := config.GetDefaultLocal()
- l, err := OpenLedger(logging.Base(), dbName, inMem, genesisInitState, cfg)
- require.NoError(t, err)
+func testEvalAppPoolingGroup(t *testing.T, schema basics.StateSchema, approvalProgram string, consensusVersion protocol.ConsensusVersion) error {
+ genBalances, addrs, _ := newTestGenesis()
+ l := newTestLedger(t, genBalances)
defer l.Close()
- newBlock := bookkeeping.MakeBlock(genesisInitState.Block.BlockHeader)
- eval, err := l.StartEvaluator(newBlock.BlockHeader, 0)
- require.NoError(t, err)
- eval.validate = true
- eval.generate = false
+ eval := l.nextBlock(t)
eval.proto = config.Consensus[consensusVersion]
- ops, err := logic.AssembleString(approvalProgram)
- require.NoError(t, err, ops.Errors)
- approval := ops.Program
- ops, err = logic.AssembleString("#pragma version 4\nint 1")
- require.NoError(t, err)
- clear := ops.Program
-
- genHash := genesisInitState.Block.BlockHeader.GenesisHash
- header := transactions.Header{
- Sender: addrs[0],
- Fee: minFee,
- FirstValid: newBlock.Round(),
- LastValid: newBlock.Round(),
- GenesisHash: genHash,
- }
- appcall1 := transactions.Transaction{
- Type: protocol.ApplicationCallTx,
- Header: header,
- ApplicationCallTxnFields: transactions.ApplicationCallTxnFields{
- GlobalStateSchema: schema,
- ApprovalProgram: approval,
- ClearStateProgram: clear,
- },
+ appcall1 := txntest.Txn{
+ Sender: addrs[0],
+ Type: protocol.ApplicationCallTx,
+ GlobalStateSchema: schema,
+ ApprovalProgram: approvalProgram,
}
- appcall2 := transactions.Transaction{
- Type: protocol.ApplicationCallTx,
- Header: header,
- ApplicationCallTxnFields: transactions.ApplicationCallTxnFields{
- ApplicationID: basics.AppIndex(1),
- },
+ appcall2 := txntest.Txn{
+ Sender: addrs[0],
+ Type: protocol.ApplicationCallTx,
+ ApplicationID: basics.AppIndex(1),
}
- appcall3 := appcall2
- appcall3.Header.Sender = addrs[1]
-
- var group transactions.TxGroup
- group.TxGroupHashes = []crypto.Digest{crypto.HashObj(appcall1), crypto.HashObj(appcall2), crypto.HashObj(appcall3)}
- appcall1.Group = crypto.HashObj(group)
- appcall2.Group = crypto.HashObj(group)
- appcall3.Group = crypto.HashObj(group)
- stxn1 := appcall1.Sign(keys[0])
- stxn2 := appcall2.Sign(keys[0])
- stxn3 := appcall3.Sign(keys[1])
-
- g := []transactions.SignedTxnWithAD{
- {
- SignedTxn: stxn1,
- },
- {
- SignedTxn: stxn2,
- },
- {
- SignedTxn: stxn3,
- },
- }
- txgroup := []transactions.SignedTxn{stxn1, stxn2, stxn3}
- err = eval.TestTransactionGroup(txgroup)
- if err != nil {
- return eval, err
+ appcall3 := txntest.Txn{
+ Sender: addrs[1],
+ Type: protocol.ApplicationCallTx,
+ ApplicationID: basics.AppIndex(1),
}
- err = eval.transactionGroup(g)
- return eval, err
+
+ return eval.txgroup(t, &appcall1, &appcall2, &appcall3)
}
// TestEvalAppPooledBudgetWithTxnGroup ensures 3 app call txns can successfully pool
@@ -704,7 +641,7 @@ func TestEvalAppPooledBudgetWithTxnGroup(t *testing.T) {
for i, param := range params {
for j, testCase := range cases {
t.Run(fmt.Sprintf("i=%d,j=%d", i, j), func(t *testing.T) {
- _, err := testEvalAppPoolingGroup(t, basics.StateSchema{NumByteSlice: 3}, testCase.prog, param)
+ err := testEvalAppPoolingGroup(t, basics.StateSchema{NumByteSlice: 3}, testCase.prog, param)
if !testCase.isSuccessV29 && reflect.DeepEqual(param, protocol.ConsensusV29) {
require.Error(t, err)
require.Contains(t, err.Error(), testCase.expectedErrorV29)
@@ -717,30 +654,250 @@ func TestEvalAppPooledBudgetWithTxnGroup(t *testing.T) {
}
}
+// BenchTxnGenerator generates transactions as long as asked for
+type BenchTxnGenerator interface {
+ // Prepare should be used for making pre-benchmark ledger initialization
+ // like accounts funding, assets or apps creation
+ Prepare(tb testing.TB, addrs []basics.Address, keys []*crypto.SignatureSecrets, rnd basics.Round, gh crypto.Digest) ([]transactions.SignedTxn, int)
+ // Txn generates a single transaction
+ Txn(tb testing.TB, addrs []basics.Address, keys []*crypto.SignatureSecrets, rnd basics.Round, gh crypto.Digest) transactions.SignedTxn
+}
+
+// BenchPaymentTxnGenerator generates payment transactions
+type BenchPaymentTxnGenerator struct {
+ counter int
+}
+
+func (g *BenchPaymentTxnGenerator) Prepare(tb testing.TB, addrs []basics.Address, keys []*crypto.SignatureSecrets, rnd basics.Round, gh crypto.Digest) ([]transactions.SignedTxn, int) {
+ return nil, 0
+}
+
+func (g *BenchPaymentTxnGenerator) Txn(tb testing.TB, addrs []basics.Address, keys []*crypto.SignatureSecrets, rnd basics.Round, gh crypto.Digest) transactions.SignedTxn {
+ sender := g.counter % len(addrs)
+ receiver := (g.counter + 1) % len(addrs)
+ // The following would create more random selection of accounts, and prevent a cache of half of the accounts..
+ // iDigest := crypto.Hash([]byte{byte(i), byte(i >> 8), byte(i >> 16), byte(i >> 24)})
+ // sender := (uint64(iDigest[0]) + uint64(iDigest[1])*256 + uint64(iDigest[2])*256*256) % uint64(len(addrs))
+ // receiver := (uint64(iDigest[4]) + uint64(iDigest[5])*256 + uint64(iDigest[6])*256*256) % uint64(len(addrs))
+
+ txn := transactions.Transaction{
+ Type: protocol.PaymentTx,
+ Header: transactions.Header{
+ Sender: addrs[sender],
+ Fee: minFee,
+ FirstValid: rnd,
+ LastValid: rnd,
+ GenesisHash: gh,
+ },
+ PaymentTxnFields: transactions.PaymentTxnFields{
+ Receiver: addrs[receiver],
+ Amount: basics.MicroAlgos{Raw: 100},
+ },
+ }
+ stxn := txn.Sign(keys[sender])
+ g.counter++
+ return stxn
+}
+
+// BenchAppTxnGenerator generates app opt in transactions
+type BenchAppOptInsTxnGenerator struct {
+ NumApps int
+ Proto protocol.ConsensusVersion
+ Program []byte
+ OptedInAccts []basics.Address
+ OptedInAcctsIndices []int
+}
+
+func (g *BenchAppOptInsTxnGenerator) Prepare(tb testing.TB, addrs []basics.Address, keys []*crypto.SignatureSecrets, rnd basics.Round, gh crypto.Digest) ([]transactions.SignedTxn, int) {
+ maxLocalSchemaEntries := config.Consensus[g.Proto].MaxLocalSchemaEntries
+ maxAppsOptedIn := config.Consensus[g.Proto].MaxAppsOptedIn
+
+ // this function might create too much transaction even to fit into a single block
+ // estimate number of smaller blocks needed in order to set LastValid properly
+ const numAccts = 10000
+ const maxTxnPerBlock = 10000
+ expectedTxnNum := g.NumApps + numAccts*maxAppsOptedIn
+ expectedNumOfBlocks := expectedTxnNum/maxTxnPerBlock + 1
+
+ createTxns := make([]transactions.SignedTxn, 0, g.NumApps)
+ for i := 0; i < g.NumApps; i++ {
+ creatorIdx := rand.Intn(len(addrs))
+ creator := addrs[creatorIdx]
+ txn := transactions.Transaction{
+ Type: protocol.ApplicationCallTx,
+ Header: transactions.Header{
+ Sender: creator,
+ Fee: minFee,
+ FirstValid: rnd,
+ LastValid: rnd + basics.Round(expectedNumOfBlocks),
+ GenesisHash: gh,
+ Note: randomNote(),
+ },
+ ApplicationCallTxnFields: transactions.ApplicationCallTxnFields{
+ ApprovalProgram: g.Program,
+ ClearStateProgram: []byte{0x02, 0x20, 0x01, 0x01, 0x22},
+ LocalStateSchema: basics.StateSchema{NumByteSlice: maxLocalSchemaEntries},
+ },
+ }
+ stxn := txn.Sign(keys[creatorIdx])
+ createTxns = append(createTxns, stxn)
+ }
+
+ appsOptedIn := make(map[basics.Address]map[basics.AppIndex]struct{}, numAccts)
+
+ optInTxns := make([]transactions.SignedTxn, 0, numAccts*maxAppsOptedIn)
+
+ for i := 0; i < numAccts; i++ {
+ var senderIdx int
+ var sender basics.Address
+ for {
+ senderIdx = rand.Intn(len(addrs))
+ sender = addrs[senderIdx]
+ if len(appsOptedIn[sender]) < maxAppsOptedIn {
+ appsOptedIn[sender] = make(map[basics.AppIndex]struct{}, maxAppsOptedIn)
+ break
+ }
+ }
+ g.OptedInAccts = append(g.OptedInAccts, sender)
+ g.OptedInAcctsIndices = append(g.OptedInAcctsIndices, senderIdx)
+
+ acctOptIns := appsOptedIn[sender]
+ for j := 0; j < maxAppsOptedIn; j++ {
+ var appIdx basics.AppIndex
+ for {
+ appIdx = basics.AppIndex(rand.Intn(g.NumApps) + 1)
+ if _, ok := acctOptIns[appIdx]; !ok {
+ acctOptIns[appIdx] = struct{}{}
+ break
+ }
+ }
+
+ txn := transactions.Transaction{
+ Type: protocol.ApplicationCallTx,
+ Header: transactions.Header{
+ Sender: sender,
+ Fee: minFee,
+ FirstValid: rnd,
+ LastValid: rnd + basics.Round(expectedNumOfBlocks),
+ GenesisHash: gh,
+ },
+ ApplicationCallTxnFields: transactions.ApplicationCallTxnFields{
+ ApplicationID: basics.AppIndex(appIdx),
+ OnCompletion: transactions.OptInOC,
+ },
+ }
+ stxn := txn.Sign(keys[senderIdx])
+ optInTxns = append(optInTxns, stxn)
+ }
+ appsOptedIn[sender] = acctOptIns
+ }
+
+ return append(createTxns, optInTxns...), maxTxnPerBlock
+}
+
+func (g *BenchAppOptInsTxnGenerator) Txn(tb testing.TB, addrs []basics.Address, keys []*crypto.SignatureSecrets, rnd basics.Round, gh crypto.Digest) transactions.SignedTxn {
+ idx := rand.Intn(len(g.OptedInAcctsIndices))
+ senderIdx := g.OptedInAcctsIndices[idx]
+ sender := addrs[senderIdx]
+ receiverIdx := rand.Intn(len(addrs))
+
+ txn := transactions.Transaction{
+ Type: protocol.PaymentTx,
+ Header: transactions.Header{
+ Sender: sender,
+ Fee: minFee,
+ FirstValid: rnd,
+ LastValid: rnd,
+ GenesisHash: gh,
+ Note: randomNote(),
+ },
+ PaymentTxnFields: transactions.PaymentTxnFields{
+ Receiver: addrs[receiverIdx],
+ Amount: basics.MicroAlgos{Raw: 100},
+ },
+ }
+ stxn := txn.Sign(keys[senderIdx])
+ return stxn
+}
+
func BenchmarkBlockEvaluatorRAMCrypto(b *testing.B) {
- benchmarkBlockEvaluator(b, true, true)
+ g := BenchPaymentTxnGenerator{}
+ benchmarkBlockEvaluator(b, true, true, protocol.ConsensusCurrentVersion, &g)
}
func BenchmarkBlockEvaluatorRAMNoCrypto(b *testing.B) {
- benchmarkBlockEvaluator(b, true, false)
+ g := BenchPaymentTxnGenerator{}
+ benchmarkBlockEvaluator(b, true, false, protocol.ConsensusCurrentVersion, &g)
}
func BenchmarkBlockEvaluatorDiskCrypto(b *testing.B) {
- benchmarkBlockEvaluator(b, false, true)
+ g := BenchPaymentTxnGenerator{}
+ benchmarkBlockEvaluator(b, false, true, protocol.ConsensusCurrentVersion, &g)
}
func BenchmarkBlockEvaluatorDiskNoCrypto(b *testing.B) {
- benchmarkBlockEvaluator(b, false, false)
+ g := BenchPaymentTxnGenerator{}
+ benchmarkBlockEvaluator(b, false, false, protocol.ConsensusCurrentVersion, &g)
+}
+
+func BenchmarkBlockEvaluatorDiskAppOptIns(b *testing.B) {
+ g := BenchAppOptInsTxnGenerator{
+ NumApps: 500,
+ Proto: protocol.ConsensusFuture,
+ Program: []byte{0x02, 0x20, 0x01, 0x01, 0x22},
+ }
+ benchmarkBlockEvaluator(b, false, false, protocol.ConsensusFuture, &g)
+}
+
+func BenchmarkBlockEvaluatorDiskFullAppOptIns(b *testing.B) {
+ // program sets all 16 available keys of len 64 bytes to same values of 64 bytes
+ source := `#pragma version 5
+ txn OnCompletion
+ int OptIn
+ ==
+ bz done
+ int 0
+ store 0 // save loop var
+loop:
+ int 0 // acct index
+ byte "012345678901234567890123456789012345678901234567890123456789ABC0"
+ int 63
+ load 0 // loop var
+ int 0x41
+ +
+ setbyte // str[63] = chr(i + 'A')
+ dup // value is the same as key
+ app_local_put
+ load 0 // loop var
+ int 1
+ +
+ dup
+ store 0 // save loop var
+ int 16
+ <
+ bnz loop
+done:
+ int 1
+`
+ ops, err := logic.AssembleString(source)
+ require.NoError(b, err)
+ prog := ops.Program
+ g := BenchAppOptInsTxnGenerator{
+ NumApps: 500,
+ Proto: protocol.ConsensusFuture,
+ Program: prog,
+ }
+ benchmarkBlockEvaluator(b, false, false, protocol.ConsensusFuture, &g)
}
// this variant focuses on benchmarking ledger.go `eval()`, the rest is setup, it runs eval() b.N times.
-func benchmarkBlockEvaluator(b *testing.B, inMem bool, withCrypto bool) {
+func benchmarkBlockEvaluator(b *testing.B, inMem bool, withCrypto bool, proto protocol.ConsensusVersion, txnSource BenchTxnGenerator) {
deadlockDisable := deadlock.Opts.Disable
deadlock.Opts.Disable = true
defer func() { deadlock.Opts.Disable = deadlockDisable }()
start := time.Now()
- genesisInitState, addrs, keys := genesis(100000)
+ genesisInitState, addrs, keys := genesisWithProto(100000, proto)
dbName := fmt.Sprintf("%s.%d", b.Name(), crypto.RandUint64())
- proto := config.Consensus[genesisInitState.Block.CurrentProtocol]
- proto.MaxTxnBytesPerBlock = 1000000000 // very big, no limit
- config.Consensus[protocol.ConsensusVersion(dbName)] = proto
+ cparams := config.Consensus[genesisInitState.Block.CurrentProtocol]
+ cparams.MaxTxnBytesPerBlock = 1000000000 // very big, no limit
+ config.Consensus[protocol.ConsensusVersion(dbName)] = cparams
genesisInitState.Block.CurrentProtocol = protocol.ConsensusVersion(dbName)
cfg := config.GetDefaultLocal()
cfg.Archival = true
@@ -753,10 +910,6 @@ func benchmarkBlockEvaluator(b *testing.B, inMem bool, withCrypto bool) {
require.NoError(b, err)
defer testLedgerCleanup(l2, dbName2, inMem)
- setupDone := time.Now()
- setupTime := setupDone.Sub(start)
- b.Logf("BenchmarkBlockEvaluator setup time %s", setupTime.String())
-
bepprof := os.Getenv("BLOCK_EVAL_PPROF")
if len(bepprof) > 0 {
profpath := dbName + "_cpuprof"
@@ -773,38 +926,83 @@ func benchmarkBlockEvaluator(b *testing.B, inMem bool, withCrypto bool) {
}()
}
- // test speed of block building
newBlock := bookkeeping.MakeBlock(genesisInitState.Block.BlockHeader)
bev, err := l.StartEvaluator(newBlock.BlockHeader, 0)
require.NoError(b, err)
genHash := genesisInitState.Block.BlockHeader.GenesisHash
+ backlogPool := execpool.MakeBacklog(nil, 0, execpool.LowPriority, nil)
+ defer backlogPool.Shutdown()
+
+ // apply initialization transations if any
+ initSignedTxns, maxTxnPerBlock := txnSource.Prepare(b, addrs, keys, newBlock.Round(), genHash)
+ if len(initSignedTxns) > 0 {
+ // all init transactions need to be written to ledger before reopening and benchmarking
+ for _, l := range []*Ledger{l, l2} {
+ l.accts.ctxCancel() // force commitSyncer to exit
+
+ // wait commitSyncer to exit
+ // the test calls commitRound directly and does not need commitSyncer/committedUpTo
+ select {
+ case <-l.accts.commitSyncerClosed:
+ break
+ }
+ }
+
+ var numBlocks uint64 = 0
+ var validatedBlock *ValidatedBlock
+
+ // there are might more transactions than MaxTxnBytesPerBlock allows
+ // so make smaller blocks to fit
+ for i, stxn := range initSignedTxns {
+ err = bev.Transaction(stxn, transactions.ApplyData{})
+ require.NoError(b, err)
+ if maxTxnPerBlock > 0 && i%maxTxnPerBlock == 0 || i == len(initSignedTxns)-1 {
+ validatedBlock, err = bev.GenerateBlock()
+ require.NoError(b, err)
+ for _, l := range []*Ledger{l, l2} {
+ err = l.AddValidatedBlock(*validatedBlock, agreement.Certificate{})
+ require.NoError(b, err)
+ }
+ newBlock = bookkeeping.MakeBlock(validatedBlock.blk.BlockHeader)
+ bev, err = l.StartEvaluator(newBlock.BlockHeader, 0)
+ require.NoError(b, err)
+ numBlocks++
+ }
+ }
+
+ // wait until everying is written and then reload ledgers in order
+ // to start reading accounts from DB and not from caches/deltas
+ var wg sync.WaitGroup
+ for _, l := range []*Ledger{l, l2} {
+ wg.Add(1)
+ // committing might take a long time, do it parallel
+ go func(l *Ledger) {
+ l.accts.accountsWriting.Add(1)
+ l.accts.commitRound(numBlocks, 0, 0)
+ l.accts.accountsWriting.Wait()
+ l.reloadLedger()
+ wg.Done()
+ }(l)
+ }
+ wg.Wait()
+
+ newBlock = bookkeeping.MakeBlock(validatedBlock.blk.BlockHeader)
+ bev, err = l.StartEvaluator(newBlock.BlockHeader, 0)
+ require.NoError(b, err)
+ }
+
+ setupDone := time.Now()
+ setupTime := setupDone.Sub(start)
+ b.Logf("BenchmarkBlockEvaluator setup time %s", setupTime.String())
+
+ // test speed of block building
numTxns := 50000
for i := 0; i < numTxns; i++ {
- sender := i % len(addrs)
- receiver := (i + 1) % len(addrs)
- // The following would create more random selection of accounts, and prevent a cache of half of the accounts..
- // iDigest := crypto.Hash([]byte{byte(i), byte(i >> 8), byte(i >> 16), byte(i >> 24)})
- // sender := (uint64(iDigest[0]) + uint64(iDigest[1])*256 + uint64(iDigest[2])*256*256) % uint64(len(addrs))
- // receiver := (uint64(iDigest[4]) + uint64(iDigest[5])*256 + uint64(iDigest[6])*256*256) % uint64(len(addrs))
- txn := transactions.Transaction{
- Type: protocol.PaymentTx,
- Header: transactions.Header{
- Sender: addrs[sender],
- Fee: minFee,
- FirstValid: newBlock.Round(),
- LastValid: newBlock.Round(),
- GenesisHash: genHash,
- },
- PaymentTxnFields: transactions.PaymentTxnFields{
- Receiver: addrs[receiver],
- Amount: basics.MicroAlgos{Raw: 100},
- },
- }
- st := txn.Sign(keys[sender])
- err = bev.Transaction(st, transactions.ApplyData{})
+ stxn := txnSource.Txn(b, addrs, keys, newBlock.Round(), genHash)
+ err = bev.Transaction(stxn, transactions.ApplyData{})
require.NoError(b, err)
}
@@ -815,15 +1013,13 @@ func benchmarkBlockEvaluator(b *testing.B, inMem bool, withCrypto bool) {
blockBuildTime := blockBuildDone.Sub(setupDone)
b.ReportMetric(float64(blockBuildTime)/float64(numTxns), "ns/block_build_tx")
- l.AddValidatedBlock(*validatedBlock, agreement.Certificate{})
+ err = l.AddValidatedBlock(*validatedBlock, agreement.Certificate{})
+ require.NoError(b, err)
avbDone := time.Now()
avbTime := avbDone.Sub(blockBuildDone)
b.ReportMetric(float64(avbTime)/float64(numTxns), "ns/AddValidatedBlock_tx")
- backlogPool := execpool.MakeBacklog(nil, 0, execpool.LowPriority, nil)
- defer backlogPool.Shutdown()
-
// test speed of block validation
// This should be the same as the eval line in ledger.go AddBlock()
// This is pulled out to isolate eval() time from db ops of AddValidatedBlock()
@@ -856,7 +1052,9 @@ func TestCowCompactCert(t *testing.T) {
blocks := make(map[basics.Round]bookkeeping.BlockHeader)
blockErr := make(map[basics.Round]error)
ml := mockLedger{balanceMap: accts0, blocks: blocks, blockErr: blockErr}
- c0 := makeRoundCowState(&ml, bookkeeping.BlockHeader{}, 0, 0)
+ c0 := makeRoundCowState(
+ &ml, bookkeeping.BlockHeader{}, config.Consensus[protocol.ConsensusCurrentVersion],
+ 0, 0)
certType = protocol.CompactCertType(1234) // bad cert type
err := c0.compactCert(certRnd, certType, cert, atRound, validate)
@@ -1040,7 +1238,7 @@ func TestModifiedAssetHoldings(t *testing.T) {
createTxn := txntest.Txn{
Type: "acfg",
Sender: addrs[0],
- Fee: basics.MicroAlgos{Raw: 2000},
+ Fee: 2000,
AssetParams: basics.AssetParams{
Total: 3,
Decimals: 0,
@@ -1054,7 +1252,7 @@ func TestModifiedAssetHoldings(t *testing.T) {
optInTxn := txntest.Txn{
Type: "axfer",
Sender: addrs[1],
- Fee: basics.MicroAlgos{Raw: 2000},
+ Fee: 2000,
XferAsset: assetid,
AssetAmount: 0,
AssetReceiver: addrs[1],
@@ -1086,7 +1284,7 @@ func TestModifiedAssetHoldings(t *testing.T) {
optOutTxn := txntest.Txn{
Type: "axfer",
Sender: addrs[1],
- Fee: basics.MicroAlgos{Raw: 1000},
+ Fee: 1000,
XferAsset: assetid,
AssetReceiver: addrs[0],
AssetCloseTo: addrs[0],
@@ -1095,7 +1293,7 @@ func TestModifiedAssetHoldings(t *testing.T) {
closeTxn := txntest.Txn{
Type: "acfg",
Sender: addrs[0],
- Fee: basics.MicroAlgos{Raw: 1000},
+ Fee: 1000,
ConfigAsset: assetid,
}
@@ -1164,7 +1362,9 @@ func newTestGenesis() (bookkeeping.GenesisBalances, []basics.Address, []*crypto.
Status: basics.NotParticipating,
}
- accts[rewards] = basics.AccountData{MicroAlgos: basics.MicroAlgos{Raw: amount}}
+ accts[rewards] = basics.AccountData{
+ MicroAlgos: basics.MicroAlgos{Raw: amount},
+ }
genBalances := bookkeeping.MakeGenesisBalances(accts, sink, rewards)
@@ -1174,22 +1374,31 @@ func newTestGenesis() (bookkeeping.GenesisBalances, []basics.Address, []*crypto.
// newTestLedger creates a in memory Ledger that is as realistic as
// possible. It has Rewards and FeeSink properly configured.
func newTestLedger(t testing.TB, balances bookkeeping.GenesisBalances) *Ledger {
+ l, _, _ := newTestLedgerImpl(t, balances, true)
+ return l
+}
+
+func newTestLedgerOnDisk(t testing.TB, balances bookkeeping.GenesisBalances) (*Ledger, string, bookkeeping.Block) {
+ return newTestLedgerImpl(t, balances, false)
+}
+
+func newTestLedgerImpl(t testing.TB, balances bookkeeping.GenesisBalances, inMem bool) (*Ledger, string, bookkeeping.Block) {
var genHash crypto.Digest
crypto.RandBytes(genHash[:])
genBlock, err := bookkeeping.MakeGenesisBlock(protocol.ConsensusFuture,
balances, "test", genHash)
-
+ require.False(t, genBlock.FeeSink.IsZero())
+ require.False(t, genBlock.RewardsPool.IsZero())
dbName := fmt.Sprintf("%s.%d", t.Name(), crypto.RandUint64())
cfg := config.GetDefaultLocal()
cfg.Archival = true
- const inMem = true
l, err := OpenLedger(logging.Base(), dbName, inMem, InitState{
Block: genBlock,
Accounts: balances.Balances,
GenesisHash: genHash,
}, cfg)
require.NoError(t, err)
- return l
+ return l, dbName, genBlock
}
// nextBlock begins evaluation of a new block, after ledger creation or endBlock()
@@ -1197,8 +1406,9 @@ func (ledger *Ledger) nextBlock(t testing.TB) *BlockEvaluator {
rnd := ledger.Latest()
hdr, err := ledger.BlockHdr(rnd)
require.NoError(t, err)
- eval, err := startEvaluator(ledger, bookkeeping.MakeBlock(hdr).BlockHeader,
- config.Consensus[hdr.CurrentProtocol], 0, false, true)
+
+ nextHdr := bookkeeping.MakeBlock(hdr).BlockHeader
+ eval, err := ledger.StartEvaluator(nextHdr, 0)
require.NoError(t, err)
return eval
}
@@ -1212,7 +1422,7 @@ func (ledger *Ledger) endBlock(t testing.TB, eval *BlockEvaluator) *ValidatedBlo
return validatedBlock
}
-// lookup gets the current accountdaa for an address
+// lookup gets the current accountdata for an address
func (ledger *Ledger) lookup(t testing.TB, addr basics.Address) basics.AccountData {
rnd := ledger.Latest()
ad, err := ledger.Lookup(rnd, addr)
@@ -1220,7 +1430,35 @@ func (ledger *Ledger) lookup(t testing.TB, addr basics.Address) basics.AccountDa
return ad
}
-func (eval *BlockEvaluator) txn(t testing.TB, txn *txntest.Txn) {
+// micros gets the current microAlgo balance for an address
+func (ledger *Ledger) micros(t testing.TB, addr basics.Address) uint64 {
+ return ledger.lookup(t, addr).MicroAlgos.Raw
+}
+
+// asa gets the current balance and optin status for some asa for an address
+func (ledger *Ledger) asa(t testing.TB, addr basics.Address, asset basics.AssetIndex) (uint64, bool) {
+ if holding, ok := ledger.lookup(t, addr).Assets[asset]; ok {
+ return holding.Amount, true
+ }
+ return 0, false
+}
+
+// asaParams gets the asset params for a given asa index
+func (ledger *Ledger) asaParams(t testing.TB, asset basics.AssetIndex) (basics.AssetParams, error) {
+ creator, ok, err := ledger.GetCreator(basics.CreatableIndex(asset), basics.AssetCreatable)
+ if err != nil {
+ return basics.AssetParams{}, err
+ }
+ if !ok {
+ return basics.AssetParams{}, fmt.Errorf("no asset (%d)", asset)
+ }
+ if params, ok := ledger.lookup(t, creator).AssetParams[asset]; ok {
+ return params, nil
+ }
+ return basics.AssetParams{}, fmt.Errorf("bad lookup (%d)", asset)
+}
+
+func (eval *BlockEvaluator) fillDefaults(txn *txntest.Txn) {
if txn.GenesisHash.IsZero() {
txn.GenesisHash = eval.genesisHash
}
@@ -1228,19 +1466,56 @@ func (eval *BlockEvaluator) txn(t testing.TB, txn *txntest.Txn) {
txn.FirstValid = eval.Round()
}
txn.FillDefaults(eval.proto)
+}
+
+func (eval *BlockEvaluator) txn(t testing.TB, txn *txntest.Txn, problem ...string) {
+ t.Helper()
+ eval.fillDefaults(txn)
stxn := txn.SignedTxn()
err := eval.testTransaction(stxn, eval.state.child(1))
- require.NoError(t, err)
- eval.Transaction(stxn, transactions.ApplyData{})
- require.NoError(t, err)
+ if err != nil {
+ if len(problem) == 1 {
+ require.Contains(t, err.Error(), problem[0])
+ } else {
+ require.NoError(t, err) // Will obviously fail
+ }
+ return
+ }
+ err = eval.Transaction(stxn, transactions.ApplyData{})
+ if err != nil {
+ if len(problem) == 1 {
+ require.Contains(t, err.Error(), problem[0])
+ } else {
+ require.NoError(t, err) // Will obviously fail
+ }
+ return
+ }
+ require.Len(t, problem, 0)
}
func (eval *BlockEvaluator) txns(t testing.TB, txns ...*txntest.Txn) {
+ t.Helper()
for _, txn := range txns {
eval.txn(t, txn)
}
}
+func (eval *BlockEvaluator) txgroup(t testing.TB, txns ...*txntest.Txn) error {
+ t.Helper()
+ for _, txn := range txns {
+ eval.fillDefaults(txn)
+ }
+ txgroup := txntest.SignedTxns(txns...)
+
+ err := eval.TestTransactionGroup(txgroup)
+ if err != nil {
+ return err
+ }
+
+ err = eval.transactionGroup(transactions.WrapSignedTxnsWithAD(txgroup))
+ return err
+}
+
func TestRewardsInAD(t *testing.T) {
partitiontest.PartitionTest(t)
@@ -1345,10 +1620,9 @@ func TestModifiedAppLocalStates(t *testing.T) {
const appid basics.AppIndex = 1
createTxn := txntest.Txn{
- Type: "appl",
- Sender: addrs[0],
- ApprovalProgram: []byte{0x02, 0x20, 0x01, 0x01, 0x22},
- ClearStateProgram: []byte{0x02, 0x20, 0x01, 0x01, 0x22},
+ Type: "appl",
+ Sender: addrs[0],
+ ApprovalProgram: "int 1",
}
optInTxn := txntest.Txn{
@@ -1402,3 +1676,156 @@ func TestModifiedAppLocalStates(t *testing.T) {
assert.False(t, created)
}
}
+
+// Test that overriding the consensus parameters effects the generated apply data.
+func TestCustomProtocolParams(t *testing.T) {
+ partitiontest.PartitionTest(t)
+
+ genesisBalances, addrs, _ := newTestGenesis()
+
+ var genHash crypto.Digest
+ crypto.RandBytes(genHash[:])
+ block, err := bookkeeping.MakeGenesisBlock(protocol.ConsensusV24,
+ genesisBalances, "test", genHash)
+
+ dbName := fmt.Sprintf("%s", t.Name())
+ cfg := config.GetDefaultLocal()
+ cfg.Archival = true
+ l, err := OpenLedger(logging.Base(), dbName, true, InitState{
+ Block: block,
+ Accounts: genesisBalances.Balances,
+ GenesisHash: genHash,
+ }, cfg)
+ require.NoError(t, err)
+ defer l.Close()
+
+ const assetid basics.AssetIndex = 1
+ proto := config.Consensus[protocol.ConsensusV24]
+
+ block = bookkeeping.MakeBlock(block.BlockHeader)
+
+ createTxn := txntest.Txn{
+ Type: "acfg",
+ Sender: addrs[0],
+ GenesisHash: block.GenesisHash(),
+ AssetParams: basics.AssetParams{
+ Total: 200,
+ Decimals: 0,
+ Manager: addrs[0],
+ Reserve: addrs[0],
+ Freeze: addrs[0],
+ Clawback: addrs[0],
+ },
+ }
+ createTxn.FillDefaults(proto)
+ createStib, err := block.BlockHeader.EncodeSignedTxn(
+ createTxn.SignedTxn(), transactions.ApplyData{})
+ require.NoError(t, err)
+
+ optInTxn := txntest.Txn{
+ Type: "axfer",
+ Sender: addrs[1],
+ GenesisHash: block.GenesisHash(),
+ XferAsset: assetid,
+ AssetAmount: 0,
+ AssetReceiver: addrs[1],
+ }
+ optInTxn.FillDefaults(proto)
+ optInStib, err := block.BlockHeader.EncodeSignedTxn(
+ optInTxn.SignedTxn(), transactions.ApplyData{})
+ require.NoError(t, err)
+
+ fundTxn := txntest.Txn{
+ Type: "axfer",
+ Sender: addrs[0],
+ GenesisHash: block.GenesisHash(),
+ XferAsset: assetid,
+ AssetAmount: 100,
+ AssetReceiver: addrs[1],
+ }
+ fundTxn.FillDefaults(proto)
+ fundStib, err := block.BlockHeader.EncodeSignedTxn(
+ fundTxn.SignedTxn(), transactions.ApplyData{})
+ require.NoError(t, err)
+
+ optOutTxn := txntest.Txn{
+ Type: "axfer",
+ Sender: addrs[1],
+ GenesisHash: block.GenesisHash(),
+ XferAsset: assetid,
+ AssetAmount: 30,
+ AssetReceiver: addrs[0],
+ AssetCloseTo: addrs[0],
+ }
+ optOutTxn.FillDefaults(proto)
+ optOutStib, err := block.BlockHeader.EncodeSignedTxn(
+ optOutTxn.SignedTxn(), transactions.ApplyData{})
+ require.NoError(t, err)
+
+ block.Payset = []transactions.SignedTxnInBlock{
+ createStib, optInStib, fundStib, optOutStib,
+ }
+
+ proto.EnableAssetCloseAmount = true
+ _, modifiedTxns, err := Eval(l, &block, proto)
+ require.NoError(t, err)
+
+ require.Equal(t, 4, len(modifiedTxns))
+ assert.Equal(t, uint64(70), modifiedTxns[3].AssetClosingAmount)
+}
+
+// TestAppInsMinBalance checks that accounts with MaxAppsOptedIn are accepted by block evaluator
+// and do not cause any MaximumMinimumBalance problems
+func TestAppInsMinBalance(t *testing.T) {
+ partitiontest.PartitionTest(t)
+
+ genBalances, addrs, _ := newTestGenesis()
+ l := newTestLedger(t, genBalances)
+ defer l.Close()
+
+ const appid basics.AppIndex = 1
+
+ maxAppsOptedIn := config.Consensus[protocol.ConsensusFuture].MaxAppsOptedIn
+ require.Greater(t, maxAppsOptedIn, 0)
+ maxAppsCreated := config.Consensus[protocol.ConsensusFuture].MaxAppsCreated
+ require.Greater(t, maxAppsCreated, 0)
+ maxLocalSchemaEntries := config.Consensus[protocol.ConsensusFuture].MaxLocalSchemaEntries
+ require.Greater(t, maxLocalSchemaEntries, uint64(0))
+
+ txnsCreate := make([]*txntest.Txn, 0, maxAppsOptedIn)
+ txnsOptIn := make([]*txntest.Txn, 0, maxAppsOptedIn)
+ appsCreated := make(map[basics.Address]int, len(addrs)-1)
+
+ acctIdx := 0
+ for i := 0; i < maxAppsOptedIn; i++ {
+ creator := addrs[acctIdx]
+ createTxn := txntest.Txn{
+ Type: protocol.ApplicationCallTx,
+ Sender: creator,
+ ApprovalProgram: "int 1",
+ LocalStateSchema: basics.StateSchema{NumByteSlice: maxLocalSchemaEntries},
+ Note: randomNote(),
+ }
+ txnsCreate = append(txnsCreate, &createTxn)
+ count := appsCreated[creator]
+ count++
+ appsCreated[creator] = count
+ if count == maxAppsCreated {
+ acctIdx++
+ }
+
+ optInTxn := txntest.Txn{
+ Type: protocol.ApplicationCallTx,
+ Sender: addrs[9],
+ ApplicationID: appid + basics.AppIndex(i),
+ OnCompletion: transactions.OptInOC,
+ }
+ txnsOptIn = append(txnsOptIn, &optInTxn)
+ }
+
+ eval := l.nextBlock(t)
+ txns := append(txnsCreate, txnsOptIn...)
+ eval.txns(t, txns...)
+ vb := l.endBlock(t, eval)
+ assert.Len(t, vb.delta.ModifiedAppLocalStates, 50)
+}