summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorJohn Jannotti <jannotti@gmail.com>2022-08-09 11:52:08 -0400
committerGitHub <noreply@github.com>2022-08-09 11:52:08 -0400
commit566be49a6e76816b392058a7f27c4ffca296a877 (patch)
tree8696a377e6c657c6981f45abe88e97cc41084047
parent2bc55c06f4dacf247b93a15f47530f57b4925d28 (diff)
AVM: Make `txn FirstValidTime` and `block` opcode available in logicsigs (#4371)algobarb/566be49
-rw-r--r--cmd/goal/clerk.go5
-rw-r--r--cmd/tealdbg/debugger_test.go1
-rw-r--r--cmd/tealdbg/local.go1
-rw-r--r--daemon/algod/api/server/v2/dryrun.go1
-rw-r--r--data/transactions/logic/assembler.go2
-rw-r--r--data/transactions/logic/eval.go40
-rw-r--r--data/transactions/logic/evalStateful_test.go24
-rw-r--r--data/transactions/logic/eval_test.go12
-rw-r--r--data/transactions/verify/txn.go18
-rw-r--r--data/transactions/verify/txn_test.go22
-rw-r--r--data/transactions/verify/verifiedTxnCache_test.go12
-rw-r--r--data/txHandler.go4
-rw-r--r--ledger/applications_test.go67
-rw-r--r--ledger/internal/eval.go4
-rw-r--r--node/node.go2
-rwxr-xr-xtest/scripts/e2e_subs/hdr-access-logicsig.sh63
-rwxr-xr-xtest/scripts/e2e_subs/hdr-access.py41
17 files changed, 250 insertions, 69 deletions
diff --git a/cmd/goal/clerk.go b/cmd/goal/clerk.go
index 75dc372c2..856662147 100644
--- a/cmd/goal/clerk.go
+++ b/cmd/goal/clerk.go
@@ -424,7 +424,7 @@ var sendCmd = &cobra.Command{
CurrentProtocol: proto,
},
}
- groupCtx, err := verify.PrepareGroupContext([]transactions.SignedTxn{uncheckedTxn}, blockHeader)
+ groupCtx, err := verify.PrepareGroupContext([]transactions.SignedTxn{uncheckedTxn}, blockHeader, nil)
if err == nil {
err = verify.LogicSigSanityCheck(&uncheckedTxn, 0, groupCtx)
}
@@ -825,7 +825,7 @@ var signCmd = &cobra.Command{
}
var groupCtx *verify.GroupContext
if lsig.Logic != nil {
- groupCtx, err = verify.PrepareGroupContext(txnGroup, contextHdr)
+ groupCtx, err = verify.PrepareGroupContext(txnGroup, contextHdr, nil)
if err != nil {
// this error has to be unsupported protocol
reportErrorf("%s: %v", txFilename, err)
@@ -1162,6 +1162,7 @@ var dryrunCmd = &cobra.Command{
reportErrorf("program size too large: %d > %d", len(txn.Lsig.Logic), params.LogicSigMaxSize)
}
ep := logic.NewEvalParams(txgroup, &params, nil)
+ ep.SigLedger = logic.NoHeaderLedger{}
err := logic.CheckSignature(i, ep)
if err != nil {
reportErrorf("program failed Check: %s", err)
diff --git a/cmd/tealdbg/debugger_test.go b/cmd/tealdbg/debugger_test.go
index 4a390d461..4f4b35ea2 100644
--- a/cmd/tealdbg/debugger_test.go
+++ b/cmd/tealdbg/debugger_test.go
@@ -103,6 +103,7 @@ func TestDebuggerSimple(t *testing.T) {
ep := logic.NewEvalParams(make([]transactions.SignedTxnWithAD, 1), &proto, nil)
ep.Debugger = debugger
+ ep.SigLedger = logic.NoHeaderLedger{}
source := `int 0
int 1
diff --git a/cmd/tealdbg/local.go b/cmd/tealdbg/local.go
index 8e92f66fd..c9cba4de3 100644
--- a/cmd/tealdbg/local.go
+++ b/cmd/tealdbg/local.go
@@ -545,6 +545,7 @@ func (r *LocalRunner) RunAll() error {
start := time.Now()
ep := logic.NewEvalParams(txngroup, &r.proto, &transactions.SpecialAddresses{})
+ ep.SigLedger = logic.NoHeaderLedger{}
configureDebugger(ep)
var last error
diff --git a/daemon/algod/api/server/v2/dryrun.go b/daemon/algod/api/server/v2/dryrun.go
index 4a27470ce..3079a71aa 100644
--- a/daemon/algod/api/server/v2/dryrun.go
+++ b/daemon/algod/api/server/v2/dryrun.go
@@ -418,6 +418,7 @@ func doDryrunRequest(dr *DryrunRequest, response *generated.DryrunResponse) {
if len(stxn.Lsig.Logic) > 0 {
var debug dryrunDebugReceiver
ep.Debugger = &debug
+ ep.SigLedger = &dl
pass, err := logic.EvalSignature(ti, ep)
var messages []string
result.Disassembly = debug.lines // Keep backwards compat
diff --git a/data/transactions/logic/assembler.go b/data/transactions/logic/assembler.go
index fcd56d2c7..e175a1703 100644
--- a/data/transactions/logic/assembler.go
+++ b/data/transactions/logic/assembler.go
@@ -2269,7 +2269,7 @@ func disassemble(dis *disassembleState, spec *OpSpec) (string, error) {
}
if strings.HasPrefix(spec.Name, "bytec_") {
b := spec.Name[len(spec.Name)-1] - byte('0')
- if int(b) < len(dis.intc) {
+ if int(b) < len(dis.bytec) {
out += fmt.Sprintf(" // %s", guessByteFormat(dis.bytec[b]))
}
}
diff --git a/data/transactions/logic/eval.go b/data/transactions/logic/eval.go
index 7334485d9..b648ff778 100644
--- a/data/transactions/logic/eval.go
+++ b/data/transactions/logic/eval.go
@@ -195,6 +195,23 @@ func ComputeMinAvmVersion(group []transactions.SignedTxnWithAD) uint64 {
return minVersion
}
+// LedgerForSignature represents the parts of Ledger that LogicSigs can see. It
+// only exposes things that consensus has already agreed upon, so it is
+// "stateless" for signature purposes.
+type LedgerForSignature interface {
+ BlockHdrCached(basics.Round) (bookkeeping.BlockHeader, error)
+}
+
+// NoHeaderLedger is intended for debugging situations in which it is reasonable
+// to preclude the use of `block` and `txn LastValidTime`
+type NoHeaderLedger struct {
+}
+
+// BlockHdrCached always errors
+func (NoHeaderLedger) BlockHdrCached(basics.Round) (bookkeeping.BlockHeader, error) {
+ return bookkeeping.BlockHeader{}, fmt.Errorf("no block header access")
+}
+
// LedgerForLogic represents ledger API for Stateful TEAL program
type LedgerForLogic interface {
AccountData(addr basics.Address) (ledgercore.AccountData, error)
@@ -239,7 +256,8 @@ type EvalParams struct {
logger logging.Logger
- Ledger LedgerForLogic
+ SigLedger LedgerForSignature
+ Ledger LedgerForLogic
// optional debugger
Debugger DebuggerHook
@@ -387,6 +405,7 @@ func NewInnerEvalParams(txg []transactions.SignedTxnWithAD, caller *EvalContext)
Specials: caller.Specials,
PooledApplicationBudget: caller.PooledApplicationBudget,
pooledAllowedInners: caller.pooledAllowedInners,
+ SigLedger: caller.SigLedger,
Ledger: caller.Ledger,
created: caller.created,
appAddrCache: caller.appAddrCache,
@@ -610,6 +629,9 @@ func EvalContract(program []byte, gi int, aid basics.AppIndex, params *EvalParam
if params.Ledger == nil {
return false, nil, errors.New("no ledger in contract eval")
}
+ if params.SigLedger == nil {
+ params.SigLedger = params.Ledger
+ }
if aid == 0 {
return false, nil, errors.New("0 appId in contract eval")
}
@@ -652,6 +674,9 @@ func EvalApp(program []byte, gi int, aid basics.AppIndex, params *EvalParams) (b
// EvalSignature evaluates the logicsig of the ith transaction in params.
// A program passes successfully if it finishes with one int element on the stack that is non-zero.
func EvalSignature(gi int, params *EvalParams) (pass bool, err error) {
+ if params.SigLedger == nil {
+ return false, errors.New("no sig ledger in signature eval")
+ }
cx := EvalContext{
EvalParams: params,
runModeFlags: modeSig,
@@ -2304,7 +2329,7 @@ func (cx *EvalContext) txnFieldToStack(stxn *transactions.SignedTxnWithAD, fs *t
if err != nil {
return sv, err
}
- hdr, err := cx.Ledger.BlockHdrCached(rnd)
+ hdr, err := cx.SigLedger.BlockHdrCached(rnd)
if err != nil {
return sv, err
}
@@ -4848,10 +4873,13 @@ func (cx *EvalContext) availableRound(r uint64) (basics.Round, error) {
if firstAvail > cx.txn.Txn.LastValid || firstAvail == 0 { // early in chain's life
firstAvail = 1
}
- current := cx.Ledger.Round()
+ lastAvail := cx.txn.Txn.FirstValid - 1
+ if lastAvail > cx.txn.Txn.FirstValid { // txn had a 0 in FirstValid
+ lastAvail = 0 // So nothing will be available
+ }
round := basics.Round(r)
- if round < firstAvail || round >= current {
- return 0, fmt.Errorf("round %d is not available. It's outside [%d-%d]", r, firstAvail, current-1)
+ if firstAvail > round || round > lastAvail {
+ return 0, fmt.Errorf("round %d is not available. It's outside [%d-%d]", r, firstAvail, lastAvail)
}
return round, nil
}
@@ -4868,7 +4896,7 @@ func opBlock(cx *EvalContext) error {
return fmt.Errorf("invalid block field %s", f)
}
- hdr, err := cx.Ledger.BlockHdrCached(round)
+ hdr, err := cx.SigLedger.BlockHdrCached(round)
if err != nil {
return err
}
diff --git a/data/transactions/logic/evalStateful_test.go b/data/transactions/logic/evalStateful_test.go
index 820f52ff4..3e051fc6b 100644
--- a/data/transactions/logic/evalStateful_test.go
+++ b/data/transactions/logic/evalStateful_test.go
@@ -52,6 +52,7 @@ func makeSampleEnvWithVersion(version uint64) (*EvalParams, *transactions.Transa
ep := defaultEvalParamsWithVersion(nil, version)
ep.TxnGroup = transactions.WrapSignedTxnsWithAD(makeSampleTxnGroup(makeSampleTxn()))
ledger := MakeLedger(map[basics.Address]uint64{})
+ ep.SigLedger = ledger
ep.Ledger = ledger
return ep, &ep.TxnGroup[0].Txn, ledger
}
@@ -2372,7 +2373,7 @@ func TestReturnTypes(t *testing.T) {
"base64_decode": `: byte "YWJjMTIzIT8kKiYoKSctPUB+"; base64_decode StdEncoding`,
"json_ref": `: byte "{\"k\": 7}"; byte "k"; json_ref JSONUint64`,
- "block": ": int 4294967200; block BlkSeed",
+ "block": "block BlkSeed",
}
/* Make sure the specialCmd tests the opcode in question */
@@ -2537,19 +2538,23 @@ func TestLatestTimestamp(t *testing.T) {
func TestBlockSeed(t *testing.T) {
ep, txn, l := makeSampleEnv()
- // makeSampleENv creates txns with fv, lv that don't actually fit the round
- // in l. Nothing in most tests cares. But the rule for `block` is
- // related to lv and the current round, so we set the fv,lv more
- // realistically.
+ // makeSampleEnv creates txns with fv, lv that don't actually fit the round
+ // in l. Nothing in most tests cares. But the rule for `block` is related
+ // to lv and fv, so we set the fv,lv more realistically.
txn.FirstValid = l.round() - 10
txn.LastValid = l.round() + 10
+ // Keep in mind that proto.MaxTxnLife is 1500 in the test proto
+
// l.round() is 0xffffffff+5 = 4294967300 in test ledger
- testApp(t, "int 4294967299; block BlkSeed; len; int 32; ==", ep) // current - 1
+
+ // These first two tests show that current-1 is not available now, though a
+ // resonable extension is to allow such access for apps (not sigs).
+ testApp(t, "int 4294967299; block BlkSeed; len; int 32; ==", ep,
+ "not available") // current - 1
testApp(t, "int 4294967300; block BlkSeed; len; int 32; ==", ep,
"not available") // can't get current round's blockseed
- // proto.MaxTxnLife is 1500 in test.
testApp(t, "int 4294967300; int 1500; -; block BlkSeed; len; int 32; ==", ep,
"not available") // 1500 back from current is more than 1500 back from lv
testApp(t, "int 4294967310; int 1500; -; block BlkSeed; len; int 32; ==", ep) // 1500 back from lv is legal
@@ -2560,6 +2565,11 @@ func TestBlockSeed(t *testing.T) {
// A little silly, as it only tests the test ledger: ensure samenes and differentness
testApp(t, "int 0xfffffff0; block BlkSeed; int 0xfffffff0; block BlkSeed; ==", ep)
testApp(t, "int 0xfffffff0; block BlkSeed; int 0xfffffff1; block BlkSeed; !=", ep)
+
+ // `block` should also work in LogicSigs, to drive home the point, blot out
+ // the normal Ledger
+ ep.Ledger = nil
+ testLogic(t, "int 0xfffffff0; block BlkTimestamp", randomnessVersion, ep)
}
func TestCurrentApplicationID(t *testing.T) {
diff --git a/data/transactions/logic/eval_test.go b/data/transactions/logic/eval_test.go
index 6ad3340a8..734f4ba6c 100644
--- a/data/transactions/logic/eval_test.go
+++ b/data/transactions/logic/eval_test.go
@@ -131,6 +131,7 @@ func defaultEvalParamsWithVersion(txn *transactions.SignedTxn, version uint64) *
Specials: &transactions.SpecialAddresses{},
Trace: &strings.Builder{},
FeeCredit: &zero,
+ SigLedger: MakeLedger(nil),
}
if txn != nil {
ep.TxnGroup[0].SignedTxn = *txn
@@ -252,16 +253,18 @@ func TestTxnFirstValidTime(t *testing.T) {
partitiontest.PartitionTest(t)
t.Parallel()
- // txn FirstValidTime is unusual. It's not really a field of a txn, but
- // since it looks at the past of the blockchain, it is "stateless", in the
- // sense that the value can not change, so it is available in logicsigs
-
ep, tx, ledger := makeSampleEnv()
// By default, test ledger uses an oddball round, ask it what round it's
// going to use and prep fv, lv accordingly.
current := ledger.Round()
+ // txn FirstValidTime is unusual. It's not really a field of a txn, but
+ // since it looks at the past of the blockchain, it is "stateless"
+
+ // Kill off ep.Ledger, to confirm it's not being used
+ ep.Ledger = nil
+
tx.FirstValid = current - 10
tx.LastValid = current + 10
testLogic(t, "txn FirstValidTime", 7, ep)
@@ -2689,6 +2692,7 @@ int 1`,
Proto: makeTestProto(),
TxnGroup: txgroup,
pastScratch: make([]*scratchSpace, 2),
+ SigLedger: MakeLedger(nil),
}
switch failCase.runMode {
diff --git a/data/transactions/verify/txn.go b/data/transactions/verify/txn.go
index 05d54caa3..1d947d31a 100644
--- a/data/transactions/verify/txn.go
+++ b/data/transactions/verify/txn.go
@@ -66,11 +66,12 @@ type GroupContext struct {
consensusParams config.ConsensusParams
minAvmVersion uint64
signedGroupTxns []transactions.SignedTxn
+ ledger logic.LedgerForSignature
}
// PrepareGroupContext prepares a verification group parameter object for a given transaction
// group.
-func PrepareGroupContext(group []transactions.SignedTxn, contextHdr bookkeeping.BlockHeader) (*GroupContext, error) {
+func PrepareGroupContext(group []transactions.SignedTxn, contextHdr bookkeeping.BlockHeader, ledger logic.LedgerForSignature) (*GroupContext, error) {
if len(group) == 0 {
return nil, nil
}
@@ -87,6 +88,7 @@ func PrepareGroupContext(group []transactions.SignedTxn, contextHdr bookkeeping.
consensusParams: consensusParams,
minAvmVersion: logic.ComputeMinAvmVersion(transactions.WrapSignedTxnsWithAD(group)),
signedGroupTxns: group,
+ ledger: ledger,
}, nil
}
@@ -132,10 +134,10 @@ func TxnBatchVerify(s *transactions.SignedTxn, txnIdx int, groupCtx *GroupContex
}
// TxnGroup verifies a []SignedTxn as being signed and having no obviously inconsistent data.
-func TxnGroup(stxs []transactions.SignedTxn, contextHdr bookkeeping.BlockHeader, cache VerifiedTransactionCache) (groupCtx *GroupContext, err error) {
+func TxnGroup(stxs []transactions.SignedTxn, contextHdr bookkeeping.BlockHeader, cache VerifiedTransactionCache, ledger logic.LedgerForSignature) (groupCtx *GroupContext, err error) {
batchVerifier := crypto.MakeBatchVerifier()
- if groupCtx, err = TxnGroupBatchVerify(stxs, contextHdr, cache, batchVerifier); err != nil {
+ if groupCtx, err = TxnGroupBatchVerify(stxs, contextHdr, cache, ledger, batchVerifier); err != nil {
return nil, err
}
@@ -152,8 +154,8 @@ func TxnGroup(stxs []transactions.SignedTxn, contextHdr bookkeeping.BlockHeader,
// TxnGroupBatchVerify verifies a []SignedTxn having no obviously inconsistent data.
// it is the caller responsibility to call batchVerifier.verify()
-func TxnGroupBatchVerify(stxs []transactions.SignedTxn, contextHdr bookkeeping.BlockHeader, cache VerifiedTransactionCache, verifier *crypto.BatchVerifier) (groupCtx *GroupContext, err error) {
- groupCtx, err = PrepareGroupContext(stxs, contextHdr)
+func TxnGroupBatchVerify(stxs []transactions.SignedTxn, contextHdr bookkeeping.BlockHeader, cache VerifiedTransactionCache, ledger logic.LedgerForSignature, verifier *crypto.BatchVerifier) (groupCtx *GroupContext, err error) {
+ groupCtx, err = PrepareGroupContext(stxs, contextHdr, ledger)
if err != nil {
return nil, err
}
@@ -293,6 +295,7 @@ func LogicSigSanityCheckBatchVerify(txn *transactions.SignedTxn, groupIndex int,
Proto: &groupCtx.consensusParams,
TxnGroup: txngroup,
MinAvmVersion: &groupCtx.minAvmVersion,
+ SigLedger: groupCtx.ledger, // won't be needed for CheckSignature
}
err := logic.CheckSignature(groupIndex, &ep)
if err != nil {
@@ -348,6 +351,7 @@ func logicSigBatchVerify(txn *transactions.SignedTxn, groupIndex int, groupCtx *
Proto: &groupCtx.consensusParams,
TxnGroup: transactions.WrapSignedTxnsWithAD(groupCtx.signedGroupTxns),
MinAvmVersion: &groupCtx.minAvmVersion,
+ SigLedger: groupCtx.ledger,
}
pass, err := logic.EvalSignature(groupIndex, &ep)
if err != nil {
@@ -369,7 +373,7 @@ func logicSigBatchVerify(txn *transactions.SignedTxn, groupIndex int, groupCtx *
// a PaysetGroups may be well-formed, but a payset might contain an overspend.
//
// This version of verify is performing the verification over the provided execution pool.
-func PaysetGroups(ctx context.Context, payset [][]transactions.SignedTxn, blkHeader bookkeeping.BlockHeader, verificationPool execpool.BacklogPool, cache VerifiedTransactionCache) (err error) {
+func PaysetGroups(ctx context.Context, payset [][]transactions.SignedTxn, blkHeader bookkeeping.BlockHeader, verificationPool execpool.BacklogPool, cache VerifiedTransactionCache, ledger logic.LedgerForSignature) (err error) {
if len(payset) == 0 {
return nil
}
@@ -406,7 +410,7 @@ func PaysetGroups(ctx context.Context, payset [][]transactions.SignedTxn, blkHea
batchVerifier := crypto.MakeBatchVerifierWithHint(len(payset))
for i, signTxnsGrp := range txnGroups {
- groupCtxs[i], grpErr = TxnGroupBatchVerify(signTxnsGrp, blkHeader, nil, batchVerifier)
+ groupCtxs[i], grpErr = TxnGroupBatchVerify(signTxnsGrp, blkHeader, nil, ledger, batchVerifier)
// abort only if it's a non-cache error.
if grpErr != nil {
return grpErr
diff --git a/data/transactions/verify/txn_test.go b/data/transactions/verify/txn_test.go
index 52cf29874..399835211 100644
--- a/data/transactions/verify/txn_test.go
+++ b/data/transactions/verify/txn_test.go
@@ -114,7 +114,7 @@ func TestSignedPayment(t *testing.T) {
payments, stxns, secrets, addrs := generateTestObjects(1, 1, 0)
payment, stxn, secret, addr := payments[0], stxns[0], secrets[0], addrs[0]
- groupCtx, err := PrepareGroupContext(stxns, blockHeader)
+ groupCtx, err := PrepareGroupContext(stxns, blockHeader, nil)
require.NoError(t, err)
require.NoError(t, payment.WellFormed(spec, proto), "generateTestObjects generated an invalid payment")
require.NoError(t, Txn(&stxn, 0, groupCtx), "generateTestObjects generated a bad signedtxn")
@@ -135,7 +135,7 @@ func TestTxnValidationEncodeDecode(t *testing.T) {
_, signed, _, _ := generateTestObjects(100, 50, 0)
for _, txn := range signed {
- groupCtx, err := PrepareGroupContext([]transactions.SignedTxn{txn}, blockHeader)
+ groupCtx, err := PrepareGroupContext([]transactions.SignedTxn{txn}, blockHeader, nil)
require.NoError(t, err)
if Txn(&txn, 0, groupCtx) != nil {
t.Errorf("signed transaction %#v did not verify", txn)
@@ -157,7 +157,7 @@ func TestTxnValidationEmptySig(t *testing.T) {
_, signed, _, _ := generateTestObjects(100, 50, 0)
for _, txn := range signed {
- groupCtx, err := PrepareGroupContext([]transactions.SignedTxn{txn}, blockHeader)
+ groupCtx, err := PrepareGroupContext([]transactions.SignedTxn{txn}, blockHeader, nil)
require.NoError(t, err)
if Txn(&txn, 0, groupCtx) != nil {
t.Errorf("signed transaction %#v did not verify", txn)
@@ -202,7 +202,7 @@ func TestTxnValidationStateProof(t *testing.T) {
},
}
- groupCtx, err := PrepareGroupContext([]transactions.SignedTxn{stxn}, blockHeader)
+ groupCtx, err := PrepareGroupContext([]transactions.SignedTxn{stxn}, blockHeader, nil)
require.NoError(t, err)
err = Txn(&stxn, 0, groupCtx)
@@ -256,7 +256,7 @@ func TestDecodeNil(t *testing.T) {
err := protocol.Decode(nilEncoding, &st)
if err == nil {
// This used to panic when run on a zero value of SignedTxn.
- groupCtx, err := PrepareGroupContext([]transactions.SignedTxn{st}, blockHeader)
+ groupCtx, err := PrepareGroupContext([]transactions.SignedTxn{st}, blockHeader, nil)
require.NoError(t, err)
Txn(&st, 0, groupCtx)
}
@@ -285,17 +285,17 @@ func TestPaysetGroups(t *testing.T) {
txnGroups := generateTransactionGroups(signedTxn, secrets, addrs)
startPaysetGroupsTime := time.Now()
- err := PaysetGroups(context.Background(), txnGroups, blkHdr, verificationPool, MakeVerifiedTransactionCache(50000))
+ err := PaysetGroups(context.Background(), txnGroups, blkHdr, verificationPool, MakeVerifiedTransactionCache(50000), nil)
require.NoError(t, err)
paysetGroupDuration := time.Now().Sub(startPaysetGroupsTime)
// break the signature and see if it fails.
txnGroups[0][0].Sig[0] = txnGroups[0][0].Sig[0] + 1
- err = PaysetGroups(context.Background(), txnGroups, blkHdr, verificationPool, MakeVerifiedTransactionCache(50000))
+ err = PaysetGroups(context.Background(), txnGroups, blkHdr, verificationPool, MakeVerifiedTransactionCache(50000), nil)
require.Error(t, err)
// ensure the rest are fine
- err = PaysetGroups(context.Background(), txnGroups[1:], blkHdr, verificationPool, MakeVerifiedTransactionCache(50000))
+ err = PaysetGroups(context.Background(), txnGroups[1:], blkHdr, verificationPool, MakeVerifiedTransactionCache(50000), nil)
require.NoError(t, err)
// test the context cancelation:
@@ -312,7 +312,7 @@ func TestPaysetGroups(t *testing.T) {
go func() {
defer close(waitCh)
cache := MakeVerifiedTransactionCache(50000)
- waitCh <- PaysetGroups(ctx, txnGroups, blkHdr, verificationPool, cache)
+ waitCh <- PaysetGroups(ctx, txnGroups, blkHdr, verificationPool, cache, nil)
}()
startPaysetGroupsTime = time.Now()
select {
@@ -366,7 +366,7 @@ func BenchmarkPaysetGroups(b *testing.B) {
cache := MakeVerifiedTransactionCache(50000)
b.ResetTimer()
- err := PaysetGroups(context.Background(), txnGroups, blkHdr, verificationPool, cache)
+ err := PaysetGroups(context.Background(), txnGroups, blkHdr, verificationPool, cache, nil)
require.NoError(b, err)
b.StopTimer()
}
@@ -422,7 +422,7 @@ func BenchmarkTxn(b *testing.B) {
b.ResetTimer()
for _, txnGroup := range txnGroups {
- groupCtx, err := PrepareGroupContext(txnGroup, blk.BlockHeader)
+ groupCtx, err := PrepareGroupContext(txnGroup, blk.BlockHeader, nil)
require.NoError(b, err)
for i, txn := range txnGroup {
err := Txn(&txn, i, groupCtx)
diff --git a/data/transactions/verify/verifiedTxnCache_test.go b/data/transactions/verify/verifiedTxnCache_test.go
index af8e36b42..35d958e35 100644
--- a/data/transactions/verify/verifiedTxnCache_test.go
+++ b/data/transactions/verify/verifiedTxnCache_test.go
@@ -34,7 +34,7 @@ func TestAddingToCache(t *testing.T) {
impl := icache.(*verifiedTransactionCache)
_, signedTxn, secrets, addrs := generateTestObjects(10, 5, 50)
txnGroups := generateTransactionGroups(signedTxn, secrets, addrs)
- groupCtx, err := PrepareGroupContext(txnGroups[0], blockHeader)
+ groupCtx, err := PrepareGroupContext(txnGroups[0], blockHeader, nil)
require.NoError(t, err)
impl.Add(txnGroups[0], groupCtx)
// make it was added.
@@ -55,7 +55,7 @@ func TestBucketCycling(t *testing.T) {
_, signedTxn, _, _ := generateTestObjects(entriesPerBucket*bucketCount*2, bucketCount, 0)
require.Equal(t, entriesPerBucket*bucketCount*2, len(signedTxn))
- groupCtx, err := PrepareGroupContext([]transactions.SignedTxn{signedTxn[0]}, blockHeader)
+ groupCtx, err := PrepareGroupContext([]transactions.SignedTxn{signedTxn[0]}, blockHeader, nil)
require.NoError(t, err)
// fill up the cache with entries.
@@ -92,7 +92,7 @@ func TestGetUnverifiedTranscationGroups50(t *testing.T) {
if i%2 == 0 {
expectedUnverifiedGroups = append(expectedUnverifiedGroups, txnGroups[i])
} else {
- groupCtx, _ := PrepareGroupContext(txnGroups[i], blockHeader)
+ groupCtx, _ := PrepareGroupContext(txnGroups[i], blockHeader, nil)
impl.Add(txnGroups[i], groupCtx)
}
}
@@ -116,7 +116,7 @@ func BenchmarkGetUnverifiedTranscationGroups50(b *testing.B) {
if i%2 == 1 {
queryTxnGroups = append(queryTxnGroups, txnGroups[i])
} else {
- groupCtx, _ := PrepareGroupContext(txnGroups[i], blockHeader)
+ groupCtx, _ := PrepareGroupContext(txnGroups[i], blockHeader, nil)
impl.Add(txnGroups[i], groupCtx)
}
}
@@ -145,7 +145,7 @@ func TestUpdatePinned(t *testing.T) {
// insert some entries.
for i := 0; i < len(txnGroups); i++ {
- groupCtx, _ := PrepareGroupContext(txnGroups[i], blockHeader)
+ groupCtx, _ := PrepareGroupContext(txnGroups[i], blockHeader, nil)
impl.Add(txnGroups[i], groupCtx)
}
@@ -174,7 +174,7 @@ func TestPinningTransactions(t *testing.T) {
// insert half of the entries.
for i := 0; i < len(txnGroups)/2; i++ {
- groupCtx, _ := PrepareGroupContext(txnGroups[i], blockHeader)
+ groupCtx, _ := PrepareGroupContext(txnGroups[i], blockHeader, nil)
impl.Add(txnGroups[i], groupCtx)
}
diff --git a/data/txHandler.go b/data/txHandler.go
index 1d4d1a500..9bc91203a 100644
--- a/data/txHandler.go
+++ b/data/txHandler.go
@@ -203,7 +203,7 @@ func (handler *TxHandler) asyncVerifySignature(arg interface{}) interface{} {
logging.Base().Warnf("Could not get header for previous block %d: %v", latest, err)
} else {
// we can't use PaysetGroups here since it's using a execpool like this go-routine and we don't want to deadlock.
- _, tx.verificationErr = verify.TxnGroup(tx.unverifiedTxGroup, latestHdr, handler.ledger.VerifiedTransactionCache())
+ _, tx.verificationErr = verify.TxnGroup(tx.unverifiedTxGroup, latestHdr, handler.ledger.VerifiedTransactionCache(), handler.ledger)
}
select {
@@ -295,7 +295,7 @@ func (handler *TxHandler) processDecoded(unverifiedTxGroup []transactions.Signed
}
unverifiedTxnGroups := bookkeeping.SignedTxnsToGroups(unverifiedTxGroup)
- err = verify.PaysetGroups(context.Background(), unverifiedTxnGroups, latestHdr, handler.txVerificationPool, handler.ledger.VerifiedTransactionCache())
+ err = verify.PaysetGroups(context.Background(), unverifiedTxnGroups, latestHdr, handler.txVerificationPool, handler.ledger.VerifiedTransactionCache(), handler.ledger)
if err != nil {
// transaction is invalid
logging.Base().Warnf("One or more transactions were malformed: %v", err)
diff --git a/ledger/applications_test.go b/ledger/applications_test.go
index 79ef7d6cd..b8fc6ad36 100644
--- a/ledger/applications_test.go
+++ b/ledger/applications_test.go
@@ -1235,3 +1235,70 @@ int 1
})
}
}
+
+// TestLogicSigValidation tests that LogicSig-signed transactions can be validated properly.
+func TestLogicSigValidation(t *testing.T) {
+ partitiontest.PartitionTest(t)
+
+ source := `#pragma version 6
+int 1
+`
+
+ a := require.New(t)
+ ops, err := logic.AssembleString(source)
+ a.NoError(err)
+ a.Greater(len(ops.Program), 1)
+ program := ops.Program
+ pd := logic.HashProgram(program)
+ lsigAddr := basics.Address(pd)
+
+ funder, err := basics.UnmarshalChecksumAddress("3LN5DBFC2UTPD265LQDP3LMTLGZCQ5M3JV7XTVTGRH5CKSVNQVDFPN6FG4")
+ a.NoError(err)
+
+ proto := config.Consensus[protocol.ConsensusCurrentVersion]
+ genesisInitState, initKeys := ledgertesting.GenerateInitState(t, protocol.ConsensusCurrentVersion, 100)
+ a.Contains(genesisInitState.Accounts, funder)
+
+ cfg := config.GetDefaultLocal()
+ l, err := OpenLedger(logging.Base(), t.Name(), true, genesisInitState, cfg)
+ a.NoError(err)
+ defer l.Close()
+
+ genesisID := t.Name()
+ txHeader := transactions.Header{
+ Sender: funder,
+ Fee: basics.MicroAlgos{Raw: proto.MinTxnFee},
+ FirstValid: l.Latest() + 1,
+ LastValid: l.Latest() + 10,
+ GenesisID: genesisID,
+ GenesisHash: genesisInitState.GenesisHash,
+ }
+
+ // fund lsig account
+ fundingPayment := transactions.Transaction{
+ Type: protocol.PaymentTx,
+ Header: txHeader,
+ PaymentTxnFields: transactions.PaymentTxnFields{
+ Receiver: lsigAddr,
+ Amount: basics.MicroAlgos{Raw: proto.MinBalance + proto.MinTxnFee},
+ },
+ }
+ err = l.appendUnvalidatedTx(t, genesisInitState.Accounts, initKeys, fundingPayment, transactions.ApplyData{})
+ a.NoError(err)
+
+ // send 0 Algos from lsig account to self
+ txHeader.Sender = lsigAddr
+ lsigPayment := transactions.Transaction{
+ Type: protocol.PaymentTx,
+ Header: txHeader,
+ PaymentTxnFields: transactions.PaymentTxnFields{
+ Receiver: lsigAddr,
+ },
+ }
+ signedLsigPayment := transactions.SignedTxn{
+ Lsig: transactions.LogicSig{Logic: program},
+ Txn: lsigPayment,
+ }
+ err = l.appendUnvalidatedSignedTx(t, genesisInitState.Accounts, signedLsigPayment, transactions.ApplyData{})
+ a.NoError(err)
+}
diff --git a/ledger/internal/eval.go b/ledger/internal/eval.go
index 93429f2e0..1944c4a69 100644
--- a/ledger/internal/eval.go
+++ b/ledger/internal/eval.go
@@ -1447,6 +1447,7 @@ type evalTxValidator struct {
txcache verify.VerifiedTransactionCache
block bookkeeping.Block
verificationPool execpool.BacklogPool
+ ledger logic.LedgerForSignature
ctx context.Context
txgroups [][]transactions.SignedTxnWithAD
@@ -1477,7 +1478,7 @@ func (validator *evalTxValidator) run() {
unverifiedTxnGroups = validator.txcache.GetUnverifiedTranscationGroups(unverifiedTxnGroups, specialAddresses, validator.block.BlockHeader.CurrentProtocol)
- err := verify.PaysetGroups(validator.ctx, unverifiedTxnGroups, validator.block.BlockHeader, validator.verificationPool, validator.txcache)
+ err := verify.PaysetGroups(validator.ctx, unverifiedTxnGroups, validator.block.BlockHeader, validator.verificationPool, validator.txcache, validator.ledger)
if err != nil {
validator.done <- err
}
@@ -1532,6 +1533,7 @@ func Eval(ctx context.Context, l LedgerForEvaluator, blk bookkeeping.Block, vali
txvalidator.txcache = txcache
txvalidator.block = blk
txvalidator.verificationPool = executionPool
+ txvalidator.ledger = l
txvalidator.ctx = validationCtx
txvalidator.txgroups = paysetgroups
diff --git a/node/node.go b/node/node.go
index bea64849f..7f0df8140 100644
--- a/node/node.go
+++ b/node/node.go
@@ -499,7 +499,7 @@ func (node *AlgorandFullNode) broadcastSignedTxGroup(txgroup []transactions.Sign
return err
}
- _, err = verify.TxnGroup(txgroup, b, node.ledger.VerifiedTransactionCache())
+ _, err = verify.TxnGroup(txgroup, b, node.ledger.VerifiedTransactionCache(), node.ledger)
if err != nil {
node.log.Warnf("malformed transaction: %v", err)
return err
diff --git a/test/scripts/e2e_subs/hdr-access-logicsig.sh b/test/scripts/e2e_subs/hdr-access-logicsig.sh
new file mode 100755
index 000000000..32c36d7b2
--- /dev/null
+++ b/test/scripts/e2e_subs/hdr-access-logicsig.sh
@@ -0,0 +1,63 @@
+#!/bin/bash
+
+filename=$(basename "$0")
+scriptname="${filename%.*}"
+date "+${scriptname} start %Y%m%d_%H%M%S"
+
+set -e
+set -x
+set -o pipefail
+
+WALLET=$1
+
+gcmd="goal -w ${WALLET}"
+
+ACCOUNT=$(${gcmd} account list|awk '{ print $3 }')
+
+cat >${TEMPDIR}/hdr.teal<<EOF
+#pragma version 7
+txn FirstValid
+int 1
+-
+block BlkTimestamp // need to make sure we don't ask for current
+
+txn FirstValid
+int 2
+-
+block BlkTimestamp
+// last two times are on stack
+-
+dup
+// difference in times is on stack twice
+
+int 1
+>
+assert
+
+int 6
+<
+EOF
+
+${gcmd} clerk compile -o ${TEMPDIR}/hdr.lsig -s -a ${ACCOUNT} ${TEMPDIR}/hdr.teal
+
+SIGACCOUNT=$(${gcmd} clerk compile -n ${TEMPDIR}/hdr.teal|awk '{ print $2 }')
+
+# Avoid rewards by giving less than an algo
+${gcmd} clerk send --amount 900000 --from ${ACCOUNT} --to ${SIGACCOUNT}
+
+function balance {
+ acct=$1; shift
+ goal account balance -a "$acct" | awk '{print $1}'
+}
+
+[ "$(balance "$SIGACCOUNT")" = 900000 ]
+
+# Don't let goal set lastvalid so far in the future, that prevents `block` access
+${gcmd} clerk send --amount 10 --from ${SIGACCOUNT} --to ${ACCOUNT} --lastvalid 100 -o ${TEMPDIR}/hdr.tx
+
+${gcmd} clerk sign -i ${TEMPDIR}/hdr.tx -o ${TEMPDIR}/hdr.stx --program ${TEMPDIR}/hdr.teal
+
+${gcmd} clerk rawsend -f ${TEMPDIR}/hdr.stx
+
+# remove min fee + 10
+[ "$(balance "$SIGACCOUNT")" = 898990 ]
diff --git a/test/scripts/e2e_subs/hdr-access.py b/test/scripts/e2e_subs/hdr-access.py
index 067c80523..4da7856a3 100755
--- a/test/scripts/e2e_subs/hdr-access.py
+++ b/test/scripts/e2e_subs/hdr-access.py
@@ -32,19 +32,35 @@ assert err
assert "invalid txn field FirstValidTime" in str(err), err
-# Test that the block timestamp from two blocks ago is between 2 and 5
-# (inclusive) seconds before the previous block timestamp. devMode
-# might mess this test up. This works because FirstValid is set to
-# the last committed block by SDK, not the next coming one.
+# Can't access two behind FirstValid because LastValid is 1000 after
teal = """
#pragma version 7
txn FirstValid
+ int 2
+ -
block BlkTimestamp
+"""
+txinfo, err = goal.app_create(joe, goal.assemble(teal))
+assert "not available" in str(err), err
+# We want to manipulate lastvalid, so we need to turn off autosend
+goal.autosend = False
+
+# We will be able to access two blocks, by setting lv explcitly. So we
+# test that the block timestamp from two blocks ago is between 2 and 5
+# (inclusive) seconds before the previous block timestamp. devMode
+# might mess this test up.
+teal = """
+#pragma version 7
txn FirstValid
int 1
-
block BlkTimestamp
+
+ txn FirstValid
+ int 2
+ -
+ block BlkTimestamp
// last two times are on stack
-
dup
@@ -58,23 +74,6 @@ teal = """
<
"""
checktimes = goal.assemble(teal)
-txinfo, err = goal.app_create(joe, checktimes)
-assert not err, err
-
-# Can't access two behind FirstValid because LastValid is 1000 after
-teal = """
-#pragma version 7
- txn FirstValid
- int 2
- -
- block BlkTimestamp
-"""
-txinfo, err = goal.app_create(joe, goal.assemble(teal))
-assert "not available" in str(err), err
-
-# We want to manipulate lastvalid, so we need to turn off autosend
-goal.autosend = False
-
tx = goal.app_create(joe, goal.assemble(teal))
tx.last_valid_round = tx.last_valid_round - 800
txinfo, err = goal.send(tx)