diff --git a/data/pools/transactionPool.go b/data/pools/transactionPool.go index 664e273804..4295e82ff1 100644 --- a/data/pools/transactionPool.go +++ b/data/pools/transactionPool.go @@ -170,6 +170,9 @@ var ErrStaleBlockAssemblyRequest = fmt.Errorf("AssembleBlock: requested block as // Reset resets the content of the transaction pool func (pool *TransactionPool) Reset() { + pool.mu.Lock() + defer pool.mu.Unlock() + defer pool.cond.Broadcast() pool.pendingTxids = make(map[transactions.Txid]transactions.SignedTxn) pool.pendingTxGroups = nil pool.rememberedTxids = make(map[transactions.Txid]transactions.SignedTxn) diff --git a/ledger/acctupdates.go b/ledger/acctupdates.go index fa8fc44dc9..16d933fbb5 100644 --- a/ledger/acctupdates.go +++ b/ledger/acctupdates.go @@ -20,13 +20,9 @@ import ( "container/heap" "context" "database/sql" - "encoding/hex" "fmt" "io" - "os" - "path/filepath" "sort" - "strconv" "sync" "sync/atomic" "time" @@ -35,7 +31,6 @@ import ( "github.com/algorand/go-algorand/config" "github.com/algorand/go-algorand/crypto" - "github.com/algorand/go-algorand/crypto/merkletrie" "github.com/algorand/go-algorand/data/basics" "github.com/algorand/go-algorand/data/bookkeeping" "github.com/algorand/go-algorand/data/transactions" @@ -52,24 +47,8 @@ const ( balancesFlushInterval = 5 * time.Second // pendingDeltasFlushThreshold is the deltas count threshold above we flush the pending balances regardless of the flush interval. pendingDeltasFlushThreshold = 128 - // trieRebuildAccountChunkSize defines the number of accounts that would get read at a single chunk - // before added to the trie during trie construction - trieRebuildAccountChunkSize = 16384 - // trieRebuildCommitFrequency defines the number of accounts that would get added before we call evict to commit the changes and adjust the memory cache. - trieRebuildCommitFrequency = 65536 - // trieAccumulatedChangesFlush defines the number of pending changes that would be applied to the merkle trie before - // we attempt to commit them to disk while writing a batch of rounds balances to disk. - trieAccumulatedChangesFlush = 256 ) -// trieCachedNodesCount defines how many balances trie nodes we would like to keep around in memory. -// value was calibrated using BenchmarkCalibrateCacheNodeSize -var trieCachedNodesCount = 9000 - -// merkleCommitterNodesPerPage controls how many nodes will be stored in a single page -// value was calibrated using BenchmarkCalibrateNodesPerPage -var merkleCommitterNodesPerPage = int64(116) - // baseAccountsPendingAccountsBufferSize defines the size of the base account pending accounts buffer size. // At the beginning of a new round, the entries from this buffer are being flushed into the base accounts map. const baseAccountsPendingAccountsBufferSize = 100000 @@ -99,14 +78,6 @@ const initializingAccountCachesMessageTimeout = 3 * time.Second // where we end up batching up to 1000 rounds in a single update. const accountsUpdatePerRoundHighWatermark = 1 * time.Second -// TrieMemoryConfig is the memory configuration setup used for the merkle trie. -var TrieMemoryConfig = merkletrie.MemoryConfig{ - NodesCountPerPage: merkleCommitterNodesPerPage, - CachedNodesCount: trieCachedNodesCount, - PageFillFactor: 0.95, - MaxChildrenPagesThreshold: 64, -} - // A modifiedAccount represents an account that has been modified since // the persistent state stored in the account DB (i.e., in the range of // rounds covered by the accountUpdates tracker). @@ -125,19 +96,9 @@ type modifiedAccount struct { type accountUpdates struct { // constant variables ( initialized on initialize, and never changed afterward ) - // dbDirectory is the directory where the ledger and block sql file resides as well as the parent directory for the catchup files to be generated - dbDirectory string - - // catchpointInterval is the configured interval at which the accountUpdates would generate catchpoint labels and catchpoint files. - catchpointInterval uint64 - // archivalLedger determines whether the associated ledger was configured as archival ledger or not. archivalLedger bool - // catchpointFileHistoryLength defines how many catchpoint files we want to store back. - // 0 means don't store any, -1 mean unlimited and positive number suggest the number of most recent catchpoint files. - catchpointFileHistoryLength int - // dynamic variables // Connection to the database. @@ -172,44 +133,14 @@ type accountUpdates struct { // i.e., totals is one longer than deltas. roundTotals []ledgercore.AccountTotals - // roundDigest stores the digest of the block for every round starting with dbRound and every round after it. - roundDigest []crypto.Digest - // log copied from ledger log logging.Logger - // lastFlushTime is the time we last flushed updates to - // the accounts DB (bumping dbRound). - lastFlushTime time.Time - // ledger is the source ledger, which is used to synchronize // the rounds at which we need to flush the balances to disk // in favor of the catchpoint to be generated. ledger ledgerForTracker - // The Trie tracking the current account balances. Always matches the balances that were - // written to the database. - balancesTrie *merkletrie.Trie - - // The last catchpoint label that was written to the database. Should always align with what's in the database. - // note that this is the last catchpoint *label* and not the catchpoint file. - lastCatchpointLabel string - - // catchpointWriting help to synchronize the catchpoint file writing. When this atomic variable is 0, no writing is going on. - // Any non-zero value indicates a catchpoint being written. - catchpointWriting int32 - - // catchpointSlowWriting suggest to the accounts writer that it should finish writing up the catchpoint file ASAP. - // when this channel is closed, the accounts writer would try and complete the writing as soon as possible. - // otherwise, it would take it's time and perform periodic sleeps between chunks processing. - catchpointSlowWriting chan struct{} - - // ctx is the context for the committing go-routine. It's also used as the "parent" of the catchpoint generation operation. - ctx context.Context - - // ctxCancel is the canceling function for canceling the committing go-routine ( i.e. signaling the committing go-routine that it's time to abort ) - ctxCancel context.CancelFunc - // deltasAccum stores the accumulated deltas for every round starting dbRound-1. deltasAccum []int @@ -225,12 +156,6 @@ type accountUpdates struct { // baseAccounts stores the most recently used accounts, at exactly dbRound baseAccounts lruAccounts - // the synchronous mode that would be used for the account database. - synchronousMode db.SynchronousMode - - // the synchronous mode that would be used while the accounts database is being rebuilt. - accountsRebuildSynchronousMode db.SynchronousMode - // logAccountUpdatesMetrics is a flag for enable/disable metrics logging logAccountUpdatesMetrics bool @@ -281,44 +206,16 @@ func (e *MismatchingDatabaseRoundError) Error() string { } // initialize initializes the accountUpdates structure -func (au *accountUpdates) initialize(cfg config.Local, dbPathPrefix string) { - au.dbDirectory = filepath.Dir(dbPathPrefix) +func (au *accountUpdates) initialize(cfg config.Local) { au.archivalLedger = cfg.Archival - switch cfg.CatchpointTracking { - case -1: - au.catchpointInterval = 0 - default: - // give a warning, then fall thought - logging.Base().Warnf("accountUpdates: the CatchpointTracking field in the config.json file contains an invalid value (%d). The default value of 0 would be used instead.", cfg.CatchpointTracking) - fallthrough - case 0: - if au.archivalLedger { - au.catchpointInterval = cfg.CatchpointInterval - } else { - au.catchpointInterval = 0 - } - case 1: - au.catchpointInterval = cfg.CatchpointInterval - } - - au.catchpointFileHistoryLength = cfg.CatchpointFileHistoryLength - if cfg.CatchpointFileHistoryLength < -1 { - au.catchpointFileHistoryLength = -1 - } au.accountsReadCond = sync.NewCond(au.accountsMu.RLocker()) - au.synchronousMode = db.SynchronousMode(cfg.LedgerSynchronousMode) - au.accountsRebuildSynchronousMode = db.SynchronousMode(cfg.AccountsRebuildSynchronousMode) // log metrics au.logAccountUpdatesMetrics = cfg.EnableAccountUpdatesStats au.logAccountUpdatesInterval = cfg.AccountUpdatesStatsInterval } -func (au *accountUpdates) catchpointEnabled() bool { - return au.catchpointInterval != 0 -} - // loadFromDisk is the 2nd level initialization, and is required before the accountUpdates becomes functional // The close function is expected to be call in pair with loadFromDisk func (au *accountUpdates) loadFromDisk(l ledgerForTracker, lastBalancesRound basics.Round) error { @@ -326,32 +223,10 @@ func (au *accountUpdates) loadFromDisk(l ledgerForTracker, lastBalancesRound bas defer au.accountsMu.Unlock() au.cachedDBRound = lastBalancesRound - lastestBlockRound := l.Latest() err := au.initializeFromDisk(l, lastBalancesRound) if err != nil { return err } - - writingCatchpointRound, _, err := au.accountsq.readCatchpointStateUint64(context.Background(), catchpointStateWritingCatchpoint) - if err != nil { - return err - } - - writingCatchpointDigest, err := au.initializeCaches(lastBalancesRound, lastestBlockRound, basics.Round(writingCatchpointRound)) - if err != nil { - return err - } - - if writingCatchpointRound != 0 && au.catchpointEnabled() { - au.generateCatchpoint(basics.Round(writingCatchpointRound), au.lastCatchpointLabel, writingCatchpointDigest, time.Duration(0)) - } - - au.voters = &votersTracker{} - err = au.voters.loadFromDisk(l, au) - if err != nil { - return err - } - return nil } @@ -360,19 +235,10 @@ func (au *accountUpdates) close() { if au.voters != nil { au.voters.close() } - if au.ctxCancel != nil { - au.ctxCancel() - } au.baseAccounts.prune(0) } -// IsWritingCatchpointFile returns true when a catchpoint file is being generated. The function is used by the catchup service -// to avoid memory pressure until the catchpoint file writing is complete. -func (au *accountUpdates) IsWritingCatchpointFile() bool { - return atomic.LoadInt32(&au.catchpointWriting) != 0 -} - // LookupWithRewards returns the account data for a given address at a given round. // Note that the function doesn't update the account with the rewards, // even while it does return the AccountData which represent the "rewarded" account data. @@ -608,24 +474,22 @@ func (au *accountUpdates) onlineTop(rnd basics.Round, voteRnd basics.Round, n ui } } -// GetLastCatchpointLabel retrieves the last catchpoint label that was stored to the database. -func (au *accountUpdates) GetLastCatchpointLabel() string { - au.accountsMu.RLock() - defer au.accountsMu.RUnlock() - return au.lastCatchpointLabel -} - // GetCreatorForRound returns the creator for a given asset/app index at a given round func (au *accountUpdates) GetCreatorForRound(rnd basics.Round, cidx basics.CreatableIndex, ctype basics.CreatableType) (creator basics.Address, ok bool, err error) { return au.getCreatorForRound(rnd, cidx, ctype, true /* take the lock */) } -func (au *accountUpdates) committedUpTo(committedRound basics.Round) (retRound basics.Round) { +// committedUpTo implements the ledgerTracker interface for accountUpdates. +// The method informs the tracker that committedRound and all it's previous rounds have +// been committed to the block database. The method returns what is the oldest round +// number that can be removed from the blocks database as well as the lookback that this +// tracker maintains. +func (au *accountUpdates) committedUpTo(committedRound basics.Round) (retRound, lookback basics.Round) { au.accountsMu.RLock() defer au.accountsMu.RUnlock() retRound = basics.Round(0) - lookback := basics.Round(config.Consensus[au.versions[len(au.versions)-1]].MaxBalLookback) + lookback = basics.Round(config.Consensus[au.versions[len(au.versions)-1]].MaxBalLookback) if committedRound < lookback { return } @@ -640,61 +504,23 @@ func (au *accountUpdates) committedUpTo(committedRound basics.Round) (retRound b // operation to a syncer goroutine. The one caveat is that when storing a catchpoint round, we would want to // wait until the catchpoint creation is done, so that the persistence of the catchpoint file would have an // uninterrupted view of the balances at a given point of time. -func (au *accountUpdates) produceCommittingTask(committedRound basics.Round, dbRound basics.Round) (dc deferredCommit) { - var isCatchpointRound, hasMultipleIntermediateCatchpoint bool +func (au *accountUpdates) produceCommittingTask(committedRound basics.Round, dbRound basics.Round, dcr *deferredCommitRange) *deferredCommitRange { var offset uint64 au.accountsMu.RLock() defer au.accountsMu.RUnlock() - var pendingDeltas int - lookback := basics.Round(config.Consensus[au.versions[len(au.versions)-1]].MaxBalLookback) - if committedRound < lookback { - return + if committedRound < dcr.lookback { + return nil } - newBase := committedRound - lookback + newBase := committedRound - dcr.lookback if newBase <= dbRound { // Already forgotten - return + return nil } if newBase > dbRound+basics.Round(len(au.deltas)) { - au.log.Panicf("produceCommittingTask: block %d too far in the future, lookback %d, dbRound %d (cached %d), deltas %d", committedRound, lookback, dbRound, au.cachedDBRound, len(au.deltas)) - } - - hasIntermediateCatchpoint := false - hasMultipleIntermediateCatchpoint = false - // check if there was a catchpoint between au.dbRound+lookback and newBase+lookback - if au.catchpointInterval > 0 { - nextCatchpointRound := ((uint64(dbRound+lookback) + au.catchpointInterval) / au.catchpointInterval) * au.catchpointInterval - - if nextCatchpointRound < uint64(newBase+lookback) { - mostRecentCatchpointRound := (uint64(committedRound) / au.catchpointInterval) * au.catchpointInterval - newBase = basics.Round(nextCatchpointRound) - lookback - if mostRecentCatchpointRound > nextCatchpointRound { - hasMultipleIntermediateCatchpoint = true - // skip if there is more than one catchpoint in queue - newBase = basics.Round(mostRecentCatchpointRound) - lookback - } - hasIntermediateCatchpoint = true - } - } - - // if we're still writing the previous balances, we can't move forward yet. - if au.IsWritingCatchpointFile() { - // if we hit this path, it means that we're still writing a catchpoint. - // see if the new delta range contains another catchpoint. - if hasIntermediateCatchpoint { - // check if we're already attempting to perform fast-writing. - select { - case <-au.catchpointSlowWriting: - // yes, we're already doing fast-writing. - default: - // no, we're not yet doing fast writing, make it so. - close(au.catchpointSlowWriting) - } - } - return + au.log.Panicf("produceCommittingTask: block %d too far in the future, lookback %d, dbRound %d (cached %d), deltas %d", committedRound, dcr.lookback, dbRound, au.cachedDBRound, len(au.deltas)) } if au.voters != nil { @@ -705,38 +531,15 @@ func (au *accountUpdates) produceCommittingTask(committedRound basics.Round, dbR offset = au.consecutiveVersion(offset) - // check to see if this is a catchpoint round - isCatchpointRound = au.isCatchpointRound(offset, dbRound, lookback) - // calculate the number of pending deltas - pendingDeltas = au.deltasAccum[offset] - au.deltasAccum[0] - - // If we recently flushed, wait to aggregate some more blocks. - // ( unless we're creating a catchpoint, in which case we want to flush it right away - // so that all the instances of the catchpoint would contain exactly the same data ) - flushTime := time.Now() - if !flushTime.After(au.lastFlushTime.Add(balancesFlushInterval)) && !isCatchpointRound && pendingDeltas < pendingDeltasFlushThreshold { - return - } - - if isCatchpointRound && au.archivalLedger { - // store non-zero ( all ones ) into the catchpointWriting atomic variable to indicate that a catchpoint is being written ( or, queued to be written ) - atomic.StoreInt32(&au.catchpointWriting, int32(-1)) - au.catchpointSlowWriting = make(chan struct{}, 1) - if hasMultipleIntermediateCatchpoint { - close(au.catchpointSlowWriting) - } - } + dcr.pendingDeltas = au.deltasAccum[offset] - au.deltasAccum[0] // submit committing task only if offset is non-zero in addition to // 1) no pending catchpoint writes // 2) batching requirements meet or catchpoint round - dc = deferredCommit{ - offset: offset, - dbRound: dbRound, - lookback: lookback, - } - return + dcr.oldBase = dbRound + dcr.offset = offset + return dcr } func (au *accountUpdates) consecutiveVersion(offset uint64) uint64 { @@ -799,64 +602,6 @@ func (r *readCloseSizer) Size() (int64, error) { return r.size, nil } -// GetCatchpointStream returns a ReadCloseSizer to the catchpoint file associated with the provided round -func (au *accountUpdates) GetCatchpointStream(round basics.Round) (ReadCloseSizer, error) { - dbFileName := "" - fileSize := int64(0) - start := time.Now() - ledgerGetcatchpointCount.Inc(nil) - err := au.dbs.Rdb.Atomic(func(ctx context.Context, tx *sql.Tx) (err error) { - dbFileName, _, fileSize, err = getCatchpoint(tx, round) - return - }) - ledgerGetcatchpointMicros.AddMicrosecondsSince(start, nil) - if err != nil && err != sql.ErrNoRows { - // we had some sql error. - return nil, fmt.Errorf("accountUpdates: getCatchpointStream: unable to lookup catchpoint %d: %v", round, err) - } - if dbFileName != "" { - catchpointPath := filepath.Join(au.dbDirectory, dbFileName) - file, err := os.OpenFile(catchpointPath, os.O_RDONLY, 0666) - if err == nil && file != nil { - return &readCloseSizer{ReadCloser: file, size: fileSize}, nil - } - // else, see if this is a file-not-found error - if os.IsNotExist(err) { - // the database told us that we have this file.. but we couldn't find it. - // delete it from the database. - err := au.saveCatchpointFile(round, "", 0, "") - if err != nil { - au.log.Warnf("accountUpdates: getCatchpointStream: unable to delete missing catchpoint entry: %v", err) - return nil, err - } - - return nil, ledgercore.ErrNoEntry{} - } - // it's some other error. - return nil, fmt.Errorf("accountUpdates: getCatchpointStream: unable to open catchpoint file '%s' %v", catchpointPath, err) - } - - // if the database doesn't know about that round, see if we have that file anyway: - fileName := filepath.Join("catchpoints", catchpointRoundToPath(round)) - catchpointPath := filepath.Join(au.dbDirectory, fileName) - file, err := os.OpenFile(catchpointPath, os.O_RDONLY, 0666) - if err == nil && file != nil { - // great, if found that we should have had this in the database.. add this one now : - fileInfo, err := file.Stat() - if err != nil { - // we couldn't get the stat, so just return with the file. - return &readCloseSizer{ReadCloser: file, size: -1}, nil - } - - err = au.saveCatchpointFile(round, fileName, fileInfo.Size(), "") - if err != nil { - au.log.Warnf("accountUpdates: getCatchpointStream: unable to save missing catchpoint entry: %v", err) - } - return &readCloseSizer{ReadCloser: file, size: fileInfo.Size()}, nil - } - return nil, ledgercore.ErrNoEntry{} -} - // functions below this line are all internal functions // accountUpdatesLedgerEvaluator is a "ledger emulator" which is used *only* by initializeCaches, as a way to shortcut @@ -932,171 +677,6 @@ func (au *accountUpdates) latestTotalsImpl() (basics.Round, ledgercore.AccountTo return rnd, au.roundTotals[offset], nil } -// initializeCaches fills up the accountUpdates cache with the most recent ~320 blocks ( on normal execution ). -// the method also support balances recovery in cases where the difference between the lastBalancesRound and the lastestBlockRound -// is far greater than 320; in these cases, it would flush to disk periodically in order to avoid high memory consumption. -func (au *accountUpdates) initializeCaches(lastBalancesRound, lastestBlockRound, writingCatchpointRound basics.Round) (catchpointBlockDigest crypto.Digest, err error) { - var blk bookkeeping.Block - var delta ledgercore.StateDelta - - accLedgerEval := accountUpdatesLedgerEvaluator{ - au: au, - } - if lastBalancesRound < lastestBlockRound { - accLedgerEval.prevHeader, err = au.ledger.BlockHdr(lastBalancesRound) - if err != nil { - return - } - } - - skipAccountCacheMessage := make(chan struct{}) - writeAccountCacheMessageCompleted := make(chan struct{}) - defer func() { - close(skipAccountCacheMessage) - select { - case <-writeAccountCacheMessageCompleted: - if err == nil { - au.log.Infof("initializeCaches completed initializing account data caches") - } - default: - } - }() - - // this goroutine logs a message once if the parent function have not completed in initializingAccountCachesMessageTimeout seconds. - // the message is important, since we're blocking on the ledger block database here, and we want to make sure that we log a message - // within the above timeout. - go func() { - select { - case <-time.After(initializingAccountCachesMessageTimeout): - au.log.Infof("initializeCaches is initializing account data caches") - close(writeAccountCacheMessageCompleted) - case <-skipAccountCacheMessage: - } - }() - - blocksStream := make(chan bookkeeping.Block, initializeCachesReadaheadBlocksStream) - blockEvalFailed := make(chan struct{}, 1) - var blockRetrievalError error - go func() { - defer close(blocksStream) - for roundNumber := lastBalancesRound + 1; roundNumber <= lastestBlockRound; roundNumber++ { - blk, blockRetrievalError = au.ledger.Block(roundNumber) - if blockRetrievalError != nil { - return - } - select { - case blocksStream <- blk: - case <-blockEvalFailed: - return - } - } - }() - - lastFlushedRound := lastBalancesRound - const accountsCacheLoadingMessageInterval = 5 * time.Second - lastProgressMessage := time.Now().Add(-accountsCacheLoadingMessageInterval / 2) - - // rollbackSynchronousMode ensures that we switch to "fast writing mode" when we start flushing out rounds to disk, and that - // we exit this mode when we're done. - rollbackSynchronousMode := false - defer func() { - if rollbackSynchronousMode { - // restore default synchronous mode - au.dbs.Wdb.SetSynchronousMode(context.Background(), au.synchronousMode, au.synchronousMode >= db.SynchronousModeFull) - } - }() - - for blk := range blocksStream { - delta, err = au.ledger.trackerEvalVerified(blk, &accLedgerEval) - if err != nil { - close(blockEvalFailed) - return - } - - au.newBlockImpl(blk, delta) - - if blk.Round() == basics.Round(writingCatchpointRound) { - catchpointBlockDigest = blk.Digest() - } - - // flush to disk if any of the following applies: - // 1. if we have loaded up more than initializeCachesRoundFlushInterval rounds since the last time we flushed the data to disk - // 2. if we completed the loading and we loaded up more than 320 rounds. - flushIntervalExceed := blk.Round()-lastFlushedRound > initializeCachesRoundFlushInterval - loadCompleted := (lastestBlockRound == blk.Round() && lastBalancesRound+basics.Round(blk.ConsensusProtocol().MaxBalLookback) < lastestBlockRound) - if flushIntervalExceed || loadCompleted { - // adjust the last flush time, so that we would not hold off the flushing due to "working too fast" - au.lastFlushTime = time.Now().Add(-balancesFlushInterval) - - if !rollbackSynchronousMode { - // switch to rebuild synchronous mode to improve performance - au.dbs.Wdb.SetSynchronousMode(context.Background(), au.accountsRebuildSynchronousMode, au.accountsRebuildSynchronousMode >= db.SynchronousModeFull) - - // flip the switch to rollback the synchronous mode once we're done. - rollbackSynchronousMode = true - } - - var roundsBehind basics.Round - func() { - // The unlocking/relocking here isn't very elegant, but it does get the work done : - // this method is called on either startup or when fast catchup is complete. In the former usecase, the - // locking here is not really needed since the system is only starting up, and there are no other - // consumers for the accounts update. On the latter usecase, the function would always have exactly 320 rounds, - // and therefore this wouldn't be an issue. - // However, to make sure we're not missing any other future codepath, unlocking here and re-locking later on is a pretty - // safe bet. - // Wrapping locking/unlocking within a function scope prevents possible loss of the expected accountsMu state in case of panicing inside scheduleCommit. - au.accountsMu.Unlock() - defer au.accountsMu.Lock() - - // flush the account data - // TODO: figure out how to move it the upper level - au.ledger.scheduleCommit(blk.Round()) - - // wait for the writing to complete. - au.ledger.waitAccountsWriting() - - // The au.dbRound after writing should be ~320 behind the block round. - roundsBehind = blk.Round() - au.cachedDBRound - }() - - // are we too far behind ? ( taking into consideration the catchpoint writing, which can stall the writing for quite a bit ) - if roundsBehind > initializeCachesRoundFlushInterval+basics.Round(au.catchpointInterval) { - // we're unable to persist changes. This is unexpected, but there is no point in keep trying batching additional changes since any further changes - // would just accumulate in memory. - close(blockEvalFailed) - au.log.Errorf("initializeCaches was unable to fill up the account caches accounts round = %d, block round = %d. See above error for more details.", au.cachedDBRound, blk.Round()) - err = fmt.Errorf("initializeCaches failed to initialize the account data caches") - return - } - - // and once we flushed it to disk, update the lastFlushedRound - lastFlushedRound = blk.Round() - } - - // if enough time have passed since the last time we wrote a message to the log file then give the user an update about the progess. - if time.Now().Sub(lastProgressMessage) > accountsCacheLoadingMessageInterval { - // drop the initial message if we're got to this point since a message saying "still initializing" that comes after "is initializing" doesn't seems to be right. - select { - case skipAccountCacheMessage <- struct{}{}: - // if we got to this point, we should be able to close the writeAccountCacheMessageCompleted channel to have the "completed initializing" message written. - close(writeAccountCacheMessageCompleted) - default: - } - au.log.Infof("initializeCaches is still initializing account data caches, %d rounds loaded out of %d rounds", blk.Round()-lastBalancesRound, lastestBlockRound-lastBalancesRound) - lastProgressMessage = time.Now() - } - - // prepare for the next iteration. - accLedgerEval.prevHeader = *delta.Hdr - } - - if blockRetrievalError != nil { - err = blockRetrievalError - } - return -} - // initializeFromDisk performs the atomic operation of loading the accounts data information from disk // and preparing the accountUpdates for operation. func (au *accountUpdates) initializeFromDisk(l ledgerForTracker, lastBalancesRound basics.Round) (err error) { @@ -1107,10 +687,6 @@ func (au *accountUpdates) initializeFromDisk(l ledgerForTracker, lastBalancesRou start := time.Now() ledgerAccountsinitCount.Inc(nil) err = au.dbs.Wdb.Atomic(func(ctx context.Context, tx *sql.Tx) error { - err0 := au.accountsInitializeHashes(ctx, tx, lastBalancesRound) - if err0 != nil { - return err0 - } totals, err0 := accountsTotals(tx, false) if err0 != nil { return err0 @@ -1129,10 +705,6 @@ func (au *accountUpdates) initializeFromDisk(l ledgerForTracker, lastBalancesRou if err != nil { return } - au.lastCatchpointLabel, _, err = au.accountsq.readCatchpointStateString(context.Background(), catchpointStateLastCatchpoint) - if err != nil { - return - } hdr, err := l.BlockHdr(lastBalancesRound) if err != nil { @@ -1145,231 +717,11 @@ func (au *accountUpdates) initializeFromDisk(l ledgerForTracker, lastBalancesRou au.accounts = make(map[basics.Address]modifiedAccount) au.creatables = make(map[basics.CreatableIndex]ledgercore.ModifiedCreatable) au.deltasAccum = []int{0} - au.roundDigest = nil - - au.catchpointWriting = 0 - // keep these channel closed if we're not generating catchpoint - au.catchpointSlowWriting = make(chan struct{}, 1) - close(au.catchpointSlowWriting) - au.ctx, au.ctxCancel = context.WithCancel(context.Background()) au.baseAccounts.init(au.log, baseAccountsPendingAccountsBufferSize, baseAccountsPendingAccountsWarnThreshold) return } -// accountHashBuilder calculates the hash key used for the trie by combining the account address and the account data -func accountHashBuilder(addr basics.Address, accountData basics.AccountData, encodedAccountData []byte) []byte { - hash := make([]byte, 4+crypto.DigestSize) - // write out the lowest 32 bits of the reward base. This should improve the caching of the trie by allowing - // recent updated to be in-cache, and "older" nodes will be left alone. - for i, rewards := 3, accountData.RewardsBase; i >= 0; i, rewards = i-1, rewards>>8 { - // the following takes the rewards & 255 -> hash[i] - hash[i] = byte(rewards) - } - entryHash := crypto.Hash(append(addr[:], encodedAccountData[:]...)) - copy(hash[4:], entryHash[:]) - return hash[:] -} - -// accountsInitializeHashes initializes account hashes. -// as part of the initialization, it tests if a hash table matches to account base and updates the former. -func (au *accountUpdates) accountsInitializeHashes(ctx context.Context, tx *sql.Tx, rnd basics.Round) error { - hashRound, err := accountsHashRound(tx) - if err != nil { - return err - } - - if hashRound != rnd { - // if the hashed round is different then the base round, something was modified, and the accounts aren't in sync - // with the hashes. - err = resetAccountHashes(tx) - if err != nil { - return err - } - // if catchpoint is disabled on this node, we could complete the initialization right here. - if !au.catchpointEnabled() { - return nil - } - } - - // create the merkle trie for the balances - committer, err := MakeMerkleCommitter(tx, false) - if err != nil { - return fmt.Errorf("accountsInitialize was unable to makeMerkleCommitter: %v", err) - } - - trie, err := merkletrie.MakeTrie(committer, TrieMemoryConfig) - if err != nil { - return fmt.Errorf("accountsInitialize was unable to MakeTrie: %v", err) - } - - // we might have a database that was previously initialized, and now we're adding the balances trie. In that case, we need to add all the existing balances to this trie. - // we can figure this out by examining the hash of the root: - rootHash, err := trie.RootHash() - if err != nil { - return fmt.Errorf("accountsInitialize was unable to retrieve trie root hash: %v", err) - } - - if rootHash.IsZero() { - au.log.Infof("accountsInitialize rebuilding merkle trie for round %d", rnd) - accountBuilderIt := makeOrderedAccountsIter(tx, trieRebuildAccountChunkSize) - defer accountBuilderIt.Close(ctx) - startTrieBuildTime := time.Now() - accountsCount := 0 - lastRebuildTime := startTrieBuildTime - pendingAccounts := 0 - totalOrderedAccounts := 0 - for { - accts, processedRows, err := accountBuilderIt.Next(ctx) - if err == sql.ErrNoRows { - // the account builder would return sql.ErrNoRows when no more data is available. - break - } else if err != nil { - return err - } - - if len(accts) > 0 { - accountsCount += len(accts) - pendingAccounts += len(accts) - for _, acct := range accts { - added, err := trie.Add(acct.digest) - if err != nil { - return fmt.Errorf("accountsInitialize was unable to add changes to trie: %v", err) - } - if !added { - au.log.Warnf("accountsInitialize attempted to add duplicate hash '%s' to merkle trie for account %v", hex.EncodeToString(acct.digest), acct.address) - } - } - - if pendingAccounts >= trieRebuildCommitFrequency { - // this trie Evict will commit using the current transaction. - // if anything goes wrong, it will still get rolled back. - _, err = trie.Evict(true) - if err != nil { - return fmt.Errorf("accountsInitialize was unable to commit changes to trie: %v", err) - } - pendingAccounts = 0 - } - - if time.Now().Sub(lastRebuildTime) > 5*time.Second { - // let the user know that the trie is still being rebuilt. - au.log.Infof("accountsInitialize still building the trie, and processed so far %d accounts", accountsCount) - lastRebuildTime = time.Now() - } - } else if processedRows > 0 { - totalOrderedAccounts += processedRows - // if it's not ordered, we can ignore it for now; we'll just increase the counters and emit logs periodically. - if time.Now().Sub(lastRebuildTime) > 5*time.Second { - // let the user know that the trie is still being rebuilt. - au.log.Infof("accountsInitialize still building the trie, and hashed so far %d accounts", totalOrderedAccounts) - lastRebuildTime = time.Now() - } - } - } - - // this trie Evict will commit using the current transaction. - // if anything goes wrong, it will still get rolled back. - _, err = trie.Evict(true) - if err != nil { - return fmt.Errorf("accountsInitialize was unable to commit changes to trie: %v", err) - } - - // we've just updated the merkle trie, update the hashRound to reflect that. - err = updateAccountsHashRound(tx, rnd) - if err != nil { - return fmt.Errorf("accountsInitialize was unable to update the account hash round to %d: %v", rnd, err) - } - - au.log.Infof("accountsInitialize rebuilt the merkle trie with %d entries in %v", accountsCount, time.Now().Sub(startTrieBuildTime)) - } - au.balancesTrie = trie - return nil -} - -// deleteStoredCatchpoints iterates over the storedcatchpoints table and deletes all the files stored on disk. -// once all the files have been deleted, it would go ahead and remove the entries from the table. -func deleteStoredCatchpoints(ctx context.Context, dbQueries *accountsDbQueries, dbDirectory string) (err error) { - catchpointsFilesChunkSize := 50 - for { - fileNames, err := dbQueries.getOldestCatchpointFiles(ctx, catchpointsFilesChunkSize, 0) - if err != nil { - return err - } - if len(fileNames) == 0 { - break - } - - for round, fileName := range fileNames { - absCatchpointFileName := filepath.Join(dbDirectory, fileName) - err = os.Remove(absCatchpointFileName) - if err == nil || os.IsNotExist(err) { - // it's ok if the file doesn't exist. just remove it from the database and we'll be good to go. - } else { - // we can't delete the file, abort - - return fmt.Errorf("unable to delete old catchpoint file '%s' : %v", absCatchpointFileName, err) - } - // clear the entry from the database - err = dbQueries.storeCatchpoint(ctx, round, "", "", 0) - if err != nil { - return err - } - } - } - return nil -} - -// accountsUpdateBalances applies the given compactAccountDeltas to the merkle trie -func (au *accountUpdates) accountsUpdateBalances(accountsDeltas compactAccountDeltas) (err error) { - if !au.catchpointEnabled() { - return nil - } - var added, deleted bool - accumulatedChanges := 0 - - for i := 0; i < accountsDeltas.len(); i++ { - addr, delta := accountsDeltas.getByIdx(i) - if !delta.old.accountData.IsZero() { - deleteHash := accountHashBuilder(addr, delta.old.accountData, protocol.Encode(&delta.old.accountData)) - deleted, err = au.balancesTrie.Delete(deleteHash) - if err != nil { - return fmt.Errorf("failed to delete hash '%s' from merkle trie for account %v: %w", hex.EncodeToString(deleteHash), addr, err) - } - if !deleted { - au.log.Warnf("failed to delete hash '%s' from merkle trie for account %v", hex.EncodeToString(deleteHash), addr) - } else { - accumulatedChanges++ - } - } - - if !delta.new.IsZero() { - addHash := accountHashBuilder(addr, delta.new, protocol.Encode(&delta.new)) - added, err = au.balancesTrie.Add(addHash) - if err != nil { - return fmt.Errorf("attempted to add duplicate hash '%s' to merkle trie for account %v: %w", hex.EncodeToString(addHash), addr, err) - } - if !added { - au.log.Warnf("attempted to add duplicate hash '%s' to merkle trie for account %v", hex.EncodeToString(addHash), addr) - } else { - accumulatedChanges++ - } - } - } - if accumulatedChanges >= trieAccumulatedChangesFlush { - accumulatedChanges = 0 - _, err = au.balancesTrie.Commit() - if err != nil { - return - } - } - - // write it all to disk. - if accumulatedChanges > 0 { - _, err = au.balancesTrie.Commit() - } - - return -} - // newBlockImpl is the accountUpdates implementation of the ledgerTracker interface. This is the "internal" facing function // which assumes that no lock need to be taken. func (au *accountUpdates) newBlockImpl(blk bookkeeping.Block, delta ledgercore.StateDelta) { @@ -1386,7 +738,6 @@ func (au *accountUpdates) newBlockImpl(blk bookkeeping.Block, delta ledgercore.S au.deltas = append(au.deltas, delta.Accts) au.versions = append(au.versions, blk.CurrentProtocol) au.creatableDeltas = append(au.creatableDeltas, delta.Creatables) - au.roundDigest = append(au.roundDigest, blk.Digest()) au.deltasAccum = append(au.deltasAccum, delta.Accts.Len()+au.deltasAccum[len(au.deltasAccum)-1]) au.baseAccounts.flushPendingWrites() @@ -1676,14 +1027,6 @@ func (au *accountUpdates) getCreatorForRound(rnd basics.Round, cidx basics.Creat } } -// accountsCreateCatchpointLabel creates a catchpoint label and write it. -func (au *accountUpdates) accountsCreateCatchpointLabel(committedRound basics.Round, totals ledgercore.AccountTotals, ledgerBlockDigest crypto.Digest, trieBalancesHash crypto.Digest) (label string, err error) { - cpLabel := ledgercore.MakeCatchpointLabel(committedRound, ledgerBlockDigest, trieBalancesHash, totals) - label = cpLabel.String() - _, err = au.accountsq.writeCatchpointStateString(context.Background(), catchpointStateLastCatchpoint, label) - return -} - // roundOffset calculates the offset of the given round compared to the current dbRound. Requires that the lock would be taken. func (au *accountUpdates) roundOffset(rnd basics.Round) (offset uint64, err error) { if rnd < au.cachedDBRound { @@ -1703,19 +1046,8 @@ func (au *accountUpdates) roundOffset(rnd basics.Round) (offset uint64, err erro return off, nil } -func (au *accountUpdates) isCatchpointRound(offset uint64, dbRound basics.Round, lookback basics.Round) bool { - return ((offset + uint64(lookback+dbRound)) > 0) && (au.catchpointInterval != 0) && ((uint64((offset + uint64(lookback+dbRound))) % au.catchpointInterval) == 0) -} - func (au *accountUpdates) handleUnorderedCommit(offset uint64, dbRound basics.Round, lookback basics.Round) { - // if this is an archival ledger, we might need to update the catchpointWriting variable. - if au.archivalLedger { - // determine if this was a catchpoint round - if au.isCatchpointRound(offset, dbRound, lookback) { - // it was a catchpoint round, so update the catchpointWriting to indicate that we're done. - atomic.StoreInt32(&au.catchpointWriting, 0) - } - } + } // prepareCommit prepares data to write to the database a "chunk" of rounds, and update the cached dbRound accordingly. @@ -1729,13 +1061,9 @@ func (au *accountUpdates) prepareCommit(dcc *deferredCommitContext) error { } offset := dcc.offset - dbRound := dcc.oldBase - lookback := dcc.lookback au.accountsMu.RLock() - dcc.isCatchpointRound = au.isCatchpointRound(offset, dbRound, lookback) - // create a copy of the deltas, round totals and protos for the range we're going to flush. dcc.deltas = make([]ledgercore.AccountDeltas, offset) creatableDeltas := make([]map[basics.CreatableIndex]ledgercore.ModifiedCreatable, offset) @@ -1752,15 +1080,11 @@ func (au *accountUpdates) prepareCommit(dcc *deferredCommitContext) error { // here would prevent us from "forgetting" to update this variable later on. // The same is repeated in commitRound on errors. if dcc.isCatchpointRound && au.archivalLedger { - atomic.StoreInt32(&au.catchpointWriting, 0) + atomic.StoreInt32(dcc.catchpointWriting, 0) } return fmt.Errorf("attempted to commit series of rounds with non-uniform consensus versions") } - if dcc.isCatchpointRound { - dcc.committedRoundDigest = au.roundDigest[offset+uint64(lookback)-1] - } - // compact all the deltas - when we're trying to persist multiple rounds, we might have the same account // being updated multiple times. When that happen, we can safely omit the intermediate updates. dcc.compactAccountDeltas = makeCompactAccountDeltas(dcc.deltas, au.baseAccounts) @@ -1780,39 +1104,17 @@ func (au *accountUpdates) prepareCommit(dcc *deferredCommitContext) error { // commitRound closure is called within the same transaction for all trackers // it receives current offset and dbRound func (au *accountUpdates) commitRound(ctx context.Context, tx *sql.Tx, dcc *deferredCommitContext) (err error) { - treeTargetRound := basics.Round(0) offset := dcc.offset dbRound := dcc.oldBase defer func() { if err != nil { if dcc.isCatchpointRound && au.archivalLedger { - atomic.StoreInt32(&au.catchpointWriting, 0) + atomic.StoreInt32(dcc.catchpointWriting, 0) } } }() - if au.catchpointEnabled() { - var mc *MerkleCommitter - mc, err = MakeMerkleCommitter(tx, false) - if err != nil { - return - } - - var trie *merkletrie.Trie - if au.balancesTrie == nil { - trie, err = merkletrie.MakeTrie(mc, TrieMemoryConfig) - if err != nil { - au.log.Warnf("unable to create merkle trie during committedUpTo: %v", err) - return err - } - au.balancesTrie = trie - } else { - au.balancesTrie.SetCommitter(mc) - } - treeTargetRound = dbRound + basics.Round(offset) - } - _, err = db.ResetTransactionWarnDeadline(ctx, tx, time.Now().Add(accountsUpdatePerRoundHighWatermark*time.Duration(offset))) if err != nil { return err @@ -1837,18 +1139,7 @@ func (au *accountUpdates) commitRound(ctx context.Context, tx *sql.Tx, dcc *defe } if dcc.updateStats { - dcc.stats.MerkleTrieUpdateDuration = time.Duration(time.Now().UnixNano()) - } - - err = au.accountsUpdateBalances(dcc.compactAccountDeltas) - if err != nil { - return err - } - - if dcc.updateStats { - now := time.Duration(time.Now().UnixNano()) - dcc.stats.MerkleTrieUpdateDuration = now - dcc.stats.MerkleTrieUpdateDuration - dcc.stats.AccountsWritingDuration = now + dcc.stats.AccountsWritingDuration = time.Duration(time.Now().UnixNano()) } // the updates of the actual account data is done last since the accountsNewRound would modify the compactDeltas old values @@ -1862,22 +1153,10 @@ func (au *accountUpdates) commitRound(ctx context.Context, tx *sql.Tx, dcc *defe dcc.stats.AccountsWritingDuration = time.Duration(time.Now().UnixNano()) - dcc.stats.AccountsWritingDuration } - err = updateAccountsHashRound(tx, treeTargetRound) - if err != nil { - return err - } - - if dcc.isCatchpointRound { - dcc.trieBalancesHash, err = au.balancesTrie.RootHash() - if err != nil { - return err - } - } - return } -func (au *accountUpdates) postCommit(dcc deferredCommitContext) { +func (au *accountUpdates) postCommit(ctx context.Context, dcc *deferredCommitContext) { if dcc.updateStats { spentDuration := dcc.stats.DatabaseCommitDuration + dcc.stats.AccountsWritingDuration + dcc.stats.MerkleTrieUpdateDuration + dcc.stats.OldAccountPreloadDuration dcc.stats.DatabaseCommitDuration = time.Duration(time.Now().UnixNano()) - spentDuration @@ -1885,28 +1164,9 @@ func (au *accountUpdates) postCommit(dcc deferredCommitContext) { offset := dcc.offset dbRound := dcc.oldBase - lookback := dcc.lookback newBase := dcc.newBase - var catchpointLabel string - var err error - if dcc.isCatchpointRound { - catchpointLabel, err = au.accountsCreateCatchpointLabel(dbRound+basics.Round(offset)+lookback, dcc.roundTotals, dcc.committedRoundDigest, dcc.trieBalancesHash) - if err != nil { - au.log.Warnf("commitRound : unable to create a catchpoint label: %v", err) - } - } - if au.balancesTrie != nil { - _, err = au.balancesTrie.Evict(false) - if err != nil { - au.log.Warnf("merkle trie failed to evict: %v", err) - } - } - - if dcc.isCatchpointRound && catchpointLabel != "" { - au.lastCatchpointLabel = catchpointLabel - } - updatingBalancesDuration := time.Since(dcc.flushTime) + dcc.updatingBalancesDuration = time.Since(dcc.flushTime) if dcc.updateStats { dcc.stats.MemoryUpdatesDuration = time.Duration(time.Now().UnixNano()) @@ -1956,12 +1216,10 @@ func (au *accountUpdates) postCommit(dcc deferredCommitContext) { au.deltas = au.deltas[offset:] au.deltasAccum = au.deltasAccum[offset:] - au.roundDigest = au.roundDigest[offset:] au.versions = au.versions[offset:] au.roundTotals = au.roundTotals[offset:] au.creatableDeltas = au.creatableDeltas[offset:] au.cachedDBRound = newBase - au.lastFlushTime = dcc.flushTime au.accountsMu.Unlock() @@ -1971,12 +1229,6 @@ func (au *accountUpdates) postCommit(dcc deferredCommitContext) { au.accountsReadCond.Broadcast() - if dcc.isCatchpointRound && au.archivalLedger && catchpointLabel != "" { - // generate the catchpoint file. This need to be done inline so that it will block any new accounts that from being written. - // the generateCatchpoint expects that the accounts data would not be modified in the background during it's execution. - au.generateCatchpoint(basics.Round(offset)+dbRound+lookback, catchpointLabel, dcc.committedRoundDigest, updatingBalancesDuration) - } - // log telemetry event if dcc.updateStats { dcc.stats.StartRound = uint64(dbRound) @@ -1987,13 +1239,6 @@ func (au *accountUpdates) postCommit(dcc deferredCommitContext) { var details struct{} au.log.Metrics(telemetryspec.Accounts, dcc.stats, details) } - - // in scheduleCommit, we expect that this function to update the catchpointWriting when - // it's on a catchpoint round and it's an archival ledger. Doing this in a deferred function - // here would prevent us from "forgetting" to update this variable later on. - if dcc.isCatchpointRound && au.archivalLedger { - atomic.StoreInt32(&au.catchpointWriting, 0) - } } // compactCreatableDeltas takes an array of creatables map deltas ( one array entry per round ), and compact the array into a single @@ -2032,180 +1277,6 @@ func (au *accountUpdates) latest() basics.Round { return au.cachedDBRound + basics.Round(len(au.deltas)) } -// generateCatchpoint generates a single catchpoint file -func (au *accountUpdates) generateCatchpoint(committedRound basics.Round, label string, committedRoundDigest crypto.Digest, updatingBalancesDuration time.Duration) { - beforeGeneratingCatchpointTime := time.Now() - catchpointGenerationStats := telemetryspec.CatchpointGenerationEventDetails{ - BalancesWriteTime: uint64(updatingBalancesDuration.Nanoseconds()), - } - - // the retryCatchpointCreation is used to repeat the catchpoint file generation in case the node crashed / aborted during startup - // before the catchpoint file generation could be completed. - retryCatchpointCreation := false - au.log.Debugf("accountUpdates: generateCatchpoint: generating catchpoint for round %d", committedRound) - defer func() { - if !retryCatchpointCreation { - // clear the writingCatchpoint flag - _, err := au.accountsq.writeCatchpointStateUint64(context.Background(), catchpointStateWritingCatchpoint, uint64(0)) - if err != nil { - au.log.Warnf("accountUpdates: generateCatchpoint unable to clear catchpoint state '%s' for round %d: %v", catchpointStateWritingCatchpoint, committedRound, err) - } - } - }() - - _, err := au.accountsq.writeCatchpointStateUint64(context.Background(), catchpointStateWritingCatchpoint, uint64(committedRound)) - if err != nil { - au.log.Warnf("accountUpdates: generateCatchpoint unable to write catchpoint state '%s' for round %d: %v", catchpointStateWritingCatchpoint, committedRound, err) - return - } - - relCatchpointFileName := filepath.Join("catchpoints", catchpointRoundToPath(committedRound)) - absCatchpointFileName := filepath.Join(au.dbDirectory, relCatchpointFileName) - - more := true - const shortChunkExecutionDuration = 50 * time.Millisecond - const longChunkExecutionDuration = 1 * time.Second - var chunkExecutionDuration time.Duration - select { - case <-au.catchpointSlowWriting: - chunkExecutionDuration = longChunkExecutionDuration - default: - chunkExecutionDuration = shortChunkExecutionDuration - } - - var catchpointWriter *catchpointWriter - start := time.Now() - ledgerGeneratecatchpointCount.Inc(nil) - err = au.dbs.Rdb.Atomic(func(ctx context.Context, tx *sql.Tx) (err error) { - catchpointWriter = makeCatchpointWriter(au.ctx, absCatchpointFileName, tx, committedRound, committedRoundDigest, label) - for more { - stepCtx, stepCancelFunction := context.WithTimeout(au.ctx, chunkExecutionDuration) - writeStepStartTime := time.Now() - more, err = catchpointWriter.WriteStep(stepCtx) - // accumulate the actual time we've spent writing in this step. - catchpointGenerationStats.CPUTime += uint64(time.Now().Sub(writeStepStartTime).Nanoseconds()) - stepCancelFunction() - if more && err == nil { - // we just wrote some data, but there is more to be written. - // go to sleep for while. - // before going to sleep, extend the transaction timeout so that we won't get warnings: - db.ResetTransactionWarnDeadline(ctx, tx, time.Now().Add(1*time.Second)) - select { - case <-time.After(100 * time.Millisecond): - // increase the time slot allocated for writing the catchpoint, but stop when we get to the longChunkExecutionDuration limit. - // this would allow the catchpoint writing speed to ramp up while still leaving some cpu available. - chunkExecutionDuration *= 2 - if chunkExecutionDuration > longChunkExecutionDuration { - chunkExecutionDuration = longChunkExecutionDuration - } - case <-au.ctx.Done(): - retryCatchpointCreation = true - err2 := catchpointWriter.Abort() - if err2 != nil { - return fmt.Errorf("error removing catchpoint file : %v", err2) - } - return nil - case <-au.catchpointSlowWriting: - chunkExecutionDuration = longChunkExecutionDuration - } - } - if err != nil { - err = fmt.Errorf("unable to create catchpoint : %v", err) - err2 := catchpointWriter.Abort() - if err2 != nil { - au.log.Warnf("accountUpdates: generateCatchpoint: error removing catchpoint file : %v", err2) - } - return - } - } - return - }) - ledgerGeneratecatchpointMicros.AddMicrosecondsSince(start, nil) - - if err != nil { - au.log.Warnf("accountUpdates: generateCatchpoint: %v", err) - return - } - if catchpointWriter == nil { - au.log.Warnf("accountUpdates: generateCatchpoint: nil catchpointWriter") - return - } - - err = au.saveCatchpointFile(committedRound, relCatchpointFileName, catchpointWriter.GetSize(), catchpointWriter.GetCatchpoint()) - if err != nil { - au.log.Warnf("accountUpdates: generateCatchpoint: unable to save catchpoint: %v", err) - return - } - catchpointGenerationStats.FileSize = uint64(catchpointWriter.GetSize()) - catchpointGenerationStats.WritingDuration = uint64(time.Now().Sub(beforeGeneratingCatchpointTime).Nanoseconds()) - catchpointGenerationStats.AccountsCount = catchpointWriter.GetTotalAccounts() - catchpointGenerationStats.CatchpointLabel = catchpointWriter.GetCatchpoint() - au.log.EventWithDetails(telemetryspec.Accounts, telemetryspec.CatchpointGenerationEvent, catchpointGenerationStats) - au.log.With("writingDuration", catchpointGenerationStats.WritingDuration). - With("CPUTime", catchpointGenerationStats.CPUTime). - With("balancesWriteTime", catchpointGenerationStats.BalancesWriteTime). - With("accountsCount", catchpointGenerationStats.AccountsCount). - With("fileSize", catchpointGenerationStats.FileSize). - With("catchpointLabel", catchpointGenerationStats.CatchpointLabel). - Infof("Catchpoint file was generated") -} - -// catchpointRoundToPath calculate the catchpoint file path for a given round -func catchpointRoundToPath(rnd basics.Round) string { - irnd := int64(rnd) / 256 - outStr := "" - for irnd > 0 { - outStr = filepath.Join(outStr, fmt.Sprintf("%02x", irnd%256)) - irnd = irnd / 256 - } - outStr = filepath.Join(outStr, strconv.FormatInt(int64(rnd), 10)+".catchpoint") - return outStr -} - -// saveCatchpointFile stores the provided fileName as the stored catchpoint for the given round. -// after a successful insert operation to the database, it would delete up to 2 old entries, as needed. -// deleting 2 entries while inserting single entry allow us to adjust the size of the backing storage and have the -// database and storage realign. -func (au *accountUpdates) saveCatchpointFile(round basics.Round, fileName string, fileSize int64, catchpoint string) (err error) { - if au.catchpointFileHistoryLength != 0 { - err = au.accountsq.storeCatchpoint(context.Background(), round, fileName, catchpoint, fileSize) - if err != nil { - au.log.Warnf("accountUpdates: saveCatchpoint: unable to save catchpoint: %v", err) - return - } - } else { - err = os.Remove(fileName) - if err != nil { - au.log.Warnf("accountUpdates: saveCatchpoint: unable to remove file (%s): %v", fileName, err) - return - } - } - if au.catchpointFileHistoryLength == -1 { - return - } - var filesToDelete map[basics.Round]string - filesToDelete, err = au.accountsq.getOldestCatchpointFiles(context.Background(), 2, au.catchpointFileHistoryLength) - if err != nil { - return fmt.Errorf("unable to delete catchpoint file, getOldestCatchpointFiles failed : %v", err) - } - for round, fileToDelete := range filesToDelete { - absCatchpointFileName := filepath.Join(au.dbDirectory, fileToDelete) - err = os.Remove(absCatchpointFileName) - if err == nil || os.IsNotExist(err) { - // it's ok if the file doesn't exist. just remove it from the database and we'll be good to go. - err = nil - } else { - // we can't delete the file, abort - - return fmt.Errorf("unable to delete old catchpoint file '%s' : %v", absCatchpointFileName, err) - } - err = au.accountsq.storeCatchpoint(context.Background(), round, "", "", 0) - if err != nil { - return fmt.Errorf("unable to delete old catchpoint entry '%s' : %v", fileToDelete, err) - } - } - return -} - // the vacuumDatabase performs a full vacuum of the accounts database. func (au *accountUpdates) vacuumDatabase(ctx context.Context) (err error) { // vaccumming the database would modify the some of the tables rowid, so we need to make sure any stored in-memory diff --git a/ledger/acctupdates_test.go b/ledger/acctupdates_test.go index 8428062869..27306af539 100644 --- a/ledger/acctupdates_test.go +++ b/ledger/acctupdates_test.go @@ -24,7 +24,6 @@ import ( "fmt" "io/ioutil" "os" - "path/filepath" "runtime" "strings" "sync" @@ -111,8 +110,12 @@ func (ml *mockLedgerForTracker) fork(t testing.TB) *mockLedgerForTracker { log: dblogger, blocks: make([]blockEntry, len(ml.blocks)), deltas: make([]ledgercore.StateDelta, len(ml.deltas)), + accts: make(map[basics.Address]basics.AccountData), filename: fn, } + for k, v := range ml.accts { + newLedgerTracker.accts[k] = v + } copy(newLedgerTracker.blocks, ml.blocks) copy(newLedgerTracker.deltas, ml.deltas) @@ -195,14 +198,6 @@ func (ml *mockLedgerForTracker) trackerLog() logging.Logger { return ml.log } -func (ml *mockLedgerForTracker) waitAccountsWriting() { - ml.trackers.waitAccountsWriting() -} - -func (ml *mockLedgerForTracker) scheduleCommit(rnd basics.Round) { - ml.trackers.scheduleCommit(rnd) -} - func (ml *mockLedgerForTracker) GenesisHash() crypto.Digest { if len(ml.blocks) > 0 { return ml.blocks[0].block.GenesisHash() @@ -251,18 +246,20 @@ func (au *accountUpdates) allBalances(rnd basics.Round) (bals map[basics.Address func newAcctUpdates(tb testing.TB, l *mockLedgerForTracker, conf config.Local, dbPathPrefix string) *accountUpdates { au := &accountUpdates{} - au.initialize(conf, ".") - _, err := trackerDBInitialize(l, au.catchpointEnabled(), au.dbDirectory) + au.initialize(conf) + _, err := trackerDBInitialize(l, false, ".") require.NoError(tb, err) - l.trackers.initialize(au, l, []ledgerTracker{au}) + l.trackers.initialize(l, []ledgerTracker{au}, conf) + err = l.trackers.loadFromDisk(l) + require.NoError(tb, err) return au } func checkAcctUpdates(t *testing.T, au *accountUpdates, base basics.Round, latestRnd basics.Round, accts []map[basics.Address]basics.AccountData, rewards []uint64, proto config.ConsensusParams) { latest := au.latest() - require.Equal(t, latest, latestRnd) + require.Equal(t, latestRnd, latest) _, err := au.Totals(latest + 1) require.Error(t, err) @@ -386,8 +383,6 @@ func TestAcctUpdates(t *testing.T) { conf := config.GetDefaultLocal() au := newAcctUpdates(t, ml, conf, ".") - err := au.loadFromDisk(ml, 0) - require.NoError(t, err) defer au.close() // cover 10 genesis blocks @@ -439,16 +434,16 @@ func TestAcctUpdates(t *testing.T) { for i := basics.Round(0); i < 15; i++ { // Clear the timer to ensure a flush - au.lastFlushTime = time.Time{} + ml.trackers.lastFlushTime = time.Time{} - ml.scheduleCommit(basics.Round(proto.MaxBalLookback) + i) - ml.waitAccountsWriting() + ml.trackers.committedUpTo(basics.Round(proto.MaxBalLookback) + i) + ml.trackers.waitAccountsWriting() checkAcctUpdates(t, au, i, basics.Round(proto.MaxBalLookback+14), accts, rewardsLevels, proto) } // check the account totals. var dbRound basics.Round - err = ml.dbs.Rdb.Atomic(func(ctx context.Context, tx *sql.Tx) (err error) { + err := ml.dbs.Rdb.Atomic(func(ctx context.Context, tx *sql.Tx) (err error) { dbRound, err = accountsRound(tx) return }) @@ -468,7 +463,6 @@ func TestAcctUpdates(t *testing.T) { require.NoError(t, err) require.Equal(t, expectedTotals, actualTotals) } - func TestAcctUpdatesFastUpdates(t *testing.T) { partitiontest.PartitionTest(t) @@ -496,8 +490,6 @@ func TestAcctUpdatesFastUpdates(t *testing.T) { conf := config.GetDefaultLocal() conf.CatchpointInterval = 1 au := newAcctUpdates(t, ml, conf, ".") - err := au.loadFromDisk(ml, 0) - require.NoError(t, err) defer au.close() // cover 10 genesis blocks @@ -541,7 +533,7 @@ func TestAcctUpdatesFastUpdates(t *testing.T) { wg.Add(1) go func(round basics.Round) { defer wg.Done() - ml.scheduleCommit(round) + ml.trackers.committedUpTo(round) }(i) } wg.Wait() @@ -585,8 +577,6 @@ func BenchmarkBalancesChanges(b *testing.B) { conf := config.GetDefaultLocal() au := newAcctUpdates(b, ml, conf, ".") - err := au.loadFromDisk(ml, 0) - require.NoError(b, err) defer au.close() // cover initialRounds genesis blocks @@ -629,18 +619,18 @@ func BenchmarkBalancesChanges(b *testing.B) { } for i := proto.MaxBalLookback; i < proto.MaxBalLookback+initialRounds; i++ { // Clear the timer to ensure a flush - au.lastFlushTime = time.Time{} - ml.scheduleCommit(basics.Round(i)) + ml.trackers.lastFlushTime = time.Time{} + ml.trackers.committedUpTo(basics.Round(i)) } - ml.waitAccountsWriting() + ml.trackers.waitAccountsWriting() b.ResetTimer() startTime := time.Now() for i := proto.MaxBalLookback + initialRounds; i < proto.MaxBalLookback+uint64(b.N); i++ { // Clear the timer to ensure a flush - au.lastFlushTime = time.Time{} - ml.scheduleCommit(basics.Round(i)) + ml.trackers.lastFlushTime = time.Time{} + ml.trackers.committedUpTo(basics.Round(i)) } - ml.waitAccountsWriting() + ml.trackers.waitAccountsWriting() deltaTime := time.Now().Sub(startTime) if deltaTime > time.Second { return @@ -718,8 +708,6 @@ func TestLargeAccountCountCatchpointGeneration(t *testing.T) { conf.CatchpointInterval = 1 conf.Archival = true au := newAcctUpdates(t, ml, conf, ".") - err := au.loadFromDisk(ml, 0) - require.NoError(t, err) defer au.close() // cover 10 genesis blocks @@ -756,9 +744,9 @@ func TestLargeAccountCountCatchpointGeneration(t *testing.T) { accts = append(accts, totals) rewardsLevels = append(rewardsLevels, rewardLevel) - ml.scheduleCommit(i) + ml.trackers.committedUpTo(i) if i%2 == 1 { - ml.waitAccountsWriting() + ml.trackers.waitAccountsWriting() } } } @@ -823,9 +811,6 @@ func TestAcctUpdatesUpdatesCorrectness(t *testing.T) { conf := config.GetDefaultLocal() au := newAcctUpdates(t, ml, conf, ".") - - err := au.loadFromDisk(ml, 0) - require.NoError(t, err) defer au.close() // cover 10 genesis blocks @@ -918,10 +903,10 @@ func TestAcctUpdatesUpdatesCorrectness(t *testing.T) { delta.Accts.Upsert(addr, ad) } au.newBlock(blk, delta) - ml.scheduleCommit(i) + ml.trackers.committedUpTo(i) } lastRound := i - 1 - ml.waitAccountsWriting() + ml.trackers.waitAccountsWriting() for idx, addr := range moneyAccounts { balance, validThrough, err := au.LookupWithoutRewards(lastRound, addr) @@ -941,53 +926,6 @@ func TestAcctUpdatesUpdatesCorrectness(t *testing.T) { t.Run("DiskDB", testFunction) } -// TestAcctUpdatesDeleteStoredCatchpoints - The goal of this test is to verify that the deleteStoredCatchpoints function works correctly. -// it doing so by filling up the storedcatchpoints with dummy catchpoint file entries, as well as creating these dummy files on disk. -// ( the term dummy is only because these aren't real catchpoint files, but rather a zero-length file ). Then, the test call the function -// and ensures that it did not errored, the catchpoint files were correctly deleted, and that deleteStoredCatchpoints contains no more -// entries. -func TestAcctUpdatesDeleteStoredCatchpoints(t *testing.T) { - partitiontest.PartitionTest(t) - - accts := []map[basics.Address]basics.AccountData{ledgertesting.RandomAccounts(20, true)} - - ml := makeMockLedgerForTracker(t, true, 10, protocol.ConsensusCurrentVersion, accts) - defer ml.Close() - - conf := config.GetDefaultLocal() - conf.CatchpointInterval = 1 - au := newAcctUpdates(t, ml, conf, ".") - - err := au.loadFromDisk(ml, 0) - require.NoError(t, err) - defer au.close() - - dummyCatchpointFilesToCreate := 42 - - for i := 0; i < dummyCatchpointFilesToCreate; i++ { - f, err := os.Create(fmt.Sprintf("./dummy_catchpoint_file-%d", i)) - require.NoError(t, err) - err = f.Close() - require.NoError(t, err) - } - - for i := 0; i < dummyCatchpointFilesToCreate; i++ { - err := au.accountsq.storeCatchpoint(context.Background(), basics.Round(i), fmt.Sprintf("./dummy_catchpoint_file-%d", i), "", 0) - require.NoError(t, err) - } - err = deleteStoredCatchpoints(context.Background(), au.accountsq, au.dbDirectory) - require.NoError(t, err) - - for i := 0; i < dummyCatchpointFilesToCreate; i++ { - // ensure that all the files were deleted. - _, err := os.Open(fmt.Sprintf("./dummy_catchpoint_file-%d", i)) - require.True(t, os.IsNotExist(err)) - } - fileNames, err := au.accountsq.getOldestCatchpointFiles(context.Background(), dummyCatchpointFilesToCreate, 0) - require.NoError(t, err) - require.Equal(t, 0, len(fileNames)) -} - // listAndCompareComb lists the assets/applications and then compares against the expected // It repeats with different combinations of the limit parameters func listAndCompareComb(t *testing.T, au *accountUpdates, expected map[basics.CreatableIndex]ledgercore.ModifiedCreatable) { @@ -1167,94 +1105,6 @@ func TestListCreatables(t *testing.T) { listAndCompareComb(t, au, expectedDbImage) } -func TestIsWritingCatchpointFile(t *testing.T) { - partitiontest.PartitionTest(t) - - au := &accountUpdates{} - - au.catchpointWriting = -1 - ans := au.IsWritingCatchpointFile() - require.True(t, ans) - - au.catchpointWriting = 0 - ans = au.IsWritingCatchpointFile() - require.False(t, ans) -} - -func TestGetCatchpointStream(t *testing.T) { - partitiontest.PartitionTest(t) - - accts := []map[basics.Address]basics.AccountData{ledgertesting.RandomAccounts(20, true)} - - ml := makeMockLedgerForTracker(t, true, 10, protocol.ConsensusCurrentVersion, accts) - defer ml.Close() - - conf := config.GetDefaultLocal() - conf.CatchpointInterval = 1 - au := newAcctUpdates(t, ml, conf, ".") - - err := au.loadFromDisk(ml, 0) - require.NoError(t, err) - defer au.close() - - filesToCreate := 4 - - temporaryDirectroy, err := ioutil.TempDir(os.TempDir(), "catchpoints") - require.NoError(t, err) - defer func() { - os.RemoveAll(temporaryDirectroy) - }() - catchpointsDirectory := filepath.Join(temporaryDirectroy, "catchpoints") - err = os.Mkdir(catchpointsDirectory, 0777) - require.NoError(t, err) - - au.dbDirectory = temporaryDirectroy - - // Create the catchpoint files with dummy data - for i := 0; i < filesToCreate; i++ { - fileName := filepath.Join("catchpoints", fmt.Sprintf("%d.catchpoint", i)) - data := []byte{byte(i), byte(i + 1), byte(i + 2)} - err = ioutil.WriteFile(filepath.Join(temporaryDirectroy, fileName), data, 0666) - require.NoError(t, err) - - // Store the catchpoint into the database - err := au.accountsq.storeCatchpoint(context.Background(), basics.Round(i), fileName, "", int64(len(data))) - require.NoError(t, err) - } - - dataRead := make([]byte, 3) - var n int - - // File on disk, and database has the record - reader, err := au.GetCatchpointStream(basics.Round(1)) - n, err = reader.Read(dataRead) - require.NoError(t, err) - require.Equal(t, 3, n) - outData := []byte{1, 2, 3} - require.Equal(t, outData, dataRead) - len, err := reader.Size() - require.NoError(t, err) - require.Equal(t, int64(3), len) - - // File deleted, but record in the database - err = os.Remove(filepath.Join(temporaryDirectroy, "catchpoints", "2.catchpoint")) - reader, err = au.GetCatchpointStream(basics.Round(2)) - require.Equal(t, ledgercore.ErrNoEntry{}, err) - require.Nil(t, reader) - - // File on disk, but database lost the record - err = au.accountsq.storeCatchpoint(context.Background(), basics.Round(3), "", "", 0) - reader, err = au.GetCatchpointStream(basics.Round(3)) - n, err = reader.Read(dataRead) - require.NoError(t, err) - require.Equal(t, 3, n) - outData = []byte{3, 4, 5} - require.Equal(t, outData, dataRead) - - err = deleteStoredCatchpoints(context.Background(), au.accountsq, au.dbDirectory) - require.NoError(t, err) -} - func accountsAll(tx *sql.Tx) (bals map[basics.Address]basics.AccountData, err error) { rows, err := tx.Query("SELECT address, data FROM accountbase") if err != nil { @@ -1312,9 +1162,6 @@ func BenchmarkLargeMerkleTrieRebuild(b *testing.B) { cfg := config.GetDefaultLocal() cfg.Archival = true au := newAcctUpdates(b, ml, cfg, ".") - - err := au.loadFromDisk(ml, 0) - require.NoError(b, err) defer au.close() // at this point, the database was created. We want to fill the accounts data @@ -1336,7 +1183,7 @@ func BenchmarkLargeMerkleTrieRebuild(b *testing.B) { require.NoError(b, err) } - err = ml.dbs.Wdb.Atomic(func(ctx context.Context, tx *sql.Tx) (err error) { + err := ml.dbs.Wdb.Atomic(func(ctx context.Context, tx *sql.Tx) (err error) { return updateAccountsHashRound(tx, 1) }) require.NoError(b, err) @@ -1350,72 +1197,6 @@ func BenchmarkLargeMerkleTrieRebuild(b *testing.B) { b.ReportMetric(float64(accountsNumber), "entries/trie") } -func BenchmarkLargeCatchpointWriting(b *testing.B) { - proto := config.Consensus[protocol.ConsensusCurrentVersion] - - accts := []map[basics.Address]basics.AccountData{ledgertesting.RandomAccounts(5, true)} - - pooldata := basics.AccountData{} - pooldata.MicroAlgos.Raw = 1000 * 1000 * 1000 * 1000 - pooldata.Status = basics.NotParticipating - accts[0][testPoolAddr] = pooldata - - sinkdata := basics.AccountData{} - sinkdata.MicroAlgos.Raw = 1000 * 1000 * 1000 * 1000 - sinkdata.Status = basics.NotParticipating - accts[0][testSinkAddr] = sinkdata - - ml := makeMockLedgerForTracker(b, true, 10, protocol.ConsensusCurrentVersion, accts) - defer ml.Close() - - cfg := config.GetDefaultLocal() - cfg.Archival = true - au := newAcctUpdates(b, ml, cfg, ".") - - temporaryDirectroy, err := ioutil.TempDir(os.TempDir(), "catchpoints") - require.NoError(b, err) - defer func() { - os.RemoveAll(temporaryDirectroy) - }() - catchpointsDirectory := filepath.Join(temporaryDirectroy, "catchpoints") - err = os.Mkdir(catchpointsDirectory, 0777) - require.NoError(b, err) - - au.dbDirectory = temporaryDirectroy - - err = au.loadFromDisk(ml, 0) - require.NoError(b, err) - defer au.close() - - // at this point, the database was created. We want to fill the accounts data - accountsNumber := 6000000 * b.N - err = ml.dbs.Wdb.Atomic(func(ctx context.Context, tx *sql.Tx) (err error) { - for i := 0; i < accountsNumber-5-2; { // subtract the account we've already created above, plus the sink/reward - var updates compactAccountDeltas - for k := 0; i < accountsNumber-5-2 && k < 1024; k++ { - addr := ledgertesting.RandomAddress() - acctData := basics.AccountData{} - acctData.MicroAlgos.Raw = 1 - updates.upsert(addr, accountDelta{new: acctData}) - i++ - } - - _, err = accountsNewRound(tx, updates, nil, proto, basics.Round(1)) - if err != nil { - return - } - } - - return updateAccountsHashRound(tx, 1) - }) - require.NoError(b, err) - - b.ResetTimer() - au.generateCatchpoint(basics.Round(0), "0#ABCD", crypto.Digest{}, time.Second) - b.StopTimer() - b.ReportMetric(float64(accountsNumber), "accounts") -} - func BenchmarkCompactDeltas(b *testing.B) { b.Run("account-deltas", func(b *testing.B) { if b.N < 500 { @@ -1507,135 +1288,6 @@ func TestCompactDeltas(t *testing.T) { } -func TestReproducibleCatchpointLabels(t *testing.T) { - partitiontest.PartitionTest(t) - - if runtime.GOARCH == "arm" || runtime.GOARCH == "arm64" { - t.Skip("This test is too slow on ARM and causes travis builds to time out") - } - // create new protocol version, which has lower lookback - testProtocolVersion := protocol.ConsensusVersion("test-protocol-TestReproducibleCatchpointLabels") - protoParams := config.Consensus[protocol.ConsensusCurrentVersion] - protoParams.MaxBalLookback = 32 - protoParams.SeedLookback = 2 - protoParams.SeedRefreshInterval = 8 - config.Consensus[testProtocolVersion] = protoParams - defer func() { - delete(config.Consensus, testProtocolVersion) - }() - - accts := []map[basics.Address]basics.AccountData{ledgertesting.RandomAccounts(20, true)} - rewardsLevels := []uint64{0} - - pooldata := basics.AccountData{} - pooldata.MicroAlgos.Raw = 100 * 1000 * 1000 * 1000 * 1000 - pooldata.Status = basics.NotParticipating - accts[0][testPoolAddr] = pooldata - - sinkdata := basics.AccountData{} - sinkdata.MicroAlgos.Raw = 1000 * 1000 * 1000 * 1000 - sinkdata.Status = basics.NotParticipating - accts[0][testSinkAddr] = sinkdata - - ml := makeMockLedgerForTracker(t, false, 1, testProtocolVersion, accts) - defer ml.Close() - - cfg := config.GetDefaultLocal() - cfg.CatchpointInterval = 50 - cfg.CatchpointTracking = 1 - au := newAcctUpdates(t, ml, cfg, ".") - - err := au.loadFromDisk(ml, 0) - require.NoError(t, err) - defer au.close() - - rewardLevel := uint64(0) - - const testCatchpointLabelsCount = 5 - - // lastCreatableID stores asset or app max used index to get rid of conflicts - lastCreatableID := crypto.RandUint64() % 512 - knownCreatables := make(map[basics.CreatableIndex]bool) - catchpointLabels := make(map[basics.Round]string) - ledgerHistory := make(map[basics.Round]*mockLedgerForTracker) - roundDeltas := make(map[basics.Round]ledgercore.StateDelta) - for i := basics.Round(1); i <= basics.Round(testCatchpointLabelsCount*cfg.CatchpointInterval); i++ { - rewardLevelDelta := crypto.RandUint64() % 5 - rewardLevel += rewardLevelDelta - var updates ledgercore.AccountDeltas - var totals map[basics.Address]basics.AccountData - base := accts[i-1] - updates, totals, lastCreatableID = ledgertesting.RandomDeltasBalancedFull(1, base, rewardLevel, lastCreatableID) - prevTotals, err := au.Totals(basics.Round(i - 1)) - require.NoError(t, err) - - newPool := totals[testPoolAddr] - newPool.MicroAlgos.Raw -= prevTotals.RewardUnits() * rewardLevelDelta - updates.Upsert(testPoolAddr, newPool) - totals[testPoolAddr] = newPool - - newTotals := ledgertesting.CalculateNewRoundAccountTotals(t, updates, rewardLevel, protoParams, base, prevTotals) - - blk := bookkeeping.Block{ - BlockHeader: bookkeeping.BlockHeader{ - Round: basics.Round(i), - }, - } - blk.RewardsLevel = rewardLevel - blk.CurrentProtocol = testProtocolVersion - delta := ledgercore.MakeStateDelta(&blk.BlockHeader, 0, updates.Len(), 0) - delta.Accts.MergeAccounts(updates) - delta.Creatables = creatablesFromUpdates(base, updates, knownCreatables) - delta.Totals = newTotals - - au.newBlock(blk, delta) - ml.scheduleCommit(i) - ml.addMockBlock(blockEntry{block: blk}, delta) - accts = append(accts, totals) - rewardsLevels = append(rewardsLevels, rewardLevel) - roundDeltas[i] = delta - - // if this is a catchpoint round, save the label. - if uint64(i)%cfg.CatchpointInterval == 0 { - ml.waitAccountsWriting() - catchpointLabels[i] = au.GetLastCatchpointLabel() - ledgerHistory[i] = ml.fork(t) - ledgerHistory[i].trackers.initialize(ml.trackers.driver, ledgerHistory[i], []ledgerTracker{au}) - ledgerHistory[i].trackers.dbRound = ml.trackers.dbRound - defer ledgerHistory[i].Close() - } - } - - // test in revese what happens when we try to repeat the exact same blocks. - // start off with the catchpoint before the last one - startingRound := basics.Round((testCatchpointLabelsCount - 1) * cfg.CatchpointInterval) - for ; startingRound > basics.Round(cfg.CatchpointInterval); startingRound -= basics.Round(cfg.CatchpointInterval) { - au.close() - ml2 := ledgerHistory[startingRound] - err := au.loadFromDisk(ml2, ml2.trackers.dbRound) - require.NoError(t, err) - - for i := startingRound + 1; i <= basics.Round(testCatchpointLabelsCount*cfg.CatchpointInterval); i++ { - blk := bookkeeping.Block{ - BlockHeader: bookkeeping.BlockHeader{ - Round: basics.Round(i), - }, - } - blk.RewardsLevel = rewardsLevels[i] - blk.CurrentProtocol = testProtocolVersion - delta := roundDeltas[i] - au.newBlock(blk, delta) - ml2.scheduleCommit(i) - - // if this is a catchpoint round, check the label. - if uint64(i)%cfg.CatchpointInterval == 0 { - ml2.waitAccountsWriting() - require.Equal(t, catchpointLabels[i], au.GetLastCatchpointLabel()) - } - } - } -} - // TestCachesInitialization test the functionality of the initializeCaches cache. func TestCachesInitialization(t *testing.T) { partitiontest.PartitionTest(t) @@ -1666,9 +1318,6 @@ func TestCachesInitialization(t *testing.T) { conf := config.GetDefaultLocal() au := newAcctUpdates(t, ml, conf, ".") - err := au.loadFromDisk(ml, 0) - require.NoError(t, err) - // cover initialRounds genesis blocks rewardLevel := uint64(0) for i := 1; i < int(initialRounds); i++ { @@ -1705,8 +1354,8 @@ func TestCachesInitialization(t *testing.T) { delta.Totals = accumulateTotals(t, protocol.ConsensusCurrentVersion, []map[basics.Address]basics.AccountData{totals}, rewardLevel) ml.addMockBlock(blockEntry{block: blk}, delta) au.newBlock(blk, delta) - ml.scheduleCommit(basics.Round(i)) - ml.waitAccountsWriting() + ml.trackers.committedUpTo(basics.Round(i)) + ml.trackers.waitAccountsWriting() accts = append(accts, totals) rewardsLevels = append(rewardsLevels, rewardLevel) } @@ -1727,8 +1376,6 @@ func TestCachesInitialization(t *testing.T) { conf = config.GetDefaultLocal() au = newAcctUpdates(t, ml2, conf, ".") defer au.close() - err = au.loadFromDisk(ml2, 0) - require.NoError(t, err) // make sure the deltas array end up containing only the most recent 320 rounds. require.Equal(t, int(proto.MaxBalLookback), len(au.deltas)) @@ -1845,8 +1492,8 @@ func TestSplittingConsensusVersionCommits(t *testing.T) { rewardsLevels = append(rewardsLevels, rewardLevel) } // now, commit and verify that the produceCommittingTask method broken the range correctly. - ml.scheduleCommit(lastRoundToWrite) - ml.waitAccountsWriting() + ml.trackers.committedUpTo(lastRoundToWrite) + ml.trackers.waitAccountsWriting() require.Equal(t, basics.Round(initialRounds+extraRounds)-1, au.cachedDBRound) } @@ -1961,8 +1608,8 @@ func TestSplittingConsensusVersionCommitsBoundry(t *testing.T) { rewardsLevels = append(rewardsLevels, rewardLevel) } // now, commit and verify that the produceCommittingTask method broken the range correctly. - ml.scheduleCommit(endOfFirstNewProtocolSegment) - ml.waitAccountsWriting() + ml.trackers.committedUpTo(endOfFirstNewProtocolSegment) + ml.trackers.waitAccountsWriting() require.Equal(t, basics.Round(initialRounds+extraRounds)-1, au.cachedDBRound) // write additional extraRounds elements and verify these can be flushed. @@ -1996,8 +1643,8 @@ func TestSplittingConsensusVersionCommitsBoundry(t *testing.T) { accts = append(accts, totals) rewardsLevels = append(rewardsLevels, rewardLevel) } - ml.scheduleCommit(endOfFirstNewProtocolSegment + basics.Round(extraRounds)) - ml.waitAccountsWriting() + ml.trackers.committedUpTo(endOfFirstNewProtocolSegment + basics.Round(extraRounds)) + ml.trackers.waitAccountsWriting() require.Equal(t, basics.Round(initialRounds+2*extraRounds), au.cachedDBRound) } diff --git a/ledger/applications_test.go b/ledger/applications_test.go index 6d299e2606..7768329689 100644 --- a/ledger/applications_test.go +++ b/ledger/applications_test.go @@ -19,6 +19,7 @@ package ledger import ( "encoding/hex" "testing" + "time" "github.com/stretchr/testify/require" @@ -33,16 +34,9 @@ import ( ) func commitRound(offset uint64, dbRound basics.Round, l *Ledger) { - l.trackers.accountsWriting.Add(1) - l.trackers.commitRound(deferredCommit{offset, dbRound, 0}) - l.trackers.accountsWriting.Wait() -} - -func stopCommitSyncer(l *Ledger) { - l.trackers.ctxCancel() // force commitSyncer to exit - // wait commitSyncer to exit - // the test calls commitRound directly and does not need commitSyncer/committedUpTo - <-l.trackers.commitSyncerClosed + l.trackers.lastFlushTime = time.Time{} + l.trackers.scheduleCommit(l.Latest(), l.Latest()-(dbRound+basics.Round(offset))) + l.trackers.waitAccountsWriting() } // test ensures that @@ -138,8 +132,6 @@ return` a.NoError(err) defer l.Close() - stopCommitSyncer(l) - txHeader := transactions.Header{ Sender: creator, Fee: basics.MicroAlgos{Raw: proto.MinTxnFee * 2}, @@ -220,10 +212,12 @@ return` var buf []byte err = l.accts.accountsq.lookupStmt.QueryRow(creator[:]).Scan(&rowid, &dbRound, &buf) a.NoError(err) + a.Equal(basics.Round(4), dbRound) a.Equal(expectedCreator, buf) err = l.accts.accountsq.lookupStmt.QueryRow(userOptin[:]).Scan(&rowid, &dbRound, &buf) a.NoError(err) + a.Equal(basics.Round(4), dbRound) a.Equal(expectedUserOptIn, buf) pad, err := l.accts.accountsq.lookup(userOptin) a.NoError(err) @@ -234,6 +228,7 @@ return` err = l.accts.accountsq.lookupStmt.QueryRow(userLocal[:]).Scan(&rowid, &dbRound, &buf) a.NoError(err) + a.Equal(basics.Round(4), dbRound) a.Equal(expectedUserLocal, buf) ad, err = l.Lookup(dbRound, userLocal) @@ -348,8 +343,6 @@ return` a.NoError(err) defer l.Close() - stopCommitSyncer(l) - genesisID := t.Name() txHeader := transactions.Header{ Sender: creator, @@ -593,8 +586,6 @@ return` a.NoError(err) defer l.Close() - stopCommitSyncer(l) - genesisID := t.Name() txHeader := transactions.Header{ Sender: creator, @@ -744,8 +735,6 @@ return` a.NoError(err) defer l.Close() - stopCommitSyncer(l) - genesisID := t.Name() txHeader := transactions.Header{ Sender: creator, @@ -937,8 +926,6 @@ func testAppAccountDeltaIndicesCompatibility(t *testing.T, source string, accoun a.NoError(err) defer l.Close() - stopCommitSyncer(l) - genesisID := t.Name() txHeader := transactions.Header{ Sender: creator, diff --git a/ledger/archival_test.go b/ledger/archival_test.go index 99df0fd033..9d4f3b15d0 100644 --- a/ledger/archival_test.go +++ b/ledger/archival_test.go @@ -89,14 +89,6 @@ func (wl *wrappedLedger) trackerLog() logging.Logger { return wl.l.trackerLog() } -func (wl *wrappedLedger) scheduleCommit(basics.Round) { - return -} - -func (wl *wrappedLedger) waitAccountsWriting() { - return -} - func (wl *wrappedLedger) GenesisHash() crypto.Digest { return wl.l.GenesisHash() } @@ -804,8 +796,8 @@ func checkTrackers(t *testing.T, wl *wrappedLedger, rnd basics.Round) (basics.Ro for _, trk := range wl.l.trackers.trackers { if au, ok := trk.(*accountUpdates); ok { wl.l.trackers.waitAccountsWriting() - minSave = trk.committedUpTo(rnd) - wl.l.trackers.scheduleCommit(rnd) + minSave, _ = trk.committedUpTo(rnd) + wl.l.trackers.committedUpTo(rnd) wl.l.trackers.waitAccountsWriting() if minSave < minMinSave { minMinSave = minSave @@ -818,9 +810,9 @@ func checkTrackers(t *testing.T, wl *wrappedLedger, rnd basics.Round) (basics.Ro au = cleanTracker.(*accountUpdates) cfg := config.GetDefaultLocal() cfg.Archival = true - au.initialize(cfg, "") + au.initialize(cfg) } else { - minSave = trk.committedUpTo(rnd) + minSave, _ = trk.committedUpTo(rnd) if minSave < minMinSave { minMinSave = minSave } diff --git a/ledger/bulletin.go b/ledger/bulletin.go index bf711a605c..1e95ee2ab7 100644 --- a/ledger/bulletin.go +++ b/ledger/bulletin.go @@ -92,7 +92,7 @@ func (b *bulletin) close() { func (b *bulletin) newBlock(blk bookkeeping.Block, delta ledgercore.StateDelta) { } -func (b *bulletin) committedUpTo(rnd basics.Round) basics.Round { +func (b *bulletin) committedUpTo(rnd basics.Round) (retRound, lookback basics.Round) { b.mu.Lock() defer b.mu.Unlock() @@ -106,7 +106,7 @@ func (b *bulletin) committedUpTo(rnd basics.Round) basics.Round { } b.latestRound = rnd - return rnd + return rnd, basics.Round(0) } func (b *bulletin) prepareCommit(dcc *deferredCommitContext) error { @@ -117,8 +117,11 @@ func (b *bulletin) commitRound(context.Context, *sql.Tx, *deferredCommitContext) return nil } -func (b *bulletin) postCommit(deferredCommitContext) { +func (b *bulletin) postCommit(ctx context.Context, dcc *deferredCommitContext) { } func (b *bulletin) handleUnorderedCommit(uint64, basics.Round, basics.Round) { } +func (b *bulletin) produceCommittingTask(committedRound basics.Round, dbRound basics.Round, dcr *deferredCommitRange) *deferredCommitRange { + return dcr +} diff --git a/ledger/catchpointtracker.go b/ledger/catchpointtracker.go new file mode 100644 index 0000000000..e1e0d2b5e3 --- /dev/null +++ b/ledger/catchpointtracker.go @@ -0,0 +1,901 @@ +// Copyright (C) 2019-2021 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 ( + "context" + "database/sql" + "encoding/hex" + "fmt" + "os" + "path/filepath" + "strconv" + "sync/atomic" + "time" + + "github.com/algorand/go-deadlock" + + "github.com/algorand/go-algorand/config" + "github.com/algorand/go-algorand/crypto" + "github.com/algorand/go-algorand/crypto/merkletrie" + "github.com/algorand/go-algorand/data/basics" + "github.com/algorand/go-algorand/data/bookkeeping" + "github.com/algorand/go-algorand/ledger/ledgercore" + "github.com/algorand/go-algorand/logging" + "github.com/algorand/go-algorand/logging/telemetryspec" + "github.com/algorand/go-algorand/protocol" + "github.com/algorand/go-algorand/util/db" +) + +// trieCachedNodesCount defines how many balances trie nodes we would like to keep around in memory. +// value was calibrated using BenchmarkCalibrateCacheNodeSize +var trieCachedNodesCount = 9000 + +// merkleCommitterNodesPerPage controls how many nodes will be stored in a single page +// value was calibrated using BenchmarkCalibrateNodesPerPage +var merkleCommitterNodesPerPage = int64(116) + +const ( + // trieRebuildAccountChunkSize defines the number of accounts that would get read at a single chunk + // before added to the trie during trie construction + trieRebuildAccountChunkSize = 16384 + // trieRebuildCommitFrequency defines the number of accounts that would get added before we call evict to commit the changes and adjust the memory cache. + trieRebuildCommitFrequency = 65536 + // trieAccumulatedChangesFlush defines the number of pending changes that would be applied to the merkle trie before + // we attempt to commit them to disk while writing a batch of rounds balances to disk. + trieAccumulatedChangesFlush = 256 +) + +// TrieMemoryConfig is the memory configuration setup used for the merkle trie. +var TrieMemoryConfig = merkletrie.MemoryConfig{ + NodesCountPerPage: merkleCommitterNodesPerPage, + CachedNodesCount: trieCachedNodesCount, + PageFillFactor: 0.95, + MaxChildrenPagesThreshold: 64, +} + +type catchpointTracker struct { + // dbDirectory is the directory where the ledger and block sql file resides as well as the parent directory for the catchup files to be generated + dbDirectory string + + // catchpointInterval is the configured interval at which the accountUpdates would generate catchpoint labels and catchpoint files. + catchpointInterval uint64 + + // catchpointFileHistoryLength defines how many catchpoint files we want to store back. + // 0 means don't store any, -1 mean unlimited and positive number suggest the number of most recent catchpoint files. + catchpointFileHistoryLength int + + // archivalLedger determines whether the associated ledger was configured as archival ledger or not. + archivalLedger bool + + // Prepared SQL statements for fast accounts DB lookups. + accountsq *accountsDbQueries + + // log copied from ledger + log logging.Logger + + // Connection to the database. + dbs db.Pair + + // The last catchpoint label that was written to the database. Should always align with what's in the database. + // note that this is the last catchpoint *label* and not the catchpoint file. + lastCatchpointLabel string + + // catchpointSlowWriting suggest to the accounts writer that it should finish writing up the catchpoint file ASAP. + // when this channel is closed, the accounts writer would try and complete the writing as soon as possible. + // otherwise, it would take it's time and perform periodic sleeps between chunks processing. + catchpointSlowWriting chan struct{} + + // catchpointWriting help to synchronize the catchpoint file writing. When this atomic variable is 0, no writing is going on. + // Any non-zero value indicates a catchpoint being written, or scheduled to be written. + catchpointWriting int32 + + // The Trie tracking the current account balances. Always matches the balances that were + // written to the database. + balancesTrie *merkletrie.Trie + + // catchpointsMu is the synchronization mutex for accessing the various non-static variables. + catchpointsMu deadlock.RWMutex + + // roundDigest stores the digest of the block for every round starting with dbRound and every round after it. + roundDigest []crypto.Digest +} + +// initialize initializes the catchpointTracker structure +func (ct *catchpointTracker) initialize(cfg config.Local, dbPathPrefix string) { + ct.dbDirectory = filepath.Dir(dbPathPrefix) + ct.archivalLedger = cfg.Archival + switch cfg.CatchpointTracking { + case -1: + ct.catchpointInterval = 0 + default: + // give a warning, then fall thought + logging.Base().Warnf("catchpointTracker: the CatchpointTracking field in the config.json file contains an invalid value (%d). The default value of 0 would be used instead.", cfg.CatchpointTracking) + fallthrough + case 0: + if ct.archivalLedger { + ct.catchpointInterval = cfg.CatchpointInterval + } else { + ct.catchpointInterval = 0 + } + case 1: + ct.catchpointInterval = cfg.CatchpointInterval + } + + ct.catchpointFileHistoryLength = cfg.CatchpointFileHistoryLength + if cfg.CatchpointFileHistoryLength < -1 { + ct.catchpointFileHistoryLength = -1 + } +} + +// GetLastCatchpointLabel retrieves the last catchpoint label that was stored to the database. +func (ct *catchpointTracker) GetLastCatchpointLabel() string { + ct.catchpointsMu.RLock() + defer ct.catchpointsMu.RUnlock() + return ct.lastCatchpointLabel +} + +// loadFromDisk loads the state of a tracker from persistent +// storage. The ledger argument allows loadFromDisk to load +// blocks from the database, or access its own state. The +// ledgerForTracker interface abstracts away the details of +// ledger internals so that individual trackers can be tested +// in isolation. +func (ct *catchpointTracker) loadFromDisk(l ledgerForTracker, lastBalancesRound basics.Round) (err error) { + ct.log = l.trackerLog() + ct.dbs = l.trackerDB() + + err = ct.dbs.Wdb.Atomic(func(ctx context.Context, tx *sql.Tx) error { + err0 := ct.accountsInitializeHashes(ctx, tx, lastBalancesRound) + if err0 != nil { + return err0 + } + return nil + }) + + if err != nil { + return err + } + + ct.accountsq, err = accountsInitDbQueries(ct.dbs.Rdb.Handle, ct.dbs.Wdb.Handle) + if err != nil { + return + } + + ct.lastCatchpointLabel, _, err = ct.accountsq.readCatchpointStateString(context.Background(), catchpointStateLastCatchpoint) + if err != nil { + return + } + + writingCatchpointRound, _, err := ct.accountsq.readCatchpointStateUint64(context.Background(), catchpointStateWritingCatchpoint) + if err != nil { + return err + } + if writingCatchpointRound == 0 || !ct.catchpointEnabled() { + return nil + } + var dbRound basics.Round + // make sure that the database is at the desired round. + err = ct.dbs.Rdb.Atomic(func(ctx context.Context, tx *sql.Tx) (err error) { + dbRound, err = accountsRound(tx) + return + }) + if err != nil { + return err + } + if dbRound != basics.Round(writingCatchpointRound) { + return nil + } + + blk, err := l.Block(dbRound) + if err != nil { + return err + } + blockHeaderDigest := blk.Digest() + + ct.catchpointWriting = 0 + // keep these channel closed if we're not generating catchpoint + ct.catchpointSlowWriting = make(chan struct{}, 1) + close(ct.catchpointSlowWriting) + ct.roundDigest = nil + + ct.generateCatchpoint(context.Background(), basics.Round(writingCatchpointRound), ct.lastCatchpointLabel, blockHeaderDigest, time.Duration(0)) + return nil +} + +// newBlock informs the tracker of a new block from round +// rnd and a given ledgercore.StateDelta as produced by BlockEvaluator. +func (ct *catchpointTracker) newBlock(blk bookkeeping.Block, delta ledgercore.StateDelta) { + ct.catchpointsMu.Lock() + defer ct.catchpointsMu.Unlock() + ct.roundDigest = append(ct.roundDigest, blk.Digest()) +} + +// committedUpTo implements the ledgerTracker interface for catchpointTracker. +// The method informs the tracker that committedRound and all it's previous rounds have +// been committed to the block database. The method returns what is the oldest round +// number that can be removed from the blocks database as well as the lookback that this +// tracker maintains. +func (ct *catchpointTracker) committedUpTo(rnd basics.Round) (retRound, lookback basics.Round) { + return rnd, basics.Round(0) +} + +func (ct *catchpointTracker) produceCommittingTask(committedRound basics.Round, dbRound basics.Round, dcr *deferredCommitRange) *deferredCommitRange { + var hasMultipleIntermediateCatchpoint, hasIntermediateCatchpoint bool + + newBase := dcr.oldBase + basics.Round(dcr.offset) + + // check if there was a catchpoint between dcc.oldBase+lookback and dcc.oldBase+offset+lookback + if ct.catchpointInterval > 0 { + nextCatchpointRound := ((uint64(dcr.oldBase+dcr.lookback) + ct.catchpointInterval) / ct.catchpointInterval) * ct.catchpointInterval + + if nextCatchpointRound < uint64(dcr.oldBase+dcr.lookback)+dcr.offset { + mostRecentCatchpointRound := (uint64(committedRound) / ct.catchpointInterval) * ct.catchpointInterval + newBase = basics.Round(nextCatchpointRound) - dcr.lookback + if mostRecentCatchpointRound > nextCatchpointRound { + hasMultipleIntermediateCatchpoint = true + // skip if there is more than one catchpoint in queue + newBase = basics.Round(mostRecentCatchpointRound) - dcr.lookback + } + hasIntermediateCatchpoint = true + } + } + + // if we're still writing the previous balances, we can't move forward yet. + if ct.IsWritingCatchpointFile() { + // if we hit this path, it means that we're still writing a catchpoint. + // see if the new delta range contains another catchpoint. + if hasIntermediateCatchpoint { + // check if we're already attempting to perform fast-writing. + select { + case <-ct.catchpointSlowWriting: + // yes, we're already doing fast-writing. + default: + // no, we're not yet doing fast writing, make it so. + close(ct.catchpointSlowWriting) + } + } + return nil + } + + dcr.offset = uint64(newBase - dcr.oldBase) + + // check to see if this is a catchpoint round + dcr.isCatchpointRound = ct.isCatchpointRound(dcr.offset, dcr.oldBase, dcr.lookback) + + if dcr.isCatchpointRound && ct.archivalLedger { + // store non-zero ( all ones ) into the catchpointWriting atomic variable to indicate that a catchpoint is being written ( or, queued to be written ) + atomic.StoreInt32(&ct.catchpointWriting, int32(-1)) + ct.catchpointSlowWriting = make(chan struct{}, 1) + if hasMultipleIntermediateCatchpoint { + close(ct.catchpointSlowWriting) + } + } + + dcr.catchpointWriting = &ct.catchpointWriting + + return dcr +} + +// prepareCommit, commitRound and postCommit are called when it is time to commit tracker's data. +// If an error returned the process is aborted. +func (ct *catchpointTracker) prepareCommit(dcc *deferredCommitContext) error { + ct.catchpointsMu.RLock() + defer ct.catchpointsMu.RUnlock() + if dcc.isCatchpointRound { + dcc.committedRoundDigest = ct.roundDigest[dcc.offset+uint64(dcc.lookback)-1] + } + return nil +} + +func (ct *catchpointTracker) commitRound(ctx context.Context, tx *sql.Tx, dcc *deferredCommitContext) (err error) { + treeTargetRound := basics.Round(0) + offset := dcc.offset + dbRound := dcc.oldBase + + defer func() { + if err != nil { + if dcc.isCatchpointRound && ct.archivalLedger { + atomic.StoreInt32(&ct.catchpointWriting, 0) + } + } + }() + + if ct.catchpointEnabled() { + var mc *MerkleCommitter + mc, err = MakeMerkleCommitter(tx, false) + if err != nil { + return + } + + var trie *merkletrie.Trie + if ct.balancesTrie == nil { + trie, err = merkletrie.MakeTrie(mc, TrieMemoryConfig) + if err != nil { + ct.log.Warnf("unable to create merkle trie during committedUpTo: %v", err) + return err + } + ct.balancesTrie = trie + } else { + ct.balancesTrie.SetCommitter(mc) + } + treeTargetRound = dbRound + basics.Round(offset) + } + + if dcc.updateStats { + dcc.stats.MerkleTrieUpdateDuration = time.Duration(time.Now().UnixNano()) + } + + err = ct.accountsUpdateBalances(dcc.compactAccountDeltas) + if err != nil { + return err + } + + if dcc.updateStats { + now := time.Duration(time.Now().UnixNano()) + dcc.stats.MerkleTrieUpdateDuration = now - dcc.stats.MerkleTrieUpdateDuration + } + + err = updateAccountsHashRound(tx, treeTargetRound) + if err != nil { + return err + } + + if dcc.isCatchpointRound { + dcc.trieBalancesHash, err = ct.balancesTrie.RootHash() + if err != nil { + return err + } + } + return nil +} + +func (ct *catchpointTracker) postCommit(ctx context.Context, dcc *deferredCommitContext) { + var err error + if dcc.isCatchpointRound { + dcc.catchpointLabel, err = ct.accountsCreateCatchpointLabel(dcc.newBase+dcc.lookback, dcc.roundTotals, dcc.committedRoundDigest, dcc.trieBalancesHash) + if err != nil { + ct.log.Warnf("commitRound : unable to create a catchpoint label: %v", err) + } + } + if ct.balancesTrie != nil { + _, err = ct.balancesTrie.Evict(false) + if err != nil { + ct.log.Warnf("merkle trie failed to evict: %v", err) + } + } + + if dcc.isCatchpointRound && dcc.catchpointLabel != "" { + ct.lastCatchpointLabel = dcc.catchpointLabel + } + dcc.updatingBalancesDuration = time.Since(dcc.flushTime) + + if dcc.updateStats { + dcc.stats.MemoryUpdatesDuration = time.Duration(time.Now().UnixNano()) + } + + ct.catchpointsMu.Lock() + + ct.roundDigest = ct.roundDigest[dcc.offset:] + + ct.catchpointsMu.Unlock() + + if dcc.isCatchpointRound && ct.archivalLedger && dcc.catchpointLabel != "" { + // generate the catchpoint file. This need to be done inline so that it will block any new accounts that from being written. + // the generateCatchpoint expects that the accounts data would not be modified in the background during it's execution. + ct.generateCatchpoint(ctx, basics.Round(dcc.offset)+dcc.oldBase+dcc.lookback, dcc.catchpointLabel, dcc.committedRoundDigest, dcc.updatingBalancesDuration) + } + // in scheduleCommit, we expect that this function to update the catchpointWriting when + // it's on a catchpoint round and it's an archival ledger. Doing this in a deferred function + // here would prevent us from "forgetting" to update this variable later on. + if dcc.isCatchpointRound && ct.archivalLedger { + atomic.StoreInt32(dcc.catchpointWriting, 0) + } +} + +// handleUnorderedCommit is a special method for handling deferred commits that are out of order. +// Tracker might update own state in this case. For example, account updates tracker cancels +// scheduled catchpoint writing that deferred commit. +func (ct *catchpointTracker) handleUnorderedCommit(offset uint64, dbRound basics.Round, lookback basics.Round) { + // if this is an archival ledger, we might need to update the catchpointWriting variable. + if ct.archivalLedger { + // determine if this was a catchpoint round + if ct.isCatchpointRound(offset, dbRound, lookback) { + // it was a catchpoint round, so update the catchpointWriting to indicate that we're done. + atomic.StoreInt32(&ct.catchpointWriting, 0) + } + } +} + +// close terminates the tracker, reclaiming any resources +// like open database connections or goroutines. close may +// be called even if loadFromDisk() is not called or does +// not succeed. +func (ct *catchpointTracker) close() { + +} + +// accountsUpdateBalances applies the given compactAccountDeltas to the merkle trie +func (ct *catchpointTracker) accountsUpdateBalances(accountsDeltas compactAccountDeltas) (err error) { + if !ct.catchpointEnabled() { + return nil + } + var added, deleted bool + accumulatedChanges := 0 + + for i := 0; i < accountsDeltas.len(); i++ { + addr, delta := accountsDeltas.getByIdx(i) + if !delta.old.accountData.IsZero() { + deleteHash := accountHashBuilder(addr, delta.old.accountData, protocol.Encode(&delta.old.accountData)) + deleted, err = ct.balancesTrie.Delete(deleteHash) + if err != nil { + return fmt.Errorf("failed to delete hash '%s' from merkle trie for account %v: %w", hex.EncodeToString(deleteHash), addr, err) + } + if !deleted { + ct.log.Warnf("failed to delete hash '%s' from merkle trie for account %v", hex.EncodeToString(deleteHash), addr) + } else { + accumulatedChanges++ + } + } + + if !delta.new.IsZero() { + addHash := accountHashBuilder(addr, delta.new, protocol.Encode(&delta.new)) + added, err = ct.balancesTrie.Add(addHash) + if err != nil { + return fmt.Errorf("attempted to add duplicate hash '%s' to merkle trie for account %v: %w", hex.EncodeToString(addHash), addr, err) + } + if !added { + ct.log.Warnf("attempted to add duplicate hash '%s' to merkle trie for account %v", hex.EncodeToString(addHash), addr) + } else { + accumulatedChanges++ + } + } + } + if accumulatedChanges >= trieAccumulatedChangesFlush { + accumulatedChanges = 0 + _, err = ct.balancesTrie.Commit() + if err != nil { + return + } + } + + // write it all to disk. + if accumulatedChanges > 0 { + _, err = ct.balancesTrie.Commit() + } + + return +} + +// IsWritingCatchpointFile returns true when a catchpoint file is being generated. The function is used by the catchup service +// to avoid memory pressure until the catchpoint file writing is complete. +func (ct *catchpointTracker) IsWritingCatchpointFile() bool { + return atomic.LoadInt32(&ct.catchpointWriting) != 0 +} + +// isCatchpointRound returns true if the round at the given offset, dbRound with the provided lookback should be a catchpoint round. +func (ct *catchpointTracker) isCatchpointRound(offset uint64, dbRound basics.Round, lookback basics.Round) bool { + return ((offset + uint64(lookback+dbRound)) > 0) && (ct.catchpointInterval != 0) && ((uint64((offset + uint64(lookback+dbRound))) % ct.catchpointInterval) == 0) +} + +// accountsCreateCatchpointLabel creates a catchpoint label and write it. +func (ct *catchpointTracker) accountsCreateCatchpointLabel(committedRound basics.Round, totals ledgercore.AccountTotals, ledgerBlockDigest crypto.Digest, trieBalancesHash crypto.Digest) (label string, err error) { + cpLabel := ledgercore.MakeCatchpointLabel(committedRound, ledgerBlockDigest, trieBalancesHash, totals) + label = cpLabel.String() + _, err = ct.accountsq.writeCatchpointStateString(context.Background(), catchpointStateLastCatchpoint, label) + return +} + +// generateCatchpoint generates a single catchpoint file +func (ct *catchpointTracker) generateCatchpoint(ctx context.Context, committedRound basics.Round, label string, committedRoundDigest crypto.Digest, updatingBalancesDuration time.Duration) { + beforeGeneratingCatchpointTime := time.Now() + catchpointGenerationStats := telemetryspec.CatchpointGenerationEventDetails{ + BalancesWriteTime: uint64(updatingBalancesDuration.Nanoseconds()), + } + + // the retryCatchpointCreation is used to repeat the catchpoint file generation in case the node crashed / aborted during startup + // before the catchpoint file generation could be completed. + retryCatchpointCreation := false + ct.log.Debugf("accountUpdates: generateCatchpoint: generating catchpoint for round %d", committedRound) + defer func() { + if !retryCatchpointCreation { + // clear the writingCatchpoint flag + _, err := ct.accountsq.writeCatchpointStateUint64(context.Background(), catchpointStateWritingCatchpoint, uint64(0)) + if err != nil { + ct.log.Warnf("accountUpdates: generateCatchpoint unable to clear catchpoint state '%s' for round %d: %v", catchpointStateWritingCatchpoint, committedRound, err) + } + } + }() + + _, err := ct.accountsq.writeCatchpointStateUint64(context.Background(), catchpointStateWritingCatchpoint, uint64(committedRound)) + if err != nil { + ct.log.Warnf("accountUpdates: generateCatchpoint unable to write catchpoint state '%s' for round %d: %v", catchpointStateWritingCatchpoint, committedRound, err) + return + } + + relCatchpointFileName := filepath.Join("catchpoints", catchpointRoundToPath(committedRound)) + absCatchpointFileName := filepath.Join(ct.dbDirectory, relCatchpointFileName) + + more := true + const shortChunkExecutionDuration = 50 * time.Millisecond + const longChunkExecutionDuration = 1 * time.Second + var chunkExecutionDuration time.Duration + select { + case <-ct.catchpointSlowWriting: + chunkExecutionDuration = longChunkExecutionDuration + default: + chunkExecutionDuration = shortChunkExecutionDuration + } + + var catchpointWriter *catchpointWriter + start := time.Now() + ledgerGeneratecatchpointCount.Inc(nil) + err = ct.dbs.Rdb.Atomic(func(dbCtx context.Context, tx *sql.Tx) (err error) { + catchpointWriter = makeCatchpointWriter(ctx, absCatchpointFileName, tx, committedRound, committedRoundDigest, label) + for more { + stepCtx, stepCancelFunction := context.WithTimeout(ctx, chunkExecutionDuration) + writeStepStartTime := time.Now() + more, err = catchpointWriter.WriteStep(stepCtx) + // accumulate the actual time we've spent writing in this step. + catchpointGenerationStats.CPUTime += uint64(time.Since(writeStepStartTime).Nanoseconds()) + stepCancelFunction() + if more && err == nil { + // we just wrote some data, but there is more to be written. + // go to sleep for while. + // before going to sleep, extend the transaction timeout so that we won't get warnings: + _, err0 := db.ResetTransactionWarnDeadline(dbCtx, tx, time.Now().Add(1*time.Second)) + if err0 != nil { + ct.log.Warnf("catchpointTracker: generateCatchpoint: failed to reset transaction warn deadline : %v", err0) + } + select { + case <-time.After(100 * time.Millisecond): + // increase the time slot allocated for writing the catchpoint, but stop when we get to the longChunkExecutionDuration limit. + // this would allow the catchpoint writing speed to ramp up while still leaving some cpu available. + chunkExecutionDuration *= 2 + if chunkExecutionDuration > longChunkExecutionDuration { + chunkExecutionDuration = longChunkExecutionDuration + } + case <-ctx.Done(): + retryCatchpointCreation = true + err2 := catchpointWriter.Abort() + if err2 != nil { + return fmt.Errorf("error removing catchpoint file : %v", err2) + } + return nil + case <-ct.catchpointSlowWriting: + chunkExecutionDuration = longChunkExecutionDuration + } + } + if err != nil { + err = fmt.Errorf("unable to create catchpoint : %v", err) + err2 := catchpointWriter.Abort() + if err2 != nil { + ct.log.Warnf("accountUpdates: generateCatchpoint: error removing catchpoint file : %v", err2) + } + return + } + } + return + }) + ledgerGeneratecatchpointMicros.AddMicrosecondsSince(start, nil) + + if err != nil { + ct.log.Warnf("accountUpdates: generateCatchpoint: %v", err) + return + } + if catchpointWriter == nil { + ct.log.Warnf("accountUpdates: generateCatchpoint: nil catchpointWriter") + return + } + + err = ct.saveCatchpointFile(committedRound, relCatchpointFileName, catchpointWriter.GetSize(), catchpointWriter.GetCatchpoint()) + if err != nil { + ct.log.Warnf("accountUpdates: generateCatchpoint: unable to save catchpoint: %v", err) + return + } + catchpointGenerationStats.FileSize = uint64(catchpointWriter.GetSize()) + catchpointGenerationStats.WritingDuration = uint64(time.Since(beforeGeneratingCatchpointTime).Nanoseconds()) + catchpointGenerationStats.AccountsCount = catchpointWriter.GetTotalAccounts() + catchpointGenerationStats.CatchpointLabel = catchpointWriter.GetCatchpoint() + ct.log.EventWithDetails(telemetryspec.Accounts, telemetryspec.CatchpointGenerationEvent, catchpointGenerationStats) + ct.log.With("writingDuration", catchpointGenerationStats.WritingDuration). + With("CPUTime", catchpointGenerationStats.CPUTime). + With("balancesWriteTime", catchpointGenerationStats.BalancesWriteTime). + With("accountsCount", catchpointGenerationStats.AccountsCount). + With("fileSize", catchpointGenerationStats.FileSize). + With("catchpointLabel", catchpointGenerationStats.CatchpointLabel). + Infof("Catchpoint file was generated") +} + +// catchpointRoundToPath calculate the catchpoint file path for a given round +func catchpointRoundToPath(rnd basics.Round) string { + irnd := int64(rnd) / 256 + outStr := "" + for irnd > 0 { + outStr = filepath.Join(outStr, fmt.Sprintf("%02x", irnd%256)) + irnd = irnd / 256 + } + outStr = filepath.Join(outStr, strconv.FormatInt(int64(rnd), 10)+".catchpoint") + return outStr +} + +// saveCatchpointFile stores the provided fileName as the stored catchpoint for the given round. +// after a successful insert operation to the database, it would delete up to 2 old entries, as needed. +// deleting 2 entries while inserting single entry allow us to adjust the size of the backing storage and have the +// database and storage realign. +func (ct *catchpointTracker) saveCatchpointFile(round basics.Round, fileName string, fileSize int64, catchpoint string) (err error) { + if ct.catchpointFileHistoryLength != 0 { + err = ct.accountsq.storeCatchpoint(context.Background(), round, fileName, catchpoint, fileSize) + if err != nil { + ct.log.Warnf("accountUpdates: saveCatchpoint: unable to save catchpoint: %v", err) + return + } + } else { + err = os.Remove(fileName) + if err != nil { + ct.log.Warnf("accountUpdates: saveCatchpoint: unable to remove file (%s): %v", fileName, err) + return + } + } + if ct.catchpointFileHistoryLength == -1 { + return + } + var filesToDelete map[basics.Round]string + filesToDelete, err = ct.accountsq.getOldestCatchpointFiles(context.Background(), 2, ct.catchpointFileHistoryLength) + if err != nil { + return fmt.Errorf("unable to delete catchpoint file, getOldestCatchpointFiles failed : %v", err) + } + for round, fileToDelete := range filesToDelete { + absCatchpointFileName := filepath.Join(ct.dbDirectory, fileToDelete) + err = os.Remove(absCatchpointFileName) + if err == nil || os.IsNotExist(err) { + // it's ok if the file doesn't exist. just remove it from the database and we'll be good to go. + err = nil + } else { + // we can't delete the file, abort - + return fmt.Errorf("unable to delete old catchpoint file '%s' : %v", absCatchpointFileName, err) + } + err = ct.accountsq.storeCatchpoint(context.Background(), round, "", "", 0) + if err != nil { + return fmt.Errorf("unable to delete old catchpoint entry '%s' : %v", fileToDelete, err) + } + } + return +} + +// GetCatchpointStream returns a ReadCloseSizer to the catchpoint file associated with the provided round +func (ct *catchpointTracker) GetCatchpointStream(round basics.Round) (ReadCloseSizer, error) { + dbFileName := "" + fileSize := int64(0) + start := time.Now() + ledgerGetcatchpointCount.Inc(nil) + err := ct.dbs.Rdb.Atomic(func(ctx context.Context, tx *sql.Tx) (err error) { + dbFileName, _, fileSize, err = getCatchpoint(tx, round) + return + }) + ledgerGetcatchpointMicros.AddMicrosecondsSince(start, nil) + if err != nil && err != sql.ErrNoRows { + // we had some sql error. + return nil, fmt.Errorf("accountUpdates: getCatchpointStream: unable to lookup catchpoint %d: %v", round, err) + } + if dbFileName != "" { + catchpointPath := filepath.Join(ct.dbDirectory, dbFileName) + file, err := os.OpenFile(catchpointPath, os.O_RDONLY, 0666) + if err == nil && file != nil { + return &readCloseSizer{ReadCloser: file, size: fileSize}, nil + } + // else, see if this is a file-not-found error + if os.IsNotExist(err) { + // the database told us that we have this file.. but we couldn't find it. + // delete it from the database. + err := ct.saveCatchpointFile(round, "", 0, "") + if err != nil { + ct.log.Warnf("accountUpdates: getCatchpointStream: unable to delete missing catchpoint entry: %v", err) + return nil, err + } + + return nil, ledgercore.ErrNoEntry{} + } + // it's some other error. + return nil, fmt.Errorf("accountUpdates: getCatchpointStream: unable to open catchpoint file '%s' %v", catchpointPath, err) + } + + // if the database doesn't know about that round, see if we have that file anyway: + fileName := filepath.Join("catchpoints", catchpointRoundToPath(round)) + catchpointPath := filepath.Join(ct.dbDirectory, fileName) + file, err := os.OpenFile(catchpointPath, os.O_RDONLY, 0666) + if err == nil && file != nil { + // great, if found that we should have had this in the database.. add this one now : + fileInfo, err := file.Stat() + if err != nil { + // we couldn't get the stat, so just return with the file. + return &readCloseSizer{ReadCloser: file, size: -1}, nil + } + + err = ct.saveCatchpointFile(round, fileName, fileInfo.Size(), "") + if err != nil { + ct.log.Warnf("accountUpdates: getCatchpointStream: unable to save missing catchpoint entry: %v", err) + } + return &readCloseSizer{ReadCloser: file, size: fileInfo.Size()}, nil + } + return nil, ledgercore.ErrNoEntry{} +} + +// deleteStoredCatchpoints iterates over the storedcatchpoints table and deletes all the files stored on disk. +// once all the files have been deleted, it would go ahead and remove the entries from the table. +func deleteStoredCatchpoints(ctx context.Context, dbQueries *accountsDbQueries, dbDirectory string) (err error) { + catchpointsFilesChunkSize := 50 + for { + fileNames, err := dbQueries.getOldestCatchpointFiles(ctx, catchpointsFilesChunkSize, 0) + if err != nil { + return err + } + if len(fileNames) == 0 { + break + } + + for round, fileName := range fileNames { + absCatchpointFileName := filepath.Join(dbDirectory, fileName) + err = os.Remove(absCatchpointFileName) + if err == nil || os.IsNotExist(err) { + // it's ok if the file doesn't exist. just remove it from the database and we'll be good to go. + } else { + // we can't delete the file, abort - + return fmt.Errorf("unable to delete old catchpoint file '%s' : %v", absCatchpointFileName, err) + } + // clear the entry from the database + err = dbQueries.storeCatchpoint(ctx, round, "", "", 0) + if err != nil { + return err + } + } + } + return nil +} + +// accountHashBuilder calculates the hash key used for the trie by combining the account address and the account data +func accountHashBuilder(addr basics.Address, accountData basics.AccountData, encodedAccountData []byte) []byte { + hash := make([]byte, 4+crypto.DigestSize) + // write out the lowest 32 bits of the reward base. This should improve the caching of the trie by allowing + // recent updated to be in-cache, and "older" nodes will be left alone. + for i, rewards := 3, accountData.RewardsBase; i >= 0; i, rewards = i-1, rewards>>8 { + // the following takes the rewards & 255 -> hash[i] + hash[i] = byte(rewards) + } + entryHash := crypto.Hash(append(addr[:], encodedAccountData[:]...)) + copy(hash[4:], entryHash[:]) + return hash[:] +} + +func (ct *catchpointTracker) catchpointEnabled() bool { + return ct.catchpointInterval != 0 +} + +// accountsInitializeHashes initializes account hashes. +// as part of the initialization, it tests if a hash table matches to account base and updates the former. +func (ct *catchpointTracker) accountsInitializeHashes(ctx context.Context, tx *sql.Tx, rnd basics.Round) error { + hashRound, err := accountsHashRound(tx) + if err != nil { + return err + } + + if hashRound != rnd { + // if the hashed round is different then the base round, something was modified, and the accounts aren't in sync + // with the hashes. + err = resetAccountHashes(tx) + if err != nil { + return err + } + // if catchpoint is disabled on this node, we could complete the initialization right here. + if !ct.catchpointEnabled() { + return nil + } + } + + // create the merkle trie for the balances + committer, err := MakeMerkleCommitter(tx, false) + if err != nil { + return fmt.Errorf("accountsInitialize was unable to makeMerkleCommitter: %v", err) + } + + trie, err := merkletrie.MakeTrie(committer, TrieMemoryConfig) + if err != nil { + return fmt.Errorf("accountsInitialize was unable to MakeTrie: %v", err) + } + + // we might have a database that was previously initialized, and now we're adding the balances trie. In that case, we need to add all the existing balances to this trie. + // we can figure this out by examining the hash of the root: + rootHash, err := trie.RootHash() + if err != nil { + return fmt.Errorf("accountsInitialize was unable to retrieve trie root hash: %v", err) + } + + if rootHash.IsZero() { + ct.log.Infof("accountsInitialize rebuilding merkle trie for round %d", rnd) + accountBuilderIt := makeOrderedAccountsIter(tx, trieRebuildAccountChunkSize) + defer accountBuilderIt.Close(ctx) + startTrieBuildTime := time.Now() + accountsCount := 0 + lastRebuildTime := startTrieBuildTime + pendingAccounts := 0 + totalOrderedAccounts := 0 + for { + accts, processedRows, err := accountBuilderIt.Next(ctx) + if err == sql.ErrNoRows { + // the account builder would return sql.ErrNoRows when no more data is available. + break + } else if err != nil { + return err + } + + if len(accts) > 0 { + accountsCount += len(accts) + pendingAccounts += len(accts) + for _, acct := range accts { + added, err := trie.Add(acct.digest) + if err != nil { + return fmt.Errorf("accountsInitialize was unable to add changes to trie: %v", err) + } + if !added { + ct.log.Warnf("accountsInitialize attempted to add duplicate hash '%s' to merkle trie for account %v", hex.EncodeToString(acct.digest), acct.address) + } + } + + if pendingAccounts >= trieRebuildCommitFrequency { + // this trie Evict will commit using the current transaction. + // if anything goes wrong, it will still get rolled back. + _, err = trie.Evict(true) + if err != nil { + return fmt.Errorf("accountsInitialize was unable to commit changes to trie: %v", err) + } + pendingAccounts = 0 + } + + if time.Since(lastRebuildTime) > 5*time.Second { + // let the user know that the trie is still being rebuilt. + ct.log.Infof("accountsInitialize still building the trie, and processed so far %d accounts", accountsCount) + lastRebuildTime = time.Now() + } + } else if processedRows > 0 { + totalOrderedAccounts += processedRows + // if it's not ordered, we can ignore it for now; we'll just increase the counters and emit logs periodically. + if time.Since(lastRebuildTime) > 5*time.Second { + // let the user know that the trie is still being rebuilt. + ct.log.Infof("accountsInitialize still building the trie, and hashed so far %d accounts", totalOrderedAccounts) + lastRebuildTime = time.Now() + } + } + } + + // this trie Evict will commit using the current transaction. + // if anything goes wrong, it will still get rolled back. + _, err = trie.Evict(true) + if err != nil { + return fmt.Errorf("accountsInitialize was unable to commit changes to trie: %v", err) + } + + // we've just updated the merkle trie, update the hashRound to reflect that. + err = updateAccountsHashRound(tx, rnd) + if err != nil { + return fmt.Errorf("accountsInitialize was unable to update the account hash round to %d: %v", rnd, err) + } + + ct.log.Infof("accountsInitialize rebuilt the merkle trie with %d entries in %v", accountsCount, time.Since(startTrieBuildTime)) + } + ct.balancesTrie = trie + return nil +} diff --git a/ledger/catchpointtracker_test.go b/ledger/catchpointtracker_test.go new file mode 100644 index 0000000000..bad4a101b6 --- /dev/null +++ b/ledger/catchpointtracker_test.go @@ -0,0 +1,375 @@ +// Copyright (C) 2019-2021 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 ( + "context" + "database/sql" + "fmt" + "io/ioutil" + "os" + "path/filepath" + "runtime" + "testing" + "time" + + "github.com/stretchr/testify/require" + + "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/bookkeeping" + "github.com/algorand/go-algorand/ledger/ledgercore" + ledgertesting "github.com/algorand/go-algorand/ledger/testing" + "github.com/algorand/go-algorand/protocol" + "github.com/algorand/go-algorand/test/partitiontest" +) + +func TestIsWritingCatchpointFile(t *testing.T) { + partitiontest.PartitionTest(t) + + ct := &catchpointTracker{} + + ct.catchpointWriting = -1 + ans := ct.IsWritingCatchpointFile() + require.True(t, ans) + + ct.catchpointWriting = 0 + ans = ct.IsWritingCatchpointFile() + require.False(t, ans) +} + +func newCatchpointTracker(tb testing.TB, l *mockLedgerForTracker, conf config.Local, dbPathPrefix string) *catchpointTracker { + au := &accountUpdates{} + ct := &catchpointTracker{} + au.initialize(conf) + ct.initialize(conf, dbPathPrefix) + _, err := trackerDBInitialize(l, ct.catchpointEnabled(), dbPathPrefix) + require.NoError(tb, err) + + err = l.trackers.initialize(l, []ledgerTracker{au, ct}, conf) + require.NoError(tb, err) + err = l.trackers.loadFromDisk(l) + require.NoError(tb, err) + return ct +} + +func TestGetCatchpointStream(t *testing.T) { + partitiontest.PartitionTest(t) + + accts := []map[basics.Address]basics.AccountData{ledgertesting.RandomAccounts(20, true)} + + ml := makeMockLedgerForTracker(t, true, 10, protocol.ConsensusCurrentVersion, accts) + defer ml.Close() + + conf := config.GetDefaultLocal() + conf.CatchpointInterval = 1 + ct := newCatchpointTracker(t, ml, conf, ".") + defer ct.close() + + filesToCreate := 4 + + temporaryDirectroy, err := ioutil.TempDir(os.TempDir(), "catchpoints") + require.NoError(t, err) + defer func() { + os.RemoveAll(temporaryDirectroy) + }() + catchpointsDirectory := filepath.Join(temporaryDirectroy, "catchpoints") + err = os.Mkdir(catchpointsDirectory, 0777) + require.NoError(t, err) + + ct.dbDirectory = temporaryDirectroy + + // Create the catchpoint files with dummy data + for i := 0; i < filesToCreate; i++ { + fileName := filepath.Join("catchpoints", fmt.Sprintf("%d.catchpoint", i)) + data := []byte{byte(i), byte(i + 1), byte(i + 2)} + err = ioutil.WriteFile(filepath.Join(temporaryDirectroy, fileName), data, 0666) + require.NoError(t, err) + + // Store the catchpoint into the database + err := ct.accountsq.storeCatchpoint(context.Background(), basics.Round(i), fileName, "", int64(len(data))) + require.NoError(t, err) + } + + dataRead := make([]byte, 3) + var n int + + // File on disk, and database has the record + reader, err := ct.GetCatchpointStream(basics.Round(1)) + n, err = reader.Read(dataRead) + require.NoError(t, err) + require.Equal(t, 3, n) + outData := []byte{1, 2, 3} + require.Equal(t, outData, dataRead) + len, err := reader.Size() + require.NoError(t, err) + require.Equal(t, int64(3), len) + + // File deleted, but record in the database + err = os.Remove(filepath.Join(temporaryDirectroy, "catchpoints", "2.catchpoint")) + reader, err = ct.GetCatchpointStream(basics.Round(2)) + require.Equal(t, ledgercore.ErrNoEntry{}, err) + require.Nil(t, reader) + + // File on disk, but database lost the record + err = ct.accountsq.storeCatchpoint(context.Background(), basics.Round(3), "", "", 0) + reader, err = ct.GetCatchpointStream(basics.Round(3)) + n, err = reader.Read(dataRead) + require.NoError(t, err) + require.Equal(t, 3, n) + outData = []byte{3, 4, 5} + require.Equal(t, outData, dataRead) + + err = deleteStoredCatchpoints(context.Background(), ct.accountsq, ct.dbDirectory) + require.NoError(t, err) +} + +// TestAcctUpdatesDeleteStoredCatchpoints - The goal of this test is to verify that the deleteStoredCatchpoints function works correctly. +// it doing so by filling up the storedcatchpoints with dummy catchpoint file entries, as well as creating these dummy files on disk. +// ( the term dummy is only because these aren't real catchpoint files, but rather a zero-length file ). Then, the test call the function +// and ensures that it did not errored, the catchpoint files were correctly deleted, and that deleteStoredCatchpoints contains no more +// entries. +func TestAcctUpdatesDeleteStoredCatchpoints(t *testing.T) { + partitiontest.PartitionTest(t) + + accts := []map[basics.Address]basics.AccountData{ledgertesting.RandomAccounts(20, true)} + + ml := makeMockLedgerForTracker(t, true, 10, protocol.ConsensusCurrentVersion, accts) + defer ml.Close() + + conf := config.GetDefaultLocal() + conf.CatchpointInterval = 1 + ct := newCatchpointTracker(t, ml, conf, ".") + defer ct.close() + + dummyCatchpointFilesToCreate := 42 + + for i := 0; i < dummyCatchpointFilesToCreate; i++ { + f, err := os.Create(fmt.Sprintf("./dummy_catchpoint_file-%d", i)) + require.NoError(t, err) + err = f.Close() + require.NoError(t, err) + } + + for i := 0; i < dummyCatchpointFilesToCreate; i++ { + err := ct.accountsq.storeCatchpoint(context.Background(), basics.Round(i), fmt.Sprintf("./dummy_catchpoint_file-%d", i), "", 0) + require.NoError(t, err) + } + err := deleteStoredCatchpoints(context.Background(), ct.accountsq, ct.dbDirectory) + require.NoError(t, err) + + for i := 0; i < dummyCatchpointFilesToCreate; i++ { + // ensure that all the files were deleted. + _, err := os.Open(fmt.Sprintf("./dummy_catchpoint_file-%d", i)) + require.True(t, os.IsNotExist(err)) + } + fileNames, err := ct.accountsq.getOldestCatchpointFiles(context.Background(), dummyCatchpointFilesToCreate, 0) + require.NoError(t, err) + require.Equal(t, 0, len(fileNames)) +} + +func BenchmarkLargeCatchpointWriting(b *testing.B) { + proto := config.Consensus[protocol.ConsensusCurrentVersion] + + accts := []map[basics.Address]basics.AccountData{ledgertesting.RandomAccounts(5, true)} + + pooldata := basics.AccountData{} + pooldata.MicroAlgos.Raw = 1000 * 1000 * 1000 * 1000 + pooldata.Status = basics.NotParticipating + accts[0][testPoolAddr] = pooldata + + sinkdata := basics.AccountData{} + sinkdata.MicroAlgos.Raw = 1000 * 1000 * 1000 * 1000 + sinkdata.Status = basics.NotParticipating + accts[0][testSinkAddr] = sinkdata + + ml := makeMockLedgerForTracker(b, true, 10, protocol.ConsensusCurrentVersion, accts) + defer ml.Close() + + cfg := config.GetDefaultLocal() + cfg.Archival = true + ct := catchpointTracker{} + ct.initialize(cfg, ".") + + temporaryDirectroy, err := ioutil.TempDir(os.TempDir(), "catchpoints") + require.NoError(b, err) + defer func() { + os.RemoveAll(temporaryDirectroy) + }() + catchpointsDirectory := filepath.Join(temporaryDirectroy, "catchpoints") + err = os.Mkdir(catchpointsDirectory, 0777) + require.NoError(b, err) + + ct.dbDirectory = temporaryDirectroy + + err = ct.loadFromDisk(ml, 0) + require.NoError(b, err) + defer ct.close() + + // at this point, the database was created. We want to fill the accounts data + accountsNumber := 6000000 * b.N + err = ml.dbs.Wdb.Atomic(func(ctx context.Context, tx *sql.Tx) (err error) { + for i := 0; i < accountsNumber-5-2; { // subtract the account we've already created above, plus the sink/reward + var updates compactAccountDeltas + for k := 0; i < accountsNumber-5-2 && k < 1024; k++ { + addr := ledgertesting.RandomAddress() + acctData := basics.AccountData{} + acctData.MicroAlgos.Raw = 1 + updates.upsert(addr, accountDelta{new: acctData}) + i++ + } + + _, err = accountsNewRound(tx, updates, nil, proto, basics.Round(1)) + if err != nil { + return + } + } + + return updateAccountsHashRound(tx, 1) + }) + require.NoError(b, err) + + b.ResetTimer() + ct.generateCatchpoint(context.Background(), basics.Round(0), "0#ABCD", crypto.Digest{}, time.Second) + b.StopTimer() + b.ReportMetric(float64(accountsNumber), "accounts") +} + +func TestReproducibleCatchpointLabels(t *testing.T) { + partitiontest.PartitionTest(t) + + if runtime.GOARCH == "arm" || runtime.GOARCH == "arm64" { + t.Skip("This test is too slow on ARM and causes travis builds to time out") + } + // create new protocol version, which has lower lookback + testProtocolVersion := protocol.ConsensusVersion("test-protocol-TestReproducibleCatchpointLabels") + protoParams := config.Consensus[protocol.ConsensusCurrentVersion] + protoParams.MaxBalLookback = 32 + protoParams.SeedLookback = 2 + protoParams.SeedRefreshInterval = 8 + config.Consensus[testProtocolVersion] = protoParams + defer func() { + delete(config.Consensus, testProtocolVersion) + }() + + accts := []map[basics.Address]basics.AccountData{ledgertesting.RandomAccounts(20, true)} + rewardsLevels := []uint64{0} + + pooldata := basics.AccountData{} + pooldata.MicroAlgos.Raw = 100 * 1000 * 1000 * 1000 * 1000 + pooldata.Status = basics.NotParticipating + accts[0][testPoolAddr] = pooldata + + sinkdata := basics.AccountData{} + sinkdata.MicroAlgos.Raw = 1000 * 1000 * 1000 * 1000 + sinkdata.Status = basics.NotParticipating + accts[0][testSinkAddr] = sinkdata + + ml := makeMockLedgerForTracker(t, false, 1, testProtocolVersion, accts) + defer ml.Close() + + cfg := config.GetDefaultLocal() + cfg.CatchpointInterval = 50 + cfg.CatchpointTracking = 1 + ct := newCatchpointTracker(t, ml, cfg, ".") + au := ml.trackers.accts + defer ct.close() + + rewardLevel := uint64(0) + + const testCatchpointLabelsCount = 5 + + // lastCreatableID stores asset or app max used index to get rid of conflicts + lastCreatableID := crypto.RandUint64() % 512 + knownCreatables := make(map[basics.CreatableIndex]bool) + catchpointLabels := make(map[basics.Round]string) + ledgerHistory := make(map[basics.Round]*mockLedgerForTracker) + roundDeltas := make(map[basics.Round]ledgercore.StateDelta) + for i := basics.Round(1); i <= basics.Round(testCatchpointLabelsCount*cfg.CatchpointInterval); i++ { + rewardLevelDelta := crypto.RandUint64() % 5 + rewardLevel += rewardLevelDelta + var updates ledgercore.AccountDeltas + var totals map[basics.Address]basics.AccountData + base := accts[i-1] + updates, totals, lastCreatableID = ledgertesting.RandomDeltasBalancedFull(1, base, rewardLevel, lastCreatableID) + prevTotals, err := au.Totals(basics.Round(i - 1)) + require.NoError(t, err) + + newPool := totals[testPoolAddr] + newPool.MicroAlgos.Raw -= prevTotals.RewardUnits() * rewardLevelDelta + updates.Upsert(testPoolAddr, newPool) + totals[testPoolAddr] = newPool + + newTotals := ledgertesting.CalculateNewRoundAccountTotals(t, updates, rewardLevel, protoParams, base, prevTotals) + + blk := bookkeeping.Block{ + BlockHeader: bookkeeping.BlockHeader{ + Round: basics.Round(i), + }, + } + blk.RewardsLevel = rewardLevel + blk.CurrentProtocol = testProtocolVersion + delta := ledgercore.MakeStateDelta(&blk.BlockHeader, 0, updates.Len(), 0) + delta.Accts.MergeAccounts(updates) + delta.Creatables = creatablesFromUpdates(base, updates, knownCreatables) + delta.Totals = newTotals + + ml.trackers.newBlock(blk, delta) + ml.trackers.committedUpTo(i) + ml.addMockBlock(blockEntry{block: blk}, delta) + accts = append(accts, totals) + rewardsLevels = append(rewardsLevels, rewardLevel) + roundDeltas[i] = delta + + // if this is a catchpoint round, save the label. + if uint64(i)%cfg.CatchpointInterval == 0 { + ml.trackers.waitAccountsWriting() + catchpointLabels[i] = ct.GetLastCatchpointLabel() + ledgerHistory[i] = ml.fork(t) + defer ledgerHistory[i].Close() + } + } + + // test in revese what happens when we try to repeat the exact same blocks. + // start off with the catchpoint before the last one + startingRound := basics.Round((testCatchpointLabelsCount - 1) * cfg.CatchpointInterval) + for ; startingRound > basics.Round(cfg.CatchpointInterval); startingRound -= basics.Round(cfg.CatchpointInterval) { + au.close() + ml2 := ledgerHistory[startingRound] + + ct := newCatchpointTracker(t, ml2, cfg, ".") + for i := startingRound + 1; i <= basics.Round(testCatchpointLabelsCount*cfg.CatchpointInterval); i++ { + blk := bookkeeping.Block{ + BlockHeader: bookkeeping.BlockHeader{ + Round: basics.Round(i), + }, + } + blk.RewardsLevel = rewardsLevels[i] + blk.CurrentProtocol = testProtocolVersion + delta := roundDeltas[i] + ml2.trackers.newBlock(blk, delta) + ml2.trackers.committedUpTo(i) + + // if this is a catchpoint round, check the label. + if uint64(i)%cfg.CatchpointInterval == 0 { + ml2.trackers.waitAccountsWriting() + require.Equal(t, catchpointLabels[i], ct.GetLastCatchpointLabel()) + } + } + } +} diff --git a/ledger/evalbench_test.go b/ledger/evalbench_test.go index 09f047e9e9..52974c7d37 100644 --- a/ledger/evalbench_test.go +++ b/ledger/evalbench_test.go @@ -350,10 +350,6 @@ func benchmarkBlockEvaluator(b *testing.B, inMem bool, withCrypto bool, proto pr // apply initialization transations if any initSignedTxns, maxTxnPerBlock := txnSource.Prepare(b, addrs, keys, newBlock.Round(), genHash) if len(initSignedTxns) > 0 { - // all init transactions need to be written to ledger before reopening and benchmarking - for _, l := range []*Ledger{l, l2} { - stopCommitSyncer(l) - } var numBlocks uint64 = 0 var validatedBlock *ledgercore.ValidatedBlock diff --git a/ledger/ledger.go b/ledger/ledger.go index 2d163de49e..4c0ececaa1 100644 --- a/ledger/ledger.go +++ b/ledger/ledger.go @@ -75,11 +75,12 @@ type Ledger struct { genesisProto config.ConsensusParams // State-machine trackers - accts accountUpdates - txTail txTail - bulletin bulletin - notifier blockNotifier - metrics metricsTracker + accts accountUpdates + catchpoint catchpointTracker + txTail txTail + bulletin bulletin + notifier blockNotifier + metrics metricsTracker trackers trackerRegistry trackerMu deadlock.RWMutex @@ -153,7 +154,8 @@ func OpenLedger( l.genesisAccounts = make(map[basics.Address]basics.AccountData) } - l.accts.initialize(cfg, dbPathPrefix) + l.accts.initialize(cfg) + l.catchpoint.initialize(cfg, dbPathPrefix) err = l.reloadLedger() if err != nil { @@ -188,21 +190,22 @@ func (l *Ledger) reloadLedger() error { } // init tracker db - trackerDBInitParams, err := trackerDBInitialize(l, l.accts.catchpointEnabled(), l.accts.dbDirectory) + trackerDBInitParams, err := trackerDBInitialize(l, l.catchpoint.catchpointEnabled(), l.catchpoint.dbDirectory) if err != nil { return err } // set account updates tracker as a driver to calculate tracker db round and committing offsets trackers := []ledgerTracker{ - &l.accts, // update the balances - &l.txTail, // update the transaction tail, tracking the recent 1000 txn - &l.bulletin, // provide closed channel signaling support for completed rounds - &l.notifier, // send OnNewBlocks to subscribers - &l.metrics, // provides metrics reporting support + &l.accts, // update the balances + &l.catchpoint, // catchpoints tracker : update catchpoint labels, create catchpoint files + &l.txTail, // update the transaction tail, tracking the recent 1000 txn + &l.bulletin, // provide closed channel signaling support for completed rounds + &l.notifier, // send OnNewBlocks to subscribers + &l.metrics, // provides metrics reporting support } - err = l.trackers.initialize(&l.accts, l, trackers) + err = l.trackers.initialize(l, trackers, l.cfg) if err != nil { return err } @@ -399,25 +402,15 @@ func (l *Ledger) notifyCommit(r basics.Round) basics.Round { minToSave = 0 } - l.trackers.scheduleCommit(r) - return minToSave } -func (l *Ledger) waitAccountsWriting() { - l.trackers.waitAccountsWriting() -} - -func (l *Ledger) scheduleCommit(rnd basics.Round) { - l.trackers.scheduleCommit(rnd) -} - // GetLastCatchpointLabel returns the latest catchpoint label that was written to the // database. func (l *Ledger) GetLastCatchpointLabel() string { l.trackerMu.RLock() defer l.trackerMu.RUnlock() - return l.accts.GetLastCatchpointLabel() + return l.catchpoint.GetLastCatchpointLabel() } // GetCreatorForRound takes a CreatableIndex and a CreatableType and tries to @@ -658,7 +651,7 @@ func (l *Ledger) GetCatchpointCatchupState(ctx context.Context) (state Catchpoin func (l *Ledger) GetCatchpointStream(round basics.Round) (ReadCloseSizer, error) { l.trackerMu.RLock() defer l.trackerMu.RUnlock() - return l.accts.GetCatchpointStream(round) + return l.catchpoint.GetCatchpointStream(round) } // ledgerForTracker methods @@ -688,7 +681,7 @@ func (l *Ledger) trackerEvalVerified(blk bookkeeping.Block, accUpdatesLedger int func (l *Ledger) IsWritingCatchpointFile() bool { l.trackerMu.RLock() defer l.trackerMu.RUnlock() - return l.accts.IsWritingCatchpointFile() + return l.catchpoint.IsWritingCatchpointFile() } // VerifiedTransactionCache returns the verify.VerifiedTransactionCache diff --git a/ledger/ledger_test.go b/ledger/ledger_test.go index 623c71c8f2..9fa3c08bd9 100644 --- a/ledger/ledger_test.go +++ b/ledger/ledger_test.go @@ -1426,7 +1426,7 @@ func TestGetLastCatchpointLabel(t *testing.T) { // set some value lastCatchpointLabel := "someCatchpointLabel" - ledger.accts.lastCatchpointLabel = lastCatchpointLabel + ledger.catchpoint.lastCatchpointLabel = lastCatchpointLabel // verify the value is returned require.Equal(t, lastCatchpointLabel, ledger.GetLastCatchpointLabel()) diff --git a/ledger/metrics.go b/ledger/metrics.go index 1f61c55b87..55a84d5632 100644 --- a/ledger/metrics.go +++ b/ledger/metrics.go @@ -50,8 +50,8 @@ func (mt *metricsTracker) newBlock(blk bookkeeping.Block, delta ledgercore.State mt.ledgerRewardClaimsTotal.Add(float64(1), map[string]string{}) } -func (mt *metricsTracker) committedUpTo(committedRnd basics.Round) basics.Round { - return committedRnd +func (mt *metricsTracker) committedUpTo(committedRnd basics.Round) (retRound, lookback basics.Round) { + return committedRnd, basics.Round(0) } func (mt *metricsTracker) prepareCommit(dcc *deferredCommitContext) error { @@ -62,8 +62,11 @@ func (mt *metricsTracker) commitRound(context.Context, *sql.Tx, *deferredCommitC return nil } -func (mt *metricsTracker) postCommit(deferredCommitContext) { +func (mt *metricsTracker) postCommit(ctx context.Context, dcc *deferredCommitContext) { } func (mt *metricsTracker) handleUnorderedCommit(uint64, basics.Round, basics.Round) { } +func (mt *metricsTracker) produceCommittingTask(committedRound basics.Round, dbRound basics.Round, dcr *deferredCommitRange) *deferredCommitRange { + return dcr +} diff --git a/ledger/notifier.go b/ledger/notifier.go index 3f29ae47e5..e922c73e17 100644 --- a/ledger/notifier.go +++ b/ledger/notifier.go @@ -110,8 +110,8 @@ func (bn *blockNotifier) newBlock(blk bookkeeping.Block, delta ledgercore.StateD bn.cond.Broadcast() } -func (bn *blockNotifier) committedUpTo(rnd basics.Round) basics.Round { - return rnd +func (bn *blockNotifier) committedUpTo(rnd basics.Round) (retRound, lookback basics.Round) { + return rnd, basics.Round(0) } func (bn *blockNotifier) prepareCommit(dcc *deferredCommitContext) error { @@ -122,8 +122,12 @@ func (bn *blockNotifier) commitRound(context.Context, *sql.Tx, *deferredCommitCo return nil } -func (bn *blockNotifier) postCommit(deferredCommitContext) { +func (bn *blockNotifier) postCommit(ctx context.Context, dcc *deferredCommitContext) { } func (bn *blockNotifier) handleUnorderedCommit(uint64, basics.Round, basics.Round) { } + +func (bn *blockNotifier) produceCommittingTask(committedRound basics.Round, dbRound basics.Round, dcr *deferredCommitRange) *deferredCommitRange { + return dcr +} diff --git a/ledger/tracker.go b/ledger/tracker.go index 0ff2ba970a..855995665d 100644 --- a/ledger/tracker.go +++ b/ledger/tracker.go @@ -19,6 +19,7 @@ package ledger import ( "context" "database/sql" + "errors" "fmt" "reflect" "sync" @@ -62,32 +63,51 @@ type ledgerTracker interface { // blocks from the database, or access its own state. The // ledgerForTracker interface abstracts away the details of // ledger internals so that individual trackers can be tested - // in isolation. + // in isolation. The provided round number represents the + // current accounts storage round number. loadFromDisk(ledgerForTracker, basics.Round) error - // newBlock informs the tracker of a new block from round - // rnd and a given ledgercore.StateDelta as produced by BlockEvaluator. + // newBlock informs the tracker of a new block along with + // a given ledgercore.StateDelta as produced by BlockEvaluator. newBlock(blk bookkeeping.Block, delta ledgercore.StateDelta) - // committedUpTo informs the tracker that the database has + // committedUpTo informs the tracker that the block database has // committed all blocks up to and including rnd to persistent - // storage (the SQL database). This can allow the tracker + // storage. This can allow the tracker // to garbage-collect state that will not be needed. // // committedUpTo() returns the round number of the earliest - // block that this tracker needs to be stored in the ledger - // for subsequent calls to loadFromDisk(). All blocks with - // round numbers before that may be deleted to save space, - // and the tracker is expected to still function after a - // restart and a call to loadFromDisk(). For example, - // returning 0 means that no blocks can be deleted. - committedUpTo(basics.Round) basics.Round + // block that this tracker needs to be stored in the block + // database for subsequent calls to loadFromDisk(). + // All blocks with round numbers before that may be deleted to + // save space, and the tracker is expected to still function + // after a restart and a call to loadFromDisk(). + // For example, returning 0 means that no blocks can be deleted. + // Separetly, the method returns the lookback that is being + // maintained by the tracker. + committedUpTo(basics.Round) (minRound, lookback basics.Round) + + // produceCommittingTask prepares a deferredCommitRange; Preparing a deferredCommitRange is a joint + // effort, and all the trackers contribute to that effort. All the trackers are being handed a + // pointer to the deferredCommitRange, and have the ability to either modify it, or return a + // nil. If nil is returned, the commit would be skipped. + produceCommittingTask(committedRound basics.Round, dbRound basics.Round, dcr *deferredCommitRange) *deferredCommitRange // prepareCommit, commitRound and postCommit are called when it is time to commit tracker's data. // If an error returned the process is aborted. + + // prepareCommit aligns the data structures stored in the deferredCommitContext with the current + // state of the tracker. It allows the tracker to decide what data is going to be persisted + // on the coming commitRound. prepareCommit(*deferredCommitContext) error + // commitRound is called for each of the trackers after a deferredCommitContext was agreed upon + // by all the prepareCommit calls. The commitRound is being executed within a single transactional + // context, and so, if any of the tracker's commitRound calls fails, the transaction is rolled back. commitRound(context.Context, *sql.Tx, *deferredCommitContext) error - postCommit(deferredCommitContext) + // postCommit is called only on a successful commitRound. In that case, each of the trackers have + // the chance to update it's internal data structures, knowing that the given deferredCommitContext + // has completed. An optional context is provided for long-running operations. + postCommit(context.Context, *deferredCommitContext) // handleUnorderedCommit is a special method for handling deferred commits that are out of order. // Tracker might update own state in this case. For example, account updates tracker cancels @@ -115,22 +135,21 @@ type ledgerForTracker interface { GenesisHash() crypto.Digest GenesisProto() config.ConsensusParams GenesisAccounts() map[basics.Address]basics.AccountData - - // TODO: temporary? - scheduleCommit(basics.Round) - waitAccountsWriting() } type trackerRegistry struct { trackers []ledgerTracker - driver *accountUpdates + // the accts has some exceptional usages in the tracker registry. + accts *accountUpdates // ctx is the context for the committing go-routine. ctx context.Context // ctxCancel is the canceling function for canceling the committing go-routine ( i.e. signaling the committing go-routine that it's time to abort ) ctxCancel context.CancelFunc - // committedOffset is the offset at which we'd like to persist all the previous account information to disk. - committedOffset chan deferredCommit + + // deferredCommits is the channel of pending deferred commits + deferredCommits chan *deferredCommitContext + // commitSyncerClosed is the blocking channel for synchronizing closing the commitSyncer goroutine. Once it's closed, the // commitSyncer can be assumed to have aborted. commitSyncerClosed chan struct{} @@ -145,14 +164,45 @@ type trackerRegistry struct { dbs db.Pair log logging.Logger + // the synchronous mode that would be used for the account database. + synchronousMode db.SynchronousMode + + // the synchronous mode that would be used while the accounts database is being rebuilt. + accountsRebuildSynchronousMode db.SynchronousMode + mu deadlock.RWMutex + + // lastFlushTime is the time we last flushed updates to + // the accounts DB (bumping dbRound). + lastFlushTime time.Time +} + +// deferredCommitRange is used during the calls to produceCommittingTask, and used as a data structure +// to syncronize the various trackers and create a uniformity around which rounds need to be persisted +// next. +type deferredCommitRange struct { + offset uint64 + oldBase basics.Round + lookback basics.Round + + // pendingDeltas is the number of accounts that were modified within this commit context. + // note that in this number we might have the same account being modified several times. + pendingDeltas int + + isCatchpointRound bool + + // catchpointWriting is a pointer to a varible with the same name in the catchpointTracker. + // it's used in order to reset the catchpointWriting flag from the acctupdates's + // prepareCommit/commitRound ( which is called before the corresponding catchpoint tracker method ) + catchpointWriting *int32 } +// deferredCommitContext is used in order to syncornize the persistence of a given deferredCommitRange. +// prepareCommit, commitRound and postCommit are all using it to exchange data. type deferredCommitContext struct { - offset uint64 - oldBase basics.Round + deferredCommitRange + newBase basics.Round - lookback basics.Round flushTime time.Time genesisProto config.ConsensusParams @@ -164,17 +214,18 @@ type deferredCommitContext struct { updatedPersistedAccounts []persistedAccountData - isCatchpointRound bool - committedRoundDigest crypto.Digest - trieBalancesHash crypto.Digest + committedRoundDigest crypto.Digest + trieBalancesHash crypto.Digest + updatingBalancesDuration time.Duration + catchpointLabel string stats telemetryspec.AccountsUpdateMetrics updateStats bool } -func (tr *trackerRegistry) initialize(au *accountUpdates, l ledgerForTracker, trackers []ledgerTracker) (err error) { - tr.driver = au +var errMissingAccountUpdateTracker = errors.New("initializeTrackerCaches : called without a valid accounts update tracker") +func (tr *trackerRegistry) initialize(l ledgerForTracker, trackers []ledgerTracker, cfg config.Local) (err error) { tr.dbs = l.trackerDB() tr.log = l.trackerLog() @@ -188,11 +239,20 @@ func (tr *trackerRegistry) initialize(au *accountUpdates, l ledgerForTracker, tr } tr.ctx, tr.ctxCancel = context.WithCancel(context.Background()) - tr.committedOffset = make(chan deferredCommit, 1) + tr.deferredCommits = make(chan *deferredCommitContext, 1) tr.commitSyncerClosed = make(chan struct{}) - go tr.commitSyncer(tr.committedOffset) + tr.synchronousMode = db.SynchronousMode(cfg.LedgerSynchronousMode) + tr.accountsRebuildSynchronousMode = db.SynchronousMode(cfg.AccountsRebuildSynchronousMode) + go tr.commitSyncer(tr.deferredCommits) + + tr.trackers = append([]ledgerTracker{}, trackers...) - tr.trackers = append(tr.trackers, trackers...) + for _, tracker := range tr.trackers { + if accts, ok := tracker.(*accountUpdates); ok { + tr.accts = accts + break + } + } return } @@ -210,7 +270,14 @@ func (tr *trackerRegistry) loadFromDisk(l ledgerForTracker) error { } } - return nil + err := tr.initializeTrackerCaches(l) + if err != nil { + return err + } + // the votes have a special dependency on the account updates, so we need to initialize these separetly. + tr.accts.voters = &votersTracker{} + err = tr.accts.voters.loadFromDisk(l, tr.accts) + return err } func (tr *trackerRegistry) newBlock(blk bookkeeping.Block, delta ledgercore.StateDelta) { @@ -221,26 +288,56 @@ func (tr *trackerRegistry) newBlock(blk bookkeeping.Block, delta ledgercore.Stat func (tr *trackerRegistry) committedUpTo(rnd basics.Round) basics.Round { minBlock := rnd - + maxLookback := basics.Round(0) for _, lt := range tr.trackers { - retain := lt.committedUpTo(rnd) - if retain < minBlock { - minBlock = retain + retainRound, lookback := lt.committedUpTo(rnd) + if retainRound < minBlock { + minBlock = retainRound + } + if lookback > maxLookback { + maxLookback = lookback } } + tr.scheduleCommit(rnd, maxLookback) + return minBlock } -func (tr *trackerRegistry) scheduleCommit(blockqRound basics.Round) { +func (tr *trackerRegistry) scheduleCommit(blockqRound, maxLookback basics.Round) { tr.mu.RLock() dbRound := tr.dbRound tr.mu.RUnlock() - dc := tr.driver.produceCommittingTask(blockqRound, dbRound) - if dc.offset != 0 { + dcc := &deferredCommitContext{ + deferredCommitRange: deferredCommitRange{ + lookback: maxLookback, + }, + } + cdr := &dcc.deferredCommitRange + for _, lt := range tr.trackers { + cdr = lt.produceCommittingTask(blockqRound, dbRound, cdr) + if cdr == nil { + break + } + } + if cdr != nil { + dcc.deferredCommitRange = *cdr + } + + tr.mu.RLock() + // If we recently flushed, wait to aggregate some more blocks. + // ( unless we're creating a catchpoint, in which case we want to flush it right away + // so that all the instances of the catchpoint would contain exactly the same data ) + flushTime := time.Now() + if dcc != nil && !flushTime.After(tr.lastFlushTime.Add(balancesFlushInterval)) && !dcc.isCatchpointRound && dcc.pendingDeltas < pendingDeltasFlushThreshold { + dcc = nil + } + tr.mu.RUnlock() + + if dcc != nil { tr.accountsWriting.Add(1) - tr.committedOffset <- dc + tr.deferredCommits <- dcc } } @@ -265,20 +362,20 @@ func (tr *trackerRegistry) close() { lt.close() } tr.trackers = nil - tr.driver = nil + tr.accts = nil } // commitSyncer is the syncer go-routine function which perform the database updates. Internally, it dequeues deferredCommits and // send the tasks to commitRound for completing the operation. -func (tr *trackerRegistry) commitSyncer(deferredCommits chan deferredCommit) { +func (tr *trackerRegistry) commitSyncer(deferredCommits chan *deferredCommitContext) { defer close(tr.commitSyncerClosed) for { select { - case committedOffset, ok := <-deferredCommits: + case commit, ok := <-deferredCommits: if !ok { return } - tr.commitRound(committedOffset) + tr.commitRound(commit) case <-tr.ctx.Done(): // drain the pending commits queue: drained := false @@ -295,14 +392,14 @@ func (tr *trackerRegistry) commitSyncer(deferredCommits chan deferredCommit) { } } -func (tr *trackerRegistry) commitRound(dc deferredCommit) { +// commitRound commits the given deferredCommitContext via the trackers. +func (tr *trackerRegistry) commitRound(dcc *deferredCommitContext) { defer tr.accountsWriting.Done() - tr.mu.RLock() - offset := dc.offset - dbRound := dc.dbRound - lookback := dc.lookback + offset := dcc.offset + dbRound := dcc.oldBase + lookback := dcc.lookback // we can exit right away, as this is the result of mis-ordered call to committedUpTo. if tr.dbRound < dbRound || offset < uint64(tr.dbRound-dbRound) { @@ -328,16 +425,13 @@ func (tr *trackerRegistry) commitRound(dc deferredCommit) { dbRound = tr.dbRound newBase := basics.Round(offset) + dbRound - dcc := deferredCommitContext{ - offset: offset, - oldBase: dbRound, - newBase: newBase, - lookback: lookback, - flushTime: time.Now(), - } + dcc.offset = offset + dcc.oldBase = dbRound + dcc.newBase = newBase + dcc.flushTime = time.Now() for _, lt := range tr.trackers { - err := lt.prepareCommit(&dcc) + err := lt.prepareCommit(dcc) if err != nil { tr.log.Errorf(err.Error()) tr.mu.RUnlock() @@ -350,7 +444,7 @@ func (tr *trackerRegistry) commitRound(dc deferredCommit) { ledgerCommitroundCount.Inc(nil) err := tr.dbs.Wdb.Atomic(func(ctx context.Context, tx *sql.Tx) (err error) { for _, lt := range tr.trackers { - err0 := lt.commitRound(ctx, tx, &dcc) + err0 := lt.commitRound(ctx, tx, dcc) if err0 != nil { return err0 } @@ -373,7 +467,185 @@ func (tr *trackerRegistry) commitRound(dc deferredCommit) { tr.mu.Lock() tr.dbRound = newBase for _, lt := range tr.trackers { - lt.postCommit(dcc) + lt.postCommit(tr.ctx, dcc) } + tr.lastFlushTime = dcc.flushTime tr.mu.Unlock() + +} + +// initializeTrackerCaches fills up the accountUpdates cache with the most recent ~320 blocks ( on normal execution ). +// the method also support balances recovery in cases where the difference between the lastBalancesRound and the lastestBlockRound +// is far greater than 320; in these cases, it would flush to disk periodically in order to avoid high memory consumption. +func (tr *trackerRegistry) initializeTrackerCaches(l ledgerForTracker) (err error) { + lastestBlockRound := l.Latest() + lastBalancesRound := tr.dbRound + + var blk bookkeeping.Block + var delta ledgercore.StateDelta + + if tr.accts == nil { + return errMissingAccountUpdateTracker + } + + accLedgerEval := accountUpdatesLedgerEvaluator{ + au: tr.accts, + } + + if lastBalancesRound < lastestBlockRound { + accLedgerEval.prevHeader, err = l.BlockHdr(lastBalancesRound) + if err != nil { + return err + } + } + + skipAccountCacheMessage := make(chan struct{}) + writeAccountCacheMessageCompleted := make(chan struct{}) + defer func() { + close(skipAccountCacheMessage) + select { + case <-writeAccountCacheMessageCompleted: + if err == nil { + tr.log.Infof("initializeTrackerCaches completed initializing account data caches") + } + default: + } + }() + + catchpointInterval := uint64(0) + for _, tracker := range tr.trackers { + if catchpointTracker, ok := tracker.(*catchpointTracker); ok { + catchpointInterval = catchpointTracker.catchpointInterval + break + } + } + + // this goroutine logs a message once if the parent function have not completed in initializingAccountCachesMessageTimeout seconds. + // the message is important, since we're blocking on the ledger block database here, and we want to make sure that we log a message + // within the above timeout. + go func() { + select { + case <-time.After(initializingAccountCachesMessageTimeout): + tr.log.Infof("initializeTrackerCaches is initializing account data caches") + close(writeAccountCacheMessageCompleted) + case <-skipAccountCacheMessage: + } + }() + + blocksStream := make(chan bookkeeping.Block, initializeCachesReadaheadBlocksStream) + blockEvalFailed := make(chan struct{}, 1) + var blockRetrievalError error + go func() { + defer close(blocksStream) + for roundNumber := lastBalancesRound + 1; roundNumber <= lastestBlockRound; roundNumber++ { + blk, blockRetrievalError = l.Block(roundNumber) + if blockRetrievalError != nil { + return + } + select { + case blocksStream <- blk: + case <-blockEvalFailed: + return + } + } + }() + + lastFlushedRound := lastBalancesRound + const accountsCacheLoadingMessageInterval = 5 * time.Second + lastProgressMessage := time.Now().Add(-accountsCacheLoadingMessageInterval / 2) + + // rollbackSynchronousMode ensures that we switch to "fast writing mode" when we start flushing out rounds to disk, and that + // we exit this mode when we're done. + rollbackSynchronousMode := false + defer func() { + if rollbackSynchronousMode { + // restore default synchronous mode + err0 := tr.dbs.Wdb.SetSynchronousMode(context.Background(), tr.synchronousMode, tr.synchronousMode >= db.SynchronousModeFull) + // override the returned error only in case there is no error - since this + // operation has a lower criticality. + if err == nil { + err = err0 + } + } + }() + + for blk := range blocksStream { + delta, err = l.trackerEvalVerified(blk, &accLedgerEval) + if err != nil { + close(blockEvalFailed) + return + } + tr.newBlock(blk, delta) + + // flush to disk if any of the following applies: + // 1. if we have loaded up more than initializeCachesRoundFlushInterval rounds since the last time we flushed the data to disk + // 2. if we completed the loading and we loaded up more than 320 rounds. + flushIntervalExceed := blk.Round()-lastFlushedRound > initializeCachesRoundFlushInterval + loadCompleted := (lastestBlockRound == blk.Round() && lastBalancesRound+basics.Round(blk.ConsensusProtocol().MaxBalLookback) < lastestBlockRound) + if flushIntervalExceed || loadCompleted { + // adjust the last flush time, so that we would not hold off the flushing due to "working too fast" + tr.lastFlushTime = time.Now().Add(-balancesFlushInterval) + + if !rollbackSynchronousMode { + // switch to rebuild synchronous mode to improve performance + err0 := tr.dbs.Wdb.SetSynchronousMode(context.Background(), tr.accountsRebuildSynchronousMode, tr.accountsRebuildSynchronousMode >= db.SynchronousModeFull) + if err0 != nil { + tr.log.Warnf("initializeTrackerCaches was unable to switch to rbuild synchronous mode : %v", err0) + } else { + // flip the switch to rollback the synchronous mode once we're done. + rollbackSynchronousMode = true + } + } + + var roundsBehind basics.Round + + // flush the account data + tr.scheduleCommit(blk.Round(), basics.Round(config.Consensus[blk.BlockHeader.CurrentProtocol].MaxBalLookback)) + // wait for the writing to complete. + tr.waitAccountsWriting() + + func() { + tr.mu.RLock() + defer tr.mu.RUnlock() + + // The au.dbRound after writing should be ~320 behind the block round. + roundsBehind = blk.Round() - tr.dbRound + }() + + // are we too far behind ? ( taking into consideration the catchpoint writing, which can stall the writing for quite a bit ) + if roundsBehind > initializeCachesRoundFlushInterval+basics.Round(catchpointInterval) { + // we're unable to persist changes. This is unexpected, but there is no point in keep trying batching additional changes since any further changes + // would just accumulate in memory. + close(blockEvalFailed) + tr.log.Errorf("initializeTrackerCaches was unable to fill up the account caches accounts round = %d, block round = %d. See above error for more details.", blk.Round()-roundsBehind, blk.Round()) + err = fmt.Errorf("initializeTrackerCaches failed to initialize the account data caches") + return + } + + // and once we flushed it to disk, update the lastFlushedRound + lastFlushedRound = blk.Round() + } + + // if enough time have passed since the last time we wrote a message to the log file then give the user an update about the progess. + if time.Since(lastProgressMessage) > accountsCacheLoadingMessageInterval { + // drop the initial message if we're got to this point since a message saying "still initializing" that comes after "is initializing" doesn't seems to be right. + select { + case skipAccountCacheMessage <- struct{}{}: + // if we got to this point, we should be able to close the writeAccountCacheMessageCompleted channel to have the "completed initializing" message written. + close(writeAccountCacheMessageCompleted) + default: + } + tr.log.Infof("initializeTrackerCaches is still initializing account data caches, %d rounds loaded out of %d rounds", blk.Round()-lastBalancesRound, lastestBlockRound-lastBalancesRound) + lastProgressMessage = time.Now() + } + + // prepare for the next iteration. + accLedgerEval.prevHeader = *delta.Hdr + } + + if blockRetrievalError != nil { + err = blockRetrievalError + } + return + } diff --git a/ledger/txtail.go b/ledger/txtail.go index 49d6c89b66..a5d77e49cd 100644 --- a/ledger/txtail.go +++ b/ledger/txtail.go @@ -143,7 +143,7 @@ func (t *txTail) newBlock(blk bookkeeping.Block, delta ledgercore.StateDelta) { } } -func (t *txTail) committedUpTo(rnd basics.Round) basics.Round { +func (t *txTail) committedUpTo(rnd basics.Round) (retRound, lookback basics.Round) { maxlife := basics.Round(t.recent[rnd].proto.MaxTxnLife) for r := range t.recent { if r+maxlife < rnd { @@ -154,7 +154,7 @@ func (t *txTail) committedUpTo(rnd basics.Round) basics.Round { delete(t.lastValid, t.lowWaterMark) } - return (rnd + 1).SubSaturate(maxlife) + return (rnd + 1).SubSaturate(maxlife), basics.Round(0) } func (t *txTail) prepareCommit(*deferredCommitContext) error { @@ -165,12 +165,16 @@ func (t *txTail) commitRound(context.Context, *sql.Tx, *deferredCommitContext) e return nil } -func (t *txTail) postCommit(deferredCommitContext) { +func (t *txTail) postCommit(ctx context.Context, dcc *deferredCommitContext) { } func (t *txTail) handleUnorderedCommit(uint64, basics.Round, basics.Round) { } +func (t *txTail) produceCommittingTask(committedRound basics.Round, dbRound basics.Round, dcr *deferredCommitRange) *deferredCommitRange { + return dcr +} + // txtailMissingRound is returned by checkDup when requested for a round number below the low watermark type txtailMissingRound struct { round basics.Round diff --git a/ledger/voters.go b/ledger/voters.go index 3d887d1446..8986040725 100644 --- a/ledger/voters.go +++ b/ledger/voters.go @@ -142,7 +142,7 @@ func (vt *votersTracker) loadTree(hdr bookkeeping.BlockHeader) { onlineAccounts := ledgercore.TopOnlineAccounts(vt.au.onlineTop) err := tr.LoadTree(onlineAccounts, hdr) if err != nil { - vt.au.log.Warnf("votersTracker.loadTree(%d): %v", hdr.Round, err) + vt.l.trackerLog().Warnf("votersTracker.loadTree(%d): %v", hdr.Round, err) tr.BroadcastError(err) } @@ -179,7 +179,7 @@ func (vt *votersTracker) newBlock(hdr bookkeeping.BlockHeader) { if (r+proto.CompactCertVotersLookback)%proto.CompactCertRounds == 0 { _, ok := vt.round[basics.Round(r)] if ok { - vt.au.log.Errorf("votersTracker.newBlock: round %d already present", r) + vt.l.trackerLog().Errorf("votersTracker.newBlock: round %d already present", r) } else { vt.loadTree(hdr) } diff --git a/test/e2e-go/cli/goal/expect/catchpointCatchupTest.exp b/test/e2e-go/cli/goal/expect/catchpointCatchupTest.exp index 39f47e12c9..f52e57a214 100644 --- a/test/e2e-go/cli/goal/expect/catchpointCatchupTest.exp +++ b/test/e2e-go/cli/goal/expect/catchpointCatchupTest.exp @@ -108,15 +108,25 @@ if { [catch { ::AlgorandGoal::StartNode $TEST_ROOT_DIR/Node False $WEBPROXY_LISTEN_ADDRESS + # once the node is started we can clear the ::GLOBAL_TEST_ALGO_DIR, so that shutdown would be done as a network. + unset ::GLOBAL_TEST_ALGO_DIR + ::AlgorandGoal::WaitForRound 1 $TEST_ROOT_DIR/Node set CATCHPOINT [::AlgorandGoal::GetNodeLastCatchpoint $TEST_ROOT_DIR/Primary] puts "Catchpoint is $CATCHPOINT" + regexp -nocase {([0-9]*)#[A-Z2-7]*} $CATCHPOINT CATCHPOINT_ROUND CATCHPOINT_ROUND + + puts "Catchpoint round is $CATCHPOINT_ROUND" + + # wait for the primary to reach $CATCHPOINT_ROUND + 5, so that the catchpoint file would be saved + ::AlgorandGoal::WaitForRound [expr {int($CATCHPOINT_ROUND + 5)}] $TEST_ROOT_DIR/Primary + ::AlgorandGoal::StartCatchup $TEST_ROOT_DIR/Node $CATCHPOINT - ::AlgorandGoal::WaitForRound 37 $TEST_ROOT_DIR/Node + ::AlgorandGoal::WaitForRound $CATCHPOINT_ROUND $TEST_ROOT_DIR/Node ::AlgorandGoal::StopNode $TEST_ROOT_DIR/Node @@ -164,6 +174,9 @@ if { [catch { ::AlgorandGoal::StartNode $TEST_ROOT_DIR/Node False $WEBPROXY_LISTEN_ADDRESS + # once the node is started we can clear the ::GLOBAL_TEST_ALGO_DIR, so that shutdown would be done as a network. + set ::GLOBAL_TEST_ALGO_DIR "" + ::AlgorandGoal::WaitForRound 38 $TEST_ROOT_DIR/Node ::AlgorandGoal::StopNode $TEST_ROOT_DIR/Node diff --git a/test/e2e-go/cli/goal/expect/goalExpectCommon.exp b/test/e2e-go/cli/goal/expect/goalExpectCommon.exp index 273d02db91..345c1be6f2 100644 --- a/test/e2e-go/cli/goal/expect/goalExpectCommon.exp +++ b/test/e2e-go/cli/goal/expect/goalExpectCommon.exp @@ -934,7 +934,17 @@ proc ::AlgorandGoal::WaitForRound { WAIT_FOR_ROUND_NUMBER NODE_DATA_DIR } { -re {Genesis ID: (\w+)} {set GENESIS_ID $expect_out(1,string); exp_continue } -re {Genesis hash: ([A-Za-z0-9+/]+={0,2})} {set GENESIS_HASH $expect_out(1,string); exp_continue } -re {Catchpoint: ([0-9]*#[A-Z2-7]*)} { set CATCHPOINT $expect_out(1,string); exp_continue } - eof { catch wait result; if { [lindex $result 3] != 0 } { ::AlgorandGoal::Abort "failed to wait for round : error code [lindex $result 3]"} } + eof { + catch wait result; + if { [lindex $result 3] != 0 } { + log_user 1 + set errLog [exec cat $NODE_DATA_DIR/algod-err.log] + puts "$NODE_DATA_DIR/algod-err.log :\r\n$errLog" + set nodeLog [exec -- tail -n 30 $NODE_DATA_DIR/node.log] + puts "$NODE_DATA_DIR/node.log :\r\n$nodeLog" + ::AlgorandGoal::Abort "failed to wait for round : error code [lindex $result 3]" + } + } } log_user 1 if { $BLOCK > -1 } {