From eb2acd436ce822a30e84ab0533c7169470d122b6 Mon Sep 17 00:00:00 2001 From: John Jannotti Date: Tue, 8 Oct 2024 13:29:17 -0400 Subject: [PATCH 01/19] Heartbeat transaction type Some TODOS: 1) Lock it down. Singleton group? Top-level? 2) Make it free for challenged nodes. 3) Get the heartbeat service going to keep those nodes online. 4) Move the crypto earlier --- data/basics/userBalance.go | 10 - data/transactions/heartbeat.go | 42 +++ data/transactions/logic/assembler.go | 10 + data/transactions/logic/crypto_test.go | 18 +- data/transactions/logic/eval_test.go | 2 +- data/transactions/transaction.go | 5 + heartbeat/abstractions.go | 54 ++++ heartbeat/service.go | 159 +++++++++++ heartbeat/service_test.go | 347 +++++++++++++++++++++++++ ledger/apply/apply.go | 7 +- ledger/apply/heartbeat.go | 56 ++++ ledger/apply/heartbeat_test.go | 90 +++++++ ledger/apply/mockBalances_test.go | 23 ++ ledger/eval/eval.go | 30 ++- ledger/eval/eval_test.go | 28 +- ledger/ledger.go | 2 +- node/node.go | 5 + protocol/txntype.go | 3 + stateproof/builder.go | 2 +- 19 files changed, 842 insertions(+), 51 deletions(-) create mode 100644 data/transactions/heartbeat.go create mode 100644 heartbeat/abstractions.go create mode 100644 heartbeat/service.go create mode 100644 heartbeat/service_test.go create mode 100644 ledger/apply/heartbeat.go create mode 100644 ledger/apply/heartbeat_test.go diff --git a/data/basics/userBalance.go b/data/basics/userBalance.go index d8f86aea54..11457d8d57 100644 --- a/data/basics/userBalance.go +++ b/data/basics/userBalance.go @@ -19,7 +19,6 @@ package basics import ( "encoding/binary" "fmt" - "reflect" "github.com/algorand/go-algorand/config" "github.com/algorand/go-algorand/crypto" @@ -581,15 +580,6 @@ func (u OnlineAccountData) KeyDilution(proto config.ConsensusParams) uint64 { return proto.DefaultKeyDilution } -// IsZero checks if an AccountData value is the same as its zero value. -func (u AccountData) IsZero() bool { - if u.Assets != nil && len(u.Assets) == 0 { - u.Assets = nil - } - - return reflect.DeepEqual(u, AccountData{}) -} - // NormalizedOnlineBalance returns a “normalized” balance for this account. // // The normalization compensates for rewards that have not yet been applied, diff --git a/data/transactions/heartbeat.go b/data/transactions/heartbeat.go new file mode 100644 index 0000000000..9acf847734 --- /dev/null +++ b/data/transactions/heartbeat.go @@ -0,0 +1,42 @@ +// Copyright (C) 2019-2024 Algorand, Inc. +// This file is part of go-algorand +// +// go-algorand is free software: you can redistribute it and/or modify +// it under the terms of the GNU Affero General Public License as +// published by the Free Software Foundation, either version 3 of the +// License, or (at your option) any later version. +// +// go-algorand is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Affero General Public License for more details. +// +// You should have received a copy of the GNU Affero General Public License +// along with go-algorand. If not, see . + +package transactions + +import ( + "github.com/algorand/go-algorand/crypto" + "github.com/algorand/go-algorand/data/basics" + "github.com/algorand/go-algorand/data/committee" +) + +// HeartbeatTxnFields captures the fields used for an account to prove it is +// online (really, it proves that an entity with the account's part keys is able +// to submit transactions, so it should be able to propose/vote.) +type HeartbeatTxnFields struct { + _struct struct{} `codec:",omitempty,omitemptyarray"` + + // HeartbeatAddress is the account this txn is proving onlineness for. + HeartbeatAddress basics.Address `codec:"hbad"` + + // Proof is a signature using HeartbeatAddress's partkey, thereby showing it is online. + Proof crypto.OneTimeSignature `codec:"hbprf"` + + // Seed must be the block seed for the block before this transaction's + // firstValid. It is supplied in the transaction so that Proof can be + // checked at submit time without a ledger lookup, and must be checked at + // evaluation time for equality with the actual blockseed. + Seed committee.Seed `codec:"hbsd"` +} diff --git a/data/transactions/logic/assembler.go b/data/transactions/logic/assembler.go index a707e23f65..c321e981ae 100644 --- a/data/transactions/logic/assembler.go +++ b/data/transactions/logic/assembler.go @@ -2738,6 +2738,16 @@ func AssembleString(text string) (*OpStream, error) { return AssembleStringWithVersion(text, assemblerNoVersion) } +// MustAssemble assembles a program an panics on error. It is useful for +// defining globals. +func MustAssemble(text string) []byte { + ops, err := AssembleString(text) + if err != nil { + panic(err) + } + return ops.Program +} + // AssembleStringWithVersion takes an entire program in a string and // assembles it to bytecode using the assembler version specified. If // version is assemblerNoVersion it uses #pragma version or fallsback diff --git a/data/transactions/logic/crypto_test.go b/data/transactions/logic/crypto_test.go index 0ba695dce5..1298f34c9f 100644 --- a/data/transactions/logic/crypto_test.go +++ b/data/transactions/logic/crypto_test.go @@ -217,13 +217,17 @@ pop // output`, "int 1"}, } } +func randSeed() crypto.Seed { + var s crypto.Seed + crypto.RandBytes(s[:]) + return s +} + func TestEd25519verify(t *testing.T) { partitiontest.PartitionTest(t) t.Parallel() - var s crypto.Seed - crypto.RandBytes(s[:]) - c := crypto.GenerateSignatureSecrets(s) + c := crypto.GenerateSignatureSecrets(randSeed()) msg := "62fdfc072182654f163f5f0f9a621d729566c74d0aa413bf009c9800418c19cd" data, err := hex.DecodeString(msg) require.NoError(t, err) @@ -262,9 +266,7 @@ func TestEd25519VerifyBare(t *testing.T) { partitiontest.PartitionTest(t) t.Parallel() - var s crypto.Seed - crypto.RandBytes(s[:]) - c := crypto.GenerateSignatureSecrets(s) + c := crypto.GenerateSignatureSecrets(randSeed()) msg := "62fdfc072182654f163f5f0f9a621d729566c74d0aa413bf009c9800418c19cd" data, err := hex.DecodeString(msg) require.NoError(t, err) @@ -743,9 +745,7 @@ func BenchmarkEd25519Verifyx1(b *testing.B) { crypto.RandBytes(buffer[:]) data = append(data, buffer) - var s crypto.Seed //generate programs and signatures - crypto.RandBytes(s[:]) - secret := crypto.GenerateSignatureSecrets(s) + secret := crypto.GenerateSignatureSecrets(randSeed()) //generate programs and signatures pk := basics.Address(secret.SignatureVerifier) pkStr := pk.String() ops, err := AssembleStringWithVersion(fmt.Sprintf(`arg 0 diff --git a/data/transactions/logic/eval_test.go b/data/transactions/logic/eval_test.go index c8f7a8bc5f..701acedaab 100644 --- a/data/transactions/logic/eval_test.go +++ b/data/transactions/logic/eval_test.go @@ -415,7 +415,7 @@ func TestBlankStackSufficient(t *testing.T) { spec := opsByOpcode[v][i] argLen := len(spec.Arg.Types) blankStackLen := len(blankStack) - require.GreaterOrEqual(t, blankStackLen, argLen) + require.GreaterOrEqual(t, blankStackLen, argLen, spec.Name) } }) } diff --git a/data/transactions/transaction.go b/data/transactions/transaction.go index 4a6d5b6603..a17066c3a5 100644 --- a/data/transactions/transaction.go +++ b/data/transactions/transaction.go @@ -100,6 +100,7 @@ type Transaction struct { AssetFreezeTxnFields ApplicationCallTxnFields StateProofTxnFields + HeartbeatTxnFields } // ApplyData contains information about the transaction's execution. @@ -598,6 +599,10 @@ func (tx Transaction) WellFormed(spec SpecialAddresses, proto config.ConsensusPa nonZeroFields[protocol.StateProofTx] = true } + if tx.HeartbeatTxnFields != (HeartbeatTxnFields{}) { + nonZeroFields[protocol.HeartbeatTx] = true + } + for t, nonZero := range nonZeroFields { if nonZero && t != tx.Type { return fmt.Errorf("transaction of type %v has non-zero fields for type %v", tx.Type, t) diff --git a/heartbeat/abstractions.go b/heartbeat/abstractions.go new file mode 100644 index 0000000000..52206a0ef7 --- /dev/null +++ b/heartbeat/abstractions.go @@ -0,0 +1,54 @@ +// Copyright (C) 2019-2023 Algorand, Inc. +// This file is part of go-algorand +// +// go-algorand is free software: you can redistribute it and/or modify +// it under the terms of the GNU Affero General Public License as +// published by the Free Software Foundation, either version 3 of the +// License, or (at your option) any later version. +// +// go-algorand is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Affero General Public License for more details. +// +// You should have received a copy of the GNU Affero General Public License +// along with go-algorand. If not, see . + +package heartbeat + +import ( + "github.com/algorand/go-algorand/data/account" + "github.com/algorand/go-algorand/data/basics" + "github.com/algorand/go-algorand/data/bookkeeping" + "github.com/algorand/go-algorand/data/transactions" + "github.com/algorand/go-algorand/ledger/ledgercore" +) + +// txnBroadcaster is an interface that captures the node's ability to broadcast +// a new transaction. +type txnBroadcaster interface { + BroadcastInternalSignedTxGroup([]transactions.SignedTxn) error +} + +// ledger represents the aspects of the "real" Ledger that heartbeat needs. +// to interact with. +type ledger interface { + // LastRound tells the round is ready for checking + LastRound() basics.Round + + // WaitMem allows the Service to wait for the results of a round to be available + WaitMem(r basics.Round) chan struct{} + + // BlockHdr allows the service access to consensus values + BlockHdr(r basics.Round) (bookkeeping.BlockHeader, error) + + // LookupAccount allows the Service to observe accounts for suspension + LookupAccount(round basics.Round, addr basics.Address) (data ledgercore.AccountData, validThrough basics.Round, withoutRewards basics.MicroAlgos, err error) +} + +// partipants captures the aspects of the AccountManager that are used by this +// package. Service must be able to find out which accounts to monitor and have +// access to their part keys to construct heartbeats. +type participants interface { + Keys(rnd basics.Round) []account.ParticipationRecordForRound +} diff --git a/heartbeat/service.go b/heartbeat/service.go new file mode 100644 index 0000000000..8f9775cc50 --- /dev/null +++ b/heartbeat/service.go @@ -0,0 +1,159 @@ +// Copyright (C) 2019-2023 Algorand, Inc. +// This file is part of go-algorand +// +// go-algorand is free software: you can redistribute it and/or modify +// it under the terms of the GNU Affero General Public License as +// published by the Free Software Foundation, either version 3 of the +// License, or (at your option) any later version. +// +// go-algorand is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Affero General Public License for more details. +// +// You should have received a copy of the GNU Affero General Public License +// along with go-algorand. If not, see . + +package heartbeat + +import ( + "context" + "fmt" + "sync" + + "github.com/algorand/go-algorand/config" + "github.com/algorand/go-algorand/data/account" + "github.com/algorand/go-algorand/data/basics" + "github.com/algorand/go-algorand/data/transactions" + "github.com/algorand/go-algorand/data/transactions/logic" + "github.com/algorand/go-algorand/ledger/eval" + "github.com/algorand/go-algorand/logging" + "github.com/algorand/go-algorand/protocol" +) + +// Service emits keep-alive heartbeats for accts that are in danger of +// suspension. +type Service struct { + // addresses that should be monitored for suspension + accts participants + // current status and balances + ledger ledger + // where to send the heartbeats + bcast txnBroadcaster + + // infrastructure + ctx context.Context + stop context.CancelFunc + wg sync.WaitGroup + log logging.Logger +} + +// NewService creates a heartbeat service. It will need to know which accounts +// to emit heartbeats for, and how to create the heartbeats. +func NewService(accts participants, ledger ledger, bcast txnBroadcaster, log logging.Logger) (s *Service) { + s = &Service{ + accts: accts, + ledger: ledger, + bcast: bcast, + log: log.With("Context", "heartbeat"), + } + return s +} + +// Start starts the goroutines for the Service. +func (s *Service) Start() { + s.ctx, s.stop = context.WithCancel(context.Background()) + s.wg.Add(1) + go s.loop() +} + +// Stop any goroutines associated with this worker. +func (s *Service) Stop() { + s.stop() + s.wg.Wait() +} + +// findChallenged() returns a list of accounts that need a heartbeat because +// they have been challenged. +func (s *Service) findChallenged(rules config.ProposerPayoutRules) []account.ParticipationRecordForRound { + current := s.ledger.LastRound() + var found []account.ParticipationRecordForRound + + ch := eval.ActiveChallenge(rules, current, s.ledger) + for _, pr := range s.accts.Keys(current + 1) { // only look at accounts we have part keys for + acct, _, _, err := s.ledger.LookupAccount(current, pr.Account) + fmt.Printf(" %v is %s at %d\n", pr.Account, acct.Status, current) + if err != nil { + s.log.Errorf("error looking up %v: %v", pr.Account, err) + continue + } + if acct.Status == basics.Online { + lastSeen := max(acct.LastProposed, acct.LastHeartbeat) + if eval.FailsChallenge(ch, pr.Account, lastSeen) { + found = append(found, pr) + } + } + /* If we add a grace period to suspension for absenteeism, then we could + also make it free to heartbeat during that period. */ + } + return found +} + +// loop monitors for any of Service's participants being suspended. If they are, +// it tries to being them back online by emitting a heartbeat transaction. It +// could try to predict an upcoming suspension, which would prevent the +// suspension from ever occurring, but that would be considerably more complex +// both to avoid emitting repeated heartbeats, and to ensure the prediction and +// the suspension logic match. This feels like a cleaner end-to-end test, at +// the cost of lost couple rounds of participation. (Though suspension is +// designed to be extremely unlikely anyway.) +func (s *Service) loop() { + defer s.wg.Done() + latest := s.ledger.LastRound() + for { + // exit if Done, else wait for next round + select { + case <-s.ctx.Done(): + return + case <-s.ledger.WaitMem(latest + 1): + } + + latest = s.ledger.LastRound() + + hdr, err := s.ledger.BlockHdr(latest) + if err != nil { + s.log.Errorf("heartbeat service could not fetch block header for round %d: %v", latest, err) + continue // Try again next round, I guess? + } + proto := config.Consensus[hdr.CurrentProtocol] + + for _, pr := range s.findChallenged(proto.Payouts) { + stxn := s.prepareHeartbeat(pr.Account, latest, hdr.GenesisHash) + err = s.bcast.BroadcastInternalSignedTxGroup([]transactions.SignedTxn{stxn}) + if err != nil { + s.log.Errorf("error broadcasting heartbeat %v for %v: %v", stxn, pr.Account, err) + } + } + } +} + +// AcceptingByteCode is the source to a logic signature that will accept anything (except rekeying). +var acceptingByteCode = logic.MustAssemble(` +#pragma version 11 +txn RekeyTo; global ZeroAddress; == +`) +var acceptingSender = basics.Address(logic.HashProgram(acceptingByteCode)) + +func (s *Service) prepareHeartbeat(address basics.Address, latest basics.Round, genHash [32]byte) transactions.SignedTxn { + var stxn transactions.SignedTxn + stxn.Lsig = transactions.LogicSig{Logic: acceptingByteCode} + stxn.Txn.Type = protocol.HeartbeatTx + stxn.Txn.Header = transactions.Header{ + Sender: acceptingSender, + FirstValid: latest + 1, + LastValid: latest + 1 + 100, // maybe use the grace period? + GenesisHash: genHash, + } + + return stxn +} diff --git a/heartbeat/service_test.go b/heartbeat/service_test.go new file mode 100644 index 0000000000..aef54dd29f --- /dev/null +++ b/heartbeat/service_test.go @@ -0,0 +1,347 @@ +// Copyright (C) 2019-2023 Algorand, Inc. +// This file is part of go-algorand +// +// go-algorand is free software: you can redistribute it and/or modify +// it under the terms of the GNU Affero General Public License as +// published by the Free Software Foundation, either version 3 of the +// License, or (at your option) any later version. +// +// go-algorand is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Affero General Public License for more details. +// +// You should have received a copy of the GNU Affero General Public License +// along with go-algorand. If not, see . + +package heartbeat + +import ( + "fmt" + "testing" + "time" + + "github.com/algorand/go-algorand/data/account" + "github.com/algorand/go-algorand/data/basics" + "github.com/algorand/go-algorand/data/bookkeeping" + "github.com/algorand/go-algorand/data/transactions" + "github.com/algorand/go-algorand/ledger/ledgercore" + "github.com/algorand/go-algorand/logging" + "github.com/algorand/go-algorand/protocol" + "github.com/algorand/go-algorand/test/partitiontest" + "github.com/algorand/go-deadlock" + "github.com/stretchr/testify/require" +) + +type mockParticipants struct { + accts map[basics.Address]struct{} +} + +func (p *mockParticipants) Keys(rnd basics.Round) []account.ParticipationRecordForRound { + var ret []account.ParticipationRecordForRound + for addr, _ := range p.accts { + ret = append(ret, account.ParticipationRecordForRound{ + ParticipationRecord: account.ParticipationRecord{ + ParticipationID: [32]byte{}, + Account: addr, + FirstValid: 0, + LastValid: 0, + KeyDilution: 0, + LastVote: 0, + LastBlockProposal: 0, + }, + }) + } + return ret +} + +func (p *mockParticipants) add(addr basics.Address) { + if p.accts == nil { + p.accts = make(map[basics.Address]struct{}) + } + p.accts[addr] = struct{}{} +} + +type table map[basics.Address]ledgercore.AccountData + +type mockedLedger struct { + mu deadlock.Mutex + waiters map[basics.Round]chan struct{} + history []table + version protocol.ConsensusVersion +} + +func newMockedLedger() mockedLedger { + return mockedLedger{ + waiters: make(map[basics.Round]chan struct{}), + history: []table{nil}, // some genesis accounts could go here + version: protocol.ConsensusFuture, + } + +} + +func (l *mockedLedger) LastRound() basics.Round { + l.mu.Lock() + defer l.mu.Unlock() + return l.lastRound() +} +func (l *mockedLedger) lastRound() basics.Round { + return basics.Round(len(l.history) - 1) +} + +func (l *mockedLedger) WaitMem(r basics.Round) chan struct{} { + l.mu.Lock() + defer l.mu.Unlock() + + if l.waiters[r] == nil { + l.waiters[r] = make(chan struct{}) + } + + // Return an already-closed channel if we already have the block. + if r <= l.lastRound() { + close(l.waiters[r]) + retChan := l.waiters[r] + delete(l.waiters, r) + return retChan + } + + return l.waiters[r] +} + +// BlockHdr allows the service access to consensus values +func (l *mockedLedger) BlockHdr(r basics.Round) (bookkeeping.BlockHeader, error) { + var hdr bookkeeping.BlockHeader + hdr.Round = r + hdr.CurrentProtocol = l.version + return hdr, nil +} + +func (l *mockedLedger) addBlock(delta table) error { + l.mu.Lock() + defer l.mu.Unlock() + + fmt.Printf("addBlock %d\n", l.lastRound()+1) + l.history = append(l.history, delta) + + for r, ch := range l.waiters { + switch { + case r < l.lastRound(): + fmt.Printf("%d < %d\n", r, l.lastRound()) + panic("why is there a waiter for an old block?") + case r == l.lastRound(): + close(ch) + delete(l.waiters, r) + case r > l.lastRound(): + /* waiter keeps waiting */ + } + } + return nil +} + +func (l *mockedLedger) LookupAccount(round basics.Round, addr basics.Address) (ledgercore.AccountData, basics.Round, basics.MicroAlgos, error) { + l.mu.Lock() + defer l.mu.Unlock() + + if round > l.lastRound() { + panic("mockedLedger.LookupAccount: future round") + } + + for r := round; r <= round; r-- { + if acct, ok := l.history[r][addr]; ok { + more := basics.MicroAlgos{Raw: acct.MicroAlgos.Raw + 1} + return acct, round, more, nil + } + } + return ledgercore.AccountData{}, round, basics.MicroAlgos{}, nil +} + +// waitFor confirms that the Service made it through the last block in the +// ledger and is waiting for the next. The Service is written such that it +// operates properly without this sort of wait, but for testing, we often want +// to wait so that we can confirm that the Service *didn't* do something. +func (l *mockedLedger) waitFor(s *Service, a *require.Assertions) { + a.Eventually(func() bool { // delay and confirm that the service advances to wait for next block + _, ok := l.waiters[l.LastRound()+1] + return ok + }, time.Second, 10*time.Millisecond) +} + +type txnSink [][]transactions.SignedTxn + +func (ts *txnSink) BroadcastInternalSignedTxGroup(group []transactions.SignedTxn) error { + fmt.Printf("sinking %+v\n", group[0].Txn.Header) + *ts = append(*ts, group) + return nil +} + +func TestStartStop(t *testing.T) { + partitiontest.PartitionTest(t) + t.Parallel() + + a := require.New(t) + sink := txnSink{} + accts := &mockParticipants{} + ledger := newMockedLedger() + s := NewService(accts, &ledger, &sink, logging.TestingLog(t)) + a.NotNil(s) + a.NoError(ledger.addBlock(nil)) + s.Start() + a.NoError(ledger.addBlock(nil)) + s.Stop() +} + +func makeBlock(r basics.Round) bookkeeping.Block { + return bookkeeping.Block{ + BlockHeader: bookkeeping.BlockHeader{Round: r}, + Payset: []transactions.SignedTxnInBlock{}, + } +} + +func TestHeartBeatOnlyWhenSuspended(t *testing.T) { + partitiontest.PartitionTest(t) + t.Parallel() + + a := require.New(t) + sink := txnSink{} + accts := &mockParticipants{} + ledger := newMockedLedger() + s := NewService(accts, &ledger, &sink, logging.TestingLog(t)) + s.Start() + + // ensure Donor can pay + a.NoError(ledger.addBlock(table{Donor: ledgercore.AccountData{ + AccountBaseData: ledgercore.AccountBaseData{ + MicroAlgos: basics.MicroAlgos{Raw: 1_000_000}, + }, + }})) + a.Empty(sink) + + joe := basics.Address{1, 1} + accts.add(joe) + + acct := ledgercore.AccountData{} + + a.NoError(ledger.addBlock(table{joe: acct})) + ledger.waitFor(s, a) + a.Empty(sink) + + acct.Status = basics.Online + + a.NoError(ledger.addBlock(table{joe: acct})) + a.Empty(sink) + + acct.Status = basics.Suspended + + a.NoError(ledger.addBlock(table{joe: acct})) + ledger.waitFor(s, a) + a.Len(sink, 1) // only one heartbeat so far + a.Len(sink[0], 1) // will probably end up being 3 to pay for `heartbeat` opcode + + s.Stop() +} + +func TestHeartBeatOnlyWhenDonorFunded(t *testing.T) { + partitiontest.PartitionTest(t) + t.Parallel() + + a := require.New(t) + sink := txnSink{} + accts := &mockParticipants{} + ledger := newMockedLedger() + s := NewService(accts, &ledger, &sink, logging.TestingLog(t)) + s.Start() + + joe := basics.Address{1, 1} + accts.add(joe) + + acct := ledgercore.AccountData{} + + a.NoError(ledger.addBlock(table{joe: acct})) + a.Empty(sink) + + acct.Status = basics.Suspended + + a.NoError(ledger.addBlock(table{joe: acct})) + ledger.waitFor(s, a) + a.Empty(sink) // no funded donor, no heartbeat + + // Donor exists, has enough for fee, but not enough when MBR is considered + a.NoError(ledger.addBlock(table{Donor: ledgercore.AccountData{ + AccountBaseData: ledgercore.AccountBaseData{ + MicroAlgos: basics.MicroAlgos{Raw: 100_000}, + }, + }})) + a.NoError(ledger.addBlock(table{joe: acct})) + ledger.waitFor(s, a) + a.Empty(sink) + + a.NoError(ledger.addBlock(table{Donor: ledgercore.AccountData{ + AccountBaseData: ledgercore.AccountBaseData{ + MicroAlgos: basics.MicroAlgos{Raw: 200_000}, + }, + }})) + ledger.waitFor(s, a) + a.Len(sink, 1) // only one heartbeat so far + a.Len(sink[0], 1) // will probably end up being 3 to pay for `heartbeat` opcode + s.Stop() +} + +func TestHeartBeatForm(t *testing.T) { + partitiontest.PartitionTest(t) + t.Parallel() + + a := require.New(t) + sink := txnSink{} + accts := &mockParticipants{} + ledger := newMockedLedger() + s := NewService(accts, &ledger, &sink, logging.TestingLog(t)) + s.Start() + + joe := basics.Address{1, 1} + accts.add(joe) + + // Fund the donor, suspend joe + a.NoError(ledger.addBlock(table{ + Donor: ledgercore.AccountData{ + AccountBaseData: ledgercore.AccountBaseData{ + MicroAlgos: basics.MicroAlgos{Raw: 200_000}, + }, + }, + joe: ledgercore.AccountData{ + AccountBaseData: ledgercore.AccountBaseData{ + Status: basics.Suspended, + MicroAlgos: basics.MicroAlgos{Raw: 2_000_000}, + }, + }, + })) + ledger.waitFor(s, a) + a.Len(sink, 1) // only one heartbeat so far + a.Len(sink[0], 1) // will probably end up being 3 to pay for `heartbeat` opcode + + grp := sink[0] + require.Equal(t, grp[0].Txn.Sender, Donor) + require.Equal(t, grp[0].Lsig, transactions.LogicSig{Logic: DonorByteCode}) + + a.NoError(ledger.addBlock(nil)) + ledger.waitFor(s, a) + a.Len(sink, 2) // still suspended, another heartbeat + inc := sink[0] + inc[0].Txn.FirstValid++ + inc[0].Txn.LastValid++ + a.Equal(inc, sink[1]) + + // mark joe online again + a.NoError(ledger.addBlock(table{ + joe: ledgercore.AccountData{ + AccountBaseData: ledgercore.AccountBaseData{ + Status: basics.Online, + MicroAlgos: basics.MicroAlgos{Raw: 2_000_000}, + }, + }, + })) + ledger.waitFor(s, a) + a.Len(sink, 2) // no further heartbeat + + s.Stop() + +} diff --git a/ledger/apply/apply.go b/ledger/apply/apply.go index dfa61b2632..ecc96c967f 100644 --- a/ledger/apply/apply.go +++ b/ledger/apply/apply.go @@ -25,9 +25,14 @@ import ( "github.com/algorand/go-algorand/ledger/ledgercore" ) +// HdrProvider allows fetching old block headers +type HdrProvider interface { + BlockHdr(r basics.Round) (bookkeeping.BlockHeader, error) +} + // StateProofsApplier allows fetching and updating state-proofs state on the ledger type StateProofsApplier interface { - BlockHdr(r basics.Round) (bookkeeping.BlockHeader, error) + HdrProvider GetStateProofNextRound() basics.Round SetStateProofNextRound(rnd basics.Round) GetStateProofVerificationContext(stateProofLastAttestedRound basics.Round) (*ledgercore.StateProofVerificationContext, error) diff --git a/ledger/apply/heartbeat.go b/ledger/apply/heartbeat.go new file mode 100644 index 0000000000..1c8a55f4b9 --- /dev/null +++ b/ledger/apply/heartbeat.go @@ -0,0 +1,56 @@ +// Copyright (C) 2019-2024 Algorand, Inc. +// This file is part of go-algorand +// +// go-algorand is free software: you can redistribute it and/or modify +// it under the terms of the GNU Affero General Public License as +// published by the Free Software Foundation, either version 3 of the +// License, or (at your option) any later version. +// +// go-algorand is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Affero General Public License for more details. +// +// You should have received a copy of the GNU Affero General Public License +// along with go-algorand. If not, see . + +package apply + +import ( + "errors" + "fmt" + + "github.com/algorand/go-algorand/data/basics" + "github.com/algorand/go-algorand/data/transactions" +) + +// Heartbeat applies a Heartbeat transaction using the Balances interface. +func Heartbeat(hb transactions.HeartbeatTxnFields, header transactions.Header, balances Balances, provider HdrProvider, round basics.Round) error { + // Get the account's balance entry + account, err := balances.Get(hb.HeartbeatAddress, false) + + sv := account.VoteID + if sv.IsEmpty() { + return fmt.Errorf("HeartbeatAddress %s has has no voting keys\n", hb.HeartbeatAddress) + } + id := basics.OneTimeIDForRound(header.LastValid, account.VoteKeyDilution) + + hdr, err := provider.BlockHdr(header.FirstValid - 1) + if err != nil { + return err + } + + if !sv.Verify(id, hdr.Seed, hb.Proof) { + return errors.New("Improper heartbeat") + } + + account.LastHeartbeat = round + + // Write the updated entry + err = balances.Put(hb.HeartbeatAddress, account) + if err != nil { + return err + } + + return nil +} diff --git a/ledger/apply/heartbeat_test.go b/ledger/apply/heartbeat_test.go new file mode 100644 index 0000000000..e6fe7d66a9 --- /dev/null +++ b/ledger/apply/heartbeat_test.go @@ -0,0 +1,90 @@ +// Copyright (C) 2019-2024 Algorand, Inc. +// This file is part of go-algorand +// +// go-algorand is free software: you can redistribute it and/or modify +// it under the terms of the GNU Affero General Public License as +// published by the Free Software Foundation, either version 3 of the +// License, or (at your option) any later version. +// +// go-algorand is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Affero General Public License for more details. +// +// You should have received a copy of the GNU Affero General Public License +// along with go-algorand. If not, see . + +package apply + +import ( + "testing" + + "github.com/stretchr/testify/require" + + "github.com/algorand/go-algorand/crypto" + "github.com/algorand/go-algorand/data/basics" + "github.com/algorand/go-algorand/data/bookkeeping" + "github.com/algorand/go-algorand/data/committee" + "github.com/algorand/go-algorand/data/transactions" + "github.com/algorand/go-algorand/protocol" + "github.com/algorand/go-algorand/test/partitiontest" +) + +func TestHeartbeat(t *testing.T) { + partitiontest.PartitionTest(t) + t.Parallel() + + // Creator + sender := basics.Address{0x01} + voter := basics.Address{0x02} + const keyDilution = 777 + + fv := basics.Round(100) + lv := basics.Round(1000) + + id := basics.OneTimeIDForRound(lv, keyDilution) + otss := crypto.GenerateOneTimeSignatureSecrets(1, 2) // This will cover rounds 1-2*777 + + mockBal := makeMockBalancesWithAccounts(protocol.ConsensusFuture, map[basics.Address]basics.AccountData{ + sender: { + MicroAlgos: basics.MicroAlgos{Raw: 10_000_000}, + }, + voter: { + Status: basics.Online, + MicroAlgos: basics.MicroAlgos{Raw: 100_000_000}, + VoteID: otss.OneTimeSignatureVerifier, + VoteKeyDilution: keyDilution, + IncentiveEligible: true, + }, + }) + + seed := committee.Seed{0x01, 0x02, 0x03} + mockHdr := makeMockHeaders(bookkeeping.BlockHeader{ + Round: fv - 1, + Seed: seed, + }) + + tx := transactions.Transaction{ + Type: protocol.HeartbeatTx, + Header: transactions.Header{ + Sender: sender, + Fee: basics.MicroAlgos{Raw: 1}, + FirstValid: fv, + LastValid: lv, + }, + HeartbeatTxnFields: transactions.HeartbeatTxnFields{ + HeartbeatAddress: voter, + Proof: otss.Sign(id, seed), + Seed: seed, + }, + } + + rnd := basics.Round(150) + err := Heartbeat(tx.HeartbeatTxnFields, tx.Header, mockBal, mockHdr, rnd) + require.NoError(t, err) + + after, err := mockBal.Get(voter, false) + require.NoError(t, err) + require.Equal(t, rnd, after.LastHeartbeat) + require.Zero(t, after.LastProposed) // unchanged +} diff --git a/ledger/apply/mockBalances_test.go b/ledger/apply/mockBalances_test.go index 43af5fa11d..dd77b66e7f 100644 --- a/ledger/apply/mockBalances_test.go +++ b/ledger/apply/mockBalances_test.go @@ -17,8 +17,11 @@ package apply import ( + "fmt" + "github.com/algorand/go-algorand/config" "github.com/algorand/go-algorand/data/basics" + "github.com/algorand/go-algorand/data/bookkeeping" "github.com/algorand/go-algorand/data/transactions" "github.com/algorand/go-algorand/data/transactions/logic" "github.com/algorand/go-algorand/ledger/ledgercore" @@ -270,3 +273,23 @@ func (b *mockCreatableBalances) HasAssetParams(addr basics.Address, aidx basics. _, ok = acct.AssetParams[aidx] return } + +type mockHeaders struct { + b map[basics.Round]bookkeeping.BlockHeader +} + +// makeMockHeaders takes a bunch of BlockHeaders and returns a HdrProivder for them. +func makeMockHeaders(hdrs ...bookkeeping.BlockHeader) mockHeaders { + b := make(map[basics.Round]bookkeeping.BlockHeader) + for _, hdr := range hdrs { + b[hdr.Round] = hdr + } + return mockHeaders{b: b} +} + +func (m mockHeaders) BlockHdr(r basics.Round) (bookkeeping.BlockHeader, error) { + if hdr, ok := m.b[r]; ok { + return hdr, nil + } + return bookkeeping.BlockHeader{}, fmt.Errorf("Round %v is not present\n", r) +} diff --git a/ledger/eval/eval.go b/ledger/eval/eval.go index 859b62922f..419a960692 100644 --- a/ledger/eval/eval.go +++ b/ledger/eval/eval.go @@ -1285,6 +1285,9 @@ func (eval *BlockEvaluator) applyTransaction(tx transactions.Transaction, cow *r // Validation of the StateProof transaction before applying will only occur in validate mode. err = apply.StateProof(tx.StateProofTxnFields, tx.Header.FirstValid, cow, eval.validate) + case protocol.HeartbeatTx: + err = apply.Heartbeat(tx.HeartbeatTxnFields, tx.Header, cow, cow, cow.Round()) + default: err = fmt.Errorf("unknown transaction type %v", tx.Type) } @@ -1618,7 +1621,7 @@ func (eval *BlockEvaluator) generateKnockOfflineAccountsList() { updates := &eval.block.ParticipationUpdates - ch := activeChallenge(&eval.proto, uint64(eval.Round()), eval.state) + ch := ActiveChallenge(eval.proto.Payouts, eval.Round(), eval.state) for _, accountAddr := range eval.state.modifiedAccounts() { acctData, found := eval.state.mods.Accts.GetData(accountAddr) @@ -1648,7 +1651,7 @@ func (eval *BlockEvaluator) generateKnockOfflineAccountsList() { if acctData.Status == basics.Online { lastSeen := max(acctData.LastProposed, acctData.LastHeartbeat) if isAbsent(eval.state.prevTotals.Online.Money, acctData.MicroAlgos, lastSeen, current) || - failsChallenge(ch, accountAddr, lastSeen) { + FailsChallenge(ch, accountAddr, lastSeen) { updates.AbsentParticipationAccounts = append( updates.AbsentParticipationAccounts, accountAddr, @@ -1709,20 +1712,19 @@ type headerSource interface { BlockHdr(round basics.Round) (bookkeeping.BlockHeader, error) } -func activeChallenge(proto *config.ConsensusParams, current uint64, headers headerSource) challenge { - rules := proto.Payouts +func ActiveChallenge(rules config.ProposerPayoutRules, current basics.Round, headers headerSource) challenge { // are challenges active? - if rules.ChallengeInterval == 0 || current < rules.ChallengeInterval { + interval := basics.Round(rules.ChallengeInterval) + if rules.ChallengeInterval == 0 || current < interval { return challenge{} } - lastChallenge := current - (current % rules.ChallengeInterval) + lastChallenge := current - (current % interval) + grace := basics.Round(rules.ChallengeGracePeriod) // challenge is in effect if we're after one grace period, but before the 2nd ends. - if current <= lastChallenge+rules.ChallengeGracePeriod || - current > lastChallenge+2*rules.ChallengeGracePeriod { + if current <= lastChallenge+grace || current > lastChallenge+2*grace { return challenge{} } - round := basics.Round(lastChallenge) - challengeHdr, err := headers.BlockHdr(round) + challengeHdr, err := headers.BlockHdr(lastChallenge) if err != nil { panic(err) } @@ -1731,10 +1733,10 @@ func activeChallenge(proto *config.ConsensusParams, current uint64, headers head if challengeProto.Payouts != rules { return challenge{} } - return challenge{round, challengeHdr.Seed, rules.ChallengeBits} + return challenge{lastChallenge, challengeHdr.Seed, rules.ChallengeBits} } -func failsChallenge(ch challenge, address basics.Address, lastSeen basics.Round) bool { +func FailsChallenge(ch challenge, address basics.Address, lastSeen basics.Round) bool { return ch.round != 0 && bitsMatch(ch.seed[:], address[:], ch.bits) && lastSeen < ch.round } @@ -1805,7 +1807,7 @@ func (eval *BlockEvaluator) validateAbsentOnlineAccounts() error { // For consistency with expired account handling, we preclude duplicates addressSet := make(map[basics.Address]bool, suspensionCount) - ch := activeChallenge(&eval.proto, uint64(eval.Round()), eval.state) + ch := ActiveChallenge(eval.proto.Payouts, eval.Round(), eval.state) for _, accountAddr := range eval.block.ParticipationUpdates.AbsentParticipationAccounts { if _, exists := addressSet[accountAddr]; exists { @@ -1826,7 +1828,7 @@ func (eval *BlockEvaluator) validateAbsentOnlineAccounts() error { if isAbsent(eval.state.prevTotals.Online.Money, acctData.MicroAlgos, lastSeen, eval.Round()) { continue // ok. it's "normal absent" } - if failsChallenge(ch, accountAddr, lastSeen) { + if FailsChallenge(ch, accountAddr, lastSeen) { continue // ok. it's "challenge absent" } return fmt.Errorf("proposed absent account %v is not absent in %d, %d", diff --git a/ledger/eval/eval_test.go b/ledger/eval/eval_test.go index 77a477b3c0..620770326b 100644 --- a/ledger/eval/eval_test.go +++ b/ledger/eval/eval_test.go @@ -1644,16 +1644,16 @@ func TestFailsChallenge(t *testing.T) { a := assert.New(t) // a valid challenge, with 4 matching bits, and an old last seen - a.True(failsChallenge(challenge{round: 11, seed: [32]byte{0xb0, 0xb4}, bits: 4}, basics.Address{0xbf, 0x34}, 10)) + a.True(FailsChallenge(challenge{round: 11, seed: [32]byte{0xb0, 0xb4}, bits: 4}, basics.Address{0xbf, 0x34}, 10)) // challenge isn't "on" - a.False(failsChallenge(challenge{round: 0, seed: [32]byte{0xb0, 0xb4}, bits: 4}, basics.Address{0xbf, 0x34}, 10)) + a.False(FailsChallenge(challenge{round: 0, seed: [32]byte{0xb0, 0xb4}, bits: 4}, basics.Address{0xbf, 0x34}, 10)) // node has appeared more recently - a.False(failsChallenge(challenge{round: 11, seed: [32]byte{0xb0, 0xb4}, bits: 4}, basics.Address{0xbf, 0x34}, 12)) + a.False(FailsChallenge(challenge{round: 11, seed: [32]byte{0xb0, 0xb4}, bits: 4}, basics.Address{0xbf, 0x34}, 12)) // bits don't match - a.False(failsChallenge(challenge{round: 11, seed: [32]byte{0xb0, 0xb4}, bits: 4}, basics.Address{0xcf, 0x34}, 10)) + a.False(FailsChallenge(challenge{round: 11, seed: [32]byte{0xb0, 0xb4}, bits: 4}, basics.Address{0xcf, 0x34}, 10)) // no enough bits match - a.False(failsChallenge(challenge{round: 11, seed: [32]byte{0xb0, 0xb4}, bits: 5}, basics.Address{0xbf, 0x34}, 10)) + a.False(FailsChallenge(challenge{round: 11, seed: [32]byte{0xb0, 0xb4}, bits: 5}, basics.Address{0xbf, 0x34}, 10)) } type singleSource bookkeeping.BlockHeader @@ -1673,27 +1673,27 @@ func TestActiveChallenge(t *testing.T) { CurrentProtocol: protocol.ConsensusFuture, }, } - now := config.Consensus[nowHeader.CurrentProtocol] + rules := config.Consensus[nowHeader.CurrentProtocol].Payouts // simplest test. when interval=X and grace=G, X+G+1 is a challenge - inChallenge := now.Payouts.ChallengeInterval + now.Payouts.ChallengeGracePeriod + 1 - ch := activeChallenge(&now, inChallenge, singleSource(nowHeader)) + inChallenge := rules.ChallengeInterval + rules.ChallengeGracePeriod + 1 + ch := ActiveChallenge(rules, inChallenge, singleSource(nowHeader)) a.NotZero(ch.round) // all rounds before that have no challenge for r := uint64(1); r < inChallenge; r++ { - ch := activeChallenge(&now, r, singleSource(nowHeader)) + ch := ActiveChallenge(rules, r, singleSource(nowHeader)) a.Zero(ch.round, r) } // ChallengeGracePeriod rounds allow challenges starting with inChallenge - for r := inChallenge; r < inChallenge+now.Payouts.ChallengeGracePeriod; r++ { - ch := activeChallenge(&now, r, singleSource(nowHeader)) - a.EqualValues(ch.round, now.Payouts.ChallengeInterval) + for r := inChallenge; r < inChallenge+rules.ChallengeGracePeriod; r++ { + ch := ActiveChallenge(rules, r, singleSource(nowHeader)) + a.EqualValues(ch.round, rules.ChallengeInterval) } // And the next round is again challenge-less - ch = activeChallenge(&now, inChallenge+now.Payouts.ChallengeGracePeriod, singleSource(nowHeader)) + ch = ActiveChallenge(rules, inChallenge+rules.ChallengeGracePeriod, singleSource(nowHeader)) a.Zero(ch.round) // ignore challenge if upgrade happened @@ -1703,6 +1703,6 @@ func TestActiveChallenge(t *testing.T) { CurrentProtocol: protocol.ConsensusV39, }, } - ch = activeChallenge(&now, inChallenge, singleSource(oldHeader)) + ch = ActiveChallenge(rules, inChallenge, singleSource(oldHeader)) a.Zero(ch.round) } diff --git a/ledger/ledger.go b/ledger/ledger.go index 2f10724fee..6832ac9ec7 100644 --- a/ledger/ledger.go +++ b/ledger/ledger.go @@ -717,7 +717,7 @@ func (l *Ledger) Block(rnd basics.Round) (blk bookkeeping.Block, err error) { func (l *Ledger) BlockHdr(rnd basics.Round) (blk bookkeeping.BlockHeader, err error) { // Expected availability range in txTail.blockHeader is [Latest - MaxTxnLife, Latest] - // allowing (MaxTxnLife + 1) = 1001 rounds back loopback. + // allowing (MaxTxnLife + 1) = 1001 rounds lookback. // The depth besides the MaxTxnLife is controlled by DeeperBlockHeaderHistory parameter // and currently set to 1. // Explanation: diff --git a/node/node.go b/node/node.go index dddb3203e3..d3ce99901b 100644 --- a/node/node.go +++ b/node/node.go @@ -43,6 +43,7 @@ import ( "github.com/algorand/go-algorand/data/pools" "github.com/algorand/go-algorand/data/transactions" "github.com/algorand/go-algorand/data/transactions/verify" + "github.com/algorand/go-algorand/heartbeat" "github.com/algorand/go-algorand/ledger" "github.com/algorand/go-algorand/ledger/ledgercore" "github.com/algorand/go-algorand/ledger/simulation" @@ -155,6 +156,8 @@ type AlgorandFullNode struct { stateProofWorker *stateproof.Worker partHandles []db.Accessor + + heartbeatService *heartbeat.Service } // TxnWithStatus represents information about a single transaction, @@ -338,6 +341,8 @@ func MakeFull(log logging.Logger, rootDir string, cfg config.Local, phonebookAdd node.stateProofWorker = stateproof.NewWorker(node.genesisDirs.StateproofGenesisDir, node.log, node.accountManager, node.ledger.Ledger, node.net, node) + node.heartbeatService = heartbeat.NewService(node.accountManager, node.ledger, node, node.log) + return node, err } diff --git a/protocol/txntype.go b/protocol/txntype.go index 76cb2dc406..ee2d085dcb 100644 --- a/protocol/txntype.go +++ b/protocol/txntype.go @@ -47,6 +47,9 @@ const ( // StateProofTx records a state proof StateProofTx TxType = "stpf" + // HeartbeatTx demonstrates the account is alive + HeartbeatTx TxType = "hb" + // UnknownTx signals an error UnknownTx TxType = "unknown" ) diff --git a/stateproof/builder.go b/stateproof/builder.go index 317e813602..a97ec752c6 100644 --- a/stateproof/builder.go +++ b/stateproof/builder.go @@ -669,7 +669,7 @@ func (spw *Worker) tryBroadcast() { latestHeader, err := spw.ledger.BlockHdr(firstValid) if err != nil { - spw.log.Warnf("spw.tryBroadcast: could not fetch block header for round %d failed: %v", firstValid, err) + spw.log.Warnf("spw.tryBroadcast: could not fetch block header for round %d: %v", firstValid, err) break } From 6f3f3c15e2eee5bc9a45eb92893ea21053bb83ad Mon Sep 17 00:00:00 2001 From: John Jannotti Date: Tue, 15 Oct 2024 11:53:59 -0400 Subject: [PATCH 02/19] Small CI fixups --- data/transactions/logic/ledger_test.go | 29 ++++- heartbeat/abstractions.go | 6 +- heartbeat/service.go | 2 +- heartbeat/service_test.go | 155 ++++++------------------- 4 files changed, 65 insertions(+), 127 deletions(-) diff --git a/data/transactions/logic/ledger_test.go b/data/transactions/logic/ledger_test.go index 3dcead5e51..b16694d48c 100644 --- a/data/transactions/logic/ledger_test.go +++ b/data/transactions/logic/ledger_test.go @@ -46,9 +46,14 @@ import ( ) type balanceRecord struct { - addr basics.Address - auth basics.Address - balance uint64 + addr basics.Address + auth basics.Address + balance uint64 + voting basics.VotingData + + proposed basics.Round // The last round that this account proposed the accepted block + heartbeat basics.Round // The last round that this account sent a heartbeat to show it was online. + locals map[basics.AppIndex]basics.TealKeyValue holdings map[basics.AssetIndex]basics.AssetHolding mods map[basics.AppIndex]map[string]basics.ValueDelta @@ -119,6 +124,18 @@ func (l *Ledger) NewAccount(addr basics.Address, balance uint64) { l.balances[addr] = newBalanceRecord(addr, balance) } +// NewVoting sets VoteID on the account. Could expand to set other voting data +// if that became useful in tests. +func (l *Ledger) NewVoting(addr basics.Address, voteID crypto.OneTimeSignatureVerifier) { + br, ok := l.balances[addr] + if !ok { + br = newBalanceRecord(addr, 0) + } + br.voting.VoteID = voteID + br.voting.VoteKeyDilution = 10_000 + l.balances[addr] = br +} + // NewApp add a new AVM app to the Ledger. In most uses, it only sets up the id // and schema but no code, as testing will want to try many different code // sequences. @@ -312,7 +329,11 @@ func (l *Ledger) AccountData(addr basics.Address) (ledgercore.AccountData, error TotalBoxes: uint64(boxesTotal), TotalBoxBytes: uint64(boxBytesTotal), + + LastProposed: br.proposed, + LastHeartbeat: br.heartbeat, }, + VotingData: br.voting, }, nil } @@ -952,6 +973,8 @@ func (l *Ledger) Get(addr basics.Address, withPendingRewards bool) (basics.Accou Assets: map[basics.AssetIndex]basics.AssetHolding{}, AppLocalStates: map[basics.AppIndex]basics.AppLocalState{}, AppParams: map[basics.AppIndex]basics.AppParams{}, + LastProposed: br.proposed, + LastHeartbeat: br.heartbeat, }, nil } diff --git a/heartbeat/abstractions.go b/heartbeat/abstractions.go index 52206a0ef7..9ccecb6fb9 100644 --- a/heartbeat/abstractions.go +++ b/heartbeat/abstractions.go @@ -1,4 +1,4 @@ -// Copyright (C) 2019-2023 Algorand, Inc. +// Copyright (C) 2019-2024 Algorand, Inc. // This file is part of go-algorand // // go-algorand is free software: you can redistribute it and/or modify @@ -30,8 +30,8 @@ type txnBroadcaster interface { BroadcastInternalSignedTxGroup([]transactions.SignedTxn) error } -// ledger represents the aspects of the "real" Ledger that heartbeat needs. -// to interact with. +// ledger represents the aspects of the "real" Ledger that the heartbeat service +// needs to interact with type ledger interface { // LastRound tells the round is ready for checking LastRound() basics.Round diff --git a/heartbeat/service.go b/heartbeat/service.go index 8f9775cc50..48d99cb959 100644 --- a/heartbeat/service.go +++ b/heartbeat/service.go @@ -1,4 +1,4 @@ -// Copyright (C) 2019-2023 Algorand, Inc. +// Copyright (C) 2019-2024 Algorand, Inc. // This file is part of go-algorand // // go-algorand is free software: you can redistribute it and/or modify diff --git a/heartbeat/service_test.go b/heartbeat/service_test.go index aef54dd29f..2246bde174 100644 --- a/heartbeat/service_test.go +++ b/heartbeat/service_test.go @@ -1,4 +1,4 @@ -// Copyright (C) 2019-2023 Algorand, Inc. +// Copyright (C) 2019-2024 Algorand, Inc. // This file is part of go-algorand // // go-algorand is free software: you can redistribute it and/or modify @@ -21,6 +21,7 @@ import ( "testing" "time" + "github.com/algorand/go-algorand/config" "github.com/algorand/go-algorand/data/account" "github.com/algorand/go-algorand/data/basics" "github.com/algorand/go-algorand/data/bookkeeping" @@ -69,6 +70,7 @@ type mockedLedger struct { waiters map[basics.Round]chan struct{} history []table version protocol.ConsensusVersion + hdrs map[basics.Round]bookkeeping.BlockHeader } func newMockedLedger() mockedLedger { @@ -110,12 +112,25 @@ func (l *mockedLedger) WaitMem(r basics.Round) chan struct{} { // BlockHdr allows the service access to consensus values func (l *mockedLedger) BlockHdr(r basics.Round) (bookkeeping.BlockHeader, error) { + if r > l.LastRound() { + return bookkeeping.BlockHeader{}, fmt.Errorf("%d is beyond current block (%d)", r, l.LastRound()) + } + if hdr, ok := l.hdrs[r]; ok { + return hdr, nil + } + // just return a simple hdr var hdr bookkeeping.BlockHeader hdr.Round = r hdr.CurrentProtocol = l.version return hdr, nil } +// addHeader places a block header into the ledger's history. It is used to make +// challenges occur as we'd like. +func (l *mockedLedger) addHeader(hdr bookkeeping.BlockHeader) { + l.hdrs[hdr.Round] = hdr +} + func (l *mockedLedger) addBlock(delta table) error { l.mu.Lock() defer l.mu.Unlock() @@ -197,7 +212,7 @@ func makeBlock(r basics.Round) bookkeeping.Block { } } -func TestHeartBeatOnlyWhenSuspended(t *testing.T) { +func TestHeartBeatOnlyWhenChallenged(t *testing.T) { partitiontest.PartitionTest(t) t.Parallel() @@ -208,15 +223,8 @@ func TestHeartBeatOnlyWhenSuspended(t *testing.T) { s := NewService(accts, &ledger, &sink, logging.TestingLog(t)) s.Start() - // ensure Donor can pay - a.NoError(ledger.addBlock(table{Donor: ledgercore.AccountData{ - AccountBaseData: ledgercore.AccountBaseData{ - MicroAlgos: basics.MicroAlgos{Raw: 1_000_000}, - }, - }})) - a.Empty(sink) - - joe := basics.Address{1, 1} + // joe is a simple, non-online account, service will not heartbeat + joe := basics.Address{0xcc} // 0xcc will matter when we set the challenge accts.add(joe) acct := ledgercore.AccountData{} @@ -225,123 +233,30 @@ func TestHeartBeatOnlyWhenSuspended(t *testing.T) { ledger.waitFor(s, a) a.Empty(sink) + // now joe is online, but not challenged, so no heartbeat acct.Status = basics.Online a.NoError(ledger.addBlock(table{joe: acct})) a.Empty(sink) - acct.Status = basics.Suspended + // now we have to make it seem like joe has been challenged. We obtain the + // payout rules to find the first challenge round, skip forward to it, then + // go forward half a grace period. Only then should the service heartbeat + hdr, err := ledger.BlockHdr(ledger.LastRound()) + a.NoError(err) + rules := config.Consensus[hdr.CurrentProtocol].Payouts + for ledger.LastRound() < basics.Round(rules.ChallengeInterval) { + a.NoError(ledger.addBlock(table{})) + ledger.waitFor(s, a) + a.Empty(sink) + } a.NoError(ledger.addBlock(table{joe: acct})) ledger.waitFor(s, a) - a.Len(sink, 1) // only one heartbeat so far - a.Len(sink[0], 1) // will probably end up being 3 to pay for `heartbeat` opcode + a.Len(sink, 1) // only one heartbeat so far + a.Len(sink[0], 1) + a.Equal(sink[0][0].Txn.Type, protocol.HeartbeatTx) + a.Equal(sink[0][0].Txn.HeartbeatAddress, joe) s.Stop() } - -func TestHeartBeatOnlyWhenDonorFunded(t *testing.T) { - partitiontest.PartitionTest(t) - t.Parallel() - - a := require.New(t) - sink := txnSink{} - accts := &mockParticipants{} - ledger := newMockedLedger() - s := NewService(accts, &ledger, &sink, logging.TestingLog(t)) - s.Start() - - joe := basics.Address{1, 1} - accts.add(joe) - - acct := ledgercore.AccountData{} - - a.NoError(ledger.addBlock(table{joe: acct})) - a.Empty(sink) - - acct.Status = basics.Suspended - - a.NoError(ledger.addBlock(table{joe: acct})) - ledger.waitFor(s, a) - a.Empty(sink) // no funded donor, no heartbeat - - // Donor exists, has enough for fee, but not enough when MBR is considered - a.NoError(ledger.addBlock(table{Donor: ledgercore.AccountData{ - AccountBaseData: ledgercore.AccountBaseData{ - MicroAlgos: basics.MicroAlgos{Raw: 100_000}, - }, - }})) - a.NoError(ledger.addBlock(table{joe: acct})) - ledger.waitFor(s, a) - a.Empty(sink) - - a.NoError(ledger.addBlock(table{Donor: ledgercore.AccountData{ - AccountBaseData: ledgercore.AccountBaseData{ - MicroAlgos: basics.MicroAlgos{Raw: 200_000}, - }, - }})) - ledger.waitFor(s, a) - a.Len(sink, 1) // only one heartbeat so far - a.Len(sink[0], 1) // will probably end up being 3 to pay for `heartbeat` opcode - s.Stop() -} - -func TestHeartBeatForm(t *testing.T) { - partitiontest.PartitionTest(t) - t.Parallel() - - a := require.New(t) - sink := txnSink{} - accts := &mockParticipants{} - ledger := newMockedLedger() - s := NewService(accts, &ledger, &sink, logging.TestingLog(t)) - s.Start() - - joe := basics.Address{1, 1} - accts.add(joe) - - // Fund the donor, suspend joe - a.NoError(ledger.addBlock(table{ - Donor: ledgercore.AccountData{ - AccountBaseData: ledgercore.AccountBaseData{ - MicroAlgos: basics.MicroAlgos{Raw: 200_000}, - }, - }, - joe: ledgercore.AccountData{ - AccountBaseData: ledgercore.AccountBaseData{ - Status: basics.Suspended, - MicroAlgos: basics.MicroAlgos{Raw: 2_000_000}, - }, - }, - })) - ledger.waitFor(s, a) - a.Len(sink, 1) // only one heartbeat so far - a.Len(sink[0], 1) // will probably end up being 3 to pay for `heartbeat` opcode - - grp := sink[0] - require.Equal(t, grp[0].Txn.Sender, Donor) - require.Equal(t, grp[0].Lsig, transactions.LogicSig{Logic: DonorByteCode}) - - a.NoError(ledger.addBlock(nil)) - ledger.waitFor(s, a) - a.Len(sink, 2) // still suspended, another heartbeat - inc := sink[0] - inc[0].Txn.FirstValid++ - inc[0].Txn.LastValid++ - a.Equal(inc, sink[1]) - - // mark joe online again - a.NoError(ledger.addBlock(table{ - joe: ledgercore.AccountData{ - AccountBaseData: ledgercore.AccountBaseData{ - Status: basics.Online, - MicroAlgos: basics.MicroAlgos{Raw: 2_000_000}, - }, - }, - })) - ledger.waitFor(s, a) - a.Len(sink, 2) // no further heartbeat - - s.Stop() - -} From 018c8e2cf5f5d021b8e83c183f553375eea4dd08 Mon Sep 17 00:00:00 2001 From: John Jannotti Date: Tue, 15 Oct 2024 13:19:40 -0400 Subject: [PATCH 03/19] break dependency on transactions package --- data/committee/common_test.go | 65 +++++++------------------------ data/committee/credential_test.go | 20 +++++----- ledger/eval/eval_test.go | 8 ++-- 3 files changed, 29 insertions(+), 64 deletions(-) diff --git a/data/committee/common_test.go b/data/committee/common_test.go index 1f7e7bd373..eef035b8e2 100644 --- a/data/committee/common_test.go +++ b/data/committee/common_test.go @@ -24,7 +24,6 @@ import ( "github.com/algorand/go-algorand/config" "github.com/algorand/go-algorand/crypto" "github.com/algorand/go-algorand/data/basics" - "github.com/algorand/go-algorand/data/transactions" "github.com/algorand/go-algorand/protocol" ) @@ -33,40 +32,33 @@ type selectionParameterListFn func(addr []basics.Address) (bool, []BalanceRecord var proto = config.Consensus[protocol.ConsensusCurrentVersion] -func newAccount(t testing.TB, gen io.Reader, latest basics.Round, keyBatchesForward uint) (basics.Address, *crypto.SignatureSecrets, *crypto.VrfPrivkey, *crypto.OneTimeSignatureSecrets) { +func newAccount(t testing.TB, gen io.Reader, latest basics.Round, keyBatchesForward uint) (basics.Address, *crypto.SignatureSecrets, *crypto.VrfPrivkey) { var seed crypto.Seed gen.Read(seed[:]) s := crypto.GenerateSignatureSecrets(seed) _, v := crypto.VrfKeygenFromSeed(seed) - o := crypto.GenerateOneTimeSignatureSecrets(basics.OneTimeIDForRound(latest, proto.DefaultKeyDilution).Batch, uint64(keyBatchesForward)) addr := basics.Address(s.SignatureVerifier) - return addr, s, &v, o + return addr, s, &v } -func signTx(s *crypto.SignatureSecrets, t transactions.Transaction) transactions.SignedTxn { - return t.Sign(s) -} - -// testingenv creates a random set of participating accounts and random transactions between them, and -// the associated selection parameters for use testing committee membership and credential validation. -// seedGen is provided as an external source of randomness for the selection seed and transaction notes; -// if the caller persists seedGen between calls to testingenv, each iteration that calls testingenv will -// exercise a new selection seed. -func testingenv(t testing.TB, numAccounts, numTxs int, seedGen io.Reader) (selectionParameterFn, selectionParameterListFn, basics.Round, []basics.Address, []*crypto.SignatureSecrets, []*crypto.VrfPrivkey, []*crypto.OneTimeSignatureSecrets, []transactions.SignedTxn) { +// testingenv creates a random set of participating accounts and the associated +// selection parameters for use testing committee membership and credential +// validation. seedGen is provided as an external source of randomness for the +// selection seed; if the caller persists seedGen between calls to testingenv, +// each iteration that calls testingenv will exercise a new selection seed. +// formerly, testingenv, generated transactions and one-time secrets as well, +// but they were not used by the tests. +func testingenv(t testing.TB, numAccounts, numTxs int, seedGen io.Reader) (selectionParameterFn, selectionParameterListFn, basics.Round, []basics.Address, []*crypto.SignatureSecrets, []*crypto.VrfPrivkey) { return testingenvMoreKeys(t, numAccounts, numTxs, uint(5), seedGen) } -func testingenvMoreKeys(t testing.TB, numAccounts, numTxs int, keyBatchesForward uint, seedGen io.Reader) (selectionParameterFn, selectionParameterListFn, basics.Round, []basics.Address, []*crypto.SignatureSecrets, []*crypto.VrfPrivkey, []*crypto.OneTimeSignatureSecrets, []transactions.SignedTxn) { +func testingenvMoreKeys(t testing.TB, numAccounts, numTxs int, keyBatchesForward uint, seedGen io.Reader) (selectionParameterFn, selectionParameterListFn, basics.Round, []basics.Address, []*crypto.SignatureSecrets, []*crypto.VrfPrivkey) { if seedGen == nil { seedGen = rand.New(rand.NewSource(1)) // same source as setting GODEBUG=randautoseed=0, same as pre-Go 1.20 default seed } P := numAccounts // n accounts - TXs := numTxs // n txns maxMoneyAtStart := 100000 // max money start minMoneyAtStart := 10000 // max money start - transferredMoney := 100 // max money/txn - maxFee := 10 // max maxFee/txn - E := basics.Round(50) // max round // generate accounts genesis := make(map[basics.Address]basics.AccountData) @@ -74,16 +66,14 @@ func testingenvMoreKeys(t testing.TB, numAccounts, numTxs int, keyBatchesForward addrs := make([]basics.Address, P) secrets := make([]*crypto.SignatureSecrets, P) vrfSecrets := make([]*crypto.VrfPrivkey, P) - otSecrets := make([]*crypto.OneTimeSignatureSecrets, P) proto := config.Consensus[protocol.ConsensusCurrentVersion] lookback := basics.Round(2*proto.SeedRefreshInterval + proto.SeedLookback + 1) var total basics.MicroAlgos for i := 0; i < P; i++ { - addr, sigSec, vrfSec, otSec := newAccount(t, gen, lookback, keyBatchesForward) + addr, sigSec, vrfSec := newAccount(t, gen, lookback, keyBatchesForward) addrs[i] = addr secrets[i] = sigSec vrfSecrets[i] = vrfSec - otSecrets[i] = otSec startamt := uint64(minMoneyAtStart + (gen.Int() % (maxMoneyAtStart - minMoneyAtStart))) short := addr @@ -91,7 +81,6 @@ func testingenvMoreKeys(t testing.TB, numAccounts, numTxs int, keyBatchesForward Status: basics.Online, MicroAlgos: basics.MicroAlgos{Raw: startamt}, SelectionID: vrfSec.Pubkey(), - VoteID: otSec.OneTimeSignatureVerifier, } total.Raw += startamt } @@ -99,32 +88,8 @@ func testingenvMoreKeys(t testing.TB, numAccounts, numTxs int, keyBatchesForward var seed Seed seedGen.Read(seed[:]) - tx := make([]transactions.SignedTxn, TXs) - for i := 0; i < TXs; i++ { - send := gen.Int() % P - recv := gen.Int() % P - - saddr := addrs[send] - raddr := addrs[recv] - amt := basics.MicroAlgos{Raw: uint64(gen.Int() % transferredMoney)} - fee := basics.MicroAlgos{Raw: uint64(gen.Int() % maxFee)} - - t := transactions.Transaction{ - Type: protocol.PaymentTx, - Header: transactions.Header{ - Sender: saddr, - Fee: fee, - FirstValid: 0, - LastValid: E, - Note: make([]byte, 4), - }, - PaymentTxnFields: transactions.PaymentTxnFields{ - Receiver: raddr, - Amount: amt, - }, - } - seedGen.Read(t.Note) // to match output from previous versions, which shared global RNG for seed & note - tx[i] = t.Sign(secrets[send]) + for i := 0; i < numTxs; i++ { + seedGen.Read(make([]byte, 4)) // to match output from previous versions, which shared global RNG for seed & note } selParams := func(addr basics.Address) (bool, BalanceRecord, Seed, basics.MicroAlgos) { @@ -149,7 +114,7 @@ func testingenvMoreKeys(t testing.TB, numAccounts, numTxs int, keyBatchesForward return } - return selParams, selParamsList, lookback, addrs, secrets, vrfSecrets, otSecrets, tx + return selParams, selParamsList, lookback, addrs, secrets, vrfSecrets } /* TODO deprecate these types after they have been removed successfully */ diff --git a/data/committee/credential_test.go b/data/committee/credential_test.go index da2be625cd..b646efdf0a 100644 --- a/data/committee/credential_test.go +++ b/data/committee/credential_test.go @@ -35,7 +35,7 @@ func TestAccountSelected(t *testing.T) { seedGen := rand.New(rand.NewSource(1)) N := 1 for i := 0; i < N; i++ { - selParams, _, round, addresses, _, vrfSecrets, _, _ := testingenv(t, 100, 2000, seedGen) + selParams, _, round, addresses, _, vrfSecrets := testingenv(t, 100, 2000, seedGen) period := Period(0) leaders := uint64(0) @@ -98,7 +98,7 @@ func TestAccountSelected(t *testing.T) { func TestRichAccountSelected(t *testing.T) { partitiontest.PartitionTest(t) - selParams, _, round, addresses, _, vrfSecrets, _, _ := testingenv(t, 10, 2000, nil) + selParams, _, round, addresses, _, vrfSecrets := testingenv(t, 10, 2000, nil) period := Period(0) ok, record, selectionSeed, _ := selParams(addresses[0]) @@ -159,7 +159,7 @@ func TestPoorAccountSelectedLeaders(t *testing.T) { failsLeaders := 0 leaders := make([]uint64, N) for i := 0; i < N; i++ { - selParams, _, round, addresses, _, vrfSecrets, _, _ := testingenv(t, 100, 2000, seedGen) + selParams, _, round, addresses, _, vrfSecrets := testingenv(t, 100, 2000, seedGen) period := Period(0) for j := range addresses { ok, record, selectionSeed, _ := selParams(addresses[j]) @@ -207,7 +207,7 @@ func TestPoorAccountSelectedCommittee(t *testing.T) { N := 1 committee := uint64(0) for i := 0; i < N; i++ { - selParams, _, round, addresses, _, vrfSecrets, _, _ := testingenv(t, 100, 2000, seedGen) + selParams, _, round, addresses, _, vrfSecrets := testingenv(t, 100, 2000, seedGen) period := Period(0) step := Cert @@ -250,10 +250,10 @@ func TestNoMoneyAccountNotSelected(t *testing.T) { seedGen := rand.New(rand.NewSource(1)) N := 1 for i := 0; i < N; i++ { - selParams, _, round, addresses, _, _, _, _ := testingenv(t, 10, 2000, seedGen) + selParams, _, round, addresses, _, _ := testingenv(t, 10, 2000, seedGen) lookback := basics.Round(2*proto.SeedRefreshInterval + proto.SeedLookback + 1) gen := rand.New(rand.NewSource(2)) - _, _, zeroVRFSecret, _ := newAccount(t, gen, lookback, 5) + _, _, zeroVRFSecret := newAccount(t, gen, lookback, 5) period := Period(0) ok, record, selectionSeed, _ := selParams(addresses[i]) if !ok { @@ -281,7 +281,7 @@ func TestNoMoneyAccountNotSelected(t *testing.T) { func TestLeadersSelected(t *testing.T) { partitiontest.PartitionTest(t) - selParams, _, round, addresses, _, vrfSecrets, _, _ := testingenv(t, 100, 2000, nil) + selParams, _, round, addresses, _, vrfSecrets := testingenv(t, 100, 2000, nil) period := Period(0) step := Propose @@ -313,7 +313,7 @@ func TestLeadersSelected(t *testing.T) { func TestCommitteeSelected(t *testing.T) { partitiontest.PartitionTest(t) - selParams, _, round, addresses, _, vrfSecrets, _, _ := testingenv(t, 100, 2000, nil) + selParams, _, round, addresses, _, vrfSecrets := testingenv(t, 100, 2000, nil) period := Period(0) step := Soft @@ -345,7 +345,7 @@ func TestCommitteeSelected(t *testing.T) { func TestAccountNotSelected(t *testing.T) { partitiontest.PartitionTest(t) - selParams, _, round, addresses, _, vrfSecrets, _, _ := testingenv(t, 100, 2000, nil) + selParams, _, round, addresses, _, vrfSecrets := testingenv(t, 100, 2000, nil) period := Period(0) leaders := uint64(0) for i := range addresses { @@ -375,7 +375,7 @@ func TestAccountNotSelected(t *testing.T) { // TODO update to remove VRF verification overhead func BenchmarkSortition(b *testing.B) { - selParams, _, round, addresses, _, vrfSecrets, _, _ := testingenv(b, 100, 2000, nil) + selParams, _, round, addresses, _, vrfSecrets := testingenv(b, 100, 2000, nil) period := Period(0) step := Soft diff --git a/ledger/eval/eval_test.go b/ledger/eval/eval_test.go index 620770326b..d2cf2e2e6e 100644 --- a/ledger/eval/eval_test.go +++ b/ledger/eval/eval_test.go @@ -1676,24 +1676,24 @@ func TestActiveChallenge(t *testing.T) { rules := config.Consensus[nowHeader.CurrentProtocol].Payouts // simplest test. when interval=X and grace=G, X+G+1 is a challenge - inChallenge := rules.ChallengeInterval + rules.ChallengeGracePeriod + 1 + inChallenge := basics.Round(rules.ChallengeInterval + rules.ChallengeGracePeriod + 1) ch := ActiveChallenge(rules, inChallenge, singleSource(nowHeader)) a.NotZero(ch.round) // all rounds before that have no challenge - for r := uint64(1); r < inChallenge; r++ { + for r := basics.Round(1); r < inChallenge; r++ { ch := ActiveChallenge(rules, r, singleSource(nowHeader)) a.Zero(ch.round, r) } // ChallengeGracePeriod rounds allow challenges starting with inChallenge - for r := inChallenge; r < inChallenge+rules.ChallengeGracePeriod; r++ { + for r := inChallenge; r < inChallenge+basics.Round(rules.ChallengeGracePeriod); r++ { ch := ActiveChallenge(rules, r, singleSource(nowHeader)) a.EqualValues(ch.round, rules.ChallengeInterval) } // And the next round is again challenge-less - ch = ActiveChallenge(rules, inChallenge+rules.ChallengeGracePeriod, singleSource(nowHeader)) + ch = ActiveChallenge(rules, inChallenge+basics.Round(rules.ChallengeGracePeriod), singleSource(nowHeader)) a.Zero(ch.round) // ignore challenge if upgrade happened From c2c7262ad6cdadca49fd57645ad7d714fd8605b6 Mon Sep 17 00:00:00 2001 From: John Jannotti Date: Tue, 15 Oct 2024 13:44:06 -0400 Subject: [PATCH 04/19] linting --- ledger/apply/heartbeat.go | 5 ++++- ledger/apply/mockBalances_test.go | 2 +- ledger/eval/eval.go | 5 +++++ 3 files changed, 10 insertions(+), 2 deletions(-) diff --git a/ledger/apply/heartbeat.go b/ledger/apply/heartbeat.go index 1c8a55f4b9..bce788cf68 100644 --- a/ledger/apply/heartbeat.go +++ b/ledger/apply/heartbeat.go @@ -28,10 +28,13 @@ import ( func Heartbeat(hb transactions.HeartbeatTxnFields, header transactions.Header, balances Balances, provider HdrProvider, round basics.Round) error { // Get the account's balance entry account, err := balances.Get(hb.HeartbeatAddress, false) + if err != nil { + return err + } sv := account.VoteID if sv.IsEmpty() { - return fmt.Errorf("HeartbeatAddress %s has has no voting keys\n", hb.HeartbeatAddress) + return fmt.Errorf("heartbeat address %s has no voting keys\n", hb.HeartbeatAddress) } id := basics.OneTimeIDForRound(header.LastValid, account.VoteKeyDilution) diff --git a/ledger/apply/mockBalances_test.go b/ledger/apply/mockBalances_test.go index dd77b66e7f..91f918cf80 100644 --- a/ledger/apply/mockBalances_test.go +++ b/ledger/apply/mockBalances_test.go @@ -291,5 +291,5 @@ func (m mockHeaders) BlockHdr(r basics.Round) (bookkeeping.BlockHeader, error) { if hdr, ok := m.b[r]; ok { return hdr, nil } - return bookkeeping.BlockHeader{}, fmt.Errorf("Round %v is not present\n", r) + return bookkeeping.BlockHeader{}, fmt.Errorf("round %v is not present\n", r) } diff --git a/ledger/eval/eval.go b/ledger/eval/eval.go index 419a960692..0014bc0044 100644 --- a/ledger/eval/eval.go +++ b/ledger/eval/eval.go @@ -1712,6 +1712,9 @@ type headerSource interface { BlockHdr(round basics.Round) (bookkeeping.BlockHeader, error) } +// ActiveChallenge returns details about the Challenge that was last issued if +// it is still in effect (that is, between one and two grace periods from +// issue). Otherwise it returns the zero value. func ActiveChallenge(rules config.ProposerPayoutRules, current basics.Round, headers headerSource) challenge { // are challenges active? interval := basics.Round(rules.ChallengeInterval) @@ -1736,6 +1739,8 @@ func ActiveChallenge(rules config.ProposerPayoutRules, current basics.Round, hea return challenge{lastChallenge, challengeHdr.Seed, rules.ChallengeBits} } +// FailsChallenge returns true iff ch is in effect, matches address, and +// lastSeen is before the challenge issue. func FailsChallenge(ch challenge, address basics.Address, lastSeen basics.Round) bool { return ch.round != 0 && bitsMatch(ch.seed[:], address[:], ch.bits) && lastSeen < ch.round } From ada82224e027e9a8ad09e8ec9e034510b4fdd2c8 Mon Sep 17 00:00:00 2001 From: John Jannotti Date: Mon, 21 Oct 2024 13:02:54 -0400 Subject: [PATCH 05/19] Add in the actual heartbeat crypto, simple test of the service --- config/consensus.go | 9 +- data/bookkeeping/block_test.go | 24 +- data/committee/common_test.go | 8 +- data/committee/credential_test.go | 3 +- data/transactions/heartbeat.go | 10 +- data/transactions/msgp_gen.go | 440 ++++++++++++++++++++------ data/transactions/msgp_gen_test.go | 60 ++++ data/transactions/transaction.go | 5 + data/transactions/transaction_test.go | 15 + heartbeat/service.go | 35 +- heartbeat/service_test.go | 122 +++---- ledger/apply/heartbeat.go | 11 +- ledger/apply/heartbeat_test.go | 6 +- ledger/apply/mockBalances_test.go | 2 +- ledger/apptxn_test.go | 6 +- ledger/eval/eval.go | 51 ++- ledger/eval/eval_test.go | 20 +- ledger/eval_simple_test.go | 6 +- 18 files changed, 604 insertions(+), 229 deletions(-) diff --git a/config/consensus.go b/config/consensus.go index f86e45e831..58678571ab 100644 --- a/config/consensus.go +++ b/config/consensus.go @@ -540,6 +540,9 @@ type ConsensusParams struct { // occur, extra funds need to be put into the FeeSink. The bonus amount // decays exponentially. Bonus BonusPlan + + // Heartbeat support + Heartbeat bool } // ProposerPayoutRules puts several related consensus parameters in one place. The same @@ -1513,7 +1516,7 @@ func initConsensusProtocols() { vFuture.LogicSigVersion = 11 // When moving this to a release, put a new higher LogicSigVersion here vFuture.Payouts.Enabled = true - vFuture.Payouts.Percent = 75 + vFuture.Payouts.Percent = 50 vFuture.Payouts.GoOnlineFee = 2_000_000 // 2 algos vFuture.Payouts.MinBalance = 30_000_000_000 // 30,000 algos vFuture.Payouts.MaxBalance = 70_000_000_000_000 // 70M algos @@ -1524,7 +1527,9 @@ func initConsensusProtocols() { vFuture.Bonus.BaseAmount = 10_000_000 // 10 Algos // 2.9 sec rounds gives about 10.8M rounds per year. - vFuture.Bonus.DecayInterval = 250_000 // .99^(10.8/0.25) ~ .648. So 35% decay per year + vFuture.Bonus.DecayInterval = 1_000_000 // .99^(10.8M/1M) ~ .897. So ~10% decay per year + + vFuture.Heartbeat = true Consensus[protocol.ConsensusFuture] = vFuture diff --git a/data/bookkeeping/block_test.go b/data/bookkeeping/block_test.go index 3c305b3c3b..bc8aec6a7a 100644 --- a/data/bookkeeping/block_test.go +++ b/data/bookkeeping/block_test.go @@ -1013,11 +1013,11 @@ func TestFirstYearsBonus(t *testing.T) { fmt.Printf("paid %d algos\n", suma) fmt.Printf("bonus start: %d end: %d\n", plan.BaseAmount, bonus) - // pays about 88M algos - a.InDelta(88_500_000, suma, 100_000) + // pays about 103.5M algos + a.InDelta(103_500_000, suma, 100_000) - // decline about 35% - a.InDelta(0.65, float64(bonus)/float64(plan.BaseAmount), 0.01) + // decline about 10% + a.InDelta(0.90, float64(bonus)/float64(plan.BaseAmount), 0.01) // year 2 for i := 0; i < yearRounds; i++ { @@ -1033,11 +1033,11 @@ func TestFirstYearsBonus(t *testing.T) { fmt.Printf("paid %d algos after 2 years\n", sum2) fmt.Printf("bonus end: %d\n", bonus) - // pays about 146M algos (total for 2 years) - a.InDelta(145_700_000, sum2, 100_000) + // pays about 196M algos (total for 2 years) + a.InDelta(196_300_000, sum2, 100_000) - // decline about 58% - a.InDelta(0.42, float64(bonus)/float64(plan.BaseAmount), 0.01) + // decline to about 81% + a.InDelta(0.81, float64(bonus)/float64(plan.BaseAmount), 0.01) // year 3 for i := 0; i < yearRounds; i++ { @@ -1053,9 +1053,9 @@ func TestFirstYearsBonus(t *testing.T) { fmt.Printf("paid %d algos after 3 years\n", sum3) fmt.Printf("bonus end: %d\n", bonus) - // pays about 182M algos (total for 3 years) - a.InDelta(182_600_000, sum3, 100_000) + // pays about 279M algos (total for 3 years) + a.InDelta(279_500_000, sum3, 100_000) - // declined to about 27% (but foundation funding probably gone anyway) - a.InDelta(0.27, float64(bonus)/float64(plan.BaseAmount), 0.01) + // declined to about 72% (but foundation funding probably gone anyway) + a.InDelta(0.72, float64(bonus)/float64(plan.BaseAmount), 0.01) } diff --git a/data/committee/common_test.go b/data/committee/common_test.go index eef035b8e2..8566a9cd2a 100644 --- a/data/committee/common_test.go +++ b/data/committee/common_test.go @@ -32,7 +32,7 @@ type selectionParameterListFn func(addr []basics.Address) (bool, []BalanceRecord var proto = config.Consensus[protocol.ConsensusCurrentVersion] -func newAccount(t testing.TB, gen io.Reader, latest basics.Round, keyBatchesForward uint) (basics.Address, *crypto.SignatureSecrets, *crypto.VrfPrivkey) { +func newAccount(t testing.TB, gen io.Reader) (basics.Address, *crypto.SignatureSecrets, *crypto.VrfPrivkey) { var seed crypto.Seed gen.Read(seed[:]) s := crypto.GenerateSignatureSecrets(seed) @@ -49,10 +49,10 @@ func newAccount(t testing.TB, gen io.Reader, latest basics.Round, keyBatchesForw // formerly, testingenv, generated transactions and one-time secrets as well, // but they were not used by the tests. func testingenv(t testing.TB, numAccounts, numTxs int, seedGen io.Reader) (selectionParameterFn, selectionParameterListFn, basics.Round, []basics.Address, []*crypto.SignatureSecrets, []*crypto.VrfPrivkey) { - return testingenvMoreKeys(t, numAccounts, numTxs, uint(5), seedGen) + return testingenvMoreKeys(t, numAccounts, numTxs, seedGen) } -func testingenvMoreKeys(t testing.TB, numAccounts, numTxs int, keyBatchesForward uint, seedGen io.Reader) (selectionParameterFn, selectionParameterListFn, basics.Round, []basics.Address, []*crypto.SignatureSecrets, []*crypto.VrfPrivkey) { +func testingenvMoreKeys(t testing.TB, numAccounts, numTxs int, seedGen io.Reader) (selectionParameterFn, selectionParameterListFn, basics.Round, []basics.Address, []*crypto.SignatureSecrets, []*crypto.VrfPrivkey) { if seedGen == nil { seedGen = rand.New(rand.NewSource(1)) // same source as setting GODEBUG=randautoseed=0, same as pre-Go 1.20 default seed } @@ -70,7 +70,7 @@ func testingenvMoreKeys(t testing.TB, numAccounts, numTxs int, keyBatchesForward lookback := basics.Round(2*proto.SeedRefreshInterval + proto.SeedLookback + 1) var total basics.MicroAlgos for i := 0; i < P; i++ { - addr, sigSec, vrfSec := newAccount(t, gen, lookback, keyBatchesForward) + addr, sigSec, vrfSec := newAccount(t, gen) addrs[i] = addr secrets[i] = sigSec vrfSecrets[i] = vrfSec diff --git a/data/committee/credential_test.go b/data/committee/credential_test.go index b646efdf0a..bbabac62e9 100644 --- a/data/committee/credential_test.go +++ b/data/committee/credential_test.go @@ -251,9 +251,8 @@ func TestNoMoneyAccountNotSelected(t *testing.T) { N := 1 for i := 0; i < N; i++ { selParams, _, round, addresses, _, _ := testingenv(t, 10, 2000, seedGen) - lookback := basics.Round(2*proto.SeedRefreshInterval + proto.SeedLookback + 1) gen := rand.New(rand.NewSource(2)) - _, _, zeroVRFSecret := newAccount(t, gen, lookback, 5) + _, _, zeroVRFSecret := newAccount(t, gen) period := Period(0) ok, record, selectionSeed, _ := selParams(addresses[i]) if !ok { diff --git a/data/transactions/heartbeat.go b/data/transactions/heartbeat.go index 9acf847734..873a905079 100644 --- a/data/transactions/heartbeat.go +++ b/data/transactions/heartbeat.go @@ -29,14 +29,14 @@ type HeartbeatTxnFields struct { _struct struct{} `codec:",omitempty,omitemptyarray"` // HeartbeatAddress is the account this txn is proving onlineness for. - HeartbeatAddress basics.Address `codec:"hbad"` + HbAddress basics.Address `codec:"hbad"` - // Proof is a signature using HeartbeatAddress's partkey, thereby showing it is online. - Proof crypto.OneTimeSignature `codec:"hbprf"` + // HbProof is a signature using HeartbeatAddress's partkey, thereby showing it is online. + HbProof crypto.OneTimeSignature `codec:"hbprf"` - // Seed must be the block seed for the block before this transaction's + // HbSeed must be the block seed for the block before this transaction's // firstValid. It is supplied in the transaction so that Proof can be // checked at submit time without a ledger lookup, and must be checked at // evaluation time for equality with the actual blockseed. - Seed committee.Seed `codec:"hbsd"` + HbSeed committee.Seed `codec:"hbsd"` } diff --git a/data/transactions/msgp_gen.go b/data/transactions/msgp_gen.go index 7cc22db08a..f7ea0e8fbb 100644 --- a/data/transactions/msgp_gen.go +++ b/data/transactions/msgp_gen.go @@ -12,6 +12,7 @@ import ( "github.com/algorand/go-algorand/crypto/merklesignature" "github.com/algorand/go-algorand/crypto/stateproof" "github.com/algorand/go-algorand/data/basics" + "github.com/algorand/go-algorand/data/committee" "github.com/algorand/go-algorand/data/stateproofmsg" "github.com/algorand/go-algorand/protocol" ) @@ -97,6 +98,16 @@ import ( // |-----> (*) MsgIsZero // |-----> HeaderMaxSize() // +// HeartbeatTxnFields +// |-----> (*) MarshalMsg +// |-----> (*) CanMarshalMsg +// |-----> (*) UnmarshalMsg +// |-----> (*) UnmarshalMsgWithState +// |-----> (*) CanUnmarshalMsg +// |-----> (*) Msgsize +// |-----> (*) MsgIsZero +// |-----> HeartbeatTxnFieldsMaxSize() +// // KeyregTxnFields // |-----> (*) MarshalMsg // |-----> (*) CanMarshalMsg @@ -2907,6 +2918,172 @@ func HeaderMaxSize() (s int) { return } +// MarshalMsg implements msgp.Marshaler +func (z *HeartbeatTxnFields) MarshalMsg(b []byte) (o []byte) { + o = msgp.Require(b, z.Msgsize()) + // omitempty: check for empty values + zb0001Len := uint32(3) + var zb0001Mask uint8 /* 4 bits */ + if (*z).HbAddress.MsgIsZero() { + zb0001Len-- + zb0001Mask |= 0x2 + } + if (*z).HbProof.MsgIsZero() { + zb0001Len-- + zb0001Mask |= 0x4 + } + if (*z).HbSeed.MsgIsZero() { + zb0001Len-- + zb0001Mask |= 0x8 + } + // variable map header, size zb0001Len + o = append(o, 0x80|uint8(zb0001Len)) + if zb0001Len != 0 { + if (zb0001Mask & 0x2) == 0 { // if not empty + // string "hbad" + o = append(o, 0xa4, 0x68, 0x62, 0x61, 0x64) + o = (*z).HbAddress.MarshalMsg(o) + } + if (zb0001Mask & 0x4) == 0 { // if not empty + // string "hbprf" + o = append(o, 0xa5, 0x68, 0x62, 0x70, 0x72, 0x66) + o = (*z).HbProof.MarshalMsg(o) + } + if (zb0001Mask & 0x8) == 0 { // if not empty + // string "hbsd" + o = append(o, 0xa4, 0x68, 0x62, 0x73, 0x64) + o = (*z).HbSeed.MarshalMsg(o) + } + } + return +} + +func (_ *HeartbeatTxnFields) CanMarshalMsg(z interface{}) bool { + _, ok := (z).(*HeartbeatTxnFields) + return ok +} + +// UnmarshalMsg implements msgp.Unmarshaler +func (z *HeartbeatTxnFields) UnmarshalMsgWithState(bts []byte, st msgp.UnmarshalState) (o []byte, err error) { + if st.AllowableDepth == 0 { + err = msgp.ErrMaxDepthExceeded{} + return + } + st.AllowableDepth-- + var field []byte + _ = field + var zb0001 int + var zb0002 bool + zb0001, zb0002, bts, err = msgp.ReadMapHeaderBytes(bts) + if _, ok := err.(msgp.TypeError); ok { + zb0001, zb0002, bts, err = msgp.ReadArrayHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + if zb0001 > 0 { + zb0001-- + bts, err = (*z).HbAddress.UnmarshalMsgWithState(bts, st) + if err != nil { + err = msgp.WrapError(err, "struct-from-array", "HbAddress") + return + } + } + if zb0001 > 0 { + zb0001-- + bts, err = (*z).HbProof.UnmarshalMsgWithState(bts, st) + if err != nil { + err = msgp.WrapError(err, "struct-from-array", "HbProof") + return + } + } + if zb0001 > 0 { + zb0001-- + bts, err = (*z).HbSeed.UnmarshalMsgWithState(bts, st) + if err != nil { + err = msgp.WrapError(err, "struct-from-array", "HbSeed") + return + } + } + if zb0001 > 0 { + err = msgp.ErrTooManyArrayFields(zb0001) + if err != nil { + err = msgp.WrapError(err, "struct-from-array") + return + } + } + } else { + if err != nil { + err = msgp.WrapError(err) + return + } + if zb0002 { + (*z) = HeartbeatTxnFields{} + } + for zb0001 > 0 { + zb0001-- + field, bts, err = msgp.ReadMapKeyZC(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + switch string(field) { + case "hbad": + bts, err = (*z).HbAddress.UnmarshalMsgWithState(bts, st) + if err != nil { + err = msgp.WrapError(err, "HbAddress") + return + } + case "hbprf": + bts, err = (*z).HbProof.UnmarshalMsgWithState(bts, st) + if err != nil { + err = msgp.WrapError(err, "HbProof") + return + } + case "hbsd": + bts, err = (*z).HbSeed.UnmarshalMsgWithState(bts, st) + if err != nil { + err = msgp.WrapError(err, "HbSeed") + return + } + default: + err = msgp.ErrNoField(string(field)) + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + } + o = bts + return +} + +func (z *HeartbeatTxnFields) UnmarshalMsg(bts []byte) (o []byte, err error) { + return z.UnmarshalMsgWithState(bts, msgp.DefaultUnmarshalState) +} +func (_ *HeartbeatTxnFields) CanUnmarshalMsg(z interface{}) bool { + _, ok := (z).(*HeartbeatTxnFields) + return ok +} + +// Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message +func (z *HeartbeatTxnFields) Msgsize() (s int) { + s = 1 + 5 + (*z).HbAddress.Msgsize() + 6 + (*z).HbProof.Msgsize() + 5 + (*z).HbSeed.Msgsize() + return +} + +// MsgIsZero returns whether this is a zero value +func (z *HeartbeatTxnFields) MsgIsZero() bool { + return ((*z).HbAddress.MsgIsZero()) && ((*z).HbProof.MsgIsZero()) && ((*z).HbSeed.MsgIsZero()) +} + +// MaxSize returns a maximum valid message size for this message type +func HeartbeatTxnFieldsMaxSize() (s int) { + s = 1 + 5 + basics.AddressMaxSize() + 6 + crypto.OneTimeSignatureMaxSize() + 5 + committee.SeedMaxSize() + return +} + // MarshalMsg implements msgp.Marshaler func (z *KeyregTxnFields) MarshalMsg(b []byte) (o []byte) { o = msgp.Require(b, z.Msgsize()) @@ -4982,216 +5159,228 @@ func StateProofTxnFieldsMaxSize() (s int) { func (z *Transaction) MarshalMsg(b []byte) (o []byte) { o = msgp.Require(b, z.Msgsize()) // omitempty: check for empty values - zb0007Len := uint32(46) - var zb0007Mask uint64 /* 55 bits */ + zb0007Len := uint32(49) + var zb0007Mask uint64 /* 59 bits */ if (*z).AssetTransferTxnFields.AssetAmount == 0 { zb0007Len-- - zb0007Mask |= 0x200 + zb0007Mask |= 0x400 } if (*z).AssetTransferTxnFields.AssetCloseTo.MsgIsZero() { zb0007Len-- - zb0007Mask |= 0x400 + zb0007Mask |= 0x800 } if (*z).AssetFreezeTxnFields.AssetFrozen == false { zb0007Len-- - zb0007Mask |= 0x800 + zb0007Mask |= 0x1000 } if (*z).PaymentTxnFields.Amount.MsgIsZero() { zb0007Len-- - zb0007Mask |= 0x1000 + zb0007Mask |= 0x2000 } if len((*z).ApplicationCallTxnFields.ApplicationArgs) == 0 { zb0007Len-- - zb0007Mask |= 0x2000 + zb0007Mask |= 0x4000 } if (*z).ApplicationCallTxnFields.OnCompletion == 0 { zb0007Len-- - zb0007Mask |= 0x4000 + zb0007Mask |= 0x8000 } if len((*z).ApplicationCallTxnFields.ApprovalProgram) == 0 { zb0007Len-- - zb0007Mask |= 0x8000 + zb0007Mask |= 0x10000 } if (*z).AssetConfigTxnFields.AssetParams.MsgIsZero() { zb0007Len-- - zb0007Mask |= 0x10000 + zb0007Mask |= 0x20000 } if len((*z).ApplicationCallTxnFields.ForeignAssets) == 0 { zb0007Len-- - zb0007Mask |= 0x20000 + zb0007Mask |= 0x40000 } if len((*z).ApplicationCallTxnFields.Accounts) == 0 { zb0007Len-- - zb0007Mask |= 0x40000 + zb0007Mask |= 0x80000 } if len((*z).ApplicationCallTxnFields.Boxes) == 0 { zb0007Len-- - zb0007Mask |= 0x80000 + zb0007Mask |= 0x100000 } if (*z).ApplicationCallTxnFields.ExtraProgramPages == 0 { zb0007Len-- - zb0007Mask |= 0x100000 + zb0007Mask |= 0x200000 } if len((*z).ApplicationCallTxnFields.ForeignApps) == 0 { zb0007Len-- - zb0007Mask |= 0x200000 + zb0007Mask |= 0x400000 } if (*z).ApplicationCallTxnFields.GlobalStateSchema.MsgIsZero() { zb0007Len-- - zb0007Mask |= 0x400000 + zb0007Mask |= 0x800000 } if (*z).ApplicationCallTxnFields.ApplicationID.MsgIsZero() { zb0007Len-- - zb0007Mask |= 0x800000 + zb0007Mask |= 0x1000000 } if (*z).ApplicationCallTxnFields.LocalStateSchema.MsgIsZero() { zb0007Len-- - zb0007Mask |= 0x1000000 + zb0007Mask |= 0x2000000 } if len((*z).ApplicationCallTxnFields.ClearStateProgram) == 0 { zb0007Len-- - zb0007Mask |= 0x2000000 + zb0007Mask |= 0x4000000 } if (*z).AssetTransferTxnFields.AssetReceiver.MsgIsZero() { zb0007Len-- - zb0007Mask |= 0x4000000 + zb0007Mask |= 0x8000000 } if (*z).AssetTransferTxnFields.AssetSender.MsgIsZero() { zb0007Len-- - zb0007Mask |= 0x8000000 + zb0007Mask |= 0x10000000 } if (*z).AssetConfigTxnFields.ConfigAsset.MsgIsZero() { zb0007Len-- - zb0007Mask |= 0x10000000 + zb0007Mask |= 0x20000000 } if (*z).PaymentTxnFields.CloseRemainderTo.MsgIsZero() { zb0007Len-- - zb0007Mask |= 0x20000000 + zb0007Mask |= 0x40000000 } if (*z).AssetFreezeTxnFields.FreezeAccount.MsgIsZero() { zb0007Len-- - zb0007Mask |= 0x40000000 + zb0007Mask |= 0x80000000 } if (*z).AssetFreezeTxnFields.FreezeAsset.MsgIsZero() { zb0007Len-- - zb0007Mask |= 0x80000000 + zb0007Mask |= 0x100000000 } if (*z).Header.Fee.MsgIsZero() { zb0007Len-- - zb0007Mask |= 0x100000000 + zb0007Mask |= 0x200000000 } if (*z).Header.FirstValid.MsgIsZero() { zb0007Len-- - zb0007Mask |= 0x200000000 + zb0007Mask |= 0x400000000 } if (*z).Header.GenesisID == "" { zb0007Len-- - zb0007Mask |= 0x400000000 + zb0007Mask |= 0x800000000 } if (*z).Header.GenesisHash.MsgIsZero() { zb0007Len-- - zb0007Mask |= 0x800000000 + zb0007Mask |= 0x1000000000 } if (*z).Header.Group.MsgIsZero() { zb0007Len-- - zb0007Mask |= 0x1000000000 + zb0007Mask |= 0x2000000000 + } + if (*z).HeartbeatTxnFields.HbAddress.MsgIsZero() { + zb0007Len-- + zb0007Mask |= 0x4000000000 + } + if (*z).HeartbeatTxnFields.HbProof.MsgIsZero() { + zb0007Len-- + zb0007Mask |= 0x8000000000 + } + if (*z).HeartbeatTxnFields.HbSeed.MsgIsZero() { + zb0007Len-- + zb0007Mask |= 0x10000000000 } if (*z).Header.LastValid.MsgIsZero() { zb0007Len-- - zb0007Mask |= 0x2000000000 + zb0007Mask |= 0x20000000000 } if (*z).Header.Lease == ([32]byte{}) { zb0007Len-- - zb0007Mask |= 0x4000000000 + zb0007Mask |= 0x40000000000 } if (*z).KeyregTxnFields.Nonparticipation == false { zb0007Len-- - zb0007Mask |= 0x8000000000 + zb0007Mask |= 0x80000000000 } if len((*z).Header.Note) == 0 { zb0007Len-- - zb0007Mask |= 0x10000000000 + zb0007Mask |= 0x100000000000 } if (*z).PaymentTxnFields.Receiver.MsgIsZero() { zb0007Len-- - zb0007Mask |= 0x20000000000 + zb0007Mask |= 0x200000000000 } if (*z).Header.RekeyTo.MsgIsZero() { zb0007Len-- - zb0007Mask |= 0x40000000000 + zb0007Mask |= 0x400000000000 } if (*z).KeyregTxnFields.SelectionPK.MsgIsZero() { zb0007Len-- - zb0007Mask |= 0x80000000000 + zb0007Mask |= 0x800000000000 } if (*z).Header.Sender.MsgIsZero() { zb0007Len-- - zb0007Mask |= 0x100000000000 + zb0007Mask |= 0x1000000000000 } if (*z).StateProofTxnFields.StateProof.MsgIsZero() { zb0007Len-- - zb0007Mask |= 0x200000000000 + zb0007Mask |= 0x2000000000000 } if (*z).StateProofTxnFields.Message.MsgIsZero() { zb0007Len-- - zb0007Mask |= 0x400000000000 + zb0007Mask |= 0x4000000000000 } if (*z).KeyregTxnFields.StateProofPK.MsgIsZero() { zb0007Len-- - zb0007Mask |= 0x800000000000 + zb0007Mask |= 0x8000000000000 } if (*z).StateProofTxnFields.StateProofType.MsgIsZero() { zb0007Len-- - zb0007Mask |= 0x1000000000000 + zb0007Mask |= 0x10000000000000 } if (*z).Type.MsgIsZero() { zb0007Len-- - zb0007Mask |= 0x2000000000000 + zb0007Mask |= 0x20000000000000 } if (*z).KeyregTxnFields.VoteFirst.MsgIsZero() { zb0007Len-- - zb0007Mask |= 0x4000000000000 + zb0007Mask |= 0x40000000000000 } if (*z).KeyregTxnFields.VoteKeyDilution == 0 { zb0007Len-- - zb0007Mask |= 0x8000000000000 + zb0007Mask |= 0x80000000000000 } if (*z).KeyregTxnFields.VotePK.MsgIsZero() { zb0007Len-- - zb0007Mask |= 0x10000000000000 + zb0007Mask |= 0x100000000000000 } if (*z).KeyregTxnFields.VoteLast.MsgIsZero() { zb0007Len-- - zb0007Mask |= 0x20000000000000 + zb0007Mask |= 0x200000000000000 } if (*z).AssetTransferTxnFields.XferAsset.MsgIsZero() { zb0007Len-- - zb0007Mask |= 0x40000000000000 + zb0007Mask |= 0x400000000000000 } // variable map header, size zb0007Len o = msgp.AppendMapHeader(o, zb0007Len) if zb0007Len != 0 { - if (zb0007Mask & 0x200) == 0 { // if not empty + if (zb0007Mask & 0x400) == 0 { // if not empty // string "aamt" o = append(o, 0xa4, 0x61, 0x61, 0x6d, 0x74) o = msgp.AppendUint64(o, (*z).AssetTransferTxnFields.AssetAmount) } - if (zb0007Mask & 0x400) == 0 { // if not empty + if (zb0007Mask & 0x800) == 0 { // if not empty // string "aclose" o = append(o, 0xa6, 0x61, 0x63, 0x6c, 0x6f, 0x73, 0x65) o = (*z).AssetTransferTxnFields.AssetCloseTo.MarshalMsg(o) } - if (zb0007Mask & 0x800) == 0 { // if not empty + if (zb0007Mask & 0x1000) == 0 { // if not empty // string "afrz" o = append(o, 0xa4, 0x61, 0x66, 0x72, 0x7a) o = msgp.AppendBool(o, (*z).AssetFreezeTxnFields.AssetFrozen) } - if (zb0007Mask & 0x1000) == 0 { // if not empty + if (zb0007Mask & 0x2000) == 0 { // if not empty // string "amt" o = append(o, 0xa3, 0x61, 0x6d, 0x74) o = (*z).PaymentTxnFields.Amount.MarshalMsg(o) } - if (zb0007Mask & 0x2000) == 0 { // if not empty + if (zb0007Mask & 0x4000) == 0 { // if not empty // string "apaa" o = append(o, 0xa4, 0x61, 0x70, 0x61, 0x61) if (*z).ApplicationCallTxnFields.ApplicationArgs == nil { @@ -5203,22 +5392,22 @@ func (z *Transaction) MarshalMsg(b []byte) (o []byte) { o = msgp.AppendBytes(o, (*z).ApplicationCallTxnFields.ApplicationArgs[zb0002]) } } - if (zb0007Mask & 0x4000) == 0 { // if not empty + if (zb0007Mask & 0x8000) == 0 { // if not empty // string "apan" o = append(o, 0xa4, 0x61, 0x70, 0x61, 0x6e) o = msgp.AppendUint64(o, uint64((*z).ApplicationCallTxnFields.OnCompletion)) } - if (zb0007Mask & 0x8000) == 0 { // if not empty + if (zb0007Mask & 0x10000) == 0 { // if not empty // string "apap" o = append(o, 0xa4, 0x61, 0x70, 0x61, 0x70) o = msgp.AppendBytes(o, (*z).ApplicationCallTxnFields.ApprovalProgram) } - if (zb0007Mask & 0x10000) == 0 { // if not empty + if (zb0007Mask & 0x20000) == 0 { // if not empty // string "apar" o = append(o, 0xa4, 0x61, 0x70, 0x61, 0x72) o = (*z).AssetConfigTxnFields.AssetParams.MarshalMsg(o) } - if (zb0007Mask & 0x20000) == 0 { // if not empty + if (zb0007Mask & 0x40000) == 0 { // if not empty // string "apas" o = append(o, 0xa4, 0x61, 0x70, 0x61, 0x73) if (*z).ApplicationCallTxnFields.ForeignAssets == nil { @@ -5230,7 +5419,7 @@ func (z *Transaction) MarshalMsg(b []byte) (o []byte) { o = (*z).ApplicationCallTxnFields.ForeignAssets[zb0006].MarshalMsg(o) } } - if (zb0007Mask & 0x40000) == 0 { // if not empty + if (zb0007Mask & 0x80000) == 0 { // if not empty // string "apat" o = append(o, 0xa4, 0x61, 0x70, 0x61, 0x74) if (*z).ApplicationCallTxnFields.Accounts == nil { @@ -5242,7 +5431,7 @@ func (z *Transaction) MarshalMsg(b []byte) (o []byte) { o = (*z).ApplicationCallTxnFields.Accounts[zb0003].MarshalMsg(o) } } - if (zb0007Mask & 0x80000) == 0 { // if not empty + if (zb0007Mask & 0x100000) == 0 { // if not empty // string "apbx" o = append(o, 0xa4, 0x61, 0x70, 0x62, 0x78) if (*z).ApplicationCallTxnFields.Boxes == nil { @@ -5276,12 +5465,12 @@ func (z *Transaction) MarshalMsg(b []byte) (o []byte) { } } } - if (zb0007Mask & 0x100000) == 0 { // if not empty + if (zb0007Mask & 0x200000) == 0 { // if not empty // string "apep" o = append(o, 0xa4, 0x61, 0x70, 0x65, 0x70) o = msgp.AppendUint32(o, (*z).ApplicationCallTxnFields.ExtraProgramPages) } - if (zb0007Mask & 0x200000) == 0 { // if not empty + if (zb0007Mask & 0x400000) == 0 { // if not empty // string "apfa" o = append(o, 0xa4, 0x61, 0x70, 0x66, 0x61) if (*z).ApplicationCallTxnFields.ForeignApps == nil { @@ -5293,167 +5482,182 @@ func (z *Transaction) MarshalMsg(b []byte) (o []byte) { o = (*z).ApplicationCallTxnFields.ForeignApps[zb0004].MarshalMsg(o) } } - if (zb0007Mask & 0x400000) == 0 { // if not empty + if (zb0007Mask & 0x800000) == 0 { // if not empty // string "apgs" o = append(o, 0xa4, 0x61, 0x70, 0x67, 0x73) o = (*z).ApplicationCallTxnFields.GlobalStateSchema.MarshalMsg(o) } - if (zb0007Mask & 0x800000) == 0 { // if not empty + if (zb0007Mask & 0x1000000) == 0 { // if not empty // string "apid" o = append(o, 0xa4, 0x61, 0x70, 0x69, 0x64) o = (*z).ApplicationCallTxnFields.ApplicationID.MarshalMsg(o) } - if (zb0007Mask & 0x1000000) == 0 { // if not empty + if (zb0007Mask & 0x2000000) == 0 { // if not empty // string "apls" o = append(o, 0xa4, 0x61, 0x70, 0x6c, 0x73) o = (*z).ApplicationCallTxnFields.LocalStateSchema.MarshalMsg(o) } - if (zb0007Mask & 0x2000000) == 0 { // if not empty + if (zb0007Mask & 0x4000000) == 0 { // if not empty // string "apsu" o = append(o, 0xa4, 0x61, 0x70, 0x73, 0x75) o = msgp.AppendBytes(o, (*z).ApplicationCallTxnFields.ClearStateProgram) } - if (zb0007Mask & 0x4000000) == 0 { // if not empty + if (zb0007Mask & 0x8000000) == 0 { // if not empty // string "arcv" o = append(o, 0xa4, 0x61, 0x72, 0x63, 0x76) o = (*z).AssetTransferTxnFields.AssetReceiver.MarshalMsg(o) } - if (zb0007Mask & 0x8000000) == 0 { // if not empty + if (zb0007Mask & 0x10000000) == 0 { // if not empty // string "asnd" o = append(o, 0xa4, 0x61, 0x73, 0x6e, 0x64) o = (*z).AssetTransferTxnFields.AssetSender.MarshalMsg(o) } - if (zb0007Mask & 0x10000000) == 0 { // if not empty + if (zb0007Mask & 0x20000000) == 0 { // if not empty // string "caid" o = append(o, 0xa4, 0x63, 0x61, 0x69, 0x64) o = (*z).AssetConfigTxnFields.ConfigAsset.MarshalMsg(o) } - if (zb0007Mask & 0x20000000) == 0 { // if not empty + if (zb0007Mask & 0x40000000) == 0 { // if not empty // string "close" o = append(o, 0xa5, 0x63, 0x6c, 0x6f, 0x73, 0x65) o = (*z).PaymentTxnFields.CloseRemainderTo.MarshalMsg(o) } - if (zb0007Mask & 0x40000000) == 0 { // if not empty + if (zb0007Mask & 0x80000000) == 0 { // if not empty // string "fadd" o = append(o, 0xa4, 0x66, 0x61, 0x64, 0x64) o = (*z).AssetFreezeTxnFields.FreezeAccount.MarshalMsg(o) } - if (zb0007Mask & 0x80000000) == 0 { // if not empty + if (zb0007Mask & 0x100000000) == 0 { // if not empty // string "faid" o = append(o, 0xa4, 0x66, 0x61, 0x69, 0x64) o = (*z).AssetFreezeTxnFields.FreezeAsset.MarshalMsg(o) } - if (zb0007Mask & 0x100000000) == 0 { // if not empty + if (zb0007Mask & 0x200000000) == 0 { // if not empty // string "fee" o = append(o, 0xa3, 0x66, 0x65, 0x65) o = (*z).Header.Fee.MarshalMsg(o) } - if (zb0007Mask & 0x200000000) == 0 { // if not empty + if (zb0007Mask & 0x400000000) == 0 { // if not empty // string "fv" o = append(o, 0xa2, 0x66, 0x76) o = (*z).Header.FirstValid.MarshalMsg(o) } - if (zb0007Mask & 0x400000000) == 0 { // if not empty + if (zb0007Mask & 0x800000000) == 0 { // if not empty // string "gen" o = append(o, 0xa3, 0x67, 0x65, 0x6e) o = msgp.AppendString(o, (*z).Header.GenesisID) } - if (zb0007Mask & 0x800000000) == 0 { // if not empty + if (zb0007Mask & 0x1000000000) == 0 { // if not empty // string "gh" o = append(o, 0xa2, 0x67, 0x68) o = (*z).Header.GenesisHash.MarshalMsg(o) } - if (zb0007Mask & 0x1000000000) == 0 { // if not empty + if (zb0007Mask & 0x2000000000) == 0 { // if not empty // string "grp" o = append(o, 0xa3, 0x67, 0x72, 0x70) o = (*z).Header.Group.MarshalMsg(o) } - if (zb0007Mask & 0x2000000000) == 0 { // if not empty + if (zb0007Mask & 0x4000000000) == 0 { // if not empty + // string "hbad" + o = append(o, 0xa4, 0x68, 0x62, 0x61, 0x64) + o = (*z).HeartbeatTxnFields.HbAddress.MarshalMsg(o) + } + if (zb0007Mask & 0x8000000000) == 0 { // if not empty + // string "hbprf" + o = append(o, 0xa5, 0x68, 0x62, 0x70, 0x72, 0x66) + o = (*z).HeartbeatTxnFields.HbProof.MarshalMsg(o) + } + if (zb0007Mask & 0x10000000000) == 0 { // if not empty + // string "hbsd" + o = append(o, 0xa4, 0x68, 0x62, 0x73, 0x64) + o = (*z).HeartbeatTxnFields.HbSeed.MarshalMsg(o) + } + if (zb0007Mask & 0x20000000000) == 0 { // if not empty // string "lv" o = append(o, 0xa2, 0x6c, 0x76) o = (*z).Header.LastValid.MarshalMsg(o) } - if (zb0007Mask & 0x4000000000) == 0 { // if not empty + if (zb0007Mask & 0x40000000000) == 0 { // if not empty // string "lx" o = append(o, 0xa2, 0x6c, 0x78) o = msgp.AppendBytes(o, ((*z).Header.Lease)[:]) } - if (zb0007Mask & 0x8000000000) == 0 { // if not empty + if (zb0007Mask & 0x80000000000) == 0 { // if not empty // string "nonpart" o = append(o, 0xa7, 0x6e, 0x6f, 0x6e, 0x70, 0x61, 0x72, 0x74) o = msgp.AppendBool(o, (*z).KeyregTxnFields.Nonparticipation) } - if (zb0007Mask & 0x10000000000) == 0 { // if not empty + if (zb0007Mask & 0x100000000000) == 0 { // if not empty // string "note" o = append(o, 0xa4, 0x6e, 0x6f, 0x74, 0x65) o = msgp.AppendBytes(o, (*z).Header.Note) } - if (zb0007Mask & 0x20000000000) == 0 { // if not empty + if (zb0007Mask & 0x200000000000) == 0 { // if not empty // string "rcv" o = append(o, 0xa3, 0x72, 0x63, 0x76) o = (*z).PaymentTxnFields.Receiver.MarshalMsg(o) } - if (zb0007Mask & 0x40000000000) == 0 { // if not empty + if (zb0007Mask & 0x400000000000) == 0 { // if not empty // string "rekey" o = append(o, 0xa5, 0x72, 0x65, 0x6b, 0x65, 0x79) o = (*z).Header.RekeyTo.MarshalMsg(o) } - if (zb0007Mask & 0x80000000000) == 0 { // if not empty + if (zb0007Mask & 0x800000000000) == 0 { // if not empty // string "selkey" o = append(o, 0xa6, 0x73, 0x65, 0x6c, 0x6b, 0x65, 0x79) o = (*z).KeyregTxnFields.SelectionPK.MarshalMsg(o) } - if (zb0007Mask & 0x100000000000) == 0 { // if not empty + if (zb0007Mask & 0x1000000000000) == 0 { // if not empty // string "snd" o = append(o, 0xa3, 0x73, 0x6e, 0x64) o = (*z).Header.Sender.MarshalMsg(o) } - if (zb0007Mask & 0x200000000000) == 0 { // if not empty + if (zb0007Mask & 0x2000000000000) == 0 { // if not empty // string "sp" o = append(o, 0xa2, 0x73, 0x70) o = (*z).StateProofTxnFields.StateProof.MarshalMsg(o) } - if (zb0007Mask & 0x400000000000) == 0 { // if not empty + if (zb0007Mask & 0x4000000000000) == 0 { // if not empty // string "spmsg" o = append(o, 0xa5, 0x73, 0x70, 0x6d, 0x73, 0x67) o = (*z).StateProofTxnFields.Message.MarshalMsg(o) } - if (zb0007Mask & 0x800000000000) == 0 { // if not empty + if (zb0007Mask & 0x8000000000000) == 0 { // if not empty // string "sprfkey" o = append(o, 0xa7, 0x73, 0x70, 0x72, 0x66, 0x6b, 0x65, 0x79) o = (*z).KeyregTxnFields.StateProofPK.MarshalMsg(o) } - if (zb0007Mask & 0x1000000000000) == 0 { // if not empty + if (zb0007Mask & 0x10000000000000) == 0 { // if not empty // string "sptype" o = append(o, 0xa6, 0x73, 0x70, 0x74, 0x79, 0x70, 0x65) o = (*z).StateProofTxnFields.StateProofType.MarshalMsg(o) } - if (zb0007Mask & 0x2000000000000) == 0 { // if not empty + if (zb0007Mask & 0x20000000000000) == 0 { // if not empty // string "type" o = append(o, 0xa4, 0x74, 0x79, 0x70, 0x65) o = (*z).Type.MarshalMsg(o) } - if (zb0007Mask & 0x4000000000000) == 0 { // if not empty + if (zb0007Mask & 0x40000000000000) == 0 { // if not empty // string "votefst" o = append(o, 0xa7, 0x76, 0x6f, 0x74, 0x65, 0x66, 0x73, 0x74) o = (*z).KeyregTxnFields.VoteFirst.MarshalMsg(o) } - if (zb0007Mask & 0x8000000000000) == 0 { // if not empty + if (zb0007Mask & 0x80000000000000) == 0 { // if not empty // string "votekd" o = append(o, 0xa6, 0x76, 0x6f, 0x74, 0x65, 0x6b, 0x64) o = msgp.AppendUint64(o, (*z).KeyregTxnFields.VoteKeyDilution) } - if (zb0007Mask & 0x10000000000000) == 0 { // if not empty + if (zb0007Mask & 0x100000000000000) == 0 { // if not empty // string "votekey" o = append(o, 0xa7, 0x76, 0x6f, 0x74, 0x65, 0x6b, 0x65, 0x79) o = (*z).KeyregTxnFields.VotePK.MarshalMsg(o) } - if (zb0007Mask & 0x20000000000000) == 0 { // if not empty + if (zb0007Mask & 0x200000000000000) == 0 { // if not empty // string "votelst" o = append(o, 0xa7, 0x76, 0x6f, 0x74, 0x65, 0x6c, 0x73, 0x74) o = (*z).KeyregTxnFields.VoteLast.MarshalMsg(o) } - if (zb0007Mask & 0x40000000000000) == 0 { // if not empty + if (zb0007Mask & 0x400000000000000) == 0 { // if not empty // string "xaid" o = append(o, 0xa4, 0x78, 0x61, 0x69, 0x64) o = (*z).AssetTransferTxnFields.XferAsset.MarshalMsg(o) @@ -6086,6 +6290,30 @@ func (z *Transaction) UnmarshalMsgWithState(bts []byte, st msgp.UnmarshalState) return } } + if zb0007 > 0 { + zb0007-- + bts, err = (*z).HeartbeatTxnFields.HbAddress.UnmarshalMsgWithState(bts, st) + if err != nil { + err = msgp.WrapError(err, "struct-from-array", "HbAddress") + return + } + } + if zb0007 > 0 { + zb0007-- + bts, err = (*z).HeartbeatTxnFields.HbProof.UnmarshalMsgWithState(bts, st) + if err != nil { + err = msgp.WrapError(err, "struct-from-array", "HbProof") + return + } + } + if zb0007 > 0 { + zb0007-- + bts, err = (*z).HeartbeatTxnFields.HbSeed.UnmarshalMsgWithState(bts, st) + if err != nil { + err = msgp.WrapError(err, "struct-from-array", "HbSeed") + return + } + } if zb0007 > 0 { err = msgp.ErrTooManyArrayFields(zb0007) if err != nil { @@ -6618,6 +6846,24 @@ func (z *Transaction) UnmarshalMsgWithState(bts []byte, st msgp.UnmarshalState) err = msgp.WrapError(err, "Message") return } + case "hbad": + bts, err = (*z).HeartbeatTxnFields.HbAddress.UnmarshalMsgWithState(bts, st) + if err != nil { + err = msgp.WrapError(err, "HbAddress") + return + } + case "hbprf": + bts, err = (*z).HeartbeatTxnFields.HbProof.UnmarshalMsgWithState(bts, st) + if err != nil { + err = msgp.WrapError(err, "HbProof") + return + } + case "hbsd": + bts, err = (*z).HeartbeatTxnFields.HbSeed.UnmarshalMsgWithState(bts, st) + if err != nil { + err = msgp.WrapError(err, "HbSeed") + return + } default: err = msgp.ErrNoField(string(field)) if err != nil { @@ -6661,13 +6907,13 @@ func (z *Transaction) Msgsize() (s int) { for zb0006 := range (*z).ApplicationCallTxnFields.ForeignAssets { s += (*z).ApplicationCallTxnFields.ForeignAssets[zb0006].Msgsize() } - s += 5 + (*z).ApplicationCallTxnFields.LocalStateSchema.Msgsize() + 5 + (*z).ApplicationCallTxnFields.GlobalStateSchema.Msgsize() + 5 + msgp.BytesPrefixSize + len((*z).ApplicationCallTxnFields.ApprovalProgram) + 5 + msgp.BytesPrefixSize + len((*z).ApplicationCallTxnFields.ClearStateProgram) + 5 + msgp.Uint32Size + 7 + (*z).StateProofTxnFields.StateProofType.Msgsize() + 3 + (*z).StateProofTxnFields.StateProof.Msgsize() + 6 + (*z).StateProofTxnFields.Message.Msgsize() + s += 5 + (*z).ApplicationCallTxnFields.LocalStateSchema.Msgsize() + 5 + (*z).ApplicationCallTxnFields.GlobalStateSchema.Msgsize() + 5 + msgp.BytesPrefixSize + len((*z).ApplicationCallTxnFields.ApprovalProgram) + 5 + msgp.BytesPrefixSize + len((*z).ApplicationCallTxnFields.ClearStateProgram) + 5 + msgp.Uint32Size + 7 + (*z).StateProofTxnFields.StateProofType.Msgsize() + 3 + (*z).StateProofTxnFields.StateProof.Msgsize() + 6 + (*z).StateProofTxnFields.Message.Msgsize() + 5 + (*z).HeartbeatTxnFields.HbAddress.Msgsize() + 6 + (*z).HeartbeatTxnFields.HbProof.Msgsize() + 5 + (*z).HeartbeatTxnFields.HbSeed.Msgsize() return } // MsgIsZero returns whether this is a zero value func (z *Transaction) MsgIsZero() bool { - return ((*z).Type.MsgIsZero()) && ((*z).Header.Sender.MsgIsZero()) && ((*z).Header.Fee.MsgIsZero()) && ((*z).Header.FirstValid.MsgIsZero()) && ((*z).Header.LastValid.MsgIsZero()) && (len((*z).Header.Note) == 0) && ((*z).Header.GenesisID == "") && ((*z).Header.GenesisHash.MsgIsZero()) && ((*z).Header.Group.MsgIsZero()) && ((*z).Header.Lease == ([32]byte{})) && ((*z).Header.RekeyTo.MsgIsZero()) && ((*z).KeyregTxnFields.VotePK.MsgIsZero()) && ((*z).KeyregTxnFields.SelectionPK.MsgIsZero()) && ((*z).KeyregTxnFields.StateProofPK.MsgIsZero()) && ((*z).KeyregTxnFields.VoteFirst.MsgIsZero()) && ((*z).KeyregTxnFields.VoteLast.MsgIsZero()) && ((*z).KeyregTxnFields.VoteKeyDilution == 0) && ((*z).KeyregTxnFields.Nonparticipation == false) && ((*z).PaymentTxnFields.Receiver.MsgIsZero()) && ((*z).PaymentTxnFields.Amount.MsgIsZero()) && ((*z).PaymentTxnFields.CloseRemainderTo.MsgIsZero()) && ((*z).AssetConfigTxnFields.ConfigAsset.MsgIsZero()) && ((*z).AssetConfigTxnFields.AssetParams.MsgIsZero()) && ((*z).AssetTransferTxnFields.XferAsset.MsgIsZero()) && ((*z).AssetTransferTxnFields.AssetAmount == 0) && ((*z).AssetTransferTxnFields.AssetSender.MsgIsZero()) && ((*z).AssetTransferTxnFields.AssetReceiver.MsgIsZero()) && ((*z).AssetTransferTxnFields.AssetCloseTo.MsgIsZero()) && ((*z).AssetFreezeTxnFields.FreezeAccount.MsgIsZero()) && ((*z).AssetFreezeTxnFields.FreezeAsset.MsgIsZero()) && ((*z).AssetFreezeTxnFields.AssetFrozen == false) && ((*z).ApplicationCallTxnFields.ApplicationID.MsgIsZero()) && ((*z).ApplicationCallTxnFields.OnCompletion == 0) && (len((*z).ApplicationCallTxnFields.ApplicationArgs) == 0) && (len((*z).ApplicationCallTxnFields.Accounts) == 0) && (len((*z).ApplicationCallTxnFields.ForeignApps) == 0) && (len((*z).ApplicationCallTxnFields.Boxes) == 0) && (len((*z).ApplicationCallTxnFields.ForeignAssets) == 0) && ((*z).ApplicationCallTxnFields.LocalStateSchema.MsgIsZero()) && ((*z).ApplicationCallTxnFields.GlobalStateSchema.MsgIsZero()) && (len((*z).ApplicationCallTxnFields.ApprovalProgram) == 0) && (len((*z).ApplicationCallTxnFields.ClearStateProgram) == 0) && ((*z).ApplicationCallTxnFields.ExtraProgramPages == 0) && ((*z).StateProofTxnFields.StateProofType.MsgIsZero()) && ((*z).StateProofTxnFields.StateProof.MsgIsZero()) && ((*z).StateProofTxnFields.Message.MsgIsZero()) + return ((*z).Type.MsgIsZero()) && ((*z).Header.Sender.MsgIsZero()) && ((*z).Header.Fee.MsgIsZero()) && ((*z).Header.FirstValid.MsgIsZero()) && ((*z).Header.LastValid.MsgIsZero()) && (len((*z).Header.Note) == 0) && ((*z).Header.GenesisID == "") && ((*z).Header.GenesisHash.MsgIsZero()) && ((*z).Header.Group.MsgIsZero()) && ((*z).Header.Lease == ([32]byte{})) && ((*z).Header.RekeyTo.MsgIsZero()) && ((*z).KeyregTxnFields.VotePK.MsgIsZero()) && ((*z).KeyregTxnFields.SelectionPK.MsgIsZero()) && ((*z).KeyregTxnFields.StateProofPK.MsgIsZero()) && ((*z).KeyregTxnFields.VoteFirst.MsgIsZero()) && ((*z).KeyregTxnFields.VoteLast.MsgIsZero()) && ((*z).KeyregTxnFields.VoteKeyDilution == 0) && ((*z).KeyregTxnFields.Nonparticipation == false) && ((*z).PaymentTxnFields.Receiver.MsgIsZero()) && ((*z).PaymentTxnFields.Amount.MsgIsZero()) && ((*z).PaymentTxnFields.CloseRemainderTo.MsgIsZero()) && ((*z).AssetConfigTxnFields.ConfigAsset.MsgIsZero()) && ((*z).AssetConfigTxnFields.AssetParams.MsgIsZero()) && ((*z).AssetTransferTxnFields.XferAsset.MsgIsZero()) && ((*z).AssetTransferTxnFields.AssetAmount == 0) && ((*z).AssetTransferTxnFields.AssetSender.MsgIsZero()) && ((*z).AssetTransferTxnFields.AssetReceiver.MsgIsZero()) && ((*z).AssetTransferTxnFields.AssetCloseTo.MsgIsZero()) && ((*z).AssetFreezeTxnFields.FreezeAccount.MsgIsZero()) && ((*z).AssetFreezeTxnFields.FreezeAsset.MsgIsZero()) && ((*z).AssetFreezeTxnFields.AssetFrozen == false) && ((*z).ApplicationCallTxnFields.ApplicationID.MsgIsZero()) && ((*z).ApplicationCallTxnFields.OnCompletion == 0) && (len((*z).ApplicationCallTxnFields.ApplicationArgs) == 0) && (len((*z).ApplicationCallTxnFields.Accounts) == 0) && (len((*z).ApplicationCallTxnFields.ForeignApps) == 0) && (len((*z).ApplicationCallTxnFields.Boxes) == 0) && (len((*z).ApplicationCallTxnFields.ForeignAssets) == 0) && ((*z).ApplicationCallTxnFields.LocalStateSchema.MsgIsZero()) && ((*z).ApplicationCallTxnFields.GlobalStateSchema.MsgIsZero()) && (len((*z).ApplicationCallTxnFields.ApprovalProgram) == 0) && (len((*z).ApplicationCallTxnFields.ClearStateProgram) == 0) && ((*z).ApplicationCallTxnFields.ExtraProgramPages == 0) && ((*z).StateProofTxnFields.StateProofType.MsgIsZero()) && ((*z).StateProofTxnFields.StateProof.MsgIsZero()) && ((*z).StateProofTxnFields.Message.MsgIsZero()) && ((*z).HeartbeatTxnFields.HbAddress.MsgIsZero()) && ((*z).HeartbeatTxnFields.HbProof.MsgIsZero()) && ((*z).HeartbeatTxnFields.HbSeed.MsgIsZero()) } // MaxSize returns a maximum valid message size for this message type @@ -6689,7 +6935,7 @@ func TransactionMaxSize() (s int) { s += 5 // Calculating size of slice: z.ApplicationCallTxnFields.ForeignAssets s += msgp.ArrayHeaderSize + ((encodedMaxForeignAssets) * (basics.AssetIndexMaxSize())) - s += 5 + basics.StateSchemaMaxSize() + 5 + basics.StateSchemaMaxSize() + 5 + msgp.BytesPrefixSize + config.MaxAvailableAppProgramLen + 5 + msgp.BytesPrefixSize + config.MaxAvailableAppProgramLen + 5 + msgp.Uint32Size + 7 + protocol.StateProofTypeMaxSize() + 3 + stateproof.StateProofMaxSize() + 6 + stateproofmsg.MessageMaxSize() + s += 5 + basics.StateSchemaMaxSize() + 5 + basics.StateSchemaMaxSize() + 5 + msgp.BytesPrefixSize + config.MaxAvailableAppProgramLen + 5 + msgp.BytesPrefixSize + config.MaxAvailableAppProgramLen + 5 + msgp.Uint32Size + 7 + protocol.StateProofTypeMaxSize() + 3 + stateproof.StateProofMaxSize() + 6 + stateproofmsg.MessageMaxSize() + 5 + basics.AddressMaxSize() + 6 + crypto.OneTimeSignatureMaxSize() + 5 + committee.SeedMaxSize() return } diff --git a/data/transactions/msgp_gen_test.go b/data/transactions/msgp_gen_test.go index 0ce6b29c38..49ed14f6e3 100644 --- a/data/transactions/msgp_gen_test.go +++ b/data/transactions/msgp_gen_test.go @@ -494,6 +494,66 @@ func BenchmarkUnmarshalHeader(b *testing.B) { } } +func TestMarshalUnmarshalHeartbeatTxnFields(t *testing.T) { + partitiontest.PartitionTest(t) + v := HeartbeatTxnFields{} + bts := v.MarshalMsg(nil) + left, err := v.UnmarshalMsg(bts) + if err != nil { + t.Fatal(err) + } + if len(left) > 0 { + t.Errorf("%d bytes left over after UnmarshalMsg(): %q", len(left), left) + } + + left, err = msgp.Skip(bts) + if err != nil { + t.Fatal(err) + } + if len(left) > 0 { + t.Errorf("%d bytes left over after Skip(): %q", len(left), left) + } +} + +func TestRandomizedEncodingHeartbeatTxnFields(t *testing.T) { + protocol.RunEncodingTest(t, &HeartbeatTxnFields{}) +} + +func BenchmarkMarshalMsgHeartbeatTxnFields(b *testing.B) { + v := HeartbeatTxnFields{} + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + v.MarshalMsg(nil) + } +} + +func BenchmarkAppendMsgHeartbeatTxnFields(b *testing.B) { + v := HeartbeatTxnFields{} + bts := make([]byte, 0, v.Msgsize()) + bts = v.MarshalMsg(bts[0:0]) + b.SetBytes(int64(len(bts))) + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + bts = v.MarshalMsg(bts[0:0]) + } +} + +func BenchmarkUnmarshalHeartbeatTxnFields(b *testing.B) { + v := HeartbeatTxnFields{} + bts := v.MarshalMsg(nil) + b.ReportAllocs() + b.SetBytes(int64(len(bts))) + b.ResetTimer() + for i := 0; i < b.N; i++ { + _, err := v.UnmarshalMsg(bts) + if err != nil { + b.Fatal(err) + } + } +} + func TestMarshalUnmarshalKeyregTxnFields(t *testing.T) { partitiontest.PartitionTest(t) v := KeyregTxnFields{} diff --git a/data/transactions/transaction.go b/data/transactions/transaction.go index a17066c3a5..363747a996 100644 --- a/data/transactions/transaction.go +++ b/data/transactions/transaction.go @@ -566,6 +566,11 @@ func (tx Transaction) WellFormed(spec SpecialAddresses, proto config.ConsensusPa return errLeaseMustBeZeroInStateproofTxn } + case protocol.HeartbeatTx: + if !proto.Heartbeat { + return fmt.Errorf("heartbeat transaction not supported") + } + default: return fmt.Errorf("unknown tx type %v", tx.Type) } diff --git a/data/transactions/transaction_test.go b/data/transactions/transaction_test.go index 08dd145a8c..1dbb2e316a 100644 --- a/data/transactions/transaction_test.go +++ b/data/transactions/transaction_test.go @@ -591,6 +591,21 @@ func TestWellFormedErrors(t *testing.T) { proto: protoV36, expectedError: nil, }, + { + tx: Transaction{ + Type: protocol.HeartbeatTx, + Header: okHeader, + }, + proto: protoV36, + expectedError: fmt.Errorf("heartbeat transaction not supported"), + }, + { + tx: Transaction{ + Type: protocol.HeartbeatTx, + Header: okHeader, + }, + proto: futureProto, + }, } for _, usecase := range usecases { err := usecase.tx.WellFormed(SpecialAddresses{}, usecase.proto) diff --git a/heartbeat/service.go b/heartbeat/service.go index 48d99cb959..15c771a68d 100644 --- a/heartbeat/service.go +++ b/heartbeat/service.go @@ -24,6 +24,7 @@ import ( "github.com/algorand/go-algorand/config" "github.com/algorand/go-algorand/data/account" "github.com/algorand/go-algorand/data/basics" + "github.com/algorand/go-algorand/data/bookkeeping" "github.com/algorand/go-algorand/data/transactions" "github.com/algorand/go-algorand/data/transactions/logic" "github.com/algorand/go-algorand/ledger/eval" @@ -77,9 +78,13 @@ func (s *Service) Stop() { // they have been challenged. func (s *Service) findChallenged(rules config.ProposerPayoutRules) []account.ParticipationRecordForRound { current := s.ledger.LastRound() - var found []account.ParticipationRecordForRound - ch := eval.ActiveChallenge(rules, current, s.ledger) + ch := eval.FindChallenge(rules, current, s.ledger, eval.ChRisky) + if ch.IsZero() { + return nil + } + + var found []account.ParticipationRecordForRound for _, pr := range s.accts.Keys(current + 1) { // only look at accounts we have part keys for acct, _, _, err := s.ledger.LookupAccount(current, pr.Account) fmt.Printf(" %v is %s at %d\n", pr.Account, acct.Status, current) @@ -89,7 +94,8 @@ func (s *Service) findChallenged(rules config.ProposerPayoutRules) []account.Par } if acct.Status == basics.Online { lastSeen := max(acct.LastProposed, acct.LastHeartbeat) - if eval.FailsChallenge(ch, pr.Account, lastSeen) { + fmt.Printf(" %v was last seen at %d\n", pr.Account, lastSeen) + if ch.Failed(pr.Account, lastSeen) { found = append(found, pr) } } @@ -120,15 +126,15 @@ func (s *Service) loop() { latest = s.ledger.LastRound() - hdr, err := s.ledger.BlockHdr(latest) + lastHdr, err := s.ledger.BlockHdr(latest) if err != nil { s.log.Errorf("heartbeat service could not fetch block header for round %d: %v", latest, err) continue // Try again next round, I guess? } - proto := config.Consensus[hdr.CurrentProtocol] + proto := config.Consensus[lastHdr.CurrentProtocol] for _, pr := range s.findChallenged(proto.Payouts) { - stxn := s.prepareHeartbeat(pr.Account, latest, hdr.GenesisHash) + stxn := s.prepareHeartbeat(pr, lastHdr) err = s.bcast.BroadcastInternalSignedTxGroup([]transactions.SignedTxn{stxn}) if err != nil { s.log.Errorf("error broadcasting heartbeat %v for %v: %v", stxn, pr.Account, err) @@ -137,22 +143,29 @@ func (s *Service) loop() { } } -// AcceptingByteCode is the source to a logic signature that will accept anything (except rekeying). +// acceptingByteCode is the byte code to a logic signature that will accept anything (except rekeying). var acceptingByteCode = logic.MustAssemble(` #pragma version 11 txn RekeyTo; global ZeroAddress; == `) var acceptingSender = basics.Address(logic.HashProgram(acceptingByteCode)) -func (s *Service) prepareHeartbeat(address basics.Address, latest basics.Round, genHash [32]byte) transactions.SignedTxn { +func (s *Service) prepareHeartbeat(pr account.ParticipationRecordForRound, latest bookkeeping.BlockHeader) transactions.SignedTxn { var stxn transactions.SignedTxn stxn.Lsig = transactions.LogicSig{Logic: acceptingByteCode} stxn.Txn.Type = protocol.HeartbeatTx stxn.Txn.Header = transactions.Header{ Sender: acceptingSender, - FirstValid: latest + 1, - LastValid: latest + 1 + 100, // maybe use the grace period? - GenesisHash: genHash, + FirstValid: latest.Round + 1, + LastValid: latest.Round + 1 + 100, // maybe use the grace period? + GenesisHash: latest.GenesisHash, + } + + id := basics.OneTimeIDForRound(latest.Round+1, pr.KeyDilution) + stxn.Txn.HeartbeatTxnFields = transactions.HeartbeatTxnFields{ + HbAddress: pr.Account, + HbProof: pr.Voting.Sign(id, latest.Seed), + HbSeed: latest.Seed, } return stxn diff --git a/heartbeat/service_test.go b/heartbeat/service_test.go index 2246bde174..566645a5c4 100644 --- a/heartbeat/service_test.go +++ b/heartbeat/service_test.go @@ -22,9 +22,11 @@ import ( "time" "github.com/algorand/go-algorand/config" + "github.com/algorand/go-algorand/crypto" "github.com/algorand/go-algorand/data/account" "github.com/algorand/go-algorand/data/basics" "github.com/algorand/go-algorand/data/bookkeeping" + "github.com/algorand/go-algorand/data/committee" "github.com/algorand/go-algorand/data/transactions" "github.com/algorand/go-algorand/ledger/ledgercore" "github.com/algorand/go-algorand/logging" @@ -34,52 +36,27 @@ import ( "github.com/stretchr/testify/require" ) -type mockParticipants struct { - accts map[basics.Address]struct{} -} - -func (p *mockParticipants) Keys(rnd basics.Round) []account.ParticipationRecordForRound { - var ret []account.ParticipationRecordForRound - for addr, _ := range p.accts { - ret = append(ret, account.ParticipationRecordForRound{ - ParticipationRecord: account.ParticipationRecord{ - ParticipationID: [32]byte{}, - Account: addr, - FirstValid: 0, - LastValid: 0, - KeyDilution: 0, - LastVote: 0, - LastBlockProposal: 0, - }, - }) - } - return ret -} - -func (p *mockParticipants) add(addr basics.Address) { - if p.accts == nil { - p.accts = make(map[basics.Address]struct{}) - } - p.accts[addr] = struct{}{} -} - type table map[basics.Address]ledgercore.AccountData type mockedLedger struct { mu deadlock.Mutex waiters map[basics.Round]chan struct{} history []table - version protocol.ConsensusVersion - hdrs map[basics.Round]bookkeeping.BlockHeader + hdr bookkeeping.BlockHeader + + participants map[basics.Address]*crypto.OneTimeSignatureSecrets } func newMockedLedger() mockedLedger { return mockedLedger{ waiters: make(map[basics.Round]chan struct{}), history: []table{nil}, // some genesis accounts could go here - version: protocol.ConsensusFuture, + hdr: bookkeeping.BlockHeader{ + UpgradeState: bookkeeping.UpgradeState{ + CurrentProtocol: protocol.ConsensusFuture, + }, + }, } - } func (l *mockedLedger) LastRound() basics.Round { @@ -115,22 +92,12 @@ func (l *mockedLedger) BlockHdr(r basics.Round) (bookkeeping.BlockHeader, error) if r > l.LastRound() { return bookkeeping.BlockHeader{}, fmt.Errorf("%d is beyond current block (%d)", r, l.LastRound()) } - if hdr, ok := l.hdrs[r]; ok { - return hdr, nil - } - // just return a simple hdr - var hdr bookkeeping.BlockHeader + // return the template hdr, with round + hdr := l.hdr hdr.Round = r - hdr.CurrentProtocol = l.version return hdr, nil } -// addHeader places a block header into the ledger's history. It is used to make -// challenges occur as we'd like. -func (l *mockedLedger) addHeader(hdr bookkeeping.BlockHeader) { - l.hdrs[hdr.Round] = hdr -} - func (l *mockedLedger) addBlock(delta table) error { l.mu.Lock() defer l.mu.Unlock() @@ -181,6 +148,38 @@ func (l *mockedLedger) waitFor(s *Service, a *require.Assertions) { }, time.Second, 10*time.Millisecond) } +func (l *mockedLedger) Keys(rnd basics.Round) []account.ParticipationRecordForRound { + var ret []account.ParticipationRecordForRound + for addr, secrets := range l.participants { + if rnd > l.LastRound() { // Usually we're looking for key material for a future round + rnd = l.LastRound() + } + acct, _, _, err := l.LookupAccount(rnd, addr) + if err != nil { + panic(err.Error()) + } + + ret = append(ret, account.ParticipationRecordForRound{ + ParticipationRecord: account.ParticipationRecord{ + ParticipationID: [32]byte{}, + Account: addr, + Voting: secrets, + FirstValid: acct.VoteFirstValid, + LastValid: acct.VoteLastValid, + KeyDilution: acct.VoteKeyDilution, + }, + }) + } + return ret +} + +func (l *mockedLedger) addParticipant(addr basics.Address, otss *crypto.OneTimeSignatureSecrets) { + if l.participants == nil { + l.participants = make(map[basics.Address]*crypto.OneTimeSignatureSecrets) + } + l.participants[addr] = otss +} + type txnSink [][]transactions.SignedTxn func (ts *txnSink) BroadcastInternalSignedTxGroup(group []transactions.SignedTxn) error { @@ -195,9 +194,8 @@ func TestStartStop(t *testing.T) { a := require.New(t) sink := txnSink{} - accts := &mockParticipants{} ledger := newMockedLedger() - s := NewService(accts, &ledger, &sink, logging.TestingLog(t)) + s := NewService(&ledger, &ledger, &sink, logging.TestingLog(t)) a.NotNil(s) a.NoError(ledger.addBlock(nil)) s.Start() @@ -218,14 +216,14 @@ func TestHeartBeatOnlyWhenChallenged(t *testing.T) { a := require.New(t) sink := txnSink{} - accts := &mockParticipants{} ledger := newMockedLedger() - s := NewService(accts, &ledger, &sink, logging.TestingLog(t)) + s := NewService(&ledger, &ledger, &sink, logging.TestingLog(t)) s.Start() - // joe is a simple, non-online account, service will not heartbeat - joe := basics.Address{0xcc} // 0xcc will matter when we set the challenge - accts.add(joe) + joe := basics.Address{0xcc} // 0xcc will matter when we set the challenge + mary := basics.Address{0xaa} // 0xaa will matter when we set the challenge + ledger.addParticipant(joe, nil) + ledger.addParticipant(mary, nil) acct := ledgercore.AccountData{} @@ -233,19 +231,27 @@ func TestHeartBeatOnlyWhenChallenged(t *testing.T) { ledger.waitFor(s, a) a.Empty(sink) - // now joe is online, but not challenged, so no heartbeat + // now they are online, but not challenged, so no heartbeat acct.Status = basics.Online - - a.NoError(ledger.addBlock(table{joe: acct})) + acct.VoteKeyDilution = 100 + otss := crypto.GenerateOneTimeSignatureSecrets( + basics.OneTimeIDForRound(ledger.LastRound(), acct.VoteKeyDilution).Batch, + 5) + acct.VoteID = otss.OneTimeSignatureVerifier + ledger.addParticipant(joe, otss) + ledger.addParticipant(mary, otss) + + a.NoError(ledger.addBlock(table{joe: acct, mary: acct})) a.Empty(sink) // now we have to make it seem like joe has been challenged. We obtain the // payout rules to find the first challenge round, skip forward to it, then // go forward half a grace period. Only then should the service heartbeat hdr, err := ledger.BlockHdr(ledger.LastRound()) + ledger.hdr.Seed = committee.Seed{0xc8} // share 5 bits with 0xcc a.NoError(err) rules := config.Consensus[hdr.CurrentProtocol].Payouts - for ledger.LastRound() < basics.Round(rules.ChallengeInterval) { + for ledger.LastRound() < basics.Round(rules.ChallengeInterval+rules.ChallengeGracePeriod/2) { a.NoError(ledger.addBlock(table{})) ledger.waitFor(s, a) a.Empty(sink) @@ -253,10 +259,10 @@ func TestHeartBeatOnlyWhenChallenged(t *testing.T) { a.NoError(ledger.addBlock(table{joe: acct})) ledger.waitFor(s, a) - a.Len(sink, 1) // only one heartbeat so far + a.Len(sink, 1) // only one heartbeat (for joe) a.Len(sink[0], 1) a.Equal(sink[0][0].Txn.Type, protocol.HeartbeatTx) - a.Equal(sink[0][0].Txn.HeartbeatAddress, joe) + a.Equal(sink[0][0].Txn.HbAddress, joe) s.Stop() } diff --git a/ledger/apply/heartbeat.go b/ledger/apply/heartbeat.go index bce788cf68..7a2020006f 100644 --- a/ledger/apply/heartbeat.go +++ b/ledger/apply/heartbeat.go @@ -27,14 +27,14 @@ import ( // Heartbeat applies a Heartbeat transaction using the Balances interface. func Heartbeat(hb transactions.HeartbeatTxnFields, header transactions.Header, balances Balances, provider HdrProvider, round basics.Round) error { // Get the account's balance entry - account, err := balances.Get(hb.HeartbeatAddress, false) + account, err := balances.Get(hb.HbAddress, false) if err != nil { return err } sv := account.VoteID if sv.IsEmpty() { - return fmt.Errorf("heartbeat address %s has no voting keys\n", hb.HeartbeatAddress) + return fmt.Errorf("heartbeat address %s has no voting keys", hb.HbAddress) } id := basics.OneTimeIDForRound(header.LastValid, account.VoteKeyDilution) @@ -42,15 +42,18 @@ func Heartbeat(hb transactions.HeartbeatTxnFields, header transactions.Header, b if err != nil { return err } + if hdr.Seed != hb.HbSeed { + return fmt.Errorf("provided seed %v does not match round %d's seed %v", hb.HbSeed, header.FirstValid-1, hdr.Seed) + } - if !sv.Verify(id, hdr.Seed, hb.Proof) { + if !sv.Verify(id, hdr.Seed, hb.HbProof) { return errors.New("Improper heartbeat") } account.LastHeartbeat = round // Write the updated entry - err = balances.Put(hb.HeartbeatAddress, account) + err = balances.Put(hb.HbAddress, account) if err != nil { return err } diff --git a/ledger/apply/heartbeat_test.go b/ledger/apply/heartbeat_test.go index e6fe7d66a9..42f9cd4732 100644 --- a/ledger/apply/heartbeat_test.go +++ b/ledger/apply/heartbeat_test.go @@ -73,9 +73,9 @@ func TestHeartbeat(t *testing.T) { LastValid: lv, }, HeartbeatTxnFields: transactions.HeartbeatTxnFields{ - HeartbeatAddress: voter, - Proof: otss.Sign(id, seed), - Seed: seed, + HbAddress: voter, + HbProof: otss.Sign(id, seed), + HbSeed: seed, }, } diff --git a/ledger/apply/mockBalances_test.go b/ledger/apply/mockBalances_test.go index 91f918cf80..a5f636fc08 100644 --- a/ledger/apply/mockBalances_test.go +++ b/ledger/apply/mockBalances_test.go @@ -291,5 +291,5 @@ func (m mockHeaders) BlockHdr(r basics.Round) (bookkeeping.BlockHeader, error) { if hdr, ok := m.b[r]; ok { return hdr, nil } - return bookkeeping.BlockHeader{}, fmt.Errorf("round %v is not present\n", r) + return bookkeeping.BlockHeader{}, fmt.Errorf("round %v is not present", r) } diff --git a/ledger/apptxn_test.go b/ledger/apptxn_test.go index a7b3b15214..fce41b00a3 100644 --- a/ledger/apptxn_test.go +++ b/ledger/apptxn_test.go @@ -104,9 +104,9 @@ func TestPayAction(t *testing.T) { dl.t.Log("postsink", postsink, "postprop", postprop) if ver >= payoutsVer { - bonus := 10_000_000 // config/consensus.go - assert.EqualValues(t, bonus-500, presink-postsink) // based on 75% in config/consensus.go - require.EqualValues(t, bonus+1500, postprop-preprop) + bonus := 10_000_000 // config/consensus.go + assert.EqualValues(t, bonus-1000, presink-postsink) // based on 50% in config/consensus.go + require.EqualValues(t, bonus+1000, postprop-preprop) } else { require.EqualValues(t, 2000, postsink-presink) // no payouts yet } diff --git a/ledger/eval/eval.go b/ledger/eval/eval.go index 0014bc0044..6365a40936 100644 --- a/ledger/eval/eval.go +++ b/ledger/eval/eval.go @@ -1621,7 +1621,7 @@ func (eval *BlockEvaluator) generateKnockOfflineAccountsList() { updates := &eval.block.ParticipationUpdates - ch := ActiveChallenge(eval.proto.Payouts, eval.Round(), eval.state) + ch := FindChallenge(eval.proto.Payouts, eval.Round(), eval.state, ChActive) for _, accountAddr := range eval.state.modifiedAccounts() { acctData, found := eval.state.mods.Accts.GetData(accountAddr) @@ -1651,7 +1651,7 @@ func (eval *BlockEvaluator) generateKnockOfflineAccountsList() { if acctData.Status == basics.Online { lastSeen := max(acctData.LastProposed, acctData.LastHeartbeat) if isAbsent(eval.state.prevTotals.Online.Money, acctData.MicroAlgos, lastSeen, current) || - FailsChallenge(ch, accountAddr, lastSeen) { + ch.Failed(accountAddr, lastSeen) { updates.AbsentParticipationAccounts = append( updates.AbsentParticipationAccounts, accountAddr, @@ -1712,10 +1712,19 @@ type headerSource interface { BlockHdr(round basics.Round) (bookkeeping.BlockHeader, error) } -// ActiveChallenge returns details about the Challenge that was last issued if -// it is still in effect (that is, between one and two grace periods from -// issue). Otherwise it returns the zero value. -func ActiveChallenge(rules config.ProposerPayoutRules, current basics.Round, headers headerSource) challenge { +// ChallengePeriod indicates which part of the challenge period is under discussion. +type ChallengePeriod int + +const ( + // ChRisky indicates that a challenge is in effect, and the initial grace period is running out. + ChRisky ChallengePeriod = iota + // ChActive indicates that a challenege is in effect, and the grace period + // has run out, so accounts can be suspended + ChActive +) + +// FindChallenge returns the Challenge that was last issued if it's in the period requested. +func FindChallenge(rules config.ProposerPayoutRules, current basics.Round, headers headerSource, period ChallengePeriod) challenge { // are challenges active? interval := basics.Round(rules.ChallengeInterval) if rules.ChallengeInterval == 0 || current < interval { @@ -1723,9 +1732,18 @@ func ActiveChallenge(rules config.ProposerPayoutRules, current basics.Round, hea } lastChallenge := current - (current % interval) grace := basics.Round(rules.ChallengeGracePeriod) - // challenge is in effect if we're after one grace period, but before the 2nd ends. - if current <= lastChallenge+grace || current > lastChallenge+2*grace { - return challenge{} + // FindChallenge is structured this way, instead of returning the challenge + // and letting the caller determine the period it cares about, to avoid + // using BlockHdr unnecessarily. + switch period { + case ChRisky: + if current <= lastChallenge+grace/2 || current > lastChallenge+grace { + return challenge{} + } + case ChActive: + if current <= lastChallenge+grace || current > lastChallenge+2*grace { + return challenge{} + } } challengeHdr, err := headers.BlockHdr(lastChallenge) if err != nil { @@ -1739,9 +1757,14 @@ func ActiveChallenge(rules config.ProposerPayoutRules, current basics.Round, hea return challenge{lastChallenge, challengeHdr.Seed, rules.ChallengeBits} } -// FailsChallenge returns true iff ch is in effect, matches address, and -// lastSeen is before the challenge issue. -func FailsChallenge(ch challenge, address basics.Address, lastSeen basics.Round) bool { +// IsZero returns true if the challenge is empty (used to indicate no challenege) +func (ch challenge) IsZero() bool { + return ch == challenge{} +} + +// Failed returns true iff ch is in effect, matches address, and lastSeen is +// before the challenge issue. +func (ch challenge) Failed(address basics.Address, lastSeen basics.Round) bool { return ch.round != 0 && bitsMatch(ch.seed[:], address[:], ch.bits) && lastSeen < ch.round } @@ -1812,7 +1835,7 @@ func (eval *BlockEvaluator) validateAbsentOnlineAccounts() error { // For consistency with expired account handling, we preclude duplicates addressSet := make(map[basics.Address]bool, suspensionCount) - ch := ActiveChallenge(eval.proto.Payouts, eval.Round(), eval.state) + ch := FindChallenge(eval.proto.Payouts, eval.Round(), eval.state, ChActive) for _, accountAddr := range eval.block.ParticipationUpdates.AbsentParticipationAccounts { if _, exists := addressSet[accountAddr]; exists { @@ -1833,7 +1856,7 @@ func (eval *BlockEvaluator) validateAbsentOnlineAccounts() error { if isAbsent(eval.state.prevTotals.Online.Money, acctData.MicroAlgos, lastSeen, eval.Round()) { continue // ok. it's "normal absent" } - if FailsChallenge(ch, accountAddr, lastSeen) { + if ch.Failed(accountAddr, lastSeen) { continue // ok. it's "challenge absent" } return fmt.Errorf("proposed absent account %v is not absent in %d, %d", diff --git a/ledger/eval/eval_test.go b/ledger/eval/eval_test.go index d2cf2e2e6e..f0ba85102c 100644 --- a/ledger/eval/eval_test.go +++ b/ledger/eval/eval_test.go @@ -1644,16 +1644,16 @@ func TestFailsChallenge(t *testing.T) { a := assert.New(t) // a valid challenge, with 4 matching bits, and an old last seen - a.True(FailsChallenge(challenge{round: 11, seed: [32]byte{0xb0, 0xb4}, bits: 4}, basics.Address{0xbf, 0x34}, 10)) + a.True(challenge{round: 11, seed: [32]byte{0xb0, 0xb4}, bits: 4}.Failed(basics.Address{0xbf, 0x34}, 10)) // challenge isn't "on" - a.False(FailsChallenge(challenge{round: 0, seed: [32]byte{0xb0, 0xb4}, bits: 4}, basics.Address{0xbf, 0x34}, 10)) + a.False(challenge{round: 0, seed: [32]byte{0xb0, 0xb4}, bits: 4}.Failed(basics.Address{0xbf, 0x34}, 10)) // node has appeared more recently - a.False(FailsChallenge(challenge{round: 11, seed: [32]byte{0xb0, 0xb4}, bits: 4}, basics.Address{0xbf, 0x34}, 12)) + a.False(challenge{round: 11, seed: [32]byte{0xb0, 0xb4}, bits: 4}.Failed(basics.Address{0xbf, 0x34}, 12)) // bits don't match - a.False(FailsChallenge(challenge{round: 11, seed: [32]byte{0xb0, 0xb4}, bits: 4}, basics.Address{0xcf, 0x34}, 10)) + a.False(challenge{round: 11, seed: [32]byte{0xb0, 0xb4}, bits: 4}.Failed(basics.Address{0xcf, 0x34}, 10)) // no enough bits match - a.False(FailsChallenge(challenge{round: 11, seed: [32]byte{0xb0, 0xb4}, bits: 5}, basics.Address{0xbf, 0x34}, 10)) + a.False(challenge{round: 11, seed: [32]byte{0xb0, 0xb4}, bits: 5}.Failed(basics.Address{0xbf, 0x34}, 10)) } type singleSource bookkeeping.BlockHeader @@ -1677,23 +1677,23 @@ func TestActiveChallenge(t *testing.T) { // simplest test. when interval=X and grace=G, X+G+1 is a challenge inChallenge := basics.Round(rules.ChallengeInterval + rules.ChallengeGracePeriod + 1) - ch := ActiveChallenge(rules, inChallenge, singleSource(nowHeader)) + ch := FindChallenge(rules, inChallenge, singleSource(nowHeader), ChActive) a.NotZero(ch.round) // all rounds before that have no challenge for r := basics.Round(1); r < inChallenge; r++ { - ch := ActiveChallenge(rules, r, singleSource(nowHeader)) + ch := FindChallenge(rules, r, singleSource(nowHeader), ChActive) a.Zero(ch.round, r) } // ChallengeGracePeriod rounds allow challenges starting with inChallenge for r := inChallenge; r < inChallenge+basics.Round(rules.ChallengeGracePeriod); r++ { - ch := ActiveChallenge(rules, r, singleSource(nowHeader)) + ch := FindChallenge(rules, r, singleSource(nowHeader), ChActive) a.EqualValues(ch.round, rules.ChallengeInterval) } // And the next round is again challenge-less - ch = ActiveChallenge(rules, inChallenge+basics.Round(rules.ChallengeGracePeriod), singleSource(nowHeader)) + ch = FindChallenge(rules, inChallenge+basics.Round(rules.ChallengeGracePeriod), singleSource(nowHeader), ChActive) a.Zero(ch.round) // ignore challenge if upgrade happened @@ -1703,6 +1703,6 @@ func TestActiveChallenge(t *testing.T) { CurrentProtocol: protocol.ConsensusV39, }, } - ch = ActiveChallenge(rules, inChallenge, singleSource(oldHeader)) + ch = FindChallenge(rules, inChallenge, singleSource(oldHeader), ChActive) a.Zero(ch.round) } diff --git a/ledger/eval_simple_test.go b/ledger/eval_simple_test.go index 972821c26c..bc88da2622 100644 --- a/ledger/eval_simple_test.go +++ b/ledger/eval_simple_test.go @@ -280,15 +280,15 @@ func TestPayoutFees(t *testing.T) { // new fields are in the header require.EqualValues(t, 2000, vb.Block().FeesCollected.Raw) require.EqualValues(t, bonus1, vb.Block().Bonus.Raw) - require.EqualValues(t, bonus1+1_500, vb.Block().ProposerPayout().Raw) + require.EqualValues(t, bonus1+1_000, vb.Block().ProposerPayout().Raw) // This last one is really only testing the "fake" agreement that // happens in dl.endBlock(). require.EqualValues(t, proposer, vb.Block().Proposer()) // At the end of the block, part of the fees + bonus have been moved to // the proposer. - require.EqualValues(t, bonus1+1500, postprop-preprop) // based on 75% in config/consensus.go - require.EqualValues(t, bonus1-500, presink-postsink) + require.EqualValues(t, bonus1+1_000, postprop-preprop) // based on 75% in config/consensus.go + require.EqualValues(t, bonus1-1_000, presink-postsink) require.Equal(t, prp.LastProposed, dl.generator.Latest()) } else { require.False(t, dl.generator.GenesisProto().Payouts.Enabled) From 79d070159cf4b00e22ba8fcd06cb30a36e8ed790 Mon Sep 17 00:00:00 2001 From: John Jannotti Date: Wed, 23 Oct 2024 15:03:47 -0400 Subject: [PATCH 06/19] DoubleLedger tests on heartbeat --- data/txntest/txn.go | 10 +++ ledger/apply/heartbeat.go | 31 ++++++-- ledger/heartbeat_test.go | 145 ++++++++++++++++++++++++++++++++++++++ 3 files changed, 182 insertions(+), 4 deletions(-) create mode 100644 ledger/heartbeat_test.go diff --git a/data/txntest/txn.go b/data/txntest/txn.go index aea4de005b..5b07ad03d5 100644 --- a/data/txntest/txn.go +++ b/data/txntest/txn.go @@ -26,6 +26,7 @@ import ( "github.com/algorand/go-algorand/crypto/merklesignature" "github.com/algorand/go-algorand/crypto/stateproof" "github.com/algorand/go-algorand/data/basics" + "github.com/algorand/go-algorand/data/committee" "github.com/algorand/go-algorand/data/stateproofmsg" "github.com/algorand/go-algorand/data/transactions" "github.com/algorand/go-algorand/data/transactions/logic" @@ -91,6 +92,10 @@ type Txn struct { StateProofType protocol.StateProofType StateProof stateproof.StateProof StateProofMsg stateproofmsg.Message + + HbAddress basics.Address + HbProof crypto.OneTimeSignature + HbSeed committee.Seed } // internalCopy "finishes" a shallow copy done by a simple Go assignment by @@ -281,6 +286,11 @@ func (tx Txn) Txn() transactions.Transaction { StateProof: tx.StateProof, Message: tx.StateProofMsg, }, + HeartbeatTxnFields: transactions.HeartbeatTxnFields{ + HbAddress: tx.HbAddress, + HbProof: tx.HbProof, + HbSeed: tx.HbSeed, + }, } } diff --git a/ledger/apply/heartbeat.go b/ledger/apply/heartbeat.go index 7a2020006f..4e858f9990 100644 --- a/ledger/apply/heartbeat.go +++ b/ledger/apply/heartbeat.go @@ -17,7 +17,6 @@ package apply import ( - "errors" "fmt" "github.com/algorand/go-algorand/data/basics" @@ -32,12 +31,36 @@ func Heartbeat(hb transactions.HeartbeatTxnFields, header transactions.Header, b return err } - sv := account.VoteID + // Note the contrast with agreement. We are using the account's _current_ + // partkey to verify the heartbeat. This is required because we can only + // look 320 rounds back for voting information. If a heartbeat was delayed a + // few rounds (even 1), we could not ask "what partkey was in effect at + // firstValid-320?" Using the current keys means that an account that + // changes keys would invalidate any heartbeats it has already sent out + // (that haven't been evaluated yet). Maybe more importantly, after going + // offline, an account can no longer heartbeat, since it has no _current_ + // keys. Yet it is still expected to vote for 320 rounds. Therefore, + // challenges do not apply to accounts that are offline (even if they should + // still be voting). + + // Conjure up an OnlineAccountData from current state, for convenience of + // oad.KeyDilution(). + oad := basics.OnlineAccountData{ + VotingData: account.VotingData, + } + + sv := oad.VoteID if sv.IsEmpty() { return fmt.Errorf("heartbeat address %s has no voting keys", hb.HbAddress) } - id := basics.OneTimeIDForRound(header.LastValid, account.VoteKeyDilution) + kd := oad.KeyDilution(balances.ConsensusParams()) + + // heartbeats are expected to sign with the partkey for their last-valid round + id := basics.OneTimeIDForRound(header.LastValid, kd) + // heartbeats sign a message consisting of the BlockSeed of the round before + // first-valid, to discourage unsavory behaviour like presigning a bunch of + // heartbeats for later use keeping an unavailable account online. hdr, err := provider.BlockHdr(header.FirstValid - 1) if err != nil { return err @@ -47,7 +70,7 @@ func Heartbeat(hb transactions.HeartbeatTxnFields, header transactions.Header, b } if !sv.Verify(id, hdr.Seed, hb.HbProof) { - return errors.New("Improper heartbeat") + return fmt.Errorf("heartbeat failed verification with VoteID %v", sv) } account.LastHeartbeat = round diff --git a/ledger/heartbeat_test.go b/ledger/heartbeat_test.go new file mode 100644 index 0000000000..6de5d1f111 --- /dev/null +++ b/ledger/heartbeat_test.go @@ -0,0 +1,145 @@ +// Copyright (C) 2019-2024 Algorand, Inc. +// This file is part of go-algorand +// +// go-algorand is free software: you can redistribute it and/or modify +// it under the terms of the GNU Affero General Public License as +// published by the Free Software Foundation, either version 3 of the +// License, or (at your option) any later version. +// +// go-algorand is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Affero General Public License for more details. +// +// You should have received a copy of the GNU Affero General Public License +// along with go-algorand. If not, see . + +package ledger + +import ( + "testing" + + "github.com/algorand/go-algorand/config" + "github.com/algorand/go-algorand/crypto" + "github.com/algorand/go-algorand/data/basics" + "github.com/algorand/go-algorand/data/txntest" + ledgertesting "github.com/algorand/go-algorand/ledger/testing" + "github.com/algorand/go-algorand/protocol" + "github.com/algorand/go-algorand/test/partitiontest" + "github.com/stretchr/testify/require" +) + +/* Tests within the `apply` package test the effects of heartbeats, while test + here are closer to integration tests, they test heartbeats in the context of + a more realistic ledger. */ + +// TestHearbeat exercises heartbeat transactions +func TestHeartBeat(t *testing.T) { + partitiontest.PartitionTest(t) + t.Parallel() + + genBalances, addrs, _ := ledgertesting.NewTestGenesis(func(cfg *ledgertesting.GenesisCfg) { + cfg.OnlineCount = 2 // addrs[0] and addrs[1] will be online + }) + heartbeatsBegin := 40 + + ledgertesting.TestConsensusRange(t, heartbeatsBegin, 0, + func(t *testing.T, ver int, cv protocol.ConsensusVersion, cfg config.Local) { + dl := NewDoubleLedger(t, genBalances, cv, cfg) + defer dl.Close() + + // empty HbAddress means ZeroAddress, and it's not online + dl.txn(&txntest.Txn{Type: "hb", Sender: addrs[1]}, + "AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAY5HFKQ has no voting keys") + + // addrs[2] is not online, it has no voting keys + dl.txn(&txntest.Txn{Type: "hb", Sender: addrs[1], HbAddress: addrs[2]}, + addrs[2].String()+" has no voting keys") + + // addrs[1] is online, it has voting keys, but seed is missing + dl.txn(&txntest.Txn{Type: "hb", Sender: addrs[1], HbAddress: addrs[1]}, + "does not match round 0's seed") + + // NewTestGenesis creates random VoterID. Verification will fail. + b0, err := dl.generator.BlockHdr(0) + require.NoError(t, err) + dl.txn(&txntest.Txn{ + Type: "hb", + Sender: addrs[1], + HbAddress: addrs[1], + HbSeed: b0.Seed, + }, + "heartbeat failed verification with") + + // keyreg addr[1] so we have a valid VoterID + const kd = 10 + firstID := basics.OneTimeIDForRound(1, kd) + otss := crypto.GenerateOneTimeSignatureSecrets(firstID.Batch, 5) + dl.txn(&txntest.Txn{ + Type: "keyreg", + Sender: addrs[1], + VotePK: otss.OneTimeSignatureVerifier, + SelectionPK: crypto.VrfPubkey([32]byte{0x01}), // must be non-zero + VoteKeyDilution: kd, + }) + + // Supply and sign the wrong HbSeed + dl.txn(&txntest.Txn{ + Type: "hb", + Sender: addrs[1], + HbAddress: addrs[1], + HbSeed: b0.Seed, + HbProof: otss.Sign(firstID, b0.Seed), + }, + "does not match round 1's seed") + + b1, err := dl.generator.BlockHdr(1) + require.NoError(t, err) + + // Supply the right seed, but sign something else. We're also now + // setting LastValid and the proper OneTimeIDForRound, so that these + // tests are failing for the reasons described, not that. + dl.txn(&txntest.Txn{ + Type: "hb", + LastValid: 30, + Sender: addrs[1], + HbAddress: addrs[1], + HbSeed: b1.Seed, + HbProof: otss.Sign(basics.OneTimeIDForRound(30, kd), b0.Seed), + }, + "failed verification") + + // Sign the right seed, but supply something else + dl.txn(&txntest.Txn{ + Type: "hb", + LastValid: 30, + Sender: addrs[1], + HbAddress: addrs[1], + HbSeed: b0.Seed, + HbProof: otss.Sign(basics.OneTimeIDForRound(30, kd), b1.Seed), + }, + "does not match round 1's") + + // Mismatch the last valid and OneTimeIDForRound + dl.txn(&txntest.Txn{ + Type: "hb", + LastValid: 29, + Sender: addrs[1], + HbAddress: addrs[1], + HbSeed: b1.Seed, + HbProof: otss.Sign(basics.OneTimeIDForRound(30, kd), b1.Seed), + }, + "failed verification") + + // now we can make a real heartbeat, with a properly signed blockseed + dl.txn(&txntest.Txn{ + Type: "hb", + LastValid: 30, + Sender: addrs[1], + HbAddress: addrs[1], + HbSeed: b1.Seed, + HbProof: otss.Sign(basics.OneTimeIDForRound(30, kd), b1.Seed), + }) + + }) +} From d38d43ca3604e1d5d6d9f727374d0e2325889f69 Mon Sep 17 00:00:00 2001 From: John Jannotti Date: Tue, 29 Oct 2024 16:14:35 -0400 Subject: [PATCH 07/19] Heartbeat service fixes and test --- data/transactions/verify/txn.go | 11 +- heartbeat/service.go | 23 ++- node/node.go | 3 + stateproof/worker.go | 4 +- .../features/incentives/challenge_test.go | 173 ++++++++++++++++++ test/framework/fixtures/libgoalFixture.go | 8 + test/testdata/nettemplates/Challenges.json | 44 +++++ util/set.go | 34 ++++ util/set_test.go | 75 ++++++++ 9 files changed, 359 insertions(+), 16 deletions(-) create mode 100644 test/e2e-go/features/incentives/challenge_test.go create mode 100644 test/testdata/nettemplates/Challenges.json create mode 100644 util/set_test.go diff --git a/data/transactions/verify/txn.go b/data/transactions/verify/txn.go index d0f38bca90..893f29450e 100644 --- a/data/transactions/verify/txn.go +++ b/data/transactions/verify/txn.go @@ -220,10 +220,15 @@ func txnGroupBatchPrep(stxs []transactions.SignedTxn, contextHdr *bookkeeping.Bl prepErr.err = fmt.Errorf("transaction %+v invalid : %w", stxn, prepErr.err) return nil, prepErr } - if stxn.Txn.Type != protocol.StateProofTx { - minFeeCount++ - } feesPaid = basics.AddSaturate(feesPaid, stxn.Txn.Fee.Raw) + if stxn.Txn.Type == protocol.StateProofTx { + continue + } + if stxn.Txn.Type == protocol.HeartbeatTx && len(stxs) == 1 { + // TODO: Only allow free HB if the HbAddress is challenged + continue + } + minFeeCount++ } feeNeeded, overflow := basics.OMul(groupCtx.consensusParams.MinTxnFee, minFeeCount) if overflow { diff --git a/heartbeat/service.go b/heartbeat/service.go index 15c771a68d..446a51c346 100644 --- a/heartbeat/service.go +++ b/heartbeat/service.go @@ -43,34 +43,36 @@ type Service struct { bcast txnBroadcaster // infrastructure - ctx context.Context - stop context.CancelFunc - wg sync.WaitGroup - log logging.Logger + ctx context.Context + shutdown context.CancelFunc + wg sync.WaitGroup + log logging.Logger } // NewService creates a heartbeat service. It will need to know which accounts // to emit heartbeats for, and how to create the heartbeats. -func NewService(accts participants, ledger ledger, bcast txnBroadcaster, log logging.Logger) (s *Service) { - s = &Service{ +func NewService(accts participants, ledger ledger, bcast txnBroadcaster, log logging.Logger) *Service { + return &Service{ accts: accts, ledger: ledger, bcast: bcast, log: log.With("Context", "heartbeat"), } - return s } // Start starts the goroutines for the Service. func (s *Service) Start() { - s.ctx, s.stop = context.WithCancel(context.Background()) + s.ctx, s.shutdown = context.WithCancel(context.Background()) s.wg.Add(1) + s.log.Info("starting heartbeat service") go s.loop() } // Stop any goroutines associated with this worker. func (s *Service) Stop() { - s.stop() + s.log.Debug("heartbeat service is stopping") + defer s.log.Debug("heartbeat service has stopped") + s.shutdown() s.wg.Wait() } @@ -96,6 +98,7 @@ func (s *Service) findChallenged(rules config.ProposerPayoutRules) []account.Par lastSeen := max(acct.LastProposed, acct.LastHeartbeat) fmt.Printf(" %v was last seen at %d\n", pr.Account, lastSeen) if ch.Failed(pr.Account, lastSeen) { + fmt.Printf(" %v needs a heartbeat\n", pr.Account) found = append(found, pr) } } @@ -161,7 +164,7 @@ func (s *Service) prepareHeartbeat(pr account.ParticipationRecordForRound, lates GenesisHash: latest.GenesisHash, } - id := basics.OneTimeIDForRound(latest.Round+1, pr.KeyDilution) + id := basics.OneTimeIDForRound(stxn.Txn.LastValid, pr.KeyDilution) stxn.Txn.HeartbeatTxnFields = transactions.HeartbeatTxnFields{ HbAddress: pr.Account, HbProof: pr.Voting.Sign(id, latest.Seed), diff --git a/node/node.go b/node/node.go index d3ce99901b..f536742c22 100644 --- a/node/node.go +++ b/node/node.go @@ -385,6 +385,7 @@ func (node *AlgorandFullNode) Start() error { node.ledgerService.Start() node.txHandler.Start() node.stateProofWorker.Start() + node.heartbeatService.Start() err := startNetwork() if err != nil { return err @@ -1226,6 +1227,7 @@ func (node *AlgorandFullNode) SetCatchpointCatchupMode(catchpointCatchupMode boo node.net.ClearHandlers() node.net.ClearValidatorHandlers() node.stateProofWorker.Stop() + node.heartbeatService.Stop() node.txHandler.Stop() node.agreementService.Shutdown() node.catchupService.Stop() @@ -1253,6 +1255,7 @@ func (node *AlgorandFullNode) SetCatchpointCatchupMode(catchpointCatchupMode boo node.ledgerService.Start() node.txHandler.Start() node.stateProofWorker.Start() + node.heartbeatService.Start() // Set up a context we can use to cancel goroutines on Stop() node.ctx, node.cancelCtx = context.WithCancel(context.Background()) diff --git a/stateproof/worker.go b/stateproof/worker.go index f74e118f58..163ec214e0 100644 --- a/stateproof/worker.go +++ b/stateproof/worker.go @@ -95,9 +95,7 @@ func NewWorker(genesisDir string, log logging.Logger, accts Accounts, ledger Led // Start starts the goroutines for the worker. func (spw *Worker) Start() { - ctx, cancel := context.WithCancel(context.Background()) - spw.ctx = ctx - spw.shutdown = cancel + spw.ctx, spw.shutdown = context.WithCancel(context.Background()) spw.signedCh = make(chan struct{}, 1) err := spw.initDb(spw.inMemory) diff --git a/test/e2e-go/features/incentives/challenge_test.go b/test/e2e-go/features/incentives/challenge_test.go new file mode 100644 index 0000000000..f45a1b6e2b --- /dev/null +++ b/test/e2e-go/features/incentives/challenge_test.go @@ -0,0 +1,173 @@ +// Copyright (C) 2019-2024 Algorand, Inc. +// This file is part of go-algorand +// +// go-algorand is free software: you can redistribute it and/or modify +// it under the terms of the GNU Affero General Public License as +// published by the Free Software Foundation, either version 3 of the +// License, or (at your option) any later version. +// +// go-algorand is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Affero General Public License for more details. +// +// You should have received a copy of the GNU Affero General Public License +// along with go-algorand. If not, see . + +package suspension + +import ( + "fmt" + "path/filepath" + "testing" + "time" + + "github.com/stretchr/testify/require" + + "github.com/algorand/go-algorand/config" + "github.com/algorand/go-algorand/daemon/algod/api/server/v2/generated/model" + "github.com/algorand/go-algorand/data/basics" + "github.com/algorand/go-algorand/libgoal" + "github.com/algorand/go-algorand/protocol" + "github.com/algorand/go-algorand/test/framework/fixtures" + "github.com/algorand/go-algorand/test/partitiontest" + "github.com/algorand/go-algorand/util" +) + +// TestChallenges ensures that accounts are knocked off if they don't respond to +// a challenge, and that algod responds for accounts it knows (keepign them online) +func TestChallenges(t *testing.T) { + partitiontest.PartitionTest(t) + defer fixtures.ShutdownSynchronizedTest(t) + + t.Parallel() + a := require.New(fixtures.SynchronizedTest(t)) + + // Overview of this test: + // Use a consensus protocol with challenge interval=50, grace period=10, bits=2. + // Start a three-node network. One relay, two nodes with 4 accounts each + // At round 50, ~2 nodes will be challenged. + + const interval = 50 + const grace = 10 + + var fixture fixtures.RestClientFixture + // Speed up rounds, keep lookback > 2 * grace period + fixture.FasterConsensus(protocol.ConsensusFuture, time.Second, 32) + fixture.AlterConsensus(protocol.ConsensusFuture, + func(cp config.ConsensusParams) config.ConsensusParams { + cp.Payouts.ChallengeInterval = 50 + cp.Payouts.ChallengeGracePeriod = 10 + cp.Payouts.ChallengeBits = 2 + return cp + }) + fixture.Setup(t, filepath.Join("nettemplates", "Challenges.json")) + defer fixture.Shutdown() + + clientAndAccounts := func(name string) (libgoal.Client, []model.Account) { + c := fixture.GetLibGoalClientForNamedNode(name) + accounts, err := fixture.GetNodeWalletsSortedByBalance(c) + a.NoError(err) + a.Len(accounts, 4) + fmt.Printf("Client %s has %v\n", name, accounts) + return c, accounts + } + + c1, accounts1 := clientAndAccounts("Node1") + c2, accounts2 := clientAndAccounts("Node2") + + // By re-regging, we become eligible for suspension (normal + challenges) + // TODO: Confirm that rereg is required for challenge suspensions + for _, account := range accounts1 { + rekeyreg(&fixture, a, c1, account.Address) + } + for _, account := range accounts2 { + rekeyreg(&fixture, a, c2, account.Address) + } + + // turn off node 1, so it can't heartbeat + a.NoError(c1.FullStop()) + + // Advance to first challenge round, check the blockseed + err := fixture.WaitForRoundWithTimeout(interval) + a.NoError(err) + + blk, err := c2.BookkeepingBlock(interval) + a.NoError(err) + challenge := blk.BlockHeader.Seed[0] & 0xA0 // high two bits + + challenged1 := util.MakeSet[model.Account]() + for _, account := range accounts1 { + abytes, err := basics.UnmarshalChecksumAddress(account.Address) + a.NoError(err) + if abytes[0]&0xA0 == challenge { + fmt.Printf("%v of node 1 was challenged %v by %v\n", account.Address, abytes[0], challenge) + challenged1.Add(account) + } + } + + challenged2 := util.MakeSet[model.Account]() + for _, account := range accounts2 { + abytes, err := basics.UnmarshalChecksumAddress(account.Address) + a.NoError(err) + if abytes[0]&0xA0 == challenge { + fmt.Printf("%v of node 2 was challenged %v by %v\n", account.Address, abytes[0], challenge) + challenged2.Add(account) + } + } + + allChallenged := util.Union(challenged1, challenged2) + + // TODO: unroll this loop and notice the heartbeat transactions from node 2 + err = fixture.WaitForRoundWithTimeout(interval + grace) + a.NoError(err) + + // challenged accounts are still online + for account := range allChallenged { + data, err := c2.AccountData(account.Address) + a.NoError(err) + a.Equal(basics.Online, data.Status) + a.NotZero(data.VoteID) + a.True(data.IncentiveEligible) + } + + err = fixture.WaitForRoundWithTimeout(interval + grace + 1) + a.NoError(err) + + // The challenged nodes need be "noticed" to be suspended. TODO: Remove this + // section when we have prompt suspensions. + source := accounts2[0] // always pay from operational account on node 2 + for account := range allChallenged { + fmt.Printf("pay %v\n", account.Address) + txn, err := c2.SendPaymentFromUnencryptedWallet(source.Address, account.Address, 1000, 0, nil) + a.NoError(err) + info, err := fixture.WaitForConfirmedTxn(uint64(txn.LastValid), txn.ID().String()) + a.NoError(err) + + blk, err := c2.BookkeepingBlock(*info.ConfirmedRound) + a.NoError(err) + a.Len(blk.AbsentParticipationAccounts, 1) + a.Equal(blk.AbsentParticipationAccounts[0].String(), account.Address) + } + + // node 1 challenged accounts are suspended because node 1 is off + for account := range challenged1 { + fmt.Printf("check1 %v\n", account.Address) + data, err := c2.AccountData(account.Address) + a.NoError(err) + a.Equal(basics.Offline, data.Status, account.Address) + a.NotZero(data.VoteID, account.Address) + a.False(data.IncentiveEligible, account.Address) // suspension turns off flag + } + + // node 2 challenged accounts are not suspended (saved by heartbeat) + for account := range challenged2 { + fmt.Printf("check2 %v\n", account.Address) + data, err := c2.AccountData(account.Address) + a.NoError(err) + a.Equal(basics.Online, data.Status, account.Address) + a.NotZero(data.VoteID, account.Address) + a.True(data.IncentiveEligible, account.Address) + } + +} diff --git a/test/framework/fixtures/libgoalFixture.go b/test/framework/fixtures/libgoalFixture.go index bd4f615ae7..0a88153989 100644 --- a/test/framework/fixtures/libgoalFixture.go +++ b/test/framework/fixtures/libgoalFixture.go @@ -67,6 +67,14 @@ func (f *RestClientFixture) SetConsensus(consensus config.ConsensusProtocols) { f.consensus = consensus } +func (f *RestClientFixture) AlterConsensus(ver protocol.ConsensusVersion, alter func(config.ConsensusParams) config.ConsensusParams) { + if f.consensus == nil { + f.consensus = make(config.ConsensusProtocols) + } + consensus := config.Consensus[ver] + f.consensus[ver] = alter(consensus) +} + // FasterConsensus speeds up the given consensus version in two ways. The seed // refresh lookback is set to 8 (instead of 80), so the 320 round balance // lookback becomes 32. And, if the architecture implies it can be handled, diff --git a/test/testdata/nettemplates/Challenges.json b/test/testdata/nettemplates/Challenges.json new file mode 100644 index 0000000000..6519033e9c --- /dev/null +++ b/test/testdata/nettemplates/Challenges.json @@ -0,0 +1,44 @@ +{ + "Genesis": { + "NetworkName": "tbd", + "ConsensusProtocol": "future", + "LastPartKeyRound": 500, + "Wallets": [ + { "Name": "Relay", "Stake": 92, "Online": true }, + { "Name": "Wallet0", "Stake": 1, "Online": true }, + { "Name": "Wallet1", "Stake": 1, "Online": true }, + { "Name": "Wallet2", "Stake": 1, "Online": true }, + { "Name": "Wallet3", "Stake": 1, "Online": true }, + { "Name": "Wallet4", "Stake": 1, "Online": true }, + { "Name": "Wallet5", "Stake": 1, "Online": true }, + { "Name": "Wallet6", "Stake": 1, "Online": true }, + { "Name": "Wallet7", "Stake": 1, "Online": true } + ], + "RewardsPoolBalance": 0 + }, + "Nodes": [ + { + "Name": "Relay", + "Wallets": [{ "Name": "Relay", "ParticipationOnly": false }], + "IsRelay": true + }, + { + "Name": "Node1", + "Wallets": [ + { "Name": "Wallet0", "ParticipationOnly": false }, + { "Name": "Wallet1", "ParticipationOnly": false }, + { "Name": "Wallet2", "ParticipationOnly": false }, + { "Name": "Wallet3", "ParticipationOnly": false } + ] + }, + { + "Name": "Node2", + "Wallets": [ + { "Name": "Wallet4", "ParticipationOnly": false }, + { "Name": "Wallet5", "ParticipationOnly": false }, + { "Name": "Wallet6", "ParticipationOnly": false }, + { "Name": "Wallet7", "ParticipationOnly": false } + ] + } + ] +} diff --git a/util/set.go b/util/set.go index 6851299c46..54ee92a050 100644 --- a/util/set.go +++ b/util/set.go @@ -40,3 +40,37 @@ func (s Set[T]) Contains(elem T) (exists bool) { _, exists = s[elem] return } + +// Union constructs a new set, containing all elements from the given sets. nil +// is never returned +func Union[T comparable](sets ...Set[T]) Set[T] { + union := make(Set[T]) + for _, set := range sets { + for elem := range set { + union.Add(elem) + } + } + return union +} + +// Intersection constructs a new set, containing all elements that appear in all +// given sets. nil is never returned. +func Intersection[T comparable](sets ...Set[T]) Set[T] { + var intersection = make(Set[T]) + if len(sets) == 0 { + return intersection + } + for elem := range sets[0] { + inAll := true + for _, set := range sets[1:] { + if _, exists := set[elem]; !exists { + inAll = false + break + } + } + if inAll { + intersection.Add(elem) + } + } + return intersection +} diff --git a/util/set_test.go b/util/set_test.go new file mode 100644 index 0000000000..86df9c5464 --- /dev/null +++ b/util/set_test.go @@ -0,0 +1,75 @@ +// Copyright (C) 2019-2024 Algorand, Inc. +// This file is part of go-algorand +// +// go-algorand is free software: you can redistribute it and/or modify +// it under the terms of the GNU Affero General Public License as +// published by the Free Software Foundation, either version 3 of the +// License, or (at your option) any later version. +// +// go-algorand is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Affero General Public License for more details. +// +// You should have received a copy of the GNU Affero General Public License +// along with go-algorand. If not, see . + +package util + +import ( + "testing" + + "github.com/algorand/go-algorand/test/partitiontest" + "github.com/stretchr/testify/require" +) + +func TestMakeSet(t *testing.T) { + partitiontest.PartitionTest(t) + t.Parallel() + + s := MakeSet(1, 2, 3) + require.True(t, s.Contains(1)) + require.True(t, s.Contains(2)) + require.True(t, s.Contains(3)) + require.False(t, s.Contains(4)) + + s = MakeSet[int]() + require.NotNil(t, s) + require.False(t, s.Contains(1)) + require.False(t, s.Contains(4)) +} + +func TestSetAdd(t *testing.T) { + partitiontest.PartitionTest(t) + t.Parallel() + + s := MakeSet[int]() + s.Add(6) + require.False(t, s.Contains(1)) + require.True(t, s.Contains(6)) + s.Add(6) + require.False(t, s.Contains(1)) + require.True(t, s.Contains(6)) +} + +func TestSetOps(t *testing.T) { + partitiontest.PartitionTest(t) + t.Parallel() + + empty := MakeSet[string]() + abc := MakeSet("a", "b", "c") + cde := MakeSet("c", "d", "e") + + require.Equal(t, abc, Union(abc)) + require.Equal(t, abc, Union(empty, abc)) + require.Equal(t, abc, Union(abc, empty, abc)) + require.NotNil(t, Union(empty, empty, empty)) + require.Equal(t, empty, Union(empty, empty, empty)) + + require.Equal(t, abc, Intersection(abc, abc)) + require.NotNil(t, Intersection(abc, empty)) + require.Equal(t, empty, Intersection(abc, empty)) + require.Equal(t, empty, Intersection(empty, abc)) + require.Equal(t, MakeSet("c"), Intersection(abc, cde)) + require.Equal(t, MakeSet("c"), Intersection(cde, abc, cde)) +} From 5191eb500e3b396cb9f1fa041c4c3f23cc6170e8 Mon Sep 17 00:00:00 2001 From: John Jannotti Date: Tue, 12 Nov 2024 10:52:38 -0500 Subject: [PATCH 08/19] Code review --- heartbeat/abstractions.go | 2 +- heartbeat/service.go | 5 +---- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/heartbeat/abstractions.go b/heartbeat/abstractions.go index 9ccecb6fb9..a03aa26b6e 100644 --- a/heartbeat/abstractions.go +++ b/heartbeat/abstractions.go @@ -46,7 +46,7 @@ type ledger interface { LookupAccount(round basics.Round, addr basics.Address) (data ledgercore.AccountData, validThrough basics.Round, withoutRewards basics.MicroAlgos, err error) } -// partipants captures the aspects of the AccountManager that are used by this +// participants captures the aspects of the AccountManager that are used by this // package. Service must be able to find out which accounts to monitor and have // access to their part keys to construct heartbeats. type participants interface { diff --git a/heartbeat/service.go b/heartbeat/service.go index 446a51c346..f13280b4e0 100644 --- a/heartbeat/service.go +++ b/heartbeat/service.go @@ -18,7 +18,6 @@ package heartbeat import ( "context" - "fmt" "sync" "github.com/algorand/go-algorand/config" @@ -89,16 +88,14 @@ func (s *Service) findChallenged(rules config.ProposerPayoutRules) []account.Par var found []account.ParticipationRecordForRound for _, pr := range s.accts.Keys(current + 1) { // only look at accounts we have part keys for acct, _, _, err := s.ledger.LookupAccount(current, pr.Account) - fmt.Printf(" %v is %s at %d\n", pr.Account, acct.Status, current) if err != nil { s.log.Errorf("error looking up %v: %v", pr.Account, err) continue } if acct.Status == basics.Online { lastSeen := max(acct.LastProposed, acct.LastHeartbeat) - fmt.Printf(" %v was last seen at %d\n", pr.Account, lastSeen) if ch.Failed(pr.Account, lastSeen) { - fmt.Printf(" %v needs a heartbeat\n", pr.Account) + s.log.Infof(" %v needs a heartbeat\n", pr.Account) found = append(found, pr) } } From 1f93f491603ee1df8b857601d8d2279f766cbef2 Mon Sep 17 00:00:00 2001 From: John Jannotti Date: Tue, 19 Nov 2024 12:58:50 -0500 Subject: [PATCH 09/19] Test heartbeat service, restrict free heartbeats --- agreement/gossip/networkFull_test.go | 2 +- catchup/universalFetcher.go | 2 +- cmd/goal/clerk.go | 3 +- cmd/loadgenerator/main.go | 11 +- daemon/algod/api/client/restClient.go | 90 +++++++++++-- data/transactions/verify/txn.go | 4 +- data/transactions/verify/txn_test.go | 4 +- .../verify/verifiedTxnCache_test.go | 2 +- heartbeat/service.go | 8 +- heartbeat/service_test.go | 8 +- ledger/apply/challenge.go | 116 +++++++++++++++++ ledger/apply/challenge_test.go | 121 +++++++++++++++++ ledger/apply/heartbeat.go | 28 +++- ledger/apply/heartbeat_test.go | 119 +++++++++++++++-- ledger/apply/mockBalances_test.go | 16 ++- ledger/eval/eval.go | 101 +-------------- ledger/eval/eval_test.go | 92 ------------- ledger/ledgercore/accountdata.go | 7 +- libgoal/libgoal.go | 51 +++----- network/connPerfMon_test.go | 4 +- .../features/accountPerf/sixMillion_test.go | 5 +- .../features/catchup/basicCatchup_test.go | 19 ++- .../catchup/catchpointCatchup_test.go | 18 ++- .../catchup/stateproofsCatchup_test.go | 12 +- .../features/followernode/syncDeltas_test.go | 6 +- .../features/followernode/syncRestart_test.go | 2 +- .../features/incentives/challenge_test.go | 121 +++++++++-------- .../onlineOfflineParticipation_test.go | 4 +- .../partitionRecovery_test.go | 9 +- test/e2e-go/restAPI/other/misc_test.go | 2 +- .../restAPI/simulate/simulateRestAPI_test.go | 6 +- .../upgrades/application_support_test.go | 4 +- test/e2e-go/upgrades/rekey_support_test.go | 2 +- test/framework/fixtures/libgoalFixture.go | 122 ++++-------------- test/framework/fixtures/restClientFixture.go | 92 ++++--------- util/db/dbutil.go | 2 +- 36 files changed, 675 insertions(+), 540 deletions(-) create mode 100644 ledger/apply/challenge.go create mode 100644 ledger/apply/challenge_test.go diff --git a/agreement/gossip/networkFull_test.go b/agreement/gossip/networkFull_test.go index 6507a6cb82..e64811d6a9 100644 --- a/agreement/gossip/networkFull_test.go +++ b/agreement/gossip/networkFull_test.go @@ -103,7 +103,7 @@ func spinNetwork(t *testing.T, nodesCount int, cfg config.Local) ([]*networkImpl break } } - log.Infof("network established, %d nodes connected in %s", nodesCount, time.Now().Sub(start).String()) + log.Infof("network established, %d nodes connected in %s", nodesCount, time.Since(start).String()) return networkImpls, msgCounters } diff --git a/catchup/universalFetcher.go b/catchup/universalFetcher.go index c7a8a9a4cf..fd99bcc612 100644 --- a/catchup/universalFetcher.go +++ b/catchup/universalFetcher.go @@ -88,7 +88,7 @@ func (uf *universalBlockFetcher) fetchBlock(ctx context.Context, round basics.Ro } else { return nil, nil, time.Duration(0), fmt.Errorf("fetchBlock: UniversalFetcher only supports HTTPPeer and UnicastPeer") } - downloadDuration = time.Now().Sub(blockDownloadStartTime) + downloadDuration = time.Since(blockDownloadStartTime) block, cert, err := processBlockBytes(fetchedBuf, round, address) if err != nil { return nil, nil, time.Duration(0), err diff --git a/cmd/goal/clerk.go b/cmd/goal/clerk.go index a69ed5be98..1a2495007d 100644 --- a/cmd/goal/clerk.go +++ b/cmd/goal/clerk.go @@ -221,8 +221,7 @@ func waitForCommit(client libgoal.Client, txid string, transactionLastValidRound } reportInfof(infoTxPending, txid, stat.LastRound) - // WaitForRound waits until round "stat.LastRound+1" is committed - stat, err = client.WaitForRound(stat.LastRound) + stat, err = client.WaitForRound(stat.LastRound + 1) if err != nil { return model.PendingTransactionResponse{}, fmt.Errorf(errorRequestFail, err) } diff --git a/cmd/loadgenerator/main.go b/cmd/loadgenerator/main.go index 6b82887695..df142de4ce 100644 --- a/cmd/loadgenerator/main.go +++ b/cmd/loadgenerator/main.go @@ -200,22 +200,23 @@ func waitForRound(restClient client.RestClient, cfg config, spendingRound bool) time.Sleep(1 * time.Second) continue } - if isSpendRound(cfg, nodeStatus.LastRound) == spendingRound { + lastRound := nodeStatus.LastRound + if isSpendRound(cfg, lastRound) == spendingRound { // time to send transactions. return } if spendingRound { - fmt.Printf("Last round %d, waiting for spending round %d\n", nodeStatus.LastRound, nextSpendRound(cfg, nodeStatus.LastRound)) + fmt.Printf("Last round %d, waiting for spending round %d\n", lastRound, nextSpendRound(cfg, nodeStatus.LastRound)) } for { // wait for the next round. - nodeStatus, err = restClient.WaitForBlock(basics.Round(nodeStatus.LastRound)) + err = restClient.WaitForRoundWithTimeout(lastRound + 1) if err != nil { fmt.Fprintf(os.Stderr, "unable to wait for next round node status : %v", err) - time.Sleep(1 * time.Second) break } - if isSpendRound(cfg, nodeStatus.LastRound) == spendingRound { + lastRound++ + if isSpendRound(cfg, lastRound) == spendingRound { // time to send transactions. return } diff --git a/daemon/algod/api/client/restClient.go b/daemon/algod/api/client/restClient.go index c349d3ecbf..e60ab1d36d 100644 --- a/daemon/algod/api/client/restClient.go +++ b/daemon/algod/api/client/restClient.go @@ -26,6 +26,7 @@ import ( "net/http" "net/url" "strings" + "time" "github.com/google/go-querystring/query" @@ -39,6 +40,8 @@ import ( "github.com/algorand/go-algorand/ledger/eval" "github.com/algorand/go-algorand/ledger/ledgercore" "github.com/algorand/go-algorand/protocol" + "github.com/algorand/go-algorand/rpcs" + "github.com/algorand/go-algorand/test/e2e-go/globals" ) const ( @@ -283,12 +286,77 @@ func (client RestClient) Status() (response model.NodeStatusResponse, err error) return } -// WaitForBlock returns the node status after waiting for the given round. -func (client RestClient) WaitForBlock(round basics.Round) (response model.NodeStatusResponse, err error) { +// WaitForBlockAfter returns the node status after trying to wait for the given +// round+1. This REST API has the documented misfeatures of returning after 1 +// minute, regardless of whether the given block has been reached. +func (client RestClient) WaitForBlockAfter(round basics.Round) (response model.NodeStatusResponse, err error) { err = client.get(&response, fmt.Sprintf("/v2/status/wait-for-block-after/%d/", round), nil) return } +// WaitForRound returns the node status after waiting for the given round. +func (client RestClient) WaitForRound(round uint64, waitTime time.Duration) (status model.NodeStatusResponse, err error) { + timeout := time.NewTimer(waitTime) + for { + status, err = client.Status() + if err != nil { + return + } + + if status.LastRound >= round { + return + } + select { + case <-timeout.C: + return model.NodeStatusResponse{}, fmt.Errorf("timeout waiting for round %v with last round = %v", round, status.LastRound) + case <-time.After(200 * time.Millisecond): + } + } +} + +const singleRoundMaxTime = globals.MaxTimePerRound * 40 + +// WaitForRoundWithTimeout waits for a given round to be reached. As it +// waits, it returns early with an error if the wait time for any round exceeds +// globals.MaxTimePerRound so we can alert when we're getting "hung" waiting. +func (client RestClient) WaitForRoundWithTimeout(roundToWaitFor uint64) error { + status, err := client.Status() + if err != nil { + return err + } + lastRound := status.LastRound + + // If node is already at or past target round, we're done + if lastRound >= roundToWaitFor { + return nil + } + + roundComplete := make(chan error, 2) + + for nextRound := lastRound + 1; lastRound < roundToWaitFor; nextRound++ { + roundStarted := time.Now() + + go func(done chan error) { + stat, err := client.WaitForRound(nextRound, singleRoundMaxTime) + lastRound = stat.LastRound + done <- err + }(roundComplete) + + select { + case lastError := <-roundComplete: + if lastError != nil { + close(roundComplete) + return lastError + } + case <-time.After(singleRoundMaxTime): + // we've timed out. + time := time.Since(roundStarted) + return fmt.Errorf("fixture.WaitForRound took %3.2f seconds between round %d and %d", time.Seconds(), lastRound, nextRound) + } + } + return nil +} + // HealthCheck does a health check on the potentially running node, // returning an error if the API is down func (client RestClient) HealthCheck() error { @@ -301,14 +369,6 @@ func (client RestClient) ReadyCheck() error { return client.get(nil, "/ready", nil) } -// StatusAfterBlock waits for a block to occur then returns the StatusResponse after that block -// blocks on the node end -// Not supported -func (client RestClient) StatusAfterBlock(blockNum uint64) (response model.NodeStatusResponse, err error) { - err = client.get(&response, fmt.Sprintf("/v2/status/wait-for-block-after/%d", blockNum), nil) - return -} - type pendingTransactionsParams struct { Max uint64 `url:"max"` Format string `url:"format"` @@ -557,6 +617,16 @@ func (client RestClient) RawBlock(round uint64) (response []byte, err error) { return } +// EncodedBlockCert takes a round and returns its parsed block and certificate +func (client RestClient) EncodedBlockCert(round uint64) (blockCert rpcs.EncodedBlockCert, err error) { + resp, err := client.RawBlock(round) + if err != nil { + return + } + err = protocol.Decode(resp, &blockCert) + return +} + // Shutdown requests the node to shut itself down func (client RestClient) Shutdown() (err error) { response := 1 diff --git a/data/transactions/verify/txn.go b/data/transactions/verify/txn.go index 893f29450e..518528a3bc 100644 --- a/data/transactions/verify/txn.go +++ b/data/transactions/verify/txn.go @@ -224,8 +224,8 @@ func txnGroupBatchPrep(stxs []transactions.SignedTxn, contextHdr *bookkeeping.Bl if stxn.Txn.Type == protocol.StateProofTx { continue } - if stxn.Txn.Type == protocol.HeartbeatTx && len(stxs) == 1 { - // TODO: Only allow free HB if the HbAddress is challenged + if stxn.Txn.Type == protocol.HeartbeatTx && stxn.Txn.Group.IsZero() { + // in apply.Heartbeat, we further confirm that the heartbeat is for a challenged node continue } minFeeCount++ diff --git a/data/transactions/verify/txn_test.go b/data/transactions/verify/txn_test.go index 282a031097..5946399d0b 100644 --- a/data/transactions/verify/txn_test.go +++ b/data/transactions/verify/txn_test.go @@ -575,7 +575,7 @@ func TestPaysetGroups(t *testing.T) { startPaysetGroupsTime := time.Now() err := PaysetGroups(context.Background(), txnGroups, blkHdr, verificationPool, MakeVerifiedTransactionCache(50000), nil) require.NoError(t, err) - paysetGroupDuration := time.Now().Sub(startPaysetGroupsTime) + paysetGroupDuration := time.Since(startPaysetGroupsTime) // break the signature and see if it fails. txnGroups[0][0].Sig[0] = txnGroups[0][0].Sig[0] + 1 @@ -609,7 +609,7 @@ func TestPaysetGroups(t *testing.T) { // channel is closed without a return require.Failf(t, "Channel got closed ?!", "") } else { - actualDuration := time.Now().Sub(startPaysetGroupsTime) + actualDuration := time.Since(startPaysetGroupsTime) if err == nil { if actualDuration > 4*time.Second { // it took at least 2.5 seconds more than it should have had! diff --git a/data/transactions/verify/verifiedTxnCache_test.go b/data/transactions/verify/verifiedTxnCache_test.go index d27510fe6a..03f5cac288 100644 --- a/data/transactions/verify/verifiedTxnCache_test.go +++ b/data/transactions/verify/verifiedTxnCache_test.go @@ -127,7 +127,7 @@ func BenchmarkGetUnverifiedTransactionGroups50(b *testing.B) { for i := 0; i < measuringMultipler; i++ { impl.GetUnverifiedTransactionGroups(queryTxnGroups, spec, protocol.ConsensusCurrentVersion) } - duration := time.Now().Sub(startTime) + duration := time.Since(startTime) // calculate time per 10K verified entries: t := int(duration*10000) / (measuringMultipler * b.N) b.ReportMetric(float64(t)/float64(time.Millisecond), "ms/10K_cache_compares") diff --git a/heartbeat/service.go b/heartbeat/service.go index f13280b4e0..d43fd196b1 100644 --- a/heartbeat/service.go +++ b/heartbeat/service.go @@ -26,7 +26,7 @@ import ( "github.com/algorand/go-algorand/data/bookkeeping" "github.com/algorand/go-algorand/data/transactions" "github.com/algorand/go-algorand/data/transactions/logic" - "github.com/algorand/go-algorand/ledger/eval" + "github.com/algorand/go-algorand/ledger/apply" "github.com/algorand/go-algorand/logging" "github.com/algorand/go-algorand/protocol" ) @@ -80,7 +80,7 @@ func (s *Service) Stop() { func (s *Service) findChallenged(rules config.ProposerPayoutRules) []account.ParticipationRecordForRound { current := s.ledger.LastRound() - ch := eval.FindChallenge(rules, current, s.ledger, eval.ChRisky) + ch := apply.FindChallenge(rules, current, s.ledger, apply.ChRisky) if ch.IsZero() { return nil } @@ -93,8 +93,7 @@ func (s *Service) findChallenged(rules config.ProposerPayoutRules) []account.Par continue } if acct.Status == basics.Online { - lastSeen := max(acct.LastProposed, acct.LastHeartbeat) - if ch.Failed(pr.Account, lastSeen) { + if ch.Failed(pr.Account, acct.LastSeen()) { s.log.Infof(" %v needs a heartbeat\n", pr.Account) found = append(found, pr) } @@ -135,6 +134,7 @@ func (s *Service) loop() { for _, pr := range s.findChallenged(proto.Payouts) { stxn := s.prepareHeartbeat(pr, lastHdr) + s.log.Infof("sending heartbeat %v for %v\n", stxn.Txn.HeartbeatTxnFields, pr.Account) err = s.bcast.BroadcastInternalSignedTxGroup([]transactions.SignedTxn{stxn}) if err != nil { s.log.Errorf("error broadcasting heartbeat %v for %v: %v", stxn, pr.Account, err) diff --git a/heartbeat/service_test.go b/heartbeat/service_test.go index 566645a5c4..3422ffdea4 100644 --- a/heartbeat/service_test.go +++ b/heartbeat/service_test.go @@ -210,7 +210,7 @@ func makeBlock(r basics.Round) bookkeeping.Block { } } -func TestHeartBeatOnlyWhenChallenged(t *testing.T) { +func TestHeartbeatOnlyWhenChallenged(t *testing.T) { partitiontest.PartitionTest(t) t.Parallel() @@ -234,9 +234,9 @@ func TestHeartBeatOnlyWhenChallenged(t *testing.T) { // now they are online, but not challenged, so no heartbeat acct.Status = basics.Online acct.VoteKeyDilution = 100 - otss := crypto.GenerateOneTimeSignatureSecrets( - basics.OneTimeIDForRound(ledger.LastRound(), acct.VoteKeyDilution).Batch, - 5) + startBatch := basics.OneTimeIDForRound(ledger.LastRound(), acct.VoteKeyDilution).Batch + const batches = 50 // gives 50 * kd rounds = 5000 + otss := crypto.GenerateOneTimeSignatureSecrets(startBatch, batches) acct.VoteID = otss.OneTimeSignatureVerifier ledger.addParticipant(joe, otss) ledger.addParticipant(mary, otss) diff --git a/ledger/apply/challenge.go b/ledger/apply/challenge.go new file mode 100644 index 0000000000..fa060879e6 --- /dev/null +++ b/ledger/apply/challenge.go @@ -0,0 +1,116 @@ +// Copyright (C) 2019-2024 Algorand, Inc. +// This file is part of go-algorand +// +// go-algorand is free software: you can redistribute it and/or modify +// it under the terms of the GNU Affero General Public License as +// published by the Free Software Foundation, either version 3 of the +// License, or (at your option) any later version. +// +// go-algorand is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Affero General Public License for more details. +// +// You should have received a copy of the GNU Affero General Public License +// along with go-algorand. If not, see . + +package apply + +import ( + "math/bits" + + "github.com/algorand/go-algorand/config" + "github.com/algorand/go-algorand/data/basics" + "github.com/algorand/go-algorand/data/bookkeeping" + "github.com/algorand/go-algorand/data/committee" +) + +// ChallengePeriod indicates which part of the challenge period is under discussion. +type ChallengePeriod int + +const ( + // ChRisky indicates that a challenge is in effect, and the initial grace period is running out. + ChRisky ChallengePeriod = iota + // ChActive indicates that a challenege is in effect, and the grace period + // has run out, so accounts can be suspended + ChActive +) + +type challenge struct { + // round is when the challenge occurred. 0 means this is not a challenge. + round basics.Round + // accounts that match the first `bits` of `seed` must propose or heartbeat to stay online + seed committee.Seed + bits int +} + +type headerSource interface { + BlockHdr(round basics.Round) (bookkeeping.BlockHeader, error) +} + +// FindChallenge returns the Challenge that was last issued if it's in the period requested. +func FindChallenge(rules config.ProposerPayoutRules, current basics.Round, headers headerSource, period ChallengePeriod) challenge { + // are challenges active? + interval := basics.Round(rules.ChallengeInterval) + if rules.ChallengeInterval == 0 || current < interval { + return challenge{} + } + lastChallenge := current - (current % interval) + grace := basics.Round(rules.ChallengeGracePeriod) + // FindChallenge is structured this way, instead of returning the challenge + // and letting the caller determine the period it cares about, to avoid + // using BlockHdr unnecessarily. + switch period { + case ChRisky: + if current <= lastChallenge+grace/2 || current > lastChallenge+grace { + return challenge{} + } + case ChActive: + if current <= lastChallenge+grace || current > lastChallenge+2*grace { + return challenge{} + } + } + challengeHdr, err := headers.BlockHdr(lastChallenge) + if err != nil { + panic(err) + } + challengeProto := config.Consensus[challengeHdr.CurrentProtocol] + // challenge is not considered if rules have changed since that round + if challengeProto.Payouts != rules { + return challenge{} + } + return challenge{lastChallenge, challengeHdr.Seed, rules.ChallengeBits} +} + +// IsZero returns true if the challenge is empty (used to indicate no challenege) +func (ch challenge) IsZero() bool { + return ch == challenge{} +} + +// Failed returns true iff ch is in effect, matches address, and lastSeen is +// before the challenge issue. +func (ch challenge) Failed(address basics.Address, lastSeen basics.Round) bool { + return ch.round != 0 && bitsMatch(ch.seed[:], address[:], ch.bits) && lastSeen < ch.round +} + +// bitsMatch checks if the first n bits of two byte slices match. Written to +// work on arbitrary slices, but we expect that n is small. Only user today +// calls with n=5. +func bitsMatch(a, b []byte, n int) bool { + // Ensure n is a valid number of bits to compare + if n < 0 || n > len(a)*8 || n > len(b)*8 { + return false + } + + // Compare entire bytes when n is bigger than 8 + for i := 0; i < n/8; i++ { + if a[i] != b[i] { + return false + } + } + remaining := n % 8 + if remaining == 0 { + return true + } + return bits.LeadingZeros8(a[n/8]^b[n/8]) >= remaining +} diff --git a/ledger/apply/challenge_test.go b/ledger/apply/challenge_test.go new file mode 100644 index 0000000000..3114b6f935 --- /dev/null +++ b/ledger/apply/challenge_test.go @@ -0,0 +1,121 @@ +// Copyright (C) 2019-2024 Algorand, Inc. +// This file is part of go-algorand +// +// go-algorand is free software: you can redistribute it and/or modify +// it under the terms of the GNU Affero General Public License as +// published by the Free Software Foundation, either version 3 of the +// License, or (at your option) any later version. +// +// go-algorand is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Affero General Public License for more details. +// +// You should have received a copy of the GNU Affero General Public License +// along with go-algorand. If not, see . + +package apply + +import ( + "testing" + + "github.com/algorand/go-algorand/config" + "github.com/algorand/go-algorand/data/basics" + "github.com/algorand/go-algorand/data/bookkeeping" + "github.com/algorand/go-algorand/protocol" + "github.com/algorand/go-algorand/test/partitiontest" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func TestBitsMatch(t *testing.T) { + partitiontest.PartitionTest(t) + t.Parallel() + + for b := 0; b <= 6; b++ { + require.True(t, bitsMatch([]byte{0x1}, []byte{0x2}, b), "%d", b) + } + require.False(t, bitsMatch([]byte{0x1}, []byte{0x2}, 7)) + require.False(t, bitsMatch([]byte{0x1}, []byte{0x2}, 8)) + require.False(t, bitsMatch([]byte{0x1}, []byte{0x2}, 9)) + + for b := 0; b <= 12; b++ { + require.True(t, bitsMatch([]byte{0x1, 0xff, 0xaa}, []byte{0x1, 0xf0}, b), "%d", b) + } + require.False(t, bitsMatch([]byte{0x1, 0xff, 0xaa}, []byte{0x1, 0xf0}, 13)) + + // on a byte boundary + require.True(t, bitsMatch([]byte{0x1}, []byte{0x1}, 8)) + require.False(t, bitsMatch([]byte{0x1}, []byte{0x1}, 9)) + require.True(t, bitsMatch([]byte{0x1, 0xff}, []byte{0x1, 0x00}, 8)) + require.False(t, bitsMatch([]byte{0x1, 0xff}, []byte{0x1, 00}, 9)) +} + +func TestFailsChallenge(t *testing.T) { + partitiontest.PartitionTest(t) + t.Parallel() + a := assert.New(t) + + // a valid challenge, with 4 matching bits, and an old last seen + a.True(challenge{round: 11, seed: [32]byte{0xb0, 0xb4}, bits: 4}.Failed(basics.Address{0xbf, 0x34}, 10)) + + // challenge isn't "on" + a.False(challenge{round: 0, seed: [32]byte{0xb0, 0xb4}, bits: 4}.Failed(basics.Address{0xbf, 0x34}, 10)) + // node has appeared more recently + a.False(challenge{round: 11, seed: [32]byte{0xb0, 0xb4}, bits: 4}.Failed(basics.Address{0xbf, 0x34}, 12)) + // bits don't match + a.False(challenge{round: 11, seed: [32]byte{0xb0, 0xb4}, bits: 4}.Failed(basics.Address{0xcf, 0x34}, 10)) + // no enough bits match + a.False(challenge{round: 11, seed: [32]byte{0xb0, 0xb4}, bits: 5}.Failed(basics.Address{0xbf, 0x34}, 10)) +} + +type singleSource bookkeeping.BlockHeader + +func (ss singleSource) BlockHdr(r basics.Round) (bookkeeping.BlockHeader, error) { + return bookkeeping.BlockHeader(ss), nil +} + +func TestActiveChallenge(t *testing.T) { + partitiontest.PartitionTest(t) + t.Parallel() + a := assert.New(t) + + nowHeader := bookkeeping.BlockHeader{ + UpgradeState: bookkeeping.UpgradeState{ + // Here the rules are on, so they certainly differ from rules in oldHeader's params + CurrentProtocol: protocol.ConsensusFuture, + }, + } + rules := config.Consensus[nowHeader.CurrentProtocol].Payouts + + // simplest test. when interval=X and grace=G, X+G+1 is a challenge + inChallenge := basics.Round(rules.ChallengeInterval + rules.ChallengeGracePeriod + 1) + ch := FindChallenge(rules, inChallenge, singleSource(nowHeader), ChActive) + a.NotZero(ch.round) + + // all rounds before that have no challenge + for r := basics.Round(1); r < inChallenge; r++ { + ch := FindChallenge(rules, r, singleSource(nowHeader), ChActive) + a.Zero(ch.round, r) + } + + // ChallengeGracePeriod rounds allow challenges starting with inChallenge + for r := inChallenge; r < inChallenge+basics.Round(rules.ChallengeGracePeriod); r++ { + ch := FindChallenge(rules, r, singleSource(nowHeader), ChActive) + a.EqualValues(ch.round, rules.ChallengeInterval) + } + + // And the next round is again challenge-less + ch = FindChallenge(rules, inChallenge+basics.Round(rules.ChallengeGracePeriod), singleSource(nowHeader), ChActive) + a.Zero(ch.round) + + // ignore challenge if upgrade happened + oldHeader := bookkeeping.BlockHeader{ + UpgradeState: bookkeeping.UpgradeState{ + // We need a version from before payouts got turned on + CurrentProtocol: protocol.ConsensusV39, + }, + } + ch = FindChallenge(rules, inChallenge, singleSource(oldHeader), ChActive) + a.Zero(ch.round) +} diff --git a/ledger/apply/heartbeat.go b/ledger/apply/heartbeat.go index 4e858f9990..afb9af257a 100644 --- a/ledger/apply/heartbeat.go +++ b/ledger/apply/heartbeat.go @@ -31,6 +31,32 @@ func Heartbeat(hb transactions.HeartbeatTxnFields, header transactions.Header, b return err } + // In txnGroupBatchPrep, we do not charge for singleton (Group.IsZero) + // heartbeats. But we only _want_ to allow free heartbeats if the account is + // under challenge. If this is an underpaid singleton heartbeat, reject it + // unless the account is under challenge. + + proto := balances.ConsensusParams() + if header.Fee.Raw < proto.MinTxnFee && header.Group.IsZero() { + kind := "free" + if header.Fee.Raw > 0 { + kind = "cheap" + } + if account.Status != basics.Online { + return fmt.Errorf("%s heartbeat is not allowed for %s %+v", kind, account.Status, hb.HbAddress) + } + if !account.IncentiveEligible { + return fmt.Errorf("%s heartbeat is not allowed for ineligible %+v", kind, hb.HbAddress) + } + ch := FindChallenge(proto.Payouts, round, provider, ChRisky) + if ch.round == 0 { + return fmt.Errorf("%s heartbeat for %s is not allowed with no challenge", kind, hb.HbAddress) + } + if !ch.Failed(hb.HbAddress, account.LastSeen()) { + return fmt.Errorf("%s heartbeat for %s is not challenged by %+v", kind, hb.HbAddress, ch) + } + } + // Note the contrast with agreement. We are using the account's _current_ // partkey to verify the heartbeat. This is required because we can only // look 320 rounds back for voting information. If a heartbeat was delayed a @@ -53,7 +79,7 @@ func Heartbeat(hb transactions.HeartbeatTxnFields, header transactions.Header, b if sv.IsEmpty() { return fmt.Errorf("heartbeat address %s has no voting keys", hb.HbAddress) } - kd := oad.KeyDilution(balances.ConsensusParams()) + kd := oad.KeyDilution(proto) // heartbeats are expected to sign with the partkey for their last-valid round id := basics.OneTimeIDForRound(header.LastValid, kd) diff --git a/ledger/apply/heartbeat_test.go b/ledger/apply/heartbeat_test.go index 42f9cd4732..f8607d915c 100644 --- a/ledger/apply/heartbeat_test.go +++ b/ledger/apply/heartbeat_test.go @@ -17,15 +17,17 @@ package apply import ( + "fmt" "testing" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/algorand/go-algorand/crypto" "github.com/algorand/go-algorand/data/basics" "github.com/algorand/go-algorand/data/bookkeeping" "github.com/algorand/go-algorand/data/committee" - "github.com/algorand/go-algorand/data/transactions" + "github.com/algorand/go-algorand/data/txntest" "github.com/algorand/go-algorand/protocol" "github.com/algorand/go-algorand/test/partitiontest" ) @@ -64,23 +66,33 @@ func TestHeartbeat(t *testing.T) { Seed: seed, }) - tx := transactions.Transaction{ - Type: protocol.HeartbeatTx, - Header: transactions.Header{ - Sender: sender, - Fee: basics.MicroAlgos{Raw: 1}, - FirstValid: fv, - LastValid: lv, - }, - HeartbeatTxnFields: transactions.HeartbeatTxnFields{ - HbAddress: voter, - HbProof: otss.Sign(id, seed), - HbSeed: seed, - }, + test := txntest.Txn{ + Type: protocol.HeartbeatTx, + Sender: sender, + Fee: basics.MicroAlgos{Raw: 1}, + FirstValid: fv, + LastValid: lv, + HbAddress: voter, + HbProof: otss.Sign(id, seed), + HbSeed: seed, } + tx := test.Txn() + rnd := basics.Round(150) + // no fee err := Heartbeat(tx.HeartbeatTxnFields, tx.Header, mockBal, mockHdr, rnd) + require.ErrorContains(t, err, "cheap heartbeat") + + test.Fee = basics.MicroAlgos{Raw: 10} + tx = test.Txn() + // just as bad: cheap + err = Heartbeat(tx.HeartbeatTxnFields, tx.Header, mockBal, mockHdr, rnd) + require.ErrorContains(t, err, "cheap heartbeat") + + test.Fee = 1000 + tx = test.Txn() + err = Heartbeat(tx.HeartbeatTxnFields, tx.Header, mockBal, mockHdr, rnd) require.NoError(t, err) after, err := mockBal.Get(voter, false) @@ -88,3 +100,82 @@ func TestHeartbeat(t *testing.T) { require.Equal(t, rnd, after.LastHeartbeat) require.Zero(t, after.LastProposed) // unchanged } + +// TestCheapRules ensures a heartbeat can only have a low fee if the account +// being heartbeat for is online, under risk of suspension by challenge, and +// incentive eligible. +func TestCheapRules(t *testing.T) { + type tcase struct { + rnd basics.Round + addrStart byte + status basics.Status + incentiveEligble bool + err string + } + + // Grace period is 200. For the second half of the grace period (1101-1200), + // the heartbeat is free for online, incentive eligible, challenged accounts. + cases := []tcase{ + // test of range + {1100, 0x01, basics.Online, true, "no challenge"}, + {1101, 0x01, basics.Online, true, ""}, + {1200, 0x01, basics.Online, true, ""}, + {1201, 0x01, basics.Online, true, "no challenge"}, + + // test of the other requirements + {1101, 0xf1, basics.Online, true, "not challenged by"}, + {1101, 0x01, basics.Offline, true, "not allowed for Offline"}, + {1101, 0x01, basics.Online, false, "not allowed for ineligible"}, + } + for _, tc := range cases { + const keyDilution = 777 + + lv := basics.Round(tc.rnd + 10) + + id := basics.OneTimeIDForRound(lv, keyDilution) + otss := crypto.GenerateOneTimeSignatureSecrets(1, 10) // This will cover rounds 1-10*777 + + sender := basics.Address{0x01} + voter := basics.Address{tc.addrStart} + mockBal := makeMockBalancesWithAccounts(protocol.ConsensusFuture, map[basics.Address]basics.AccountData{ + sender: { + MicroAlgos: basics.MicroAlgos{Raw: 10_000_000}, + }, + voter: { + Status: tc.status, + MicroAlgos: basics.MicroAlgos{Raw: 100_000_000}, + VoteID: otss.OneTimeSignatureVerifier, + VoteKeyDilution: keyDilution, + IncentiveEligible: tc.incentiveEligble, + }, + }) + + seed := committee.Seed{0x01, 0x02, 0x03} + mockHdr := makeMockHeaders() + mockHdr.setFallback(bookkeeping.BlockHeader{ + UpgradeState: bookkeeping.UpgradeState{ + CurrentProtocol: protocol.ConsensusFuture, + }, + Seed: seed, + }) + txn := txntest.Txn{ + Type: protocol.HeartbeatTx, + Sender: sender, + Fee: basics.MicroAlgos{Raw: 1}, + FirstValid: tc.rnd - 10, + LastValid: tc.rnd + 10, + HbAddress: voter, + HbProof: otss.Sign(id, seed), + HbSeed: seed, + } + + tx := txn.Txn() + fmt.Printf("tc %+v\n", tc) + err := Heartbeat(tx.HeartbeatTxnFields, tx.Header, mockBal, mockHdr, tc.rnd) + if tc.err == "" { + assert.NoError(t, err) + } else { + assert.ErrorContains(t, err, tc.err, "%+v", tc) + } + } +} diff --git a/ledger/apply/mockBalances_test.go b/ledger/apply/mockBalances_test.go index a5f636fc08..a18500341e 100644 --- a/ledger/apply/mockBalances_test.go +++ b/ledger/apply/mockBalances_test.go @@ -275,7 +275,8 @@ func (b *mockCreatableBalances) HasAssetParams(addr basics.Address, aidx basics. } type mockHeaders struct { - b map[basics.Round]bookkeeping.BlockHeader + perRound map[basics.Round]bookkeeping.BlockHeader + fallback *bookkeeping.BlockHeader } // makeMockHeaders takes a bunch of BlockHeaders and returns a HdrProivder for them. @@ -284,12 +285,21 @@ func makeMockHeaders(hdrs ...bookkeeping.BlockHeader) mockHeaders { for _, hdr := range hdrs { b[hdr.Round] = hdr } - return mockHeaders{b: b} + return mockHeaders{perRound: b} } func (m mockHeaders) BlockHdr(r basics.Round) (bookkeeping.BlockHeader, error) { - if hdr, ok := m.b[r]; ok { + if hdr, ok := m.perRound[r]; ok { return hdr, nil } + if m.fallback != nil { + copy := *m.fallback + copy.Round = r + return copy, nil + } return bookkeeping.BlockHeader{}, fmt.Errorf("round %v is not present", r) } + +func (m *mockHeaders) setFallback(hdr bookkeeping.BlockHeader) { + m.fallback = &hdr +} diff --git a/ledger/eval/eval.go b/ledger/eval/eval.go index f98ed4cb5d..2e5c8f00a3 100644 --- a/ledger/eval/eval.go +++ b/ledger/eval/eval.go @@ -21,7 +21,6 @@ import ( "errors" "fmt" "math" - "math/bits" "sync" "github.com/algorand/go-algorand/agreement" @@ -29,7 +28,6 @@ import ( "github.com/algorand/go-algorand/crypto" "github.com/algorand/go-algorand/data/basics" "github.com/algorand/go-algorand/data/bookkeeping" - "github.com/algorand/go-algorand/data/committee" "github.com/algorand/go-algorand/data/transactions" "github.com/algorand/go-algorand/data/transactions/logic" "github.com/algorand/go-algorand/data/transactions/verify" @@ -1609,14 +1607,6 @@ func (eval *BlockEvaluator) proposerPayout() (basics.MicroAlgos, error) { return basics.MinA(total, available), nil } -type challenge struct { - // round is when the challenge occurred. 0 means this is not a challenge. - round basics.Round - // accounts that match the first `bits` of `seed` must propose or heartbeat to stay online - seed committee.Seed - bits int -} - // generateKnockOfflineAccountsList creates the lists of expired or absent // participation accounts by traversing over the modified accounts in the state // deltas and testing if any of them needs to be reset/suspended. Expiration @@ -1641,7 +1631,7 @@ func (eval *BlockEvaluator) generateKnockOfflineAccountsList(participating []bas updates := &eval.block.ParticipationUpdates - ch := FindChallenge(eval.proto.Payouts, current, eval.state, ChActive) + ch := apply.FindChallenge(eval.proto.Payouts, current, eval.state, apply.ChActive) onlineStake, err := eval.state.onlineStake() if err != nil { logging.Base().Errorf("unable to fetch online stake, no knockoffs: %v", err) @@ -1754,28 +1744,6 @@ func (eval *BlockEvaluator) generateKnockOfflineAccountsList(participating []bas } } -// bitsMatch checks if the first n bits of two byte slices match. Written to -// work on arbitrary slices, but we expect that n is small. Only user today -// calls with n=5. -func bitsMatch(a, b []byte, n int) bool { - // Ensure n is a valid number of bits to compare - if n < 0 || n > len(a)*8 || n > len(b)*8 { - return false - } - - // Compare entire bytes when n is bigger than 8 - for i := 0; i < n/8; i++ { - if a[i] != b[i] { - return false - } - } - remaining := n % 8 - if remaining == 0 { - return true - } - return bits.LeadingZeros8(a[n/8]^b[n/8]) >= remaining -} - func isAbsent(totalOnlineStake basics.MicroAlgos, acctStake basics.MicroAlgos, lastSeen basics.Round, current basics.Round) bool { // Don't consider accounts that were online when payouts went into effect as // absent. They get noticed the next time they propose or keyreg, which @@ -1793,66 +1761,6 @@ func isAbsent(totalOnlineStake basics.MicroAlgos, acctStake basics.MicroAlgos, l return lastSeen+basics.Round(allowableLag) < current } -type headerSource interface { - BlockHdr(round basics.Round) (bookkeeping.BlockHeader, error) -} - -// ChallengePeriod indicates which part of the challenge period is under discussion. -type ChallengePeriod int - -const ( - // ChRisky indicates that a challenge is in effect, and the initial grace period is running out. - ChRisky ChallengePeriod = iota - // ChActive indicates that a challenege is in effect, and the grace period - // has run out, so accounts can be suspended - ChActive -) - -// FindChallenge returns the Challenge that was last issued if it's in the period requested. -func FindChallenge(rules config.ProposerPayoutRules, current basics.Round, headers headerSource, period ChallengePeriod) challenge { - // are challenges active? - interval := basics.Round(rules.ChallengeInterval) - if rules.ChallengeInterval == 0 || current < interval { - return challenge{} - } - lastChallenge := current - (current % interval) - grace := basics.Round(rules.ChallengeGracePeriod) - // FindChallenge is structured this way, instead of returning the challenge - // and letting the caller determine the period it cares about, to avoid - // using BlockHdr unnecessarily. - switch period { - case ChRisky: - if current <= lastChallenge+grace/2 || current > lastChallenge+grace { - return challenge{} - } - case ChActive: - if current <= lastChallenge+grace || current > lastChallenge+2*grace { - return challenge{} - } - } - challengeHdr, err := headers.BlockHdr(lastChallenge) - if err != nil { - panic(err) - } - challengeProto := config.Consensus[challengeHdr.CurrentProtocol] - // challenge is not considered if rules have changed since that round - if challengeProto.Payouts != rules { - return challenge{} - } - return challenge{lastChallenge, challengeHdr.Seed, rules.ChallengeBits} -} - -// IsZero returns true if the challenge is empty (used to indicate no challenege) -func (ch challenge) IsZero() bool { - return ch == challenge{} -} - -// Failed returns true iff ch is in effect, matches address, and lastSeen is -// before the challenge issue. -func (ch challenge) Failed(address basics.Address, lastSeen basics.Round) bool { - return ch.round != 0 && bitsMatch(ch.seed[:], address[:], ch.bits) && lastSeen < ch.round -} - // validateExpiredOnlineAccounts tests the expired online accounts specified in ExpiredParticipationAccounts, and verify // that they have all expired and need to be reset. func (eval *BlockEvaluator) validateExpiredOnlineAccounts() error { @@ -1920,7 +1828,7 @@ func (eval *BlockEvaluator) validateAbsentOnlineAccounts() error { // For consistency with expired account handling, we preclude duplicates addressSet := make(map[basics.Address]bool, suspensionCount) - ch := FindChallenge(eval.proto.Payouts, eval.Round(), eval.state, ChActive) + ch := apply.FindChallenge(eval.proto.Payouts, eval.Round(), eval.state, apply.ChActive) totalOnlineStake, err := eval.state.onlineStake() if err != nil { logging.Base().Errorf("unable to fetch online stake, can't check knockoffs: %v", err) @@ -1948,15 +1856,14 @@ func (eval *BlockEvaluator) validateAbsentOnlineAccounts() error { return fmt.Errorf("proposed absent account %v with zero algos", accountAddr) } - lastSeen := max(acctData.LastProposed, acctData.LastHeartbeat) oad, lErr := eval.state.lookupAgreement(accountAddr) if lErr != nil { return fmt.Errorf("unable to check absent account: %v", accountAddr) } - if isAbsent(totalOnlineStake, oad.VotingStake(), lastSeen, eval.Round()) { + if isAbsent(totalOnlineStake, oad.VotingStake(), acctData.LastSeen(), eval.Round()) { continue // ok. it's "normal absent" } - if ch.Failed(accountAddr, lastSeen) { + if ch.Failed(accountAddr, acctData.LastSeen()) { continue // ok. it's "challenge absent" } return fmt.Errorf("proposed absent account %v is not absent in %d, %d", diff --git a/ledger/eval/eval_test.go b/ledger/eval/eval_test.go index 8c8e2cf233..d33996b2d8 100644 --- a/ledger/eval/eval_test.go +++ b/ledger/eval/eval_test.go @@ -1605,29 +1605,6 @@ func TestExpiredAccountGeneration(t *testing.T) { require.NotZero(t, propAcct.StateProofID) } -func TestBitsMatch(t *testing.T) { - partitiontest.PartitionTest(t) - t.Parallel() - - for b := 0; b <= 6; b++ { - require.True(t, bitsMatch([]byte{0x1}, []byte{0x2}, b), "%d", b) - } - require.False(t, bitsMatch([]byte{0x1}, []byte{0x2}, 7)) - require.False(t, bitsMatch([]byte{0x1}, []byte{0x2}, 8)) - require.False(t, bitsMatch([]byte{0x1}, []byte{0x2}, 9)) - - for b := 0; b <= 12; b++ { - require.True(t, bitsMatch([]byte{0x1, 0xff, 0xaa}, []byte{0x1, 0xf0}, b), "%d", b) - } - require.False(t, bitsMatch([]byte{0x1, 0xff, 0xaa}, []byte{0x1, 0xf0}, 13)) - - // on a byte boundary - require.True(t, bitsMatch([]byte{0x1}, []byte{0x1}, 8)) - require.False(t, bitsMatch([]byte{0x1}, []byte{0x1}, 9)) - require.True(t, bitsMatch([]byte{0x1, 0xff}, []byte{0x1, 0x00}, 8)) - require.False(t, bitsMatch([]byte{0x1, 0xff}, []byte{0x1, 00}, 9)) -} - func TestIsAbsent(t *testing.T) { partitiontest.PartitionTest(t) t.Parallel() @@ -1646,72 +1623,3 @@ func TestIsAbsent(t *testing.T) { a.False(absent(1000, 10, 0, 6000)) a.False(absent(1000, 10, 0, 6001)) } - -func TestFailsChallenge(t *testing.T) { - partitiontest.PartitionTest(t) - t.Parallel() - a := assert.New(t) - - // a valid challenge, with 4 matching bits, and an old last seen - a.True(challenge{round: 11, seed: [32]byte{0xb0, 0xb4}, bits: 4}.Failed(basics.Address{0xbf, 0x34}, 10)) - - // challenge isn't "on" - a.False(challenge{round: 0, seed: [32]byte{0xb0, 0xb4}, bits: 4}.Failed(basics.Address{0xbf, 0x34}, 10)) - // node has appeared more recently - a.False(challenge{round: 11, seed: [32]byte{0xb0, 0xb4}, bits: 4}.Failed(basics.Address{0xbf, 0x34}, 12)) - // bits don't match - a.False(challenge{round: 11, seed: [32]byte{0xb0, 0xb4}, bits: 4}.Failed(basics.Address{0xcf, 0x34}, 10)) - // no enough bits match - a.False(challenge{round: 11, seed: [32]byte{0xb0, 0xb4}, bits: 5}.Failed(basics.Address{0xbf, 0x34}, 10)) -} - -type singleSource bookkeeping.BlockHeader - -func (ss singleSource) BlockHdr(r basics.Round) (bookkeeping.BlockHeader, error) { - return bookkeeping.BlockHeader(ss), nil -} - -func TestActiveChallenge(t *testing.T) { - partitiontest.PartitionTest(t) - t.Parallel() - a := assert.New(t) - - nowHeader := bookkeeping.BlockHeader{ - UpgradeState: bookkeeping.UpgradeState{ - // Here the rules are on, so they certainly differ from rules in oldHeader's params - CurrentProtocol: protocol.ConsensusFuture, - }, - } - rules := config.Consensus[nowHeader.CurrentProtocol].Payouts - - // simplest test. when interval=X and grace=G, X+G+1 is a challenge - inChallenge := basics.Round(rules.ChallengeInterval + rules.ChallengeGracePeriod + 1) - ch := FindChallenge(rules, inChallenge, singleSource(nowHeader), ChActive) - a.NotZero(ch.round) - - // all rounds before that have no challenge - for r := basics.Round(1); r < inChallenge; r++ { - ch := FindChallenge(rules, r, singleSource(nowHeader), ChActive) - a.Zero(ch.round, r) - } - - // ChallengeGracePeriod rounds allow challenges starting with inChallenge - for r := inChallenge; r < inChallenge+basics.Round(rules.ChallengeGracePeriod); r++ { - ch := FindChallenge(rules, r, singleSource(nowHeader), ChActive) - a.EqualValues(ch.round, rules.ChallengeInterval) - } - - // And the next round is again challenge-less - ch = FindChallenge(rules, inChallenge+basics.Round(rules.ChallengeGracePeriod), singleSource(nowHeader), ChActive) - a.Zero(ch.round) - - // ignore challenge if upgrade happened - oldHeader := bookkeeping.BlockHeader{ - UpgradeState: bookkeeping.UpgradeState{ - // We need a version from before payouts got turned on - CurrentProtocol: protocol.ConsensusV39, - }, - } - ch = FindChallenge(rules, inChallenge, singleSource(oldHeader), ChActive) - a.Zero(ch.round) -} diff --git a/ledger/ledgercore/accountdata.go b/ledger/ledgercore/accountdata.go index 5b17730122..ea7b150a6e 100644 --- a/ledger/ledgercore/accountdata.go +++ b/ledger/ledgercore/accountdata.go @@ -135,10 +135,15 @@ func (u *AccountData) Suspend() { } // Suspended returns true if the account is suspended (offline with keys) -func (u *AccountData) Suspended() bool { +func (u AccountData) Suspended() bool { return u.Status == basics.Offline && !u.VoteID.IsEmpty() } +// LastSeen returns the last round that the account was seen online +func (u AccountData) LastSeen() basics.Round { + return max(u.LastProposed, u.LastHeartbeat) +} + // MinBalance computes the minimum balance requirements for an account based on // some consensus parameters. MinBalance should correspond roughly to how much // storage the account is allowed to store on disk. diff --git a/libgoal/libgoal.go b/libgoal/libgoal.go index f3f1c67192..e7739e085c 100644 --- a/libgoal/libgoal.go +++ b/libgoal/libgoal.go @@ -28,7 +28,6 @@ import ( v2 "github.com/algorand/go-algorand/daemon/algod/api/server/v2" kmdclient "github.com/algorand/go-algorand/daemon/kmd/client" "github.com/algorand/go-algorand/ledger/ledgercore" - "github.com/algorand/go-algorand/rpcs" "github.com/algorand/go-algorand/config" "github.com/algorand/go-algorand/crypto" @@ -831,53 +830,43 @@ func (c *Client) Block(round uint64) (resp v2.BlockResponseJSON, err error) { // RawBlock takes a round and returns its block func (c *Client) RawBlock(round uint64) (resp []byte, err error) { algod, err := c.ensureAlgodClient() - if err == nil { - resp, err = algod.RawBlock(round) - } - return -} - -// EncodedBlockCert takes a round and returns its parsed block and certificate -func (c *Client) EncodedBlockCert(round uint64) (blockCert rpcs.EncodedBlockCert, err error) { - algod, err := c.ensureAlgodClient() - if err == nil { - var resp []byte - resp, err = algod.RawBlock(round) - if err == nil { - err = protocol.Decode(resp, &blockCert) - if err != nil { - return - } - } + if err != nil { + return } - return + return algod.RawBlock(round) } // BookkeepingBlock takes a round and returns its block func (c *Client) BookkeepingBlock(round uint64) (block bookkeeping.Block, err error) { - blockCert, err := c.EncodedBlockCert(round) - if err == nil { - return blockCert.Block, nil + algod, err := c.ensureAlgodClient() + if err != nil { + return } - return + blockCert, err := algod.EncodedBlockCert(round) + if err != nil { + return + } + return blockCert.Block, nil } // HealthCheck returns an error if something is wrong func (c *Client) HealthCheck() error { algod, err := c.ensureAlgodClient() - if err == nil { - err = algod.HealthCheck() + if err != nil { + return err } - return err + return algod.HealthCheck() } -// WaitForRound takes a round, waits until it appears and returns its status. This function blocks. +// WaitForRound takes a round, waits up to one minute, for it to appear and +// returns the node status. This function blocks and fails if the block does not +// appear in one minute. func (c *Client) WaitForRound(round uint64) (resp model.NodeStatusResponse, err error) { algod, err := c.ensureAlgodClient() - if err == nil { - resp, err = algod.StatusAfterBlock(round) + if err != nil { + return } - return + return algod.WaitForRound(round, time.Minute) } // GetBalance takes an address and returns its total balance; if the address doesn't exist, it returns 0. diff --git a/network/connPerfMon_test.go b/network/connPerfMon_test.go index 560be72a96..4c2bc5f034 100644 --- a/network/connPerfMon_test.go +++ b/network/connPerfMon_test.go @@ -103,14 +103,14 @@ func TestConnMonitorStageTiming(t *testing.T) { startTestTime := time.Now().UnixNano() perfMonitor := makeConnectionPerformanceMonitor([]Tag{protocol.AgreementVoteTag}) // measure measuring overhead. - measuringOverhead := time.Now().Sub(time.Now()) + measuringOverhead := time.Since(time.Now()) perfMonitor.Reset(peers) for msgIdx, msg := range msgPool { msg.Received += startTestTime beforeNotify := time.Now() beforeNotifyStage := perfMonitor.stage perfMonitor.Notify(&msg) - notifyTime := time.Now().Sub(beforeNotify) + notifyTime := time.Since(beforeNotify) stageTimings[beforeNotifyStage] += notifyTime stageNotifyCalls[beforeNotifyStage]++ if perfMonitor.GetPeersStatistics() != nil { diff --git a/test/e2e-go/features/accountPerf/sixMillion_test.go b/test/e2e-go/features/accountPerf/sixMillion_test.go index 946d1b24b6..94feb3e9eb 100644 --- a/test/e2e-go/features/accountPerf/sixMillion_test.go +++ b/test/e2e-go/features/accountPerf/sixMillion_test.go @@ -1024,13 +1024,10 @@ func checkPoint(counter, firstValid, tLife uint64, force bool, fixture *fixtures if verbose { fmt.Printf("Waiting for round %d...", int(lastRound)) } - nodeStat, err := fixture.AlgodClient.WaitForBlock(basics.Round(lastRound - 1)) + nodeStat, err := fixture.AlgodClient.WaitForRound(lastRound, time.Minute) if err != nil { return 0, 0, fmt.Errorf("failed to wait for block %d : %w", lastRound, err) } - if nodeStat.LastRound < lastRound { - return 0, 0, fmt.Errorf("failed to wait for block %d : node is at round %d", lastRound, nodeStat.LastRound) - } return 0, nodeStat.LastRound + 1, nil } return counter, firstValid, nil diff --git a/test/e2e-go/features/catchup/basicCatchup_test.go b/test/e2e-go/features/catchup/basicCatchup_test.go index 2e3ac87943..adc8c43f18 100644 --- a/test/e2e-go/features/catchup/basicCatchup_test.go +++ b/test/e2e-go/features/catchup/basicCatchup_test.go @@ -56,9 +56,8 @@ func TestBasicCatchup(t *testing.T) { a.NoError(err) // Let the network make some progress - a.NoError(err) waitForRound := uint64(3) - err = fixture.ClientWaitForRoundWithTimeout(fixture.GetAlgodClientForController(nc), waitForRound) + err = fixture.GetAlgodClientForController(nc).WaitForRoundWithTimeout(3) a.NoError(err) // Now spin up third node @@ -71,7 +70,7 @@ func TestBasicCatchup(t *testing.T) { defer shutdownClonedNode(cloneDataDir, &fixture, t) // Now, catch up - err = fixture.LibGoalFixture.ClientWaitForRoundWithTimeout(cloneClient, waitForRound) + _, err = cloneClient.WaitForRound(waitForRound) a.NoError(err) } @@ -155,7 +154,7 @@ func runCatchupOverGossip(t fixtures.TestingTB, // Let the secondary make progress up to round 3, while the primary was never startred ( hence, it's on round = 0) waitForRound := uint64(3) - err = fixture.ClientWaitForRoundWithTimeout(fixture.GetAlgodClientForController(nc), waitForRound) + err = fixture.GetAlgodClientForController(nc).WaitForRoundWithTimeout(waitForRound) a.NoError(err) // stop the secondary, which is on round 3 or more. @@ -167,7 +166,7 @@ func runCatchupOverGossip(t fixtures.TestingTB, a.NoError(err) // Now, catch up - err = fixture.LibGoalFixture.ClientWaitForRoundWithTimeout(lg, waitForRound) + _, err = lg.WaitForRound(waitForRound) a.NoError(err) waitStart := time.Now() @@ -184,7 +183,7 @@ func runCatchupOverGossip(t fixtures.TestingTB, break } - if time.Now().Sub(waitStart) > time.Minute { + if time.Since(waitStart) > time.Minute { // it's taking too long. a.FailNow("Waiting too long for catchup to complete") } @@ -258,7 +257,7 @@ func TestStoppedCatchupOnUnsupported(t *testing.T) { // Let the network make some progress a.NoError(err) waitForRound := uint64(3) // UpgradeVoteRounds + DefaultUpgradeWaitRounds - err = fixture.ClientWaitForRoundWithTimeout(fixture.GetAlgodClientForController(nc), waitForRound) + err = fixture.GetAlgodClientForController(nc).WaitForRoundWithTimeout(waitForRound) a.NoError(err) // Now spin up third node @@ -274,7 +273,7 @@ func TestStoppedCatchupOnUnsupported(t *testing.T) { defer shutdownClonedNode(cloneDataDir, &fixture, t) // Now, catch up - err = fixture.LibGoalFixture.ClientWaitForRoundWithTimeout(cloneClient, waitForRound) + _, err = cloneClient.WaitForRound(waitForRound) a.NoError(err) timeout := time.NewTimer(20 * time.Second) @@ -374,7 +373,7 @@ func TestBasicCatchupCompletes(t *testing.T) { a.NoError(err) // Wait for the network to make some progess. - err = fixture.ClientWaitForRoundWithTimeout(fixture.GetAlgodClientForController(nc), waitForRound) + err = fixture.GetAlgodClientForController(nc).WaitForRoundWithTimeout(waitForRound) a.NoError(err) // Start the third node to catchup. @@ -384,7 +383,7 @@ func TestBasicCatchupCompletes(t *testing.T) { defer shutdownClonedNode(cloneDataDir, &fixture, t) // Wait for it to catchup - err = fixture.LibGoalFixture.ClientWaitForRoundWithTimeout(cloneClient, waitForRound) + _, err = cloneClient.WaitForRound(waitForRound) a.NoError(err) // Calculate the catchup time diff --git a/test/e2e-go/features/catchup/catchpointCatchup_test.go b/test/e2e-go/features/catchup/catchpointCatchup_test.go index 3a1eefedc4..0a1d522cac 100644 --- a/test/e2e-go/features/catchup/catchpointCatchup_test.go +++ b/test/e2e-go/features/catchup/catchpointCatchup_test.go @@ -46,7 +46,7 @@ import ( const basicTestCatchpointInterval = 4 func waitForCatchpointGeneration(t *testing.T, fixture *fixtures.RestClientFixture, client client.RestClient, catchpointRound basics.Round) string { - err := fixture.ClientWaitForRoundWithTimeout(client, uint64(catchpointRound+1)) + err := client.WaitForRoundWithTimeout(uint64(catchpointRound + 1)) if err != nil { return "" } @@ -212,7 +212,7 @@ func startCatchpointGeneratingNode(a *require.Assertions, fixture *fixtures.Rest restClient := fixture.GetAlgodClientForController(nodeController) // We don't want to start using the node without it being properly initialized. - err = fixture.ClientWaitForRoundWithTimeout(restClient, 1) + err = restClient.WaitForRoundWithTimeout(1) a.NoError(err) return nodeController, restClient, &errorsCollector @@ -239,7 +239,7 @@ func startCatchpointUsingNode(a *require.Assertions, fixture *fixtures.RestClien restClient := fixture.GetAlgodClientForController(nodeController) // We don't want to start using the node without it being properly initialized. - err = fixture.ClientWaitForRoundWithTimeout(restClient, 1) + err = restClient.WaitForRoundWithTimeout(1) a.NoError(err) return nodeController, restClient, wp, &errorsCollector @@ -263,7 +263,7 @@ func startCatchpointNormalNode(a *require.Assertions, fixture *fixtures.RestClie restClient := fixture.GetAlgodClientForController(nodeController) // We don't want to start using the node without it being properly initialized. - err = fixture.ClientWaitForRoundWithTimeout(restClient, 1) + err = restClient.WaitForRoundWithTimeout(1) a.NoError(err) return nodeController, restClient, &errorsCollector @@ -365,7 +365,7 @@ func TestBasicCatchpointCatchup(t *testing.T) { _, err = usingNodeRestClient.Catchup(catchpointLabel, 0) a.NoError(err) - err = fixture.ClientWaitForRoundWithTimeout(usingNodeRestClient, uint64(targetCatchpointRound+1)) + err = usingNodeRestClient.WaitForRoundWithTimeout(uint64(targetCatchpointRound + 1)) a.NoError(err) // ensure the raw block can be downloaded (including cert) @@ -438,7 +438,7 @@ func TestCatchpointLabelGeneration(t *testing.T) { primaryNodeRestClient := fixture.GetAlgodClientForController(primaryNode) log.Infof("Building ledger history..") for { - err = fixture.ClientWaitForRound(primaryNodeRestClient, currentRound, 45*time.Second) + _, err = primaryNodeRestClient.WaitForRound(currentRound+1, 45*time.Second) a.NoError(err) if targetRound <= currentRound { break @@ -553,8 +553,7 @@ func TestNodeTxHandlerRestart(t *testing.T) { // Wait for the network to start making progress again primaryNodeRestClient := fixture.GetAlgodClientForController(primaryNode) - err = fixture.ClientWaitForRound(primaryNodeRestClient, targetRound, - 10*catchpointCatchupProtocol.AgreementFilterTimeout) + _, err = primaryNodeRestClient.WaitForRound(targetRound, 10*catchpointCatchupProtocol.AgreementFilterTimeout) a.NoError(err) // let the 2nd client send a transaction @@ -674,8 +673,7 @@ func TestReadyEndpoint(t *testing.T) { // Wait for the network to start making progress again primaryNodeRestClient := fixture.GetAlgodClientForController(primaryNode) - err = fixture.ClientWaitForRound(primaryNodeRestClient, targetRound, - 10*catchpointCatchupProtocol.AgreementFilterTimeout) + _, err = primaryNodeRestClient.WaitForRound(targetRound, 10*catchpointCatchupProtocol.AgreementFilterTimeout) a.NoError(err) // The primary node has reached the target round, diff --git a/test/e2e-go/features/catchup/stateproofsCatchup_test.go b/test/e2e-go/features/catchup/stateproofsCatchup_test.go index 5dcbc11452..f9639abeb1 100644 --- a/test/e2e-go/features/catchup/stateproofsCatchup_test.go +++ b/test/e2e-go/features/catchup/stateproofsCatchup_test.go @@ -115,7 +115,7 @@ func TestStateProofInReplayCatchpoint(t *testing.T) { } // wait for fastcatchup to complete and the node is synced - err = fixture.ClientWaitForRoundWithTimeout(usingNodeRestClient, uint64(targetCatchpointRound+1)) + err = usingNodeRestClient.WaitForRoundWithTimeout(uint64(targetCatchpointRound + 1)) a.NoError(err) primaryLibGoal := fixture.GetLibGoalClientFromNodeController(primaryNode) @@ -174,7 +174,7 @@ func TestStateProofAfterCatchpoint(t *testing.T) { roundAfterSPGeneration := targetCatchpointRound.RoundUpToMultipleOf(basics.Round(consensusParams.StateProofInterval)) + basics.Round(consensusParams.StateProofInterval/2) - err = fixture.ClientWaitForRoundWithTimeout(usingNodeRestClient, uint64(roundAfterSPGeneration)) + err = usingNodeRestClient.WaitForRoundWithTimeout(uint64(roundAfterSPGeneration)) a.NoError(err) primaryLibGoal := fixture.GetLibGoalClientFromNodeController(primaryNode) @@ -234,14 +234,14 @@ func TestSendSigsAfterCatchpointCatchup(t *testing.T) { primaryNodeAddr, err := primaryNode.GetListeningAddress() a.NoError(err) - err = fixture.ClientWaitForRoundWithTimeout(primaryNodeRestClient, 3) + err = primaryNodeRestClient.WaitForRoundWithTimeout(3) a.NoError(err) normalNode, normalNodeRestClient, normalNodeEC := startCatchpointNormalNode(a, &fixture, "Node1", primaryNodeAddr) defer normalNodeEC.Print() defer normalNode.StopAlgod() - err = fixture.ClientWaitForRoundWithTimeout(normalNodeRestClient, 3) + err = normalNodeRestClient.WaitForRoundWithTimeout(3) a.NoError(err) // at this point PrimaryNode and Node1 would pass round 3. Before running Node2 we remove block 2 from Primary database. @@ -267,7 +267,7 @@ func TestSendSigsAfterCatchpointCatchup(t *testing.T) { _, err = usingNodeRestClient.Catchup(catchpointLabel, 0) a.NoError(err) - err = fixture.ClientWaitForRoundWithTimeout(usingNodeRestClient, uint64(targetCatchpointRound)+1) + err = usingNodeRestClient.WaitForRoundWithTimeout(uint64(targetCatchpointRound) + 1) a.NoError(err) lastNormalRound, err := fixture.GetLibGoalClientFromNodeController(normalNode).CurrentRound() @@ -280,7 +280,7 @@ func TestSendSigsAfterCatchpointCatchup(t *testing.T) { lastNormalNodeSignedRound := basics.Round(lastNormalRound).RoundDownToMultipleOf(basics.Round(consensusParams.StateProofInterval)) lastNormalNextStateProofRound := lastNormalNodeSignedRound + basics.Round(consensusParams.StateProofInterval) targetRound := lastNormalNextStateProofRound + basics.Round(consensusParams.StateProofInterval*2) - err = fixture.ClientWaitForRoundWithTimeout(usingNodeRestClient, uint64(targetRound)) + err = usingNodeRestClient.WaitForRoundWithTimeout(uint64(targetRound)) a.NoError(err) primaryClient := fixture.GetLibGoalClientFromNodeController(primaryNode) diff --git a/test/e2e-go/features/followernode/syncDeltas_test.go b/test/e2e-go/features/followernode/syncDeltas_test.go index af27c7dda7..d1458b7451 100644 --- a/test/e2e-go/features/followernode/syncDeltas_test.go +++ b/test/e2e-go/features/followernode/syncDeltas_test.go @@ -74,7 +74,7 @@ func TestBasicSyncMode(t *testing.T) { // Let the network make some progress waitForRound := uint64(5) - err = fixture.ClientWaitForRoundWithTimeout(fixture.GetAlgodClientForController(nc), waitForRound) + err = fixture.GetAlgodClientForController(nc).WaitForRoundWithTimeout(waitForRound) a.NoError(err) // Get the follower client, and exercise the sync/ledger functionality @@ -88,7 +88,7 @@ func TestBasicSyncMode(t *testing.T) { a.NoError(err) a.Equal(round, rResp.Round) // make some progress to round - err = fixture.ClientWaitForRoundWithTimeout(followClient, round) + err = followClient.WaitForRoundWithTimeout(round) a.NoError(err) // retrieve state delta gResp, err := followClient.GetLedgerStateDelta(round) @@ -113,6 +113,6 @@ func TestBasicSyncMode(t *testing.T) { err = followClient.SetSyncRound(round + 1) a.NoError(err) } - err = fixture.LibGoalFixture.ClientWaitForRoundWithTimeout(fixture.LibGoalClient, waitForRound) + err = fixture.WaitForRoundWithTimeout(waitForRound) a.NoError(err) } diff --git a/test/e2e-go/features/followernode/syncRestart_test.go b/test/e2e-go/features/followernode/syncRestart_test.go index 589bb7b53c..1aa5b2560d 100644 --- a/test/e2e-go/features/followernode/syncRestart_test.go +++ b/test/e2e-go/features/followernode/syncRestart_test.go @@ -62,7 +62,7 @@ func TestSyncRestart(t *testing.T) { waitTill := func(node string, round uint64) { controller, err := fixture.GetNodeController(node) a.NoError(err) - err = fixture.ClientWaitForRoundWithTimeout(fixture.GetAlgodClientForController(controller), round) + err = fixture.GetAlgodClientForController(controller).WaitForRoundWithTimeout(round) a.NoError(err) } diff --git a/test/e2e-go/features/incentives/challenge_test.go b/test/e2e-go/features/incentives/challenge_test.go index f45a1b6e2b..51586eab76 100644 --- a/test/e2e-go/features/incentives/challenge_test.go +++ b/test/e2e-go/features/incentives/challenge_test.go @@ -27,6 +27,7 @@ import ( "github.com/algorand/go-algorand/config" "github.com/algorand/go-algorand/daemon/algod/api/server/v2/generated/model" "github.com/algorand/go-algorand/data/basics" + "github.com/algorand/go-algorand/data/bookkeeping" "github.com/algorand/go-algorand/libgoal" "github.com/algorand/go-algorand/protocol" "github.com/algorand/go-algorand/test/framework/fixtures" @@ -48,17 +49,19 @@ func TestChallenges(t *testing.T) { // Start a three-node network. One relay, two nodes with 4 accounts each // At round 50, ~2 nodes will be challenged. + const lookback = 32 const interval = 50 const grace = 10 + const mask = 0x80 var fixture fixtures.RestClientFixture // Speed up rounds, keep lookback > 2 * grace period - fixture.FasterConsensus(protocol.ConsensusFuture, time.Second, 32) + fixture.FasterConsensus(protocol.ConsensusFuture, time.Second, lookback) fixture.AlterConsensus(protocol.ConsensusFuture, func(cp config.ConsensusParams) config.ConsensusParams { cp.Payouts.ChallengeInterval = 50 cp.Payouts.ChallengeGracePeriod = 10 - cp.Payouts.ChallengeBits = 2 + cp.Payouts.ChallengeBits = 1 // half of nodes should get challenged return cp }) fixture.Setup(t, filepath.Join("nettemplates", "Challenges.json")) @@ -78,6 +81,10 @@ func TestChallenges(t *testing.T) { // By re-regging, we become eligible for suspension (normal + challenges) // TODO: Confirm that rereg is required for challenge suspensions + + err := fixture.WaitForRoundWithTimeout(interval - lookback) // Make all LastHeartbeats > interval, < 2*interval + a.NoError(err) + for _, account := range accounts1 { rekeyreg(&fixture, a, c1, account.Address) } @@ -88,86 +95,94 @@ func TestChallenges(t *testing.T) { // turn off node 1, so it can't heartbeat a.NoError(c1.FullStop()) - // Advance to first challenge round, check the blockseed - err := fixture.WaitForRoundWithTimeout(interval) + current, err := c2.CurrentRound() a.NoError(err) + // Get them all done so that their inflated LastHeartbeat comes before the + // next challenge. + a.Less(current+lookback, 2*uint64(interval)) + + // We need to wait for the first challenge that happens after the keyreg + // LastHeartbeat has passed. Example: current is 40, so the lastPossible + // LastHeartbeat is 72. Interval is 50, so challengeRound is 100. - blk, err := c2.BookkeepingBlock(interval) + // 100 = 40 + 32 + (50-22) = 72 + 28 + lastPossible := current + lookback + challengeRound := lastPossible + (interval - lastPossible%interval) + + // Advance to challenge round, check the blockseed + err = fixture.WaitForRoundWithTimeout(challengeRound) + a.NoError(err) + blk, err := c2.BookkeepingBlock(challengeRound) a.NoError(err) - challenge := blk.BlockHeader.Seed[0] & 0xA0 // high two bits + challenge := blk.BlockHeader.Seed[0] & mask // high bit - challenged1 := util.MakeSet[model.Account]() + challenged1 := util.MakeSet[basics.Address]() for _, account := range accounts1 { - abytes, err := basics.UnmarshalChecksumAddress(account.Address) + address, err := basics.UnmarshalChecksumAddress(account.Address) a.NoError(err) - if abytes[0]&0xA0 == challenge { - fmt.Printf("%v of node 1 was challenged %v by %v\n", account.Address, abytes[0], challenge) - challenged1.Add(account) + if address[0]&mask == challenge { + fmt.Printf("%v of node 1 was challenged %v by %v\n", address, address[0], challenge) + challenged1.Add(address) } } + require.NotEmpty(t, challenged1, "rerun the test") // TODO: remove. - challenged2 := util.MakeSet[model.Account]() + challenged2 := util.MakeSet[basics.Address]() for _, account := range accounts2 { - abytes, err := basics.UnmarshalChecksumAddress(account.Address) + address, err := basics.UnmarshalChecksumAddress(account.Address) a.NoError(err) - if abytes[0]&0xA0 == challenge { - fmt.Printf("%v of node 2 was challenged %v by %v\n", account.Address, abytes[0], challenge) - challenged2.Add(account) + if address[0]&mask == challenge { + fmt.Printf("%v of node 2 was challenged %v by %v\n", address, address[0], challenge) + challenged2.Add(address) } } + require.NotEmpty(t, challenged2, "rerun the test") // TODO: remove. allChallenged := util.Union(challenged1, challenged2) - // TODO: unroll this loop and notice the heartbeat transactions from node 2 - err = fixture.WaitForRoundWithTimeout(interval + grace) - a.NoError(err) - - // challenged accounts are still online - for account := range allChallenged { - data, err := c2.AccountData(account.Address) + // All challenged nodes are still online + for address := range allChallenged { + data, err := c2.AccountData(address.String()) a.NoError(err) - a.Equal(basics.Online, data.Status) + a.Equal(basics.Online, data.Status, "%v %d", address.String(), data.LastHeartbeat) a.NotZero(data.VoteID) a.True(data.IncentiveEligible) } - err = fixture.WaitForRoundWithTimeout(interval + grace + 1) - a.NoError(err) - - // The challenged nodes need be "noticed" to be suspended. TODO: Remove this - // section when we have prompt suspensions. - source := accounts2[0] // always pay from operational account on node 2 - for account := range allChallenged { - fmt.Printf("pay %v\n", account.Address) - txn, err := c2.SendPaymentFromUnencryptedWallet(source.Address, account.Address, 1000, 0, nil) - a.NoError(err) - info, err := fixture.WaitForConfirmedTxn(uint64(txn.LastValid), txn.ID().String()) - a.NoError(err) + // In the second half of the grace period, Node 2 should heartbeat for its accounts + beated := util.MakeSet[basics.Address]() + fixture.WithEveryBlock(challengeRound+grace/2, challengeRound+grace, func(block bookkeeping.Block) { + for _, txn := range block.Payset { + hb := txn.Txn.HeartbeatTxnFields + fmt.Printf("Heartbeat txn %v\n", hb) + a.True(challenged2.Contains(hb.HbAddress)) // only Node 2 is alive + a.False(beated.Contains(hb.HbAddress)) // beat only once + beated.Add(hb.HbAddress) + } + a.Empty(block.AbsentParticipationAccounts) // nobody suspended during grace + }) + a.Equal(challenged2, beated) - blk, err := c2.BookkeepingBlock(*info.ConfirmedRound) - a.NoError(err) - a.Len(blk.AbsentParticipationAccounts, 1) - a.Equal(blk.AbsentParticipationAccounts[0].String(), account.Address) - } + blk, err = fixture.WaitForBlockWithTimeout(challengeRound + grace + 1) + a.NoError(err) + a.Equal(challenged1, util.MakeSet(blk.AbsentParticipationAccounts...)) // node 1 challenged accounts are suspended because node 1 is off - for account := range challenged1 { - fmt.Printf("check1 %v\n", account.Address) - data, err := c2.AccountData(account.Address) + for address := range challenged1 { + data, err := c2.AccountData(address.String()) a.NoError(err) - a.Equal(basics.Offline, data.Status, account.Address) - a.NotZero(data.VoteID, account.Address) - a.False(data.IncentiveEligible, account.Address) // suspension turns off flag + a.Equal(basics.Offline, data.Status, address) + a.NotZero(data.VoteID, address) + a.False(data.IncentiveEligible, address) // suspension turns off flag } // node 2 challenged accounts are not suspended (saved by heartbeat) - for account := range challenged2 { - fmt.Printf("check2 %v\n", account.Address) - data, err := c2.AccountData(account.Address) + for address := range challenged2 { + data, err := c2.AccountData(address.String()) a.NoError(err) - a.Equal(basics.Online, data.Status, account.Address) - a.NotZero(data.VoteID, account.Address) - a.True(data.IncentiveEligible, account.Address) + a.Equal(basics.Online, data.Status, address) + a.NotZero(data.VoteID, address) + a.True(data.IncentiveEligible, address) } } diff --git a/test/e2e-go/features/participation/onlineOfflineParticipation_test.go b/test/e2e-go/features/participation/onlineOfflineParticipation_test.go index 0b38fe76ff..21a701139a 100644 --- a/test/e2e-go/features/participation/onlineOfflineParticipation_test.go +++ b/test/e2e-go/features/participation/onlineOfflineParticipation_test.go @@ -216,7 +216,7 @@ func TestNewAccountCanGoOnlineAndParticipate(t *testing.T) { // Need to wait for funding to take effect on selection, then we can see if we're participating // Stop before the account should become eligible for selection so we can ensure it wasn't - err = fixture.ClientWaitForRound(fixture.AlgodClient, uint64(accountProposesStarting-1), + err = fixture.WaitForRound(uint64(accountProposesStarting-1), time.Duration(uint64(globals.MaxTimePerRound)*uint64(accountProposesStarting-1))) a.NoError(err) @@ -226,7 +226,7 @@ func TestNewAccountCanGoOnlineAndParticipate(t *testing.T) { a.False(blockWasProposed, "account should not be selected until BalLookback (round %d) passes", int(accountProposesStarting-1)) // Now wait until the round where the funded account will be used. - err = fixture.ClientWaitForRound(fixture.AlgodClient, uint64(accountProposesStarting), 10*globals.MaxTimePerRound) + err = fixture.WaitForRound(uint64(accountProposesStarting), 10*globals.MaxTimePerRound) a.NoError(err) blockWasProposedByNewAccountRecently := fixture.VerifyBlockProposedRange(newAccount, int(accountProposesStarting), 1) diff --git a/test/e2e-go/features/partitionRecovery/partitionRecovery_test.go b/test/e2e-go/features/partitionRecovery/partitionRecovery_test.go index 21ce3bdf0d..e3429490c4 100644 --- a/test/e2e-go/features/partitionRecovery/partitionRecovery_test.go +++ b/test/e2e-go/features/partitionRecovery/partitionRecovery_test.go @@ -57,7 +57,7 @@ func TestBasicPartitionRecovery(t *testing.T) { // Let the network make some progress waitForRound := uint64(3) - err = fixture.ClientWaitForRoundWithTimeout(fixture.GetAlgodClientForController(nc), waitForRound) + err = fixture.GetAlgodClientForController(nc).WaitForRoundWithTimeout(waitForRound) a.NoError(err) // Now stop 2nd node @@ -133,7 +133,7 @@ func runTestWithStaggeredStopStart(t *testing.T, fixture *fixtures.RestClientFix // Let the network make some progress waitForRound := uint64(3) - err = fixture.ClientWaitForRoundWithTimeout(fixture.GetAlgodClientForController(nc1), waitForRound) + err = fixture.GetAlgodClientForController(nc1).WaitForRoundWithTimeout(waitForRound) a.NoError(err) // Stop Node1 @@ -196,7 +196,7 @@ func TestBasicPartitionRecoveryPartOffline(t *testing.T) { // Let the network make some progress waitForRound := uint64(3) - err = fixture.ClientWaitForRoundWithTimeout(fixture.GetAlgodClientForController(nc1), waitForRound) + err = fixture.GetAlgodClientForController(nc1).WaitForRoundWithTimeout(waitForRound) a.NoError(err) // Stop Node1 @@ -264,8 +264,7 @@ func TestPartitionHalfOffline(t *testing.T) { // Let the network make some progress client := fixture.LibGoalClient - waitForRound := uint64(3) - err = fixture.ClientWaitForRoundWithTimeout(fixture.GetAlgodClientForController(nc1), waitForRound) + err = fixture.GetAlgodClientForController(nc1).WaitForRoundWithTimeout(3) a.NoError(err) // Stop nodes with 50% of stake diff --git a/test/e2e-go/restAPI/other/misc_test.go b/test/e2e-go/restAPI/other/misc_test.go index eeaff9fcd1..23e805dc25 100644 --- a/test/e2e-go/restAPI/other/misc_test.go +++ b/test/e2e-go/restAPI/other/misc_test.go @@ -62,7 +62,7 @@ func TestDisabledAPIConfig(t *testing.T) { a.NoError(err) testClient := client.MakeRestClient(url, "") // empty token - _, err = testClient.WaitForBlock(1) + err = testClient.WaitForRoundWithTimeout(1) assert.NoError(t, err) _, err = testClient.Block(1) assert.NoError(t, err) diff --git a/test/e2e-go/restAPI/simulate/simulateRestAPI_test.go b/test/e2e-go/restAPI/simulate/simulateRestAPI_test.go index 66601c1737..b058b510e4 100644 --- a/test/e2e-go/restAPI/simulate/simulateRestAPI_test.go +++ b/test/e2e-go/restAPI/simulate/simulateRestAPI_test.go @@ -53,7 +53,7 @@ func TestSimulateTxnTracerDevMode(t *testing.T) { testClient := localFixture.LibGoalClient - _, err := testClient.WaitForRound(1) + _, err := testClient.Status() a.NoError(err) wh, err := testClient.GetUnencryptedWalletHandle() @@ -288,11 +288,11 @@ int 1` // Let the primary node make some progress primaryClient := fixture.GetAlgodClientForController(nc) - err = fixture.ClientWaitForRoundWithTimeout(primaryClient, followerSyncRound+uint64(cfg.MaxAcctLookback)) + err = primaryClient.WaitForRoundWithTimeout(followerSyncRound + uint64(cfg.MaxAcctLookback)) a.NoError(err) // Let follower node progress as far as it can - err = fixture.ClientWaitForRoundWithTimeout(followClient, followerSyncRound+uint64(cfg.MaxAcctLookback)-1) + err = followClient.WaitForRoundWithTimeout(followerSyncRound + uint64(cfg.MaxAcctLookback) - 1) a.NoError(err) simulateRequest := v2.PreEncodedSimulateRequest{ diff --git a/test/e2e-go/upgrades/application_support_test.go b/test/e2e-go/upgrades/application_support_test.go index 549a82c5ab..c41ad84166 100644 --- a/test/e2e-go/upgrades/application_support_test.go +++ b/test/e2e-go/upgrades/application_support_test.go @@ -180,7 +180,7 @@ int 1 curStatus, err = client.Status() a.NoError(err) - a.Less(int64(time.Now().Sub(startLoopTime)), int64(3*time.Minute)) + a.Less(int64(time.Since(startLoopTime)), int64(3*time.Minute)) time.Sleep(time.Duration(smallLambdaMs) * time.Millisecond) } @@ -438,7 +438,7 @@ int 1 curStatus, err = client.Status() a.NoError(err) - a.Less(int64(time.Now().Sub(startLoopTime)), int64(3*time.Minute)) + a.Less(int64(time.Since(startLoopTime)), int64(3*time.Minute)) time.Sleep(time.Duration(smallLambdaMs) * time.Millisecond) round = curStatus.LastRound } diff --git a/test/e2e-go/upgrades/rekey_support_test.go b/test/e2e-go/upgrades/rekey_support_test.go index 0dcec41545..cc3eca018c 100644 --- a/test/e2e-go/upgrades/rekey_support_test.go +++ b/test/e2e-go/upgrades/rekey_support_test.go @@ -150,7 +150,7 @@ func TestRekeyUpgrade(t *testing.T) { curStatus, err = client.Status() a.NoError(err) - a.Less(int64(time.Now().Sub(startLoopTime)), int64(3*time.Minute)) + a.Less(int64(time.Since(startLoopTime)), int64(3*time.Minute)) time.Sleep(time.Duration(smallLambdaMs) * time.Millisecond) round = curStatus.LastRound } diff --git a/test/framework/fixtures/libgoalFixture.go b/test/framework/fixtures/libgoalFixture.go index 0a88153989..c05a59ff1f 100644 --- a/test/framework/fixtures/libgoalFixture.go +++ b/test/framework/fixtures/libgoalFixture.go @@ -42,7 +42,6 @@ import ( "github.com/algorand/go-algorand/netdeploy" "github.com/algorand/go-algorand/nodecontrol" "github.com/algorand/go-algorand/protocol" - "github.com/algorand/go-algorand/test/e2e-go/globals" "github.com/algorand/go-algorand/util/db" ) @@ -67,12 +66,12 @@ func (f *RestClientFixture) SetConsensus(consensus config.ConsensusProtocols) { f.consensus = consensus } +// AlterConsensus allows the caller to modify the consensus settings for a given version. func (f *RestClientFixture) AlterConsensus(ver protocol.ConsensusVersion, alter func(config.ConsensusParams) config.ConsensusParams) { if f.consensus == nil { f.consensus = make(config.ConsensusProtocols) } - consensus := config.Consensus[ver] - f.consensus[ver] = alter(consensus) + f.consensus[ver] = alter(f.ConsensusParamsFromVer(ver)) } // FasterConsensus speeds up the given consensus version in two ways. The seed @@ -80,21 +79,19 @@ func (f *RestClientFixture) AlterConsensus(ver protocol.ConsensusVersion, alter // lookback becomes 32. And, if the architecture implies it can be handled, // round times are shortened by lowering vote timeouts. func (f *RestClientFixture) FasterConsensus(ver protocol.ConsensusVersion, timeout time.Duration, lookback basics.Round) { - if f.consensus == nil { - f.consensus = make(config.ConsensusProtocols) - } - fast := config.Consensus[ver] - // balanceRound is 4 * SeedRefreshInterval - if lookback%4 != 0 { - panic(fmt.Sprintf("lookback must be a multiple of 4, got %d", lookback)) - } - fast.SeedRefreshInterval = uint64(lookback) / 4 - // and speed up the rounds while we're at it - if runtime.GOARCH == "amd64" || runtime.GOARCH == "arm64" { - fast.AgreementFilterTimeoutPeriod0 = timeout - fast.AgreementFilterTimeout = timeout - } - f.consensus[ver] = fast + f.AlterConsensus(ver, func(fast config.ConsensusParams) config.ConsensusParams { + // balanceRound is 4 * SeedRefreshInterval + if lookback%4 != 0 { + panic(fmt.Sprintf("lookback must be a multiple of 4, got %d", lookback)) + } + fast.SeedRefreshInterval = uint64(lookback) / 4 + // and speed up the rounds while we're at it + if runtime.GOARCH == "amd64" || runtime.GOARCH == "arm64" { + fast.AgreementFilterTimeoutPeriod0 = timeout + fast.AgreementFilterTimeout = timeout + } + return fast + }) } // Setup is called to initialize the test fixture for the test(s) @@ -460,75 +457,6 @@ func (f *LibGoalFixture) GetParticipationOnlyAccounts(lg libgoal.Client) []accou return f.clientPartKeys[lg.DataDir()] } -// WaitForRoundWithTimeout waits for a given round to reach. The implementation also ensures to limit the wait time for each round to the -// globals.MaxTimePerRound so we can alert when we're getting "hung" before waiting for all the expected rounds to reach. -func (f *LibGoalFixture) WaitForRoundWithTimeout(roundToWaitFor uint64) error { - return f.ClientWaitForRoundWithTimeout(f.LibGoalClient, roundToWaitFor) -} - -// ClientWaitForRoundWithTimeout waits for a given round to be reached by the specific client/node. The implementation -// also ensures to limit the wait time for each round to the globals.MaxTimePerRound so we can alert when we're -// getting "hung" before waiting for all the expected rounds to reach. -func (f *LibGoalFixture) ClientWaitForRoundWithTimeout(client libgoal.Client, roundToWaitFor uint64) error { - status, err := client.Status() - require.NoError(f.t, err) - lastRound := status.LastRound - - // If node is already at or past target round, we're done - if lastRound >= roundToWaitFor { - return nil - } - - roundTime := globals.MaxTimePerRound * 10 // For first block, we wait much longer - roundComplete := make(chan error, 2) - - for nextRound := lastRound + 1; lastRound < roundToWaitFor; { - roundStarted := time.Now() - - go func(done chan error) { - err := f.ClientWaitForRound(client, nextRound, roundTime) - done <- err - }(roundComplete) - - select { - case lastError := <-roundComplete: - if lastError != nil { - close(roundComplete) - return lastError - } - case <-time.After(roundTime): - // we've timed out. - time := time.Now().Sub(roundStarted) - return fmt.Errorf("fixture.WaitForRound took %3.2f seconds between round %d and %d", time.Seconds(), lastRound, nextRound) - } - - roundTime = singleRoundMaxTime - lastRound++ - nextRound++ - } - return nil -} - -// ClientWaitForRound waits up to the specified amount of time for -// the network to reach or pass the specified round, on the specific client/node -func (f *LibGoalFixture) ClientWaitForRound(client libgoal.Client, round uint64, waitTime time.Duration) error { - timeout := time.NewTimer(waitTime) - for { - status, err := client.Status() - if err != nil { - return err - } - if status.LastRound >= round { - return nil - } - select { - case <-timeout.C: - return fmt.Errorf("timeout waiting for round %v", round) - case <-time.After(200 * time.Millisecond): - } - } -} - // CurrentConsensusParams returns the consensus parameters for the currently active protocol func (f *LibGoalFixture) CurrentConsensusParams() (consensus config.ConsensusParams, err error) { status, err := f.LibGoalClient.Status() @@ -540,20 +468,20 @@ func (f *LibGoalFixture) CurrentConsensusParams() (consensus config.ConsensusPar } // ConsensusParams returns the consensus parameters for the protocol from the specified round -func (f *LibGoalFixture) ConsensusParams(round uint64) (consensus config.ConsensusParams, err error) { +func (f *LibGoalFixture) ConsensusParams(round uint64) (config.ConsensusParams, error) { block, err := f.LibGoalClient.BookkeepingBlock(round) if err != nil { - return + return config.ConsensusParams{}, err } - version := protocol.ConsensusVersion(block.CurrentProtocol) - if f.consensus != nil { - consensus, has := f.consensus[version] - if has { - return consensus, nil - } + return f.ConsensusParamsFromVer(block.CurrentProtocol), nil +} + +// ConsensusParamsFromVer looks up a consensus version, allowing for override +func (f *LibGoalFixture) ConsensusParamsFromVer(cv protocol.ConsensusVersion) config.ConsensusParams { + if consensus, has := f.consensus[cv]; has { + return consensus } - consensus = config.Consensus[version] - return + return config.Consensus[cv] } // CurrentMinFeeAndBalance returns the MinTxnFee and MinBalance for the currently active protocol diff --git a/test/framework/fixtures/restClientFixture.go b/test/framework/fixtures/restClientFixture.go index 473df25d38..fb1a26d31b 100644 --- a/test/framework/fixtures/restClientFixture.go +++ b/test/framework/fixtures/restClientFixture.go @@ -25,6 +25,7 @@ import ( "github.com/stretchr/testify/require" "github.com/algorand/go-algorand/data/basics" + "github.com/algorand/go-algorand/data/bookkeeping" "github.com/algorand/go-algorand/netdeploy" "github.com/algorand/go-algorand/protocol" @@ -34,7 +35,6 @@ import ( "github.com/algorand/go-algorand/libgoal" "github.com/algorand/go-algorand/nodecontrol" - "github.com/algorand/go-algorand/test/e2e-go/globals" "github.com/algorand/go-algorand/util/tokens" ) @@ -80,79 +80,37 @@ func (f *RestClientFixture) GetAlgodClientForController(nc nodecontrol.NodeContr // WaitForRound waits up to the specified amount of time for // the network to reach or pass the specified round func (f *RestClientFixture) WaitForRound(round uint64, waitTime time.Duration) error { - return f.ClientWaitForRound(f.AlgodClient, round, waitTime) + _, err := f.AlgodClient.WaitForRound(round, waitTime) + return err } -// ClientWaitForRound waits up to the specified amount of time for -// the network to reach or pass the specified round, on the specific client/node -func (f *RestClientFixture) ClientWaitForRound(client client.RestClient, round uint64, waitTime time.Duration) error { - timeout := time.NewTimer(waitTime) - for { - status, err := client.Status() - if err != nil { - return err - } - - if status.LastRound >= round { - return nil - } - select { - case <-timeout.C: - return fmt.Errorf("timeout waiting for round %v with last round = %v", round, status.LastRound) - case <-time.After(200 * time.Millisecond): - } +// WithEveryBlock calls the provided function for every block from first to last. +func (f *RestClientFixture) WithEveryBlock(first, last uint64, visit func(bookkeeping.Block)) { + for round := first; round <= last; round++ { + err := f.WaitForRoundWithTimeout(round) + require.NoError(f.t, err) + block, err := f.AlgodClient.Block(round) + require.NoError(f.t, err) + visit(block.Block) } } // WaitForRoundWithTimeout waits for a given round to reach. The implementation also ensures to limit the wait time for each round to the // globals.MaxTimePerRound so we can alert when we're getting "hung" before waiting for all the expected rounds to reach. func (f *RestClientFixture) WaitForRoundWithTimeout(roundToWaitFor uint64) error { - return f.ClientWaitForRoundWithTimeout(f.AlgodClient, roundToWaitFor) + return f.AlgodClient.WaitForRoundWithTimeout(roundToWaitFor) } -const singleRoundMaxTime = globals.MaxTimePerRound * 40 - -// ClientWaitForRoundWithTimeout waits for a given round to be reached by the specific client/node. The implementation -// also ensures to limit the wait time for each round to the globals.MaxTimePerRound so we can alert when we're -// getting "hung" before waiting for all the expected rounds to reach. -func (f *RestClientFixture) ClientWaitForRoundWithTimeout(client client.RestClient, roundToWaitFor uint64) error { - status, err := client.Status() - require.NoError(f.t, err) - lastRound := status.LastRound - - // If node is already at or past target round, we're done - if lastRound >= roundToWaitFor { - return nil +// WaitForBlockWithTimeout waits for a given round and returns its block. +func (f *RestClientFixture) WaitForBlockWithTimeout(roundToWaitFor uint64) (bookkeeping.Block, error) { + if err := f.AlgodClient.WaitForRoundWithTimeout(roundToWaitFor); err != nil { + return bookkeeping.Block{}, err } - - roundTime := globals.MaxTimePerRound * 10 // For first block, we wait much longer - roundComplete := make(chan error, 2) - - for nextRound := lastRound + 1; lastRound < roundToWaitFor; { - roundStarted := time.Now() - - go func(done chan error) { - err := f.ClientWaitForRound(client, nextRound, roundTime) - done <- err - }(roundComplete) - - select { - case lastError := <-roundComplete: - if lastError != nil { - close(roundComplete) - return lastError - } - case <-time.After(roundTime): - // we've timed out. - time := time.Now().Sub(roundStarted) - return fmt.Errorf("fixture.WaitForRound took %3.2f seconds between round %d and %d", time.Seconds(), lastRound, nextRound) - } - - roundTime = singleRoundMaxTime - lastRound++ - nextRound++ + both, err := f.AlgodClient.EncodedBlockCert(roundToWaitFor) + if err != nil { + return bookkeeping.Block{}, err } - return nil + return both.Block, nil } // GetFirstAccount returns the first account from listing local accounts @@ -367,17 +325,15 @@ func (f *RestClientFixture) SendMoneyAndWaitFromWallet(walletHandle, walletPassw // VerifyBlockProposedRange checks the rounds starting at fromRounds and moving backwards checking countDownNumRounds rounds if any // blocks were proposed by address -func (f *RestClientFixture) VerifyBlockProposedRange(account string, fromRound, countDownNumRounds int) (blockWasProposed bool) { - c := f.LibGoalClient +func (f *RestClientFixture) VerifyBlockProposedRange(account string, fromRound, countDownNumRounds int) bool { for i := 0; i < countDownNumRounds; i++ { - cert, err := c.EncodedBlockCert(uint64(fromRound - i)) + cert, err := f.AlgodClient.EncodedBlockCert(uint64(fromRound - i)) require.NoError(f.t, err, "client failed to get block %d", fromRound-i) if cert.Certificate.Proposal.OriginalProposer.GetUserAddress() == account { - blockWasProposed = true - break + return true } } - return + return false } // VerifyBlockProposed checks the last searchRange blocks to see if any blocks were proposed by address diff --git a/util/db/dbutil.go b/util/db/dbutil.go index a6e524464d..8a52862c54 100644 --- a/util/db/dbutil.go +++ b/util/db/dbutil.go @@ -311,7 +311,7 @@ func (db *Accessor) AtomicContext(ctx context.Context, fn idemFn, extras ...inte } if time.Now().After(atomicDeadline) { - db.getDecoratedLogger(fn, extras).Warnf("dbatomic: tx surpassed expected deadline by %v", time.Now().Sub(atomicDeadline)) + db.getDecoratedLogger(fn, extras).Warnf("dbatomic: tx surpassed expected deadline by %v", time.Since(atomicDeadline)) } return } From 75f9afb1d56b8a1cb501a5a639560806a277892b Mon Sep 17 00:00:00 2001 From: John Jannotti Date: Thu, 21 Nov 2024 12:32:00 -0500 Subject: [PATCH 10/19] Add some extra checks on cheap heartbeats --- ledger/apply/heartbeat.go | 15 ++++++++++++++- ledger/apply/heartbeat_test.go | 25 +++++++++++++++++-------- 2 files changed, 31 insertions(+), 9 deletions(-) diff --git a/ledger/apply/heartbeat.go b/ledger/apply/heartbeat.go index afb9af257a..3ef01d408f 100644 --- a/ledger/apply/heartbeat.go +++ b/ledger/apply/heartbeat.go @@ -42,11 +42,24 @@ func Heartbeat(hb transactions.HeartbeatTxnFields, header transactions.Header, b if header.Fee.Raw > 0 { kind = "cheap" } + + // These first checks are a little draconian. The idea is not let these + // free transactions do anything except their exact intended purpose. + if len(header.Note) > 0 { + return fmt.Errorf("%s heartbeat is not allowed to have a note", kind) + } + if header.Lease != [32]byte{} { + return fmt.Errorf("%s heartbeat is not allowed to have a lease", kind) + } + if !header.RekeyTo.IsZero() { + return fmt.Errorf("%s heartbeat is not allowed to rekey", kind) + } + if account.Status != basics.Online { return fmt.Errorf("%s heartbeat is not allowed for %s %+v", kind, account.Status, hb.HbAddress) } if !account.IncentiveEligible { - return fmt.Errorf("%s heartbeat is not allowed for ineligible %+v", kind, hb.HbAddress) + return fmt.Errorf("%s heartbeat is not allowed when not IncentiveEligible %+v", kind, hb.HbAddress) } ch := FindChallenge(proto.Payouts, round, provider, ChRisky) if ch.round == 0 { diff --git a/ledger/apply/heartbeat_test.go b/ledger/apply/heartbeat_test.go index f8607d915c..3f43025a7c 100644 --- a/ledger/apply/heartbeat_test.go +++ b/ledger/apply/heartbeat_test.go @@ -110,22 +110,28 @@ func TestCheapRules(t *testing.T) { addrStart byte status basics.Status incentiveEligble bool + note []byte + lease [32]byte + rekey [32]byte err string } - + empty := [32]byte{} // Grace period is 200. For the second half of the grace period (1101-1200), // the heartbeat is free for online, incentive eligible, challenged accounts. cases := []tcase{ // test of range - {1100, 0x01, basics.Online, true, "no challenge"}, - {1101, 0x01, basics.Online, true, ""}, - {1200, 0x01, basics.Online, true, ""}, - {1201, 0x01, basics.Online, true, "no challenge"}, + {1100, 0x01, basics.Online, true, nil, empty, empty, "no challenge"}, + {1101, 0x01, basics.Online, true, nil, empty, empty, ""}, + {1200, 0x01, basics.Online, true, nil, empty, empty, ""}, + {1201, 0x01, basics.Online, true, nil, empty, empty, "no challenge"}, // test of the other requirements - {1101, 0xf1, basics.Online, true, "not challenged by"}, - {1101, 0x01, basics.Offline, true, "not allowed for Offline"}, - {1101, 0x01, basics.Online, false, "not allowed for ineligible"}, + {1101, 0x01, basics.Online, true, []byte("note"), empty, empty, "not allowed to have a note"}, + {1101, 0x01, basics.Online, true, nil, [32]byte{'l', 'e', 'a', 's', 'e'}, empty, "not allowed to have a lease"}, + {1101, 0x01, basics.Online, true, nil, empty, [32]byte{'r', 'e', 'k', 'e', 'y'}, "not allowed to rekey"}, + {1101, 0xf1, basics.Online, true, nil, empty, empty, "not challenged by"}, + {1101, 0x01, basics.Offline, true, nil, empty, empty, "not allowed for Offline"}, + {1101, 0x01, basics.Online, false, nil, empty, empty, "not allowed when not IncentiveEligible"}, } for _, tc := range cases { const keyDilution = 777 @@ -164,6 +170,9 @@ func TestCheapRules(t *testing.T) { Fee: basics.MicroAlgos{Raw: 1}, FirstValid: tc.rnd - 10, LastValid: tc.rnd + 10, + Lease: tc.lease, + Note: tc.note, + RekeyTo: tc.rekey, HbAddress: voter, HbProof: otss.Sign(id, seed), HbSeed: seed, From 674915ee097a06a686f1ce7e05e65141c09b9d6e Mon Sep 17 00:00:00 2001 From: John Jannotti Date: Thu, 21 Nov 2024 12:56:18 -0500 Subject: [PATCH 11/19] Introduce HeartbeatProof The HeartbeatProof type just makes the Heartbeat transactions a little cleaner - they lack the legacy field that OneTimeSignature can't remove. --- crypto/msgp_gen.go | 236 +++++++++++++++++++++++++++++++++ crypto/msgp_gen_test.go | 60 +++++++++ crypto/onetimesig.go | 39 ++++++ data/transactions/heartbeat.go | 2 +- data/transactions/msgp_gen.go | 4 +- data/txntest/txn.go | 2 +- ledger/apply/heartbeat.go | 2 +- ledger/apply/heartbeat_test.go | 4 +- 8 files changed, 342 insertions(+), 7 deletions(-) diff --git a/crypto/msgp_gen.go b/crypto/msgp_gen.go index ab5bdceb88..fc279029a0 100644 --- a/crypto/msgp_gen.go +++ b/crypto/msgp_gen.go @@ -111,6 +111,16 @@ import ( // |-----> MsgIsZero // |-----> HashTypeMaxSize() // +// HeartbeatProof +// |-----> (*) MarshalMsg +// |-----> (*) CanMarshalMsg +// |-----> (*) UnmarshalMsg +// |-----> (*) UnmarshalMsgWithState +// |-----> (*) CanUnmarshalMsg +// |-----> (*) Msgsize +// |-----> (*) MsgIsZero +// |-----> HeartbeatProofMaxSize() +// // MasterDerivationKey // |-----> (*) MarshalMsg // |-----> (*) CanMarshalMsg @@ -1169,6 +1179,232 @@ func HashTypeMaxSize() (s int) { return } +// MarshalMsg implements msgp.Marshaler +func (z *HeartbeatProof) MarshalMsg(b []byte) (o []byte) { + o = msgp.Require(b, z.Msgsize()) + // omitempty: check for empty values + zb0006Len := uint32(5) + var zb0006Mask uint8 /* 6 bits */ + if (*z).PK == (ed25519PublicKey{}) { + zb0006Len-- + zb0006Mask |= 0x2 + } + if (*z).PK1Sig == (ed25519Signature{}) { + zb0006Len-- + zb0006Mask |= 0x4 + } + if (*z).PK2 == (ed25519PublicKey{}) { + zb0006Len-- + zb0006Mask |= 0x8 + } + if (*z).PK2Sig == (ed25519Signature{}) { + zb0006Len-- + zb0006Mask |= 0x10 + } + if (*z).Sig == (ed25519Signature{}) { + zb0006Len-- + zb0006Mask |= 0x20 + } + // variable map header, size zb0006Len + o = append(o, 0x80|uint8(zb0006Len)) + if zb0006Len != 0 { + if (zb0006Mask & 0x2) == 0 { // if not empty + // string "p" + o = append(o, 0xa1, 0x70) + o = msgp.AppendBytes(o, ((*z).PK)[:]) + } + if (zb0006Mask & 0x4) == 0 { // if not empty + // string "p1s" + o = append(o, 0xa3, 0x70, 0x31, 0x73) + o = msgp.AppendBytes(o, ((*z).PK1Sig)[:]) + } + if (zb0006Mask & 0x8) == 0 { // if not empty + // string "p2" + o = append(o, 0xa2, 0x70, 0x32) + o = msgp.AppendBytes(o, ((*z).PK2)[:]) + } + if (zb0006Mask & 0x10) == 0 { // if not empty + // string "p2s" + o = append(o, 0xa3, 0x70, 0x32, 0x73) + o = msgp.AppendBytes(o, ((*z).PK2Sig)[:]) + } + if (zb0006Mask & 0x20) == 0 { // if not empty + // string "s" + o = append(o, 0xa1, 0x73) + o = msgp.AppendBytes(o, ((*z).Sig)[:]) + } + } + return +} + +func (_ *HeartbeatProof) CanMarshalMsg(z interface{}) bool { + _, ok := (z).(*HeartbeatProof) + return ok +} + +// UnmarshalMsg implements msgp.Unmarshaler +func (z *HeartbeatProof) UnmarshalMsgWithState(bts []byte, st msgp.UnmarshalState) (o []byte, err error) { + if st.AllowableDepth == 0 { + err = msgp.ErrMaxDepthExceeded{} + return + } + st.AllowableDepth-- + var field []byte + _ = field + var zb0006 int + var zb0007 bool + zb0006, zb0007, bts, err = msgp.ReadMapHeaderBytes(bts) + if _, ok := err.(msgp.TypeError); ok { + zb0006, zb0007, bts, err = msgp.ReadArrayHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + if zb0006 > 0 { + zb0006-- + bts, err = msgp.ReadExactBytes(bts, ((*z).Sig)[:]) + if err != nil { + err = msgp.WrapError(err, "struct-from-array", "Sig") + return + } + } + if zb0006 > 0 { + zb0006-- + bts, err = msgp.ReadExactBytes(bts, ((*z).PK)[:]) + if err != nil { + err = msgp.WrapError(err, "struct-from-array", "PK") + return + } + } + if zb0006 > 0 { + zb0006-- + bts, err = msgp.ReadExactBytes(bts, ((*z).PK2)[:]) + if err != nil { + err = msgp.WrapError(err, "struct-from-array", "PK2") + return + } + } + if zb0006 > 0 { + zb0006-- + bts, err = msgp.ReadExactBytes(bts, ((*z).PK1Sig)[:]) + if err != nil { + err = msgp.WrapError(err, "struct-from-array", "PK1Sig") + return + } + } + if zb0006 > 0 { + zb0006-- + bts, err = msgp.ReadExactBytes(bts, ((*z).PK2Sig)[:]) + if err != nil { + err = msgp.WrapError(err, "struct-from-array", "PK2Sig") + return + } + } + if zb0006 > 0 { + err = msgp.ErrTooManyArrayFields(zb0006) + if err != nil { + err = msgp.WrapError(err, "struct-from-array") + return + } + } + } else { + if err != nil { + err = msgp.WrapError(err) + return + } + if zb0007 { + (*z) = HeartbeatProof{} + } + for zb0006 > 0 { + zb0006-- + field, bts, err = msgp.ReadMapKeyZC(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + switch string(field) { + case "s": + bts, err = msgp.ReadExactBytes(bts, ((*z).Sig)[:]) + if err != nil { + err = msgp.WrapError(err, "Sig") + return + } + case "p": + bts, err = msgp.ReadExactBytes(bts, ((*z).PK)[:]) + if err != nil { + err = msgp.WrapError(err, "PK") + return + } + case "p2": + bts, err = msgp.ReadExactBytes(bts, ((*z).PK2)[:]) + if err != nil { + err = msgp.WrapError(err, "PK2") + return + } + case "p1s": + bts, err = msgp.ReadExactBytes(bts, ((*z).PK1Sig)[:]) + if err != nil { + err = msgp.WrapError(err, "PK1Sig") + return + } + case "p2s": + bts, err = msgp.ReadExactBytes(bts, ((*z).PK2Sig)[:]) + if err != nil { + err = msgp.WrapError(err, "PK2Sig") + return + } + default: + err = msgp.ErrNoField(string(field)) + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + } + o = bts + return +} + +func (z *HeartbeatProof) UnmarshalMsg(bts []byte) (o []byte, err error) { + return z.UnmarshalMsgWithState(bts, msgp.DefaultUnmarshalState) +} +func (_ *HeartbeatProof) CanUnmarshalMsg(z interface{}) bool { + _, ok := (z).(*HeartbeatProof) + return ok +} + +// Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message +func (z *HeartbeatProof) Msgsize() (s int) { + s = 1 + 2 + msgp.ArrayHeaderSize + (64 * (msgp.ByteSize)) + 2 + msgp.ArrayHeaderSize + (32 * (msgp.ByteSize)) + 3 + msgp.ArrayHeaderSize + (32 * (msgp.ByteSize)) + 4 + msgp.ArrayHeaderSize + (64 * (msgp.ByteSize)) + 4 + msgp.ArrayHeaderSize + (64 * (msgp.ByteSize)) + return +} + +// MsgIsZero returns whether this is a zero value +func (z *HeartbeatProof) MsgIsZero() bool { + return ((*z).Sig == (ed25519Signature{})) && ((*z).PK == (ed25519PublicKey{})) && ((*z).PK2 == (ed25519PublicKey{})) && ((*z).PK1Sig == (ed25519Signature{})) && ((*z).PK2Sig == (ed25519Signature{})) +} + +// MaxSize returns a maximum valid message size for this message type +func HeartbeatProofMaxSize() (s int) { + s = 1 + 2 + // Calculating size of array: z.Sig + s += msgp.ArrayHeaderSize + ((64) * (msgp.ByteSize)) + s += 2 + // Calculating size of array: z.PK + s += msgp.ArrayHeaderSize + ((32) * (msgp.ByteSize)) + s += 3 + // Calculating size of array: z.PK2 + s += msgp.ArrayHeaderSize + ((32) * (msgp.ByteSize)) + s += 4 + // Calculating size of array: z.PK1Sig + s += msgp.ArrayHeaderSize + ((64) * (msgp.ByteSize)) + s += 4 + // Calculating size of array: z.PK2Sig + s += msgp.ArrayHeaderSize + ((64) * (msgp.ByteSize)) + return +} + // MarshalMsg implements msgp.Marshaler func (z *MasterDerivationKey) MarshalMsg(b []byte) (o []byte) { o = msgp.Require(b, z.Msgsize()) diff --git a/crypto/msgp_gen_test.go b/crypto/msgp_gen_test.go index b3fb95150b..0105a58f1d 100644 --- a/crypto/msgp_gen_test.go +++ b/crypto/msgp_gen_test.go @@ -434,6 +434,66 @@ func BenchmarkUnmarshalHashFactory(b *testing.B) { } } +func TestMarshalUnmarshalHeartbeatProof(t *testing.T) { + partitiontest.PartitionTest(t) + v := HeartbeatProof{} + bts := v.MarshalMsg(nil) + left, err := v.UnmarshalMsg(bts) + if err != nil { + t.Fatal(err) + } + if len(left) > 0 { + t.Errorf("%d bytes left over after UnmarshalMsg(): %q", len(left), left) + } + + left, err = msgp.Skip(bts) + if err != nil { + t.Fatal(err) + } + if len(left) > 0 { + t.Errorf("%d bytes left over after Skip(): %q", len(left), left) + } +} + +func TestRandomizedEncodingHeartbeatProof(t *testing.T) { + protocol.RunEncodingTest(t, &HeartbeatProof{}) +} + +func BenchmarkMarshalMsgHeartbeatProof(b *testing.B) { + v := HeartbeatProof{} + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + v.MarshalMsg(nil) + } +} + +func BenchmarkAppendMsgHeartbeatProof(b *testing.B) { + v := HeartbeatProof{} + bts := make([]byte, 0, v.Msgsize()) + bts = v.MarshalMsg(bts[0:0]) + b.SetBytes(int64(len(bts))) + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + bts = v.MarshalMsg(bts[0:0]) + } +} + +func BenchmarkUnmarshalHeartbeatProof(b *testing.B) { + v := HeartbeatProof{} + bts := v.MarshalMsg(nil) + b.ReportAllocs() + b.SetBytes(int64(len(bts))) + b.ResetTimer() + for i := 0; i < b.N; i++ { + _, err := v.UnmarshalMsg(bts) + if err != nil { + b.Fatal(err) + } + } +} + func TestMarshalUnmarshalMasterDerivationKey(t *testing.T) { partitiontest.PartitionTest(t) v := MasterDerivationKey{} diff --git a/crypto/onetimesig.go b/crypto/onetimesig.go index d05ccaa961..a817e590fe 100644 --- a/crypto/onetimesig.go +++ b/crypto/onetimesig.go @@ -57,6 +57,45 @@ type OneTimeSignature struct { PK2Sig ed25519Signature `codec:"p2s"` } +// A HeartbeatProof is functionally equivalent to a OneTimeSignature, but it has +// been cleaned up for use as a transaction field in heartbeat transactions. +type HeartbeatProof struct { + _struct struct{} `codec:",omitempty,omitemptyarray"` + + // Sig is a signature of msg under the key PK. + Sig ed25519Signature `codec:"s"` + PK ed25519PublicKey `codec:"p"` + + // PK2 is used to verify a two-level ephemeral signature. + PK2 ed25519PublicKey `codec:"p2"` + // PK1Sig is a signature of OneTimeSignatureSubkeyOffsetID(PK, Batch, Offset) under the key PK2. + PK1Sig ed25519Signature `codec:"p1s"` + // PK2Sig is a signature of OneTimeSignatureSubkeyBatchID(PK2, Batch) under the master key (OneTimeSignatureVerifier). + PK2Sig ed25519Signature `codec:"p2s"` +} + +// ToOneTimeSignature converts a HeartbeatProof to a OneTimeSignature. +func (hbp HeartbeatProof) ToOneTimeSignature() OneTimeSignature { + return OneTimeSignature{ + Sig: hbp.Sig, + PK: hbp.PK, + PK2: hbp.PK2, + PK1Sig: hbp.PK1Sig, + PK2Sig: hbp.PK2Sig, + } +} + +// ToHeartbeatProof converts a OneTimeSignature to a HeartbeatProof. +func (ots OneTimeSignature) ToHeartbeatProof() HeartbeatProof { + return HeartbeatProof{ + Sig: ots.Sig, + PK: ots.PK, + PK2: ots.PK2, + PK1Sig: ots.PK1Sig, + PK2Sig: ots.PK2Sig, + } +} + // A OneTimeSignatureSubkeyBatchID identifies an ephemeralSubkey of a batch // for the purposes of signing it with the top-level master key. type OneTimeSignatureSubkeyBatchID struct { diff --git a/data/transactions/heartbeat.go b/data/transactions/heartbeat.go index 873a905079..48a4df6c69 100644 --- a/data/transactions/heartbeat.go +++ b/data/transactions/heartbeat.go @@ -32,7 +32,7 @@ type HeartbeatTxnFields struct { HbAddress basics.Address `codec:"hbad"` // HbProof is a signature using HeartbeatAddress's partkey, thereby showing it is online. - HbProof crypto.OneTimeSignature `codec:"hbprf"` + HbProof crypto.HeartbeatProof `codec:"hbprf"` // HbSeed must be the block seed for the block before this transaction's // firstValid. It is supplied in the transaction so that Proof can be diff --git a/data/transactions/msgp_gen.go b/data/transactions/msgp_gen.go index f7ea0e8fbb..edc229bffe 100644 --- a/data/transactions/msgp_gen.go +++ b/data/transactions/msgp_gen.go @@ -3080,7 +3080,7 @@ func (z *HeartbeatTxnFields) MsgIsZero() bool { // MaxSize returns a maximum valid message size for this message type func HeartbeatTxnFieldsMaxSize() (s int) { - s = 1 + 5 + basics.AddressMaxSize() + 6 + crypto.OneTimeSignatureMaxSize() + 5 + committee.SeedMaxSize() + s = 1 + 5 + basics.AddressMaxSize() + 6 + crypto.HeartbeatProofMaxSize() + 5 + committee.SeedMaxSize() return } @@ -6935,7 +6935,7 @@ func TransactionMaxSize() (s int) { s += 5 // Calculating size of slice: z.ApplicationCallTxnFields.ForeignAssets s += msgp.ArrayHeaderSize + ((encodedMaxForeignAssets) * (basics.AssetIndexMaxSize())) - s += 5 + basics.StateSchemaMaxSize() + 5 + basics.StateSchemaMaxSize() + 5 + msgp.BytesPrefixSize + config.MaxAvailableAppProgramLen + 5 + msgp.BytesPrefixSize + config.MaxAvailableAppProgramLen + 5 + msgp.Uint32Size + 7 + protocol.StateProofTypeMaxSize() + 3 + stateproof.StateProofMaxSize() + 6 + stateproofmsg.MessageMaxSize() + 5 + basics.AddressMaxSize() + 6 + crypto.OneTimeSignatureMaxSize() + 5 + committee.SeedMaxSize() + s += 5 + basics.StateSchemaMaxSize() + 5 + basics.StateSchemaMaxSize() + 5 + msgp.BytesPrefixSize + config.MaxAvailableAppProgramLen + 5 + msgp.BytesPrefixSize + config.MaxAvailableAppProgramLen + 5 + msgp.Uint32Size + 7 + protocol.StateProofTypeMaxSize() + 3 + stateproof.StateProofMaxSize() + 6 + stateproofmsg.MessageMaxSize() + 5 + basics.AddressMaxSize() + 6 + crypto.HeartbeatProofMaxSize() + 5 + committee.SeedMaxSize() return } diff --git a/data/txntest/txn.go b/data/txntest/txn.go index 5b07ad03d5..d734f47576 100644 --- a/data/txntest/txn.go +++ b/data/txntest/txn.go @@ -94,7 +94,7 @@ type Txn struct { StateProofMsg stateproofmsg.Message HbAddress basics.Address - HbProof crypto.OneTimeSignature + HbProof crypto.HeartbeatProof HbSeed committee.Seed } diff --git a/ledger/apply/heartbeat.go b/ledger/apply/heartbeat.go index 3ef01d408f..a0e57849bf 100644 --- a/ledger/apply/heartbeat.go +++ b/ledger/apply/heartbeat.go @@ -108,7 +108,7 @@ func Heartbeat(hb transactions.HeartbeatTxnFields, header transactions.Header, b return fmt.Errorf("provided seed %v does not match round %d's seed %v", hb.HbSeed, header.FirstValid-1, hdr.Seed) } - if !sv.Verify(id, hdr.Seed, hb.HbProof) { + if !sv.Verify(id, hdr.Seed, hb.HbProof.ToOneTimeSignature()) { return fmt.Errorf("heartbeat failed verification with VoteID %v", sv) } diff --git a/ledger/apply/heartbeat_test.go b/ledger/apply/heartbeat_test.go index 3f43025a7c..f4468ab6e7 100644 --- a/ledger/apply/heartbeat_test.go +++ b/ledger/apply/heartbeat_test.go @@ -73,7 +73,7 @@ func TestHeartbeat(t *testing.T) { FirstValid: fv, LastValid: lv, HbAddress: voter, - HbProof: otss.Sign(id, seed), + HbProof: otss.Sign(id, seed).ToHeartbeatProof(), HbSeed: seed, } @@ -174,7 +174,7 @@ func TestCheapRules(t *testing.T) { Note: tc.note, RekeyTo: tc.rekey, HbAddress: voter, - HbProof: otss.Sign(id, seed), + HbProof: otss.Sign(id, seed).ToHeartbeatProof(), HbSeed: seed, } From ba3c96c9eb94a2d8c006684a38b502de130dc10d Mon Sep 17 00:00:00 2001 From: John Jannotti Date: Thu, 21 Nov 2024 14:11:47 -0500 Subject: [PATCH 12/19] missed a conversion --- heartbeat/service.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/heartbeat/service.go b/heartbeat/service.go index d43fd196b1..741d004272 100644 --- a/heartbeat/service.go +++ b/heartbeat/service.go @@ -164,7 +164,7 @@ func (s *Service) prepareHeartbeat(pr account.ParticipationRecordForRound, lates id := basics.OneTimeIDForRound(stxn.Txn.LastValid, pr.KeyDilution) stxn.Txn.HeartbeatTxnFields = transactions.HeartbeatTxnFields{ HbAddress: pr.Account, - HbProof: pr.Voting.Sign(id, latest.Seed), + HbProof: pr.Voting.Sign(id, latest.Seed).ToHeartbeatProof(), HbSeed: latest.Seed, } From 9fa93f0c460fec7250613f06b1ba29d28978c221 Mon Sep 17 00:00:00 2001 From: John Jannotti Date: Thu, 21 Nov 2024 14:19:26 -0500 Subject: [PATCH 13/19] more conversions --- ledger/heartbeat_test.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/ledger/heartbeat_test.go b/ledger/heartbeat_test.go index 6de5d1f111..e2ed851783 100644 --- a/ledger/heartbeat_test.go +++ b/ledger/heartbeat_test.go @@ -89,7 +89,7 @@ func TestHeartBeat(t *testing.T) { Sender: addrs[1], HbAddress: addrs[1], HbSeed: b0.Seed, - HbProof: otss.Sign(firstID, b0.Seed), + HbProof: otss.Sign(firstID, b0.Seed).ToHeartbeatProof(), }, "does not match round 1's seed") @@ -105,7 +105,7 @@ func TestHeartBeat(t *testing.T) { Sender: addrs[1], HbAddress: addrs[1], HbSeed: b1.Seed, - HbProof: otss.Sign(basics.OneTimeIDForRound(30, kd), b0.Seed), + HbProof: otss.Sign(basics.OneTimeIDForRound(30, kd), b0.Seed).ToHeartbeatProof(), }, "failed verification") @@ -116,7 +116,7 @@ func TestHeartBeat(t *testing.T) { Sender: addrs[1], HbAddress: addrs[1], HbSeed: b0.Seed, - HbProof: otss.Sign(basics.OneTimeIDForRound(30, kd), b1.Seed), + HbProof: otss.Sign(basics.OneTimeIDForRound(30, kd), b1.Seed).ToHeartbeatProof(), }, "does not match round 1's") @@ -127,7 +127,7 @@ func TestHeartBeat(t *testing.T) { Sender: addrs[1], HbAddress: addrs[1], HbSeed: b1.Seed, - HbProof: otss.Sign(basics.OneTimeIDForRound(30, kd), b1.Seed), + HbProof: otss.Sign(basics.OneTimeIDForRound(30, kd), b1.Seed).ToHeartbeatProof(), }, "failed verification") @@ -138,7 +138,7 @@ func TestHeartBeat(t *testing.T) { Sender: addrs[1], HbAddress: addrs[1], HbSeed: b1.Seed, - HbProof: otss.Sign(basics.OneTimeIDForRound(30, kd), b1.Seed), + HbProof: otss.Sign(basics.OneTimeIDForRound(30, kd), b1.Seed).ToHeartbeatProof(), }) }) From ffe70457343b5c1709caccc6e6d656479c326016 Mon Sep 17 00:00:00 2001 From: John Jannotti Date: Fri, 22 Nov 2024 13:37:38 -0500 Subject: [PATCH 14/19] Bump LastHeartbeat if account receives a lot of money --- ledger/eval/eval.go | 20 +++++ .../features/incentives/whalejoin_test.go | 78 +++++++++++++++++-- 2 files changed, 92 insertions(+), 6 deletions(-) diff --git a/ledger/eval/eval.go b/ledger/eval/eval.go index 2e5c8f00a3..75404d36fe 100644 --- a/ledger/eval/eval.go +++ b/ledger/eval/eval.go @@ -608,6 +608,7 @@ func (cs *roundCowState) Move(from basics.Address, to basics.Address, amt basics if overflowed { return fmt.Errorf("overspend (account %v, data %+v, tried to spend %v)", from, fromBal, amt) } + fromBalNew = cs.autoHeartbeat(fromBal, fromBalNew) err = cs.putAccount(from, fromBalNew) if err != nil { return err @@ -636,6 +637,7 @@ func (cs *roundCowState) Move(from basics.Address, to basics.Address, amt basics if overflowed { return fmt.Errorf("balance overflow (account %v, data %+v, was going to receive %v)", to, toBal, amt) } + toBalNew = cs.autoHeartbeat(toBal, toBalNew) err = cs.putAccount(to, toBalNew) if err != nil { return err @@ -645,6 +647,24 @@ func (cs *roundCowState) Move(from basics.Address, to basics.Address, amt basics return nil } +// autoHeartbeat compares `before` and `after`, returning a new AccountData +// based on `after` but with an updated `LastHeartbeat` if `after` shows enough +// balance increase to risk a false positive suspension for absenteeism. +func (cs *roundCowState) autoHeartbeat(before, after ledgercore.AccountData) ledgercore.AccountData { + // No need to adjust unless account is suspendable + if after.Status != basics.Online || !after.IncentiveEligible { + return after + } + + // Adjust only if balance has doubled + twice, o := basics.OMul(before.MicroAlgos.Raw, 2) + if !o && twice < after.MicroAlgos.Raw { + lookback := agreement.BalanceLookback(cs.ConsensusParams()) + after.LastHeartbeat = cs.Round() + lookback + } + return after +} + func (cs *roundCowState) ConsensusParams() config.ConsensusParams { return cs.proto } diff --git a/test/e2e-go/features/incentives/whalejoin_test.go b/test/e2e-go/features/incentives/whalejoin_test.go index b3e8dac479..783a01ca33 100644 --- a/test/e2e-go/features/incentives/whalejoin_test.go +++ b/test/e2e-go/features/incentives/whalejoin_test.go @@ -24,6 +24,7 @@ import ( "github.com/stretchr/testify/require" + v2 "github.com/algorand/go-algorand/daemon/algod/api/server/v2" "github.com/algorand/go-algorand/daemon/algod/api/server/v2/generated/model" "github.com/algorand/go-algorand/data/basics" "github.com/algorand/go-algorand/data/transactions" @@ -36,8 +37,8 @@ import ( // TestWhaleJoin shows a "whale" with more stake than is currently online can go // online without immediate suspension. This tests for a bug we had where we // calcululated expected proposal interval using the _old_ totals, rather than -// the totals following the keyreg. So big joiner could be expected to propose -// in the same block they joined. +// the totals following the keyreg. So big joiner was being expected to propose +// in the same block it joined. func TestWhaleJoin(t *testing.T) { partitiontest.PartitionTest(t) defer fixtures.ShutdownSynchronizedTest(t) @@ -185,6 +186,65 @@ func TestBigJoin(t *testing.T) { // is looking for. } +// TestBigIncrease shows when an incentive eligible account receives a lot of +// algos, they are not immediately suspended. We also check the details of the +// mechanism - that LastHeartbeat is incremented when such an account doubles +// its balance in a single pay. +func TestBigIncrease(t *testing.T) { + partitiontest.PartitionTest(t) + defer fixtures.ShutdownSynchronizedTest(t) + + t.Parallel() + a := require.New(fixtures.SynchronizedTest(t)) + + var fixture fixtures.RestClientFixture + const lookback = 32 + fixture.FasterConsensus(protocol.ConsensusFuture, time.Second/2, lookback) + fixture.Setup(t, filepath.Join("nettemplates", "Payouts.json")) + defer fixture.Shutdown() + + // Overview of this test: + // 0. spend wallet01 down so it has a very small percent of stake + // 1. rereg wallet01 so it is suspendable + // 2. move almost all of wallet15's money to wallet01 + // 3. check that c1.LastHeart is set to 32 rounds later + // 4. wait 40 rounds ensure c1 stays online + + clientAndAccount := func(name string) (libgoal.Client, model.Account) { + c := fixture.GetLibGoalClientForNamedNode(name) + accounts, err := fixture.GetNodeWalletsSortedByBalance(c) + a.NoError(err) + a.Len(accounts, 1) + fmt.Printf("Client %s is %v\n", name, accounts[0].Address) + return c, accounts[0] + } + + c1, account01 := clientAndAccount("Node01") + c15, account15 := clientAndAccount("Node15") + + // We need to spend 01 down so that it has nearly no stake. That way, it + // certainly will not have proposed by pure luck just before the critical + // round. If we don't do that, 1/16 of stake is enough that it will probably + // have a fairly recent proposal, and not get knocked off. + pay(&fixture, a, c1, account01.Address, account15.Address, 99*account01.Amount/100) + + rekeyreg(&fixture, a, c1, account01.Address) + + // 2. Wait lookback rounds + wait(&fixture, a, lookback) + + tx := pay(&fixture, a, c15, account15.Address, account01.Address, 50*account15.Amount/100) + data, err := c15.AccountData(account01.Address) + a.NoError(err) + a.EqualValues(*tx.ConfirmedRound+lookback, data.LastHeartbeat) + + wait(&fixture, a, lookback+5) + data, err = c15.AccountData(account01.Address) + a.NoError(err) + a.Equal(basics.Online, data.Status) + a.True(data.IncentiveEligible) +} + func wait(f *fixtures.RestClientFixture, a *require.Assertions, count uint64) { res, err := f.AlgodClient.Status() a.NoError(err) @@ -192,12 +252,18 @@ func wait(f *fixtures.RestClientFixture, a *require.Assertions, count uint64) { a.NoError(f.WaitForRoundWithTimeout(round)) } -func zeroPay(f *fixtures.RestClientFixture, a *require.Assertions, - c libgoal.Client, address string) { - pay, err := c.SendPaymentFromUnencryptedWallet(address, address, 1000, 0, nil) +func pay(f *fixtures.RestClientFixture, a *require.Assertions, + c libgoal.Client, from string, to string, amount uint64) v2.PreEncodedTxInfo { + pay, err := c.SendPaymentFromUnencryptedWallet(from, to, 1000, amount, nil) a.NoError(err) - _, err = f.WaitForConfirmedTxn(uint64(pay.LastValid), pay.ID().String()) + tx, err := f.WaitForConfirmedTxn(uint64(pay.LastValid), pay.ID().String()) a.NoError(err) + return tx +} + +func zeroPay(f *fixtures.RestClientFixture, a *require.Assertions, + c libgoal.Client, address string) { + pay(f, a, c, address, address, 0) } // Go offline, but return the key material so it's easy to go back online From cbe9c5b6cf33c1bfda45984dac8c4504b65ec22f Mon Sep 17 00:00:00 2001 From: John Jannotti Date: Fri, 22 Nov 2024 15:45:24 -0500 Subject: [PATCH 15/19] Update data/transactions/logic/assembler.go Co-authored-by: Gary Malouf <982483+gmalouf@users.noreply.github.com> --- data/transactions/logic/assembler.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/data/transactions/logic/assembler.go b/data/transactions/logic/assembler.go index c321e981ae..9c625d3d56 100644 --- a/data/transactions/logic/assembler.go +++ b/data/transactions/logic/assembler.go @@ -2738,7 +2738,7 @@ func AssembleString(text string) (*OpStream, error) { return AssembleStringWithVersion(text, assemblerNoVersion) } -// MustAssemble assembles a program an panics on error. It is useful for +// MustAssemble assembles a program and panics on error. It is useful for // defining globals. func MustAssemble(text string) []byte { ops, err := AssembleString(text) From 2c0975f07ddfd98b8dfe2a90360baa231e543a81 Mon Sep 17 00:00:00 2001 From: John Jannotti Date: Mon, 25 Nov 2024 17:42:17 -0500 Subject: [PATCH 16/19] First cut --- heartbeat/README.md | 176 ++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 176 insertions(+) create mode 100644 heartbeat/README.md diff --git a/heartbeat/README.md b/heartbeat/README.md new file mode 100644 index 0000000000..02fec181f3 --- /dev/null +++ b/heartbeat/README.md @@ -0,0 +1,176 @@ +# Block Payouts, Suspensions, and Heartbeats + +Running a validator node on Algorand is a relatively lightweight operation. Therefore, participation +in consensus was not compensated. There was an expectation that financial motivated holders of Algos +would run nodes in order to help secure their holdings. + +Although simple participation is not terribly resource intensive, running _any_ service with high +uptime becomes expensive when one considers that it should be monitored for uptime, be somewhat +over-provisioned to handle unexpected load spikes, and plans need to be in place to restart in the +face of hardware failure (or the accounts should leave consensus properly). + +With those burdens in mind, fewer Algo holders chose to run participation nodes than would be +preferred to provide security against well-financed bad actors. To alleviate this problem, a +mechanism to reward block proposers has been created. With these _block payouts_ in place, large +Algo holders are incentivized to run participation nodes in order to earn more Algos, increasing +security for the entire Algorand network. + +With the financial incentive to run participation nodes comes the risk that some nodes may be +operated without sufficient care. Therefore, a mechanism to _suspend_ nodes that appear to +performing poorly (or not at all). Appearances can be deceiving, however. Since Algorand is a +probabilistic consensus protocol, pure chance might lead to a node appearing to be delinquent. A new +transaction type, the _heartbeat_, allows a node to explicitly indicate that it is online even if it +does not propose blocks due to "bad luck". + +# Payouts + +Payouts are made in every block, if the proposer has opted into receiving them, has an Algo balance +in an appropriate range, and has not been suspended for poor behavior since opting-in. The size of +the payout is indicated in the block header, and comes from the `FeeSink`. The block payout consist +of two components. First, a portion of the block fees (currently 50%) are paid to the proposer. +This component incentives fuller blocks which lead to larger payouts. Second, a _bonus_ payout is +made according to a exponentially decaying formula. This bonus is (intentionally) unsustainable +from protocol fees. It is expected that the Algorand Foundation will seed the `FeeSink` with +sufficient funds to allow the bonuses to be paid out according to the formula for several years. If +the `FeeSink` has insufficient funds for the sum of these components, the payout will be as high as +possible while maintaining the `FeeSink`'s minimum balance. These calculations are performed in +`endOfBlock` in `eval/eval.go`. + +To opt-in to receiving block payouts, an account includes an extra fee in the `keyreg` +transaction. The amount is controlled by the consensus parameter `Payouts.GoOnlineFee`. When such a +fee is included, a new account state bit, `IncentiveEligible` is set to true. + +Even when an account is `IncentiveEligible` there is a proposal-time check of the account's online +stake. If the account has too much or too little, no payout is performed (though +`IncentiveEligible` remains true). As explained below, this check occurs in `agreement` code in +`payoutEligible()`. The balance check is performed on the _online_ stake, that is the stake from 320 +rounds earlier, so a clever proposer can not move Algos in the round it proposes in order to receive +the payout. Finally, in an interesting corner case, a proposing account could be closed at proposal +time, since voting is based on the earlier balance. Such an account receives no payout, even if its +balances was in the proper range 320 rounds ago. + +A surprising complication in the implementation of these payouts is that when a block is prepared by +a node, it does not know which account is the proposer. Until now, `algod` could prepare a single +block which would be used by any of the accounts it was participating for. The block would be +handed off to `agreement` which would manipulate the block only to add the appropriate block seed +(which depended upon the proposer). That interaction between `eval` and `agreement` was widened +(see `WithProposer()`) to allow `agreement` to modify the block to include the proper `Proposer`, +and to zero the `ProposerPayout` if the account that proposed was not actually eligible to receive a +payout. + +# Suspensions + +Accounts can be _suspended_ for poor behavior. There are two forms of poor behavior that can lead +to suspension. First, an account is considered _absent_ if it fails to propose as often as it +should. Second, an account can be suspended for failing to respond to a _challenge_ issued by the +network at random. + +## Absenteeism + +An account can be expected to propose once every `n = TotalOnlineStake/AccountOnlineStake` rounds. +For example, a node with 2% of online stake ought to propose once every 50 rounds. Of course the +actual proposer is chosen by random sortition. To make false positive suspensions unlikely, a node +is considered absent if it fails to produce a block over the course of `10n` rounds. + +The suspension mechanism is implemented in `generateKnockOfflineAccountsList` in `eval/eval.go`. It +is closely modeled on the mechanism that knocks accounts offline if their voting keys have expired. +An absent account is added to the `AbsentParticipationAccounts` list of the block header. When +evaluating a block, accounts in `AbsentParticipationAccounts` are suspended by changing their +`Status` to `Offline` and setting `IncentiveEligible` to false, but retaining their voting keys. + +### Keyreg and `LastHeartbeat` + +As described so far, 320 rounds after a `keyreg` to go online, an account suddenly is expected to +have proposed more recently than 10 times its new expected interval. That would be impossible, since +it was not online until that round. Therefore, when a `keyreg` is used to go online and become +`IncentiveEligible`, the account's `LastHeartbeat` field is set 320 rounds into the future. In +effect, the account is treated as though it proposed in the first round it is online. + +### Large Algo increases and `LastHeartbeat` + +A similar problem can occur when an online account receives Algos. 320 rounds after receiving the +new Algos, the account's expected proposal interval will shrink. If, for example, such an account +increases by a factor of 10, then it is reasonably likely that it will not have proposed recently +enough, and will be suspended immediately. To mitigate this risk, any time an online, +`IncentiveEligible` account balance doubles from a single `Pay`, its `LastHeartbeat` is incremented +to 320 rounds past the current round. + +## Challenges + +The absenteeism checks quickly suspend a high-value account if it becomes inoperative. For example, +and account with 2% of stake can be marked absent after 500 rounds (about 24 minutes). After +suspension, the effect on consensus is mitigated after 320 more rounds (about 15 +minutes). Therefore, the suspension mechanism makes Algorand significantly more robust in the face +of operational errors. + +However, the absenteeism mechanism is very slow to notice small accounts. An account with 30,000 +Algos might represent 1/100,000 or less of total stake. It would only be considered absent after a +million or more rounds without a proposal. At current network speeds, this about a month. With such +slow detection, a financially motived entity might make the decision to run a node even if they lack +the wherewithal to run the node with excellent uptime. A worst case scenario might be a node that is +turned off daily, overnight. Such a node would generate profit for the runner, would probably never +be marked offline by the absenteeism mechanism, yet would impact consensus negatively. Algorand +can't make progress with 1/3 of nodes offline at any given time for a nightly rest. + +To combat this scenario, the network generates random _challenges_ periodically. Every +`Payouts.ChallengeInterval` rounds (currently 1000), a random selected portion (currently 1/32) of +all online accounts are challenged. They must _heartbeat_ within `Payouts.ChallengeGracePeriod` +rounds (currently 200), or they will be subject to suspension. With the current consensus +parameters, nodes can be expected to be challenged daily. When suspended, accounts must `keyreg` +with the `GoOnlineFee` in order to receive block payouts again, so it becomes unprofitable for +these low-stake nodes to operate with poor uptimes. + +# Heartbeats + +The absenteeism mechanism is subject to rare false positives. The challenge mechanism explicitly +requires an affirmative response from nodes to indicate they are operating properly on behalf of a +challenged account. Both of these needs are addressed by a new transaction type --- _Heartbeat_. A +Heartbeat transaction contains a signature (`HbProof`) of a recent block seed (`HbSeed`) under the +participation key of the account (`HbAddress`) in question. Note that the account being heartbeat +for is _not_ the `Sender` of the transaction, which can be any address. + +It is relatively easy for a bad actor to emit Heartbeats for its accounts without actually +participating. However, there is no financial incentive to do so. Pretending to be operational when +offline does not earn block payouts. Furthermore, running a server to monitor the block chain to +notice challenges and gather the recent blockseed is not significantly cheap that simply running a +functional node. It is _already_ possible for malicious, well-resourced accounts to cause consensus +difficulties by going online without actually participating. Heartbeats do not mitigate that +risk. But these mechanisms have been designed to avoid _motivating_ such behavior, so that they can +accomplish their actual goal of noticing poor behavior stemming from _inadvertent_ operational +problems. + +## Free Heartbeats + +Challenges occur frequently, so it important that `algod` can easily send Heartbeats as +required. How should these transactions be paid for? Many accounts, especially high-value accounts, +would not want to keep their spending keys available for automatic use by `algod`. Further, creating +(and keeping funded) a low-value side account to pay for Heartbeats would be an annoying operational +overhead. Therefore, when required by challenges, heartbeat transactions do not require a fee. +Therefore, any account, even an unfunded logigsig, can send heartbeats for an account under +challenge. + +The conditions for a free Heartbeat are: + +1. The Heartbeat is not part of a larger group, and has a zero `GroupID`. +1. The `HbAddress` is Online and under challenge with the grace period at least half over. +1. The `HbAddress` is `IncentiveEligible`. +1. There is no `Note`, `Lease`, or `RekeyTo`. + +## Heartbeat Service + +The Heartbeat Service (`heartbeat/service.go`) watches the state of all acounts for which `algod` +has participation keys. If any of those acounts meets the requirements above, a heartbeat +transaction is sent, starting with the round following half a grace period from the challenge. It +uses the (presumably unfunded) logicsig that does nothing except preclude rekey operations. + +The heartbeat service does _not_ heartbeat if an account is unlucky and threatened to be considered +absent. We presume such false postives to be so unlikely that, if they occur, the node must be +brought back online manually. It would be reasonable to consider: + +1. Making heartbeats free for accounts that are "nearly absent". + +or + +2. Allowing for paid heartbeats by the heartbeat service when configured with access to a funded + account's spending key. + From d8e3989f00ecc8cea4277132416f31f3c2bcccc0 Mon Sep 17 00:00:00 2001 From: John Jannotti Date: Tue, 26 Nov 2024 11:32:01 -0500 Subject: [PATCH 17/19] CR from @gmalouf --- config/consensus_test.go | 5 +++++ heartbeat/README.md | 11 +++++------ ledger/apply/heartbeat_test.go | 3 +++ 3 files changed, 13 insertions(+), 6 deletions(-) diff --git a/config/consensus_test.go b/config/consensus_test.go index d7fbed4085..635ad5699c 100644 --- a/config/consensus_test.go +++ b/config/consensus_test.go @@ -37,6 +37,11 @@ func TestConsensusParams(t *testing.T) { if params.ApplyData && params.PaysetCommit == PaysetCommitUnsupported { t.Errorf("Protocol %s: ApplyData with PaysetCommitUnsupported", proto) } + + // To figure out challenges, nodes must be able to lookup headers up to two GracePeriods back + if 2*params.Payouts.ChallengeGracePeriod > params.MaxTxnLife+params.DeeperBlockHeaderHistory { + t.Errorf("Protocol %s: Grace period is too long", proto) + } } } diff --git a/heartbeat/README.md b/heartbeat/README.md index 02fec181f3..acee4ed1e4 100644 --- a/heartbeat/README.md +++ b/heartbeat/README.md @@ -16,7 +16,7 @@ Algo holders are incentivized to run participation nodes in order to earn more A security for the entire Algorand network. With the financial incentive to run participation nodes comes the risk that some nodes may be -operated without sufficient care. Therefore, a mechanism to _suspend_ nodes that appear to +operated without sufficient care. Therefore, a mechanism to _suspend_ nodes that appear to be performing poorly (or not at all). Appearances can be deceiving, however. Since Algorand is a probabilistic consensus protocol, pure chance might lead to a node appearing to be delinquent. A new transaction type, the _heartbeat_, allows a node to explicitly indicate that it is online even if it @@ -105,7 +105,7 @@ of operational errors. However, the absenteeism mechanism is very slow to notice small accounts. An account with 30,000 Algos might represent 1/100,000 or less of total stake. It would only be considered absent after a -million or more rounds without a proposal. At current network speeds, this about a month. With such +million or more rounds without a proposal. At current network speeds, this is about a month. With such slow detection, a financially motived entity might make the decision to run a node even if they lack the wherewithal to run the node with excellent uptime. A worst case scenario might be a node that is turned off daily, overnight. Such a node would generate profit for the runner, would probably never @@ -132,7 +132,7 @@ for is _not_ the `Sender` of the transaction, which can be any address. It is relatively easy for a bad actor to emit Heartbeats for its accounts without actually participating. However, there is no financial incentive to do so. Pretending to be operational when offline does not earn block payouts. Furthermore, running a server to monitor the block chain to -notice challenges and gather the recent blockseed is not significantly cheap that simply running a +notice challenges and gather the recent blockseed is not significantly cheaper that simply running a functional node. It is _already_ possible for malicious, well-resourced accounts to cause consensus difficulties by going online without actually participating. Heartbeats do not mitigate that risk. But these mechanisms have been designed to avoid _motivating_ such behavior, so that they can @@ -159,13 +159,13 @@ The conditions for a free Heartbeat are: ## Heartbeat Service The Heartbeat Service (`heartbeat/service.go`) watches the state of all acounts for which `algod` -has participation keys. If any of those acounts meets the requirements above, a heartbeat +has participation keys. If any of those accounts meets the requirements above, a heartbeat transaction is sent, starting with the round following half a grace period from the challenge. It uses the (presumably unfunded) logicsig that does nothing except preclude rekey operations. The heartbeat service does _not_ heartbeat if an account is unlucky and threatened to be considered absent. We presume such false postives to be so unlikely that, if they occur, the node must be -brought back online manually. It would be reasonable to consider: +brought back online manually. It would be reasonable to consider in the future: 1. Making heartbeats free for accounts that are "nearly absent". @@ -173,4 +173,3 @@ or 2. Allowing for paid heartbeats by the heartbeat service when configured with access to a funded account's spending key. - diff --git a/ledger/apply/heartbeat_test.go b/ledger/apply/heartbeat_test.go index f4468ab6e7..9310876e2d 100644 --- a/ledger/apply/heartbeat_test.go +++ b/ledger/apply/heartbeat_test.go @@ -105,6 +105,9 @@ func TestHeartbeat(t *testing.T) { // being heartbeat for is online, under risk of suspension by challenge, and // incentive eligible. func TestCheapRules(t *testing.T) { + partitiontest.PartitionTest(t) + t.Parallel() + type tcase struct { rnd basics.Round addrStart byte From 974f3578026d810bf20183bc77e583cf7ce549a2 Mon Sep 17 00:00:00 2001 From: John Jannotti Date: Tue, 26 Nov 2024 12:17:14 -0500 Subject: [PATCH 18/19] Use firstValid blockseed in heartbeat More careful heartbeat in service.go so we don't heartbeat every round, and we use a lower firstValid in case we are slightly ahead of the nodes we send it to. We don't want them to drop it. --- heartbeat/README.md | 7 ++++--- heartbeat/service.go | 23 +++++++++++++++++------ ledger/apply/heartbeat.go | 8 ++++---- ledger/apply/heartbeat_test.go | 2 +- 4 files changed, 26 insertions(+), 14 deletions(-) diff --git a/heartbeat/README.md b/heartbeat/README.md index acee4ed1e4..23c05b670f 100644 --- a/heartbeat/README.md +++ b/heartbeat/README.md @@ -125,9 +125,10 @@ these low-stake nodes to operate with poor uptimes. The absenteeism mechanism is subject to rare false positives. The challenge mechanism explicitly requires an affirmative response from nodes to indicate they are operating properly on behalf of a challenged account. Both of these needs are addressed by a new transaction type --- _Heartbeat_. A -Heartbeat transaction contains a signature (`HbProof`) of a recent block seed (`HbSeed`) under the -participation key of the account (`HbAddress`) in question. Note that the account being heartbeat -for is _not_ the `Sender` of the transaction, which can be any address. +Heartbeat transaction contains a signature (`HbProof`) of the blockseed (`HbSeed`) of the +transaction's firstValid block under the participation key of the account (`HbAddress`) in +question. Note that the account being heartbeat for is _not_ the `Sender` of the transaction, which +can be any address. It is relatively easy for a bad actor to emit Heartbeats for its accounts without actually participating. However, there is no financial incentive to do so. Pretending to be operational when diff --git a/heartbeat/service.go b/heartbeat/service.go index 741d004272..c3d66239e2 100644 --- a/heartbeat/service.go +++ b/heartbeat/service.go @@ -77,9 +77,7 @@ func (s *Service) Stop() { // findChallenged() returns a list of accounts that need a heartbeat because // they have been challenged. -func (s *Service) findChallenged(rules config.ProposerPayoutRules) []account.ParticipationRecordForRound { - current := s.ledger.LastRound() - +func (s *Service) findChallenged(rules config.ProposerPayoutRules, current basics.Round) []account.ParticipationRecordForRound { ch := apply.FindChallenge(rules, current, s.ledger, apply.ChRisky) if ch.IsZero() { return nil @@ -114,6 +112,7 @@ func (s *Service) findChallenged(rules config.ProposerPayoutRules) []account.Par // designed to be extremely unlikely anyway.) func (s *Service) loop() { defer s.wg.Done() + suppress := make(map[basics.Address]basics.Round) latest := s.ledger.LastRound() for { // exit if Done, else wait for next round @@ -132,12 +131,20 @@ func (s *Service) loop() { } proto := config.Consensus[lastHdr.CurrentProtocol] - for _, pr := range s.findChallenged(proto.Payouts) { + for _, pr := range s.findChallenged(proto.Payouts, latest) { + if suppress[pr.Account] > latest { + continue + } stxn := s.prepareHeartbeat(pr, lastHdr) s.log.Infof("sending heartbeat %v for %v\n", stxn.Txn.HeartbeatTxnFields, pr.Account) err = s.bcast.BroadcastInternalSignedTxGroup([]transactions.SignedTxn{stxn}) if err != nil { s.log.Errorf("error broadcasting heartbeat %v for %v: %v", stxn, pr.Account, err) + } else { + // Don't bother heartbeating again until the last one expires. + // If it is accepted, we won't need to (because we won't be + // under challenge any more). + suppress[pr.Account] = stxn.Txn.LastValid } } } @@ -150,14 +157,18 @@ txn RekeyTo; global ZeroAddress; == `) var acceptingSender = basics.Address(logic.HashProgram(acceptingByteCode)) +// hbLifetime is somewhat short. It seems better to try several times during the +// grace period than to try a single time with a longer lifetime. +const hbLifetime = 10 + func (s *Service) prepareHeartbeat(pr account.ParticipationRecordForRound, latest bookkeeping.BlockHeader) transactions.SignedTxn { var stxn transactions.SignedTxn stxn.Lsig = transactions.LogicSig{Logic: acceptingByteCode} stxn.Txn.Type = protocol.HeartbeatTx stxn.Txn.Header = transactions.Header{ Sender: acceptingSender, - FirstValid: latest.Round + 1, - LastValid: latest.Round + 1 + 100, // maybe use the grace period? + FirstValid: latest.Round, + LastValid: latest.Round + hbLifetime, GenesisHash: latest.GenesisHash, } diff --git a/ledger/apply/heartbeat.go b/ledger/apply/heartbeat.go index a0e57849bf..9e1056dc3c 100644 --- a/ledger/apply/heartbeat.go +++ b/ledger/apply/heartbeat.go @@ -97,15 +97,15 @@ func Heartbeat(hb transactions.HeartbeatTxnFields, header transactions.Header, b // heartbeats are expected to sign with the partkey for their last-valid round id := basics.OneTimeIDForRound(header.LastValid, kd) - // heartbeats sign a message consisting of the BlockSeed of the round before - // first-valid, to discourage unsavory behaviour like presigning a bunch of + // heartbeats sign a message consisting of the BlockSeed of the first-valid + // round, to discourage unsavory behaviour like presigning a bunch of // heartbeats for later use keeping an unavailable account online. - hdr, err := provider.BlockHdr(header.FirstValid - 1) + hdr, err := provider.BlockHdr(header.FirstValid) if err != nil { return err } if hdr.Seed != hb.HbSeed { - return fmt.Errorf("provided seed %v does not match round %d's seed %v", hb.HbSeed, header.FirstValid-1, hdr.Seed) + return fmt.Errorf("provided seed %v does not match round %d's seed %v", hb.HbSeed, header.FirstValid, hdr.Seed) } if !sv.Verify(id, hdr.Seed, hb.HbProof.ToOneTimeSignature()) { diff --git a/ledger/apply/heartbeat_test.go b/ledger/apply/heartbeat_test.go index 9310876e2d..f7d3845b12 100644 --- a/ledger/apply/heartbeat_test.go +++ b/ledger/apply/heartbeat_test.go @@ -62,7 +62,7 @@ func TestHeartbeat(t *testing.T) { seed := committee.Seed{0x01, 0x02, 0x03} mockHdr := makeMockHeaders(bookkeeping.BlockHeader{ - Round: fv - 1, + Round: fv, Seed: seed, }) From 1cb12d3c0233a86939fb0735d502d7588bdab58f Mon Sep 17 00:00:00 2001 From: John Jannotti Date: Wed, 27 Nov 2024 11:38:24 -0500 Subject: [PATCH 19/19] Adjust tests and readme for signing FirstValid seed --- heartbeat/README.md | 16 +++++--- ledger/heartbeat_test.go | 88 ++++++++++++++++++++++------------------ ledger/simple_test.go | 2 +- 3 files changed, 60 insertions(+), 46 deletions(-) diff --git a/heartbeat/README.md b/heartbeat/README.md index 23c05b670f..f1a522928d 100644 --- a/heartbeat/README.md +++ b/heartbeat/README.md @@ -126,19 +126,23 @@ The absenteeism mechanism is subject to rare false positives. The challenge mec requires an affirmative response from nodes to indicate they are operating properly on behalf of a challenged account. Both of these needs are addressed by a new transaction type --- _Heartbeat_. A Heartbeat transaction contains a signature (`HbProof`) of the blockseed (`HbSeed`) of the -transaction's firstValid block under the participation key of the account (`HbAddress`) in +transaction's FirstValid block under the participation key of the account (`HbAddress`) in question. Note that the account being heartbeat for is _not_ the `Sender` of the transaction, which -can be any address. +can be any address. Signing a recent block seed makes it more difficult to pre-sign heartbeats that +another machine might send on your behalf. Signing the FirstValid's blockseed (rather than +FirstValid-1) simply enforces a best practice: emit a transaction with FirstValid set to a committed +round, not a future round, avoiding a race. The node you send transactions to might not have +committed your latest round yet. It is relatively easy for a bad actor to emit Heartbeats for its accounts without actually participating. However, there is no financial incentive to do so. Pretending to be operational when offline does not earn block payouts. Furthermore, running a server to monitor the block chain to notice challenges and gather the recent blockseed is not significantly cheaper that simply running a functional node. It is _already_ possible for malicious, well-resourced accounts to cause consensus -difficulties by going online without actually participating. Heartbeats do not mitigate that -risk. But these mechanisms have been designed to avoid _motivating_ such behavior, so that they can -accomplish their actual goal of noticing poor behavior stemming from _inadvertent_ operational -problems. +difficulties by putting significant stake online without actually participating. Heartbeats do not +mitigate that risk. But these mechanisms have been designed to avoid _motivating_ such behavior, so +that they can accomplish their actual goal of noticing poor behavior stemming from _inadvertent_ +operational problems. ## Free Heartbeats diff --git a/ledger/heartbeat_test.go b/ledger/heartbeat_test.go index e2ed851783..2af0310fef 100644 --- a/ledger/heartbeat_test.go +++ b/ledger/heartbeat_test.go @@ -34,7 +34,7 @@ import ( a more realistic ledger. */ // TestHearbeat exercises heartbeat transactions -func TestHeartBeat(t *testing.T) { +func TestHeartbeat(t *testing.T) { partitiontest.PartitionTest(t) t.Parallel() @@ -48,6 +48,8 @@ func TestHeartBeat(t *testing.T) { dl := NewDoubleLedger(t, genBalances, cv, cfg) defer dl.Close() + dl.txns() // tests involving seed are easier if we have the first block in ledger + // empty HbAddress means ZeroAddress, and it's not online dl.txn(&txntest.Txn{Type: "hb", Sender: addrs[1]}, "AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAY5HFKQ has no voting keys") @@ -57,17 +59,18 @@ func TestHeartBeat(t *testing.T) { addrs[2].String()+" has no voting keys") // addrs[1] is online, it has voting keys, but seed is missing - dl.txn(&txntest.Txn{Type: "hb", Sender: addrs[1], HbAddress: addrs[1]}, - "does not match round 0's seed") + dl.txn(&txntest.Txn{Type: "hb", Sender: addrs[1], HbAddress: addrs[1], FirstValid: 1}, + "does not match round 1's seed") // NewTestGenesis creates random VoterID. Verification will fail. - b0, err := dl.generator.BlockHdr(0) + b1, err := dl.generator.BlockHdr(1) require.NoError(t, err) dl.txn(&txntest.Txn{ - Type: "hb", - Sender: addrs[1], - HbAddress: addrs[1], - HbSeed: b0.Seed, + Type: "hb", + Sender: addrs[1], + HbAddress: addrs[1], + HbSeed: b1.Seed, + FirstValid: 1, }, "heartbeat failed verification with") @@ -84,61 +87,68 @@ func TestHeartBeat(t *testing.T) { }) // Supply and sign the wrong HbSeed + wrong := b1.Seed + wrong[0]++ dl.txn(&txntest.Txn{ - Type: "hb", - Sender: addrs[1], - HbAddress: addrs[1], - HbSeed: b0.Seed, - HbProof: otss.Sign(firstID, b0.Seed).ToHeartbeatProof(), + Type: "hb", + Sender: addrs[1], + HbAddress: addrs[1], + HbSeed: wrong, + HbProof: otss.Sign(firstID, wrong).ToHeartbeatProof(), + FirstValid: 1, }, "does not match round 1's seed") - b1, err := dl.generator.BlockHdr(1) + b2, err := dl.generator.BlockHdr(2) require.NoError(t, err) // Supply the right seed, but sign something else. We're also now // setting LastValid and the proper OneTimeIDForRound, so that these // tests are failing for the reasons described, not that. dl.txn(&txntest.Txn{ - Type: "hb", - LastValid: 30, - Sender: addrs[1], - HbAddress: addrs[1], - HbSeed: b1.Seed, - HbProof: otss.Sign(basics.OneTimeIDForRound(30, kd), b0.Seed).ToHeartbeatProof(), + Type: "hb", + LastValid: 30, + Sender: addrs[1], + HbAddress: addrs[1], + HbSeed: b2.Seed, + HbProof: otss.Sign(basics.OneTimeIDForRound(30, kd), wrong).ToHeartbeatProof(), + FirstValid: 2, }, "failed verification") // Sign the right seed, but supply something else dl.txn(&txntest.Txn{ - Type: "hb", - LastValid: 30, - Sender: addrs[1], - HbAddress: addrs[1], - HbSeed: b0.Seed, - HbProof: otss.Sign(basics.OneTimeIDForRound(30, kd), b1.Seed).ToHeartbeatProof(), + Type: "hb", + LastValid: 30, + Sender: addrs[1], + HbAddress: addrs[1], + HbSeed: wrong, + HbProof: otss.Sign(basics.OneTimeIDForRound(30, kd), b2.Seed).ToHeartbeatProof(), + FirstValid: 2, }, - "does not match round 1's") + "does not match round 2's") // Mismatch the last valid and OneTimeIDForRound dl.txn(&txntest.Txn{ - Type: "hb", - LastValid: 29, - Sender: addrs[1], - HbAddress: addrs[1], - HbSeed: b1.Seed, - HbProof: otss.Sign(basics.OneTimeIDForRound(30, kd), b1.Seed).ToHeartbeatProof(), + Type: "hb", + LastValid: 29, + Sender: addrs[1], + HbAddress: addrs[1], + HbSeed: b2.Seed, + HbProof: otss.Sign(basics.OneTimeIDForRound(30, kd), b2.Seed).ToHeartbeatProof(), + FirstValid: 2, }, "failed verification") // now we can make a real heartbeat, with a properly signed blockseed dl.txn(&txntest.Txn{ - Type: "hb", - LastValid: 30, - Sender: addrs[1], - HbAddress: addrs[1], - HbSeed: b1.Seed, - HbProof: otss.Sign(basics.OneTimeIDForRound(30, kd), b1.Seed).ToHeartbeatProof(), + Type: "hb", + LastValid: 30, + Sender: addrs[1], + HbAddress: addrs[1], + HbSeed: b2.Seed, + HbProof: otss.Sign(basics.OneTimeIDForRound(30, kd), b2.Seed).ToHeartbeatProof(), + FirstValid: 2, }) }) diff --git a/ledger/simple_test.go b/ledger/simple_test.go index 0995f88ecc..328f6f1528 100644 --- a/ledger/simple_test.go +++ b/ledger/simple_test.go @@ -129,7 +129,7 @@ func txn(t testing.TB, ledger *Ledger, eval *eval.BlockEvaluator, txn *txntest.T } return } - require.True(t, len(problem) == 0 || problem[0] == "") + require.True(t, len(problem) == 0 || problem[0] == "", "Transaction did not fail. Expected: %v", problem) } func txgroup(t testing.TB, ledger *Ledger, eval *eval.BlockEvaluator, txns ...*txntest.Txn) error {