summaryrefslogtreecommitdiff
path: root/ledger/eval.go
diff options
context:
space:
mode:
Diffstat (limited to 'ledger/eval.go')
-rw-r--r--ledger/eval.go172
1 files changed, 110 insertions, 62 deletions
diff --git a/ledger/eval.go b/ledger/eval.go
index fbd287c5c..5730a4f22 100644
--- a/ledger/eval.go
+++ b/ledger/eval.go
@@ -41,6 +41,9 @@ import (
// ErrNoSpace indicates insufficient space for transaction in block
var ErrNoSpace = errors.New("block does not have space for transaction")
+// ErrRoundZero is self-explanatory
+var ErrRoundZero = errors.New("cannot start evaluator for round 0")
+
// maxPaysetHint makes sure that we don't allocate too much memory up front
// in the block evaluator, since there cannot reasonably be more than this
// many transactions in a block.
@@ -348,6 +351,7 @@ type BlockEvaluator struct {
block bookkeeping.Block
blockTxBytes int
+ specials transactions.SpecialAddresses
blockGenerated bool // prevent repeated GenerateBlock calls
@@ -383,6 +387,21 @@ func (l *Ledger) StartEvaluator(hdr bookkeeping.BlockHeader, paysetHint int) (*B
}
func startEvaluator(l ledgerForEvaluator, hdr bookkeeping.BlockHeader, proto config.ConsensusParams, paysetHint int, validate bool, generate bool) (*BlockEvaluator, error) {
+ if hdr.Round == 0 {
+ return nil, ErrRoundZero
+ }
+
+ prevHeader, err := l.BlockHdr(hdr.Round - 1)
+ if err != nil {
+ return nil, fmt.Errorf(
+ "can't evaluate block %d without previous header: %v", hdr.Round, err)
+ }
+
+ prevProto, ok := config.Consensus[prevHeader.CurrentProtocol]
+ if !ok {
+ return nil, protocol.Error(prevHeader.CurrentProtocol)
+ }
+
base := &roundCowBase{
l: l,
// round that lookups come from is previous block. We validate
@@ -390,14 +409,20 @@ func startEvaluator(l ledgerForEvaluator, hdr bookkeeping.BlockHeader, proto con
// If we are not validating, we must have previously checked
// an agreement.Certificate attesting that hdr is valid.
rnd: hdr.Round - 1,
+ txnCount: prevHeader.TxnCounter,
proto: proto,
accounts: make(map[basics.Address]basics.AccountData),
}
eval := &BlockEvaluator{
- validate: validate,
- generate: generate,
- block: bookkeeping.Block{BlockHeader: hdr},
+ validate: validate,
+ generate: generate,
+ prevHeader: prevHeader,
+ block: bookkeeping.Block{BlockHeader: hdr},
+ specials: transactions.SpecialAddresses{
+ FeeSink: hdr.FeeSink,
+ RewardsPool: hdr.RewardsPool,
+ },
proto: proto,
genesisHash: l.GenesisHash(),
l: l,
@@ -412,35 +437,18 @@ func startEvaluator(l ledgerForEvaluator, hdr bookkeeping.BlockHeader, proto con
eval.block.Payset = make([]transactions.SignedTxnInBlock, 0, paysetHint)
}
- prevProto := proto
+ base.compactCertNextRnd = eval.prevHeader.CompactCert[protocol.CompactCertBasic].CompactCertNextRound
- if hdr.Round > 0 {
- var err error
- eval.prevHeader, err = l.BlockHdr(base.rnd)
- if err != nil {
- return nil, fmt.Errorf("can't evaluate block %v without previous header: %v", hdr.Round, err)
- }
-
- base.txnCount = eval.prevHeader.TxnCounter
- base.compactCertNextRnd = eval.prevHeader.CompactCert[protocol.CompactCertBasic].CompactCertNextRound
-
- var ok bool
- prevProto, ok = config.Consensus[eval.prevHeader.CurrentProtocol]
- if !ok {
- return nil, protocol.Error(eval.prevHeader.CurrentProtocol)
- }
-
- // Check if compact certs are being enabled as of this block.
- if base.compactCertNextRnd == 0 && proto.CompactCertRounds != 0 {
- // Determine the first block that will contain a Merkle
- // commitment to the voters. We need to account for the
- // fact that the voters come from CompactCertVotersLookback
- // rounds ago.
- votersRound := (hdr.Round + basics.Round(proto.CompactCertVotersLookback)).RoundUpToMultipleOf(basics.Round(proto.CompactCertRounds))
+ // Check if compact certs are being enabled as of this block.
+ if base.compactCertNextRnd == 0 && proto.CompactCertRounds != 0 {
+ // Determine the first block that will contain a Merkle
+ // commitment to the voters. We need to account for the
+ // fact that the voters come from CompactCertVotersLookback
+ // rounds ago.
+ votersRound := (hdr.Round + basics.Round(proto.CompactCertVotersLookback)).RoundUpToMultipleOf(basics.Round(proto.CompactCertRounds))
- // The first compact cert will appear CompactCertRounds after that.
- base.compactCertNextRnd = votersRound + basics.Round(proto.CompactCertRounds)
- }
+ // The first compact cert will appear CompactCertRounds after that.
+ base.compactCertNextRnd = votersRound + basics.Round(proto.CompactCertRounds)
}
prevTotals, err := l.Totals(eval.prevHeader.Round)
@@ -465,7 +473,7 @@ func startEvaluator(l ledgerForEvaluator, hdr bookkeeping.BlockHeader, proto con
eval.block.BlockHeader.RewardsState = eval.prevHeader.NextRewardsState(hdr.Round, proto, incentivePoolData.MicroAlgos, prevTotals.RewardUnits())
}
// set the eval state with the current header
- eval.state = makeRoundCowState(base, eval.block.BlockHeader, eval.prevHeader.TimeStamp, paysetHint)
+ eval.state = makeRoundCowState(base, eval.block.BlockHeader, proto, eval.prevHeader.TimeStamp, paysetHint)
if validate {
err := eval.block.BlockHeader.PreCheck(eval.prevHeader)
@@ -551,8 +559,8 @@ func (eval *BlockEvaluator) workaroundOverspentRewards(rewardPoolBalance basics.
return
}
-// TxnCounter returns the number of transactions that have been added to the block evaluator so far.
-func (eval *BlockEvaluator) TxnCounter() int {
+// PaySetSize returns the number of top-level transactions that have been added to the block evaluator so far.
+func (eval *BlockEvaluator) PaySetSize() int {
return len(eval.block.Payset)
}
@@ -627,12 +635,7 @@ func (eval *BlockEvaluator) testTransaction(txn transactions.SignedTxn, cow *rou
return err
}
- // Well-formed on its own?
- spec := transactions.SpecialAddresses{
- FeeSink: eval.block.BlockHeader.FeeSink,
- RewardsPool: eval.block.BlockHeader.RewardsPool,
- }
- err = txn.Txn.WellFormed(spec, eval.proto)
+ err = txn.Txn.WellFormed(eval.specials, eval.proto)
if err != nil {
return fmt.Errorf("transaction %v: malformed: %v", txn.ID(), err)
}
@@ -668,12 +671,13 @@ func (eval *BlockEvaluator) TransactionGroup(txads []transactions.SignedTxnWithA
// prepareEvalParams creates a logic.EvalParams for each ApplicationCall
// transaction in the group
-func (eval *BlockEvaluator) prepareEvalParams(txgroup []transactions.SignedTxnWithAD) (res []*logic.EvalParams) {
+func (eval *BlockEvaluator) prepareEvalParams(txgroup []transactions.SignedTxnWithAD) []*logic.EvalParams {
var groupNoAD []transactions.SignedTxn
var pastSideEffects []logic.EvalSideEffects
var minTealVersion uint64
pooledApplicationBudget := uint64(0)
- res = make([]*logic.EvalParams, len(txgroup))
+ var credit uint64
+ res := make([]*logic.EvalParams, len(txgroup))
for i, txn := range txgroup {
// Ignore any non-ApplicationCall transactions
if txn.SignedTxn.Txn.Type != protocol.ApplicationCallTx {
@@ -693,19 +697,23 @@ func (eval *BlockEvaluator) prepareEvalParams(txgroup []transactions.SignedTxnWi
}
pastSideEffects = logic.MakePastSideEffects(len(txgroup))
minTealVersion = logic.ComputeMinTealVersion(groupNoAD)
+ credit, _ = transactions.FeeCredit(groupNoAD, eval.proto.MinTxnFee)
+ // intentionally ignoring error here, fees had to have been enough to get here
}
res[i] = &logic.EvalParams{
Txn: &groupNoAD[i],
Proto: &eval.proto,
TxnGroup: groupNoAD,
- GroupIndex: i,
+ GroupIndex: uint64(i),
PastSideEffects: pastSideEffects,
MinTealVersion: &minTealVersion,
PooledApplicationBudget: &pooledApplicationBudget,
+ FeeCredit: &credit,
+ Specials: &eval.specials,
}
}
- return
+ return res
}
// transactionGroup tentatively executes a group of transactions as part of this block evaluation.
@@ -726,8 +734,6 @@ func (eval *BlockEvaluator) transactionGroup(txgroup []transactions.SignedTxnWit
var groupTxBytes int
cow := eval.state.child(len(txgroup))
-
- // Prepare eval params for any ApplicationCall transactions in the group
evalParams := eval.prepareEvalParams(txgroup)
// Evaluate each transaction in the group
@@ -860,13 +866,8 @@ func (eval *BlockEvaluator) transaction(txn transactions.SignedTxn, evalParams *
}
}
- spec := transactions.SpecialAddresses{
- FeeSink: eval.block.BlockHeader.FeeSink,
- RewardsPool: eval.block.BlockHeader.RewardsPool,
- }
-
// Apply the transaction, updating the cow balances
- applyData, err := eval.applyTransaction(txn.Txn, cow, evalParams, spec, cow.txnCounter())
+ applyData, err := eval.applyTransaction(txn.Txn, cow, evalParams, cow.txnCounter())
if err != nil {
return fmt.Errorf("transaction %v: %v", txid, err)
}
@@ -903,6 +904,13 @@ func (eval *BlockEvaluator) transaction(txn transactions.SignedTxn, evalParams *
}
}
+ // We are not allowing InnerTxns to have InnerTxns yet. Error if that happens.
+ for _, itx := range applyData.EvalDelta.InnerTxns {
+ if len(itx.ApplyData.EvalDelta.InnerTxns) > 0 {
+ return fmt.Errorf("inner transaction has inner transactions %v", itx)
+ }
+ }
+
// Remember this txn
cow.addTx(txn.Txn, txid)
@@ -910,11 +918,11 @@ func (eval *BlockEvaluator) transaction(txn transactions.SignedTxn, evalParams *
}
// applyTransaction changes the balances according to this transaction.
-func (eval *BlockEvaluator) applyTransaction(tx transactions.Transaction, balances *roundCowState, evalParams *logic.EvalParams, spec transactions.SpecialAddresses, ctr uint64) (ad transactions.ApplyData, err error) {
+func (eval *BlockEvaluator) applyTransaction(tx transactions.Transaction, balances *roundCowState, evalParams *logic.EvalParams, ctr uint64) (ad transactions.ApplyData, err error) {
params := balances.ConsensusParams()
// move fee to pool
- err = balances.Move(tx.Sender, spec.FeeSink, tx.Fee, &ad.SenderRewards, nil)
+ err = balances.Move(tx.Sender, eval.specials.FeeSink, tx.Fee, &ad.SenderRewards, nil)
if err != nil {
return
}
@@ -942,19 +950,19 @@ func (eval *BlockEvaluator) applyTransaction(tx transactions.Transaction, balanc
switch tx.Type {
case protocol.PaymentTx:
- err = apply.Payment(tx.PaymentTxnFields, tx.Header, balances, spec, &ad)
+ err = apply.Payment(tx.PaymentTxnFields, tx.Header, balances, eval.specials, &ad)
case protocol.KeyRegistrationTx:
- err = apply.Keyreg(tx.KeyregTxnFields, tx.Header, balances, spec, &ad, balances.round())
+ err = apply.Keyreg(tx.KeyregTxnFields, tx.Header, balances, eval.specials, &ad, balances.round())
case protocol.AssetConfigTx:
- err = apply.AssetConfig(tx.AssetConfigTxnFields, tx.Header, balances, spec, &ad, ctr)
+ err = apply.AssetConfig(tx.AssetConfigTxnFields, tx.Header, balances, eval.specials, &ad, ctr)
case protocol.AssetTransferTx:
- err = apply.AssetTransfer(tx.AssetTransferTxnFields, tx.Header, balances, spec, &ad)
+ err = apply.AssetTransfer(tx.AssetTransferTxnFields, tx.Header, balances, eval.specials, &ad)
case protocol.AssetFreezeTx:
- err = apply.AssetFreeze(tx.AssetFreezeTxnFields, tx.Header, balances, spec, &ad)
+ err = apply.AssetFreeze(tx.AssetFreezeTxnFields, tx.Header, balances, eval.specials, &ad)
case protocol.ApplicationCallTx:
err = apply.ApplicationCall(tx.ApplicationCallTxnFields, tx.Header, balances, &ad, evalParams, ctr)
@@ -1116,7 +1124,14 @@ func (eval *BlockEvaluator) GenerateBlock() (*ValidatedBlock, error) {
delta: eval.state.deltas(),
}
eval.blockGenerated = true
- eval.state = makeRoundCowState(eval.state, eval.block.BlockHeader, eval.prevHeader.TimeStamp, len(eval.block.Payset))
+ proto, ok := config.Consensus[eval.block.BlockHeader.CurrentProtocol]
+ if !ok {
+ return nil, fmt.Errorf(
+ "unknown consensus version: %s", eval.block.BlockHeader.CurrentProtocol)
+ }
+ eval.state = makeRoundCowState(
+ eval.state, eval.block.BlockHeader, proto, eval.prevHeader.TimeStamp,
+ len(eval.block.Payset))
return &vb, nil
}
@@ -1275,7 +1290,7 @@ transactionGroupLoop:
return eval.state.deltas(), nil
}
-// loadedTransactionGroup is a helper struct to allow asyncronious loading of the account data needed by the transaction groups
+// loadedTransactionGroup is a helper struct to allow asynchronous loading of the account data needed by the transaction groups
type loadedTransactionGroup struct {
// group is the transaction group
group []transactions.SignedTxnWithAD
@@ -1286,6 +1301,21 @@ type loadedTransactionGroup struct {
err error
}
+// Return the maximum number of addresses referenced in any given transaction.
+func maxAddressesInTxn(proto *config.ConsensusParams) int {
+ return 7 + proto.MaxAppTxnAccounts
+}
+
+// Write the list of addresses referenced in `txn` to `out`. Addresses might repeat.
+func getTxnAddresses(txn *transactions.Transaction, out *[]basics.Address) {
+ *out = (*out)[:0]
+
+ *out = append(
+ *out, txn.Sender, txn.Receiver, txn.CloseRemainderTo, txn.AssetSender,
+ txn.AssetReceiver, txn.AssetCloseTo, txn.FreezeAccount)
+ *out = append(*out, txn.ApplicationCallTxnFields.Accounts...)
+}
+
// loadAccounts loads the account data for the provided transaction group list. It also loads the feeSink account and add it to the first returned transaction group.
// The order of the transaction groups returned by the channel is identical to the one in the input array.
func loadAccounts(ctx context.Context, l ledgerForEvaluator, rnd basics.Round, groups [][]transactions.SignedTxnWithAD, feeSinkAddr basics.Address, consensusParams config.ConsensusParams) chan loadedTransactionGroup {
@@ -1312,8 +1342,7 @@ func loadAccounts(ctx context.Context, l ledgerForEvaluator, rnd basics.Round, g
defer close(outChan)
accountTasks := make(map[basics.Address]*addrTask)
- maxAddressesPerTransaction := 7 + consensusParams.MaxAppTxnAccounts
- addressesCh := make(chan *addrTask, len(groups)*consensusParams.MaxTxGroupSize*maxAddressesPerTransaction)
+ addressesCh := make(chan *addrTask, len(groups)*consensusParams.MaxTxGroupSize*maxAddressesInTxn(&consensusParams))
// totalBalances counts the total number of balances over all the transaction groups
totalBalances := 0
@@ -1354,6 +1383,7 @@ func loadAccounts(ctx context.Context, l ledgerForEvaluator, rnd basics.Round, g
task := &groupTask{}
groupsReady[i] = task
for _, stxn := range group {
+ // If you add new addresses here, also add them in getTxnAddresses().
initAccount(stxn.Txn.Sender, task)
initAccount(stxn.Txn.Receiver, task)
initAccount(stxn.Txn.CloseRemainderTo, task)
@@ -1495,6 +1525,24 @@ func (vb ValidatedBlock) WithSeed(s committee.Seed) ValidatedBlock {
}
}
+// GetBlockAddresses returns all addresses referenced in `block`.
+func GetBlockAddresses(block *bookkeeping.Block) map[basics.Address]struct{} {
+ // Reserve a reasonable memory size for the map.
+ res := make(map[basics.Address]struct{}, len(block.Payset)+2)
+ res[block.FeeSink] = struct{}{}
+ res[block.RewardsPool] = struct{}{}
+
+ var refAddresses []basics.Address
+ for _, stib := range block.Payset {
+ getTxnAddresses(&stib.Txn, &refAddresses)
+ for _, address := range refAddresses {
+ res[address] = struct{}{}
+ }
+ }
+
+ return res
+}
+
// Eval evaluates a block without validation using the given `proto`. Return the state
// delta and transactions with modified apply data according to `proto`.
// This function is used by Indexer which modifies `proto` to retrieve the asset