Merge pull request #571 from wpaulino/birthday-blockstamp

wallet+waddrmgr: add migration & sanity check to populate birthday block
This commit is contained in:
Olaoluwa Osuntokun 2018-11-14 18:05:49 -08:00 committed by GitHub
commit ee93fa9871
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
9 changed files with 703 additions and 37 deletions

View file

@ -8,6 +8,7 @@ package waddrmgr
import (
"crypto/sha256"
"encoding/binary"
"errors"
"fmt"
"time"
@ -252,9 +253,10 @@ var (
watchingOnlyName = []byte("watchonly")
// Sync related key names (sync bucket).
syncedToName = []byte("syncedto")
startBlockName = []byte("startblock")
birthdayName = []byte("birthday")
syncedToName = []byte("syncedto")
startBlockName = []byte("startblock")
birthdayName = []byte("birthday")
birthdayBlockName = []byte("birthdayblock")
)
// uint32ToBytes converts a 32 bit unsigned integer into a 4-byte slice in
@ -1875,6 +1877,10 @@ func fetchBlockHash(ns walletdb.ReadBucket, height int32) (*chainhash.Hash, erro
heightBytes := make([]byte, 4)
binary.BigEndian.PutUint32(heightBytes, uint32(height))
hashBytes := bucket.Get(heightBytes)
if hashBytes == nil {
err := errors.New("block not found")
return nil, managerError(ErrBlockNotFound, errStr, err)
}
if len(hashBytes) != 32 {
err := fmt.Errorf("couldn't get hash from database")
return nil, managerError(ErrDatabase, errStr, err)
@ -1929,32 +1935,80 @@ func putStartBlock(ns walletdb.ReadWriteBucket, bs *BlockStamp) error {
// fetchBirthday loads the manager's bithday timestamp from the database.
func fetchBirthday(ns walletdb.ReadBucket) (time.Time, error) {
bucket := ns.NestedReadBucket(syncBucketName)
var t time.Time
buf := bucket.Get(birthdayName)
if len(buf) != 8 {
bucket := ns.NestedReadBucket(syncBucketName)
birthdayTimestamp := bucket.Get(birthdayName)
if len(birthdayTimestamp) != 8 {
str := "malformed birthday stored in database"
return t, managerError(ErrDatabase, str, nil)
}
t = time.Unix(int64(binary.BigEndian.Uint64(buf)), 0)
t = time.Unix(int64(binary.BigEndian.Uint64(birthdayTimestamp)), 0)
return t, nil
}
// putBirthday stores the provided birthday timestamp to the database.
func putBirthday(ns walletdb.ReadWriteBucket, t time.Time) error {
var birthdayTimestamp [8]byte
binary.BigEndian.PutUint64(birthdayTimestamp[:], uint64(t.Unix()))
bucket := ns.NestedReadWriteBucket(syncBucketName)
buf := make([]byte, 8)
binary.BigEndian.PutUint64(buf, uint64(t.Unix()))
err := bucket.Put(birthdayName, buf)
if err != nil {
if err := bucket.Put(birthdayName, birthdayTimestamp[:]); err != nil {
str := "failed to store birthday"
return managerError(ErrDatabase, str, err)
}
return nil
}
// fetchBirthdayBlock retrieves the birthday block from the database.
//
// The block is serialized as follows:
// [0:4] block height
// [4:36] block hash
// [36:44] block timestamp
func fetchBirthdayBlock(ns walletdb.ReadBucket) (BlockStamp, error) {
var block BlockStamp
bucket := ns.NestedReadBucket(syncBucketName)
birthdayBlock := bucket.Get(birthdayBlockName)
if birthdayBlock == nil {
str := "birthday block not set"
return block, managerError(ErrBirthdayBlockNotSet, str, nil)
}
if len(birthdayBlock) != 44 {
str := "malformed birthday block stored in database"
return block, managerError(ErrDatabase, str, nil)
}
block.Height = int32(binary.BigEndian.Uint32(birthdayBlock[:4]))
copy(block.Hash[:], birthdayBlock[4:36])
t := int64(binary.BigEndian.Uint64(birthdayBlock[36:]))
block.Timestamp = time.Unix(t, 0)
return block, nil
}
// putBirthdayBlock stores the provided birthday block to the database.
//
// The block is serialized as follows:
// [0:4] block height
// [4:36] block hash
// [36:44] block timestamp
func putBirthdayBlock(ns walletdb.ReadWriteBucket, block BlockStamp) error {
var birthdayBlock [44]byte
binary.BigEndian.PutUint32(birthdayBlock[:4], uint32(block.Height))
copy(birthdayBlock[4:36], block.Hash[:])
binary.BigEndian.PutUint64(birthdayBlock[36:], uint64(block.Timestamp.Unix()))
bucket := ns.NestedReadWriteBucket(syncBucketName)
if err := bucket.Put(birthdayBlockName, birthdayBlock[:]); err != nil {
str := "failed to store birthday block"
return managerError(ErrDatabase, str, err)
}
return nil
}

View file

@ -131,6 +131,14 @@ const (
// ErrScopeNotFound is returned when a target scope cannot be found
// within the database.
ErrScopeNotFound
// ErrBirthdayBlockNotSet is returned when we attempt to retrieve the
// wallet's birthday but it has not been set yet.
ErrBirthdayBlockNotSet
// ErrBlockNotFound is returned when we attempt to retrieve the hash for
// a block that we do not know of.
ErrBlockNotFound
)
// Map of ErrorCode values back to their constant names for pretty printing.

43
waddrmgr/log.go Normal file
View file

@ -0,0 +1,43 @@
package waddrmgr
import "github.com/btcsuite/btclog"
// log is a logger that is initialized with no output filters. This
// means the package will not perform any logging by default until the caller
// requests it.
var log btclog.Logger
// The default amount of logging is none.
func init() {
DisableLog()
}
// DisableLog disables all library log output. Logging output is disabled
// by default until either UseLogger or SetLogWriter are called.
func DisableLog() {
UseLogger(btclog.Disabled)
}
// UseLogger uses a specified Logger to output package logging info.
// This should be used in preference to SetLogWriter if the caller is also
// using btclog.
func UseLogger(logger btclog.Logger) {
log = logger
}
// LogClosure is a closure that can be printed with %v to be used to
// generate expensive-to-create data for a detailed log level and avoid doing
// the work if the data isn't printed.
type logClosure func() string
// String invokes the log closure and returns the results string.
func (c logClosure) String() string {
return c()
}
// newLogClosure returns a new closure over the passed function which allows
// it to be used as a parameter in a logging function that is only invoked when
// the logging level is such that the message will actually be logged.
func newLogClosure(c func() string) logClosure {
return logClosure(c)
}

View file

@ -2,7 +2,9 @@ package waddrmgr
import (
"fmt"
"time"
"github.com/btcsuite/btcd/chaincfg"
"github.com/btcsuite/btcwallet/walletdb"
"github.com/btcsuite/btcwallet/walletdb/migration"
)
@ -20,6 +22,10 @@ var versions = []migration.Version{
Number: 5,
Migration: upgradeToVersion5,
},
{
Number: 6,
Migration: populateBirthdayBlock,
},
}
// getLatestVersion returns the version number of the latest database version.
@ -254,3 +260,93 @@ func migrateRecursively(src, dst walletdb.ReadWriteBucket,
}
return nil
}
// populateBirthdayBlock is a migration that attempts to populate the birthday
// block of the wallet. This is needed so that in the event that we need to
// perform a rescan of the wallet, we can do so starting from this block, rather
// than from the genesis block.
//
// NOTE: This migration cannot guarantee the correctness of the birthday block
// being set as we do not store block timestamps, so a sanity check must be done
// upon starting the wallet to ensure we do not potentially miss any relevant
// events when rescanning.
func populateBirthdayBlock(ns walletdb.ReadWriteBucket) error {
// We'll need to jump through some hoops in order to determine the
// corresponding block height for our birthday timestamp. Since we do
// not store block timestamps, we'll need to estimate our height by
// looking at the genesis timestamp and assuming a block occurs every 10
// minutes. This can be unsafe, and cause us to actually miss on-chain
// events, so a sanity check is done before the wallet attempts to sync
// itself.
//
// We'll start by fetching our birthday timestamp.
birthdayTimestamp, err := fetchBirthday(ns)
if err != nil {
return fmt.Errorf("unable to fetch birthday timestamp: %v", err)
}
log.Infof("Setting the wallet's birthday block from timestamp=%v",
birthdayTimestamp)
// Now, we'll need to determine the timestamp of the genesis block for
// the corresponding chain.
genesisHash, err := fetchBlockHash(ns, 0)
if err != nil {
return fmt.Errorf("unable to fetch genesis block hash: %v", err)
}
var genesisTimestamp time.Time
switch *genesisHash {
case *chaincfg.MainNetParams.GenesisHash:
genesisTimestamp =
chaincfg.MainNetParams.GenesisBlock.Header.Timestamp
case *chaincfg.TestNet3Params.GenesisHash:
genesisTimestamp =
chaincfg.TestNet3Params.GenesisBlock.Header.Timestamp
case *chaincfg.RegressionNetParams.GenesisHash:
genesisTimestamp =
chaincfg.RegressionNetParams.GenesisBlock.Header.Timestamp
case *chaincfg.SimNetParams.GenesisHash:
genesisTimestamp =
chaincfg.SimNetParams.GenesisBlock.Header.Timestamp
default:
return fmt.Errorf("unknown genesis hash %v", genesisHash)
}
// With the timestamps retrieved, we can estimate a block height by
// taking the difference between them and dividing by the average block
// time (10 minutes).
birthdayHeight := int32((birthdayTimestamp.Sub(genesisTimestamp).Seconds() / 600))
// Now that we have the height estimate, we can fetch the corresponding
// block and set it as our birthday block.
birthdayHash, err := fetchBlockHash(ns, birthdayHeight)
// To ensure we record a height that is known to us from the chain,
// we'll make sure this height estimate can be found. Otherwise, we'll
// continue subtracting a day worth of blocks until we can find one.
for IsError(err, ErrBlockNotFound) {
birthdayHeight -= 144
if birthdayHeight < 0 {
birthdayHeight = 0
}
birthdayHash, err = fetchBlockHash(ns, birthdayHeight)
}
if err != nil {
return err
}
log.Infof("Estimated birthday block from timestamp=%v: height=%d, "+
"hash=%v", birthdayTimestamp, birthdayHeight, birthdayHash)
// NOTE: The timestamp of the birthday block isn't set since we do not
// store each block's timestamp.
return putBirthdayBlock(ns, BlockStamp{
Height: birthdayHeight,
Hash: *birthdayHash,
})
}

217
waddrmgr/migrations_test.go Normal file
View file

@ -0,0 +1,217 @@
package waddrmgr
import (
"bytes"
"errors"
"fmt"
"testing"
"time"
"github.com/btcsuite/btcd/chaincfg"
"github.com/btcsuite/btcwallet/walletdb"
)
// applyMigration is a helper function that allows us to assert the state of the
// top-level bucket before and after a migration. This can be used to ensure
// the correctness of migrations.
func applyMigration(t *testing.T, beforeMigration, afterMigration,
migration func(walletdb.ReadWriteBucket) error, shouldFail bool) {
t.Helper()
// We'll start by setting up our address manager backed by a database.
teardown, db, _ := setupManager(t)
defer teardown()
// First, we'll run the beforeMigration closure, which contains the
// database modifications/assertions needed before proceeding with the
// migration.
err := walletdb.Update(db, func(tx walletdb.ReadWriteTx) error {
ns := tx.ReadWriteBucket(waddrmgrNamespaceKey)
if ns == nil {
return errors.New("top-level namespace does not exist")
}
return beforeMigration(ns)
})
if err != nil {
t.Fatalf("unable to run beforeMigration func: %v", err)
}
// Then, we'll run the migration itself and fail if it does not match
// its expected result.
err = walletdb.Update(db, func(tx walletdb.ReadWriteTx) error {
ns := tx.ReadWriteBucket(waddrmgrNamespaceKey)
if ns == nil {
return errors.New("top-level namespace does not exist")
}
return migration(ns)
})
if err != nil && !shouldFail {
t.Fatalf("unable to perform migration: %v", err)
} else if err == nil && shouldFail {
t.Fatal("expected migration to fail, but did not")
}
// Finally, we'll run the afterMigration closure, which contains the
// assertions needed in order to guarantee than the migration was
// successful.
err = walletdb.Update(db, func(tx walletdb.ReadWriteTx) error {
ns := tx.ReadWriteBucket(waddrmgrNamespaceKey)
if ns == nil {
return errors.New("top-level namespace does not exist")
}
return afterMigration(ns)
})
if err != nil {
t.Fatalf("unable to run afterMigration func: %v", err)
}
}
// TestMigrationPupulateBirthdayBlock ensures that the migration to populate the
// wallet's birthday block works as intended.
func TestMigrationPopulateBirthdayBlock(t *testing.T) {
t.Parallel()
var expectedHeight int32
beforeMigration := func(ns walletdb.ReadWriteBucket) error {
// To test this migration, we'll start by writing to disk 10
// random blocks.
block := &BlockStamp{}
for i := int32(1); i <= 10; i++ {
block.Height = i
blockHash := bytes.Repeat([]byte(string(i)), 32)
copy(block.Hash[:], blockHash)
if err := putSyncedTo(ns, block); err != nil {
return err
}
}
// With the blocks inserted, we'll assume that the birthday
// block corresponds to the 7th block (out of 11) in the chain.
// To do this, we'll need to set our birthday timestamp to the
// estimated timestamp of a block that's 6 blocks after genesis.
genesisTimestamp := chaincfg.MainNetParams.GenesisBlock.Header.Timestamp
delta := time.Hour
expectedHeight = int32(delta.Seconds() / 600)
birthday := genesisTimestamp.Add(delta)
if err := putBirthday(ns, birthday); err != nil {
return err
}
// Finally, since the migration has not yet started, we should
// not be able to find the birthday block within the database.
_, err := fetchBirthdayBlock(ns)
if !IsError(err, ErrBirthdayBlockNotSet) {
return fmt.Errorf("expected ErrBirthdayBlockNotSet, "+
"got %v", err)
}
return nil
}
// After the migration has completed, we should see that the birthday
// block now exists and is set to the correct expected height.
afterMigration := func(ns walletdb.ReadWriteBucket) error {
birthdayBlock, err := fetchBirthdayBlock(ns)
if err != nil {
return err
}
if birthdayBlock.Height != expectedHeight {
return fmt.Errorf("expected birthday block with "+
"height %d, got %d", expectedHeight,
birthdayBlock.Height)
}
return nil
}
// We can now apply the migration and expect it not to fail.
applyMigration(
t, beforeMigration, afterMigration, populateBirthdayBlock,
false,
)
}
// TestMigrationPopulateBirthdayBlockEstimateTooFar ensures that the migration
// can properly detect a height estimate which the chain from our point of view
// has not yet reached.
func TestMigrationPopulateBirthdayBlockEstimateTooFar(t *testing.T) {
t.Parallel()
const numBlocks = 1000
chainParams := chaincfg.MainNetParams
var expectedHeight int32
beforeMigration := func(ns walletdb.ReadWriteBucket) error {
// To test this migration, we'll start by writing to disk 999
// random blocks to simulate a synced chain with height 1000.
block := &BlockStamp{}
for i := int32(1); i < numBlocks; i++ {
block.Height = i
blockHash := bytes.Repeat([]byte(string(i)), 32)
copy(block.Hash[:], blockHash)
if err := putSyncedTo(ns, block); err != nil {
return err
}
}
// With the blocks inserted, we'll assume that the birthday
// block corresponds to the 900th block in the chain. To do
// this, we'd need to set our birthday timestamp to the
// estimated timestamp of a block that's 899 blocks after
// genesis. However, this will not work if the average block
// time is not 10 mins, which can throw off the height estimate
// with a height longer than the chain in the event of test
// networks (testnet, regtest, etc. and not fully synced
// wallets). Instead the migration should be able to handle this
// by subtracting a days worth of blocks until finding a block
// that it is aware of.
//
// We'll have the migration assume that our birthday is at block
// 1001 in the chain. Since this block doesn't exist from the
// database's point of view, a days worth of blocks will be
// subtracted from the estimate, which should give us a valid
// block height.
genesisTimestamp := chainParams.GenesisBlock.Header.Timestamp
delta := numBlocks * 10 * time.Minute
expectedHeight = numBlocks - 144
birthday := genesisTimestamp.Add(delta)
if err := putBirthday(ns, birthday); err != nil {
return err
}
// Finally, since the migration has not yet started, we should
// not be able to find the birthday block within the database.
_, err := fetchBirthdayBlock(ns)
if !IsError(err, ErrBirthdayBlockNotSet) {
return fmt.Errorf("expected ErrBirthdayBlockNotSet, "+
"got %v", err)
}
return nil
}
// After the migration has completed, we should see that the birthday
// block now exists and is set to the correct expected height.
afterMigration := func(ns walletdb.ReadWriteBucket) error {
birthdayBlock, err := fetchBirthdayBlock(ns)
if err != nil {
return err
}
if birthdayBlock.Height != expectedHeight {
return fmt.Errorf("expected birthday block height %d, "+
"got %d", expectedHeight, birthdayBlock.Height)
}
return nil
}
// We can now apply the migration and expect it not to fail.
applyMigration(
t, beforeMigration, afterMigration, populateBirthdayBlock,
false,
)
}

View file

@ -110,3 +110,17 @@ func (m *Manager) SetBirthday(ns walletdb.ReadWriteBucket,
m.birthday = birthday
return putBirthday(ns, birthday)
}
// BirthdayBlock returns the birthday block, or earliest block a key could have
// been used, for the manager.
func (m *Manager) BirthdayBlock(ns walletdb.ReadBucket) (BlockStamp, error) {
return fetchBirthdayBlock(ns)
}
// SetBirthdayBlock sets the birthday block, or earliest time a key could have
// been used, for the manager.
func (m *Manager) SetBirthdayBlock(ns walletdb.ReadWriteBucket,
block BlockStamp) error {
return putBirthdayBlock(ns, block)
}

View file

@ -6,7 +6,9 @@ package wallet
import (
"bytes"
"fmt"
"strings"
"time"
"github.com/btcsuite/btcd/txscript"
"github.com/btcsuite/btcwallet/chain"
@ -24,12 +26,12 @@ func (w *Wallet) handleChainNotifications() {
return
}
sync := func(w *Wallet) {
sync := func(w *Wallet, birthdayStamp *waddrmgr.BlockStamp) {
// At the moment there is no recourse if the rescan fails for
// some reason, however, the wallet will not be marked synced
// and many methods will error early since the wallet is known
// to be out of date.
err := w.syncWithChain()
err := w.syncWithChain(birthdayStamp)
if err != nil && !w.ShuttingDown() {
log.Warnf("Unable to synchronize wallet to chain: %v", err)
}
@ -96,7 +98,20 @@ func (w *Wallet) handleChainNotifications() {
var err error
switch n := n.(type) {
case chain.ClientConnected:
go sync(w)
// Before attempting to sync with our backend,
// we'll make sure that our birthday block has
// been set correctly to potentially prevent
// missing relevant events.
birthdayBlock, err := w.birthdaySanityCheck()
if err != nil {
err := fmt.Errorf("unable to sanity "+
"check wallet birthday block: %v",
err)
log.Error(err)
panic(err)
}
go sync(w, birthdayBlock)
case chain.BlockConnected:
err = walletdb.Update(w.db, func(tx walletdb.ReadWriteTx) error {
return w.connectBlock(tx, wtxmgr.BlockMeta(n))
@ -329,3 +344,189 @@ func (w *Wallet) addRelevantTx(dbtx walletdb.ReadWriteTx, rec *wtxmgr.TxRecord,
return nil
}
// birthdaySanityCheck is a helper function that ensures our birthday block
// correctly reflects the birthday timestamp within a reasonable timestamp
// delta. It will be run after the wallet establishes its connection with the
// backend, but before it begins syncing. This is done as the second part to
// the wallet's address manager migration where we populate the birthday block
// to ensure we do not miss any relevant events throughout rescans.
func (w *Wallet) birthdaySanityCheck() (*waddrmgr.BlockStamp, error) {
// We'll start by acquiring our chain backend client as we'll be
// querying it for blocks.
chainClient, err := w.requireChainClient()
if err != nil {
return nil, err
}
// We'll then fetch our wallet's birthday timestamp and block.
birthdayTimestamp := w.Manager.Birthday()
var birthdayBlock waddrmgr.BlockStamp
err = walletdb.View(w.db, func(tx walletdb.ReadTx) error {
var err error
ns := tx.ReadBucket(waddrmgrNamespaceKey)
birthdayBlock, err = w.Manager.BirthdayBlock(ns)
return err
})
switch {
// If our wallet's birthday block has not been set yet, then this is our
// initial sync, so we'll defer setting it until then.
case waddrmgr.IsError(err, waddrmgr.ErrBirthdayBlockNotSet):
return nil, nil
// Otherwise, we'll return the error if there was one.
case err != nil:
return nil, err
}
log.Debugf("Starting sanity check for the wallet's birthday block "+
"from: height=%d, hash=%v", birthdayBlock.Height,
birthdayBlock.Hash)
// Now, we'll need to determine if our block correctly reflects our
// timestamp. To do so, we'll fetch the block header and check its
// timestamp in the event that the birthday block's timestamp was not
// set (this is possible if it was set through the migration, since we
// do not store block timestamps).
candidate := birthdayBlock
header, err := chainClient.GetBlockHeader(&candidate.Hash)
if err != nil {
return nil, fmt.Errorf("unable to get header for block hash "+
"%v: %v", candidate.Hash, err)
}
candidate.Timestamp = header.Timestamp
// We'll go back a day worth of blocks in the chain until we find a
// block whose timestamp is below our birthday timestamp.
heightDelta := int32(144)
for birthdayTimestamp.Before(candidate.Timestamp) {
// If the birthday block has reached genesis, then we can exit
// our search as there exists no data before this point.
if candidate.Height == 0 {
break
}
// To prevent requesting blocks out of range, we'll use a lower
// bound of the first block in the chain.
newCandidateHeight := int64(candidate.Height - heightDelta)
if newCandidateHeight < 0 {
newCandidateHeight = 0
}
// Then, we'll fetch the current candidate's hash and header to
// determine if it is valid.
hash, err := chainClient.GetBlockHash(newCandidateHeight)
if err != nil {
return nil, fmt.Errorf("unable to get block hash for "+
"height %d: %v", candidate.Height, err)
}
header, err := chainClient.GetBlockHeader(hash)
if err != nil {
return nil, fmt.Errorf("unable to get header for "+
"block hash %v: %v", candidate.Hash, err)
}
candidate.Hash = *hash
candidate.Timestamp = header.Timestamp
log.Debugf("Checking next birthday block candidate: "+
"height=%d, hash=%v, timestamp=%v",
candidate.Height, candidate.Hash,
candidate.Timestamp)
}
// To ensure we have a reasonable birthday block, we'll make sure it
// respects our birthday timestamp and it is within a reasonable delta.
// The birthday has already been adjusted to two days in the past of the
// actual birthday, so we'll make our expected delta to be within two
// hours of it to account for the network-adjusted time and prevent
// fetching more unnecessary blocks.
_, bestHeight, err := chainClient.GetBestBlock()
if err != nil {
return nil, err
}
timestampDelta := birthdayTimestamp.Sub(candidate.Timestamp)
for timestampDelta > 2*time.Hour {
// We'll determine the height for our next candidate and make
// sure it is not out of range. If it is, we'll lower our height
// delta until finding a height within range.
newHeight := candidate.Height + heightDelta
if newHeight > bestHeight {
heightDelta /= 2
// If we've exhausted all of our possible options at a
// later height, then we can assume the current birthday
// block is our best estimate.
if heightDelta == 0 {
break
}
continue
}
// We'll fetch the header for the next candidate and compare its
// timestamp.
hash, err := chainClient.GetBlockHash(int64(newHeight))
if err != nil {
return nil, fmt.Errorf("unable to get block hash for "+
"height %d: %v", candidate.Height, err)
}
header, err := chainClient.GetBlockHeader(hash)
if err != nil {
return nil, fmt.Errorf("unable to get header for "+
"block hash %v: %v", hash, err)
}
log.Debugf("Checking next birthday block candidate: "+
"height=%d, hash=%v, timestamp=%v", newHeight, hash,
header.Timestamp)
// If this block has exceeded our birthday timestamp, we'll look
// for the next candidate with a lower height delta.
if birthdayTimestamp.Before(header.Timestamp) {
heightDelta /= 2
// If we've exhausted all of our possible options at a
// later height, then we can assume the current birthday
// block is our best estimate.
if heightDelta == 0 {
break
}
continue
}
// Otherwise, this is a valid candidate, so we'll check to see
// if it meets our expected timestamp delta.
candidate.Hash = *hash
candidate.Height = newHeight
candidate.Timestamp = header.Timestamp
timestampDelta = birthdayTimestamp.Sub(header.Timestamp)
}
// At this point, we've found a valid candidate that satisfies our
// conditions above. If this is our current birthday block, then we can
// exit to avoid the additional database transaction.
if candidate.Hash.IsEqual(&birthdayBlock.Hash) {
return &candidate, nil
}
// Otherwise, we have a new, better candidate, so we'll write it to
// disk.
log.Debugf("Found a new valid wallet birthday block: height=%d, hash=%v",
candidate.Height, candidate.Hash)
err = walletdb.Update(w.db, func(tx walletdb.ReadWriteTx) error {
ns := tx.ReadWriteBucket(waddrmgrNamespaceKey)
if err := w.Manager.SetBirthdayBlock(ns, candidate); err != nil {
return err
}
return w.Manager.SetSyncedTo(ns, &candidate)
})
if err != nil {
return nil, err
}
return &candidate, nil
}

View file

@ -6,6 +6,7 @@ package wallet
import (
"github.com/btcsuite/btclog"
"github.com/btcsuite/btcwallet/waddrmgr"
"github.com/btcsuite/btcwallet/walletdb/migration"
)
@ -32,6 +33,7 @@ func UseLogger(logger btclog.Logger) {
log = logger
migration.UseLogger(logger)
waddrmgr.UseLogger(logger)
}
// LogClosure is a closure that can be printed with %v to be used to

View file

@ -318,9 +318,10 @@ func (w *Wallet) activeData(dbtx walletdb.ReadTx) ([]btcutil.Address, []wtxmgr.C
}
// syncWithChain brings the wallet up to date with the current chain server
// connection. It creates a rescan request and blocks until the rescan has
// finished.
func (w *Wallet) syncWithChain() error {
// connection. It creates a rescan request and blocks until the rescan has
// finished. The birthday block can be passed in, if set, to ensure we can
// properly detect if it gets rolled back.
func (w *Wallet) syncWithChain(birthdayStamp *waddrmgr.BlockStamp) error {
chainClient, err := w.requireChainClient()
if err != nil {
return err
@ -351,12 +352,6 @@ func (w *Wallet) syncWithChain() error {
isRecovery := w.recoveryWindow > 0
birthday := w.Manager.Birthday()
// If an initial sync is attempted, we will try and find the block stamp
// of the first block past our birthday. This will be fed into the
// rescan to ensure we catch transactions that are sent while performing
// the initial sync.
var birthdayStamp *waddrmgr.BlockStamp
// TODO(jrick): How should this handle a synced height earlier than
// the chain server best block?
@ -496,6 +491,19 @@ func (w *Wallet) syncWithChain() error {
Hash: *hash,
Timestamp: timestamp,
}
log.Debugf("Found birthday block: "+
"height=%d, hash=%v",
birthdayStamp.Height,
birthdayStamp.Hash)
err := w.Manager.SetBirthdayBlock(
ns, *birthdayStamp,
)
if err != nil {
tx.Rollback()
return err
}
}
// If we are in recovery mode and the check
@ -647,6 +655,18 @@ func (w *Wallet) syncWithChain() error {
// points at the new tip.
if birthdayStamp != nil && rollbackStamp.Height <= birthdayStamp.Height {
birthdayStamp = &rollbackStamp
log.Debugf("Found new birthday block after rollback: "+
"height=%d, hash=%v", birthdayStamp.Height,
birthdayStamp.Hash)
err := walletdb.Update(w.db, func(tx walletdb.ReadWriteTx) error {
ns := tx.ReadWriteBucket(waddrmgrNamespaceKey)
return w.Manager.SetBirthdayBlock(ns, *birthdayStamp)
})
if err != nil {
return nil
}
}
// Request notifications for connected and disconnected blocks.
@ -657,8 +677,7 @@ func (w *Wallet) syncWithChain() error {
// as well. I am leaning towards allowing off all rpcclient
// notification re-registrations, in which case the code here should be
// left as is.
err = chainClient.NotifyBlocks()
if err != nil {
if err := chainClient.NotifyBlocks(); err != nil {
return err
}
@ -2603,6 +2622,9 @@ func (w *Wallet) DumpWIFPrivateKey(addr btcutil.Address) (string, error) {
// ImportPrivateKey imports a private key to the wallet and writes the new
// wallet to disk.
//
// NOTE: If a block stamp is not provided, then the wallet's birthday will be
// set to the genesis block of the corresponding chain.
func (w *Wallet) ImportPrivateKey(scope waddrmgr.KeyScope, wif *btcutil.WIF,
bs *waddrmgr.BlockStamp, rescan bool) (string, error) {
@ -2613,18 +2635,18 @@ func (w *Wallet) ImportPrivateKey(scope waddrmgr.KeyScope, wif *btcutil.WIF,
// The starting block for the key is the genesis block unless otherwise
// specified.
var newBirthday time.Time
if bs == nil {
bs = &waddrmgr.BlockStamp{
Hash: *w.chainParams.GenesisHash,
Height: 0,
Hash: *w.chainParams.GenesisHash,
Height: 0,
Timestamp: w.chainParams.GenesisBlock.Header.Timestamp,
}
} else {
} else if bs.Timestamp.IsZero() {
// Only update the new birthday time from default value if we
// actually have timestamp info in the header.
header, err := w.chainClient.GetBlockHeader(&bs.Hash)
if err == nil {
newBirthday = header.Timestamp
bs.Timestamp = header.Timestamp
}
}
@ -2646,13 +2668,22 @@ func (w *Wallet) ImportPrivateKey(scope waddrmgr.KeyScope, wif *btcutil.WIF,
}
// We'll only update our birthday with the new one if it is
// before our current one. Otherwise, we won't rescan for
// potentially relevant chain events that occurred between them.
if newBirthday.After(w.Manager.Birthday()) {
// before our current one. Otherwise, if we do, we can
// potentially miss detecting relevant chain events that
// occurred between them while rescanning.
birthdayBlock, err := w.Manager.BirthdayBlock(addrmgrNs)
if err != nil {
return err
}
if bs.Height >= birthdayBlock.Height {
return nil
}
return w.Manager.SetBirthday(addrmgrNs, newBirthday)
err = w.Manager.SetBirthday(addrmgrNs, bs.Timestamp)
if err != nil {
return err
}
return w.Manager.SetBirthdayBlock(addrmgrNs, *bs)
})
if err != nil {
return "", err