Merge pull request #2314 from wpaulino/chainnotifier-subserver

chainntnfs+lnrpc/chainrpc: add ChainNotifier RPC subserver
This commit is contained in:
Olaoluwa Osuntokun 2019-01-21 15:19:12 -08:00 committed by GitHub
commit 375be936ce
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
29 changed files with 4587 additions and 1420 deletions

@ -1,6 +1,8 @@
package bitcoindnotify
import (
"bytes"
"encoding/hex"
"errors"
"fmt"
"strings"
@ -8,6 +10,7 @@ import (
"sync/atomic"
"github.com/btcsuite/btcd/btcjson"
"github.com/btcsuite/btcd/chaincfg"
"github.com/btcsuite/btcd/chaincfg/chainhash"
"github.com/btcsuite/btcd/wire"
"github.com/btcsuite/btcutil"
@ -23,11 +26,6 @@ const (
)
var (
// ErrChainNotifierShuttingDown is used when we are trying to
// measure a spend notification when notifier is already stopped.
ErrChainNotifierShuttingDown = errors.New("chainntnfs: system interrupt " +
"while attempting to register for spend notification.")
// ErrTransactionNotFound is an error returned when we attempt to find a
// transaction by manually scanning the chain within a specific range
// but it is not found.
@ -58,7 +56,8 @@ type BitcoindNotifier struct {
started int32 // To be used atomically.
stopped int32 // To be used atomically.
chainConn *chain.BitcoindClient
chainConn *chain.BitcoindClient
chainParams *chaincfg.Params
notificationCancels chan interface{}
notificationRegistry chan interface{}
@ -88,12 +87,15 @@ type BitcoindNotifier struct {
var _ chainntnfs.ChainNotifier = (*BitcoindNotifier)(nil)
// New returns a new BitcoindNotifier instance. This function assumes the
// bitcoind node detailed in the passed configuration is already running, and
// bitcoind node detailed in the passed configuration is already running, and
// willing to accept RPC requests and new zmq clients.
func New(chainConn *chain.BitcoindConn, spendHintCache chainntnfs.SpendHintCache,
func New(chainConn *chain.BitcoindConn, chainParams *chaincfg.Params,
spendHintCache chainntnfs.SpendHintCache,
confirmHintCache chainntnfs.ConfirmHintCache) *BitcoindNotifier {
notifier := &BitcoindNotifier{
chainParams: chainParams,
notificationCancels: make(chan interface{}),
notificationRegistry: make(chan interface{}),
@ -229,7 +231,8 @@ out:
defer b.wg.Done()
confDetails, _, err := b.historicalConfDetails(
msg.TxID, msg.StartHeight, msg.EndHeight,
msg.ConfRequest,
msg.StartHeight, msg.EndHeight,
)
if err != nil {
chainntnfs.Log.Error(err)
@ -244,7 +247,7 @@ out:
// cache at tip, since any pending
// rescans have now completed.
err = b.txNotifier.UpdateConfDetails(
*msg.TxID, confDetails,
msg.ConfRequest, confDetails,
)
if err != nil {
chainntnfs.Log.Error(err)
@ -265,29 +268,50 @@ out:
if err != nil {
chainntnfs.Log.Errorf("Rescan to "+
"determine the spend "+
"details of %v failed: %v",
msg.OutPoint, err)
"details of %v within "+
"range %d-%d failed: %v",
msg.SpendRequest,
msg.StartHeight,
msg.EndHeight, err)
}
}()
case *blockEpochRegistration:
chainntnfs.Log.Infof("New block epoch subscription")
b.blockEpochClients[msg.epochID] = msg
if msg.bestBlock != nil {
missedBlocks, err :=
chainntnfs.GetClientMissedBlocks(
b.chainConn, msg.bestBlock,
b.bestBlock.Height, true,
)
if err != nil {
msg.errorChan <- err
continue
}
for _, block := range missedBlocks {
b.notifyBlockEpochClient(msg,
block.Height, block.Hash)
}
// If the client did not provide their best
// known block, then we'll immediately dispatch
// a notification for the current tip.
if msg.bestBlock == nil {
b.notifyBlockEpochClient(
msg, b.bestBlock.Height,
b.bestBlock.Hash,
)
msg.errorChan <- nil
continue
}
// Otherwise, we'll attempt to deliver the
// backlog of notifications from their best
// known block.
missedBlocks, err := chainntnfs.GetClientMissedBlocks(
b.chainConn, msg.bestBlock,
b.bestBlock.Height, true,
)
if err != nil {
msg.errorChan <- err
continue
}
for _, block := range missedBlocks {
b.notifyBlockEpochClient(
msg, block.Height, block.Hash,
)
}
msg.errorChan <- nil
}
@ -372,14 +396,14 @@ out:
continue
}
tx := &item.TxRecord.MsgTx
tx := btcutil.NewTx(&item.TxRecord.MsgTx)
err := b.txNotifier.ProcessRelevantSpendTx(
tx, item.Block.Height,
tx, uint32(item.Block.Height),
)
if err != nil {
chainntnfs.Log.Errorf("Unable to "+
"process transaction %v: %v",
tx.TxHash(), err)
tx.Hash(), err)
}
}
@ -390,15 +414,28 @@ out:
b.wg.Done()
}
// historicalConfDetails looks up whether a transaction is already included in a
// block in the active chain and, if so, returns details about the confirmation.
func (b *BitcoindNotifier) historicalConfDetails(txid *chainhash.Hash,
// historicalConfDetails looks up whether a confirmation request (txid/output
// script) has already been included in a block in the active chain and, if so,
// returns details about said block.
func (b *BitcoindNotifier) historicalConfDetails(confRequest chainntnfs.ConfRequest,
startHeight, endHeight uint32) (*chainntnfs.TxConfirmation,
chainntnfs.TxConfStatus, error) {
// If a txid was not provided, then we should dispatch upon seeing the
// script on-chain, so we'll short-circuit straight to scanning manually
// as there doesn't exist a script index to query.
if confRequest.TxID == chainntnfs.ZeroHash {
return b.confDetailsManually(
confRequest, startHeight, endHeight,
)
}
// Otherwise, we'll dispatch upon seeing a transaction on-chain with the
// given hash.
//
// We'll first attempt to retrieve the transaction using the node's
// txindex.
txConf, txStatus, err := b.confDetailsFromTxIndex(txid)
txConf, txStatus, err := b.confDetailsFromTxIndex(&confRequest.TxID)
// We'll then check the status of the transaction lookup returned to
// determine whether we should proceed with any fallback methods.
@ -409,7 +446,7 @@ func (b *BitcoindNotifier) historicalConfDetails(txid *chainhash.Hash,
case err != nil:
chainntnfs.Log.Debugf("Failed getting conf details from "+
"index (%v), scanning manually", err)
return b.confDetailsManually(txid, startHeight, endHeight)
return b.confDetailsManually(confRequest, startHeight, endHeight)
// The transaction was found within the node's mempool.
case txStatus == chainntnfs.TxFoundMempool:
@ -440,7 +477,7 @@ func (b *BitcoindNotifier) confDetailsFromTxIndex(txid *chainhash.Hash,
// If the transaction has some or all of its confirmations required,
// then we may be able to dispatch it immediately.
tx, err := b.chainConn.GetRawTransactionVerbose(txid)
rawTxRes, err := b.chainConn.GetRawTransactionVerbose(txid)
if err != nil {
// If the transaction lookup was successful, but it wasn't found
// within the index itself, then we can exit early. We'll also
@ -461,20 +498,19 @@ func (b *BitcoindNotifier) confDetailsFromTxIndex(txid *chainhash.Hash,
// Make sure we actually retrieved a transaction that is included in a
// block. If not, the transaction must be unconfirmed (in the mempool),
// and we'll return TxFoundMempool together with a nil TxConfirmation.
if tx.BlockHash == "" {
if rawTxRes.BlockHash == "" {
return nil, chainntnfs.TxFoundMempool, nil
}
// As we need to fully populate the returned TxConfirmation struct,
// grab the block in which the transaction was confirmed so we can
// locate its exact index within the block.
blockHash, err := chainhash.NewHashFromStr(tx.BlockHash)
blockHash, err := chainhash.NewHashFromStr(rawTxRes.BlockHash)
if err != nil {
return nil, chainntnfs.TxNotFoundIndex,
fmt.Errorf("unable to get block hash %v for "+
"historical dispatch: %v", tx.BlockHash, err)
"historical dispatch: %v", rawTxRes.BlockHash, err)
}
block, err := b.chainConn.GetBlockVerbose(blockHash)
if err != nil {
return nil, chainntnfs.TxNotFoundIndex,
@ -484,36 +520,49 @@ func (b *BitcoindNotifier) confDetailsFromTxIndex(txid *chainhash.Hash,
// If the block was obtained, locate the transaction's index within the
// block so we can give the subscriber full confirmation details.
targetTxidStr := txid.String()
txidStr := txid.String()
for txIndex, txHash := range block.Tx {
if txHash == targetTxidStr {
details := &chainntnfs.TxConfirmation{
BlockHash: blockHash,
BlockHeight: uint32(block.Height),
TxIndex: uint32(txIndex),
}
return details, chainntnfs.TxFoundIndex, nil
if txHash != txidStr {
continue
}
// Deserialize the hex-encoded transaction to include it in the
// confirmation details.
rawTx, err := hex.DecodeString(rawTxRes.Hex)
if err != nil {
return nil, chainntnfs.TxFoundIndex,
fmt.Errorf("unable to deserialize tx %v: %v",
txHash, err)
}
var tx wire.MsgTx
if err := tx.Deserialize(bytes.NewReader(rawTx)); err != nil {
return nil, chainntnfs.TxFoundIndex,
fmt.Errorf("unable to deserialize tx %v: %v",
txHash, err)
}
return &chainntnfs.TxConfirmation{
Tx: &tx,
BlockHash: blockHash,
BlockHeight: uint32(block.Height),
TxIndex: uint32(txIndex),
}, chainntnfs.TxFoundIndex, nil
}
// We return an error because we should have found the transaction
// within the block, but didn't.
return nil, chainntnfs.TxNotFoundIndex,
fmt.Errorf("unable to locate tx %v in block %v", txid,
blockHash)
return nil, chainntnfs.TxNotFoundIndex, fmt.Errorf("unable to locate "+
"tx %v in block %v", txid, blockHash)
}
// confDetailsManually looks up whether a transaction is already included in a
// block in the active chain by scanning the chain's blocks, starting from the
// earliest height the transaction could have been included in, to the current
// height in the chain. If the transaction is found, its confirmation details
// are returned. Otherwise, nil is returned.
func (b *BitcoindNotifier) confDetailsManually(txid *chainhash.Hash,
// confDetailsManually looks up whether a transaction/output script has already
// been included in a block in the active chain by scanning the chain's blocks
// within the given range. If the transaction/output script is found, its
// confirmation details are returned. Otherwise, nil is returned.
func (b *BitcoindNotifier) confDetailsManually(confRequest chainntnfs.ConfRequest,
heightHint, currentHeight uint32) (*chainntnfs.TxConfirmation,
chainntnfs.TxConfStatus, error) {
targetTxidStr := txid.String()
// Begin scanning blocks at every height to determine where the
// transaction was included in.
for height := currentHeight; height >= heightHint && height > 0; height-- {
@ -522,7 +571,7 @@ func (b *BitcoindNotifier) confDetailsManually(txid *chainhash.Hash,
select {
case <-b.quit:
return nil, chainntnfs.TxNotFoundManually,
ErrChainNotifierShuttingDown
chainntnfs.ErrChainNotifierShuttingDown
default:
}
@ -533,24 +582,27 @@ func (b *BitcoindNotifier) confDetailsManually(txid *chainhash.Hash,
"with height %d", height)
}
block, err := b.chainConn.GetBlockVerbose(blockHash)
block, err := b.chainConn.GetBlock(blockHash)
if err != nil {
return nil, chainntnfs.TxNotFoundManually,
fmt.Errorf("unable to get block with hash "+
"%v: %v", blockHash, err)
}
for txIndex, txHash := range block.Tx {
// If we're able to find the transaction in this block,
// return its confirmation details.
if txHash == targetTxidStr {
details := &chainntnfs.TxConfirmation{
BlockHash: blockHash,
BlockHeight: height,
TxIndex: uint32(txIndex),
}
return details, chainntnfs.TxFoundManually, nil
// For every transaction in the block, check which one matches
// our request. If we find one that does, we can dispatch its
// confirmation details.
for txIndex, tx := range block.Transactions {
if !confRequest.MatchesTx(tx) {
continue
}
return &chainntnfs.TxConfirmation{
Tx: tx,
BlockHash: blockHash,
BlockHeight: height,
TxIndex: uint32(txIndex),
}, chainntnfs.TxFoundManually, nil
}
}
@ -620,33 +672,57 @@ func (b *BitcoindNotifier) notifyBlockEpochClient(epochClient *blockEpochRegistr
}
// RegisterSpendNtfn registers an intent to be notified once the target
// outpoint has been spent by a transaction on-chain. Once a spend of the target
// outpoint has been detected, the details of the spending event will be sent
// across the 'Spend' channel. The heightHint should represent the earliest
// height in the chain where the transaction could have been spent in.
// outpoint/output script has been spent by a transaction on-chain. When
// intending to be notified of the spend of an output script, a nil outpoint
// must be used. The heightHint should represent the earliest height in the
// chain of the transaction that spent the outpoint/output script.
//
// Once a spend of has been detected, the details of the spending event will be
// sent across the 'Spend' channel.
func (b *BitcoindNotifier) RegisterSpendNtfn(outpoint *wire.OutPoint,
pkScript []byte, heightHint uint32) (*chainntnfs.SpendEvent, error) {
// First, we'll construct a spend notification request and hand it off
// to the txNotifier.
spendID := atomic.AddUint64(&b.spendClientCounter, 1)
cancel := func() {
b.txNotifier.CancelSpend(*outpoint, spendID)
spendRequest, err := chainntnfs.NewSpendRequest(outpoint, pkScript)
if err != nil {
return nil, err
}
ntfn := &chainntnfs.SpendNtfn{
SpendID: spendID,
OutPoint: *outpoint,
PkScript: pkScript,
Event: chainntnfs.NewSpendEvent(cancel),
SpendID: spendID,
SpendRequest: spendRequest,
Event: chainntnfs.NewSpendEvent(func() {
b.txNotifier.CancelSpend(spendRequest, spendID)
}),
HeightHint: heightHint,
}
historicalDispatch, err := b.txNotifier.RegisterSpend(ntfn)
historicalDispatch, _, err := b.txNotifier.RegisterSpend(ntfn)
if err != nil {
return nil, err
}
// We'll then request the backend to notify us when it has detected the
// outpoint/output script as spent.
//
// TODO(wilmer): use LoadFilter API instead.
if spendRequest.OutPoint == chainntnfs.ZeroOutPoint {
addr, err := spendRequest.PkScript.Address(b.chainParams)
if err != nil {
return nil, err
}
addrs := []btcutil.Address{addr}
if err := b.chainConn.NotifyReceived(addrs); err != nil {
return nil, err
}
} else {
ops := []*wire.OutPoint{&spendRequest.OutPoint}
if err := b.chainConn.NotifySpent(ops); err != nil {
return nil, err
}
}
// If the txNotifier didn't return any details to perform a historical
// scan of the chain, then we can return early as there's nothing left
// for us to do.
@ -654,23 +730,39 @@ func (b *BitcoindNotifier) RegisterSpendNtfn(outpoint *wire.OutPoint,
return ntfn.Event, nil
}
// We'll then request the backend to notify us when it has detected the
// outpoint as spent.
if err := b.chainConn.NotifySpent([]*wire.OutPoint{outpoint}); err != nil {
return nil, err
// Otherwise, we'll need to dispatch a historical rescan to determine if
// the outpoint was already spent at a previous height.
//
// We'll short-circuit the path when dispatching the spend of a script,
// rather than an outpoint, as there aren't any additional checks we can
// make for scripts.
if spendRequest.OutPoint == chainntnfs.ZeroOutPoint {
select {
case b.notificationRegistry <- historicalDispatch:
case <-b.quit:
return nil, chainntnfs.ErrChainNotifierShuttingDown
}
return ntfn.Event, nil
}
// In addition to the check above, we'll also check the backend's UTXO
// set to determine whether the outpoint has been spent. If it hasn't,
// we can return to the caller as well.
txOut, err := b.chainConn.GetTxOut(&outpoint.Hash, outpoint.Index, true)
// When dispatching spends of outpoints, there are a number of checks we
// can make to start our rescan from a better height or completely avoid
// it.
//
// We'll start by checking the backend's UTXO set to determine whether
// the outpoint has been spent. If it hasn't, we can return to the
// caller as well.
txOut, err := b.chainConn.GetTxOut(
&spendRequest.OutPoint.Hash, spendRequest.OutPoint.Index, true,
)
if err != nil {
return nil, err
}
if txOut != nil {
// We'll let the txNotifier know the outpoint is still unspent
// in order to begin updating its spend hint.
err := b.txNotifier.UpdateSpendDetails(*outpoint, nil)
err := b.txNotifier.UpdateSpendDetails(spendRequest, nil)
if err != nil {
return nil, err
}
@ -678,22 +770,21 @@ func (b *BitcoindNotifier) RegisterSpendNtfn(outpoint *wire.OutPoint,
return ntfn.Event, nil
}
// Otherwise, we'll determine when the output was spent by scanning the
// chain. We'll begin by determining where to start our historical
// rescan.
// Since the outpoint was spent, as it no longer exists within the UTXO
// set, we'll determine when it happened by scanning the chain.
//
// As a minimal optimization, we'll query the backend's transaction
// index (if enabled) to determine if we have a better rescan starting
// height. We can do this as the GetRawTransaction call will return the
// hash of the block it was included in within the chain.
tx, err := b.chainConn.GetRawTransactionVerbose(&outpoint.Hash)
tx, err := b.chainConn.GetRawTransactionVerbose(&spendRequest.OutPoint.Hash)
if err != nil {
// Avoid returning an error if the transaction was not found to
// proceed with fallback methods.
jsonErr, ok := err.(*btcjson.RPCError)
if !ok || jsonErr.Code != btcjson.ErrRPCNoTxInfo {
return nil, fmt.Errorf("unable to query for "+
"txid %v: %v", outpoint.Hash, err)
return nil, fmt.Errorf("unable to query for txid %v: %v",
spendRequest.OutPoint.Hash, err)
}
}
@ -722,23 +813,24 @@ func (b *BitcoindNotifier) RegisterSpendNtfn(outpoint *wire.OutPoint,
}
// Now that we've determined the starting point of our rescan, we can
// dispatch it.
// dispatch it and return.
select {
case b.notificationRegistry <- historicalDispatch:
return ntfn.Event, nil
case <-b.quit:
return nil, ErrChainNotifierShuttingDown
return nil, chainntnfs.ErrChainNotifierShuttingDown
}
return ntfn.Event, nil
}
// disaptchSpendDetailsManually attempts to manually scan the chain within the
// given height range for a transaction that spends the given outpoint. If one
// is found, it's spending details are sent to the notifier dispatcher, which
// will then dispatch the notification to all of its clients.
// given height range for a transaction that spends the given outpoint/output
// script. If one is found, it's spending details are sent to the TxNotifier,
// which will then dispatch the notification to all of its clients.
func (b *BitcoindNotifier) dispatchSpendDetailsManually(
historicalDispatchDetails *chainntnfs.HistoricalSpendDispatch) error {
op := historicalDispatchDetails.OutPoint
spendRequest := historicalDispatchDetails.SpendRequest
startHeight := historicalDispatchDetails.StartHeight
endHeight := historicalDispatchDetails.EndHeight
@ -749,7 +841,7 @@ func (b *BitcoindNotifier) dispatchSpendDetailsManually(
// processing the next height.
select {
case <-b.quit:
return ErrChainNotifierShuttingDown
return chainntnfs.ErrChainNotifierShuttingDown
default:
}
@ -765,61 +857,75 @@ func (b *BitcoindNotifier) dispatchSpendDetailsManually(
"%v: %v", blockHash, err)
}
// Then, we'll manually go over every transaction in it and
// determine whether it spends the outpoint in question.
// Then, we'll manually go over every input in every transaction
// in it and determine whether it spends the request in
// question. If we find one, we'll dispatch the spend details.
for _, tx := range block.Transactions {
for i, txIn := range tx.TxIn {
if txIn.PreviousOutPoint != op {
continue
}
// If it does, we'll construct its spend details
// and hand them over to the TxNotifier so that
// it can properly notify its registered
// clients.
txHash := tx.TxHash()
details := &chainntnfs.SpendDetail{
SpentOutPoint: &op,
SpenderTxHash: &txHash,
SpendingTx: tx,
SpenderInputIndex: uint32(i),
SpendingHeight: int32(height),
}
return b.txNotifier.UpdateSpendDetails(
op, details,
)
matches, inputIdx, err := spendRequest.MatchesTx(tx)
if err != nil {
return err
}
if !matches {
continue
}
txHash := tx.TxHash()
details := &chainntnfs.SpendDetail{
SpentOutPoint: &tx.TxIn[inputIdx].PreviousOutPoint,
SpenderTxHash: &txHash,
SpendingTx: tx,
SpenderInputIndex: inputIdx,
SpendingHeight: int32(height),
}
return b.txNotifier.UpdateSpendDetails(
historicalDispatchDetails.SpendRequest,
details,
)
}
}
return ErrTransactionNotFound
}
// RegisterConfirmationsNtfn registers a notification with BitcoindNotifier
// which will be triggered once the txid reaches numConfs number of
// confirmations.
// RegisterConfirmationsNtfn registers an intent to be notified once the target
// txid/output script has reached numConfs confirmations on-chain. When
// intending to be notified of the confirmation of an output script, a nil txid
// must be used. The heightHint should represent the earliest height at which
// the txid/output script could have been included in the chain.
//
// Progress on the number of confirmations left can be read from the 'Updates'
// channel. Once it has reached all of its confirmations, a notification will be
// sent across the 'Confirmed' channel.
func (b *BitcoindNotifier) RegisterConfirmationsNtfn(txid *chainhash.Hash,
_ []byte, numConfs, heightHint uint32) (*chainntnfs.ConfirmationEvent, error) {
pkScript []byte,
numConfs, heightHint uint32) (*chainntnfs.ConfirmationEvent, error) {
// Construct a notification request for the transaction and send it to
// the main event loop.
confID := atomic.AddUint64(&b.confClientCounter, 1)
confRequest, err := chainntnfs.NewConfRequest(txid, pkScript)
if err != nil {
return nil, err
}
ntfn := &chainntnfs.ConfNtfn{
ConfID: atomic.AddUint64(&b.confClientCounter, 1),
TxID: txid,
ConfID: confID,
ConfRequest: confRequest,
NumConfirmations: numConfs,
Event: chainntnfs.NewConfirmationEvent(numConfs),
HeightHint: heightHint,
Event: chainntnfs.NewConfirmationEvent(numConfs, func() {
b.txNotifier.CancelConf(confRequest, confID)
}),
HeightHint: heightHint,
}
chainntnfs.Log.Infof("New confirmation subscription: "+
"txid=%v, numconfs=%v", txid, numConfs)
chainntnfs.Log.Infof("New confirmation subscription: %v, num_confs=%v",
confRequest, numConfs)
// Register the conf notification with the TxNotifier. A non-nil value
// for `dispatch` will be returned if we are required to perform a
// manual scan for the confirmation. Otherwise the notifier will begin
// watching at tip for the transaction to confirm.
dispatch, err := b.txNotifier.RegisterConf(ntfn)
dispatch, _, err := b.txNotifier.RegisterConf(ntfn)
if err != nil {
return nil, err
}
@ -832,7 +938,7 @@ func (b *BitcoindNotifier) RegisterConfirmationsNtfn(txid *chainhash.Hash,
case b.notificationRegistry <- dispatch:
return ntfn.Event, nil
case <-b.quit:
return nil, ErrChainNotifierShuttingDown
return nil, chainntnfs.ErrChainNotifierShuttingDown
}
}
@ -863,7 +969,9 @@ type epochCancel struct {
// RegisterBlockEpochNtfn returns a BlockEpochEvent which subscribes the
// caller to receive notifications, of each new block connected to the main
// chain. Clients have the option of passing in their best known block, which
// the notifier uses to check if they are behind on blocks and catch them up.
// the notifier uses to check if they are behind on blocks and catch them up. If
// they do not provide one, then a notification will be dispatched immediately
// for the current tip of the chain upon a successful registration.
func (b *BitcoindNotifier) RegisterBlockEpochNtfn(
bestBlock *chainntnfs.BlockEpoch) (*chainntnfs.BlockEpochEvent, error) {

@ -3,6 +3,7 @@
package bitcoindnotify
import (
"bytes"
"io/ioutil"
"testing"
"time"
@ -14,6 +15,20 @@ import (
"github.com/lightningnetwork/lnd/channeldb"
)
var (
testScript = []byte{
// OP_HASH160
0xA9,
// OP_DATA_20
0x14,
// <20-byte hash>
0xec, 0x6f, 0x7a, 0x5a, 0xa8, 0xf2, 0xb1, 0x0c, 0xa5, 0x15,
0x04, 0x52, 0x3a, 0x60, 0xd4, 0x03, 0x06, 0xf6, 0x96, 0xcd,
// OP_EQUAL
0x87,
}
)
func initHintCache(t *testing.T) *chainntnfs.HeightHintCache {
t.Helper()
@ -41,7 +56,10 @@ func setUpNotifier(t *testing.T, bitcoindConn *chain.BitcoindConn,
t.Helper()
notifier := New(bitcoindConn, spendHintCache, confirmHintCache)
notifier := New(
bitcoindConn, chainntnfs.NetParams, spendHintCache,
confirmHintCache,
)
if err := notifier.Start(); err != nil {
t.Fatalf("unable to start notifier: %v", err)
}
@ -104,8 +122,13 @@ func TestHistoricalConfDetailsTxIndex(t *testing.T) {
// A transaction unknown to the node should not be found within the
// txindex even if it is enabled, so we should not proceed with any
// fallback methods.
var zeroHash chainhash.Hash
_, txStatus, err := notifier.historicalConfDetails(&zeroHash, 0, 0)
var unknownHash chainhash.Hash
copy(unknownHash[:], bytes.Repeat([]byte{0x10}, 32))
unknownConfReq, err := chainntnfs.NewConfRequest(&unknownHash, testScript)
if err != nil {
t.Fatalf("unable to create conf request: %v", err)
}
_, txStatus, err := notifier.historicalConfDetails(unknownConfReq, 0, 0)
if err != nil {
t.Fatalf("unable to retrieve historical conf details: %v", err)
}
@ -120,16 +143,20 @@ func TestHistoricalConfDetailsTxIndex(t *testing.T) {
// Now, we'll create a test transaction, confirm it, and attempt to
// retrieve its confirmation details.
txid, _, err := chainntnfs.GetTestTxidAndScript(miner)
txid, pkScript, err := chainntnfs.GetTestTxidAndScript(miner)
if err != nil {
t.Fatalf("unable to create tx: %v", err)
}
if err := chainntnfs.WaitForMempoolTx(miner, txid); err != nil {
t.Fatal(err)
}
confReq, err := chainntnfs.NewConfRequest(txid, pkScript)
if err != nil {
t.Fatalf("unable to create conf request: %v", err)
}
// The transaction should be found in the mempool at this point.
_, txStatus, err = notifier.historicalConfDetails(txid, 0, 0)
_, txStatus, err = notifier.historicalConfDetails(confReq, 0, 0)
if err != nil {
t.Fatalf("unable to retrieve historical conf details: %v", err)
}
@ -151,7 +178,7 @@ func TestHistoricalConfDetailsTxIndex(t *testing.T) {
// the txindex includes the transaction just mined.
syncNotifierWithMiner(t, notifier, miner)
_, txStatus, err = notifier.historicalConfDetails(txid, 0, 0)
_, txStatus, err = notifier.historicalConfDetails(confReq, 0, 0)
if err != nil {
t.Fatalf("unable to retrieve historical conf details: %v", err)
}
@ -186,10 +213,15 @@ func TestHistoricalConfDetailsNoTxIndex(t *testing.T) {
// Since the node has its txindex disabled, we fall back to scanning the
// chain manually. A transaction unknown to the network should not be
// found.
var zeroHash chainhash.Hash
var unknownHash chainhash.Hash
copy(unknownHash[:], bytes.Repeat([]byte{0x10}, 32))
unknownConfReq, err := chainntnfs.NewConfRequest(&unknownHash, testScript)
if err != nil {
t.Fatalf("unable to create conf request: %v", err)
}
broadcastHeight := syncNotifierWithMiner(t, notifier, miner)
_, txStatus, err := notifier.historicalConfDetails(
&zeroHash, uint32(broadcastHeight), uint32(broadcastHeight),
unknownConfReq, uint32(broadcastHeight), uint32(broadcastHeight),
)
if err != nil {
t.Fatalf("unable to retrieve historical conf details: %v", err)
@ -210,8 +242,8 @@ func TestHistoricalConfDetailsNoTxIndex(t *testing.T) {
// one output, which we will manually spend. The backend node's
// transaction index should also be disabled, which we've already
// ensured above.
output, pkScript := chainntnfs.CreateSpendableOutput(t, miner)
spendTx := chainntnfs.CreateSpendTx(t, output, pkScript)
outpoint, output, privKey := chainntnfs.CreateSpendableOutput(t, miner)
spendTx := chainntnfs.CreateSpendTx(t, outpoint, output, privKey)
spendTxHash, err := miner.Node.SendRawTransaction(spendTx, true)
if err != nil {
t.Fatalf("unable to broadcast tx: %v", err)
@ -225,9 +257,13 @@ func TestHistoricalConfDetailsNoTxIndex(t *testing.T) {
// Ensure the notifier and miner are synced to the same height to ensure
// we can find the transaction when manually scanning the chain.
confReq, err := chainntnfs.NewConfRequest(&outpoint.Hash, output.PkScript)
if err != nil {
t.Fatalf("unable to create conf request: %v", err)
}
currentHeight := syncNotifierWithMiner(t, notifier, miner)
_, txStatus, err = notifier.historicalConfDetails(
&output.Hash, uint32(broadcastHeight), uint32(currentHeight),
confReq, uint32(broadcastHeight), uint32(currentHeight),
)
if err != nil {
t.Fatalf("unable to retrieve historical conf details: %v", err)

@ -4,6 +4,7 @@ import (
"errors"
"fmt"
"github.com/btcsuite/btcd/chaincfg"
"github.com/btcsuite/btcwallet/chain"
"github.com/lightningnetwork/lnd/chainntnfs"
)
@ -11,9 +12,9 @@ import (
// createNewNotifier creates a new instance of the ChainNotifier interface
// implemented by BitcoindNotifier.
func createNewNotifier(args ...interface{}) (chainntnfs.ChainNotifier, error) {
if len(args) != 3 {
if len(args) != 4 {
return nil, fmt.Errorf("incorrect number of arguments to "+
".New(...), expected 2, instead passed %v", len(args))
".New(...), expected 4, instead passed %v", len(args))
}
chainConn, ok := args[0].(*chain.BitcoindConn)
@ -22,19 +23,25 @@ func createNewNotifier(args ...interface{}) (chainntnfs.ChainNotifier, error) {
"is incorrect, expected a *chain.BitcoindConn")
}
spendHintCache, ok := args[1].(chainntnfs.SpendHintCache)
chainParams, ok := args[1].(*chaincfg.Params)
if !ok {
return nil, errors.New("second argument to bitcoindnotify.New " +
"is incorrect, expected a *chaincfg.Params")
}
spendHintCache, ok := args[2].(chainntnfs.SpendHintCache)
if !ok {
return nil, errors.New("third argument to bitcoindnotify.New " +
"is incorrect, expected a chainntnfs.SpendHintCache")
}
confirmHintCache, ok := args[2].(chainntnfs.ConfirmHintCache)
confirmHintCache, ok := args[3].(chainntnfs.ConfirmHintCache)
if !ok {
return nil, errors.New("third argument to bitcoindnotify.New " +
return nil, errors.New("fourth argument to bitcoindnotify.New " +
"is incorrect, expected a chainntnfs.ConfirmHintCache")
}
return New(chainConn, spendHintCache, confirmHintCache), nil
return New(chainConn, chainParams, spendHintCache, confirmHintCache), nil
}
// init registers a driver for the BtcdNotifier concrete implementation of the

@ -1,6 +1,8 @@
package btcdnotify
import (
"bytes"
"encoding/hex"
"errors"
"fmt"
"strings"
@ -9,6 +11,7 @@ import (
"time"
"github.com/btcsuite/btcd/btcjson"
"github.com/btcsuite/btcd/chaincfg"
"github.com/btcsuite/btcd/chaincfg/chainhash"
"github.com/btcsuite/btcd/rpcclient"
"github.com/btcsuite/btcd/wire"
@ -23,13 +26,6 @@ const (
notifierType = "btcd"
)
var (
// ErrChainNotifierShuttingDown is used when we are trying to
// measure a spend notification when notifier is already stopped.
ErrChainNotifierShuttingDown = errors.New("chainntnfs: system interrupt " +
"while attempting to register for spend notification.")
)
// chainUpdate encapsulates an update to the current main chain. This struct is
// used as an element within an unbounded queue in order to avoid blocking the
// main rpc dispatch rule.
@ -64,7 +60,8 @@ type BtcdNotifier struct {
started int32 // To be used atomically.
stopped int32 // To be used atomically.
chainConn *rpcclient.Client
chainConn *rpcclient.Client
chainParams *chaincfg.Params
notificationCancels chan interface{}
notificationRegistry chan interface{}
@ -98,10 +95,13 @@ var _ chainntnfs.ChainNotifier = (*BtcdNotifier)(nil)
// New returns a new BtcdNotifier instance. This function assumes the btcd node
// detailed in the passed configuration is already running, and willing to
// accept new websockets clients.
func New(config *rpcclient.ConnConfig, spendHintCache chainntnfs.SpendHintCache,
func New(config *rpcclient.ConnConfig, chainParams *chaincfg.Params,
spendHintCache chainntnfs.SpendHintCache,
confirmHintCache chainntnfs.ConfirmHintCache) (*BtcdNotifier, error) {
notifier := &BtcdNotifier{
chainParams: chainParams,
notificationCancels: make(chan interface{}),
notificationRegistry: make(chan interface{}),
@ -289,10 +289,10 @@ out:
case registerMsg := <-b.notificationRegistry:
switch msg := registerMsg.(type) {
case *chainntnfs.HistoricalConfDispatch:
// Look up whether the transaction is already
// included in the active chain. We'll do this
// in a goroutine to prevent blocking
// potentially long rescans.
// Look up whether the transaction/output script
// has already confirmed in the active chain.
// We'll do this in a goroutine to prevent
// blocking potentially long rescans.
//
// TODO(wilmer): add retry logic if rescan fails?
b.wg.Add(1)
@ -300,7 +300,8 @@ out:
defer b.wg.Done()
confDetails, _, err := b.historicalConfDetails(
msg.TxID, msg.StartHeight, msg.EndHeight,
msg.ConfRequest,
msg.StartHeight, msg.EndHeight,
)
if err != nil {
chainntnfs.Log.Error(err)
@ -315,7 +316,7 @@ out:
// cache at tip, since any pending
// rescans have now completed.
err = b.txNotifier.UpdateConfDetails(
*msg.TxID, confDetails,
msg.ConfRequest, confDetails,
)
if err != nil {
chainntnfs.Log.Error(err)
@ -324,23 +325,40 @@ out:
case *blockEpochRegistration:
chainntnfs.Log.Infof("New block epoch subscription")
b.blockEpochClients[msg.epochID] = msg
if msg.bestBlock != nil {
missedBlocks, err :=
chainntnfs.GetClientMissedBlocks(
b.chainConn, msg.bestBlock,
b.bestBlock.Height, true,
)
if err != nil {
msg.errorChan <- err
continue
}
for _, block := range missedBlocks {
b.notifyBlockEpochClient(msg,
block.Height, block.Hash)
}
b.blockEpochClients[msg.epochID] = msg
// If the client did not provide their best
// known block, then we'll immediately dispatch
// a notification for the current tip.
if msg.bestBlock == nil {
b.notifyBlockEpochClient(
msg, b.bestBlock.Height,
b.bestBlock.Hash,
)
msg.errorChan <- nil
continue
}
// Otherwise, we'll attempt to deliver the
// backlog of notifications from their best
// known block.
missedBlocks, err := chainntnfs.GetClientMissedBlocks(
b.chainConn, msg.bestBlock,
b.bestBlock.Height, true,
)
if err != nil {
msg.errorChan <- err
continue
}
for _, block := range missedBlocks {
b.notifyBlockEpochClient(
msg, block.Height, block.Hash,
)
}
msg.errorChan <- nil
}
@ -425,13 +443,13 @@ out:
continue
}
tx := newSpend.tx.MsgTx()
err := b.txNotifier.ProcessRelevantSpendTx(
tx, newSpend.details.Height,
newSpend.tx, uint32(newSpend.details.Height),
)
if err != nil {
chainntnfs.Log.Errorf("Unable to process "+
"transaction %v: %v", tx.TxHash(), err)
"transaction %v: %v",
newSpend.tx.Hash(), err)
}
case <-b.quit:
@ -441,15 +459,28 @@ out:
b.wg.Done()
}
// historicalConfDetails looks up whether a transaction is already included in a
// block in the active chain and, if so, returns details about the confirmation.
func (b *BtcdNotifier) historicalConfDetails(txid *chainhash.Hash,
// historicalConfDetails looks up whether a confirmation request (txid/output
// script) has already been included in a block in the active chain and, if so,
// returns details about said block.
func (b *BtcdNotifier) historicalConfDetails(confRequest chainntnfs.ConfRequest,
startHeight, endHeight uint32) (*chainntnfs.TxConfirmation,
chainntnfs.TxConfStatus, error) {
// If a txid was not provided, then we should dispatch upon seeing the
// script on-chain, so we'll short-circuit straight to scanning manually
// as there doesn't exist a script index to query.
if confRequest.TxID == chainntnfs.ZeroHash {
return b.confDetailsManually(
confRequest, startHeight, endHeight,
)
}
// Otherwise, we'll dispatch upon seeing a transaction on-chain with the
// given hash.
//
// We'll first attempt to retrieve the transaction using the node's
// txindex.
txConf, txStatus, err := b.confDetailsFromTxIndex(txid)
txConf, txStatus, err := b.confDetailsFromTxIndex(&confRequest.TxID)
// We'll then check the status of the transaction lookup returned to
// determine whether we should proceed with any fallback methods.
@ -458,9 +489,13 @@ func (b *BtcdNotifier) historicalConfDetails(txid *chainhash.Hash,
// We failed querying the index for the transaction, fall back to
// scanning manually.
case err != nil:
chainntnfs.Log.Debugf("Failed getting conf details from "+
"index (%v), scanning manually", err)
return b.confDetailsManually(txid, startHeight, endHeight)
chainntnfs.Log.Debugf("Unable to determine confirmation of %v "+
"through the backend's txindex (%v), scanning manually",
confRequest.TxID, err)
return b.confDetailsManually(
confRequest, startHeight, endHeight,
)
// The transaction was found within the node's mempool.
case txStatus == chainntnfs.TxFoundMempool:
@ -491,7 +526,7 @@ func (b *BtcdNotifier) confDetailsFromTxIndex(txid *chainhash.Hash,
// If the transaction has some or all of its confirmations required,
// then we may be able to dispatch it immediately.
tx, err := b.chainConn.GetRawTransactionVerbose(txid)
rawTxRes, err := b.chainConn.GetRawTransactionVerbose(txid)
if err != nil {
// If the transaction lookup was successful, but it wasn't found
// within the index itself, then we can exit early. We'll also
@ -512,20 +547,19 @@ func (b *BtcdNotifier) confDetailsFromTxIndex(txid *chainhash.Hash,
// Make sure we actually retrieved a transaction that is included in a
// block. If not, the transaction must be unconfirmed (in the mempool),
// and we'll return TxFoundMempool together with a nil TxConfirmation.
if tx.BlockHash == "" {
if rawTxRes.BlockHash == "" {
return nil, chainntnfs.TxFoundMempool, nil
}
// As we need to fully populate the returned TxConfirmation struct,
// grab the block in which the transaction was confirmed so we can
// locate its exact index within the block.
blockHash, err := chainhash.NewHashFromStr(tx.BlockHash)
blockHash, err := chainhash.NewHashFromStr(rawTxRes.BlockHash)
if err != nil {
return nil, chainntnfs.TxNotFoundIndex,
fmt.Errorf("unable to get block hash %v for "+
"historical dispatch: %v", tx.BlockHash, err)
"historical dispatch: %v", rawTxRes.BlockHash, err)
}
block, err := b.chainConn.GetBlockVerbose(blockHash)
if err != nil {
return nil, chainntnfs.TxNotFoundIndex,
@ -535,36 +569,49 @@ func (b *BtcdNotifier) confDetailsFromTxIndex(txid *chainhash.Hash,
// If the block was obtained, locate the transaction's index within the
// block so we can give the subscriber full confirmation details.
targetTxidStr := txid.String()
txidStr := txid.String()
for txIndex, txHash := range block.Tx {
if txHash == targetTxidStr {
details := &chainntnfs.TxConfirmation{
BlockHash: blockHash,
BlockHeight: uint32(block.Height),
TxIndex: uint32(txIndex),
}
return details, chainntnfs.TxFoundIndex, nil
if txHash != txidStr {
continue
}
// Deserialize the hex-encoded transaction to include it in the
// confirmation details.
rawTx, err := hex.DecodeString(rawTxRes.Hex)
if err != nil {
return nil, chainntnfs.TxFoundIndex,
fmt.Errorf("unable to deserialize tx %v: %v",
txHash, err)
}
var tx wire.MsgTx
if err := tx.Deserialize(bytes.NewReader(rawTx)); err != nil {
return nil, chainntnfs.TxFoundIndex,
fmt.Errorf("unable to deserialize tx %v: %v",
txHash, err)
}
return &chainntnfs.TxConfirmation{
Tx: &tx,
BlockHash: blockHash,
BlockHeight: uint32(block.Height),
TxIndex: uint32(txIndex),
}, chainntnfs.TxFoundIndex, nil
}
// We return an error because we should have found the transaction
// within the block, but didn't.
return nil, chainntnfs.TxNotFoundIndex,
fmt.Errorf("unable to locate tx %v in block %v", txid,
blockHash)
return nil, chainntnfs.TxNotFoundIndex, fmt.Errorf("unable to locate "+
"tx %v in block %v", txid, blockHash)
}
// confDetailsManually looks up whether a transaction is already included in a
// block in the active chain by scanning the chain's blocks, starting from the
// earliest height the transaction could have been included in, to the current
// height in the chain. If the transaction is found, its confirmation details
// are returned. Otherwise, nil is returned.
func (b *BtcdNotifier) confDetailsManually(txid *chainhash.Hash, startHeight,
endHeight uint32) (*chainntnfs.TxConfirmation,
// confDetailsManually looks up whether a transaction/output script has already
// been included in a block in the active chain by scanning the chain's blocks
// within the given range. If the transaction/output script is found, its
// confirmation details are returned. Otherwise, nil is returned.
func (b *BtcdNotifier) confDetailsManually(confRequest chainntnfs.ConfRequest,
startHeight, endHeight uint32) (*chainntnfs.TxConfirmation,
chainntnfs.TxConfStatus, error) {
targetTxidStr := txid.String()
// Begin scanning blocks at every height to determine where the
// transaction was included in.
for height := endHeight; height >= startHeight && height > 0; height-- {
@ -573,7 +620,7 @@ func (b *BtcdNotifier) confDetailsManually(txid *chainhash.Hash, startHeight,
select {
case <-b.quit:
return nil, chainntnfs.TxNotFoundManually,
ErrChainNotifierShuttingDown
chainntnfs.ErrChainNotifierShuttingDown
default:
}
@ -585,24 +632,27 @@ func (b *BtcdNotifier) confDetailsManually(txid *chainhash.Hash, startHeight,
}
// TODO: fetch the neutrino filters instead.
block, err := b.chainConn.GetBlockVerbose(blockHash)
block, err := b.chainConn.GetBlock(blockHash)
if err != nil {
return nil, chainntnfs.TxNotFoundManually,
fmt.Errorf("unable to get block with hash "+
"%v: %v", blockHash, err)
}
for txIndex, txHash := range block.Tx {
// If we're able to find the transaction in this block,
// return its confirmation details.
if txHash == targetTxidStr {
details := &chainntnfs.TxConfirmation{
BlockHash: blockHash,
BlockHeight: height,
TxIndex: uint32(txIndex),
}
return details, chainntnfs.TxFoundManually, nil
// For every transaction in the block, check which one matches
// our request. If we find one that does, we can dispatch its
// confirmation details.
for txIndex, tx := range block.Transactions {
if !confRequest.MatchesTx(tx) {
continue
}
return &chainntnfs.TxConfirmation{
Tx: tx,
BlockHash: blockHash,
BlockHeight: height,
TxIndex: uint32(txIndex),
}, chainntnfs.TxFoundManually, nil
}
}
@ -681,32 +731,57 @@ func (b *BtcdNotifier) notifyBlockEpochClient(epochClient *blockEpochRegistratio
}
// RegisterSpendNtfn registers an intent to be notified once the target
// outpoint has been spent by a transaction on-chain. Once a spend of the target
// outpoint has been detected, the details of the spending event will be sent
// across the 'Spend' channel. The heightHint should represent the earliest
// height in the chain where the transaction could have been spent in.
// outpoint/output script has been spent by a transaction on-chain. When
// intending to be notified of the spend of an output script, a nil outpoint
// must be used. The heightHint should represent the earliest height in the
// chain of the transaction that spent the outpoint/output script.
//
// Once a spend of has been detected, the details of the spending event will be
// sent across the 'Spend' channel.
func (b *BtcdNotifier) RegisterSpendNtfn(outpoint *wire.OutPoint,
pkScript []byte, heightHint uint32) (*chainntnfs.SpendEvent, error) {
// First, we'll construct a spend notification request and hand it off
// to the txNotifier.
spendID := atomic.AddUint64(&b.spendClientCounter, 1)
cancel := func() {
b.txNotifier.CancelSpend(*outpoint, spendID)
spendRequest, err := chainntnfs.NewSpendRequest(outpoint, pkScript)
if err != nil {
return nil, err
}
ntfn := &chainntnfs.SpendNtfn{
SpendID: spendID,
OutPoint: *outpoint,
PkScript: pkScript,
Event: chainntnfs.NewSpendEvent(cancel),
SpendID: spendID,
SpendRequest: spendRequest,
Event: chainntnfs.NewSpendEvent(func() {
b.txNotifier.CancelSpend(spendRequest, spendID)
}),
HeightHint: heightHint,
}
historicalDispatch, err := b.txNotifier.RegisterSpend(ntfn)
historicalDispatch, _, err := b.txNotifier.RegisterSpend(ntfn)
if err != nil {
return nil, err
}
// We'll then request the backend to notify us when it has detected the
// outpoint/output script as spent.
//
// TODO(wilmer): use LoadFilter API instead.
if spendRequest.OutPoint == chainntnfs.ZeroOutPoint {
addr, err := spendRequest.PkScript.Address(b.chainParams)
if err != nil {
return nil, err
}
addrs := []btcutil.Address{addr}
if err := b.chainConn.NotifyReceived(addrs); err != nil {
return nil, err
}
} else {
ops := []*wire.OutPoint{&spendRequest.OutPoint}
if err := b.chainConn.NotifySpent(ops); err != nil {
return nil, err
}
}
// If the txNotifier didn't return any details to perform a historical
// scan of the chain, then we can return early as there's nothing left
// for us to do.
@ -714,24 +789,55 @@ func (b *BtcdNotifier) RegisterSpendNtfn(outpoint *wire.OutPoint,
return ntfn.Event, nil
}
// We'll then request the backend to notify us when it has detected the
// outpoint as spent.
ops := []*wire.OutPoint{outpoint}
if err := b.chainConn.NotifySpent(ops); err != nil {
return nil, err
// Otherwise, we'll need to dispatch a historical rescan to determine if
// the outpoint was already spent at a previous height.
//
// We'll short-circuit the path when dispatching the spend of a script,
// rather than an outpoint, as there aren't any additional checks we can
// make for scripts.
if spendRequest.OutPoint == chainntnfs.ZeroOutPoint {
startHash, err := b.chainConn.GetBlockHash(
int64(historicalDispatch.StartHeight),
)
if err != nil {
return nil, err
}
// TODO(wilmer): add retry logic if rescan fails?
addr, err := spendRequest.PkScript.Address(b.chainParams)
if err != nil {
return nil, err
}
addrs := []btcutil.Address{addr}
asyncResult := b.chainConn.RescanAsync(startHash, addrs, nil)
go func() {
if rescanErr := asyncResult.Receive(); rescanErr != nil {
chainntnfs.Log.Errorf("Rescan to determine "+
"the spend details of %v failed: %v",
spendRequest, rescanErr)
}
}()
return ntfn.Event, nil
}
// In addition to the check above, we'll also check the backend's UTXO
// set to determine whether the outpoint has been spent. If it hasn't,
// we can return to the caller as well.
txOut, err := b.chainConn.GetTxOut(&outpoint.Hash, outpoint.Index, true)
// When dispatching spends of outpoints, there are a number of checks we
// can make to start our rescan from a better height or completely avoid
// it.
//
// We'll start by checking the backend's UTXO set to determine whether
// the outpoint has been spent. If it hasn't, we can return to the
// caller as well.
txOut, err := b.chainConn.GetTxOut(
&spendRequest.OutPoint.Hash, spendRequest.OutPoint.Index, true,
)
if err != nil {
return nil, err
}
if txOut != nil {
// We'll let the txNotifier know the outpoint is still unspent
// in order to begin updating its spend hint.
err := b.txNotifier.UpdateSpendDetails(*outpoint, nil)
err := b.txNotifier.UpdateSpendDetails(spendRequest, nil)
if err != nil {
return nil, err
}
@ -739,9 +845,9 @@ func (b *BtcdNotifier) RegisterSpendNtfn(outpoint *wire.OutPoint,
return ntfn.Event, nil
}
// Otherwise, we'll determine when the output was spent by scanning the
// chain. We'll begin by determining where to start our historical
// rescan.
// Since the outpoint was spent, as it no longer exists within the UTXO
// set, we'll determine when it happened by scanning the chain. We'll
// begin by fetching the block hash of our starting height.
startHash, err := b.chainConn.GetBlockHash(
int64(historicalDispatch.StartHeight),
)
@ -754,14 +860,14 @@ func (b *BtcdNotifier) RegisterSpendNtfn(outpoint *wire.OutPoint,
// index (if enabled) to determine if we have a better rescan starting
// height. We can do this as the GetRawTransaction call will return the
// hash of the block it was included in within the chain.
tx, err := b.chainConn.GetRawTransactionVerbose(&outpoint.Hash)
tx, err := b.chainConn.GetRawTransactionVerbose(&spendRequest.OutPoint.Hash)
if err != nil {
// Avoid returning an error if the transaction was not found to
// proceed with fallback methods.
jsonErr, ok := err.(*btcjson.RPCError)
if !ok || jsonErr.Code != btcjson.ErrRPCNoTxInfo {
return nil, fmt.Errorf("unable to query for "+
"txid %v: %v", outpoint.Hash, err)
return nil, fmt.Errorf("unable to query for txid %v: %v",
spendRequest.OutPoint.Hash, err)
}
}
@ -797,6 +903,9 @@ func (b *BtcdNotifier) RegisterSpendNtfn(outpoint *wire.OutPoint,
}
}
// Now that we've determined the best starting point for our rescan,
// we can go ahead and dispatch it.
//
// In order to ensure that we don't block the caller on what may be a
// long rescan, we'll launch a new goroutine to handle the async result
// of the rescan. We purposefully prevent from adding this goroutine to
@ -804,41 +913,58 @@ func (b *BtcdNotifier) RegisterSpendNtfn(outpoint *wire.OutPoint,
// asyncResult channel not being exposed.
//
// TODO(wilmer): add retry logic if rescan fails?
asyncResult := b.chainConn.RescanAsync(startHash, nil, ops)
asyncResult := b.chainConn.RescanAsync(
startHash, nil, []*wire.OutPoint{&spendRequest.OutPoint},
)
go func() {
if rescanErr := asyncResult.Receive(); rescanErr != nil {
chainntnfs.Log.Errorf("Rescan to determine the spend "+
"details of %v failed: %v", outpoint, rescanErr)
"details of %v failed: %v", spendRequest,
rescanErr)
}
}()
return ntfn.Event, nil
}
// RegisterConfirmationsNtfn registers a notification with BtcdNotifier
// which will be triggered once the txid reaches numConfs number of
// confirmations.
func (b *BtcdNotifier) RegisterConfirmationsNtfn(txid *chainhash.Hash, _ []byte,
// RegisterConfirmationsNtfn registers an intent to be notified once the target
// txid/output script has reached numConfs confirmations on-chain. When
// intending to be notified of the confirmation of an output script, a nil txid
// must be used. The heightHint should represent the earliest height at which
// the txid/output script could have been included in the chain.
//
// Progress on the number of confirmations left can be read from the 'Updates'
// channel. Once it has reached all of its confirmations, a notification will be
// sent across the 'Confirmed' channel.
func (b *BtcdNotifier) RegisterConfirmationsNtfn(txid *chainhash.Hash,
pkScript []byte,
numConfs, heightHint uint32) (*chainntnfs.ConfirmationEvent, error) {
// Construct a notification request for the transaction and send it to
// the main event loop.
confID := atomic.AddUint64(&b.confClientCounter, 1)
confRequest, err := chainntnfs.NewConfRequest(txid, pkScript)
if err != nil {
return nil, err
}
ntfn := &chainntnfs.ConfNtfn{
ConfID: atomic.AddUint64(&b.confClientCounter, 1),
TxID: txid,
ConfID: confID,
ConfRequest: confRequest,
NumConfirmations: numConfs,
Event: chainntnfs.NewConfirmationEvent(numConfs),
HeightHint: heightHint,
Event: chainntnfs.NewConfirmationEvent(numConfs, func() {
b.txNotifier.CancelConf(confRequest, confID)
}),
HeightHint: heightHint,
}
chainntnfs.Log.Infof("New confirmation subscription: "+
"txid=%v, numconfs=%v", txid, numConfs)
chainntnfs.Log.Infof("New confirmation subscription: %v, num_confs=%v ",
confRequest, numConfs)
// Register the conf notification with the TxNotifier. A non-nil value
// for `dispatch` will be returned if we are required to perform a
// manual scan for the confirmation. Otherwise the notifier will begin
// watching at tip for the transaction to confirm.
dispatch, err := b.txNotifier.RegisterConf(ntfn)
dispatch, _, err := b.txNotifier.RegisterConf(ntfn)
if err != nil {
return nil, err
}
@ -851,7 +977,7 @@ func (b *BtcdNotifier) RegisterConfirmationsNtfn(txid *chainhash.Hash, _ []byte,
case b.notificationRegistry <- dispatch:
return ntfn.Event, nil
case <-b.quit:
return nil, ErrChainNotifierShuttingDown
return nil, chainntnfs.ErrChainNotifierShuttingDown
}
}
@ -882,7 +1008,9 @@ type epochCancel struct {
// RegisterBlockEpochNtfn returns a BlockEpochEvent which subscribes the
// caller to receive notifications, of each new block connected to the main
// chain. Clients have the option of passing in their best known block, which
// the notifier uses to check if they are behind on blocks and catch them up.
// the notifier uses to check if they are behind on blocks and catch them up. If
// they do not provide one, then a notification will be dispatched immediately
// for the current tip of the chain upon a successful registration.
func (b *BtcdNotifier) RegisterBlockEpochNtfn(
bestBlock *chainntnfs.BlockEpoch) (*chainntnfs.BlockEpochEvent, error) {
@ -894,6 +1022,7 @@ func (b *BtcdNotifier) RegisterBlockEpochNtfn(
bestBlock: bestBlock,
errorChan: make(chan error, 1),
}
reg.epochQueue.Start()
// Before we send the request to the main goroutine, we'll launch a new

@ -3,6 +3,7 @@
package btcdnotify
import (
"bytes"
"io/ioutil"
"testing"
@ -12,6 +13,20 @@ import (
"github.com/lightningnetwork/lnd/channeldb"
)
var (
testScript = []byte{
// OP_HASH160
0xA9,
// OP_DATA_20
0x14,
// <20-byte hash>
0xec, 0x6f, 0x7a, 0x5a, 0xa8, 0xf2, 0xb1, 0x0c, 0xa5, 0x15,
0x04, 0x52, 0x3a, 0x60, 0xd4, 0x03, 0x06, 0xf6, 0x96, 0xcd,
// OP_EQUAL
0x87,
}
)
func initHintCache(t *testing.T) *chainntnfs.HeightHintCache {
t.Helper()
@ -37,7 +52,7 @@ func setUpNotifier(t *testing.T, h *rpctest.Harness) *BtcdNotifier {
hintCache := initHintCache(t)
rpcCfg := h.RPCConfig()
notifier, err := New(&rpcCfg, hintCache, hintCache)
notifier, err := New(&rpcCfg, chainntnfs.NetParams, hintCache, hintCache)
if err != nil {
t.Fatalf("unable to create notifier: %v", err)
}
@ -64,8 +79,13 @@ func TestHistoricalConfDetailsTxIndex(t *testing.T) {
// A transaction unknown to the node should not be found within the
// txindex even if it is enabled, so we should not proceed with any
// fallback methods.
var zeroHash chainhash.Hash
_, txStatus, err := notifier.historicalConfDetails(&zeroHash, 0, 0)
var unknownHash chainhash.Hash
copy(unknownHash[:], bytes.Repeat([]byte{0x10}, 32))
unknownConfReq, err := chainntnfs.NewConfRequest(&unknownHash, testScript)
if err != nil {
t.Fatalf("unable to create conf request: %v", err)
}
_, txStatus, err := notifier.historicalConfDetails(unknownConfReq, 0, 0)
if err != nil {
t.Fatalf("unable to retrieve historical conf details: %v", err)
}
@ -80,16 +100,20 @@ func TestHistoricalConfDetailsTxIndex(t *testing.T) {
// Now, we'll create a test transaction and attempt to retrieve its
// confirmation details.
txid, _, err := chainntnfs.GetTestTxidAndScript(harness)
txid, pkScript, err := chainntnfs.GetTestTxidAndScript(harness)
if err != nil {
t.Fatalf("unable to create tx: %v", err)
}
if err := chainntnfs.WaitForMempoolTx(harness, txid); err != nil {
t.Fatalf("unable to find tx in the mempool: %v", err)
}
confReq, err := chainntnfs.NewConfRequest(txid, pkScript)
if err != nil {
t.Fatalf("unable to create conf request: %v", err)
}
// The transaction should be found in the mempool at this point.
_, txStatus, err = notifier.historicalConfDetails(txid, 0, 0)
_, txStatus, err = notifier.historicalConfDetails(confReq, 0, 0)
if err != nil {
t.Fatalf("unable to retrieve historical conf details: %v", err)
}
@ -109,7 +133,7 @@ func TestHistoricalConfDetailsTxIndex(t *testing.T) {
t.Fatalf("unable to generate block: %v", err)
}
_, txStatus, err = notifier.historicalConfDetails(txid, 0, 0)
_, txStatus, err = notifier.historicalConfDetails(confReq, 0, 0)
if err != nil {
t.Fatalf("unable to retrieve historical conf details: %v", err)
}
@ -139,8 +163,13 @@ func TestHistoricalConfDetailsNoTxIndex(t *testing.T) {
// Since the node has its txindex disabled, we fall back to scanning the
// chain manually. A transaction unknown to the network should not be
// found.
var zeroHash chainhash.Hash
_, txStatus, err := notifier.historicalConfDetails(&zeroHash, 0, 0)
var unknownHash chainhash.Hash
copy(unknownHash[:], bytes.Repeat([]byte{0x10}, 32))
unknownConfReq, err := chainntnfs.NewConfRequest(&unknownHash, testScript)
if err != nil {
t.Fatalf("unable to create conf request: %v", err)
}
_, txStatus, err := notifier.historicalConfDetails(unknownConfReq, 0, 0)
if err != nil {
t.Fatalf("unable to retrieve historical conf details: %v", err)
}
@ -161,15 +190,19 @@ func TestHistoricalConfDetailsNoTxIndex(t *testing.T) {
t.Fatalf("unable to retrieve current height: %v", err)
}
txid, _, err := chainntnfs.GetTestTxidAndScript(harness)
txid, pkScript, err := chainntnfs.GetTestTxidAndScript(harness)
if err != nil {
t.Fatalf("unable to create tx: %v", err)
}
if err := chainntnfs.WaitForMempoolTx(harness, txid); err != nil {
t.Fatalf("unable to find tx in the mempool: %v", err)
}
confReq, err := chainntnfs.NewConfRequest(txid, pkScript)
if err != nil {
t.Fatalf("unable to create conf request: %v", err)
}
_, txStatus, err = notifier.historicalConfDetails(txid, 0, 0)
_, txStatus, err = notifier.historicalConfDetails(confReq, 0, 0)
if err != nil {
t.Fatalf("unable to retrieve historical conf details: %v", err)
}
@ -188,7 +221,7 @@ func TestHistoricalConfDetailsNoTxIndex(t *testing.T) {
}
_, txStatus, err = notifier.historicalConfDetails(
txid, uint32(currentHeight), uint32(currentHeight)+1,
confReq, uint32(currentHeight), uint32(currentHeight)+1,
)
if err != nil {
t.Fatalf("unable to retrieve historical conf details: %v", err)

@ -4,6 +4,7 @@ import (
"errors"
"fmt"
"github.com/btcsuite/btcd/chaincfg"
"github.com/btcsuite/btcd/rpcclient"
"github.com/lightningnetwork/lnd/chainntnfs"
)
@ -11,30 +12,36 @@ import (
// createNewNotifier creates a new instance of the ChainNotifier interface
// implemented by BtcdNotifier.
func createNewNotifier(args ...interface{}) (chainntnfs.ChainNotifier, error) {
if len(args) != 3 {
if len(args) != 4 {
return nil, fmt.Errorf("incorrect number of arguments to "+
".New(...), expected 2, instead passed %v", len(args))
".New(...), expected 4, instead passed %v", len(args))
}
config, ok := args[0].(*rpcclient.ConnConfig)
if !ok {
return nil, errors.New("first argument to btcdnotifier.New " +
return nil, errors.New("first argument to btcdnotify.New " +
"is incorrect, expected a *rpcclient.ConnConfig")
}
spendHintCache, ok := args[1].(chainntnfs.SpendHintCache)
chainParams, ok := args[1].(*chaincfg.Params)
if !ok {
return nil, errors.New("second argument to btcdnotifier.New " +
return nil, errors.New("second argument to btcdnotify.New " +
"is incorrect, expected a *chaincfg.Params")
}
spendHintCache, ok := args[2].(chainntnfs.SpendHintCache)
if !ok {
return nil, errors.New("third argument to btcdnotify.New " +
"is incorrect, expected a chainntnfs.SpendHintCache")
}
confirmHintCache, ok := args[2].(chainntnfs.ConfirmHintCache)
confirmHintCache, ok := args[3].(chainntnfs.ConfirmHintCache)
if !ok {
return nil, errors.New("third argument to btcdnotifier.New " +
return nil, errors.New("fourth argument to btcdnotify.New " +
"is incorrect, expected a chainntnfs.ConfirmHintCache")
}
return New(config, spendHintCache, confirmHintCache)
return New(config, chainParams, spendHintCache, confirmHintCache)
}
// init registers a driver for the BtcdNotifier concrete implementation of the

@ -4,8 +4,6 @@ import (
"bytes"
"errors"
"github.com/btcsuite/btcd/chaincfg/chainhash"
"github.com/btcsuite/btcd/wire"
bolt "github.com/coreos/bbolt"
"github.com/lightningnetwork/lnd/channeldb"
)
@ -51,16 +49,16 @@ var (
// which an outpoint could have been spent within.
type SpendHintCache interface {
// CommitSpendHint commits a spend hint for the outpoints to the cache.
CommitSpendHint(height uint32, ops ...wire.OutPoint) error
CommitSpendHint(height uint32, spendRequests ...SpendRequest) error
// QuerySpendHint returns the latest spend hint for an outpoint.
// ErrSpendHintNotFound is returned if a spend hint does not exist
// within the cache for the outpoint.
QuerySpendHint(op wire.OutPoint) (uint32, error)
QuerySpendHint(spendRequest SpendRequest) (uint32, error)
// PurgeSpendHint removes the spend hint for the outpoints from the
// cache.
PurgeSpendHint(ops ...wire.OutPoint) error
PurgeSpendHint(spendRequests ...SpendRequest) error
}
// ConfirmHintCache is an interface whose duty is to cache confirm hints for
@ -69,16 +67,16 @@ type SpendHintCache interface {
type ConfirmHintCache interface {
// CommitConfirmHint commits a confirm hint for the transactions to the
// cache.
CommitConfirmHint(height uint32, txids ...chainhash.Hash) error
CommitConfirmHint(height uint32, confRequests ...ConfRequest) error
// QueryConfirmHint returns the latest confirm hint for a transaction
// hash. ErrConfirmHintNotFound is returned if a confirm hint does not
// exist within the cache for the transaction hash.
QueryConfirmHint(txid chainhash.Hash) (uint32, error)
QueryConfirmHint(confRequest ConfRequest) (uint32, error)
// PurgeConfirmHint removes the confirm hint for the transactions from
// the cache.
PurgeConfirmHint(txids ...chainhash.Hash) error
PurgeConfirmHint(confRequests ...ConfRequest) error
}
// HeightHintCache is an implementation of the SpendHintCache and
@ -118,12 +116,15 @@ func (c *HeightHintCache) initBuckets() error {
}
// CommitSpendHint commits a spend hint for the outpoints to the cache.
func (c *HeightHintCache) CommitSpendHint(height uint32, ops ...wire.OutPoint) error {
if len(ops) == 0 {
func (c *HeightHintCache) CommitSpendHint(height uint32,
spendRequests ...SpendRequest) error {
if len(spendRequests) == 0 {
return nil
}
Log.Tracef("Updating spend hint to height %d for %v", height, ops)
Log.Tracef("Updating spend hint to height %d for %v", height,
spendRequests)
return c.db.Batch(func(tx *bolt.Tx) error {
spendHints := tx.Bucket(spendHintBucket)
@ -136,14 +137,12 @@ func (c *HeightHintCache) CommitSpendHint(height uint32, ops ...wire.OutPoint) e
return err
}
for _, op := range ops {
var outpoint bytes.Buffer
err := channeldb.WriteElement(&outpoint, op)
for _, spendRequest := range spendRequests {
spendHintKey, err := spendRequest.SpendHintKey()
if err != nil {
return err
}
err = spendHints.Put(outpoint.Bytes(), hint.Bytes())
err = spendHints.Put(spendHintKey, hint.Bytes())
if err != nil {
return err
}
@ -156,7 +155,7 @@ func (c *HeightHintCache) CommitSpendHint(height uint32, ops ...wire.OutPoint) e
// QuerySpendHint returns the latest spend hint for an outpoint.
// ErrSpendHintNotFound is returned if a spend hint does not exist within the
// cache for the outpoint.
func (c *HeightHintCache) QuerySpendHint(op wire.OutPoint) (uint32, error) {
func (c *HeightHintCache) QuerySpendHint(spendRequest SpendRequest) (uint32, error) {
var hint uint32
err := c.db.View(func(tx *bolt.Tx) error {
spendHints := tx.Bucket(spendHintBucket)
@ -164,12 +163,11 @@ func (c *HeightHintCache) QuerySpendHint(op wire.OutPoint) (uint32, error) {
return ErrCorruptedHeightHintCache
}
var outpoint bytes.Buffer
if err := channeldb.WriteElement(&outpoint, op); err != nil {
spendHintKey, err := spendRequest.SpendHintKey()
if err != nil {
return err
}
spendHint := spendHints.Get(outpoint.Bytes())
spendHint := spendHints.Get(spendHintKey)
if spendHint == nil {
return ErrSpendHintNotFound
}
@ -184,12 +182,12 @@ func (c *HeightHintCache) QuerySpendHint(op wire.OutPoint) (uint32, error) {
}
// PurgeSpendHint removes the spend hint for the outpoints from the cache.
func (c *HeightHintCache) PurgeSpendHint(ops ...wire.OutPoint) error {
if len(ops) == 0 {
func (c *HeightHintCache) PurgeSpendHint(spendRequests ...SpendRequest) error {
if len(spendRequests) == 0 {
return nil
}
Log.Tracef("Removing spend hints for %v", ops)
Log.Tracef("Removing spend hints for %v", spendRequests)
return c.db.Batch(func(tx *bolt.Tx) error {
spendHints := tx.Bucket(spendHintBucket)
@ -197,15 +195,12 @@ func (c *HeightHintCache) PurgeSpendHint(ops ...wire.OutPoint) error {
return ErrCorruptedHeightHintCache
}
for _, op := range ops {
var outpoint bytes.Buffer
err := channeldb.WriteElement(&outpoint, op)
for _, spendRequest := range spendRequests {
spendHintKey, err := spendRequest.SpendHintKey()
if err != nil {
return err
}
err = spendHints.Delete(outpoint.Bytes())
if err != nil {
if err := spendHints.Delete(spendHintKey); err != nil {
return err
}
}
@ -215,12 +210,15 @@ func (c *HeightHintCache) PurgeSpendHint(ops ...wire.OutPoint) error {
}
// CommitConfirmHint commits a confirm hint for the transactions to the cache.
func (c *HeightHintCache) CommitConfirmHint(height uint32, txids ...chainhash.Hash) error {
if len(txids) == 0 {
func (c *HeightHintCache) CommitConfirmHint(height uint32,
confRequests ...ConfRequest) error {
if len(confRequests) == 0 {
return nil
}
Log.Tracef("Updating confirm hints to height %d for %v", height, txids)
Log.Tracef("Updating confirm hints to height %d for %v", height,
confRequests)
return c.db.Batch(func(tx *bolt.Tx) error {
confirmHints := tx.Bucket(confirmHintBucket)
@ -233,14 +231,12 @@ func (c *HeightHintCache) CommitConfirmHint(height uint32, txids ...chainhash.Ha
return err
}
for _, txid := range txids {
var txHash bytes.Buffer
err := channeldb.WriteElement(&txHash, txid)
for _, confRequest := range confRequests {
confHintKey, err := confRequest.ConfHintKey()
if err != nil {
return err
}
err = confirmHints.Put(txHash.Bytes(), hint.Bytes())
err = confirmHints.Put(confHintKey, hint.Bytes())
if err != nil {
return err
}
@ -253,7 +249,7 @@ func (c *HeightHintCache) CommitConfirmHint(height uint32, txids ...chainhash.Ha
// QueryConfirmHint returns the latest confirm hint for a transaction hash.
// ErrConfirmHintNotFound is returned if a confirm hint does not exist within
// the cache for the transaction hash.
func (c *HeightHintCache) QueryConfirmHint(txid chainhash.Hash) (uint32, error) {
func (c *HeightHintCache) QueryConfirmHint(confRequest ConfRequest) (uint32, error) {
var hint uint32
err := c.db.View(func(tx *bolt.Tx) error {
confirmHints := tx.Bucket(confirmHintBucket)
@ -261,12 +257,11 @@ func (c *HeightHintCache) QueryConfirmHint(txid chainhash.Hash) (uint32, error)
return ErrCorruptedHeightHintCache
}
var txHash bytes.Buffer
if err := channeldb.WriteElement(&txHash, txid); err != nil {
confHintKey, err := confRequest.ConfHintKey()
if err != nil {
return err
}
confirmHint := confirmHints.Get(txHash.Bytes())
confirmHint := confirmHints.Get(confHintKey)
if confirmHint == nil {
return ErrConfirmHintNotFound
}
@ -282,12 +277,12 @@ func (c *HeightHintCache) QueryConfirmHint(txid chainhash.Hash) (uint32, error)
// PurgeConfirmHint removes the confirm hint for the transactions from the
// cache.
func (c *HeightHintCache) PurgeConfirmHint(txids ...chainhash.Hash) error {
if len(txids) == 0 {
func (c *HeightHintCache) PurgeConfirmHint(confRequests ...ConfRequest) error {
if len(confRequests) == 0 {
return nil
}
Log.Tracef("Removing confirm hints for %v", txids)
Log.Tracef("Removing confirm hints for %v", confRequests)
return c.db.Batch(func(tx *bolt.Tx) error {
confirmHints := tx.Bucket(confirmHintBucket)
@ -295,15 +290,12 @@ func (c *HeightHintCache) PurgeConfirmHint(txids ...chainhash.Hash) error {
return ErrCorruptedHeightHintCache
}
for _, txid := range txids {
var txHash bytes.Buffer
err := channeldb.WriteElement(&txHash, txid)
for _, confRequest := range confRequests {
confHintKey, err := confRequest.ConfHintKey()
if err != nil {
return err
}
err = confirmHints.Delete(txHash.Bytes())
if err != nil {
if err := confirmHints.Delete(confHintKey); err != nil {
return err
}
}

@ -39,7 +39,9 @@ func TestHeightHintCacheConfirms(t *testing.T) {
// Querying for a transaction hash not found within the cache should
// return an error indication so.
var unknownHash chainhash.Hash
_, err := hintCache.QueryConfirmHint(unknownHash)
copy(unknownHash[:], bytes.Repeat([]byte{0x01}, 32))
unknownConfRequest := ConfRequest{TxID: unknownHash}
_, err := hintCache.QueryConfirmHint(unknownConfRequest)
if err != ErrConfirmHintNotFound {
t.Fatalf("expected ErrConfirmHintNotFound, got: %v", err)
}
@ -48,23 +50,24 @@ func TestHeightHintCacheConfirms(t *testing.T) {
// cache with the same confirm hint.
const height = 100
const numHashes = 5
txHashes := make([]chainhash.Hash, numHashes)
confRequests := make([]ConfRequest, numHashes)
for i := 0; i < numHashes; i++ {
var txHash chainhash.Hash
copy(txHash[:], bytes.Repeat([]byte{byte(i)}, 32))
txHashes[i] = txHash
copy(txHash[:], bytes.Repeat([]byte{byte(i + 1)}, 32))
confRequests[i] = ConfRequest{TxID: txHash}
}
if err := hintCache.CommitConfirmHint(height, txHashes...); err != nil {
err = hintCache.CommitConfirmHint(height, confRequests...)
if err != nil {
t.Fatalf("unable to add entries to cache: %v", err)
}
// With the hashes committed, we'll now query the cache to ensure that
// we're able to properly retrieve the confirm hints.
for _, txHash := range txHashes {
confirmHint, err := hintCache.QueryConfirmHint(txHash)
for _, confRequest := range confRequests {
confirmHint, err := hintCache.QueryConfirmHint(confRequest)
if err != nil {
t.Fatalf("unable to query for hint: %v", err)
t.Fatalf("unable to query for hint of %v: %v", confRequest, err)
}
if confirmHint != height {
t.Fatalf("expected confirm hint %d, got %d", height,
@ -74,14 +77,14 @@ func TestHeightHintCacheConfirms(t *testing.T) {
// We'll also attempt to purge all of them in a single database
// transaction.
if err := hintCache.PurgeConfirmHint(txHashes...); err != nil {
if err := hintCache.PurgeConfirmHint(confRequests...); err != nil {
t.Fatalf("unable to remove confirm hints: %v", err)
}
// Finally, we'll attempt to query for each hash. We should expect not
// to find a hint for any of them.
for _, txHash := range txHashes {
_, err := hintCache.QueryConfirmHint(txHash)
for _, confRequest := range confRequests {
_, err := hintCache.QueryConfirmHint(confRequest)
if err != ErrConfirmHintNotFound {
t.Fatalf("expected ErrConfirmHintNotFound, got :%v", err)
}
@ -97,8 +100,9 @@ func TestHeightHintCacheSpends(t *testing.T) {
// Querying for an outpoint not found within the cache should return an
// error indication so.
var unknownOutPoint wire.OutPoint
_, err := hintCache.QuerySpendHint(unknownOutPoint)
unknownOutPoint := wire.OutPoint{Index: 1}
unknownSpendRequest := SpendRequest{OutPoint: unknownOutPoint}
_, err := hintCache.QuerySpendHint(unknownSpendRequest)
if err != ErrSpendHintNotFound {
t.Fatalf("expected ErrSpendHintNotFound, got: %v", err)
}
@ -107,21 +111,22 @@ func TestHeightHintCacheSpends(t *testing.T) {
// the same spend hint.
const height = 100
const numOutpoints = 5
var txHash chainhash.Hash
copy(txHash[:], bytes.Repeat([]byte{0xFF}, 32))
outpoints := make([]wire.OutPoint, numOutpoints)
spendRequests := make([]SpendRequest, numOutpoints)
for i := uint32(0); i < numOutpoints; i++ {
outpoints[i] = wire.OutPoint{Hash: txHash, Index: i}
spendRequests[i] = SpendRequest{
OutPoint: wire.OutPoint{Index: i + 1},
}
}
if err := hintCache.CommitSpendHint(height, outpoints...); err != nil {
t.Fatalf("unable to add entry to cache: %v", err)
err = hintCache.CommitSpendHint(height, spendRequests...)
if err != nil {
t.Fatalf("unable to add entries to cache: %v", err)
}
// With the outpoints committed, we'll now query the cache to ensure
// that we're able to properly retrieve the confirm hints.
for _, op := range outpoints {
spendHint, err := hintCache.QuerySpendHint(op)
for _, spendRequest := range spendRequests {
spendHint, err := hintCache.QuerySpendHint(spendRequest)
if err != nil {
t.Fatalf("unable to query for hint: %v", err)
}
@ -133,14 +138,14 @@ func TestHeightHintCacheSpends(t *testing.T) {
// We'll also attempt to purge all of them in a single database
// transaction.
if err := hintCache.PurgeSpendHint(outpoints...); err != nil {
if err := hintCache.PurgeSpendHint(spendRequests...); err != nil {
t.Fatalf("unable to remove spend hint: %v", err)
}
// Finally, we'll attempt to query for each outpoint. We should expect
// not to find a hint for any of them.
for _, op := range outpoints {
_, err = hintCache.QuerySpendHint(op)
for _, spendRequest := range spendRequests {
_, err = hintCache.QuerySpendHint(spendRequest)
if err != ErrSpendHintNotFound {
t.Fatalf("expected ErrSpendHintNotFound, got: %v", err)
}

@ -1,6 +1,7 @@
package chainntnfs
import (
"errors"
"fmt"
"sync"
@ -9,6 +10,12 @@ import (
"github.com/btcsuite/btcd/wire"
)
var (
// ErrChainNotifierShuttingDown is used when we are trying to
// measure a spend notification when notifier is already stopped.
ErrChainNotifierShuttingDown = errors.New("chain notifier shutting down")
)
// TxConfStatus denotes the status of a transaction's lookup.
type TxConfStatus uint8
@ -65,36 +72,44 @@ func (t TxConfStatus) String() string {
//
// Concrete implementations of ChainNotifier should be able to support multiple
// concurrent client requests, as well as multiple concurrent notification events.
// TODO(roasbeef): all events should have a Cancel() method to free up the
// resource
type ChainNotifier interface {
// RegisterConfirmationsNtfn registers an intent to be notified once
// txid reaches numConfs confirmations. We also pass in the pkScript as
// the default light client instead needs to match on scripts created
// in the block. The returned ConfirmationEvent should properly notify
// the client once the specified number of confirmations has been
// reached for the txid, as well as if the original tx gets re-org'd
// out of the mainchain. The heightHint parameter is provided as a
// convenience to light clients. The heightHint denotes the earliest
// height in the blockchain in which the target txid _could_ have been
// included in the chain. This can be used to bound the search space
// when checking to see if a notification can immediately be dispatched
// due to historical data.
// the default light client instead needs to match on scripts created in
// the block. If a nil txid is passed in, then not only should we match
// on the script, but we should also dispatch once the transaction
// containing the script reaches numConfs confirmations. This can be
// useful in instances where we only know the script in advance, but not
// the transaction containing it.
//
// The returned ConfirmationEvent should properly notify the client once
// the specified number of confirmations has been reached for the txid,
// as well as if the original tx gets re-org'd out of the mainchain. The
// heightHint parameter is provided as a convenience to light clients.
// It heightHint denotes the earliest height in the blockchain in which
// the target txid _could_ have been included in the chain. This can be
// used to bound the search space when checking to see if a notification
// can immediately be dispatched due to historical data.
//
// NOTE: Dispatching notifications to multiple clients subscribed to
// the same (txid, numConfs) tuple MUST be supported.
RegisterConfirmationsNtfn(txid *chainhash.Hash, pkScript []byte, numConfs,
heightHint uint32) (*ConfirmationEvent, error)
RegisterConfirmationsNtfn(txid *chainhash.Hash, pkScript []byte,
numConfs, heightHint uint32) (*ConfirmationEvent, error)
// RegisterSpendNtfn registers an intent to be notified once the target
// outpoint is successfully spent within a transaction. The script that
// the outpoint creates must also be specified. This allows this
// interface to be implemented by BIP 158-like filtering. The returned
// SpendEvent will receive a send on the 'Spend' transaction once a
// transaction spending the input is detected on the blockchain. The
// heightHint parameter is provided as a convenience to light clients.
// The heightHint denotes the earliest height in the blockchain in
// which the target output could have been created.
// interface to be implemented by BIP 158-like filtering. If a nil
// outpoint is passed in, then not only should we match on the script,
// but we should also dispatch once a transaction spends the output
// containing said script. This can be useful in instances where we only
// know the script in advance, but not the outpoint itself.
//
// The returned SpendEvent will receive a send on the 'Spend'
// transaction once a transaction spending the input is detected on the
// blockchain. The heightHint parameter is provided as a convenience to
// light clients. It denotes the earliest height in the blockchain in
// which the target output could have been spent.
//
// NOTE: The notification should only be triggered when the spending
// transaction receives a single confirmation.
@ -112,7 +127,9 @@ type ChainNotifier interface {
// Clients have the option of passing in their best known block.
// If they specify a block, the ChainNotifier checks whether the client
// is behind on blocks. If they are, the ChainNotifier sends a backlog
// of block notifications for the missed blocks.
// of block notifications for the missed blocks. If they do not provide
// one, then a notification will be dispatched immediately for the
// current tip of the chain upon a successful registration.
RegisterBlockEpochNtfn(*BlockEpoch) (*BlockEpochEvent, error)
// Start the ChainNotifier. Once started, the implementation should be
@ -140,6 +157,9 @@ type TxConfirmation struct {
// TxIndex is the index within the block of the ultimate confirmed
// transaction.
TxIndex uint32
// Tx is the transaction for which the notification was requested for.
Tx *wire.MsgTx
}
// ConfirmationEvent encapsulates a confirmation notification. With this struct,
@ -155,6 +175,9 @@ type TxConfirmation struct {
// If the event that the original transaction becomes re-org'd out of the main
// chain, the 'NegativeConf' will be sent upon with a value representing the
// depth of the re-org.
//
// NOTE: If the caller wishes to cancel their registered spend notification,
// the Cancel closure MUST be called.
type ConfirmationEvent struct {
// Confirmed is a channel that will be sent upon once the transaction
// has been fully confirmed. The struct sent will contain all the
@ -171,26 +194,34 @@ type ConfirmationEvent struct {
// confirmations.
Updates chan uint32
// TODO(roasbeef): all goroutines on ln channel updates should also
// have a struct chan that's closed if funding gets re-org out. Need
// to sync, to request another confirmation event ntfn, then re-open
// channel after confs.
// NegativeConf is a channel that will be sent upon if the transaction
// confirms, but is later reorged out of the chain. The integer sent
// through the channel represents the reorg depth.
//
// NOTE: This channel must be buffered.
NegativeConf chan int32
// Done is a channel that gets sent upon once the confirmation request
// is no longer under the risk of being reorged out of the chain.
//
// NOTE: This channel must be buffered.
Done chan struct{}
// Cancel is a closure that should be executed by the caller in the case
// that they wish to prematurely abandon their registered confirmation
// notification.
Cancel func()
}
// NewConfirmationEvent constructs a new ConfirmationEvent with newly opened
// channels.
func NewConfirmationEvent(numConfs uint32) *ConfirmationEvent {
func NewConfirmationEvent(numConfs uint32, cancel func()) *ConfirmationEvent {
return &ConfirmationEvent{
Confirmed: make(chan *TxConfirmation, 1),
Updates: make(chan uint32, numConfs),
NegativeConf: make(chan int32, 1),
Done: make(chan struct{}, 1),
Cancel: cancel,
}
}
@ -227,8 +258,14 @@ type SpendEvent struct {
// NOTE: This channel must be buffered.
Reorg chan struct{}
// Cancel is a closure that should be executed by the caller in the
// case that they wish to prematurely abandon their registered spend
// Done is a channel that gets sent upon once the confirmation request
// is no longer under the risk of being reorged out of the chain.
//
// NOTE: This channel must be buffered.
Done chan struct{}
// Cancel is a closure that should be executed by the caller in the case
// that they wish to prematurely abandon their registered spend
// notification.
Cancel func()
}
@ -238,6 +275,7 @@ func NewSpendEvent(cancel func()) *SpendEvent {
return &SpendEvent{
Spend: make(chan *SpendDetail, 1),
Reorg: make(chan struct{}, 1),
Done: make(chan struct{}, 1),
Cancel: cancel,
}
}
@ -267,8 +305,8 @@ type BlockEpochEvent struct {
// NOTE: This channel must be buffered.
Epochs <-chan *BlockEpoch
// Cancel is a closure that should be executed by the caller in the
// case that they wish to abandon their registered spend notification.
// Cancel is a closure that should be executed by the caller in the case
// that they wish to abandon their registered block epochs notification.
Cancel func()
}

@ -16,29 +16,18 @@ import (
"github.com/btcsuite/btcd/rpcclient"
"github.com/btcsuite/btcd/wire"
"github.com/btcsuite/btcutil"
"github.com/btcsuite/btcwallet/chain"
_ "github.com/btcsuite/btcwallet/walletdb/bdb" // Required to auto-register the boltdb walletdb implementation.
"github.com/lightninglabs/neutrino"
"github.com/lightningnetwork/lnd/chainntnfs"
"github.com/lightningnetwork/lnd/channeldb"
// Required to auto-register the bitcoind backed ChainNotifier
// implementation.
"github.com/lightningnetwork/lnd/chainntnfs/bitcoindnotify"
// Required to auto-register the btcd backed ChainNotifier
// implementation.
"github.com/lightningnetwork/lnd/chainntnfs/btcdnotify"
// Required to auto-register the neutrino backed ChainNotifier
// implementation.
"github.com/lightningnetwork/lnd/chainntnfs/neutrinonotify"
// Required to register the boltdb walletdb implementation.
"github.com/btcsuite/btcwallet/chain"
_ "github.com/btcsuite/btcwallet/walletdb/bdb"
"github.com/lightningnetwork/lnd/channeldb"
)
func testSingleConfirmationNotification(miner *rpctest.Harness,
notifier chainntnfs.TestChainNotifier, t *testing.T) {
notifier chainntnfs.TestChainNotifier, scriptDispatch bool, t *testing.T) {
// We'd like to test the case of being notified once a txid reaches
// a *single* confirmation.
@ -62,9 +51,16 @@ func testSingleConfirmationNotification(miner *rpctest.Harness,
// Now that we have a txid, register a confirmation notification with
// the chainntfn source.
numConfs := uint32(1)
confIntent, err := notifier.RegisterConfirmationsNtfn(
txid, pkScript, numConfs, uint32(currentHeight),
)
var confIntent *chainntnfs.ConfirmationEvent
if scriptDispatch {
confIntent, err = notifier.RegisterConfirmationsNtfn(
nil, pkScript, numConfs, uint32(currentHeight),
)
} else {
confIntent, err = notifier.RegisterConfirmationsNtfn(
txid, pkScript, numConfs, uint32(currentHeight),
)
}
if err != nil {
t.Fatalf("unable to register ntfn: %v", err)
}
@ -106,7 +102,7 @@ func testSingleConfirmationNotification(miner *rpctest.Harness,
}
func testMultiConfirmationNotification(miner *rpctest.Harness,
notifier chainntnfs.TestChainNotifier, t *testing.T) {
notifier chainntnfs.TestChainNotifier, scriptDispatch bool, t *testing.T) {
// We'd like to test the case of being notified once a txid reaches
// N confirmations, where N > 1.
@ -127,9 +123,16 @@ func testMultiConfirmationNotification(miner *rpctest.Harness,
}
numConfs := uint32(6)
confIntent, err := notifier.RegisterConfirmationsNtfn(
txid, pkScript, numConfs, uint32(currentHeight),
)
var confIntent *chainntnfs.ConfirmationEvent
if scriptDispatch {
confIntent, err = notifier.RegisterConfirmationsNtfn(
nil, pkScript, numConfs, uint32(currentHeight),
)
} else {
confIntent, err = notifier.RegisterConfirmationsNtfn(
txid, pkScript, numConfs, uint32(currentHeight),
)
}
if err != nil {
t.Fatalf("unable to register ntfn: %v", err)
}
@ -152,7 +155,7 @@ func testMultiConfirmationNotification(miner *rpctest.Harness,
}
func testBatchConfirmationNotification(miner *rpctest.Harness,
notifier chainntnfs.TestChainNotifier, t *testing.T) {
notifier chainntnfs.TestChainNotifier, scriptDispatch bool, t *testing.T) {
// We'd like to test a case of serving notifications to multiple
// clients, each requesting to be notified once a txid receives
@ -174,9 +177,16 @@ func testBatchConfirmationNotification(miner *rpctest.Harness,
if err != nil {
t.Fatalf("unable to create test addr: %v", err)
}
confIntent, err := notifier.RegisterConfirmationsNtfn(
txid, pkScript, numConfs, uint32(currentHeight),
)
var confIntent *chainntnfs.ConfirmationEvent
if scriptDispatch {
confIntent, err = notifier.RegisterConfirmationsNtfn(
nil, pkScript, numConfs, uint32(currentHeight),
)
} else {
confIntent, err = notifier.RegisterConfirmationsNtfn(
txid, pkScript, numConfs, uint32(currentHeight),
)
}
if err != nil {
t.Fatalf("unable to register ntfn: %v", err)
}
@ -257,13 +267,13 @@ func checkNotificationFields(ntfn *chainntnfs.SpendDetail,
}
func testSpendNotification(miner *rpctest.Harness,
notifier chainntnfs.TestChainNotifier, t *testing.T) {
notifier chainntnfs.TestChainNotifier, scriptDispatch bool, t *testing.T) {
// We'd like to test the spend notifications for all ChainNotifier
// concrete implementations.
//
// To do so, we first create a new output to our test target address.
outpoint, pkScript := chainntnfs.CreateSpendableOutput(t, miner)
outpoint, output, privKey := chainntnfs.CreateSpendableOutput(t, miner)
_, currentHeight, err := miner.Node.GetBestBlock()
if err != nil {
@ -277,9 +287,16 @@ func testSpendNotification(miner *rpctest.Harness,
const numClients = 5
spendClients := make([]*chainntnfs.SpendEvent, numClients)
for i := 0; i < numClients; i++ {
spentIntent, err := notifier.RegisterSpendNtfn(
outpoint, pkScript, uint32(currentHeight),
)
var spentIntent *chainntnfs.SpendEvent
if scriptDispatch {
spentIntent, err = notifier.RegisterSpendNtfn(
nil, output.PkScript, uint32(currentHeight),
)
} else {
spentIntent, err = notifier.RegisterSpendNtfn(
outpoint, output.PkScript, uint32(currentHeight),
)
}
if err != nil {
t.Fatalf("unable to register for spend ntfn: %v", err)
}
@ -288,7 +305,7 @@ func testSpendNotification(miner *rpctest.Harness,
}
// Next, create a new transaction spending that output.
spendingTx := chainntnfs.CreateSpendTx(t, outpoint, pkScript)
spendingTx := chainntnfs.CreateSpendTx(t, outpoint, output, privKey)
// Broadcast our spending transaction.
spenderSha, err := miner.Node.SendRawTransaction(spendingTx, true)
@ -328,9 +345,16 @@ func testSpendNotification(miner *rpctest.Harness,
// Make sure registering a client after the tx is in the mempool still
// doesn't trigger a notification.
spentIntent, err := notifier.RegisterSpendNtfn(
outpoint, pkScript, uint32(currentHeight),
)
var spentIntent *chainntnfs.SpendEvent
if scriptDispatch {
spentIntent, err = notifier.RegisterSpendNtfn(
nil, output.PkScript, uint32(currentHeight),
)
} else {
spentIntent, err = notifier.RegisterSpendNtfn(
outpoint, output.PkScript, uint32(currentHeight),
)
}
if err != nil {
t.Fatalf("unable to register for spend ntfn: %v", err)
}
@ -374,22 +398,23 @@ func testBlockEpochNotification(miner *rpctest.Harness,
// block epoch notifications.
const numBlocks = 10
const numNtfns = numBlocks + 1
const numClients = 5
var wg sync.WaitGroup
// Create numClients clients which will listen for block notifications. We
// expect each client to receive 10 notifications for each of the ten
// blocks we generate below. So we'll use a WaitGroup to synchronize the
// test.
// expect each client to receive 11 notifications, one for the current
// tip of the chain, and one for each of the ten blocks we generate
// below. So we'll use a WaitGroup to synchronize the test.
for i := 0; i < numClients; i++ {
epochClient, err := notifier.RegisterBlockEpochNtfn(nil)
if err != nil {
t.Fatalf("unable to register for epoch notification")
}
wg.Add(numBlocks)
wg.Add(numNtfns)
go func() {
for i := 0; i < numBlocks; i++ {
for i := 0; i < numNtfns; i++ {
<-epochClient.Epochs
wg.Done()
}
@ -416,7 +441,7 @@ func testBlockEpochNotification(miner *rpctest.Harness,
}
func testMultiClientConfirmationNotification(miner *rpctest.Harness,
notifier chainntnfs.TestChainNotifier, t *testing.T) {
notifier chainntnfs.TestChainNotifier, scriptDispatch bool, t *testing.T) {
// We'd like to test the case of a multiple clients registered to
// receive a confirmation notification for the same transaction.
@ -442,9 +467,16 @@ func testMultiClientConfirmationNotification(miner *rpctest.Harness,
// Register for a conf notification for the above generated txid with
// numConfsClients distinct clients.
for i := 0; i < numConfsClients; i++ {
confClient, err := notifier.RegisterConfirmationsNtfn(
txid, pkScript, numConfs, uint32(currentHeight),
)
var confClient *chainntnfs.ConfirmationEvent
if scriptDispatch {
confClient, err = notifier.RegisterConfirmationsNtfn(
nil, pkScript, numConfs, uint32(currentHeight),
)
} else {
confClient, err = notifier.RegisterConfirmationsNtfn(
txid, pkScript, numConfs, uint32(currentHeight),
)
}
if err != nil {
t.Fatalf("unable to register for confirmation: %v", err)
}
@ -479,7 +511,7 @@ func testMultiClientConfirmationNotification(miner *rpctest.Harness,
// transaction that has already been included in a block. In this case, the
// confirmation notification should be dispatched immediately.
func testTxConfirmedBeforeNtfnRegistration(miner *rpctest.Harness,
notifier chainntnfs.TestChainNotifier, t *testing.T) {
notifier chainntnfs.TestChainNotifier, scriptDispatch bool, t *testing.T) {
// First, let's send some coins to "ourself", obtaining a txid. We're
// spending from a coinbase output here, so we use the dedicated
@ -533,9 +565,16 @@ func testTxConfirmedBeforeNtfnRegistration(miner *rpctest.Harness,
// which is included in the last block. The height hint is the height before
// the block is included. This notification should fire immediately since
// only 1 confirmation is required.
ntfn1, err := notifier.RegisterConfirmationsNtfn(
txid1, pkScript1, 1, uint32(currentHeight),
)
var ntfn1 *chainntnfs.ConfirmationEvent
if scriptDispatch {
ntfn1, err = notifier.RegisterConfirmationsNtfn(
nil, pkScript1, 1, uint32(currentHeight),
)
} else {
ntfn1, err = notifier.RegisterConfirmationsNtfn(
txid1, pkScript1, 1, uint32(currentHeight),
)
}
if err != nil {
t.Fatalf("unable to register ntfn: %v", err)
}
@ -572,9 +611,16 @@ func testTxConfirmedBeforeNtfnRegistration(miner *rpctest.Harness,
// Register a confirmation notification for tx2, requiring 3 confirmations.
// This transaction is only partially confirmed, so the notification should
// not fire yet.
ntfn2, err := notifier.RegisterConfirmationsNtfn(
txid2, pkScript2, 3, uint32(currentHeight),
)
var ntfn2 *chainntnfs.ConfirmationEvent
if scriptDispatch {
ntfn2, err = notifier.RegisterConfirmationsNtfn(
nil, pkScript2, 3, uint32(currentHeight),
)
} else {
ntfn2, err = notifier.RegisterConfirmationsNtfn(
txid2, pkScript2, 3, uint32(currentHeight),
)
}
if err != nil {
t.Fatalf("unable to register ntfn: %v", err)
}
@ -600,9 +646,16 @@ func testTxConfirmedBeforeNtfnRegistration(miner *rpctest.Harness,
// Finally register a confirmation notification for tx3, requiring 1
// confirmation. Ensure that conf notifications do not refire on txs
// 1 or 2.
ntfn3, err := notifier.RegisterConfirmationsNtfn(
txid3, pkScript3, 1, uint32(currentHeight-1),
)
var ntfn3 *chainntnfs.ConfirmationEvent
if scriptDispatch {
ntfn3, err = notifier.RegisterConfirmationsNtfn(
nil, pkScript3, 1, uint32(currentHeight-1),
)
} else {
ntfn3, err = notifier.RegisterConfirmationsNtfn(
txid3, pkScript3, 1, uint32(currentHeight-1),
)
}
if err != nil {
t.Fatalf("unable to register ntfn: %v", err)
}
@ -632,7 +685,7 @@ func testTxConfirmedBeforeNtfnRegistration(miner *rpctest.Harness,
// checking for a confirmation. This should not cause the notifier to stop
// working
func testLazyNtfnConsumer(miner *rpctest.Harness,
notifier chainntnfs.TestChainNotifier, t *testing.T) {
notifier chainntnfs.TestChainNotifier, scriptDispatch bool, t *testing.T) {
// Create a transaction to be notified about. We'll register for
// notifications on this transaction but won't be prompt in checking them
@ -657,9 +710,16 @@ func testLazyNtfnConsumer(miner *rpctest.Harness,
t.Fatalf("unable to generate blocks: %v", err)
}
firstConfIntent, err := notifier.RegisterConfirmationsNtfn(
txid, pkScript, numConfs, uint32(currentHeight),
)
var firstConfIntent *chainntnfs.ConfirmationEvent
if scriptDispatch {
firstConfIntent, err = notifier.RegisterConfirmationsNtfn(
nil, pkScript, numConfs, uint32(currentHeight),
)
} else {
firstConfIntent, err = notifier.RegisterConfirmationsNtfn(
txid, pkScript, numConfs, uint32(currentHeight),
)
}
if err != nil {
t.Fatalf("unable to register ntfn: %v", err)
}
@ -686,10 +746,16 @@ func testLazyNtfnConsumer(miner *rpctest.Harness,
}
numConfs = 1
secondConfIntent, err := notifier.RegisterConfirmationsNtfn(
txid, pkScript, numConfs, uint32(currentHeight),
)
var secondConfIntent *chainntnfs.ConfirmationEvent
if scriptDispatch {
secondConfIntent, err = notifier.RegisterConfirmationsNtfn(
nil, pkScript, numConfs, uint32(currentHeight),
)
} else {
secondConfIntent, err = notifier.RegisterConfirmationsNtfn(
txid, pkScript, numConfs, uint32(currentHeight),
)
}
if err != nil {
t.Fatalf("unable to register ntfn: %v", err)
}
@ -719,16 +785,21 @@ func testLazyNtfnConsumer(miner *rpctest.Harness,
// has already been included in a block. In this case, the spend notification
// should be dispatched immediately.
func testSpendBeforeNtfnRegistration(miner *rpctest.Harness,
notifier chainntnfs.TestChainNotifier, t *testing.T) {
notifier chainntnfs.TestChainNotifier, scriptDispatch bool, t *testing.T) {
// We'd like to test the spend notifications for all ChainNotifier
// concrete implementations.
//
// To do so, we first create a new output to our test target address.
outpoint, pkScript := chainntnfs.CreateSpendableOutput(t, miner)
outpoint, output, privKey := chainntnfs.CreateSpendableOutput(t, miner)
_, heightHint, err := miner.Node.GetBestBlock()
if err != nil {
t.Fatalf("unable to get current height: %v", err)
}
// We'll then spend this output and broadcast the spend transaction.
spendingTx := chainntnfs.CreateSpendTx(t, outpoint, pkScript)
spendingTx := chainntnfs.CreateSpendTx(t, outpoint, output, privKey)
spenderSha, err := miner.Node.SendRawTransaction(spendingTx, true)
if err != nil {
t.Fatalf("unable to broadcast tx: %v", err)
@ -748,8 +819,7 @@ func testSpendBeforeNtfnRegistration(miner *rpctest.Harness,
if _, err := miner.Node.Generate(1); err != nil {
t.Fatalf("unable to generate single block: %v", err)
}
_, currentHeight, err := miner.Node.GetBestBlock()
_, spendHeight, err := miner.Node.GetBestBlock()
if err != nil {
t.Fatalf("unable to get current height: %v", err)
}
@ -763,9 +833,17 @@ func testSpendBeforeNtfnRegistration(miner *rpctest.Harness,
const numClients = 2
spendClients := make([]*chainntnfs.SpendEvent, numClients)
for i := 0; i < numClients; i++ {
spentIntent, err := notifier.RegisterSpendNtfn(
outpoint, pkScript, uint32(currentHeight),
)
var spentIntent *chainntnfs.SpendEvent
if scriptDispatch {
spentIntent, err = notifier.RegisterSpendNtfn(
nil, output.PkScript, uint32(heightHint),
)
} else {
spentIntent, err = notifier.RegisterSpendNtfn(
outpoint, output.PkScript,
uint32(heightHint),
)
}
if err != nil {
t.Fatalf("unable to register for spend ntfn: %v",
err)
@ -779,8 +857,9 @@ func testSpendBeforeNtfnRegistration(miner *rpctest.Harness,
case ntfn := <-client.Spend:
// We've received the spend nftn. So now verify
// all the fields have been set properly.
checkNotificationFields(ntfn, outpoint, spenderSha,
currentHeight, t)
checkNotificationFields(
ntfn, outpoint, spenderSha, spendHeight, t,
)
case <-time.After(30 * time.Second):
t.Fatalf("spend ntfn never received")
}
@ -824,14 +903,14 @@ func testSpendBeforeNtfnRegistration(miner *rpctest.Harness,
}
func testCancelSpendNtfn(node *rpctest.Harness,
notifier chainntnfs.TestChainNotifier, t *testing.T) {
notifier chainntnfs.TestChainNotifier, scriptDispatch bool, t *testing.T) {
// We'd like to test that once a spend notification is registered, it
// can be cancelled before the notification is dispatched.
// First, we'll start by creating a new output that we can spend
// ourselves.
outpoint, pkScript := chainntnfs.CreateSpendableOutput(t, node)
outpoint, output, privKey := chainntnfs.CreateSpendableOutput(t, node)
_, currentHeight, err := node.Node.GetBestBlock()
if err != nil {
@ -844,9 +923,16 @@ func testCancelSpendNtfn(node *rpctest.Harness,
const numClients = 2
spendClients := make([]*chainntnfs.SpendEvent, numClients)
for i := 0; i < numClients; i++ {
spentIntent, err := notifier.RegisterSpendNtfn(
outpoint, pkScript, uint32(currentHeight),
)
var spentIntent *chainntnfs.SpendEvent
if scriptDispatch {
spentIntent, err = notifier.RegisterSpendNtfn(
nil, output.PkScript, uint32(currentHeight),
)
} else {
spentIntent, err = notifier.RegisterSpendNtfn(
outpoint, output.PkScript, uint32(currentHeight),
)
}
if err != nil {
t.Fatalf("unable to register for spend ntfn: %v", err)
}
@ -855,7 +941,7 @@ func testCancelSpendNtfn(node *rpctest.Harness,
}
// Next, create a new transaction spending that output.
spendingTx := chainntnfs.CreateSpendTx(t, outpoint, pkScript)
spendingTx := chainntnfs.CreateSpendTx(t, outpoint, output, privKey)
// Before we broadcast the spending transaction, we'll cancel the
// notification of the first client.
@ -877,8 +963,8 @@ func testCancelSpendNtfn(node *rpctest.Harness,
t.Fatalf("unable to generate single block: %v", err)
}
// However, the spend notification for the first client should have
// been dispatched.
// The spend notification for the first client should have been
// dispatched.
select {
case ntfn := <-spendClients[0].Spend:
// We've received the spend nftn. So now verify all the
@ -902,7 +988,7 @@ func testCancelSpendNtfn(node *rpctest.Harness,
t.Fatalf("spend ntfn never received")
}
// However, The spend notification of the second client should NOT have
// However, the spend notification of the second client should NOT have
// been dispatched.
select {
case _, ok := <-spendClients[1].Spend:
@ -914,8 +1000,8 @@ func testCancelSpendNtfn(node *rpctest.Harness,
}
}
func testCancelEpochNtfn(node *rpctest.Harness, notifier chainntnfs.TestChainNotifier,
t *testing.T) {
func testCancelEpochNtfn(node *rpctest.Harness,
notifier chainntnfs.TestChainNotifier, t *testing.T) {
// We'd like to ensure that once a client cancels their block epoch
// notifications, no further notifications are sent over the channel
@ -964,8 +1050,8 @@ func testCancelEpochNtfn(node *rpctest.Harness, notifier chainntnfs.TestChainNot
}
}
func testReorgConf(miner *rpctest.Harness, notifier chainntnfs.TestChainNotifier,
t *testing.T) {
func testReorgConf(miner *rpctest.Harness,
notifier chainntnfs.TestChainNotifier, scriptDispatch bool, t *testing.T) {
// Set up a new miner that we can use to cause a reorg.
miner2, err := rpctest.New(chainntnfs.NetParams, nil, []string{"--txindex"})
@ -1026,9 +1112,16 @@ func testReorgConf(miner *rpctest.Harness, notifier chainntnfs.TestChainNotifier
// Now that we have a txid, register a confirmation notification with
// the chainntfn source.
numConfs := uint32(2)
confIntent, err := notifier.RegisterConfirmationsNtfn(
txid, pkScript, numConfs, uint32(currentHeight),
)
var confIntent *chainntnfs.ConfirmationEvent
if scriptDispatch {
confIntent, err = notifier.RegisterConfirmationsNtfn(
nil, pkScript, numConfs, uint32(currentHeight),
)
} else {
confIntent, err = notifier.RegisterConfirmationsNtfn(
txid, pkScript, numConfs, uint32(currentHeight),
)
}
if err != nil {
t.Fatalf("unable to register ntfn: %v", err)
}
@ -1116,18 +1209,26 @@ func testReorgConf(miner *rpctest.Harness, notifier chainntnfs.TestChainNotifier
// correctly handle outpoints whose spending transaction has been reorged out of
// the chain.
func testReorgSpend(miner *rpctest.Harness,
notifier chainntnfs.TestChainNotifier, t *testing.T) {
notifier chainntnfs.TestChainNotifier, scriptDispatch bool, t *testing.T) {
// We'll start by creating an output and registering a spend
// notification for it.
outpoint, pkScript := chainntnfs.CreateSpendableOutput(t, miner)
_, currentHeight, err := miner.Node.GetBestBlock()
outpoint, output, privKey := chainntnfs.CreateSpendableOutput(t, miner)
_, heightHint, err := miner.Node.GetBestBlock()
if err != nil {
t.Fatalf("unable to retrieve current height: %v", err)
}
spendIntent, err := notifier.RegisterSpendNtfn(
outpoint, pkScript, uint32(currentHeight),
)
var spendIntent *chainntnfs.SpendEvent
if scriptDispatch {
spendIntent, err = notifier.RegisterSpendNtfn(
nil, output.PkScript, uint32(heightHint),
)
} else {
spendIntent, err = notifier.RegisterSpendNtfn(
outpoint, output.PkScript, uint32(heightHint),
)
}
if err != nil {
t.Fatalf("unable to register for spend: %v", err)
}
@ -1174,7 +1275,7 @@ func testReorgSpend(miner *rpctest.Harness,
// Craft the spending transaction for the outpoint created above and
// confirm it under the chain of the original miner.
spendTx := chainntnfs.CreateSpendTx(t, outpoint, pkScript)
spendTx := chainntnfs.CreateSpendTx(t, outpoint, output, privKey)
spendTxHash, err := miner.Node.SendRawTransaction(spendTx, true)
if err != nil {
t.Fatalf("unable to broadcast spend tx: %v", err)
@ -1186,14 +1287,17 @@ func testReorgSpend(miner *rpctest.Harness,
if _, err := miner.Node.Generate(numBlocks); err != nil {
t.Fatalf("unable to generate blocks: %v", err)
}
_, spendHeight, err := miner.Node.GetBestBlock()
if err != nil {
t.Fatalf("unable to get spend height: %v", err)
}
// We should see a spend notification dispatched with the correct spend
// details.
select {
case spendDetails := <-spendIntent.Spend:
checkNotificationFields(
spendDetails, outpoint, spendTxHash,
currentHeight+numBlocks, t,
spendDetails, outpoint, spendTxHash, spendHeight, t,
)
case <-time.After(5 * time.Second):
t.Fatal("expected spend notification to be dispatched")
@ -1243,19 +1347,18 @@ func testReorgSpend(miner *rpctest.Harness,
if err := chainntnfs.WaitForMempoolTx(miner, spendTxHash); err != nil {
t.Fatalf("tx not relayed to miner: %v", err)
}
_, currentHeight, err = miner.Node.GetBestBlock()
if err != nil {
t.Fatalf("unable to retrieve current height: %v", err)
}
if _, err := miner.Node.Generate(numBlocks); err != nil {
t.Fatalf("unable to generate single block: %v", err)
}
_, spendHeight, err = miner.Node.GetBestBlock()
if err != nil {
t.Fatalf("unable to retrieve current height: %v", err)
}
select {
case spendDetails := <-spendIntent.Spend:
checkNotificationFields(
spendDetails, outpoint, spendTxHash,
currentHeight+numBlocks, t,
spendDetails, outpoint, spendTxHash, spendHeight, t,
)
case <-time.After(5 * time.Second):
t.Fatal("expected spend notification to be dispatched")
@ -1392,6 +1495,16 @@ func testCatchUpOnMissedBlocks(miner *rpctest.Harness,
if err != nil {
t.Fatalf("unable to register for epoch notification: %v", err)
}
// Drain the notification dispatched upon registration as we're
// not interested in it.
select {
case <-epochClient.Epochs:
case <-time.After(5 * time.Second):
t.Fatal("expected to receive epoch for current block " +
"upon registration")
}
clients = append(clients, epochClient)
}
@ -1567,6 +1680,16 @@ func testCatchUpOnMissedBlocksWithReorg(miner1 *rpctest.Harness,
if err != nil {
t.Fatalf("unable to register for epoch notification: %v", err)
}
// Drain the notification dispatched upon registration as we're
// not interested in it.
select {
case <-epochClient.Epochs:
case <-time.After(5 * time.Second):
t.Fatal("expected to receive epoch for current block " +
"upon registration")
}
clients = append(clients, epochClient)
}
@ -1642,7 +1765,13 @@ func testCatchUpOnMissedBlocksWithReorg(miner1 *rpctest.Harness,
}
}
type testCase struct {
type txNtfnTestCase struct {
name string
test func(node *rpctest.Harness, notifier chainntnfs.TestChainNotifier,
scriptDispatch bool, t *testing.T)
}
type blockNtfnTestCase struct {
name string
test func(node *rpctest.Harness, notifier chainntnfs.TestChainNotifier,
t *testing.T)
@ -1654,7 +1783,7 @@ type blockCatchupTestCase struct {
t *testing.T)
}
var ntfnTests = []testCase{
var txNtfnTests = []txNtfnTestCase{
{
name: "single conf ntfn",
test: testSingleConfirmationNotification,
@ -1672,41 +1801,44 @@ var ntfnTests = []testCase{
test: testMultiClientConfirmationNotification,
},
{
name: "spend ntfn",
test: testSpendNotification,
},
{
name: "block epoch",
test: testBlockEpochNotification,
name: "lazy ntfn consumer",
test: testLazyNtfnConsumer,
},
{
name: "historical conf dispatch",
test: testTxConfirmedBeforeNtfnRegistration,
},
{
name: "reorg conf",
test: testReorgConf,
},
{
name: "spend ntfn",
test: testSpendNotification,
},
{
name: "historical spend dispatch",
test: testSpendBeforeNtfnRegistration,
},
{
name: "reorg spend",
test: testReorgSpend,
},
{
name: "cancel spend ntfn",
test: testCancelSpendNtfn,
},
}
var blockNtfnTests = []blockNtfnTestCase{
{
name: "block epoch",
test: testBlockEpochNotification,
},
{
name: "cancel epoch ntfn",
test: testCancelEpochNtfn,
},
{
name: "lazy ntfn consumer",
test: testLazyNtfnConsumer,
},
{
name: "reorg conf",
test: testReorgConf,
},
{
name: "reorg spend",
test: testReorgSpend,
},
}
var blockCatchupTests = []blockCatchupTestCase{
@ -1746,7 +1878,8 @@ func TestInterfaces(t *testing.T) {
rpcConfig := miner.RPCConfig()
p2pAddr := miner.P2PAddress()
log.Printf("Running %v ChainNotifier interface tests", len(ntfnTests))
log.Printf("Running %v ChainNotifier interface tests",
2*len(txNtfnTests)+len(blockNtfnTests)+len(blockCatchupTests))
for _, notifierDriver := range chainntnfs.RegisteredNotifiers() {
// Initialize a height hint cache for each notifier.
@ -1777,14 +1910,16 @@ func TestInterfaces(t *testing.T) {
)
newNotifier = func() (chainntnfs.TestChainNotifier, error) {
return bitcoindnotify.New(
bitcoindConn, hintCache, hintCache,
bitcoindConn, chainntnfs.NetParams,
hintCache, hintCache,
), nil
}
case "btcd":
newNotifier = func() (chainntnfs.TestChainNotifier, error) {
return btcdnotify.New(
&rpcConfig, hintCache, hintCache,
&rpcConfig, chainntnfs.NetParams,
hintCache, hintCache,
)
}
@ -1796,7 +1931,7 @@ func TestInterfaces(t *testing.T) {
newNotifier = func() (chainntnfs.TestChainNotifier, error) {
return neutrinonotify.New(
spvNode, hintCache, hintCache,
)
), nil
}
}
@ -1813,12 +1948,30 @@ func TestInterfaces(t *testing.T) {
notifierType, err)
}
for _, ntfnTest := range ntfnTests {
testName := fmt.Sprintf("%v: %v", notifierType,
ntfnTest.name)
for _, txNtfnTest := range txNtfnTests {
for _, scriptDispatch := range []bool{false, true} {
testName := fmt.Sprintf("%v %v", notifierType,
txNtfnTest.name)
if scriptDispatch {
testName += " with script dispatch"
}
success := t.Run(testName, func(t *testing.T) {
txNtfnTest.test(
miner, notifier, scriptDispatch,
t,
)
})
if !success {
break
}
}
}
for _, blockNtfnTest := range blockNtfnTests {
testName := fmt.Sprintf("%v %v", notifierType,
blockNtfnTest.name)
success := t.Run(testName, func(t *testing.T) {
ntfnTest.test(miner, notifier, t)
blockNtfnTest.test(miner, notifier, t)
})
if !success {
break
@ -1836,7 +1989,7 @@ func TestInterfaces(t *testing.T) {
notifierType, err)
}
testName := fmt.Sprintf("%v: %v", notifierType,
testName := fmt.Sprintf("%v %v", notifierType,
blockCatchupTest.name)
success := t.Run(testName, func(t *testing.T) {

@ -11,9 +11,9 @@ import (
// createNewNotifier creates a new instance of the ChainNotifier interface
// implemented by NeutrinoNotifier.
func createNewNotifier(args ...interface{}) (chainntnfs.ChainNotifier, error) {
if len(args) != 2 {
if len(args) != 3 {
return nil, fmt.Errorf("incorrect number of arguments to "+
".New(...), expected 2, instead passed %v", len(args))
".New(...), expected 3, instead passed %v", len(args))
}
config, ok := args[0].(*neutrino.ChainService)
@ -34,7 +34,7 @@ func createNewNotifier(args ...interface{}) (chainntnfs.ChainNotifier, error) {
"is incorrect, expected a chainntfs.ConfirmHintCache")
}
return New(config, spendHintCache, confirmHintCache)
return New(config, spendHintCache, confirmHintCache), nil
}
// init registers a driver for the NeutrinoNotify concrete implementation of

@ -27,13 +27,6 @@ const (
notifierType = "neutrino"
)
var (
// ErrChainNotifierShuttingDown is used when we are trying to
// measure a spend notification when notifier is already stopped.
ErrChainNotifierShuttingDown = errors.New("chainntnfs: system interrupt " +
"while attempting to register for spend notification.")
)
// NeutrinoNotifier is a version of ChainNotifier that's backed by the neutrino
// Bitcoin light client. Unlike other implementations, this implementation
// speaks directly to the p2p network. As a result, this implementation of the
@ -51,8 +44,8 @@ type NeutrinoNotifier struct {
started int32 // To be used atomically.
stopped int32 // To be used atomically.
heightMtx sync.RWMutex
bestHeight uint32
bestBlockMtx sync.RWMutex
bestBlock chainntnfs.BlockEpoch
p2pNode *neutrino.ChainService
chainView *neutrino.Rescan
@ -69,6 +62,7 @@ type NeutrinoNotifier struct {
rescanErr <-chan error
chainUpdates *queue.ConcurrentQueue
txUpdates *queue.ConcurrentQueue
// spendHintCache is a cache used to query and update the latest height
// hints for an outpoint. Each height hint represents the earliest
@ -93,27 +87,27 @@ var _ chainntnfs.ChainNotifier = (*NeutrinoNotifier)(nil)
// NOTE: The passed neutrino node should already be running and active before
// being passed into this function.
func New(node *neutrino.ChainService, spendHintCache chainntnfs.SpendHintCache,
confirmHintCache chainntnfs.ConfirmHintCache) (*NeutrinoNotifier, error) {
confirmHintCache chainntnfs.ConfirmHintCache) *NeutrinoNotifier {
notifier := &NeutrinoNotifier{
return &NeutrinoNotifier{
notificationCancels: make(chan interface{}),
notificationRegistry: make(chan interface{}),
blockEpochClients: make(map[uint64]*blockEpochRegistration),
p2pNode: node,
p2pNode: node,
chainConn: &NeutrinoChainConn{node},
rescanErr: make(chan error),
chainUpdates: queue.NewConcurrentQueue(10),
txUpdates: queue.NewConcurrentQueue(10),
spendHintCache: spendHintCache,
confirmHintCache: confirmHintCache,
quit: make(chan struct{}),
}
return notifier, nil
}
// Start contacts the running neutrino light client and kicks off an initial
@ -132,8 +126,13 @@ func (n *NeutrinoNotifier) Start() error {
if err != nil {
return err
}
n.bestBlock.Hash = &startingPoint.Hash
n.bestBlock.Height = startingPoint.Height
n.bestHeight = uint32(startingPoint.Height)
n.txNotifier = chainntnfs.NewTxNotifier(
uint32(n.bestBlock.Height), chainntnfs.ReorgSafetyLimit,
n.confirmHintCache, n.spendHintCache,
)
// Next, we'll create our set of rescan options. Currently it's
// required that a user MUST set an addr/outpoint/txid when creating a
@ -147,24 +146,19 @@ func (n *NeutrinoNotifier) Start() error {
rpcclient.NotificationHandlers{
OnFilteredBlockConnected: n.onFilteredBlockConnected,
OnFilteredBlockDisconnected: n.onFilteredBlockDisconnected,
OnRedeemingTx: n.onRelevantTx,
},
),
neutrino.WatchInputs(zeroInput),
}
n.txNotifier = chainntnfs.NewTxNotifier(
n.bestHeight, chainntnfs.ReorgSafetyLimit, n.confirmHintCache,
n.spendHintCache,
)
n.chainConn = &NeutrinoChainConn{n.p2pNode}
// Finally, we'll create our rescan struct, start it, and launch all
// the goroutines we need to operate this ChainNotifier instance.
n.chainView = n.p2pNode.NewRescan(rescanOptions...)
n.rescanErr = n.chainView.Start()
n.chainUpdates.Start()
n.txUpdates.Start()
n.wg.Add(1)
go n.notificationDispatcher()
@ -183,6 +177,7 @@ func (n *NeutrinoNotifier) Stop() error {
n.wg.Wait()
n.chainUpdates.Stop()
n.txUpdates.Stop()
// Notify all pending clients of our shutdown by closing the related
// notification channels.
@ -226,11 +221,14 @@ func (n *NeutrinoNotifier) onFilteredBlockConnected(height int32,
// Append this new chain update to the end of the queue of new chain
// updates.
n.chainUpdates.ChanIn() <- &filteredBlock{
select {
case n.chainUpdates.ChanIn() <- &filteredBlock{
hash: header.BlockHash(),
height: uint32(height),
txns: txns,
connect: true,
}:
case <-n.quit:
}
}
@ -241,10 +239,29 @@ func (n *NeutrinoNotifier) onFilteredBlockDisconnected(height int32,
// Append this new chain update to the end of the queue of new chain
// disconnects.
n.chainUpdates.ChanIn() <- &filteredBlock{
select {
case n.chainUpdates.ChanIn() <- &filteredBlock{
hash: header.BlockHash(),
height: uint32(height),
connect: false,
}:
case <-n.quit:
}
}
// relevantTx represents a relevant transaction to the notifier that fulfills
// any outstanding spend requests.
type relevantTx struct {
tx *btcutil.Tx
details *btcjson.BlockDetails
}
// onRelevantTx is a callback that proxies relevant transaction notifications
// from the backend to the notifier's main event handler.
func (n *NeutrinoNotifier) onRelevantTx(tx *btcutil.Tx, details *btcjson.BlockDetails) {
select {
case n.txUpdates.ChanIn() <- &relevantTx{tx, details}:
case <-n.quit:
}
}
@ -293,7 +310,7 @@ out:
defer n.wg.Done()
confDetails, err := n.historicalConfDetails(
msg.TxID, msg.PkScript,
msg.ConfRequest,
msg.StartHeight, msg.EndHeight,
)
if err != nil {
@ -308,7 +325,7 @@ out:
// cache at tip, since any pending
// rescans have now completed.
err = n.txNotifier.UpdateConfDetails(
*msg.TxID, confDetails,
msg.ConfRequest, confDetails,
)
if err != nil {
chainntnfs.Log.Error(err)
@ -317,25 +334,44 @@ out:
case *blockEpochRegistration:
chainntnfs.Log.Infof("New block epoch subscription")
n.blockEpochClients[msg.epochID] = msg
if msg.bestBlock != nil {
n.heightMtx.Lock()
bestHeight := int32(n.bestHeight)
n.heightMtx.Unlock()
missedBlocks, err :=
chainntnfs.GetClientMissedBlocks(
n.chainConn, msg.bestBlock,
bestHeight, false,
)
if err != nil {
msg.errorChan <- err
continue
}
for _, block := range missedBlocks {
n.notifyBlockEpochClient(msg,
block.Height, block.Hash)
}
// If the client did not provide their best
// known block, then we'll immediately dispatch
// a notification for the current tip.
if msg.bestBlock == nil {
n.notifyBlockEpochClient(
msg, n.bestBlock.Height,
n.bestBlock.Hash,
)
msg.errorChan <- nil
continue
}
// Otherwise, we'll attempt to deliver the
// backlog of notifications from their best
// known block.
n.bestBlockMtx.Lock()
bestHeight := n.bestBlock.Height
n.bestBlockMtx.Unlock()
missedBlocks, err := chainntnfs.GetClientMissedBlocks(
n.chainConn, msg.bestBlock, bestHeight,
false,
)
if err != nil {
msg.errorChan <- err
continue
}
for _, block := range missedBlocks {
n.notifyBlockEpochClient(
msg, block.Height, block.Hash,
)
}
msg.errorChan <- nil
case *rescanFilterUpdate:
@ -350,7 +386,7 @@ out:
case item := <-n.chainUpdates.ChanOut():
update := item.(*filteredBlock)
if update.connect {
n.heightMtx.Lock()
n.bestBlockMtx.Lock()
// Since neutrino has no way of knowing what
// height to rewind to in the case of a reorged
// best known height, there is no point in
@ -359,27 +395,24 @@ out:
// the other notifiers do when they receive
// a new connected block. Therefore, we just
// compare the heights.
if update.height != n.bestHeight+1 {
if update.height != uint32(n.bestBlock.Height+1) {
// Handle the case where the notifier
// missed some blocks from its chain
// backend
chainntnfs.Log.Infof("Missed blocks, " +
"attempting to catch up")
bestBlock := chainntnfs.BlockEpoch{
Height: int32(n.bestHeight),
Hash: nil,
}
_, missedBlocks, err :=
chainntnfs.HandleMissedBlocks(
n.chainConn,
n.txNotifier,
bestBlock,
n.bestBlock,
int32(update.height),
false,
)
if err != nil {
chainntnfs.Log.Error(err)
n.heightMtx.Unlock()
n.bestBlockMtx.Unlock()
continue
}
@ -388,13 +421,13 @@ out:
n.getFilteredBlock(block)
if err != nil {
chainntnfs.Log.Error(err)
n.heightMtx.Unlock()
n.bestBlockMtx.Unlock()
continue out
}
err = n.handleBlockConnected(filteredBlock)
if err != nil {
chainntnfs.Log.Error(err)
n.heightMtx.Unlock()
n.bestBlockMtx.Unlock()
continue out
}
}
@ -405,42 +438,46 @@ out:
if err != nil {
chainntnfs.Log.Error(err)
}
n.heightMtx.Unlock()
n.bestBlockMtx.Unlock()
continue
}
n.heightMtx.Lock()
if update.height != uint32(n.bestHeight) {
n.bestBlockMtx.Lock()
if update.height != uint32(n.bestBlock.Height) {
chainntnfs.Log.Infof("Missed disconnected " +
"blocks, attempting to catch up")
}
hash, err := n.p2pNode.GetBlockHash(int64(n.bestHeight))
if err != nil {
chainntnfs.Log.Errorf("Unable to fetch block hash "+
"for height %d: %v", n.bestHeight, err)
n.heightMtx.Unlock()
continue
}
notifierBestBlock := chainntnfs.BlockEpoch{
Height: int32(n.bestHeight),
Hash: hash,
}
newBestBlock, err := chainntnfs.RewindChain(
n.chainConn, n.txNotifier, notifierBestBlock,
n.chainConn, n.txNotifier, n.bestBlock,
int32(update.height-1),
)
if err != nil {
chainntnfs.Log.Errorf("Unable to rewind chain "+
"from height %d to height %d: %v",
n.bestHeight, update.height-1, err)
n.bestBlock.Height, update.height-1, err)
}
// Set the bestHeight here in case a chain rewind
// partially completed.
n.bestHeight = uint32(newBestBlock.Height)
n.heightMtx.Unlock()
n.bestBlock = newBestBlock
n.bestBlockMtx.Unlock()
case txUpdate := <-n.txUpdates.ChanOut():
// A new relevant transaction notification has been
// received from the backend. We'll attempt to process
// it to determine if it fulfills any outstanding
// confirmation and/or spend requests and dispatch
// notifications for them.
update := txUpdate.(*relevantTx)
err := n.txNotifier.ProcessRelevantSpendTx(
update.tx, uint32(update.details.Height),
)
if err != nil {
chainntnfs.Log.Errorf("Unable to process "+
"transaction %v: %v", update.tx.Hash(),
err)
}
case err := <-n.rescanErr:
chainntnfs.Log.Errorf("Error during rescan: %v", err)
@ -452,21 +489,20 @@ out:
}
}
// historicalConfDetails looks up whether a transaction is already included in
// a block in the active chain and, if so, returns details about the
// confirmation.
func (n *NeutrinoNotifier) historicalConfDetails(targetHash *chainhash.Hash,
pkScript []byte,
// historicalConfDetails looks up whether a confirmation request (txid/output
// script) has already been included in a block in the active chain and, if so,
// returns details about said block.
func (n *NeutrinoNotifier) historicalConfDetails(confRequest chainntnfs.ConfRequest,
startHeight, endHeight uint32) (*chainntnfs.TxConfirmation, error) {
// Starting from the height hint, we'll walk forwards in the chain to
// see if this transaction has already been confirmed.
// see if this transaction/output script has already been confirmed.
for scanHeight := endHeight; scanHeight >= startHeight && scanHeight > 0; scanHeight-- {
// Ensure we haven't been requested to shut down before
// processing the next height.
select {
case <-n.quit:
return nil, ErrChainNotifierShuttingDown
return nil, chainntnfs.ErrChainNotifierShuttingDown
default:
}
@ -498,7 +534,7 @@ func (n *NeutrinoNotifier) historicalConfDetails(targetHash *chainhash.Hash,
// In the case that the filter exists, we'll attempt to see if
// any element in it matches our target public key script.
key := builder.DeriveKey(blockHash)
match, err := regFilter.Match(key, pkScript)
match, err := regFilter.Match(key, confRequest.PkScript.Script())
if err != nil {
return nil, fmt.Errorf("unable to query filter: %v", err)
}
@ -516,16 +552,21 @@ func (n *NeutrinoNotifier) historicalConfDetails(targetHash *chainhash.Hash,
if err != nil {
return nil, fmt.Errorf("unable to get block from network: %v", err)
}
for j, tx := range block.Transactions() {
txHash := tx.Hash()
if txHash.IsEqual(targetHash) {
confDetails := chainntnfs.TxConfirmation{
BlockHash: blockHash,
BlockHeight: scanHeight,
TxIndex: uint32(j),
}
return &confDetails, nil
// For every transaction in the block, check which one matches
// our request. If we find one that does, we can dispatch its
// confirmation details.
for i, tx := range block.Transactions() {
if !confRequest.MatchesTx(tx.MsgTx()) {
continue
}
return &chainntnfs.TxConfirmation{
Tx: tx.MsgTx(),
BlockHash: blockHash,
BlockHeight: scanHeight,
TxIndex: uint32(i),
}, nil
}
}
@ -535,6 +576,8 @@ func (n *NeutrinoNotifier) historicalConfDetails(targetHash *chainhash.Hash,
// handleBlockConnected applies a chain update for a new block. Any watched
// transactions included this block will processed to either send notifications
// now or after numConfirmations confs.
//
// NOTE: This method must be called with the bestBlockMtx lock held.
func (n *NeutrinoNotifier) handleBlockConnected(newBlock *filteredBlock) error {
// We'll extend the txNotifier's height with the information of this new
// block, which will handle all of the notification logic for us.
@ -553,7 +596,8 @@ func (n *NeutrinoNotifier) handleBlockConnected(newBlock *filteredBlock) error {
// registered clients whom have had notifications fulfilled. Before
// doing so, we'll make sure update our in memory state in order to
// satisfy any client requests based upon the new block.
n.bestHeight = newBlock.height
n.bestBlock.Hash = &newBlock.hash
n.bestBlock.Height = int32(newBlock.height)
n.notifyBlockEpochs(int32(newBlock.height), &newBlock.hash)
return n.txNotifier.NotifyHeight(newBlock.height)
@ -603,8 +647,12 @@ func (n *NeutrinoNotifier) notifyBlockEpochClient(epochClient *blockEpochRegistr
}
// RegisterSpendNtfn registers an intent to be notified once the target
// outpoint has been spent by a transaction on-chain. Once a spend of the
// target outpoint has been detected, the details of the spending event will be
// outpoint/output script has been spent by a transaction on-chain. When
// intending to be notified of the spend of an output script, a nil outpoint
// must be used. The heightHint should represent the earliest height in the
// chain of the transaction that spent the outpoint/output script.
//
// Once a spend of has been detected, the details of the spending event will be
// sent across the 'Spend' channel.
func (n *NeutrinoNotifier) RegisterSpendNtfn(outpoint *wire.OutPoint,
pkScript []byte, heightHint uint32) (*chainntnfs.SpendEvent, error) {
@ -612,26 +660,22 @@ func (n *NeutrinoNotifier) RegisterSpendNtfn(outpoint *wire.OutPoint,
// First, we'll construct a spend notification request and hand it off
// to the txNotifier.
spendID := atomic.AddUint64(&n.spendClientCounter, 1)
cancel := func() {
n.txNotifier.CancelSpend(*outpoint, spendID)
}
ntfn := &chainntnfs.SpendNtfn{
SpendID: spendID,
OutPoint: *outpoint,
Event: chainntnfs.NewSpendEvent(cancel),
HeightHint: heightHint,
}
historicalDispatch, err := n.txNotifier.RegisterSpend(ntfn)
spendRequest, err := chainntnfs.NewSpendRequest(outpoint, pkScript)
if err != nil {
return nil, err
}
ntfn := &chainntnfs.SpendNtfn{
SpendID: spendID,
SpendRequest: spendRequest,
Event: chainntnfs.NewSpendEvent(func() {
n.txNotifier.CancelSpend(spendRequest, spendID)
}),
HeightHint: heightHint,
}
// If the txNotifier didn't return any details to perform a historical
// scan of the chain, then we can return early as there's nothing left
// for us to do.
if historicalDispatch == nil {
return ntfn.Event, nil
historicalDispatch, txNotifierTip, err := n.txNotifier.RegisterSpend(ntfn)
if err != nil {
return nil, err
}
// To determine whether this outpoint has been spent on-chain, we'll
@ -640,45 +684,64 @@ func (n *NeutrinoNotifier) RegisterSpendNtfn(outpoint *wire.OutPoint,
// past.
//
// We'll update our filter first to ensure we can immediately detect the
// spend at tip. To do so, we'll map the script into an address
// type so we can instruct neutrino to match if the transaction
// containing the script is found in a block.
// spend at tip.
inputToWatch := neutrino.InputWithScript{
OutPoint: *outpoint,
PkScript: pkScript,
OutPoint: spendRequest.OutPoint,
PkScript: spendRequest.PkScript.Script(),
}
updateOptions := []neutrino.UpdateOption{
neutrino.AddInputs(inputToWatch),
neutrino.DisableDisconnectedNtfns(true),
}
// We'll use the txNotifier's tip as the starting point of our filter
// update. In the case of an output script spend request, we'll check if
// we should perform a historical rescan and start from there, as we
// cannot do so with GetUtxo since it matches outpoints.
rewindHeight := txNotifierTip
if historicalDispatch != nil &&
spendRequest.OutPoint == chainntnfs.ZeroOutPoint {
rewindHeight = historicalDispatch.StartHeight
}
updateOptions = append(updateOptions, neutrino.Rewind(rewindHeight))
errChan := make(chan error, 1)
select {
case n.notificationRegistry <- &rescanFilterUpdate{
updateOptions: []neutrino.UpdateOption{
neutrino.AddInputs(inputToWatch),
neutrino.Rewind(historicalDispatch.EndHeight),
neutrino.DisableDisconnectedNtfns(true),
},
errChan: errChan,
updateOptions: updateOptions,
errChan: errChan,
}:
case <-n.quit:
return nil, ErrChainNotifierShuttingDown
return nil, chainntnfs.ErrChainNotifierShuttingDown
}
select {
case err = <-errChan:
case <-n.quit:
return nil, ErrChainNotifierShuttingDown
return nil, chainntnfs.ErrChainNotifierShuttingDown
}
if err != nil {
return nil, fmt.Errorf("unable to update filter: %v", err)
}
// If the txNotifier didn't return any details to perform a historical
// scan of the chain, or if we already performed one like in the case of
// output script spend requests, then we can return early as there's
// nothing left for us to do.
if historicalDispatch == nil ||
spendRequest.OutPoint == chainntnfs.ZeroOutPoint {
return ntfn.Event, nil
}
// With the filter updated, we'll dispatch our historical rescan to
// ensure we detect the spend if it happened in the past. We'll ensure
// that neutrino is caught up to the starting height before we attempt
// to fetch the UTXO from the chain. If we're behind, then we may miss a
// notification dispatch.
for {
n.heightMtx.RLock()
currentHeight := n.bestHeight
n.heightMtx.RUnlock()
n.bestBlockMtx.RLock()
currentHeight := uint32(n.bestBlock.Height)
n.bestBlockMtx.RUnlock()
if currentHeight >= historicalDispatch.StartHeight {
break
@ -706,7 +769,7 @@ func (n *NeutrinoNotifier) RegisterSpendNtfn(outpoint *wire.OutPoint,
if spendReport != nil && spendReport.SpendingTx != nil {
spendingTxHash := spendReport.SpendingTx.TxHash()
spendDetails = &chainntnfs.SpendDetail{
SpentOutPoint: outpoint,
SpentOutPoint: &spendRequest.OutPoint,
SpenderTxHash: &spendingTxHash,
SpendingTx: spendReport.SpendingTx,
SpenderInputIndex: spendReport.SpendingInputIndex,
@ -718,7 +781,7 @@ func (n *NeutrinoNotifier) RegisterSpendNtfn(outpoint *wire.OutPoint,
// not, we'll mark our historical rescan as complete to ensure the
// outpoint's spend hint gets updated upon connected/disconnected
// blocks.
err = n.txNotifier.UpdateSpendDetails(*outpoint, spendDetails)
err = n.txNotifier.UpdateSpendDetails(spendRequest, spendDetails)
if err != nil {
return nil, err
}
@ -726,40 +789,48 @@ func (n *NeutrinoNotifier) RegisterSpendNtfn(outpoint *wire.OutPoint,
return ntfn.Event, nil
}
// RegisterConfirmationsNtfn registers a notification with NeutrinoNotifier
// which will be triggered once the txid reaches numConfs number of
// confirmations.
// RegisterConfirmationsNtfn registers an intent to be notified once the target
// txid/output script has reached numConfs confirmations on-chain. When
// intending to be notified of the confirmation of an output script, a nil txid
// must be used. The heightHint should represent the earliest height at which
// the txid/output script could have been included in the chain.
//
// Progress on the number of confirmations left can be read from the 'Updates'
// channel. Once it has reached all of its confirmations, a notification will be
// sent across the 'Confirmed' channel.
func (n *NeutrinoNotifier) RegisterConfirmationsNtfn(txid *chainhash.Hash,
pkScript []byte,
numConfs, heightHint uint32) (*chainntnfs.ConfirmationEvent, error) {
// Construct a notification request for the transaction and send it to
// the main event loop.
confID := atomic.AddUint64(&n.confClientCounter, 1)
confRequest, err := chainntnfs.NewConfRequest(txid, pkScript)
if err != nil {
return nil, err
}
ntfn := &chainntnfs.ConfNtfn{
ConfID: atomic.AddUint64(&n.confClientCounter, 1),
TxID: txid,
PkScript: pkScript,
ConfID: confID,
ConfRequest: confRequest,
NumConfirmations: numConfs,
Event: chainntnfs.NewConfirmationEvent(numConfs),
HeightHint: heightHint,
Event: chainntnfs.NewConfirmationEvent(numConfs, func() {
n.txNotifier.CancelConf(confRequest, confID)
}),
HeightHint: heightHint,
}
chainntnfs.Log.Infof("New confirmation subscription: "+
"txid=%v, numconfs=%v", txid, numConfs)
chainntnfs.Log.Infof("New confirmation subscription: %v, num_confs=%v",
confRequest, numConfs)
// Register the conf notification with the TxNotifier. A non-nil value
// for `dispatch` will be returned if we are required to perform a
// manual scan for the confirmation. Otherwise the notifier will begin
// watching at tip for the transaction to confirm.
dispatch, err := n.txNotifier.RegisterConf(ntfn)
dispatch, txNotifierTip, err := n.txNotifier.RegisterConf(ntfn)
if err != nil {
return nil, err
}
if dispatch == nil {
return ntfn.Event, nil
}
// To determine whether this transaction has confirmed on-chain, we'll
// update our filter to watch for the transaction at tip and we'll also
// dispatch a historical rescan to determine if it has confirmed in the
@ -770,7 +841,9 @@ func (n *NeutrinoNotifier) RegisterConfirmationsNtfn(txid *chainhash.Hash,
// type so we can instruct neutrino to match if the transaction
// containing the script is found in a block.
params := n.p2pNode.ChainParams()
_, addrs, _, err := txscript.ExtractPkScriptAddrs(pkScript, &params)
_, addrs, _, err := txscript.ExtractPkScriptAddrs(
confRequest.PkScript.Script(), &params,
)
if err != nil {
return nil, fmt.Errorf("unable to extract script: %v", err)
}
@ -782,30 +855,36 @@ func (n *NeutrinoNotifier) RegisterConfirmationsNtfn(txid *chainhash.Hash,
case n.notificationRegistry <- &rescanFilterUpdate{
updateOptions: []neutrino.UpdateOption{
neutrino.AddAddrs(addrs...),
neutrino.Rewind(dispatch.EndHeight),
neutrino.Rewind(txNotifierTip),
neutrino.DisableDisconnectedNtfns(true),
},
errChan: errChan,
}:
case <-n.quit:
return nil, ErrChainNotifierShuttingDown
return nil, chainntnfs.ErrChainNotifierShuttingDown
}
select {
case err = <-errChan:
case <-n.quit:
return nil, ErrChainNotifierShuttingDown
return nil, chainntnfs.ErrChainNotifierShuttingDown
}
if err != nil {
return nil, fmt.Errorf("unable to update filter: %v", err)
}
// Finally, with the filter updates, we can dispatch the historical
// If a historical rescan was not requested by the txNotifier, then we
// can return to the caller.
if dispatch == nil {
return ntfn.Event, nil
}
// Finally, with the filter updated, we can dispatch the historical
// rescan to ensure we can detect if the event happened in the past.
select {
case n.notificationRegistry <- dispatch:
case <-n.quit:
return nil, ErrChainNotifierShuttingDown
return nil, chainntnfs.ErrChainNotifierShuttingDown
}
return ntfn.Event, nil
@ -838,7 +917,9 @@ type epochCancel struct {
// RegisterBlockEpochNtfn returns a BlockEpochEvent which subscribes the
// caller to receive notifications, of each new block connected to the main
// chain. Clients have the option of passing in their best known block, which
// the notifier uses to check if they are behind on blocks and catch them up.
// the notifier uses to check if they are behind on blocks and catch them up. If
// they do not provide one, then a notification will be dispatched immediately
// for the current tip of the chain upon a successful registration.
func (n *NeutrinoNotifier) RegisterBlockEpochNtfn(
bestBlock *chainntnfs.BlockEpoch) (*chainntnfs.BlockEpochEvent, error) {

@ -53,14 +53,13 @@ func (n *NeutrinoNotifier) UnsafeStart(bestHeight int32,
n.confirmHintCache, n.spendHintCache,
)
n.chainConn = &NeutrinoChainConn{n.p2pNode}
// Finally, we'll create our rescan struct, start it, and launch all
// the goroutines we need to operate this ChainNotifier instance.
n.chainView = n.p2pNode.NewRescan(rescanOptions...)
n.rescanErr = n.chainView.Start()
n.chainUpdates.Start()
n.txUpdates.Start()
if generateBlocks != nil {
// Ensure no block notifications are pending when we start the
@ -90,7 +89,8 @@ func (n *NeutrinoNotifier) UnsafeStart(bestHeight int32,
// Run notificationDispatcher after setting the notifier's best height
// to avoid a race condition.
n.bestHeight = uint32(bestHeight)
n.bestBlock.Hash = bestHash
n.bestBlock.Height = bestHeight
n.wg.Add(1)
go n.notificationDispatcher()

@ -35,35 +35,44 @@ var (
var (
NetParams = &chaincfg.RegressionNetParams
testPrivKey = []byte{
0x81, 0xb6, 0x37, 0xd8, 0xfc, 0xd2, 0xc6, 0xda,
0x63, 0x59, 0xe6, 0x96, 0x31, 0x13, 0xa1, 0x17,
0xd, 0xe7, 0x95, 0xe4, 0xb7, 0x25, 0xb8, 0x4d,
0x1e, 0xb, 0x4c, 0xfd, 0x9e, 0xc5, 0x8c, 0xe9,
}
privKey, pubKey = btcec.PrivKeyFromBytes(btcec.S256(), testPrivKey)
addrPk, _ = btcutil.NewAddressPubKey(
pubKey.SerializeCompressed(), NetParams,
)
testAddr = addrPk.AddressPubKeyHash()
)
// GetTestTxidAndScript generate a new test transaction and returns its txid and
// the script of the output being generated.
func GetTestTxidAndScript(h *rpctest.Harness) (*chainhash.Hash, []byte, error) {
script, err := txscript.PayToAddrScript(testAddr)
// randPubKeyHashScript generates a P2PKH script that pays to the public key of
// a randomly-generated private key.
func randPubKeyHashScript() ([]byte, *btcec.PrivateKey, error) {
privKey, err := btcec.NewPrivateKey(btcec.S256())
if err != nil {
return nil, nil, err
}
output := &wire.TxOut{Value: 2e8, PkScript: script}
pubKeyHash := btcutil.Hash160(privKey.PubKey().SerializeCompressed())
addrScript, err := btcutil.NewAddressPubKeyHash(pubKeyHash, NetParams)
if err != nil {
return nil, nil, err
}
pkScript, err := txscript.PayToAddrScript(addrScript)
if err != nil {
return nil, nil, err
}
return pkScript, privKey, nil
}
// GetTestTxidAndScript generate a new test transaction and returns its txid and
// the script of the output being generated.
func GetTestTxidAndScript(h *rpctest.Harness) (*chainhash.Hash, []byte, error) {
pkScript, _, err := randPubKeyHashScript()
if err != nil {
return nil, nil, fmt.Errorf("unable to generate pkScript: %v", err)
}
output := &wire.TxOut{Value: 2e8, PkScript: pkScript}
txid, err := h.SendOutputs([]*wire.TxOut{output}, 10)
if err != nil {
return nil, nil, err
}
return txid, script, nil
return txid, pkScript, nil
}
// WaitForMempoolTx waits for the txid to be seen in the miner's mempool.
@ -107,16 +116,18 @@ func WaitForMempoolTx(miner *rpctest.Harness, txid *chainhash.Hash) error {
// CreateSpendableOutput creates and returns an output that can be spent later
// on.
func CreateSpendableOutput(t *testing.T, miner *rpctest.Harness) (*wire.OutPoint, []byte) {
func CreateSpendableOutput(t *testing.T,
miner *rpctest.Harness) (*wire.OutPoint, *wire.TxOut, *btcec.PrivateKey) {
t.Helper()
// Create a transaction that only has one output, the one destined for
// the recipient.
script, err := txscript.PayToAddrScript(testAddr)
pkScript, privKey, err := randPubKeyHashScript()
if err != nil {
t.Fatalf("unable to create p2pkh script: %v", err)
t.Fatalf("unable to generate pkScript: %v", err)
}
output := &wire.TxOut{Value: 2e8, PkScript: script}
output := &wire.TxOut{Value: 2e8, PkScript: pkScript}
txid, err := miner.SendOutputsWithoutChange([]*wire.TxOut{output}, 10)
if err != nil {
t.Fatalf("unable to create tx: %v", err)
@ -130,19 +141,22 @@ func CreateSpendableOutput(t *testing.T, miner *rpctest.Harness) (*wire.OutPoint
t.Fatalf("unable to generate single block: %v", err)
}
return wire.NewOutPoint(txid, 0), script
return wire.NewOutPoint(txid, 0), output, privKey
}
// CreateSpendTx creates a transaction spending the specified output.
func CreateSpendTx(t *testing.T, outpoint *wire.OutPoint, pkScript []byte) *wire.MsgTx {
func CreateSpendTx(t *testing.T, prevOutPoint *wire.OutPoint,
prevOutput *wire.TxOut, privKey *btcec.PrivateKey) *wire.MsgTx {
t.Helper()
spendingTx := wire.NewMsgTx(1)
spendingTx.AddTxIn(&wire.TxIn{PreviousOutPoint: *outpoint})
spendingTx.AddTxOut(&wire.TxOut{Value: 1e8, PkScript: pkScript})
spendingTx.AddTxIn(&wire.TxIn{PreviousOutPoint: *prevOutPoint})
spendingTx.AddTxOut(&wire.TxOut{Value: 1e8, PkScript: prevOutput.PkScript})
sigScript, err := txscript.SignatureScript(
spendingTx, 0, pkScript, txscript.SigHashAll, privKey, true,
spendingTx, 0, prevOutput.PkScript, txscript.SigHashAll,
privKey, true,
)
if err != nil {
t.Fatalf("unable to sign tx: %v", err)

@ -1,13 +1,16 @@
package chainntnfs
import (
"bytes"
"errors"
"fmt"
"sync"
"github.com/btcsuite/btcd/chaincfg/chainhash"
"github.com/btcsuite/btcd/txscript"
"github.com/btcsuite/btcd/wire"
"github.com/btcsuite/btcutil"
"github.com/lightningnetwork/lnd/channeldb"
)
const (
@ -23,6 +26,20 @@ const (
MaxNumConfs = ReorgSafetyLimit
)
var (
// ZeroHash is the value that should be used as the txid when
// registering for the confirmation of a script on-chain. This allows
// the notifier to match _and_ dispatch upon the inclusion of the script
// on-chain, rather than the txid.
ZeroHash chainhash.Hash
// ZeroOutPoint is the value that should be used as the outpoint when
// registering for the spend of a script on-chain. This allows the
// notifier to match _and_ dispatch upon detecting the spend of the
// script on-chain, rather than the outpoint.
ZeroOutPoint wire.OutPoint
)
var (
// ErrTxNotifierExiting is an error returned when attempting to interact
// with the TxNotifier but it been shut down.
@ -56,21 +73,23 @@ const (
)
// confNtfnSet holds all known, registered confirmation notifications for a
// single txid. If duplicates notifications are requested, only one historical
// dispatch will be spawned to ensure redundant scans are not permitted. A
// single conf detail will be constructed and dispatched to all interested
// txid/output script. If duplicates notifications are requested, only one
// historical dispatch will be spawned to ensure redundant scans are not
// permitted. A single conf detail will be constructed and dispatched to all
// interested
// clients.
type confNtfnSet struct {
// ntfns keeps tracks of all the active client notification requests for
// a transaction.
// a transaction/output script
ntfns map[uint64]*ConfNtfn
// rescanStatus represents the current rescan state for the transaction.
// rescanStatus represents the current rescan state for the
// transaction/output script.
rescanStatus rescanState
// details serves as a cache of the confirmation details of a
// transaction that we'll use to determine if a transaction has already
// confirmed at the time of registration.
// transaction that we'll use to determine if a transaction/output
// script has already confirmed at the time of registration.
details *TxConfirmation
}
@ -83,20 +102,22 @@ func newConfNtfnSet() *confNtfnSet {
}
}
// spendNtfnSet holds all known, registered spend notifications for an outpoint.
// If duplicate notifications are requested, only one historical dispatch will
// be spawned to ensure redundant scans are not permitted.
// spendNtfnSet holds all known, registered spend notifications for a spend
// request (outpoint/output script). If duplicate notifications are requested,
// only one historical dispatch will be spawned to ensure redundant scans are
// not permitted.
type spendNtfnSet struct {
// ntfns keeps tracks of all the active client notification requests for
// an outpoint.
// an outpoint/output script.
ntfns map[uint64]*SpendNtfn
// rescanStatus represents the current rescan state for the outpoint.
// rescanStatus represents the current rescan state for the spend
// request (outpoint/output script).
rescanStatus rescanState
// details serves as a cache of the spend details for an outpoint that
// we'll use to determine if an outpoint has already been spent at the
// time of registration.
// details serves as a cache of the spend details for an outpoint/output
// script that we'll use to determine if it has already been spent at
// the time of registration.
details *SpendDetail
}
@ -108,24 +129,94 @@ func newSpendNtfnSet() *spendNtfnSet {
}
}
// ConfNtfn represents a notifier client's request to receive a notification
// once the target transaction gets sufficient confirmations. The client is
// asynchronously notified via the ConfirmationEvent channels.
type ConfNtfn struct {
// ConfID uniquely identifies the confirmation notification request for
// the specified transaction.
ConfID uint64
// TxID is the hash of the transaction for which confirmation notifications
// are requested.
TxID *chainhash.Hash
// ConfRequest encapsulates a request for a confirmation notification of either
// a txid or output script.
type ConfRequest struct {
// TxID is the hash of the transaction for which confirmation
// notifications are requested. If set to a zero hash, then a
// confirmation notification will be dispatched upon inclusion of the
// _script_, rather than the txid.
TxID chainhash.Hash
// PkScript is the public key script of an outpoint created in this
// transaction.
//
// NOTE: This value MUST be set when the dispatch is to be performed
// using compact filters.
PkScript []byte
PkScript txscript.PkScript
}
// NewConfRequest creates a request for a confirmation notification of either a
// txid or output script. A nil txid or an allocated ZeroHash can be used to
// dispatch the confirmation notification on the script.
func NewConfRequest(txid *chainhash.Hash, pkScript []byte) (ConfRequest, error) {
var r ConfRequest
outputScript, err := txscript.ParsePkScript(pkScript)
if err != nil {
return r, err
}
// We'll only set a txid for which we'll dispatch a confirmation
// notification on this request if one was provided. Otherwise, we'll
// default to dispatching on the confirmation of the script instead.
if txid != nil {
r.TxID = *txid
}
r.PkScript = outputScript
return r, nil
}
// String returns the string representation of the ConfRequest.
func (r ConfRequest) String() string {
if r.TxID != ZeroHash {
return fmt.Sprintf("txid=%v", r.TxID)
}
return fmt.Sprintf("script=%v", r.PkScript)
}
// ConfHintKey returns the key that will be used to index the confirmation
// request's hint within the height hint cache.
func (r ConfRequest) ConfHintKey() ([]byte, error) {
if r.TxID == ZeroHash {
return r.PkScript.Script(), nil
}
var txid bytes.Buffer
if err := channeldb.WriteElement(&txid, r.TxID); err != nil {
return nil, err
}
return txid.Bytes(), nil
}
// MatchesTx determines whether the given transaction satisfies the confirmation
// request. If the confirmation request is for a script, then we'll check all of
// the outputs of the transaction to determine if it matches. Otherwise, we'll
// match on the txid.
func (r ConfRequest) MatchesTx(tx *wire.MsgTx) bool {
if r.TxID != ZeroHash {
return r.TxID == tx.TxHash()
}
pkScript := r.PkScript.Script()
for _, txOut := range tx.TxOut {
if bytes.Equal(txOut.PkScript, pkScript) {
return true
}
}
return false
}
// ConfNtfn represents a notifier client's request to receive a notification
// once the target transaction/ouput script gets sufficient confirmations. The
// client is asynchronously notified via the ConfirmationEvent channels.
type ConfNtfn struct {
// ConfID uniquely identifies the confirmation notification request for
// the specified transaction/output script.
ConfID uint64
// ConfRequest represents either the txid or script we should detect
// inclusion of within the chain.
ConfRequest
// NumConfirmations is the number of confirmations after which the
// notification is to be sent.
@ -144,18 +235,12 @@ type ConfNtfn struct {
}
// HistoricalConfDispatch parameterizes a manual rescan for a particular
// transaction identifier. The parameters include the start and end block
// transaction/output script. The parameters include the start and end block
// heights specifying the range of blocks to scan.
type HistoricalConfDispatch struct {
// TxID is the transaction ID to search for in the historical dispatch.
TxID *chainhash.Hash
// PkScript is a public key script from an output created by this
// transaction.
//
// NOTE: This value MUST be set when the dispatch is to be performed
// using compact filters.
PkScript []byte
// ConfRequest represents either the txid or script we should detect
// inclusion of within the chain.
ConfRequest
// StartHeight specifies the block height at which to being the
// historical rescan.
@ -166,30 +251,121 @@ type HistoricalConfDispatch struct {
EndHeight uint32
}
// SpendNtfn represents a client's request to receive a notification once an
// outpoint has been spent on-chain. The client is asynchronously notified via
// the SpendEvent channels.
type SpendNtfn struct {
// SpendID uniquely identies the spend notification request for the
// specified outpoint.
SpendID uint64
// SpendRequest encapsulates a request for a spend notification of either an
// outpoint or output script.
type SpendRequest struct {
// OutPoint is the outpoint for which a client has requested a spend
// notification for.
// notification for. If set to a zero outpoint, then a spend
// notification will be dispatched upon detecting the spend of the
// _script_, rather than the outpoint.
OutPoint wire.OutPoint
// PkScript is the script of the outpoint. This is needed in order to
// match compact filters when attempting a historical rescan to
// determine if the outpoint has already been spent.
PkScript []byte
// PkScript is the script of the outpoint. If a zero outpoint is set,
// then this can be an arbitrary script.
PkScript txscript.PkScript
}
// NewSpendRequest creates a request for a spend notification of either an
// outpoint or output script. A nil outpoint or an allocated ZeroOutPoint can be
// used to dispatch the confirmation notification on the script.
func NewSpendRequest(op *wire.OutPoint, pkScript []byte) (SpendRequest, error) {
var r SpendRequest
outputScript, err := txscript.ParsePkScript(pkScript)
if err != nil {
return r, err
}
// We'll only set an outpoint for which we'll dispatch a spend
// notification on this request if one was provided. Otherwise, we'll
// default to dispatching on the spend of the script instead.
if op != nil {
r.OutPoint = *op
}
r.PkScript = outputScript
return r, nil
}
// String returns the string representation of the SpendRequest.
func (r SpendRequest) String() string {
if r.OutPoint != ZeroOutPoint {
return fmt.Sprintf("outpoint=%v", r.OutPoint)
}
return fmt.Sprintf("script=%v", r.PkScript)
}
// SpendHintKey returns the key that will be used to index the spend request's
// hint within the height hint cache.
func (r SpendRequest) SpendHintKey() ([]byte, error) {
if r.OutPoint == ZeroOutPoint {
return r.PkScript.Script(), nil
}
var outpoint bytes.Buffer
err := channeldb.WriteElement(&outpoint, r.OutPoint)
if err != nil {
return nil, err
}
return outpoint.Bytes(), nil
}
// MatchesTx determines whether the given transaction satisfies the spend
// request. If the spend request is for an outpoint, then we'll check all of
// the outputs being spent by the inputs of the transaction to determine if it
// matches. Otherwise, we'll need to match on the output script being spent, so
// we'll recompute it for each input of the transaction to determine if it
// matches.
func (r SpendRequest) MatchesTx(tx *wire.MsgTx) (bool, uint32, error) {
if r.OutPoint != ZeroOutPoint {
for i, txIn := range tx.TxIn {
if txIn.PreviousOutPoint == r.OutPoint {
return true, uint32(i), nil
}
}
return false, 0, nil
}
for i, txIn := range tx.TxIn {
pkScript, err := txscript.ComputePkScript(
txIn.SignatureScript, txIn.Witness,
)
if err == txscript.ErrUnsupportedScriptType {
continue
}
if err != nil {
return false, 0, err
}
if bytes.Equal(pkScript.Script(), r.PkScript.Script()) {
return true, uint32(i), nil
}
}
return false, 0, nil
}
// SpendNtfn represents a client's request to receive a notification once an
// outpoint/output script has been spent on-chain. The client is asynchronously
// notified via the SpendEvent channels.
type SpendNtfn struct {
// SpendID uniquely identies the spend notification request for the
// specified outpoint/output script.
SpendID uint64
// SpendRequest represents either the outpoint or script we should
// detect the spend of.
SpendRequest
// Event contains references to the channels that the notifications are
// to be sent over.
Event *SpendEvent
// HeightHint is the earliest height in the chain that we expect to find
// the spending transaction of the specified outpoint. This value will
// be overridden by the spend hint cache if it contains an entry for it.
// the spending transaction of the specified outpoint/output script.
// This value will be overridden by the spend hint cache if it contains
// an entry for it.
HeightHint uint32
// dispatched signals whether a spend notification has been disptached
@ -198,15 +374,13 @@ type SpendNtfn struct {
}
// HistoricalSpendDispatch parameterizes a manual rescan to determine the
// spending details (if any) of an outpoint. The parameters include the start
// and end block heights specifying the range of blocks to scan.
// spending details (if any) of an outpoint/output script. The parameters
// include the start and end block heights specifying the range of blocks to
// scan.
type HistoricalSpendDispatch struct {
// OutPoint is the outpoint which we should attempt to find the spending
OutPoint wire.OutPoint
// PkScript is the script of the outpoint. This is needed in order to
// match compact filters when attempting a historical rescan.
PkScript []byte
// SpendRequest represents either the outpoint or script we should
// detect the spend of.
SpendRequest
// StartHeight specified the block height at which to begin the
// historical rescan.
@ -219,9 +393,9 @@ type HistoricalSpendDispatch struct {
// TxNotifier is a struct responsible for delivering transaction notifications
// to subscribers. These notifications can be of two different types:
// transaction confirmations and/or outpoint spends. The TxNotifier will watch
// the blockchain as new blocks come in, in order to satisfy its client
// requests.
// transaction/output script confirmations and/or outpoint/output script spends.
// The TxNotifier will watch the blockchain as new blocks come in, in order to
// satisfy its client requests.
type TxNotifier struct {
// currentHeight is the height of the tracked blockchain. It is used to
// determine the number of confirmations a tx has and ensure blocks are
@ -240,38 +414,41 @@ type TxNotifier struct {
// blocks are disconnected without being interrupted by a new block.
reorgDepth uint32
// confNotifications is an index of notification requests by transaction
// hash.
confNotifications map[chainhash.Hash]*confNtfnSet
// confNotifications is an index of confirmation notification requests
// by transaction hash/output script.
confNotifications map[ConfRequest]*confNtfnSet
// txsByInitialHeight is an index of watched transactions by the height
// that they are included at in the blockchain. This is tracked so that
// incorrect notifications are not sent if a transaction is reorged out
// of the chain and so that negative confirmations can be recognized.
txsByInitialHeight map[uint32]map[chainhash.Hash]struct{}
// confsByInitialHeight is an index of watched transactions/output
// scripts by the height that they are included at in the chain. This
// is tracked so that incorrect notifications are not sent if a
// transaction/output script is reorged out of the chain and so that
// negative confirmations can be recognized.
confsByInitialHeight map[uint32]map[ConfRequest]struct{}
// ntfnsByConfirmHeight is an index of notification requests by the
// height at which the transaction will have sufficient confirmations.
// height at which the transaction/output script will have sufficient
// confirmations.
ntfnsByConfirmHeight map[uint32]map[*ConfNtfn]struct{}
// spendNotifications is an index of all active notification requests
// per outpoint.
spendNotifications map[wire.OutPoint]*spendNtfnSet
// per outpoint/output script.
spendNotifications map[SpendRequest]*spendNtfnSet
// opsBySpendHeight is an index that keeps tracks of the spending height
// of an outpoint we are currently tracking notifications for. This is
// used in order to recover from the spending transaction of an outpoint
// spendsByHeight is an index that keeps tracks of the spending height
// of outpoints/output scripts we are currently tracking notifications
// for. This is used in order to recover from spending transactions
// being reorged out of the chain.
opsBySpendHeight map[uint32]map[wire.OutPoint]struct{}
spendsByHeight map[uint32]map[SpendRequest]struct{}
// confirmHintCache is a cache used to maintain the latest height hints
// for transactions. Each height hint represents the earliest height at
// which the transactions could have been confirmed within the chain.
// for transactions/output scripts. Each height hint represents the
// earliest height at which they scripts could have been confirmed
// within the chain.
confirmHintCache ConfirmHintCache
// spendHintCache is a cache used to maintain the latest height hints
// for outpoints. Each height hint represents the earliest height at
// which the outpoints could have been spent within the chain.
// for outpoints/output scripts. Each height hint represents the
// earliest height at which they could have been spent within the chain.
spendHintCache SpendHintCache
// quit is closed in order to signal that the notifier is gracefully
@ -292,39 +469,43 @@ func NewTxNotifier(startHeight uint32, reorgSafetyLimit uint32,
return &TxNotifier{
currentHeight: startHeight,
reorgSafetyLimit: reorgSafetyLimit,
confNotifications: make(map[chainhash.Hash]*confNtfnSet),
txsByInitialHeight: make(map[uint32]map[chainhash.Hash]struct{}),
confNotifications: make(map[ConfRequest]*confNtfnSet),
confsByInitialHeight: make(map[uint32]map[ConfRequest]struct{}),
ntfnsByConfirmHeight: make(map[uint32]map[*ConfNtfn]struct{}),
spendNotifications: make(map[wire.OutPoint]*spendNtfnSet),
opsBySpendHeight: make(map[uint32]map[wire.OutPoint]struct{}),
spendNotifications: make(map[SpendRequest]*spendNtfnSet),
spendsByHeight: make(map[uint32]map[SpendRequest]struct{}),
confirmHintCache: confirmHintCache,
spendHintCache: spendHintCache,
quit: make(chan struct{}),
}
}
// RegisterConf handles a new notification request. The client will be notified
// when the transaction gets a sufficient number of confirmations on the
// blockchain. The registration succeeds if no error is returned. If the
// returned HistoricalConfDispatch is non-nil, the caller is responsible for
// attempting to manually rescan blocks for the txid between the start and end
// heights.
// RegisterConf handles a new confirmation notification request. The client will
// be notified when the transaction/output script gets a sufficient number of
// confirmations in the blockchain. The registration succeeds if no error is
// returned. If the returned HistoricalConfDispatch is non-nil, the caller is
// responsible for attempting to manually rescan blocks for the txid/output
// script between the start and end heights. The notifier's current height is
// also returned so that backends can request to be notified of confirmations
// from this point forwards.
//
// NOTE: If the transaction has already been included in a block on the chain,
// the confirmation details must be provided with the UpdateConfDetails method,
// otherwise we will wait for the transaction to confirm even though it already
// has.
func (n *TxNotifier) RegisterConf(ntfn *ConfNtfn) (*HistoricalConfDispatch, error) {
// NOTE: If the transaction/output script has already been included in a block
// on the chain, the confirmation details must be provided with the
// UpdateConfDetails method, otherwise we will wait for the transaction/output
// script to confirm even though it already has.
func (n *TxNotifier) RegisterConf(ntfn *ConfNtfn) (*HistoricalConfDispatch,
uint32, error) {
select {
case <-n.quit:
return nil, ErrTxNotifierExiting
return nil, 0, ErrTxNotifierExiting
default:
}
// Enforce that we will not dispatch confirmations beyond the reorg
// safety limit.
if ntfn.NumConfirmations > n.reorgSafetyLimit {
return nil, ErrTxMaxConfs
return nil, 0, ErrTxMaxConfs
}
// Before proceeding to register the notification, we'll query our
@ -332,77 +513,84 @@ func (n *TxNotifier) RegisterConf(ntfn *ConfNtfn) (*HistoricalConfDispatch, erro
//
// TODO(conner): verify that all submitted height hints are identical.
startHeight := ntfn.HeightHint
hint, err := n.confirmHintCache.QueryConfirmHint(*ntfn.TxID)
hint, err := n.confirmHintCache.QueryConfirmHint(ntfn.ConfRequest)
if err == nil {
if hint > startHeight {
Log.Debugf("Using height hint %d retrieved "+
"from cache for %v", hint, *ntfn.TxID)
Log.Debugf("Using height hint %d retrieved from cache "+
"for %v", hint, ntfn.ConfRequest)
startHeight = hint
}
} else if err != ErrConfirmHintNotFound {
Log.Errorf("Unable to query confirm hint for %v: %v",
*ntfn.TxID, err)
ntfn.ConfRequest, err)
}
n.Lock()
defer n.Unlock()
confSet, ok := n.confNotifications[*ntfn.TxID]
confSet, ok := n.confNotifications[ntfn.ConfRequest]
if !ok {
// If this is the first registration for this txid, construct a
// confSet to coalesce all notifications for the same txid.
// If this is the first registration for this request, construct
// a confSet to coalesce all notifications for the same request.
confSet = newConfNtfnSet()
n.confNotifications[*ntfn.TxID] = confSet
n.confNotifications[ntfn.ConfRequest] = confSet
}
confSet.ntfns[ntfn.ConfID] = ntfn
switch confSet.rescanStatus {
// A prior rescan has already completed and we are actively watching at
// tip for this txid.
// tip for this request.
case rescanComplete:
// If conf details for this set of notifications has already
// been found, we'll attempt to deliver them immediately to this
// client.
Log.Debugf("Attempting to dispatch conf for txid=%v "+
"on registration since rescan has finished", ntfn.TxID)
return nil, n.dispatchConfDetails(ntfn, confSet.details)
// If the confirmation details for this set of notifications has
// already been found, we'll attempt to deliver them immediately
// to this client.
Log.Debugf("Attempting to dispatch confirmation for %v on "+
"registration since rescan has finished",
ntfn.ConfRequest)
return nil, n.currentHeight, n.dispatchConfDetails(
ntfn, confSet.details,
)
// A rescan is already in progress, return here to prevent dispatching
// another. When the scan returns, this notifications details will be
// another. When the rescan returns, this notification's details will be
// updated as well.
case rescanPending:
Log.Debugf("Waiting for pending rescan to finish before "+
"notifying txid=%v at tip", ntfn.TxID)
return nil, nil
"notifying %v at tip", ntfn.ConfRequest)
return nil, n.currentHeight, nil
// If no rescan has been dispatched, attempt to do so now.
case rescanNotStarted:
}
// If the provided or cached height hint indicates that the transaction
// is to be confirmed at a height greater than the conf notifier's
// current height, we'll refrain from spawning a historical dispatch.
// If the provided or cached height hint indicates that the
// transaction with the given txid/output script is to be confirmed at a
// height greater than the notifier's current height, we'll refrain from
// spawning a historical dispatch.
if startHeight > n.currentHeight {
Log.Debugf("Height hint is above current height, not dispatching "+
"historical rescan for txid=%v ", ntfn.TxID)
// Set the rescan status to complete, which will allow the conf
Log.Debugf("Height hint is above current height, not "+
"dispatching historical confirmation rescan for %v",
ntfn.ConfRequest)
// Set the rescan status to complete, which will allow the
// notifier to start delivering messages for this set
// immediately.
confSet.rescanStatus = rescanComplete
return nil, nil
return nil, n.currentHeight, nil
}
Log.Debugf("Dispatching historical rescan for txid=%v ", ntfn.TxID)
Log.Debugf("Dispatching historical confirmation rescan for %v",
ntfn.ConfRequest)
// Construct the parameters for historical dispatch, scanning the range
// of blocks between our best known height hint and the notifier's
// current height. The notifier will begin also watching for
// confirmations at tip starting with the next block.
dispatch := &HistoricalConfDispatch{
TxID: ntfn.TxID,
PkScript: ntfn.PkScript,
ConfRequest: ntfn.ConfRequest,
StartHeight: startHeight,
EndHeight: n.currentHeight,
}
@ -411,16 +599,49 @@ func (n *TxNotifier) RegisterConf(ntfn *ConfNtfn) (*HistoricalConfDispatch, erro
// registrations don't also attempt a dispatch.
confSet.rescanStatus = rescanPending
return dispatch, nil
return dispatch, n.currentHeight, nil
}
// CancelConf cancels an existing request for a spend notification of an
// outpoint/output script. The request is identified by its spend ID.
func (n *TxNotifier) CancelConf(confRequest ConfRequest, confID uint64) {
select {
case <-n.quit:
return
default:
}
n.Lock()
defer n.Unlock()
confSet, ok := n.confNotifications[confRequest]
if !ok {
return
}
ntfn, ok := confSet.ntfns[confID]
if !ok {
return
}
Log.Infof("Canceling confirmation notification: conf_id=%d, %v", confID,
confRequest)
// We'll close all the notification channels to let the client know
// their cancel request has been fulfilled.
close(ntfn.Event.Confirmed)
close(ntfn.Event.Updates)
close(ntfn.Event.NegativeConf)
delete(confSet.ntfns, confID)
}
// UpdateConfDetails attempts to update the confirmation details for an active
// notification within the notifier. This should only be used in the case of a
// transaction that has confirmed before the notifier's current height.
// transaction/output script that has confirmed before the notifier's current
// height.
//
// NOTE: The notification should be registered first to ensure notifications are
// dispatched correctly.
func (n *TxNotifier) UpdateConfDetails(txid chainhash.Hash,
func (n *TxNotifier) UpdateConfDetails(confRequest ConfRequest,
details *TxConfirmation) error {
select {
@ -434,14 +655,15 @@ func (n *TxNotifier) UpdateConfDetails(txid chainhash.Hash,
n.Lock()
defer n.Unlock()
// First, we'll determine whether we have an active notification for
// this transaction with the given ID.
confSet, ok := n.confNotifications[txid]
// First, we'll determine whether we have an active confirmation
// notification for the given txid/script.
confSet, ok := n.confNotifications[confRequest]
if !ok {
return fmt.Errorf("no notification found with TxID %v", txid)
return fmt.Errorf("confirmation notification for %v not found",
confRequest)
}
// If the conf details were already found at tip, all existing
// If the confirmation details were already found at tip, all existing
// notifications will have been dispatched or queued for dispatch. We
// can exit early to avoid sending too many notifications on the
// buffered channels.
@ -455,43 +677,47 @@ func (n *TxNotifier) UpdateConfDetails(txid chainhash.Hash,
// continue to watch for them at tip.
confSet.rescanStatus = rescanComplete
// The notifier has yet to reach the height at which the transaction was
// included in a block, so we should defer until handling it then within
// ConnectTip.
// The notifier has yet to reach the height at which the
// transaction/output script was included in a block, so we should defer
// until handling it then within ConnectTip.
if details == nil {
Log.Debugf("Conf details for txid=%v not found during "+
"historical dispatch, waiting to dispatch at tip", txid)
Log.Debugf("Confirmation details for %v not found during "+
"historical dispatch, waiting to dispatch at tip",
confRequest)
// We'll commit the current height as the confirm hint to
// prevent another potentially long rescan if we restart before
// a new block comes in.
err := n.confirmHintCache.CommitConfirmHint(
n.currentHeight, txid,
n.currentHeight, confRequest,
)
if err != nil {
// The error is not fatal as this is an optimistic
// optimization, so we'll avoid returning an error.
Log.Debugf("Unable to update confirm hint to %d for "+
"%v: %v", n.currentHeight, txid, err)
"%v: %v", n.currentHeight, confRequest, err)
}
return nil
}
if details.BlockHeight > n.currentHeight {
Log.Debugf("Conf details for txid=%v found above current "+
"height, waiting to dispatch at tip", txid)
Log.Debugf("Confirmation details for %v found above current "+
"height, waiting to dispatch at tip", confRequest)
return nil
}
Log.Debugf("Updating conf details for txid=%v details", txid)
Log.Debugf("Updating confirmation details for %v", confRequest)
err := n.confirmHintCache.CommitConfirmHint(details.BlockHeight, txid)
err := n.confirmHintCache.CommitConfirmHint(
details.BlockHeight, confRequest,
)
if err != nil {
// The error is not fatal, so we should not return an error to
// the caller.
Log.Errorf("Unable to update confirm hint to %d for %v: %v",
details.BlockHeight, txid, err)
details.BlockHeight, confRequest, err)
}
// Cache the details found in the rescan and attempt to dispatch any
@ -508,30 +734,30 @@ func (n *TxNotifier) UpdateConfDetails(txid chainhash.Hash,
}
// dispatchConfDetails attempts to cache and dispatch details to a particular
// client if the transaction has sufficiently confirmed. If the provided details
// are nil, this method will be a no-op.
// client if the transaction/output script has sufficiently confirmed. If the
// provided details are nil, this method will be a no-op.
func (n *TxNotifier) dispatchConfDetails(
ntfn *ConfNtfn, details *TxConfirmation) error {
// If no details are provided, return early as we can't dispatch.
if details == nil {
Log.Debugf("Unable to dispatch %v, no details provided",
ntfn.TxID)
ntfn.ConfRequest)
return nil
}
// Now, we'll examine whether the transaction of this
// notification request has reached its required number of
// confirmations. If it has, we'll dispatch a confirmation
// notification to the caller.
// Now, we'll examine whether the transaction/output script of this
// request has reached its required number of confirmations. If it has,
// we'll dispatch a confirmation notification to the caller.
confHeight := details.BlockHeight + ntfn.NumConfirmations - 1
if confHeight <= n.currentHeight {
Log.Infof("Dispatching %v conf notification for %v",
ntfn.NumConfirmations, ntfn.TxID)
Log.Infof("Dispatching %v confirmation notification for %v",
ntfn.NumConfirmations, ntfn.ConfRequest)
// We'll send a 0 value to the Updates channel,
// indicating that the transaction has already been
// confirmed.
// indicating that the transaction/output script has already
// been confirmed.
select {
case ntfn.Event.Updates <- 0:
case <-n.quit:
@ -545,8 +771,8 @@ func (n *TxNotifier) dispatchConfDetails(
return ErrTxNotifierExiting
}
} else {
Log.Debugf("Queueing %v conf notification for %v at tip ",
ntfn.NumConfirmations, ntfn.TxID)
Log.Debugf("Queueing %v confirmation notification for %v at tip ",
ntfn.NumConfirmations, ntfn.ConfRequest)
// Otherwise, we'll keep track of the notification
// request by the height at which we should dispatch the
@ -559,8 +785,8 @@ func (n *TxNotifier) dispatchConfDetails(
ntfnSet[ntfn] = struct{}{}
// We'll also send an update to the client of how many
// confirmations are left for the transaction to be
// confirmed.
// confirmations are left for the transaction/output script to
// be confirmed.
numConfsLeft := confHeight - n.currentHeight
select {
case ntfn.Event.Updates <- numConfsLeft:
@ -569,70 +795,73 @@ func (n *TxNotifier) dispatchConfDetails(
}
}
// As a final check, we'll also watch the transaction if it's
// still possible for it to get reorged out of the chain.
blockHeight := details.BlockHeight
reorgSafeHeight := blockHeight + n.reorgSafetyLimit
// As a final check, we'll also watch the transaction/output script if
// it's still possible for it to get reorged out of the chain.
reorgSafeHeight := details.BlockHeight + n.reorgSafetyLimit
if reorgSafeHeight > n.currentHeight {
txSet, exists := n.txsByInitialHeight[blockHeight]
txSet, exists := n.confsByInitialHeight[details.BlockHeight]
if !exists {
txSet = make(map[chainhash.Hash]struct{})
n.txsByInitialHeight[blockHeight] = txSet
txSet = make(map[ConfRequest]struct{})
n.confsByInitialHeight[details.BlockHeight] = txSet
}
txSet[*ntfn.TxID] = struct{}{}
txSet[ntfn.ConfRequest] = struct{}{}
}
return nil
}
// RegisterSpend handles a new spend notification request. The client will be
// notified once the outpoint is detected as spent within the chain.
// notified once the outpoint/output script is detected as spent within the
// chain.
//
// The registration succeeds if no error is returned. If the returned
// HistoricalSpendDisaptch is non-nil, the caller is responsible for attempting
// to determine whether the outpoint has been spent between the start and end
// heights.
// to determine whether the outpoint/output script has been spent between the
// start and end heights. The notifier's current height is also returned so that
// backends can request to be notified of spends from this point forwards.
//
// NOTE: If the outpoint has already been spent within the chain before the
// notifier's current tip, the spend details must be provided with the
// UpdateSpendDetails method, otherwise we will wait for the outpoint to
// be spent at tip, even though it already has.
func (n *TxNotifier) RegisterSpend(ntfn *SpendNtfn) (*HistoricalSpendDispatch, error) {
// NOTE: If the outpoint/output script has already been spent within the chain
// before the notifier's current tip, the spend details must be provided with
// the UpdateSpendDetails method, otherwise we will wait for the outpoint/output
// script to be spent at tip, even though it already has.
func (n *TxNotifier) RegisterSpend(ntfn *SpendNtfn) (*HistoricalSpendDispatch,
uint32, error) {
select {
case <-n.quit:
return nil, ErrTxNotifierExiting
return nil, 0, ErrTxNotifierExiting
default:
}
// Before proceeding to register the notification, we'll query our spend
// hint cache to determine whether a better one exists.
startHeight := ntfn.HeightHint
hint, err := n.spendHintCache.QuerySpendHint(ntfn.OutPoint)
hint, err := n.spendHintCache.QuerySpendHint(ntfn.SpendRequest)
if err == nil {
if hint > startHeight {
Log.Debugf("Using height hint %d retrieved from cache "+
"for %v", startHeight, ntfn.OutPoint)
"for %v", startHeight, ntfn.SpendRequest)
startHeight = hint
}
} else if err != ErrSpendHintNotFound {
Log.Errorf("Unable to query spend hint for %v: %v",
ntfn.OutPoint, err)
ntfn.SpendRequest, err)
}
n.Lock()
defer n.Unlock()
Log.Infof("New spend subscription: spend_id=%d, outpoint=%v, "+
"height_hint=%d", ntfn.SpendID, ntfn.OutPoint, ntfn.HeightHint)
Log.Infof("New spend subscription: spend_id=%d, %v, height_hint=%d",
ntfn.SpendID, ntfn.SpendRequest, ntfn.HeightHint)
// Keep track of the notification request so that we can properly
// dispatch a spend notification later on.
spendSet, ok := n.spendNotifications[ntfn.OutPoint]
spendSet, ok := n.spendNotifications[ntfn.SpendRequest]
if !ok {
// If this is the first registration for the outpoint, we'll
// If this is the first registration for the request, we'll
// construct a spendNtfnSet to coalesce all notifications.
spendSet = newSpendNtfnSet()
n.spendNotifications[ntfn.OutPoint] = spendSet
n.spendNotifications[ntfn.SpendRequest] = spendSet
}
spendSet.ntfns[ntfn.SpendID] = ntfn
@ -640,19 +869,28 @@ func (n *TxNotifier) RegisterSpend(ntfn *SpendNtfn) (*HistoricalSpendDispatch, e
// depending on the current rescan status.
switch spendSet.rescanStatus {
// If the spending details for this outpoint have already been
// determined and cached, then we can use them to immediately dispatch
// the spend notification to the client.
// If the spending details for this request have already been determined
// and cached, then we can use them to immediately dispatch the spend
// notification to the client.
case rescanComplete:
return nil, n.dispatchSpendDetails(ntfn, spendSet.details)
Log.Debugf("Attempting to dispatch spend for %v on "+
"registration since rescan has finished",
ntfn.SpendRequest)
// If there is an active rescan to determine whether the outpoint has
return nil, n.currentHeight, n.dispatchSpendDetails(
ntfn, spendSet.details,
)
// If there is an active rescan to determine whether the request has
// been spent, then we won't trigger another one.
case rescanPending:
return nil, nil
Log.Debugf("Waiting for pending rescan to finish before "+
"notifying %v at tip", ntfn.SpendRequest)
return nil, n.currentHeight, nil
// Otherwise, we'll fall through and let the caller know that a rescan
// should be dispatched to determine whether the outpoint has already
// should be dispatched to determine whether the request has already
// been spent.
case rescanNotStarted:
}
@ -663,30 +901,32 @@ func (n *TxNotifier) RegisterSpend(ntfn *SpendNtfn) (*HistoricalSpendDispatch, e
// historical rescan and wait for the spend to come in at tip.
if startHeight > n.currentHeight {
Log.Debugf("Spend hint of %d for %v is above current height %d",
startHeight, ntfn.OutPoint, n.currentHeight)
startHeight, ntfn.SpendRequest, n.currentHeight)
// We'll also set the rescan status as complete to ensure that
// spend hints for this outpoint get updated upon
// spend hints for this request get updated upon
// connected/disconnected blocks.
spendSet.rescanStatus = rescanComplete
return nil, nil
return nil, n.currentHeight, nil
}
// We'll set the rescan status to pending to ensure subsequent
// notifications don't also attempt a historical dispatch.
spendSet.rescanStatus = rescanPending
Log.Debugf("Dispatching historical spend rescan for %v",
ntfn.SpendRequest)
return &HistoricalSpendDispatch{
OutPoint: ntfn.OutPoint,
PkScript: ntfn.PkScript,
StartHeight: startHeight,
EndHeight: n.currentHeight,
}, nil
SpendRequest: ntfn.SpendRequest,
StartHeight: startHeight,
EndHeight: n.currentHeight,
}, n.currentHeight, nil
}
// CancelSpend cancels an existing request for a spend notification of an
// outpoint. The request is identified by its spend ID.
func (n *TxNotifier) CancelSpend(op wire.OutPoint, spendID uint64) {
// outpoint/output script. The request is identified by its spend ID.
func (n *TxNotifier) CancelSpend(spendRequest SpendRequest, spendID uint64) {
select {
case <-n.quit:
return
@ -696,10 +936,7 @@ func (n *TxNotifier) CancelSpend(op wire.OutPoint, spendID uint64) {
n.Lock()
defer n.Unlock()
Log.Infof("Canceling spend notification: spend_id=%d, outpoint=%v",
spendID, op)
spendSet, ok := n.spendNotifications[op]
spendSet, ok := n.spendNotifications[spendRequest]
if !ok {
return
}
@ -708,18 +945,24 @@ func (n *TxNotifier) CancelSpend(op wire.OutPoint, spendID uint64) {
return
}
Log.Infof("Canceling spend notification: spend_id=%d, %v", spendID,
spendRequest)
// We'll close all the notification channels to let the client know
// their cancel request has been fulfilled.
close(ntfn.Event.Spend)
close(ntfn.Event.Reorg)
close(ntfn.Event.Done)
delete(spendSet.ntfns, spendID)
}
// ProcessRelevantSpendTx processes a transaction provided externally. This will
// check whether the transaction is relevant to the notifier if it spends any
// outputs for which we currently have registered notifications for. If it is
// relevant, spend notifications will be dispatched to the caller.
func (n *TxNotifier) ProcessRelevantSpendTx(tx *wire.MsgTx, txHeight int32) error {
// outpoints/output scripts for which we currently have registered notifications
// for. If it is relevant, spend notifications will be dispatched to the caller.
func (n *TxNotifier) ProcessRelevantSpendTx(tx *btcutil.Tx,
blockHeight uint32) error {
select {
case <-n.quit:
return ErrTxNotifierExiting
@ -731,31 +974,26 @@ func (n *TxNotifier) ProcessRelevantSpendTx(tx *wire.MsgTx, txHeight int32) erro
n.Lock()
defer n.Unlock()
// Grab the set of active registered outpoints to determine if the
// transaction spends any of them.
spendNtfns := n.spendNotifications
// We'll use a channel to coalesce all the spend requests that this
// transaction fulfills.
type spend struct {
request *SpendRequest
details *SpendDetail
}
// We'll check if this transaction spends an output that has an existing
// spend notification for it.
for i, txIn := range tx.TxIn {
// If this input doesn't spend an existing registered outpoint,
// we'll go on to the next.
prevOut := txIn.PreviousOutPoint
if _, ok := spendNtfns[prevOut]; !ok {
continue
}
// We'll set up the onSpend filter callback to gather all the fulfilled
// spends requests within this transaction.
var spends []spend
onSpend := func(request SpendRequest, details *SpendDetail) {
spends = append(spends, spend{&request, details})
}
n.filterTx(tx, nil, blockHeight, nil, onSpend)
// Otherwise, we'll create a spend summary and send off the
// details to the notification subscribers.
txHash := tx.TxHash()
details := &SpendDetail{
SpentOutPoint: &prevOut,
SpenderTxHash: &txHash,
SpendingTx: tx,
SpenderInputIndex: uint32(i),
SpendingHeight: txHeight,
}
if err := n.updateSpendDetails(prevOut, details); err != nil {
// After the transaction has been filtered, we can finally dispatch
// notifications for each request.
for _, spend := range spends {
err := n.updateSpendDetails(*spend.request, spend.details)
if err != nil {
return err
}
}
@ -764,13 +1002,13 @@ func (n *TxNotifier) ProcessRelevantSpendTx(tx *wire.MsgTx, txHeight int32) erro
}
// UpdateSpendDetails attempts to update the spend details for all active spend
// notification requests for an outpoint. This method should be used once a
// historical scan of the chain has finished. If the historical scan did not
// find a spending transaction for the outpoint, the spend details may be nil.
// notification requests for an outpoint/output script. This method should be
// used once a historical scan of the chain has finished. If the historical scan
// did not find a spending transaction for it, the spend details may be nil.
//
// NOTE: A notification request for the outpoint must be registered first to
// ensure notifications are delivered.
func (n *TxNotifier) UpdateSpendDetails(op wire.OutPoint,
// NOTE: A notification request for the outpoint/output script must be
// registered first to ensure notifications are delivered.
func (n *TxNotifier) UpdateSpendDetails(spendRequest SpendRequest,
details *SpendDetail) error {
select {
@ -784,24 +1022,24 @@ func (n *TxNotifier) UpdateSpendDetails(op wire.OutPoint,
n.Lock()
defer n.Unlock()
return n.updateSpendDetails(op, details)
return n.updateSpendDetails(spendRequest, details)
}
// updateSpendDetails attempts to update the spend details for all active spend
// notification requests for an outpoint. This method should be used once a
// historical scan of the chain has finished. If the historical scan did not
// find a spending transaction for the outpoint, the spend details may be nil.
// notification requests for an outpoint/output script. This method should be
// used once a historical scan of the chain has finished. If the historical scan
// did not find a spending transaction for it, the spend details may be nil.
//
// NOTE: This method must be called with the TxNotifier's lock held.
func (n *TxNotifier) updateSpendDetails(op wire.OutPoint,
func (n *TxNotifier) updateSpendDetails(spendRequest SpendRequest,
details *SpendDetail) error {
// Mark the ongoing historical rescan for this outpoint as finished.
// This will allow us to update the spend hints for this outpoint at
// tip.
spendSet, ok := n.spendNotifications[op]
// Mark the ongoing historical rescan for this request as finished. This
// will allow us to update the spend hints for it at tip.
spendSet, ok := n.spendNotifications[spendRequest]
if !ok {
return fmt.Errorf("no notifications found for outpoint %v", op)
return fmt.Errorf("spend notification for %v not found",
spendRequest)
}
// If the spend details have already been found either at tip, then the
@ -811,31 +1049,33 @@ func (n *TxNotifier) updateSpendDetails(op wire.OutPoint,
return nil
}
// Since the historical rescan has completed for this outpoint, we'll
// Since the historical rescan has completed for this request, we'll
// mark its rescan status as complete in order to ensure that the
// TxNotifier can properly update its spend hints upon
// connected/disconnected blocks.
spendSet.rescanStatus = rescanComplete
// If the historical rescan was not able to find a spending transaction
// for this outpoint, then we can track the spend at tip.
// for this request, then we can track the spend at tip.
if details == nil {
// We'll commit the current height as the spend hint to prevent
// another potentially long rescan if we restart before a new
// block comes in.
err := n.spendHintCache.CommitSpendHint(n.currentHeight, op)
err := n.spendHintCache.CommitSpendHint(
n.currentHeight, spendRequest,
)
if err != nil {
// The error is not fatal as this is an optimistic
// optimization, so we'll avoid returning an error.
Log.Debugf("Unable to update spend hint to %d for %v: %v",
n.currentHeight, op, err)
n.currentHeight, spendRequest, err)
}
return nil
}
// If the historical rescan found the spending transaction for this
// outpoint, but it's at a later height than the notifier (this can
// request, but it's at a later height than the notifier (this can
// happen due to latency with the backend during a reorg), then we'll
// defer handling the notification until the notifier has caught up to
// such height.
@ -843,17 +1083,17 @@ func (n *TxNotifier) updateSpendDetails(op wire.OutPoint,
return nil
}
// Now that we've determined the outpoint has been spent, we'll commit
// Now that we've determined the request has been spent, we'll commit
// its spending height as its hint in the cache and dispatch
// notifications to all of its respective clients.
err := n.spendHintCache.CommitSpendHint(
uint32(details.SpendingHeight), op,
uint32(details.SpendingHeight), spendRequest,
)
if err != nil {
// The error is not fatal as this is an optimistic optimization,
// so we'll avoid returning an error.
Log.Debugf("Unable to update spend hint to %d for %v: %v",
details.SpendingHeight, op, err)
details.SpendingHeight, spendRequest, err)
}
spendSet.details = details
@ -877,8 +1117,8 @@ func (n *TxNotifier) dispatchSpendDetails(ntfn *SpendNtfn, details *SpendDetail)
return nil
}
Log.Infof("Dispatching spend notification for outpoint=%v at height=%d",
ntfn.OutPoint, n.currentHeight)
Log.Infof("Dispatching confirmed spend notification for %v at height=%d",
ntfn.SpendRequest, n.currentHeight)
select {
case ntfn.Event.Spend <- details:
@ -894,16 +1134,17 @@ func (n *TxNotifier) dispatchSpendDetails(ntfn *SpendNtfn, details *SpendDetail)
// through every transaction and determine if it is relevant to any of its
// clients. A transaction can be relevant in either of the following two ways:
//
// 1. One of the inputs in the transaction spends an outpoint for which we
// currently have an active spend registration for.
// 1. One of the inputs in the transaction spends an outpoint/output script
// for which we currently have an active spend registration for.
//
// 2. The transaction is a transaction for which we currently have an active
// confirmation registration for.
// 2. The transaction has a txid or output script for which we currently have
// an active confirmation registration for.
//
// In the event that the transaction is relevant, a confirmation/spend
// notification will be queued for dispatch to the relevant clients.
// Confirmation notifications will only be dispatched for transactions that have
// met the required number of confirmations required by the client.
// Confirmation notifications will only be dispatched for transactions/output
// scripts that have met the required number of confirmations required by the
// client.
//
// NOTE: In order to actually dispatch the relevant transaction notifications to
// clients, NotifyHeight must be called with the same block height in order to
@ -921,7 +1162,7 @@ func (n *TxNotifier) ConnectTip(blockHash *chainhash.Hash, blockHeight uint32,
defer n.Unlock()
if blockHeight != n.currentHeight+1 {
return fmt.Errorf("Received blocks out of order: "+
return fmt.Errorf("received blocks out of order: "+
"current height=%d, new height=%d",
n.currentHeight, blockHeight)
}
@ -931,117 +1172,238 @@ func (n *TxNotifier) ConnectTip(blockHash *chainhash.Hash, blockHeight uint32,
// First, we'll iterate over all the transactions found in this block to
// determine if it includes any relevant transactions to the TxNotifier.
for _, tx := range txns {
txHash := tx.Hash()
n.filterTx(
tx, blockHash, blockHeight, n.handleConfDetailsAtTip,
n.handleSpendDetailsAtTip,
)
}
// In order to determine if this transaction is relevant to the
// notifier, we'll check its inputs for any outstanding spend
// notifications.
for i, txIn := range tx.MsgTx().TxIn {
prevOut := txIn.PreviousOutPoint
spendSet, ok := n.spendNotifications[prevOut]
if !ok {
continue
}
// Now that we've determined which requests were confirmed and spent
// within the new block, we can update their entries in their respective
// caches, along with all of our unconfirmed and unspent requests.
n.updateHints(blockHeight)
// If we have any, we'll record its spend height so that
// notifications get dispatched to the respective
// clients.
spendDetails := &SpendDetail{
SpentOutPoint: &prevOut,
SpenderTxHash: txHash,
SpendingTx: tx.MsgTx(),
SpenderInputIndex: uint32(i),
SpendingHeight: int32(blockHeight),
}
// TODO(wilmer): cancel pending historical rescans if any?
spendSet.rescanStatus = rescanComplete
spendSet.details = spendDetails
for _, ntfn := range spendSet.ntfns {
// In the event that this notification was aware
// that the spending transaction of its outpoint
// was reorged out of the chain, we'll consume
// the reorg notification if it hasn't been
// done yet already.
// Finally, we'll clear the entries from our set of notifications for
// requests that are no longer under the risk of being reorged out of
// the chain.
if blockHeight >= n.reorgSafetyLimit {
matureBlockHeight := blockHeight - n.reorgSafetyLimit
for confRequest := range n.confsByInitialHeight[matureBlockHeight] {
confSet := n.confNotifications[confRequest]
for _, ntfn := range confSet.ntfns {
select {
case <-ntfn.Event.Reorg:
default:
case ntfn.Event.Done <- struct{}{}:
case <-n.quit:
return ErrTxNotifierExiting
}
}
// We'll note the outpoints spending height in order to
// correctly handle dispatching notifications when the
// spending transactions gets reorged out of the chain.
opSet, exists := n.opsBySpendHeight[blockHeight]
if !exists {
opSet = make(map[wire.OutPoint]struct{})
n.opsBySpendHeight[blockHeight] = opSet
delete(n.confNotifications, confRequest)
}
delete(n.confsByInitialHeight, matureBlockHeight)
for spendRequest := range n.spendsByHeight[matureBlockHeight] {
spendSet := n.spendNotifications[spendRequest]
for _, ntfn := range spendSet.ntfns {
select {
case ntfn.Event.Done <- struct{}{}:
case <-n.quit:
return ErrTxNotifierExiting
}
}
opSet[prevOut] = struct{}{}
delete(n.spendNotifications, spendRequest)
}
delete(n.spendsByHeight, matureBlockHeight)
}
return nil
}
// filterTx determines whether the transaction spends or confirms any
// outstanding pending requests. The onConf and onSpend callbacks can be used to
// retrieve all the requests fulfilled by this transaction as they occur.
func (n *TxNotifier) filterTx(tx *btcutil.Tx, blockHash *chainhash.Hash,
blockHeight uint32, onConf func(ConfRequest, *TxConfirmation),
onSpend func(SpendRequest, *SpendDetail)) {
// In order to determine if this transaction is relevant to the
// notifier, we'll check its inputs for any outstanding spend
// requests.
txHash := tx.Hash()
if onSpend != nil {
// notifyDetails is a helper closure that will construct the
// spend details of a request and hand them off to the onSpend
// callback.
notifyDetails := func(spendRequest SpendRequest,
prevOut wire.OutPoint, inputIdx uint32) {
Log.Debugf("Found spend of %v: spend_tx=%v, "+
"block_height=%d", spendRequest, txHash,
blockHeight)
onSpend(spendRequest, &SpendDetail{
SpentOutPoint: &prevOut,
SpenderTxHash: txHash,
SpendingTx: tx.MsgTx(),
SpenderInputIndex: inputIdx,
SpendingHeight: int32(blockHeight),
})
}
// Check if we have any pending notifications for this txid. If
// none are found, we can proceed to the next transaction.
confSet, ok := n.confNotifications[*txHash]
if !ok {
continue
}
Log.Debugf("Block contains txid=%v, constructing details",
txHash)
// If we have any, we'll record its confirmed height so that
// notifications get dispatched when the transaction reaches the
// clients' desired number of confirmations.
details := &TxConfirmation{
BlockHash: blockHash,
BlockHeight: blockHeight,
TxIndex: uint32(tx.Index()),
}
// TODO(wilmer): cancel pending historical rescans if any?
confSet.rescanStatus = rescanComplete
confSet.details = details
for _, ntfn := range confSet.ntfns {
// In the event that this notification was aware that
// the transaction was reorged out of the chain, we'll
// consume the reorg notification if it hasn't been done
// yet already.
select {
case <-ntfn.Event.NegativeConf:
default:
for i, txIn := range tx.MsgTx().TxIn {
// We'll re-derive the script of the output being spent
// to determine if the inputs spends any registered
// requests.
prevOut := txIn.PreviousOutPoint
pkScript, err := txscript.ComputePkScript(
txIn.SignatureScript, txIn.Witness,
)
if err != nil {
continue
}
spendRequest := SpendRequest{
OutPoint: prevOut,
PkScript: pkScript,
}
// We'll note this client's required number of
// confirmations so that we can notify them when
// expected.
confHeight := blockHeight + ntfn.NumConfirmations - 1
ntfnSet, exists := n.ntfnsByConfirmHeight[confHeight]
if !exists {
ntfnSet = make(map[*ConfNtfn]struct{})
n.ntfnsByConfirmHeight[confHeight] = ntfnSet
// If we have any, we'll record their spend height so
// that notifications get dispatched to the respective
// clients.
if _, ok := n.spendNotifications[spendRequest]; ok {
notifyDetails(spendRequest, prevOut, uint32(i))
}
ntfnSet[ntfn] = struct{}{}
// We'll also note the initial confirmation height in
// order to correctly handle dispatching notifications
// when the transaction gets reorged out of the chain.
txSet, exists := n.txsByInitialHeight[blockHeight]
if !exists {
txSet = make(map[chainhash.Hash]struct{})
n.txsByInitialHeight[blockHeight] = txSet
spendRequest.OutPoint = ZeroOutPoint
if _, ok := n.spendNotifications[spendRequest]; ok {
notifyDetails(spendRequest, prevOut, uint32(i))
}
txSet[*txHash] = struct{}{}
}
}
// Finally, now that we've determined which transactions were confirmed
// and which outpoints were spent within the new block, we can update
// their entries in their respective caches, along with all of our
// unconfirmed transactions and unspent outpoints.
n.updateHints(blockHeight)
// We'll also check its outputs to determine if there are any
// outstanding confirmation requests.
if onConf != nil {
// notifyDetails is a helper closure that will construct the
// confirmation details of a request and hand them off to the
// onConf callback.
notifyDetails := func(confRequest ConfRequest) {
Log.Debugf("Found initial confirmation of %v: "+
"height=%d, hash=%v", confRequest,
blockHeight, blockHash)
return nil
details := &TxConfirmation{
Tx: tx.MsgTx(),
BlockHash: blockHash,
BlockHeight: blockHeight,
TxIndex: uint32(tx.Index()),
}
onConf(confRequest, details)
}
for _, txOut := range tx.MsgTx().TxOut {
// We'll parse the script of the output to determine if
// we have any registered requests for it or the
// transaction itself.
pkScript, err := txscript.ParsePkScript(txOut.PkScript)
if err != nil {
continue
}
confRequest := ConfRequest{
TxID: *txHash,
PkScript: pkScript,
}
// If we have any, we'll record their confirmed height
// so that notifications get dispatched when they
// reaches the clients' desired number of confirmations.
if _, ok := n.confNotifications[confRequest]; ok {
notifyDetails(confRequest)
}
confRequest.TxID = ZeroHash
if _, ok := n.confNotifications[confRequest]; ok {
notifyDetails(confRequest)
}
}
}
}
// handleConfDetailsAtTip tracks the confirmation height of the txid/output
// script in order to properly dispatch a confirmation notification after
// meeting each request's desired number of confirmations for all current and
// future registered clients.
func (n *TxNotifier) handleConfDetailsAtTip(confRequest ConfRequest,
details *TxConfirmation) {
// TODO(wilmer): cancel pending historical rescans if any?
confSet := n.confNotifications[confRequest]
confSet.rescanStatus = rescanComplete
confSet.details = details
for _, ntfn := range confSet.ntfns {
// In the event that this notification was aware that the
// transaction/output script was reorged out of the chain, we'll
// consume the reorg notification if it hasn't been done yet
// already.
select {
case <-ntfn.Event.NegativeConf:
default:
}
// We'll note this client's required number of confirmations so
// that we can notify them when expected.
confHeight := details.BlockHeight + ntfn.NumConfirmations - 1
ntfnSet, exists := n.ntfnsByConfirmHeight[confHeight]
if !exists {
ntfnSet = make(map[*ConfNtfn]struct{})
n.ntfnsByConfirmHeight[confHeight] = ntfnSet
}
ntfnSet[ntfn] = struct{}{}
}
// We'll also note the initial confirmation height in order to correctly
// handle dispatching notifications when the transaction/output script
// gets reorged out of the chain.
txSet, exists := n.confsByInitialHeight[details.BlockHeight]
if !exists {
txSet = make(map[ConfRequest]struct{})
n.confsByInitialHeight[details.BlockHeight] = txSet
}
txSet[confRequest] = struct{}{}
}
// handleSpendDetailsAtTip tracks the spend height of the outpoint/output script
// in order to properly dispatch a spend notification for all current and future
// registered clients.
func (n *TxNotifier) handleSpendDetailsAtTip(spendRequest SpendRequest,
details *SpendDetail) {
// TODO(wilmer): cancel pending historical rescans if any?
spendSet := n.spendNotifications[spendRequest]
spendSet.rescanStatus = rescanComplete
spendSet.details = details
for _, ntfn := range spendSet.ntfns {
// In the event that this notification was aware that the
// spending transaction of its outpoint/output script was
// reorged out of the chain, we'll consume the reorg
// notification if it hasn't been done yet already.
select {
case <-ntfn.Event.Reorg:
default:
}
}
// We'll note the spending height of the request in order to correctly
// handle dispatching notifications when the spending transactions gets
// reorged out of the chain.
spendHeight := uint32(details.SpendingHeight)
opSet, exists := n.spendsByHeight[spendHeight]
if !exists {
opSet = make(map[SpendRequest]struct{})
n.spendsByHeight[spendHeight] = opSet
}
opSet[spendRequest] = struct{}{}
}
// NotifyHeight dispatches confirmation and spend notifications to the clients
@ -1052,21 +1414,21 @@ func (n *TxNotifier) NotifyHeight(height uint32) error {
defer n.Unlock()
// First, we'll dispatch an update to all of the notification clients
// for our watched transactions with the number of confirmations left at
// for our watched requests with the number of confirmations left at
// this new height.
for _, txHashes := range n.txsByInitialHeight {
for txHash := range txHashes {
confSet := n.confNotifications[txHash]
for _, confRequests := range n.confsByInitialHeight {
for confRequest := range confRequests {
confSet := n.confNotifications[confRequest]
for _, ntfn := range confSet.ntfns {
txConfHeight := confSet.details.BlockHeight +
ntfn.NumConfirmations - 1
numConfsLeft := txConfHeight - height
// Since we don't clear notifications until
// transactions are no longer under the risk of
// being reorganized out of the chain, we'll
// skip sending updates for transactions that
// have already been confirmed.
// transactions/output scripts are no longer
// under the risk of being reorganized out of
// the chain, we'll skip sending updates for
// those that have already been confirmed.
if int32(numConfsLeft) < 0 {
continue
}
@ -1080,13 +1442,13 @@ func (n *TxNotifier) NotifyHeight(height uint32) error {
}
}
// Then, we'll dispatch notifications for all the transactions that have
// Then, we'll dispatch notifications for all the requests that have
// become confirmed at this new block height.
for ntfn := range n.ntfnsByConfirmHeight[height] {
confSet := n.confNotifications[*ntfn.TxID]
confSet := n.confNotifications[ntfn.ConfRequest]
Log.Infof("Dispatching %v conf notification for %v",
ntfn.NumConfirmations, ntfn.TxID)
Log.Infof("Dispatching %v confirmation notification for %v",
ntfn.NumConfirmations, ntfn.ConfRequest)
select {
case ntfn.Event.Confirmed <- confSet.details:
@ -1097,10 +1459,10 @@ func (n *TxNotifier) NotifyHeight(height uint32) error {
}
delete(n.ntfnsByConfirmHeight, height)
// We'll also dispatch spend notifications for all the outpoints that
// Finally, we'll dispatch spend notifications for all the requests that
// were spent at this new block height.
for op := range n.opsBySpendHeight[height] {
spendSet := n.spendNotifications[op]
for spendRequest := range n.spendsByHeight[height] {
spendSet := n.spendNotifications[spendRequest]
for _, ntfn := range spendSet.ntfns {
err := n.dispatchSpendDetails(ntfn, spendSet.details)
if err != nil {
@ -1109,30 +1471,15 @@ func (n *TxNotifier) NotifyHeight(height uint32) error {
}
}
// Finally, we'll clear the entries from our set of notifications for
// transactions and outpoints that are no longer under the risk of being
// reorged out of the chain.
if height >= n.reorgSafetyLimit {
matureBlockHeight := height - n.reorgSafetyLimit
for tx := range n.txsByInitialHeight[matureBlockHeight] {
delete(n.confNotifications, tx)
}
delete(n.txsByInitialHeight, matureBlockHeight)
for op := range n.opsBySpendHeight[matureBlockHeight] {
delete(n.spendNotifications, op)
}
delete(n.opsBySpendHeight, matureBlockHeight)
}
return nil
}
// DisconnectTip handles the tip of the current chain being disconnected during
// a chain reorganization. If any watched transactions or spending transactions
// for registered outpoints were included in this block, internal structures are
// updated to ensure confirmation/spend notifications are consumed (if not
// already), and reorg notifications are dispatched instead. Confirmation/spend
// notifications will be dispatched again upon block inclusion.
// a chain reorganization. If any watched requests were included in this block,
// internal structures are updated to ensure confirmation/spend notifications
// are consumed (if not already), and reorg notifications are dispatched
// instead. Confirmation/spend notifications will be dispatched again upon block
// inclusion.
func (n *TxNotifier) DisconnectTip(blockHeight uint32) error {
select {
case <-n.quit:
@ -1152,19 +1499,19 @@ func (n *TxNotifier) DisconnectTip(blockHeight uint32) error {
n.reorgDepth++
// With the block disconnected, we'll update the confirm and spend hints
// for our transactions and outpoints to reflect the new height, except
// for those that have confirmed/spent at previous heights.
// for our notification requests to reflect the new height, except for
// those that have confirmed/spent at previous heights.
n.updateHints(blockHeight)
// We'll go through all of our watched transactions and attempt to drain
// their notification channels to ensure sending notifications to the
// clients is always non-blocking.
for initialHeight, txHashes := range n.txsByInitialHeight {
// We'll go through all of our watched confirmation requests and attempt
// to drain their notification channels to ensure sending notifications
// to the clients is always non-blocking.
for initialHeight, txHashes := range n.confsByInitialHeight {
for txHash := range txHashes {
// If the transaction has been reorged out of the chain,
// we'll make sure to remove the cached confirmation
// details to prevent notifying clients with old
// information.
// If the transaction/output script has been reorged out
// of the chain, we'll make sure to remove the cached
// confirmation details to prevent notifying clients
// with old information.
confSet := n.confNotifications[txHash]
if initialHeight == blockHeight {
confSet.details = nil
@ -1181,10 +1528,11 @@ func (n *TxNotifier) DisconnectTip(blockHeight uint32) error {
default:
}
// Then, we'll check if the current transaction
// was included in the block currently being
// disconnected. If it was, we'll need to
// dispatch a reorg notification to the client.
// Then, we'll check if the current
// transaction/output script was included in the
// block currently being disconnected. If it
// was, we'll need to dispatch a reorg
// notification to the client.
if initialHeight == blockHeight {
err := n.dispatchConfReorg(
ntfn, blockHeight,
@ -1197,15 +1545,15 @@ func (n *TxNotifier) DisconnectTip(blockHeight uint32) error {
}
}
// We'll also go through our watched outpoints and attempt to drain
// We'll also go through our watched spend requests and attempt to drain
// their dispatched notifications to ensure dispatching notifications to
// clients later on is always non-blocking. We're only interested in
// outpoints whose spending transaction was included at the height being
// clients later on is always non-blocking. We're only interested in
// requests whose spending transaction was included at the height being
// disconnected.
for op := range n.opsBySpendHeight[blockHeight] {
for op := range n.spendsByHeight[blockHeight] {
// Since the spending transaction is being reorged out of the
// chain, we'll need to clear out the spending details of the
// outpoint.
// request.
spendSet := n.spendNotifications[op]
spendSet.details = nil
@ -1219,70 +1567,68 @@ func (n *TxNotifier) DisconnectTip(blockHeight uint32) error {
}
}
// Finally, we can remove the transactions that were confirmed and the
// outpoints that were spent at the height being disconnected. We'll
// still continue to track them until they have been confirmed/spent and
// are no longer under the risk of being reorged out of the chain again.
delete(n.txsByInitialHeight, blockHeight)
delete(n.opsBySpendHeight, blockHeight)
// Finally, we can remove the requests that were confirmed and/or spent
// at the height being disconnected. We'll still continue to track them
// until they have been confirmed/spent and are no longer under the risk
// of being reorged out of the chain again.
delete(n.confsByInitialHeight, blockHeight)
delete(n.spendsByHeight, blockHeight)
return nil
}
// updateHints attempts to update the confirm and spend hints for all relevant
// transactions and outpoints respectively. The height parameter is used to
// determine which transactions and outpoints we should update based on whether
// a new block is being connected/disconnected.
// requests respectively. The height parameter is used to determine which
// requests we should update based on whether a new block is being
// connected/disconnected.
//
// NOTE: This must be called with the TxNotifier's lock held and after its
// height has already been reflected by a block being connected/disconnected.
func (n *TxNotifier) updateHints(height uint32) {
// TODO(wilmer): update under one database transaction.
//
// To update the height hint for all the required transactions under one
// database transaction, we'll gather the set of unconfirmed
// transactions along with the ones that confirmed at the height being
// To update the height hint for all the required confirmation requests
// under one database transaction, we'll gather the set of unconfirmed
// requests along with the ones that confirmed at the height being
// connected/disconnected.
txsToUpdateHints := n.unconfirmedTxs()
for confirmedTx := range n.txsByInitialHeight[height] {
txsToUpdateHints = append(txsToUpdateHints, confirmedTx)
confRequests := n.unconfirmedRequests()
for confRequest := range n.confsByInitialHeight[height] {
confRequests = append(confRequests, confRequest)
}
err := n.confirmHintCache.CommitConfirmHint(
n.currentHeight, txsToUpdateHints...,
n.currentHeight, confRequests...,
)
if err != nil {
// The error is not fatal as this is an optimistic optimization,
// so we'll avoid returning an error.
Log.Debugf("Unable to update confirm hints to %d for "+
"%v: %v", n.currentHeight, txsToUpdateHints, err)
"%v: %v", n.currentHeight, confRequests, err)
}
// Similarly, to update the height hint for all the required outpoints
// under one database transaction, we'll gather the set of unspent
// outpoints along with the ones that were spent at the height being
// connected/disconnected.
opsToUpdateHints := n.unspentOutPoints()
for spentOp := range n.opsBySpendHeight[height] {
opsToUpdateHints = append(opsToUpdateHints, spentOp)
// Similarly, to update the height hint for all the required spend
// requests under one database transaction, we'll gather the set of
// unspent requests along with the ones that were spent at the height
// being connected/disconnected.
spendRequests := n.unspentRequests()
for spendRequest := range n.spendsByHeight[height] {
spendRequests = append(spendRequests, spendRequest)
}
err = n.spendHintCache.CommitSpendHint(
n.currentHeight, opsToUpdateHints...,
)
err = n.spendHintCache.CommitSpendHint(n.currentHeight, spendRequests...)
if err != nil {
// The error is not fatal as this is an optimistic optimization,
// so we'll avoid returning an error.
Log.Debugf("Unable to update spend hints to %d for "+
"%v: %v", n.currentHeight, opsToUpdateHints, err)
"%v: %v", n.currentHeight, spendRequests, err)
}
}
// unconfirmedTxs returns the set of transactions that are still seen as
// unconfirmed by the TxNotifier.
// unconfirmedRequests returns the set of confirmation requests that are
// still seen as unconfirmed by the TxNotifier.
//
// NOTE: This method must be called with the TxNotifier's lock held.
func (n *TxNotifier) unconfirmedTxs() []chainhash.Hash {
var unconfirmedTxs []chainhash.Hash
for tx, confNtfnSet := range n.confNotifications {
func (n *TxNotifier) unconfirmedRequests() []ConfRequest {
var unconfirmed []ConfRequest
for confRequest, confNtfnSet := range n.confNotifications {
// If the notification is already aware of its confirmation
// details, or it's in the process of learning them, we'll skip
// it as we can't yet determine if it's confirmed or not.
@ -1291,19 +1637,19 @@ func (n *TxNotifier) unconfirmedTxs() []chainhash.Hash {
continue
}
unconfirmedTxs = append(unconfirmedTxs, tx)
unconfirmed = append(unconfirmed, confRequest)
}
return unconfirmedTxs
return unconfirmed
}
// unspentOutPoints returns the set of outpoints that are still seen as unspent
// by the TxNotifier.
// unspentRequests returns the set of spend requests that are still seen as
// unspent by the TxNotifier.
//
// NOTE: This method must be called with the TxNotifier's lock held.
func (n *TxNotifier) unspentOutPoints() []wire.OutPoint {
var unspentOps []wire.OutPoint
for op, spendNtfnSet := range n.spendNotifications {
func (n *TxNotifier) unspentRequests() []SpendRequest {
var unspent []SpendRequest
for spendRequest, spendNtfnSet := range n.spendNotifications {
// If the notification is already aware of its spend details, or
// it's in the process of learning them, we'll skip it as we
// can't yet determine if it's unspent or not.
@ -1312,10 +1658,10 @@ func (n *TxNotifier) unspentOutPoints() []wire.OutPoint {
continue
}
unspentOps = append(unspentOps, op)
unspent = append(unspent, spendRequest)
}
return unspentOps
return unspent
}
// dispatchConfReorg dispatches a reorg notification to the client if the
@ -1325,10 +1671,10 @@ func (n *TxNotifier) unspentOutPoints() []wire.OutPoint {
func (n *TxNotifier) dispatchConfReorg(ntfn *ConfNtfn,
heightDisconnected uint32) error {
// If the transaction's confirmation notification has yet to be
// dispatched, we'll need to clear its entry within the
// ntfnsByConfirmHeight index to prevent from notifying the client once
// the notifier reaches the confirmation height.
// If the request's confirmation notification has yet to be dispatched,
// we'll need to clear its entry within the ntfnsByConfirmHeight index
// to prevent from notifying the client once the notifier reaches the
// confirmation height.
if !ntfn.dispatched {
confHeight := heightDisconnected + ntfn.NumConfirmations - 1
ntfnSet, exists := n.ntfnsByConfirmHeight[confHeight]
@ -1404,6 +1750,7 @@ func (n *TxNotifier) TearDown() {
close(ntfn.Event.Confirmed)
close(ntfn.Event.Updates)
close(ntfn.Event.NegativeConf)
close(ntfn.Event.Done)
}
}
@ -1411,6 +1758,7 @@ func (n *TxNotifier) TearDown() {
for _, ntfn := range spendSet.ntfns {
close(ntfn.Event.Spend)
close(ntfn.Event.Reorg)
close(ntfn.Event.Done)
}
}
}

@ -1,45 +1,68 @@
package chainntnfs_test
import (
"bytes"
"sync"
"testing"
"github.com/btcsuite/btcd/chaincfg/chainhash"
"github.com/btcsuite/btcd/txscript"
"github.com/btcsuite/btcd/wire"
"github.com/btcsuite/btcutil"
"github.com/lightningnetwork/lnd/chainntnfs"
)
var (
zeroHash chainhash.Hash
zeroOutPoint wire.OutPoint
testRawScript = []byte{
// OP_HASH160
0xa9,
// OP_DATA_20
0x14,
// <20-byte script hash>
0x90, 0x1c, 0x86, 0x94, 0xc0, 0x3f, 0xaf, 0xd5,
0x52, 0x28, 0x10, 0xe0, 0x33, 0x0f, 0x26, 0xe6,
0x7a, 0x85, 0x33, 0xcd,
// OP_EQUAL
0x87,
}
testSigScript = []byte{
// OP_DATA_16
0x16,
// <22-byte redeem script>
0x00, 0x14, 0x1d, 0x7c, 0xd6, 0xc7, 0x5c, 0x2e,
0x86, 0xf4, 0xcb, 0xf9, 0x8e, 0xae, 0xd2, 0x21,
0xb3, 0x0b, 0xd9, 0xa0, 0xb9, 0x28,
}
testScript, _ = txscript.ParsePkScript(testRawScript)
)
type mockHintCache struct {
mu sync.Mutex
confHints map[chainhash.Hash]uint32
spendHints map[wire.OutPoint]uint32
confHints map[chainntnfs.ConfRequest]uint32
spendHints map[chainntnfs.SpendRequest]uint32
}
var _ chainntnfs.SpendHintCache = (*mockHintCache)(nil)
var _ chainntnfs.ConfirmHintCache = (*mockHintCache)(nil)
func (c *mockHintCache) CommitSpendHint(heightHint uint32, ops ...wire.OutPoint) error {
func (c *mockHintCache) CommitSpendHint(heightHint uint32,
spendRequests ...chainntnfs.SpendRequest) error {
c.mu.Lock()
defer c.mu.Unlock()
for _, op := range ops {
c.spendHints[op] = heightHint
for _, spendRequest := range spendRequests {
c.spendHints[spendRequest] = heightHint
}
return nil
}
func (c *mockHintCache) QuerySpendHint(op wire.OutPoint) (uint32, error) {
func (c *mockHintCache) QuerySpendHint(spendRequest chainntnfs.SpendRequest) (uint32, error) {
c.mu.Lock()
defer c.mu.Unlock()
hint, ok := c.spendHints[op]
hint, ok := c.spendHints[spendRequest]
if !ok {
return 0, chainntnfs.ErrSpendHintNotFound
}
@ -47,33 +70,35 @@ func (c *mockHintCache) QuerySpendHint(op wire.OutPoint) (uint32, error) {
return hint, nil
}
func (c *mockHintCache) PurgeSpendHint(ops ...wire.OutPoint) error {
func (c *mockHintCache) PurgeSpendHint(spendRequests ...chainntnfs.SpendRequest) error {
c.mu.Lock()
defer c.mu.Unlock()
for _, op := range ops {
delete(c.spendHints, op)
for _, spendRequest := range spendRequests {
delete(c.spendHints, spendRequest)
}
return nil
}
func (c *mockHintCache) CommitConfirmHint(heightHint uint32, txids ...chainhash.Hash) error {
func (c *mockHintCache) CommitConfirmHint(heightHint uint32,
confRequests ...chainntnfs.ConfRequest) error {
c.mu.Lock()
defer c.mu.Unlock()
for _, txid := range txids {
c.confHints[txid] = heightHint
for _, confRequest := range confRequests {
c.confHints[confRequest] = heightHint
}
return nil
}
func (c *mockHintCache) QueryConfirmHint(txid chainhash.Hash) (uint32, error) {
func (c *mockHintCache) QueryConfirmHint(confRequest chainntnfs.ConfRequest) (uint32, error) {
c.mu.Lock()
defer c.mu.Unlock()
hint, ok := c.confHints[txid]
hint, ok := c.confHints[confRequest]
if !ok {
return 0, chainntnfs.ErrConfirmHintNotFound
}
@ -81,12 +106,12 @@ func (c *mockHintCache) QueryConfirmHint(txid chainhash.Hash) (uint32, error) {
return hint, nil
}
func (c *mockHintCache) PurgeConfirmHint(txids ...chainhash.Hash) error {
func (c *mockHintCache) PurgeConfirmHint(confRequests ...chainntnfs.ConfRequest) error {
c.mu.Lock()
defer c.mu.Unlock()
for _, txid := range txids {
delete(c.confHints, txid)
for _, confRequest := range confRequests {
delete(c.confHints, confRequest)
}
return nil
@ -94,8 +119,8 @@ func (c *mockHintCache) PurgeConfirmHint(txids ...chainhash.Hash) error {
func newMockHintCache() *mockHintCache {
return &mockHintCache{
confHints: make(map[chainhash.Hash]uint32),
spendHints: make(map[wire.OutPoint]uint32),
confHints: make(map[chainntnfs.ConfRequest]uint32),
spendHints: make(map[chainntnfs.SpendRequest]uint32),
}
}
@ -112,19 +137,21 @@ func TestTxNotifierMaxConfs(t *testing.T) {
// Registering one confirmation above the maximum should fail with
// ErrTxMaxConfs.
ntfn := &chainntnfs.ConfNtfn{
ConfID: 1,
TxID: &zeroHash,
ConfRequest: chainntnfs.ConfRequest{
TxID: chainntnfs.ZeroHash,
PkScript: testScript,
},
NumConfirmations: chainntnfs.MaxNumConfs + 1,
Event: chainntnfs.NewConfirmationEvent(
chainntnfs.MaxNumConfs,
chainntnfs.MaxNumConfs, nil,
),
}
if _, err := n.RegisterConf(ntfn); err != chainntnfs.ErrTxMaxConfs {
if _, _, err := n.RegisterConf(ntfn); err != chainntnfs.ErrTxMaxConfs {
t.Fatalf("expected chainntnfs.ErrTxMaxConfs, got %v", err)
}
ntfn.NumConfirmations--
if _, err := n.RegisterConf(ntfn); err != nil {
if _, _, err := n.RegisterConf(ntfn); err != nil {
t.Fatalf("unable to register conf ntfn: %v", err)
}
}
@ -139,12 +166,6 @@ func TestTxNotifierFutureConfDispatch(t *testing.T) {
tx2NumConfs uint32 = 2
)
var (
tx1 = wire.MsgTx{Version: 1}
tx2 = wire.MsgTx{Version: 2}
tx3 = wire.MsgTx{Version: 3}
)
hintCache := newMockHintCache()
n := chainntnfs.NewTxNotifier(
10, chainntnfs.ReorgSafetyLimit, hintCache, hintCache,
@ -153,23 +174,31 @@ func TestTxNotifierFutureConfDispatch(t *testing.T) {
// Create the test transactions and register them with the TxNotifier
// before including them in a block to receive future
// notifications.
tx1Hash := tx1.TxHash()
tx1 := wire.MsgTx{Version: 1}
tx1.AddTxOut(&wire.TxOut{PkScript: testRawScript})
ntfn1 := chainntnfs.ConfNtfn{
TxID: &tx1Hash,
ConfRequest: chainntnfs.ConfRequest{
TxID: tx1.TxHash(),
PkScript: testScript,
},
NumConfirmations: tx1NumConfs,
Event: chainntnfs.NewConfirmationEvent(tx1NumConfs),
Event: chainntnfs.NewConfirmationEvent(tx1NumConfs, nil),
}
if _, err := n.RegisterConf(&ntfn1); err != nil {
if _, _, err := n.RegisterConf(&ntfn1); err != nil {
t.Fatalf("unable to register ntfn: %v", err)
}
tx2Hash := tx2.TxHash()
tx2 := wire.MsgTx{Version: 2}
tx2.AddTxOut(&wire.TxOut{PkScript: testRawScript})
ntfn2 := chainntnfs.ConfNtfn{
TxID: &tx2Hash,
ConfRequest: chainntnfs.ConfRequest{
TxID: tx2.TxHash(),
PkScript: testScript,
},
NumConfirmations: tx2NumConfs,
Event: chainntnfs.NewConfirmationEvent(tx2NumConfs),
Event: chainntnfs.NewConfirmationEvent(tx2NumConfs, nil),
}
if _, err := n.RegisterConf(&ntfn2); err != nil {
if _, _, err := n.RegisterConf(&ntfn2); err != nil {
t.Fatalf("unable to register ntfn: %v", err)
}
@ -194,7 +223,7 @@ func TestTxNotifierFutureConfDispatch(t *testing.T) {
// Include the transactions in a block and add it to the TxNotifier.
// This should confirm tx1, but not tx2.
block1 := btcutil.NewBlock(&wire.MsgBlock{
Transactions: []*wire.MsgTx{&tx1, &tx2, &tx3},
Transactions: []*wire.MsgTx{&tx1, &tx2},
})
err := n.ConnectTip(block1.Hash(), 11, block1.Transactions())
@ -227,6 +256,7 @@ func TestTxNotifierFutureConfDispatch(t *testing.T) {
BlockHash: block1.Hash(),
BlockHeight: 11,
TxIndex: 0,
Tx: &tx1,
}
assertConfDetails(t, txConf, &expectedConf)
default:
@ -257,10 +287,7 @@ func TestTxNotifierFutureConfDispatch(t *testing.T) {
// Create a new block and add it to the TxNotifier at the next height.
// This should confirm tx2.
block2 := btcutil.NewBlock(&wire.MsgBlock{
Transactions: []*wire.MsgTx{&tx3},
})
block2 := btcutil.NewBlock(&wire.MsgBlock{})
err = n.ConnectTip(block2.Hash(), 12, block2.Transactions())
if err != nil {
t.Fatalf("Failed to connect block: %v", err)
@ -301,6 +328,7 @@ func TestTxNotifierFutureConfDispatch(t *testing.T) {
BlockHash: block1.Hash(),
BlockHeight: 11,
TxIndex: 1,
Tx: &tx2,
}
assertConfDetails(t, txConf, &expectedConf)
default:
@ -335,33 +363,34 @@ func TestTxNotifierHistoricalConfDispatch(t *testing.T) {
tx1Hash := tx1.TxHash()
ntfn1 := chainntnfs.ConfNtfn{
ConfID: 0,
TxID: &tx1Hash,
ConfRequest: chainntnfs.ConfRequest{TxID: tx1Hash},
NumConfirmations: tx1NumConfs,
Event: chainntnfs.NewConfirmationEvent(tx1NumConfs),
Event: chainntnfs.NewConfirmationEvent(tx1NumConfs, nil),
}
if _, err := n.RegisterConf(&ntfn1); err != nil {
if _, _, err := n.RegisterConf(&ntfn1); err != nil {
t.Fatalf("unable to register ntfn: %v", err)
}
tx2Hash := tx2.TxHash()
ntfn2 := chainntnfs.ConfNtfn{
ConfID: 1,
TxID: &tx2Hash,
ConfRequest: chainntnfs.ConfRequest{TxID: tx2Hash},
NumConfirmations: tx2NumConfs,
Event: chainntnfs.NewConfirmationEvent(tx2NumConfs),
Event: chainntnfs.NewConfirmationEvent(tx2NumConfs, nil),
}
if _, err := n.RegisterConf(&ntfn2); err != nil {
if _, _, err := n.RegisterConf(&ntfn2); err != nil {
t.Fatalf("unable to register ntfn: %v", err)
}
// Update tx1 with its confirmation details. We should only receive one
// update since it only requires one confirmation and it already met it.
txConf1 := chainntnfs.TxConfirmation{
BlockHash: &zeroHash,
BlockHash: &chainntnfs.ZeroHash,
BlockHeight: 9,
TxIndex: 1,
Tx: &tx1,
}
err := n.UpdateConfDetails(tx1Hash, &txConf1)
err := n.UpdateConfDetails(ntfn1.ConfRequest, &txConf1)
if err != nil {
t.Fatalf("unable to update conf details: %v", err)
}
@ -390,11 +419,12 @@ func TestTxNotifierHistoricalConfDispatch(t *testing.T) {
// of confirmations, but we should receive a confirmation update
// indicating how many confirmation are left.
txConf2 := chainntnfs.TxConfirmation{
BlockHash: &zeroHash,
BlockHash: &chainntnfs.ZeroHash,
BlockHeight: 9,
TxIndex: 2,
Tx: &tx2,
}
err = n.UpdateConfDetails(tx2Hash, &txConf2)
err = n.UpdateConfDetails(ntfn2.ConfRequest, &txConf2)
if err != nil {
t.Fatalf("unable to update conf details: %v", err)
}
@ -477,10 +507,13 @@ func TestTxNotifierFutureSpendDispatch(t *testing.T) {
// We'll start off by registering for a spend notification of an
// outpoint.
ntfn := &chainntnfs.SpendNtfn{
OutPoint: zeroOutPoint,
Event: chainntnfs.NewSpendEvent(nil),
SpendRequest: chainntnfs.SpendRequest{
OutPoint: wire.OutPoint{Index: 1},
PkScript: testScript,
},
Event: chainntnfs.NewSpendEvent(nil),
}
if _, err := n.RegisterSpend(ntfn); err != nil {
if _, _, err := n.RegisterSpend(ntfn); err != nil {
t.Fatalf("unable to register spend ntfn: %v", err)
}
@ -496,7 +529,10 @@ func TestTxNotifierFutureSpendDispatch(t *testing.T) {
// above. We'll include it in the next block, which should trigger a
// spend notification.
spendTx := wire.NewMsgTx(2)
spendTx.AddTxIn(&wire.TxIn{PreviousOutPoint: zeroOutPoint})
spendTx.AddTxIn(&wire.TxIn{
PreviousOutPoint: ntfn.OutPoint,
SignatureScript: testSigScript,
})
spendTxHash := spendTx.TxHash()
block := btcutil.NewBlock(&wire.MsgBlock{
Transactions: []*wire.MsgTx{spendTx},
@ -529,7 +565,10 @@ func TestTxNotifierFutureSpendDispatch(t *testing.T) {
// spent, then we don't receive another spend notification.
prevOut := wire.OutPoint{Hash: spendTxHash, Index: 0}
spendOfSpend := wire.NewMsgTx(2)
spendOfSpend.AddTxIn(&wire.TxIn{PreviousOutPoint: prevOut})
spendOfSpend.AddTxIn(&wire.TxIn{
PreviousOutPoint: prevOut,
SignatureScript: testSigScript,
})
block = btcutil.NewBlock(&wire.MsgBlock{
Transactions: []*wire.MsgTx{spendOfSpend},
})
@ -563,9 +602,12 @@ func TestTxNotifierHistoricalSpendDispatch(t *testing.T) {
// We'll start by constructing the spending details of the outpoint
// below.
spentOutpoint := zeroOutPoint
spentOutpoint := wire.OutPoint{Index: 1}
spendTx := wire.NewMsgTx(2)
spendTx.AddTxIn(&wire.TxIn{PreviousOutPoint: zeroOutPoint})
spendTx.AddTxIn(&wire.TxIn{
PreviousOutPoint: spentOutpoint,
SignatureScript: testSigScript,
})
spendTxHash := spendTx.TxHash()
expectedSpendDetails := &chainntnfs.SpendDetail{
@ -579,10 +621,10 @@ func TestTxNotifierHistoricalSpendDispatch(t *testing.T) {
// We'll register for a spend notification of the outpoint and ensure
// that a notification isn't dispatched.
ntfn := &chainntnfs.SpendNtfn{
OutPoint: spentOutpoint,
Event: chainntnfs.NewSpendEvent(nil),
SpendRequest: chainntnfs.SpendRequest{OutPoint: spentOutpoint},
Event: chainntnfs.NewSpendEvent(nil),
}
if _, err := n.RegisterSpend(ntfn); err != nil {
if _, _, err := n.RegisterSpend(ntfn); err != nil {
t.Fatalf("unable to register spend ntfn: %v", err)
}
@ -596,7 +638,7 @@ func TestTxNotifierHistoricalSpendDispatch(t *testing.T) {
// we'll hand off the spending details of the outpoint to the notifier
// as it is not possible for it to view historical events in the chain.
// By doing this, we replicate the functionality of the ChainNotifier.
err := n.UpdateSpendDetails(ntfn.OutPoint, expectedSpendDetails)
err := n.UpdateSpendDetails(ntfn.SpendRequest, expectedSpendDetails)
if err != nil {
t.Fatalf("unable to update spend details: %v", err)
}
@ -614,7 +656,10 @@ func TestTxNotifierHistoricalSpendDispatch(t *testing.T) {
// spent, then we don't receive another spend notification.
prevOut := wire.OutPoint{Hash: spendTxHash, Index: 0}
spendOfSpend := wire.NewMsgTx(2)
spendOfSpend.AddTxIn(&wire.TxIn{PreviousOutPoint: prevOut})
spendOfSpend.AddTxIn(&wire.TxIn{
PreviousOutPoint: prevOut,
SignatureScript: testSigScript,
})
block := btcutil.NewBlock(&wire.MsgBlock{
Transactions: []*wire.MsgTx{spendOfSpend},
})
@ -650,10 +695,11 @@ func TestTxNotifierMultipleHistoricalConfRescans(t *testing.T) {
// historical view of the chain.
confNtfn1 := &chainntnfs.ConfNtfn{
ConfID: 0,
TxID: &zeroHash,
Event: chainntnfs.NewConfirmationEvent(1),
// TODO(wilmer): set pkScript.
ConfRequest: chainntnfs.ConfRequest{TxID: chainntnfs.ZeroHash},
Event: chainntnfs.NewConfirmationEvent(1, nil),
}
historicalConfDispatch1, err := n.RegisterConf(confNtfn1)
historicalConfDispatch1, _, err := n.RegisterConf(confNtfn1)
if err != nil {
t.Fatalf("unable to register spend ntfn: %v", err)
}
@ -666,10 +712,11 @@ func TestTxNotifierMultipleHistoricalConfRescans(t *testing.T) {
// since the first one is still pending.
confNtfn2 := &chainntnfs.ConfNtfn{
ConfID: 1,
TxID: &zeroHash,
Event: chainntnfs.NewConfirmationEvent(1),
// TODO(wilmer): set pkScript.
ConfRequest: chainntnfs.ConfRequest{TxID: chainntnfs.ZeroHash},
Event: chainntnfs.NewConfirmationEvent(1, nil),
}
historicalConfDispatch2, err := n.RegisterConf(confNtfn2)
historicalConfDispatch2, _, err := n.RegisterConf(confNtfn2)
if err != nil {
t.Fatalf("unable to register spend ntfn: %v", err)
}
@ -684,16 +731,17 @@ func TestTxNotifierMultipleHistoricalConfRescans(t *testing.T) {
confDetails := &chainntnfs.TxConfirmation{
BlockHeight: startingHeight - 1,
}
if err := n.UpdateConfDetails(*confNtfn2.TxID, confDetails); err != nil {
err = n.UpdateConfDetails(confNtfn2.ConfRequest, confDetails)
if err != nil {
t.Fatalf("unable to update conf details: %v", err)
}
confNtfn3 := &chainntnfs.ConfNtfn{
ConfID: 2,
TxID: &zeroHash,
Event: chainntnfs.NewConfirmationEvent(1),
ConfID: 2,
ConfRequest: chainntnfs.ConfRequest{TxID: chainntnfs.ZeroHash},
Event: chainntnfs.NewConfirmationEvent(1, nil),
}
historicalConfDispatch3, err := n.RegisterConf(confNtfn3)
historicalConfDispatch3, _, err := n.RegisterConf(confNtfn3)
if err != nil {
t.Fatalf("unable to register spend ntfn: %v", err)
}
@ -717,12 +765,15 @@ func TestTxNotifierMultipleHistoricalSpendRescans(t *testing.T) {
// The first registration for an outpoint in the notifier should request
// a historical spend rescan as it does not have a historical view of
// the chain.
ntfn1 := &chainntnfs.SpendNtfn{
SpendID: 0,
OutPoint: zeroOutPoint,
Event: chainntnfs.NewSpendEvent(nil),
spendRequest := chainntnfs.SpendRequest{
OutPoint: wire.OutPoint{Index: 1},
}
historicalDispatch1, err := n.RegisterSpend(ntfn1)
ntfn1 := &chainntnfs.SpendNtfn{
SpendID: 0,
SpendRequest: spendRequest,
Event: chainntnfs.NewSpendEvent(nil),
}
historicalDispatch1, _, err := n.RegisterSpend(ntfn1)
if err != nil {
t.Fatalf("unable to register spend ntfn: %v", err)
}
@ -734,11 +785,11 @@ func TestTxNotifierMultipleHistoricalSpendRescans(t *testing.T) {
// should not request a historical spend rescan since the first one is
// still pending.
ntfn2 := &chainntnfs.SpendNtfn{
SpendID: 1,
OutPoint: zeroOutPoint,
Event: chainntnfs.NewSpendEvent(nil),
SpendID: 1,
SpendRequest: spendRequest,
Event: chainntnfs.NewSpendEvent(nil),
}
historicalDispatch2, err := n.RegisterSpend(ntfn2)
historicalDispatch2, _, err := n.RegisterSpend(ntfn2)
if err != nil {
t.Fatalf("unable to register spend ntfn: %v", err)
}
@ -752,22 +803,22 @@ func TestTxNotifierMultipleHistoricalSpendRescans(t *testing.T) {
// cached.
spendDetails := &chainntnfs.SpendDetail{
SpentOutPoint: &ntfn2.OutPoint,
SpenderTxHash: &zeroHash,
SpenderTxHash: &chainntnfs.ZeroHash,
SpendingTx: wire.NewMsgTx(2),
SpenderInputIndex: 0,
SpendingHeight: startingHeight - 1,
}
err = n.UpdateSpendDetails(ntfn2.OutPoint, spendDetails)
err = n.UpdateSpendDetails(ntfn2.SpendRequest, spendDetails)
if err != nil {
t.Fatalf("unable to update spend details: %v", err)
}
ntfn3 := &chainntnfs.SpendNtfn{
SpendID: 2,
OutPoint: zeroOutPoint,
Event: chainntnfs.NewSpendEvent(nil),
SpendID: 2,
SpendRequest: spendRequest,
Event: chainntnfs.NewSpendEvent(nil),
}
historicalDispatch3, err := n.RegisterSpend(ntfn3)
historicalDispatch3, _, err := n.RegisterSpend(ntfn3)
if err != nil {
t.Fatalf("unable to register spend ntfn: %v", err)
}
@ -795,16 +846,23 @@ func TestTxNotifierMultipleHistoricalNtfns(t *testing.T) {
hintCache,
)
var txid chainhash.Hash
copy(txid[:], bytes.Repeat([]byte{0x01}, 32))
confRequest := chainntnfs.ConfRequest{
// TODO(wilmer): set pkScript.
TxID: txid,
}
// We'll start off by registered 5 clients for a confirmation
// notification on the same transaction.
confNtfns := make([]*chainntnfs.ConfNtfn, numNtfns)
for i := uint64(0); i < numNtfns; i++ {
confNtfns[i] = &chainntnfs.ConfNtfn{
ConfID: i,
TxID: &zeroHash,
Event: chainntnfs.NewConfirmationEvent(1),
ConfID: i,
ConfRequest: confRequest,
Event: chainntnfs.NewConfirmationEvent(1, nil),
}
if _, err := n.RegisterConf(confNtfns[i]); err != nil {
if _, _, err := n.RegisterConf(confNtfns[i]); err != nil {
t.Fatalf("unable to register conf ntfn #%d: %v", i, err)
}
}
@ -824,8 +882,9 @@ func TestTxNotifierMultipleHistoricalNtfns(t *testing.T) {
// it can stop watching at tip.
expectedConfDetails := &chainntnfs.TxConfirmation{
BlockHeight: startingHeight - 1,
Tx: wire.NewMsgTx(1),
}
err := n.UpdateConfDetails(*confNtfns[0].TxID, expectedConfDetails)
err := n.UpdateConfDetails(confNtfns[0].ConfRequest, expectedConfDetails)
if err != nil {
t.Fatalf("unable to update conf details: %v", err)
}
@ -847,11 +906,11 @@ func TestTxNotifierMultipleHistoricalNtfns(t *testing.T) {
// see a historical rescan request and the confirmation notification
// should come through immediately.
extraConfNtfn := &chainntnfs.ConfNtfn{
ConfID: numNtfns + 1,
TxID: &zeroHash,
Event: chainntnfs.NewConfirmationEvent(1),
ConfID: numNtfns + 1,
ConfRequest: confRequest,
Event: chainntnfs.NewConfirmationEvent(1, nil),
}
historicalConfRescan, err := n.RegisterConf(extraConfNtfn)
historicalConfRescan, _, err := n.RegisterConf(extraConfNtfn)
if err != nil {
t.Fatalf("unable to register conf ntfn: %v", err)
}
@ -867,14 +926,17 @@ func TestTxNotifierMultipleHistoricalNtfns(t *testing.T) {
}
// Similarly, we'll do the same thing but for spend notifications.
spendRequest := chainntnfs.SpendRequest{
OutPoint: wire.OutPoint{Index: 1},
}
spendNtfns := make([]*chainntnfs.SpendNtfn, numNtfns)
for i := uint64(0); i < numNtfns; i++ {
spendNtfns[i] = &chainntnfs.SpendNtfn{
SpendID: i,
OutPoint: zeroOutPoint,
Event: chainntnfs.NewSpendEvent(nil),
SpendID: i,
SpendRequest: spendRequest,
Event: chainntnfs.NewSpendEvent(nil),
}
if _, err := n.RegisterSpend(spendNtfns[i]); err != nil {
if _, _, err := n.RegisterSpend(spendNtfns[i]); err != nil {
t.Fatalf("unable to register spend ntfn #%d: %v", i, err)
}
}
@ -894,12 +956,12 @@ func TestTxNotifierMultipleHistoricalNtfns(t *testing.T) {
// stop watching at tip.
expectedSpendDetails := &chainntnfs.SpendDetail{
SpentOutPoint: &spendNtfns[0].OutPoint,
SpenderTxHash: &zeroHash,
SpenderTxHash: &chainntnfs.ZeroHash,
SpendingTx: wire.NewMsgTx(2),
SpenderInputIndex: 0,
SpendingHeight: startingHeight - 1,
}
err = n.UpdateSpendDetails(spendNtfns[0].OutPoint, expectedSpendDetails)
err = n.UpdateSpendDetails(spendNtfns[0].SpendRequest, expectedSpendDetails)
if err != nil {
t.Fatalf("unable to update spend details: %v", err)
}
@ -921,11 +983,11 @@ func TestTxNotifierMultipleHistoricalNtfns(t *testing.T) {
// should not see a historical rescan request and the spend notification
// should come through immediately.
extraSpendNtfn := &chainntnfs.SpendNtfn{
SpendID: numNtfns + 1,
OutPoint: zeroOutPoint,
Event: chainntnfs.NewSpendEvent(nil),
SpendID: numNtfns + 1,
SpendRequest: spendRequest,
Event: chainntnfs.NewSpendEvent(nil),
}
historicalSpendRescan, err := n.RegisterSpend(extraSpendNtfn)
historicalSpendRescan, _, err := n.RegisterSpend(extraSpendNtfn)
if err != nil {
t.Fatalf("unable to register spend ntfn: %v", err)
}
@ -941,6 +1003,92 @@ func TestTxNotifierMultipleHistoricalNtfns(t *testing.T) {
}
}
// TestTxNotifierCancelConf ensures that a confirmation notification after a
// client has canceled their intent to receive one.
func TestTxNotifierCancelConf(t *testing.T) {
t.Parallel()
const startingHeight = 10
hintCache := newMockHintCache()
n := chainntnfs.NewTxNotifier(startingHeight, 100, hintCache, hintCache)
// We'll register two notification requests. Only the second one will be
// canceled.
tx1 := wire.NewMsgTx(1)
tx1.AddTxOut(&wire.TxOut{PkScript: testRawScript})
ntfn1 := &chainntnfs.ConfNtfn{
ConfID: 1,
ConfRequest: chainntnfs.ConfRequest{
TxID: tx1.TxHash(),
PkScript: testScript,
},
NumConfirmations: 1,
Event: chainntnfs.NewConfirmationEvent(1, nil),
}
if _, _, err := n.RegisterConf(ntfn1); err != nil {
t.Fatalf("unable to register spend ntfn: %v", err)
}
tx2 := wire.NewMsgTx(2)
tx2.AddTxOut(&wire.TxOut{PkScript: testRawScript})
ntfn2 := &chainntnfs.ConfNtfn{
ConfID: 2,
ConfRequest: chainntnfs.ConfRequest{
TxID: tx2.TxHash(),
PkScript: testScript,
},
NumConfirmations: 1,
Event: chainntnfs.NewConfirmationEvent(1, nil),
}
if _, _, err := n.RegisterConf(ntfn2); err != nil {
t.Fatalf("unable to register spend ntfn: %v", err)
}
// Construct a block that will confirm both transactions.
block := btcutil.NewBlock(&wire.MsgBlock{
Transactions: []*wire.MsgTx{tx1, tx2},
})
tx1ConfDetails := &chainntnfs.TxConfirmation{
BlockHeight: startingHeight + 1,
BlockHash: block.Hash(),
TxIndex: 0,
Tx: tx1,
}
// Before extending the notifier's tip with the block above, we'll
// cancel the second request.
n.CancelConf(ntfn2.ConfRequest, ntfn2.ConfID)
err := n.ConnectTip(block.Hash(), startingHeight+1, block.Transactions())
if err != nil {
t.Fatalf("unable to connect block: %v", err)
}
if err := n.NotifyHeight(startingHeight + 1); err != nil {
t.Fatalf("unable to dispatch notifications: %v", err)
}
// The first request should still be active, so we should receive a
// confirmation notification with the correct details.
select {
case confDetails := <-ntfn1.Event.Confirmed:
assertConfDetails(t, confDetails, tx1ConfDetails)
default:
t.Fatalf("expected to receive confirmation notification")
}
// The second one, however, should not have. The event's Confrimed
// channel must have also been closed to indicate the caller that the
// TxNotifier can no longer fulfill their canceled request.
select {
case _, ok := <-ntfn2.Event.Confirmed:
if ok {
t.Fatal("expected Confirmed channel to be closed")
}
default:
t.Fatal("expected Confirmed channel to be closed")
}
}
// TestTxNotifierCancelSpend ensures that a spend notification after a client
// has canceled their intent to receive one.
func TestTxNotifierCancelSpend(t *testing.T) {
@ -956,27 +1104,35 @@ func TestTxNotifierCancelSpend(t *testing.T) {
// We'll register two notification requests. Only the second one will be
// canceled.
ntfn1 := &chainntnfs.SpendNtfn{
SpendID: 0,
OutPoint: zeroOutPoint,
Event: chainntnfs.NewSpendEvent(nil),
SpendID: 0,
SpendRequest: chainntnfs.SpendRequest{
OutPoint: wire.OutPoint{Index: 1},
PkScript: testScript,
},
Event: chainntnfs.NewSpendEvent(nil),
}
if _, err := n.RegisterSpend(ntfn1); err != nil {
if _, _, err := n.RegisterSpend(ntfn1); err != nil {
t.Fatalf("unable to register spend ntfn: %v", err)
}
ntfn2 := &chainntnfs.SpendNtfn{
SpendID: 1,
OutPoint: zeroOutPoint,
Event: chainntnfs.NewSpendEvent(nil),
SpendID: 1,
SpendRequest: chainntnfs.SpendRequest{
OutPoint: wire.OutPoint{Index: 2},
},
Event: chainntnfs.NewSpendEvent(nil),
}
if _, err := n.RegisterSpend(ntfn2); err != nil {
if _, _, err := n.RegisterSpend(ntfn2); err != nil {
t.Fatalf("unable to register spend ntfn: %v", err)
}
// Construct the spending details of the outpoint and create a dummy
// block containing it.
spendTx := wire.NewMsgTx(2)
spendTx.AddTxIn(&wire.TxIn{PreviousOutPoint: ntfn1.OutPoint})
spendTx.AddTxIn(&wire.TxIn{
PreviousOutPoint: ntfn1.OutPoint,
SignatureScript: testSigScript,
})
spendTxHash := spendTx.TxHash()
expectedSpendDetails := &chainntnfs.SpendDetail{
SpentOutPoint: &ntfn1.OutPoint,
@ -992,7 +1148,7 @@ func TestTxNotifierCancelSpend(t *testing.T) {
// Before extending the notifier's tip with the dummy block above, we'll
// cancel the second request.
n.CancelSpend(ntfn2.OutPoint, ntfn2.SpendID)
n.CancelSpend(ntfn2.SpendRequest, ntfn2.SpendID)
err := n.ConnectTip(block.Hash(), startingHeight+1, block.Transactions())
if err != nil {
@ -1036,59 +1192,68 @@ func TestTxNotifierConfReorg(t *testing.T) {
tx3NumConfs uint32 = 2
)
var (
tx1 = wire.MsgTx{Version: 1}
tx2 = wire.MsgTx{Version: 2}
tx3 = wire.MsgTx{Version: 3}
)
hintCache := newMockHintCache()
n := chainntnfs.NewTxNotifier(
7, chainntnfs.ReorgSafetyLimit, hintCache, hintCache,
)
// Tx 1 will be confirmed in block 9 and requires 2 confs.
tx1Hash := tx1.TxHash()
tx1 := wire.MsgTx{Version: 1}
tx1.AddTxOut(&wire.TxOut{PkScript: testRawScript})
ntfn1 := chainntnfs.ConfNtfn{
TxID: &tx1Hash,
ConfID: 1,
ConfRequest: chainntnfs.ConfRequest{
TxID: tx1.TxHash(),
PkScript: testScript,
},
NumConfirmations: tx1NumConfs,
Event: chainntnfs.NewConfirmationEvent(tx1NumConfs),
Event: chainntnfs.NewConfirmationEvent(tx1NumConfs, nil),
}
if _, err := n.RegisterConf(&ntfn1); err != nil {
if _, _, err := n.RegisterConf(&ntfn1); err != nil {
t.Fatalf("unable to register ntfn: %v", err)
}
if err := n.UpdateConfDetails(*ntfn1.TxID, nil); err != nil {
if err := n.UpdateConfDetails(ntfn1.ConfRequest, nil); err != nil {
t.Fatalf("unable to deliver conf details: %v", err)
}
// Tx 2 will be confirmed in block 10 and requires 1 conf.
tx2Hash := tx2.TxHash()
tx2 := wire.MsgTx{Version: 2}
tx2.AddTxOut(&wire.TxOut{PkScript: testRawScript})
ntfn2 := chainntnfs.ConfNtfn{
TxID: &tx2Hash,
ConfID: 2,
ConfRequest: chainntnfs.ConfRequest{
TxID: tx2.TxHash(),
PkScript: testScript,
},
NumConfirmations: tx2NumConfs,
Event: chainntnfs.NewConfirmationEvent(tx2NumConfs),
Event: chainntnfs.NewConfirmationEvent(tx2NumConfs, nil),
}
if _, err := n.RegisterConf(&ntfn2); err != nil {
if _, _, err := n.RegisterConf(&ntfn2); err != nil {
t.Fatalf("unable to register ntfn: %v", err)
}
if err := n.UpdateConfDetails(*ntfn2.TxID, nil); err != nil {
if err := n.UpdateConfDetails(ntfn2.ConfRequest, nil); err != nil {
t.Fatalf("unable to deliver conf details: %v", err)
}
// Tx 3 will be confirmed in block 10 and requires 2 confs.
tx3Hash := tx3.TxHash()
tx3 := wire.MsgTx{Version: 3}
tx3.AddTxOut(&wire.TxOut{PkScript: testRawScript})
ntfn3 := chainntnfs.ConfNtfn{
TxID: &tx3Hash,
ConfID: 3,
ConfRequest: chainntnfs.ConfRequest{
TxID: tx3.TxHash(),
PkScript: testScript,
},
NumConfirmations: tx3NumConfs,
Event: chainntnfs.NewConfirmationEvent(tx3NumConfs),
Event: chainntnfs.NewConfirmationEvent(tx3NumConfs, nil),
}
if _, err := n.RegisterConf(&ntfn3); err != nil {
if _, _, err := n.RegisterConf(&ntfn3); err != nil {
t.Fatalf("unable to register ntfn: %v", err)
}
if err := n.UpdateConfDetails(*ntfn3.TxID, nil); err != nil {
if err := n.UpdateConfDetails(ntfn3.ConfRequest, nil); err != nil {
t.Fatalf("unable to deliver conf details: %v", err)
}
@ -1270,6 +1435,7 @@ func TestTxNotifierConfReorg(t *testing.T) {
BlockHash: block3.Hash(),
BlockHeight: 12,
TxIndex: 0,
Tx: &tx2,
}
assertConfDetails(t, txConf, &expectedConf)
default:
@ -1300,6 +1466,7 @@ func TestTxNotifierConfReorg(t *testing.T) {
BlockHash: block3.Hash(),
BlockHeight: 12,
TxIndex: 1,
Tx: &tx3,
}
assertConfDetails(t, txConf, &expectedConf)
default:
@ -1323,31 +1490,44 @@ func TestTxNotifierSpendReorg(t *testing.T) {
// We'll have two outpoints that will be spent throughout the test. The
// first will be spent and will not experience a reorg, while the second
// one will.
op1 := zeroOutPoint
op1.Index = 1
spendRequest1 := chainntnfs.SpendRequest{
OutPoint: wire.OutPoint{Index: 1},
PkScript: testScript,
}
spendTx1 := wire.NewMsgTx(2)
spendTx1.AddTxIn(&wire.TxIn{PreviousOutPoint: op1})
spendTx1.AddTxIn(&wire.TxIn{
PreviousOutPoint: spendRequest1.OutPoint,
SignatureScript: testSigScript,
})
spendTxHash1 := spendTx1.TxHash()
expectedSpendDetails1 := &chainntnfs.SpendDetail{
SpentOutPoint: &op1,
SpentOutPoint: &spendRequest1.OutPoint,
SpenderTxHash: &spendTxHash1,
SpendingTx: spendTx1,
SpenderInputIndex: 0,
SpendingHeight: startingHeight + 1,
}
op2 := zeroOutPoint
op2.Index = 2
spendRequest2 := chainntnfs.SpendRequest{
OutPoint: wire.OutPoint{Index: 2},
PkScript: testScript,
}
spendTx2 := wire.NewMsgTx(2)
spendTx2.AddTxIn(&wire.TxIn{PreviousOutPoint: zeroOutPoint})
spendTx2.AddTxIn(&wire.TxIn{PreviousOutPoint: op2})
spendTx2.AddTxIn(&wire.TxIn{
PreviousOutPoint: chainntnfs.ZeroOutPoint,
SignatureScript: testSigScript,
})
spendTx2.AddTxIn(&wire.TxIn{
PreviousOutPoint: spendRequest2.OutPoint,
SignatureScript: testSigScript,
})
spendTxHash2 := spendTx2.TxHash()
// The second outpoint will experience a reorg and get re-spent at a
// different height, so we'll need to construct the spend details for
// before and after the reorg.
expectedSpendDetails2BeforeReorg := chainntnfs.SpendDetail{
SpentOutPoint: &op2,
SpentOutPoint: &spendRequest2.OutPoint,
SpenderTxHash: &spendTxHash2,
SpendingTx: spendTx2,
SpenderInputIndex: 1,
@ -1361,20 +1541,20 @@ func TestTxNotifierSpendReorg(t *testing.T) {
// We'll register for a spend notification for each outpoint above.
ntfn1 := &chainntnfs.SpendNtfn{
SpendID: 78,
OutPoint: op1,
Event: chainntnfs.NewSpendEvent(nil),
SpendID: 78,
SpendRequest: spendRequest1,
Event: chainntnfs.NewSpendEvent(nil),
}
if _, err := n.RegisterSpend(ntfn1); err != nil {
if _, _, err := n.RegisterSpend(ntfn1); err != nil {
t.Fatalf("unable to register spend ntfn: %v", err)
}
ntfn2 := &chainntnfs.SpendNtfn{
SpendID: 21,
OutPoint: op2,
Event: chainntnfs.NewSpendEvent(nil),
SpendID: 21,
SpendRequest: spendRequest2,
Event: chainntnfs.NewSpendEvent(nil),
}
if _, err := n.RegisterSpend(ntfn2); err != nil {
if _, _, err := n.RegisterSpend(ntfn2); err != nil {
t.Fatalf("unable to register spend ntfn: %v", err)
}
@ -1545,38 +1725,46 @@ func TestTxNotifierConfirmHintCache(t *testing.T) {
// Create two test transactions and register them for notifications.
tx1 := wire.MsgTx{Version: 1}
tx1Hash := tx1.TxHash()
tx1.AddTxOut(&wire.TxOut{PkScript: testRawScript})
ntfn1 := &chainntnfs.ConfNtfn{
TxID: &tx1Hash,
ConfID: 1,
ConfRequest: chainntnfs.ConfRequest{
TxID: tx1.TxHash(),
PkScript: testScript,
},
NumConfirmations: 1,
Event: chainntnfs.NewConfirmationEvent(1),
Event: chainntnfs.NewConfirmationEvent(1, nil),
}
tx2 := wire.MsgTx{Version: 2}
tx2Hash := tx2.TxHash()
tx2.AddTxOut(&wire.TxOut{PkScript: testRawScript})
ntfn2 := &chainntnfs.ConfNtfn{
TxID: &tx2Hash,
ConfID: 2,
ConfRequest: chainntnfs.ConfRequest{
TxID: tx2.TxHash(),
PkScript: testScript,
},
NumConfirmations: 2,
Event: chainntnfs.NewConfirmationEvent(2),
Event: chainntnfs.NewConfirmationEvent(2, nil),
}
if _, err := n.RegisterConf(ntfn1); err != nil {
if _, _, err := n.RegisterConf(ntfn1); err != nil {
t.Fatalf("unable to register tx1: %v", err)
}
if _, err := n.RegisterConf(ntfn2); err != nil {
if _, _, err := n.RegisterConf(ntfn2); err != nil {
t.Fatalf("unable to register tx2: %v", err)
}
// Both transactions should not have a height hint set, as RegisterConf
// should not alter the cache state.
_, err := hintCache.QueryConfirmHint(tx1Hash)
_, err := hintCache.QueryConfirmHint(ntfn1.ConfRequest)
if err != chainntnfs.ErrConfirmHintNotFound {
t.Fatalf("unexpected error when querying for height hint "+
"want: %v, got %v",
chainntnfs.ErrConfirmHintNotFound, err)
}
_, err = hintCache.QueryConfirmHint(tx2Hash)
_, err = hintCache.QueryConfirmHint(ntfn2.ConfRequest)
if err != chainntnfs.ErrConfirmHintNotFound {
t.Fatalf("unexpected error when querying for height hint "+
"want: %v, got %v",
@ -1602,14 +1790,14 @@ func TestTxNotifierConfirmHintCache(t *testing.T) {
// the height hints should remain unchanged. This simulates blocks
// confirming while the historical dispatch is processing the
// registration.
hint, err := hintCache.QueryConfirmHint(tx1Hash)
hint, err := hintCache.QueryConfirmHint(ntfn1.ConfRequest)
if err != chainntnfs.ErrConfirmHintNotFound {
t.Fatalf("unexpected error when querying for height hint "+
"want: %v, got %v",
chainntnfs.ErrConfirmHintNotFound, err)
}
hint, err = hintCache.QueryConfirmHint(tx2Hash)
hint, err = hintCache.QueryConfirmHint(ntfn2.ConfRequest)
if err != chainntnfs.ErrConfirmHintNotFound {
t.Fatalf("unexpected error when querying for height hint "+
"want: %v, got %v",
@ -1618,10 +1806,10 @@ func TestTxNotifierConfirmHintCache(t *testing.T) {
// Now, update the conf details reporting that the neither txn was found
// in the historical dispatch.
if err := n.UpdateConfDetails(tx1Hash, nil); err != nil {
if err := n.UpdateConfDetails(ntfn1.ConfRequest, nil); err != nil {
t.Fatalf("unable to update conf details: %v", err)
}
if err := n.UpdateConfDetails(tx2Hash, nil); err != nil {
if err := n.UpdateConfDetails(ntfn2.ConfRequest, nil); err != nil {
t.Fatalf("unable to update conf details: %v", err)
}
@ -1642,7 +1830,7 @@ func TestTxNotifierConfirmHintCache(t *testing.T) {
// Now that both notifications are waiting at tip for confirmations,
// they should have their height hints updated to the latest block
// height.
hint, err = hintCache.QueryConfirmHint(tx1Hash)
hint, err = hintCache.QueryConfirmHint(ntfn1.ConfRequest)
if err != nil {
t.Fatalf("unable to query for hint: %v", err)
}
@ -1651,7 +1839,7 @@ func TestTxNotifierConfirmHintCache(t *testing.T) {
tx1Height, hint)
}
hint, err = hintCache.QueryConfirmHint(tx2Hash)
hint, err = hintCache.QueryConfirmHint(ntfn2.ConfRequest)
if err != nil {
t.Fatalf("unable to query for hint: %v", err)
}
@ -1675,7 +1863,7 @@ func TestTxNotifierConfirmHintCache(t *testing.T) {
}
// The height hint for the first transaction should remain the same.
hint, err = hintCache.QueryConfirmHint(tx1Hash)
hint, err = hintCache.QueryConfirmHint(ntfn1.ConfRequest)
if err != nil {
t.Fatalf("unable to query for hint: %v", err)
}
@ -1686,7 +1874,7 @@ func TestTxNotifierConfirmHintCache(t *testing.T) {
// The height hint for the second transaction should now be updated to
// reflect its confirmation.
hint, err = hintCache.QueryConfirmHint(tx2Hash)
hint, err = hintCache.QueryConfirmHint(ntfn2.ConfRequest)
if err != nil {
t.Fatalf("unable to query for hint: %v", err)
}
@ -1703,7 +1891,7 @@ func TestTxNotifierConfirmHintCache(t *testing.T) {
// This should update the second transaction's height hint within the
// cache to the previous height.
hint, err = hintCache.QueryConfirmHint(tx2Hash)
hint, err = hintCache.QueryConfirmHint(ntfn2.ConfRequest)
if err != nil {
t.Fatalf("unable to query for hint: %v", err)
}
@ -1714,7 +1902,7 @@ func TestTxNotifierConfirmHintCache(t *testing.T) {
// The first transaction's height hint should remain at the original
// confirmation height.
hint, err = hintCache.QueryConfirmHint(tx2Hash)
hint, err = hintCache.QueryConfirmHint(ntfn2.ConfRequest)
if err != nil {
t.Fatalf("unable to query for hint: %v", err)
}
@ -1747,34 +1935,40 @@ func TestTxNotifierSpendHintCache(t *testing.T) {
)
// Create two test outpoints and register them for spend notifications.
op1 := wire.OutPoint{Hash: zeroHash, Index: 1}
ntfn1 := &chainntnfs.SpendNtfn{
OutPoint: op1,
Event: chainntnfs.NewSpendEvent(nil),
SpendID: 1,
SpendRequest: chainntnfs.SpendRequest{
OutPoint: wire.OutPoint{Index: 1},
PkScript: testScript,
},
Event: chainntnfs.NewSpendEvent(nil),
}
op2 := wire.OutPoint{Hash: zeroHash, Index: 2}
ntfn2 := &chainntnfs.SpendNtfn{
OutPoint: op2,
Event: chainntnfs.NewSpendEvent(nil),
SpendID: 2,
SpendRequest: chainntnfs.SpendRequest{
OutPoint: wire.OutPoint{Index: 2},
PkScript: testScript,
},
Event: chainntnfs.NewSpendEvent(nil),
}
if _, err := n.RegisterSpend(ntfn1); err != nil {
if _, _, err := n.RegisterSpend(ntfn1); err != nil {
t.Fatalf("unable to register spend for op1: %v", err)
}
if _, err := n.RegisterSpend(ntfn2); err != nil {
if _, _, err := n.RegisterSpend(ntfn2); err != nil {
t.Fatalf("unable to register spend for op2: %v", err)
}
// Both outpoints should not have a spend hint set upon registration, as
// we must first determine whether they have already been spent in the
// chain.
_, err := hintCache.QuerySpendHint(op1)
_, err := hintCache.QuerySpendHint(ntfn1.SpendRequest)
if err != chainntnfs.ErrSpendHintNotFound {
t.Fatalf("unexpected error when querying for height hint "+
"expected: %v, got %v", chainntnfs.ErrSpendHintNotFound,
err)
}
_, err = hintCache.QuerySpendHint(op2)
_, err = hintCache.QuerySpendHint(ntfn2.SpendRequest)
if err != chainntnfs.ErrSpendHintNotFound {
t.Fatalf("unexpected error when querying for height hint "+
"expected: %v, got %v", chainntnfs.ErrSpendHintNotFound,
@ -1796,13 +1990,13 @@ func TestTxNotifierSpendHintCache(t *testing.T) {
// Since we haven't called UpdateSpendDetails on any of the test
// outpoints, this implies that there is a still a pending historical
// rescan for them, so their spend hints should not be created/updated.
_, err = hintCache.QuerySpendHint(op1)
_, err = hintCache.QuerySpendHint(ntfn1.SpendRequest)
if err != chainntnfs.ErrSpendHintNotFound {
t.Fatalf("unexpected error when querying for height hint "+
"expected: %v, got %v", chainntnfs.ErrSpendHintNotFound,
err)
}
_, err = hintCache.QuerySpendHint(op2)
_, err = hintCache.QuerySpendHint(ntfn2.SpendRequest)
if err != chainntnfs.ErrSpendHintNotFound {
t.Fatalf("unexpected error when querying for height hint "+
"expected: %v, got %v", chainntnfs.ErrSpendHintNotFound,
@ -1812,17 +2006,20 @@ func TestTxNotifierSpendHintCache(t *testing.T) {
// Now, we'll simulate that their historical rescans have finished by
// calling UpdateSpendDetails. This should allow their spend hints to be
// updated upon every block connected/disconnected.
if err := n.UpdateSpendDetails(ntfn1.OutPoint, nil); err != nil {
if err := n.UpdateSpendDetails(ntfn1.SpendRequest, nil); err != nil {
t.Fatalf("unable to update spend details: %v", err)
}
if err := n.UpdateSpendDetails(ntfn2.OutPoint, nil); err != nil {
if err := n.UpdateSpendDetails(ntfn2.SpendRequest, nil); err != nil {
t.Fatalf("unable to update spend details: %v", err)
}
// We'll create a new block that only contains the spending transaction
// of the first outpoint.
spendTx1 := wire.NewMsgTx(2)
spendTx1.AddTxIn(&wire.TxIn{PreviousOutPoint: ntfn1.OutPoint})
spendTx1.AddTxIn(&wire.TxIn{
PreviousOutPoint: ntfn1.OutPoint,
SignatureScript: testSigScript,
})
block1 := btcutil.NewBlock(&wire.MsgBlock{
Transactions: []*wire.MsgTx{spendTx1},
})
@ -1837,14 +2034,14 @@ func TestTxNotifierSpendHintCache(t *testing.T) {
// Both outpoints should have their spend hints reflect the height of
// the new block being connected due to the first outpoint being spent
// at this height, and the second outpoint still being unspent.
op1Hint, err := hintCache.QuerySpendHint(ntfn1.OutPoint)
op1Hint, err := hintCache.QuerySpendHint(ntfn1.SpendRequest)
if err != nil {
t.Fatalf("unable to query for spend hint of op1: %v", err)
}
if op1Hint != op1Height {
t.Fatalf("expected hint %d, got %d", op1Height, op1Hint)
}
op2Hint, err := hintCache.QuerySpendHint(ntfn2.OutPoint)
op2Hint, err := hintCache.QuerySpendHint(ntfn2.SpendRequest)
if err != nil {
t.Fatalf("unable to query for spend hint of op2: %v", err)
}
@ -1854,7 +2051,10 @@ func TestTxNotifierSpendHintCache(t *testing.T) {
// Then, we'll create another block that spends the second outpoint.
spendTx2 := wire.NewMsgTx(2)
spendTx2.AddTxIn(&wire.TxIn{PreviousOutPoint: ntfn2.OutPoint})
spendTx2.AddTxIn(&wire.TxIn{
PreviousOutPoint: ntfn2.OutPoint,
SignatureScript: testSigScript,
})
block2 := btcutil.NewBlock(&wire.MsgBlock{
Transactions: []*wire.MsgTx{spendTx2},
})
@ -1869,14 +2069,14 @@ func TestTxNotifierSpendHintCache(t *testing.T) {
// Only the second outpoint should have its spend hint updated due to
// being spent within the new block. The first outpoint's spend hint
// should remain the same as it's already been spent before.
op1Hint, err = hintCache.QuerySpendHint(ntfn1.OutPoint)
op1Hint, err = hintCache.QuerySpendHint(ntfn1.SpendRequest)
if err != nil {
t.Fatalf("unable to query for spend hint of op1: %v", err)
}
if op1Hint != op1Height {
t.Fatalf("expected hint %d, got %d", op1Height, op1Hint)
}
op2Hint, err = hintCache.QuerySpendHint(ntfn2.OutPoint)
op2Hint, err = hintCache.QuerySpendHint(ntfn2.SpendRequest)
if err != nil {
t.Fatalf("unable to query for spend hint of op2: %v", err)
}
@ -1894,14 +2094,14 @@ func TestTxNotifierSpendHintCache(t *testing.T) {
// to the previous height, as that's where its spending transaction was
// included in within the chain. The first outpoint's spend hint should
// remain the same.
op1Hint, err = hintCache.QuerySpendHint(ntfn1.OutPoint)
op1Hint, err = hintCache.QuerySpendHint(ntfn1.SpendRequest)
if err != nil {
t.Fatalf("unable to query for spend hint of op1: %v", err)
}
if op1Hint != op1Height {
t.Fatalf("expected hint %d, got %d", op1Height, op1Hint)
}
op2Hint, err = hintCache.QuerySpendHint(ntfn2.OutPoint)
op2Hint, err = hintCache.QuerySpendHint(ntfn2.SpendRequest)
if err != nil {
t.Fatalf("unable to query for spend hint of op2: %v", err)
}
@ -1910,6 +2110,152 @@ func TestTxNotifierSpendHintCache(t *testing.T) {
}
}
// TestTxNotifierNtfnDone ensures that a notification is sent to registered
// clients through the Done channel once the notification request is no longer
// under the risk of being reorged out of the chain.
func TestTxNotifierNtfnDone(t *testing.T) {
t.Parallel()
hintCache := newMockHintCache()
const reorgSafetyLimit = 100
n := chainntnfs.NewTxNotifier(10, reorgSafetyLimit, hintCache, hintCache)
// We'll start by creating two notification requests: one confirmation
// and one spend.
confNtfn := &chainntnfs.ConfNtfn{
ConfID: 1,
ConfRequest: chainntnfs.ConfRequest{
TxID: chainntnfs.ZeroHash,
PkScript: testScript,
},
NumConfirmations: 1,
Event: chainntnfs.NewConfirmationEvent(1, nil),
}
if _, _, err := n.RegisterConf(confNtfn); err != nil {
t.Fatalf("unable to register conf ntfn: %v", err)
}
spendNtfn := &chainntnfs.SpendNtfn{
SpendID: 2,
SpendRequest: chainntnfs.SpendRequest{
OutPoint: chainntnfs.ZeroOutPoint,
PkScript: testScript,
},
Event: chainntnfs.NewSpendEvent(nil),
}
if _, _, err := n.RegisterSpend(spendNtfn); err != nil {
t.Fatalf("unable to register spend: %v", err)
}
// We'll create two transactions that will satisfy the notification
// requests above and include them in the next block of the chain.
tx := wire.NewMsgTx(1)
tx.AddTxOut(&wire.TxOut{PkScript: testRawScript})
spendTx := wire.NewMsgTx(1)
spendTx.AddTxIn(&wire.TxIn{
PreviousOutPoint: wire.OutPoint{Index: 1},
SignatureScript: testSigScript,
})
block := btcutil.NewBlock(&wire.MsgBlock{
Transactions: []*wire.MsgTx{tx, spendTx},
})
err := n.ConnectTip(block.Hash(), 11, block.Transactions())
if err != nil {
t.Fatalf("unable to connect block: %v", err)
}
if err := n.NotifyHeight(11); err != nil {
t.Fatalf("unable to dispatch notifications: %v", err)
}
// With the chain extended, we should see notifications dispatched for
// both requests.
select {
case <-confNtfn.Event.Confirmed:
default:
t.Fatal("expected to receive confirmation notification")
}
select {
case <-spendNtfn.Event.Spend:
default:
t.Fatal("expected to receive spend notification")
}
// The done notifications should not be dispatched yet as the requests
// are still under the risk of being reorged out the chain.
select {
case <-confNtfn.Event.Done:
t.Fatal("received unexpected done notification for confirmation")
case <-spendNtfn.Event.Done:
t.Fatal("received unexpected done notification for spend")
default:
}
// Now, we'll disconnect the block at tip to simulate a reorg. The reorg
// notifications should be dispatched to the respective clients.
if err := n.DisconnectTip(11); err != nil {
t.Fatalf("unable to disconnect block: %v", err)
}
select {
case <-confNtfn.Event.NegativeConf:
default:
t.Fatal("expected to receive reorg notification for confirmation")
}
select {
case <-spendNtfn.Event.Reorg:
default:
t.Fatal("expected to receive reorg notification for spend")
}
// We'll reconnect the block that satisfies both of these requests.
// We should see notifications dispatched for both once again.
err = n.ConnectTip(block.Hash(), 11, block.Transactions())
if err != nil {
t.Fatalf("unable to connect block: %v", err)
}
if err := n.NotifyHeight(11); err != nil {
t.Fatalf("unable to dispatch notifications: %v", err)
}
select {
case <-confNtfn.Event.Confirmed:
default:
t.Fatal("expected to receive confirmation notification")
}
select {
case <-spendNtfn.Event.Spend:
default:
t.Fatal("expected to receive spend notification")
}
// Finally, we'll extend the chain with blocks until the requests are no
// longer under the risk of being reorged out of the chain. We should
// expect the done notifications to be dispatched.
nextHeight := uint32(12)
for i := nextHeight; i < nextHeight+reorgSafetyLimit; i++ {
dummyBlock := btcutil.NewBlock(&wire.MsgBlock{})
if err := n.ConnectTip(dummyBlock.Hash(), i, nil); err != nil {
t.Fatalf("unable to connect block: %v", err)
}
}
select {
case <-confNtfn.Event.Done:
default:
t.Fatal("expected to receive done notification for confirmation")
}
select {
case <-spendNtfn.Event.Done:
default:
t.Fatal("expected to receive done notification for spend")
}
}
// TestTxNotifierTearDown ensures that the TxNotifier properly alerts clients
// that it is shutting down and will be unable to deliver notifications.
func TestTxNotifierTearDown(t *testing.T) {
@ -1923,19 +2269,21 @@ func TestTxNotifierTearDown(t *testing.T) {
// To begin the test, we'll register for a confirmation and spend
// notification.
confNtfn := &chainntnfs.ConfNtfn{
TxID: &zeroHash,
ConfID: 1,
ConfRequest: chainntnfs.ConfRequest{TxID: chainntnfs.ZeroHash},
NumConfirmations: 1,
Event: chainntnfs.NewConfirmationEvent(1),
Event: chainntnfs.NewConfirmationEvent(1, nil),
}
if _, err := n.RegisterConf(confNtfn); err != nil {
if _, _, err := n.RegisterConf(confNtfn); err != nil {
t.Fatalf("unable to register conf ntfn: %v", err)
}
spendNtfn := &chainntnfs.SpendNtfn{
OutPoint: zeroOutPoint,
Event: chainntnfs.NewSpendEvent(nil),
SpendID: 1,
SpendRequest: chainntnfs.SpendRequest{OutPoint: chainntnfs.ZeroOutPoint},
Event: chainntnfs.NewSpendEvent(nil),
}
if _, err := n.RegisterSpend(spendNtfn); err != nil {
if _, _, err := n.RegisterSpend(spendNtfn); err != nil {
t.Fatalf("unable to register spend ntfn: %v", err)
}
@ -1972,10 +2320,10 @@ func TestTxNotifierTearDown(t *testing.T) {
// Now that the notifier is torn down, we should no longer be able to
// register notification requests.
if _, err := n.RegisterConf(confNtfn); err == nil {
if _, _, err := n.RegisterConf(confNtfn); err == nil {
t.Fatal("expected confirmation registration to fail")
}
if _, err := n.RegisterSpend(spendNtfn); err == nil {
if _, _, err := n.RegisterSpend(spendNtfn); err == nil {
t.Fatal("expected spend registration to fail")
}
}
@ -1997,6 +2345,10 @@ func assertConfDetails(t *testing.T, result, expected *chainntnfs.TxConfirmation
t.Fatalf("Incorrect tx index in confirmation details: "+
"expected %d, got %d", expected.TxIndex, result.TxIndex)
}
if result.Tx.TxHash() != expected.Tx.TxHash() {
t.Fatalf("expected tx hash %v, got %v", expected.Tx.TxHash(),
result.Tx.TxHash())
}
}
func assertSpendDetails(t *testing.T, result, expected *chainntnfs.SpendDetail) {

@ -256,12 +256,7 @@ func newChainControlFromConfig(cfg *config, chanDB *channeldb.DB,
// Next we'll create the instances of the ChainNotifier and
// FilteredChainView interface which is backed by the neutrino
// light client.
cc.chainNotifier, err = neutrinonotify.New(
svc, hintCache, hintCache,
)
if err != nil {
return nil, nil, err
}
cc.chainNotifier = neutrinonotify.New(svc, hintCache, hintCache)
cc.chainView, err = chainview.NewCfFilteredChainView(svc)
if err != nil {
return nil, nil, err
@ -336,7 +331,7 @@ func newChainControlFromConfig(cfg *config, chanDB *channeldb.DB,
}
cc.chainNotifier = bitcoindnotify.New(
bitcoindConn, hintCache, hintCache,
bitcoindConn, activeNetParams.Params, hintCache, hintCache,
)
cc.chainView = chainview.NewBitcoindFilteredChainView(bitcoindConn)
walletConfig.ChainSource = bitcoindConn.NewBitcoindClient()
@ -446,7 +441,7 @@ func newChainControlFromConfig(cfg *config, chanDB *channeldb.DB,
DisableAutoReconnect: false,
}
cc.chainNotifier, err = btcdnotify.New(
rpcConfig, hintCache, hintCache,
rpcConfig, activeNetParams.Params, hintCache, hintCache,
)
if err != nil {
return nil, nil, err

12
go.mod

@ -7,19 +7,17 @@ require (
github.com/Yawning/aez v0.0.0-20180114000226-4dad034d9db2
github.com/aead/chacha20 v0.0.0-20180709150244-8b13a72661da // indirect
github.com/boltdb/bolt v1.3.1 // indirect
github.com/btcsuite/btcd v0.0.0-20180824064422-7d2daa5bfef28c5e282571bc06416516936115ee
github.com/btcsuite/btcd v0.0.0-20190115013929-ed77733ec07d
github.com/btcsuite/btclog v0.0.0-20170628155309-84c8d2346e9f
github.com/btcsuite/btcutil v0.0.0-20180706230648-ab6388e0c60ae4834a1f57511e20c17b5f78be4b
github.com/btcsuite/btcwallet v0.0.0-20181130221647-e59e51f8e13c
github.com/btcsuite/btcutil v0.0.0-20190112041146-bf1e1be93589
github.com/btcsuite/btcwallet v0.0.0-20190115024521-9ad115360b37
github.com/btcsuite/fastsha256 v0.0.0-20160815193821-637e65642941
github.com/btcsuite/goleveldb v1.0.0 // indirect
github.com/coreos/bbolt v0.0.0-20180223184059-7ee3ded59d4835e10f3e7d0f7603c42aa5e83820
github.com/davecgh/go-spew v1.1.1
github.com/fsnotify/fsnotify v1.4.7 // indirect
github.com/go-errors/errors v1.0.1
github.com/golang/protobuf v1.2.0
github.com/grpc-ecosystem/grpc-gateway v0.0.0-20170724004829-f2862b476edc
github.com/hpcloud/tail v1.0.0 // indirect
github.com/jackpal/gateway v1.0.4
github.com/jackpal/go-nat-pmp v0.0.0-20170405195558-28a68d0c24ad
github.com/jessevdk/go-flags v0.0.0-20170926144705-f88afde2fa19
@ -32,7 +30,7 @@ require (
github.com/juju/utils v0.0.0-20180820210520-bf9cc5bdd62d // indirect
github.com/juju/version v0.0.0-20180108022336-b64dbd566305 // indirect
github.com/kkdai/bstream v0.0.0-20181106074824-b3251f7901ec
github.com/lightninglabs/neutrino v0.0.0-20181130220745-8d09312ac266
github.com/lightninglabs/neutrino v0.0.0-20190115022559-351f5f06c6af
github.com/lightningnetwork/lightning-onion v0.0.0-20180605012408-ac4d9da8f1d6
github.com/ltcsuite/ltcd v0.0.0-20190101042124-f37f8bf35796
github.com/miekg/dns v0.0.0-20171125082028-79bfde677fa8
@ -47,9 +45,7 @@ require (
google.golang.org/genproto v0.0.0-20181127195345-31ac5d88444a
google.golang.org/grpc v1.16.0
gopkg.in/errgo.v1 v1.0.0 // indirect
gopkg.in/fsnotify.v1 v1.4.7 // indirect
gopkg.in/macaroon-bakery.v2 v2.0.1
gopkg.in/macaroon.v2 v2.0.0
gopkg.in/mgo.v2 v2.0.0-20180705113604-9856a29383ce // indirect
gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 // indirect
)

25
go.sum

@ -14,16 +14,17 @@ github.com/aead/siphash v1.0.1/go.mod h1:Nywa3cDsYNNK3gaciGTWPwHt0wlpNV15vwmswBA
github.com/boltdb/bolt v1.3.1 h1:JQmyP4ZBrce+ZQu0dY660FMfatumYDLun9hBCUVIkF4=
github.com/boltdb/bolt v1.3.1/go.mod h1:clJnj/oiGkjum5o1McbSZDSLxVThjynRyGBgiAx27Ps=
github.com/btcsuite/btcd v0.0.0-20180823030728-d81d8877b8f3/go.mod h1:Dmm/EzmjnCiweXmzRIAiUWCInVmPgjkzgv5k4tVyXiQ=
github.com/btcsuite/btcd v0.0.0-20180824064422-7d2daa5bfef28c5e282571bc06416516936115ee h1:YSiTy2Hn6a5TaPnmcyk0+OTc13E1rbmCCgo/J0LDxtA=
github.com/btcsuite/btcd v0.0.0-20180824064422-7d2daa5bfef28c5e282571bc06416516936115ee/go.mod h1:Jr9bmNVGZ7TH2Ux1QuP0ec+yGgh0gE9FIlkzQiI5bR0=
github.com/btcsuite/btcd v0.0.0-20180824064422-ed77733ec07dfc8a513741138419b8d9d3de9d2d/go.mod h1:d3C0AkH6BRcvO8T0UEPu53cnw4IbV63x1bEjildYhO0=
github.com/btcsuite/btcd v0.0.0-20190115013929-ed77733ec07d h1:xG8Pj6Y6J760xwETNmMzmlt38QSwz0BLp1cZ09g27uw=
github.com/btcsuite/btcd v0.0.0-20190115013929-ed77733ec07d/go.mod h1:d3C0AkH6BRcvO8T0UEPu53cnw4IbV63x1bEjildYhO0=
github.com/btcsuite/btclog v0.0.0-20170628155309-84c8d2346e9f h1:bAs4lUbRJpnnkd9VhRV3jjAVU7DJVjMaK+IsvSeZvFo=
github.com/btcsuite/btclog v0.0.0-20170628155309-84c8d2346e9f/go.mod h1:TdznJufoqS23FtqVCzL0ZqgP5MqXbb4fg/WgDys70nA=
github.com/btcsuite/btcutil v0.0.0-20180706230648-ab6388e0c60a/go.mod h1:+5NJ2+qvTyV9exUAL/rxXi3DcLg2Ts+ymUAY5y4NvMg=
github.com/btcsuite/btcutil v0.0.0-20180706230648-ab6388e0c60ae4834a1f57511e20c17b5f78be4b h1:sa+k743hEDlacUmuzqYlpxx4gp61C9Cf531bPaOneWo=
github.com/btcsuite/btcutil v0.0.0-20180706230648-ab6388e0c60ae4834a1f57511e20c17b5f78be4b/go.mod h1:+5NJ2+qvTyV9exUAL/rxXi3DcLg2Ts+ymUAY5y4NvMg=
github.com/btcsuite/btcutil v0.0.0-20190112041146-bf1e1be93589 h1:9A5pe5iQS+ll6R1EVLFv/y92IjrymihwITCU81aCIBQ=
github.com/btcsuite/btcutil v0.0.0-20190112041146-bf1e1be93589/go.mod h1:+5NJ2+qvTyV9exUAL/rxXi3DcLg2Ts+ymUAY5y4NvMg=
github.com/btcsuite/btcwallet v0.0.0-20180904010540-284e2e0e696e33d5be388f7f3d9a26db703e0c06/go.mod h1:/d7QHZsfUAruXuBhyPITqoYOmJ+nq35qPsJjz/aSpCg=
github.com/btcsuite/btcwallet v0.0.0-20181130221647-e59e51f8e13c h1:k39UlSgW6cR0cIEleF9Or3S4uSP7NgvRLPd/WNYScbU=
github.com/btcsuite/btcwallet v0.0.0-20181130221647-e59e51f8e13c/go.mod h1:mHSlZHtkxbCzvqKCzkQOSDb7Lc5iqoD8+pj6qc0yDBU=
github.com/btcsuite/btcwallet v0.0.0-20190115024521-9ad115360b37 h1:f8RY/x01F18bYrOvekIUDLqm9oGZFrim+wx5Z0ts/Kg=
github.com/btcsuite/btcwallet v0.0.0-20190115024521-9ad115360b37/go.mod h1:+u1ftn+QOb9qHKwsLf7rBOr0PHCo9CGA7U1WFq7VLA4=
github.com/btcsuite/fastsha256 v0.0.0-20160815193821-637e65642941 h1:kij1x2aL7VE6gtx8KMIt8PGPgI5GV9LgtHFG5KaEMPY=
github.com/btcsuite/fastsha256 v0.0.0-20160815193821-637e65642941/go.mod h1:QcFA8DZHtuIAdYKCq/BzELOaznRsCvwf4zTPmaYwaig=
github.com/btcsuite/go-socks v0.0.0-20170105172521-4720035b7bfd h1:R/opQEbFEy9JGkIguV40SvRY1uliPX8ifOvi6ICsFCw=
@ -95,9 +96,8 @@ github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI=
github.com/lightninglabs/gozmq v0.0.0-20180324010646-462a8a753885 h1:fTLuPUkaKIIV0+gA1IxiBDvDxtF8tzpSF6N6NfFGmsU=
github.com/lightninglabs/gozmq v0.0.0-20180324010646-462a8a753885/go.mod h1:KUh15naRlx/TmUMFS/p4JJrCrE6F7RGF7rsnvuu45E4=
github.com/lightninglabs/neutrino v0.0.0-20181017011010-4d6069299130/go.mod h1:KJq43Fu9ceitbJsSXMILcT4mGDNI/crKmPIkDOZXFyM=
github.com/lightninglabs/neutrino v0.0.0-20181017011010-8d09312ac266916a00d367abeaf05fbd8bccf5d8/go.mod h1:/ie0+o5CLo6NDM52EpoMAJsMPq25DqpK1APv6MzZj7U=
github.com/lightninglabs/neutrino v0.0.0-20181130220745-8d09312ac266 h1:2WHiUpriTMc4cLWvS4syzNHLS9+hPVLJjNeZuhq5HR4=
github.com/lightninglabs/neutrino v0.0.0-20181130220745-8d09312ac266/go.mod h1:/ie0+o5CLo6NDM52EpoMAJsMPq25DqpK1APv6MzZj7U=
github.com/lightninglabs/neutrino v0.0.0-20190115022559-351f5f06c6af h1:JzoYbWqwPb+PARU4LTtlohetdNa6/ocyQ0xidZQw4Hg=
github.com/lightninglabs/neutrino v0.0.0-20190115022559-351f5f06c6af/go.mod h1:aR+E6cs+FTaIwIa/WLyvNsB8FZg8TiP3r0Led+4Q4gI=
github.com/lightningnetwork/lightning-onion v0.0.0-20180605012408-ac4d9da8f1d6 h1:ONLGrYJVQdbtP6CE/ff1KNWZtygRGEh12RzonTiCzPs=
github.com/lightningnetwork/lightning-onion v0.0.0-20180605012408-ac4d9da8f1d6/go.mod h1:8EgEt4a/NUOVQd+3kk6n9aZCJ1Ssj96Pb6lCrci+6oc=
github.com/ltcsuite/ltcd v0.0.0-20190101042124-f37f8bf35796 h1:sjOGyegMIhvgfq5oaue6Td+hxZuf3tDC8lAPrFldqFw=
@ -108,8 +108,12 @@ github.com/miekg/dns v0.0.0-20171125082028-79bfde677fa8 h1:PRMAcldsl4mXKJeRNB/KV
github.com/miekg/dns v0.0.0-20171125082028-79bfde677fa8/go.mod h1:W1PPwlIAgtquWBMBEV9nkV9Cazfe8ScdGz/Lj7v3Nrg=
github.com/onsi/ginkgo v1.6.0 h1:Ix8l273rp3QzYgXSR+c8d1fTG7UPgYkOSELPhiY/YGw=
github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE=
github.com/onsi/ginkgo v1.7.0 h1:WSHQ+IS43OoUrWtD1/bbclrwK8TTH5hzp+umCiuxHgs=
github.com/onsi/ginkgo v1.7.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE=
github.com/onsi/gomega v1.4.1 h1:PZSj/UFNaVp3KxrzHOcS7oyuWA7LoOY/77yCTEFu21U=
github.com/onsi/gomega v1.4.1/go.mod h1:C1qb7wdrVGGVU+Z6iS04AVkA3Q65CEZX59MT0QO5uiA=
github.com/onsi/gomega v1.4.3 h1:RE1xgDvH7imwFD45h+u2SgIfERHlS2yNG4DObb5BSKU=
github.com/onsi/gomega v1.4.3/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY=
github.com/rogpeppe/fastuuid v0.0.0-20150106093220-6724a57986af h1:gu+uRPtBe88sKxUCEXRoeCvVG90TJmwhiqRpvdhQFng=
github.com/rogpeppe/fastuuid v0.0.0-20150106093220-6724a57986af/go.mod h1:XWv6SoW27p1b0cqNHllgS5HIMJraePCO15w5zCzIWYg=
github.com/tv42/zbase32 v0.0.0-20160707012821-501572607d02 h1:tcJ6OjwOMvExLlzrAVZute09ocAGa7KqOON60++Gz4E=
@ -126,6 +130,7 @@ golang.org/x/net v0.0.0-20180719180050-a680a1efc54d/go.mod h1:mL1N/T3taQHkDXs73r
golang.org/x/net v0.0.0-20180821023952-922f4815f713/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
golang.org/x/net v0.0.0-20180826012351-8a410e7b638d h1:g9qWBGx4puODJTMVyoPrpoxPFgVGd+z1DZwjfRu4d0I=
golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
golang.org/x/net v0.0.0-20181106065722-10aee1819953 h1:LuZIitY8waaxUfNIdtajyE/YzA/zyf0YxXG27VpLrkg=
golang.org/x/net v0.0.0-20181106065722-10aee1819953/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U=
@ -135,6 +140,8 @@ golang.org/x/sync v0.0.0-20181108010431-42b317875d0f h1:Bl/8QSvNqXvPGPGXa2z5xUTm
golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
golang.org/x/sys v0.0.0-20180821140842-3b58ed4ad339/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e h1:o3PsSEY8E4eXWkXrIP9YJALUkVZqzHJT5DOasTyn8Vs=
golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20181128092732-4ed8d59d0b35 h1:YAFjXN64LMvktoUZH9zgY4lGc/msGN7HQfoSuKCgaDU=
golang.org/x/sys v0.0.0-20181128092732-4ed8d59d0b35/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/text v0.3.0 h1:g61tztE5qeGQ89tm6NTjjM9VPIm088od1l6aSorWRWg=

@ -0,0 +1,459 @@
// +build chainrpc
package chainrpc
import (
"bytes"
"context"
"errors"
"io/ioutil"
"os"
"path/filepath"
"sync/atomic"
"github.com/btcsuite/btcd/chaincfg/chainhash"
"github.com/btcsuite/btcd/wire"
"github.com/lightningnetwork/lnd/chainntnfs"
"github.com/lightningnetwork/lnd/lnrpc"
"google.golang.org/grpc"
"gopkg.in/macaroon-bakery.v2/bakery"
)
const (
// subServerName is the name of the RPC sub-server. We'll use this name
// to register ourselves, and we also require that the main
// SubServerConfigDispatcher instance recognize this as the name of the
// config file that we need.
subServerName = "ChainRPC"
)
var (
// macaroonOps are the set of capabilities that our minted macaroon (if
// it doesn't already exist) will have.
macaroonOps = []bakery.Op{
{
Entity: "onchain",
Action: "read",
},
}
// macPermissions maps RPC calls to the permissions they require.
macPermissions = map[string][]bakery.Op{
"/chainrpc.ChainNotifier/RegisterConfirmationsNtfn": {{
Entity: "onchain",
Action: "read",
}},
"/chainrpc.ChainNotifier/RegisterSpendNtfn": {{
Entity: "onchain",
Action: "read",
}},
"/chainrpc.ChainNotifier/RegisterBlockEpochNtfn": {{
Entity: "onchain",
Action: "read",
}},
}
// DefaultChainNotifierMacFilename is the default name of the chain
// notifier macaroon that we expect to find via a file handle within the
// main configuration file in this package.
DefaultChainNotifierMacFilename = "chainnotifier.macaroon"
// ErrChainNotifierServerShuttingDown is an error returned when we are
// waiting for a notification to arrive but the chain notifier server
// has been shut down.
ErrChainNotifierServerShuttingDown = errors.New("chain notifier RPC " +
"subserver shutting down")
)
// fileExists reports whether the named file or directory exists.
func fileExists(name string) bool {
if _, err := os.Stat(name); err != nil {
if os.IsNotExist(err) {
return false
}
}
return true
}
// Server is a sub-server of the main RPC server: the chain notifier RPC. This
// RPC sub-server allows external callers to access the full chain notifier
// capabilities of lnd. This allows callers to create custom protocols, external
// to lnd, even backed by multiple distinct lnd across independent failure
// domains.
type Server struct {
started uint32
stopped uint32
cfg Config
quit chan struct{}
}
// New returns a new instance of the chainrpc ChainNotifier sub-server. We also
// return the set of permissions for the macaroons that we may create within
// this method. If the macaroons we need aren't found in the filepath, then
// we'll create them on start up. If we're unable to locate, or create the
// macaroons we need, then we'll return with an error.
func New(cfg *Config) (*Server, lnrpc.MacaroonPerms, error) {
// If the path of the chain notifier macaroon wasn't generated, then
// we'll assume that it's found at the default network directory.
if cfg.ChainNotifierMacPath == "" {
cfg.ChainNotifierMacPath = filepath.Join(
cfg.NetworkDir, DefaultChainNotifierMacFilename,
)
}
// Now that we know the full path of the chain notifier macaroon, we can
// check to see if we need to create it or not.
macFilePath := cfg.ChainNotifierMacPath
if cfg.MacService != nil && !fileExists(macFilePath) {
log.Infof("Baking macaroons for ChainNotifier RPC Server at: %v",
macFilePath)
// At this point, we know that the chain notifier macaroon
// doesn't yet, exist, so we need to create it with the help of
// the main macaroon service.
chainNotifierMac, err := cfg.MacService.Oven.NewMacaroon(
context.Background(), bakery.LatestVersion, nil,
macaroonOps...,
)
if err != nil {
return nil, nil, err
}
chainNotifierMacBytes, err := chainNotifierMac.M().MarshalBinary()
if err != nil {
return nil, nil, err
}
err = ioutil.WriteFile(macFilePath, chainNotifierMacBytes, 0644)
if err != nil {
os.Remove(macFilePath)
return nil, nil, err
}
}
return &Server{
cfg: *cfg,
quit: make(chan struct{}),
}, macPermissions, nil
}
// Compile-time checks to ensure that Server fully implements the
// ChainNotifierServer gRPC service and lnrpc.SubServer interface.
var _ ChainNotifierServer = (*Server)(nil)
var _ lnrpc.SubServer = (*Server)(nil)
// Start launches any helper goroutines required for the server to function.
//
// NOTE: This is part of the lnrpc.SubServer interface.
func (s *Server) Start() error {
if !atomic.CompareAndSwapUint32(&s.started, 0, 1) {
return nil
}
return nil
}
// Stop signals any active goroutines for a graceful closure.
//
// NOTE: This is part of the lnrpc.SubServer interface.
func (s *Server) Stop() error {
if !atomic.CompareAndSwapUint32(&s.stopped, 0, 1) {
return nil
}
close(s.quit)
return nil
}
// Name returns a unique string representation of the sub-server. This can be
// used to identify the sub-server and also de-duplicate them.
//
// NOTE: This is part of the lnrpc.SubServer interface.
func (s *Server) Name() string {
return subServerName
}
// RegisterWithRootServer will be called by the root gRPC server to direct a RPC
// sub-server to register itself with the main gRPC root server. Until this is
// called, each sub-server won't be able to have requests routed towards it.
//
// NOTE: This is part of the lnrpc.SubServer interface.
func (s *Server) RegisterWithRootServer(grpcServer *grpc.Server) error {
// We make sure that we register it with the main gRPC server to ensure
// all our methods are routed properly.
RegisterChainNotifierServer(grpcServer, s)
log.Debug("ChainNotifier RPC server successfully register with root " +
"gRPC server")
return nil
}
// RegisterConfirmationsNtfn is a synchronous response-streaming RPC that
// registers an intent for a client to be notified once a confirmation request
// has reached its required number of confirmations on-chain.
//
// A client can specify whether the confirmation request should be for a
// particular transaction by its hash or for an output script by specifying a
// zero hash.
//
// NOTE: This is part of the chainrpc.ChainNotifierService interface.
func (s *Server) RegisterConfirmationsNtfn(in *ConfRequest,
confStream ChainNotifier_RegisterConfirmationsNtfnServer) error {
// We'll start by reconstructing the RPC request into what the
// underlying ChainNotifier expects.
var txid chainhash.Hash
copy(txid[:], in.Txid)
// We'll then register for the spend notification of the request.
confEvent, err := s.cfg.ChainNotifier.RegisterConfirmationsNtfn(
&txid, in.Script, in.NumConfs, in.HeightHint,
)
if err != nil {
return err
}
defer confEvent.Cancel()
// With the request registered, we'll wait for its spend notification to
// be dispatched.
for {
select {
// The transaction satisfying the request has confirmed on-chain
// and reached its required number of confirmations. We'll
// dispatch an event to the caller indicating so.
case details, ok := <-confEvent.Confirmed:
if !ok {
return chainntnfs.ErrChainNotifierShuttingDown
}
var rawTxBuf bytes.Buffer
err := details.Tx.Serialize(&rawTxBuf)
if err != nil {
return err
}
rpcConfDetails := &ConfDetails{
RawTx: rawTxBuf.Bytes(),
BlockHash: details.BlockHash[:],
BlockHeight: details.BlockHeight,
TxIndex: details.TxIndex,
}
conf := &ConfEvent{
Event: &ConfEvent_Conf{
Conf: rpcConfDetails,
},
}
if err := confStream.Send(conf); err != nil {
return err
}
// The transaction satisfying the request has been reorged out
// of the chain, so we'll send an event describing so.
case _, ok := <-confEvent.NegativeConf:
if !ok {
return chainntnfs.ErrChainNotifierShuttingDown
}
reorg := &ConfEvent{
Event: &ConfEvent_Reorg{Reorg: &Reorg{}},
}
if err := confStream.Send(reorg); err != nil {
return err
}
// The transaction satisfying the request has confirmed and is
// no longer under the risk of being reorged out of the chain,
// so we can safely exit.
case _, ok := <-confEvent.Done:
if !ok {
return chainntnfs.ErrChainNotifierShuttingDown
}
return nil
// The response stream's context for whatever reason has been
// closed. We'll return the error indicated by the context
// itself to the caller.
case <-confStream.Context().Done():
return confStream.Context().Err()
// The server has been requested to shut down.
case <-s.quit:
return ErrChainNotifierServerShuttingDown
}
}
}
// RegisterSpendNtfn is a synchronous response-streaming RPC that registers an
// intent for a client to be notification once a spend request has been spent by
// a transaction that has confirmed on-chain.
//
// A client can specify whether the spend request should be for a particular
// outpoint or for an output script by specifying a zero outpoint.
//
// NOTE: This is part of the chainrpc.ChainNotifierService interface.
func (s *Server) RegisterSpendNtfn(in *SpendRequest,
spendStream ChainNotifier_RegisterSpendNtfnServer) error {
// We'll start by reconstructing the RPC request into what the
// underlying ChainNotifier expects.
var op *wire.OutPoint
if in.Outpoint != nil {
var txid chainhash.Hash
copy(txid[:], in.Outpoint.Hash)
op = &wire.OutPoint{Hash: txid, Index: in.Outpoint.Index}
}
// We'll then register for the spend notification of the request.
spendEvent, err := s.cfg.ChainNotifier.RegisterSpendNtfn(
op, in.Script, in.HeightHint,
)
if err != nil {
return err
}
defer spendEvent.Cancel()
// With the request registered, we'll wait for its spend notification to
// be dispatched.
for {
select {
// A transaction that spends the given has confirmed on-chain.
// We'll return an event to the caller indicating so that
// includes the details of the spending transaction.
case details, ok := <-spendEvent.Spend:
if !ok {
return chainntnfs.ErrChainNotifierShuttingDown
}
var rawSpendingTxBuf bytes.Buffer
err := details.SpendingTx.Serialize(&rawSpendingTxBuf)
if err != nil {
return err
}
rpcSpendDetails := &SpendDetails{
SpendingOutpoint: &Outpoint{
Hash: details.SpentOutPoint.Hash[:],
Index: details.SpentOutPoint.Index,
},
RawSpendingTx: rawSpendingTxBuf.Bytes(),
SpendingTxHash: details.SpenderTxHash[:],
SpendingInputIndex: details.SpenderInputIndex,
SpendingHeight: uint32(details.SpendingHeight),
}
spend := &SpendEvent{
Event: &SpendEvent_Spend{
Spend: rpcSpendDetails,
},
}
if err := spendStream.Send(spend); err != nil {
return err
}
// The spending transaction of the request has been reorged of
// the chain. We'll return an event to the caller indicating so.
case _, ok := <-spendEvent.Reorg:
if !ok {
return chainntnfs.ErrChainNotifierShuttingDown
}
reorg := &SpendEvent{
Event: &SpendEvent_Reorg{Reorg: &Reorg{}},
}
if err := spendStream.Send(reorg); err != nil {
return err
}
// The spending transaction of the requests has confirmed
// on-chain and is no longer under the risk of being reorged out
// of the chain, so we can safely exit.
case _, ok := <-spendEvent.Done:
if !ok {
return chainntnfs.ErrChainNotifierShuttingDown
}
return nil
// The response stream's context for whatever reason has been
// closed. We'll return the error indicated by the context
// itself to the caller.
case <-spendStream.Context().Done():
return spendStream.Context().Err()
// The server has been requested to shut down.
case <-s.quit:
return ErrChainNotifierServerShuttingDown
}
}
}
// RegisterBlockEpochNtfn is a synchronous response-streaming RPC that registers
// an intent for a client to be notified of blocks in the chain. The stream will
// return a hash and height tuple of a block for each new/stale block in the
// chain. It is the client's responsibility to determine whether the tuple
// returned is for a new or stale block in the chain.
//
// A client can also request a historical backlog of blocks from a particular
// point. This allows clients to be idempotent by ensuring that they do not
// missing processing a single block within the chain.
//
// NOTE: This is part of the chainrpc.ChainNotifierService interface.
func (s *Server) RegisterBlockEpochNtfn(in *BlockEpoch,
epochStream ChainNotifier_RegisterBlockEpochNtfnServer) error {
// We'll start by reconstructing the RPC request into what the
// underlying ChainNotifier expects.
var hash chainhash.Hash
copy(hash[:], in.Hash)
// If the request isn't for a zero hash and a zero height, then we
// should deliver a backlog of notifications from the given block
// (hash/height tuple) until tip, and continue delivering epochs for
// new blocks.
var blockEpoch *chainntnfs.BlockEpoch
if hash != chainntnfs.ZeroHash && in.Height != 0 {
blockEpoch = &chainntnfs.BlockEpoch{
Hash: &hash,
Height: int32(in.Height),
}
}
epochEvent, err := s.cfg.ChainNotifier.RegisterBlockEpochNtfn(blockEpoch)
if err != nil {
return err
}
defer epochEvent.Cancel()
for {
select {
// A notification for a block has been received. This block can
// either be a new block or stale.
case blockEpoch, ok := <-epochEvent.Epochs:
if !ok {
return chainntnfs.ErrChainNotifierShuttingDown
}
epoch := &BlockEpoch{
Hash: blockEpoch.Hash[:],
Height: uint32(blockEpoch.Height),
}
if err := epochStream.Send(epoch); err != nil {
return err
}
// The response stream's context for whatever reason has been
// closed. We'll return the error indicated by the context
// itself to the caller.
case <-epochStream.Context().Done():
return epochStream.Context().Err()
// The server has been requested to shut down.
case <-s.quit:
return ErrChainNotifierServerShuttingDown
}
}
}

@ -0,0 +1,1050 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// source: chainrpc/chainnotifier.proto
package chainrpc
import proto "github.com/golang/protobuf/proto"
import fmt "fmt"
import math "math"
import (
context "golang.org/x/net/context"
grpc "google.golang.org/grpc"
)
// Reference imports to suppress errors if they are not otherwise used.
var _ = proto.Marshal
var _ = fmt.Errorf
var _ = math.Inf
// This is a compile-time assertion to ensure that this generated file
// is compatible with the proto package it is being compiled against.
// A compilation error at this line likely means your copy of the
// proto package needs to be updated.
const _ = proto.ProtoPackageIsVersion2 // please upgrade the proto package
type ConfRequest struct {
//
// The transaction hash for which we should request a confirmation notification
// for. If set to a hash of all zeros, then the confirmation notification will
// be requested for the script instead.
Txid []byte `protobuf:"bytes,1,opt,name=txid,proto3" json:"txid,omitempty"`
//
// An output script within a transaction with the hash above which will be used
// by light clients to match block filters. If the transaction hash is set to a
// hash of all zeros, then a confirmation notification will be requested for
// this script instead.
Script []byte `protobuf:"bytes,2,opt,name=script,proto3" json:"script,omitempty"`
//
// The number of desired confirmations the transaction/output script should
// reach before dispatching a confirmation notification.
NumConfs uint32 `protobuf:"varint,3,opt,name=num_confs,json=numConfs,proto3" json:"num_confs,omitempty"`
//
// The earliest height in the chain for which the transaction/output script
// could have been included in a block. This should in most cases be set to the
// broadcast height of the transaction/output script.
HeightHint uint32 `protobuf:"varint,4,opt,name=height_hint,json=heightHint,proto3" json:"height_hint,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *ConfRequest) Reset() { *m = ConfRequest{} }
func (m *ConfRequest) String() string { return proto.CompactTextString(m) }
func (*ConfRequest) ProtoMessage() {}
func (*ConfRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_chainnotifier_7f5a7ad17988064c, []int{0}
}
func (m *ConfRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ConfRequest.Unmarshal(m, b)
}
func (m *ConfRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_ConfRequest.Marshal(b, m, deterministic)
}
func (dst *ConfRequest) XXX_Merge(src proto.Message) {
xxx_messageInfo_ConfRequest.Merge(dst, src)
}
func (m *ConfRequest) XXX_Size() int {
return xxx_messageInfo_ConfRequest.Size(m)
}
func (m *ConfRequest) XXX_DiscardUnknown() {
xxx_messageInfo_ConfRequest.DiscardUnknown(m)
}
var xxx_messageInfo_ConfRequest proto.InternalMessageInfo
func (m *ConfRequest) GetTxid() []byte {
if m != nil {
return m.Txid
}
return nil
}
func (m *ConfRequest) GetScript() []byte {
if m != nil {
return m.Script
}
return nil
}
func (m *ConfRequest) GetNumConfs() uint32 {
if m != nil {
return m.NumConfs
}
return 0
}
func (m *ConfRequest) GetHeightHint() uint32 {
if m != nil {
return m.HeightHint
}
return 0
}
type ConfDetails struct {
// The raw bytes of the confirmed transaction.
RawTx []byte `protobuf:"bytes,1,opt,name=raw_tx,json=rawTx,proto3" json:"raw_tx,omitempty"`
// The hash of the block in which the confirmed transaction was included in.
BlockHash []byte `protobuf:"bytes,2,opt,name=block_hash,json=blockHash,proto3" json:"block_hash,omitempty"`
// The height of the block in which the confirmed transaction was included in.
BlockHeight uint32 `protobuf:"varint,3,opt,name=block_height,json=blockHeight,proto3" json:"block_height,omitempty"`
// The index of the confirmed transaction within the transaction.
TxIndex uint32 `protobuf:"varint,4,opt,name=tx_index,json=txIndex,proto3" json:"tx_index,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *ConfDetails) Reset() { *m = ConfDetails{} }
func (m *ConfDetails) String() string { return proto.CompactTextString(m) }
func (*ConfDetails) ProtoMessage() {}
func (*ConfDetails) Descriptor() ([]byte, []int) {
return fileDescriptor_chainnotifier_7f5a7ad17988064c, []int{1}
}
func (m *ConfDetails) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ConfDetails.Unmarshal(m, b)
}
func (m *ConfDetails) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_ConfDetails.Marshal(b, m, deterministic)
}
func (dst *ConfDetails) XXX_Merge(src proto.Message) {
xxx_messageInfo_ConfDetails.Merge(dst, src)
}
func (m *ConfDetails) XXX_Size() int {
return xxx_messageInfo_ConfDetails.Size(m)
}
func (m *ConfDetails) XXX_DiscardUnknown() {
xxx_messageInfo_ConfDetails.DiscardUnknown(m)
}
var xxx_messageInfo_ConfDetails proto.InternalMessageInfo
func (m *ConfDetails) GetRawTx() []byte {
if m != nil {
return m.RawTx
}
return nil
}
func (m *ConfDetails) GetBlockHash() []byte {
if m != nil {
return m.BlockHash
}
return nil
}
func (m *ConfDetails) GetBlockHeight() uint32 {
if m != nil {
return m.BlockHeight
}
return 0
}
func (m *ConfDetails) GetTxIndex() uint32 {
if m != nil {
return m.TxIndex
}
return 0
}
type Reorg struct {
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *Reorg) Reset() { *m = Reorg{} }
func (m *Reorg) String() string { return proto.CompactTextString(m) }
func (*Reorg) ProtoMessage() {}
func (*Reorg) Descriptor() ([]byte, []int) {
return fileDescriptor_chainnotifier_7f5a7ad17988064c, []int{2}
}
func (m *Reorg) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Reorg.Unmarshal(m, b)
}
func (m *Reorg) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_Reorg.Marshal(b, m, deterministic)
}
func (dst *Reorg) XXX_Merge(src proto.Message) {
xxx_messageInfo_Reorg.Merge(dst, src)
}
func (m *Reorg) XXX_Size() int {
return xxx_messageInfo_Reorg.Size(m)
}
func (m *Reorg) XXX_DiscardUnknown() {
xxx_messageInfo_Reorg.DiscardUnknown(m)
}
var xxx_messageInfo_Reorg proto.InternalMessageInfo
type ConfEvent struct {
// Types that are valid to be assigned to Event:
// *ConfEvent_Conf
// *ConfEvent_Reorg
Event isConfEvent_Event `protobuf_oneof:"event"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *ConfEvent) Reset() { *m = ConfEvent{} }
func (m *ConfEvent) String() string { return proto.CompactTextString(m) }
func (*ConfEvent) ProtoMessage() {}
func (*ConfEvent) Descriptor() ([]byte, []int) {
return fileDescriptor_chainnotifier_7f5a7ad17988064c, []int{3}
}
func (m *ConfEvent) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ConfEvent.Unmarshal(m, b)
}
func (m *ConfEvent) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_ConfEvent.Marshal(b, m, deterministic)
}
func (dst *ConfEvent) XXX_Merge(src proto.Message) {
xxx_messageInfo_ConfEvent.Merge(dst, src)
}
func (m *ConfEvent) XXX_Size() int {
return xxx_messageInfo_ConfEvent.Size(m)
}
func (m *ConfEvent) XXX_DiscardUnknown() {
xxx_messageInfo_ConfEvent.DiscardUnknown(m)
}
var xxx_messageInfo_ConfEvent proto.InternalMessageInfo
type isConfEvent_Event interface {
isConfEvent_Event()
}
type ConfEvent_Conf struct {
Conf *ConfDetails `protobuf:"bytes,1,opt,name=conf,proto3,oneof"`
}
type ConfEvent_Reorg struct {
Reorg *Reorg `protobuf:"bytes,2,opt,name=reorg,proto3,oneof"`
}
func (*ConfEvent_Conf) isConfEvent_Event() {}
func (*ConfEvent_Reorg) isConfEvent_Event() {}
func (m *ConfEvent) GetEvent() isConfEvent_Event {
if m != nil {
return m.Event
}
return nil
}
func (m *ConfEvent) GetConf() *ConfDetails {
if x, ok := m.GetEvent().(*ConfEvent_Conf); ok {
return x.Conf
}
return nil
}
func (m *ConfEvent) GetReorg() *Reorg {
if x, ok := m.GetEvent().(*ConfEvent_Reorg); ok {
return x.Reorg
}
return nil
}
// XXX_OneofFuncs is for the internal use of the proto package.
func (*ConfEvent) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) {
return _ConfEvent_OneofMarshaler, _ConfEvent_OneofUnmarshaler, _ConfEvent_OneofSizer, []interface{}{
(*ConfEvent_Conf)(nil),
(*ConfEvent_Reorg)(nil),
}
}
func _ConfEvent_OneofMarshaler(msg proto.Message, b *proto.Buffer) error {
m := msg.(*ConfEvent)
// event
switch x := m.Event.(type) {
case *ConfEvent_Conf:
b.EncodeVarint(1<<3 | proto.WireBytes)
if err := b.EncodeMessage(x.Conf); err != nil {
return err
}
case *ConfEvent_Reorg:
b.EncodeVarint(2<<3 | proto.WireBytes)
if err := b.EncodeMessage(x.Reorg); err != nil {
return err
}
case nil:
default:
return fmt.Errorf("ConfEvent.Event has unexpected type %T", x)
}
return nil
}
func _ConfEvent_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error) {
m := msg.(*ConfEvent)
switch tag {
case 1: // event.conf
if wire != proto.WireBytes {
return true, proto.ErrInternalBadWireType
}
msg := new(ConfDetails)
err := b.DecodeMessage(msg)
m.Event = &ConfEvent_Conf{msg}
return true, err
case 2: // event.reorg
if wire != proto.WireBytes {
return true, proto.ErrInternalBadWireType
}
msg := new(Reorg)
err := b.DecodeMessage(msg)
m.Event = &ConfEvent_Reorg{msg}
return true, err
default:
return false, nil
}
}
func _ConfEvent_OneofSizer(msg proto.Message) (n int) {
m := msg.(*ConfEvent)
// event
switch x := m.Event.(type) {
case *ConfEvent_Conf:
s := proto.Size(x.Conf)
n += 1 // tag and wire
n += proto.SizeVarint(uint64(s))
n += s
case *ConfEvent_Reorg:
s := proto.Size(x.Reorg)
n += 1 // tag and wire
n += proto.SizeVarint(uint64(s))
n += s
case nil:
default:
panic(fmt.Sprintf("proto: unexpected type %T in oneof", x))
}
return n
}
type Outpoint struct {
// The hash of the transaction.
Hash []byte `protobuf:"bytes,1,opt,name=hash,proto3" json:"hash,omitempty"`
// The index of the output within the transaction.
Index uint32 `protobuf:"varint,2,opt,name=index,proto3" json:"index,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *Outpoint) Reset() { *m = Outpoint{} }
func (m *Outpoint) String() string { return proto.CompactTextString(m) }
func (*Outpoint) ProtoMessage() {}
func (*Outpoint) Descriptor() ([]byte, []int) {
return fileDescriptor_chainnotifier_7f5a7ad17988064c, []int{4}
}
func (m *Outpoint) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Outpoint.Unmarshal(m, b)
}
func (m *Outpoint) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_Outpoint.Marshal(b, m, deterministic)
}
func (dst *Outpoint) XXX_Merge(src proto.Message) {
xxx_messageInfo_Outpoint.Merge(dst, src)
}
func (m *Outpoint) XXX_Size() int {
return xxx_messageInfo_Outpoint.Size(m)
}
func (m *Outpoint) XXX_DiscardUnknown() {
xxx_messageInfo_Outpoint.DiscardUnknown(m)
}
var xxx_messageInfo_Outpoint proto.InternalMessageInfo
func (m *Outpoint) GetHash() []byte {
if m != nil {
return m.Hash
}
return nil
}
func (m *Outpoint) GetIndex() uint32 {
if m != nil {
return m.Index
}
return 0
}
type SpendRequest struct {
//
// The outpoint for which we should request a spend notification for. If set to
// a zero outpoint, then the spend notification will be requested for the
// script instead.
Outpoint *Outpoint `protobuf:"bytes,1,opt,name=outpoint,proto3" json:"outpoint,omitempty"`
//
// The output script for the outpoint above. This will be used by light clients
// to match block filters. If the outpoint is set to a zero outpoint, then a
// spend notification will be requested for this script instead.
Script []byte `protobuf:"bytes,2,opt,name=script,proto3" json:"script,omitempty"`
//
// The earliest height in the chain for which the outpoint/output script could
// have been spent. This should in most cases be set to the broadcast height of
// the outpoint/output script.
HeightHint uint32 `protobuf:"varint,3,opt,name=height_hint,json=heightHint,proto3" json:"height_hint,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *SpendRequest) Reset() { *m = SpendRequest{} }
func (m *SpendRequest) String() string { return proto.CompactTextString(m) }
func (*SpendRequest) ProtoMessage() {}
func (*SpendRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_chainnotifier_7f5a7ad17988064c, []int{5}
}
func (m *SpendRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_SpendRequest.Unmarshal(m, b)
}
func (m *SpendRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_SpendRequest.Marshal(b, m, deterministic)
}
func (dst *SpendRequest) XXX_Merge(src proto.Message) {
xxx_messageInfo_SpendRequest.Merge(dst, src)
}
func (m *SpendRequest) XXX_Size() int {
return xxx_messageInfo_SpendRequest.Size(m)
}
func (m *SpendRequest) XXX_DiscardUnknown() {
xxx_messageInfo_SpendRequest.DiscardUnknown(m)
}
var xxx_messageInfo_SpendRequest proto.InternalMessageInfo
func (m *SpendRequest) GetOutpoint() *Outpoint {
if m != nil {
return m.Outpoint
}
return nil
}
func (m *SpendRequest) GetScript() []byte {
if m != nil {
return m.Script
}
return nil
}
func (m *SpendRequest) GetHeightHint() uint32 {
if m != nil {
return m.HeightHint
}
return 0
}
type SpendDetails struct {
// The outpoint was that spent.
SpendingOutpoint *Outpoint `protobuf:"bytes,1,opt,name=spending_outpoint,json=spendingOutpoint,proto3" json:"spending_outpoint,omitempty"`
// The raw bytes of the spending transaction.
RawSpendingTx []byte `protobuf:"bytes,2,opt,name=raw_spending_tx,json=rawSpendingTx,proto3" json:"raw_spending_tx,omitempty"`
// The hash of the spending transaction.
SpendingTxHash []byte `protobuf:"bytes,3,opt,name=spending_tx_hash,json=spendingTxHash,proto3" json:"spending_tx_hash,omitempty"`
// The input of the spending transaction that fulfilled the spend request.
SpendingInputIndex uint32 `protobuf:"varint,4,opt,name=spending_input_index,json=spendingInputIndex,proto3" json:"spending_input_index,omitempty"`
// The height at which the spending transaction was included in a block.
SpendingHeight uint32 `protobuf:"varint,5,opt,name=spending_height,json=spendingHeight,proto3" json:"spending_height,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *SpendDetails) Reset() { *m = SpendDetails{} }
func (m *SpendDetails) String() string { return proto.CompactTextString(m) }
func (*SpendDetails) ProtoMessage() {}
func (*SpendDetails) Descriptor() ([]byte, []int) {
return fileDescriptor_chainnotifier_7f5a7ad17988064c, []int{6}
}
func (m *SpendDetails) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_SpendDetails.Unmarshal(m, b)
}
func (m *SpendDetails) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_SpendDetails.Marshal(b, m, deterministic)
}
func (dst *SpendDetails) XXX_Merge(src proto.Message) {
xxx_messageInfo_SpendDetails.Merge(dst, src)
}
func (m *SpendDetails) XXX_Size() int {
return xxx_messageInfo_SpendDetails.Size(m)
}
func (m *SpendDetails) XXX_DiscardUnknown() {
xxx_messageInfo_SpendDetails.DiscardUnknown(m)
}
var xxx_messageInfo_SpendDetails proto.InternalMessageInfo
func (m *SpendDetails) GetSpendingOutpoint() *Outpoint {
if m != nil {
return m.SpendingOutpoint
}
return nil
}
func (m *SpendDetails) GetRawSpendingTx() []byte {
if m != nil {
return m.RawSpendingTx
}
return nil
}
func (m *SpendDetails) GetSpendingTxHash() []byte {
if m != nil {
return m.SpendingTxHash
}
return nil
}
func (m *SpendDetails) GetSpendingInputIndex() uint32 {
if m != nil {
return m.SpendingInputIndex
}
return 0
}
func (m *SpendDetails) GetSpendingHeight() uint32 {
if m != nil {
return m.SpendingHeight
}
return 0
}
type SpendEvent struct {
// Types that are valid to be assigned to Event:
// *SpendEvent_Spend
// *SpendEvent_Reorg
Event isSpendEvent_Event `protobuf_oneof:"event"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *SpendEvent) Reset() { *m = SpendEvent{} }
func (m *SpendEvent) String() string { return proto.CompactTextString(m) }
func (*SpendEvent) ProtoMessage() {}
func (*SpendEvent) Descriptor() ([]byte, []int) {
return fileDescriptor_chainnotifier_7f5a7ad17988064c, []int{7}
}
func (m *SpendEvent) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_SpendEvent.Unmarshal(m, b)
}
func (m *SpendEvent) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_SpendEvent.Marshal(b, m, deterministic)
}
func (dst *SpendEvent) XXX_Merge(src proto.Message) {
xxx_messageInfo_SpendEvent.Merge(dst, src)
}
func (m *SpendEvent) XXX_Size() int {
return xxx_messageInfo_SpendEvent.Size(m)
}
func (m *SpendEvent) XXX_DiscardUnknown() {
xxx_messageInfo_SpendEvent.DiscardUnknown(m)
}
var xxx_messageInfo_SpendEvent proto.InternalMessageInfo
type isSpendEvent_Event interface {
isSpendEvent_Event()
}
type SpendEvent_Spend struct {
Spend *SpendDetails `protobuf:"bytes,1,opt,name=spend,proto3,oneof"`
}
type SpendEvent_Reorg struct {
Reorg *Reorg `protobuf:"bytes,2,opt,name=reorg,proto3,oneof"`
}
func (*SpendEvent_Spend) isSpendEvent_Event() {}
func (*SpendEvent_Reorg) isSpendEvent_Event() {}
func (m *SpendEvent) GetEvent() isSpendEvent_Event {
if m != nil {
return m.Event
}
return nil
}
func (m *SpendEvent) GetSpend() *SpendDetails {
if x, ok := m.GetEvent().(*SpendEvent_Spend); ok {
return x.Spend
}
return nil
}
func (m *SpendEvent) GetReorg() *Reorg {
if x, ok := m.GetEvent().(*SpendEvent_Reorg); ok {
return x.Reorg
}
return nil
}
// XXX_OneofFuncs is for the internal use of the proto package.
func (*SpendEvent) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) {
return _SpendEvent_OneofMarshaler, _SpendEvent_OneofUnmarshaler, _SpendEvent_OneofSizer, []interface{}{
(*SpendEvent_Spend)(nil),
(*SpendEvent_Reorg)(nil),
}
}
func _SpendEvent_OneofMarshaler(msg proto.Message, b *proto.Buffer) error {
m := msg.(*SpendEvent)
// event
switch x := m.Event.(type) {
case *SpendEvent_Spend:
b.EncodeVarint(1<<3 | proto.WireBytes)
if err := b.EncodeMessage(x.Spend); err != nil {
return err
}
case *SpendEvent_Reorg:
b.EncodeVarint(2<<3 | proto.WireBytes)
if err := b.EncodeMessage(x.Reorg); err != nil {
return err
}
case nil:
default:
return fmt.Errorf("SpendEvent.Event has unexpected type %T", x)
}
return nil
}
func _SpendEvent_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error) {
m := msg.(*SpendEvent)
switch tag {
case 1: // event.spend
if wire != proto.WireBytes {
return true, proto.ErrInternalBadWireType
}
msg := new(SpendDetails)
err := b.DecodeMessage(msg)
m.Event = &SpendEvent_Spend{msg}
return true, err
case 2: // event.reorg
if wire != proto.WireBytes {
return true, proto.ErrInternalBadWireType
}
msg := new(Reorg)
err := b.DecodeMessage(msg)
m.Event = &SpendEvent_Reorg{msg}
return true, err
default:
return false, nil
}
}
func _SpendEvent_OneofSizer(msg proto.Message) (n int) {
m := msg.(*SpendEvent)
// event
switch x := m.Event.(type) {
case *SpendEvent_Spend:
s := proto.Size(x.Spend)
n += 1 // tag and wire
n += proto.SizeVarint(uint64(s))
n += s
case *SpendEvent_Reorg:
s := proto.Size(x.Reorg)
n += 1 // tag and wire
n += proto.SizeVarint(uint64(s))
n += s
case nil:
default:
panic(fmt.Sprintf("proto: unexpected type %T in oneof", x))
}
return n
}
type BlockEpoch struct {
// The hash of the block.
Hash []byte `protobuf:"bytes,1,opt,name=hash,proto3" json:"hash,omitempty"`
// The height of the block.
Height uint32 `protobuf:"varint,2,opt,name=height,proto3" json:"height,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *BlockEpoch) Reset() { *m = BlockEpoch{} }
func (m *BlockEpoch) String() string { return proto.CompactTextString(m) }
func (*BlockEpoch) ProtoMessage() {}
func (*BlockEpoch) Descriptor() ([]byte, []int) {
return fileDescriptor_chainnotifier_7f5a7ad17988064c, []int{8}
}
func (m *BlockEpoch) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_BlockEpoch.Unmarshal(m, b)
}
func (m *BlockEpoch) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_BlockEpoch.Marshal(b, m, deterministic)
}
func (dst *BlockEpoch) XXX_Merge(src proto.Message) {
xxx_messageInfo_BlockEpoch.Merge(dst, src)
}
func (m *BlockEpoch) XXX_Size() int {
return xxx_messageInfo_BlockEpoch.Size(m)
}
func (m *BlockEpoch) XXX_DiscardUnknown() {
xxx_messageInfo_BlockEpoch.DiscardUnknown(m)
}
var xxx_messageInfo_BlockEpoch proto.InternalMessageInfo
func (m *BlockEpoch) GetHash() []byte {
if m != nil {
return m.Hash
}
return nil
}
func (m *BlockEpoch) GetHeight() uint32 {
if m != nil {
return m.Height
}
return 0
}
func init() {
proto.RegisterType((*ConfRequest)(nil), "chainrpc.ConfRequest")
proto.RegisterType((*ConfDetails)(nil), "chainrpc.ConfDetails")
proto.RegisterType((*Reorg)(nil), "chainrpc.Reorg")
proto.RegisterType((*ConfEvent)(nil), "chainrpc.ConfEvent")
proto.RegisterType((*Outpoint)(nil), "chainrpc.Outpoint")
proto.RegisterType((*SpendRequest)(nil), "chainrpc.SpendRequest")
proto.RegisterType((*SpendDetails)(nil), "chainrpc.SpendDetails")
proto.RegisterType((*SpendEvent)(nil), "chainrpc.SpendEvent")
proto.RegisterType((*BlockEpoch)(nil), "chainrpc.BlockEpoch")
}
// Reference imports to suppress errors if they are not otherwise used.
var _ context.Context
var _ grpc.ClientConn
// This is a compile-time assertion to ensure that this generated file
// is compatible with the grpc package it is being compiled against.
const _ = grpc.SupportPackageIsVersion4
// ChainNotifierClient is the client API for ChainNotifier service.
//
// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream.
type ChainNotifierClient interface {
//
// RegisterConfirmationsNtfn is a synchronous response-streaming RPC that
// registers an intent for a client to be notified once a confirmation request
// has reached its required number of confirmations on-chain.
//
// A client can specify whether the confirmation request should be for a
// particular transaction by its hash or for an output script by specifying a
// zero hash.
RegisterConfirmationsNtfn(ctx context.Context, in *ConfRequest, opts ...grpc.CallOption) (ChainNotifier_RegisterConfirmationsNtfnClient, error)
//
// RegisterSpendNtfn is a synchronous response-streaming RPC that registers an
// intent for a client to be notification once a spend request has been spent
// by a transaction that has confirmed on-chain.
//
// A client can specify whether the spend request should be for a particular
// outpoint or for an output script by specifying a zero outpoint.
RegisterSpendNtfn(ctx context.Context, in *SpendRequest, opts ...grpc.CallOption) (ChainNotifier_RegisterSpendNtfnClient, error)
//
// RegisterBlockEpochNtfn is a synchronous response-streaming RPC that
// registers an intent for a client to be notified of blocks in the chain. The
// stream will return a hash and height tuple of a block for each new/stale
// block in the chain. It is the client's responsibility to determine whether
// the tuple returned is for a new or stale block in the chain.
//
// A client can also request a historical backlog of blocks from a particular
// point. This allows clients to be idempotent by ensuring that they do not
// missing processing a single block within the chain.
RegisterBlockEpochNtfn(ctx context.Context, in *BlockEpoch, opts ...grpc.CallOption) (ChainNotifier_RegisterBlockEpochNtfnClient, error)
}
type chainNotifierClient struct {
cc *grpc.ClientConn
}
func NewChainNotifierClient(cc *grpc.ClientConn) ChainNotifierClient {
return &chainNotifierClient{cc}
}
func (c *chainNotifierClient) RegisterConfirmationsNtfn(ctx context.Context, in *ConfRequest, opts ...grpc.CallOption) (ChainNotifier_RegisterConfirmationsNtfnClient, error) {
stream, err := c.cc.NewStream(ctx, &_ChainNotifier_serviceDesc.Streams[0], "/chainrpc.ChainNotifier/RegisterConfirmationsNtfn", opts...)
if err != nil {
return nil, err
}
x := &chainNotifierRegisterConfirmationsNtfnClient{stream}
if err := x.ClientStream.SendMsg(in); err != nil {
return nil, err
}
if err := x.ClientStream.CloseSend(); err != nil {
return nil, err
}
return x, nil
}
type ChainNotifier_RegisterConfirmationsNtfnClient interface {
Recv() (*ConfEvent, error)
grpc.ClientStream
}
type chainNotifierRegisterConfirmationsNtfnClient struct {
grpc.ClientStream
}
func (x *chainNotifierRegisterConfirmationsNtfnClient) Recv() (*ConfEvent, error) {
m := new(ConfEvent)
if err := x.ClientStream.RecvMsg(m); err != nil {
return nil, err
}
return m, nil
}
func (c *chainNotifierClient) RegisterSpendNtfn(ctx context.Context, in *SpendRequest, opts ...grpc.CallOption) (ChainNotifier_RegisterSpendNtfnClient, error) {
stream, err := c.cc.NewStream(ctx, &_ChainNotifier_serviceDesc.Streams[1], "/chainrpc.ChainNotifier/RegisterSpendNtfn", opts...)
if err != nil {
return nil, err
}
x := &chainNotifierRegisterSpendNtfnClient{stream}
if err := x.ClientStream.SendMsg(in); err != nil {
return nil, err
}
if err := x.ClientStream.CloseSend(); err != nil {
return nil, err
}
return x, nil
}
type ChainNotifier_RegisterSpendNtfnClient interface {
Recv() (*SpendEvent, error)
grpc.ClientStream
}
type chainNotifierRegisterSpendNtfnClient struct {
grpc.ClientStream
}
func (x *chainNotifierRegisterSpendNtfnClient) Recv() (*SpendEvent, error) {
m := new(SpendEvent)
if err := x.ClientStream.RecvMsg(m); err != nil {
return nil, err
}
return m, nil
}
func (c *chainNotifierClient) RegisterBlockEpochNtfn(ctx context.Context, in *BlockEpoch, opts ...grpc.CallOption) (ChainNotifier_RegisterBlockEpochNtfnClient, error) {
stream, err := c.cc.NewStream(ctx, &_ChainNotifier_serviceDesc.Streams[2], "/chainrpc.ChainNotifier/RegisterBlockEpochNtfn", opts...)
if err != nil {
return nil, err
}
x := &chainNotifierRegisterBlockEpochNtfnClient{stream}
if err := x.ClientStream.SendMsg(in); err != nil {
return nil, err
}
if err := x.ClientStream.CloseSend(); err != nil {
return nil, err
}
return x, nil
}
type ChainNotifier_RegisterBlockEpochNtfnClient interface {
Recv() (*BlockEpoch, error)
grpc.ClientStream
}
type chainNotifierRegisterBlockEpochNtfnClient struct {
grpc.ClientStream
}
func (x *chainNotifierRegisterBlockEpochNtfnClient) Recv() (*BlockEpoch, error) {
m := new(BlockEpoch)
if err := x.ClientStream.RecvMsg(m); err != nil {
return nil, err
}
return m, nil
}
// ChainNotifierServer is the server API for ChainNotifier service.
type ChainNotifierServer interface {
//
// RegisterConfirmationsNtfn is a synchronous response-streaming RPC that
// registers an intent for a client to be notified once a confirmation request
// has reached its required number of confirmations on-chain.
//
// A client can specify whether the confirmation request should be for a
// particular transaction by its hash or for an output script by specifying a
// zero hash.
RegisterConfirmationsNtfn(*ConfRequest, ChainNotifier_RegisterConfirmationsNtfnServer) error
//
// RegisterSpendNtfn is a synchronous response-streaming RPC that registers an
// intent for a client to be notification once a spend request has been spent
// by a transaction that has confirmed on-chain.
//
// A client can specify whether the spend request should be for a particular
// outpoint or for an output script by specifying a zero outpoint.
RegisterSpendNtfn(*SpendRequest, ChainNotifier_RegisterSpendNtfnServer) error
//
// RegisterBlockEpochNtfn is a synchronous response-streaming RPC that
// registers an intent for a client to be notified of blocks in the chain. The
// stream will return a hash and height tuple of a block for each new/stale
// block in the chain. It is the client's responsibility to determine whether
// the tuple returned is for a new or stale block in the chain.
//
// A client can also request a historical backlog of blocks from a particular
// point. This allows clients to be idempotent by ensuring that they do not
// missing processing a single block within the chain.
RegisterBlockEpochNtfn(*BlockEpoch, ChainNotifier_RegisterBlockEpochNtfnServer) error
}
func RegisterChainNotifierServer(s *grpc.Server, srv ChainNotifierServer) {
s.RegisterService(&_ChainNotifier_serviceDesc, srv)
}
func _ChainNotifier_RegisterConfirmationsNtfn_Handler(srv interface{}, stream grpc.ServerStream) error {
m := new(ConfRequest)
if err := stream.RecvMsg(m); err != nil {
return err
}
return srv.(ChainNotifierServer).RegisterConfirmationsNtfn(m, &chainNotifierRegisterConfirmationsNtfnServer{stream})
}
type ChainNotifier_RegisterConfirmationsNtfnServer interface {
Send(*ConfEvent) error
grpc.ServerStream
}
type chainNotifierRegisterConfirmationsNtfnServer struct {
grpc.ServerStream
}
func (x *chainNotifierRegisterConfirmationsNtfnServer) Send(m *ConfEvent) error {
return x.ServerStream.SendMsg(m)
}
func _ChainNotifier_RegisterSpendNtfn_Handler(srv interface{}, stream grpc.ServerStream) error {
m := new(SpendRequest)
if err := stream.RecvMsg(m); err != nil {
return err
}
return srv.(ChainNotifierServer).RegisterSpendNtfn(m, &chainNotifierRegisterSpendNtfnServer{stream})
}
type ChainNotifier_RegisterSpendNtfnServer interface {
Send(*SpendEvent) error
grpc.ServerStream
}
type chainNotifierRegisterSpendNtfnServer struct {
grpc.ServerStream
}
func (x *chainNotifierRegisterSpendNtfnServer) Send(m *SpendEvent) error {
return x.ServerStream.SendMsg(m)
}
func _ChainNotifier_RegisterBlockEpochNtfn_Handler(srv interface{}, stream grpc.ServerStream) error {
m := new(BlockEpoch)
if err := stream.RecvMsg(m); err != nil {
return err
}
return srv.(ChainNotifierServer).RegisterBlockEpochNtfn(m, &chainNotifierRegisterBlockEpochNtfnServer{stream})
}
type ChainNotifier_RegisterBlockEpochNtfnServer interface {
Send(*BlockEpoch) error
grpc.ServerStream
}
type chainNotifierRegisterBlockEpochNtfnServer struct {
grpc.ServerStream
}
func (x *chainNotifierRegisterBlockEpochNtfnServer) Send(m *BlockEpoch) error {
return x.ServerStream.SendMsg(m)
}
var _ChainNotifier_serviceDesc = grpc.ServiceDesc{
ServiceName: "chainrpc.ChainNotifier",
HandlerType: (*ChainNotifierServer)(nil),
Methods: []grpc.MethodDesc{},
Streams: []grpc.StreamDesc{
{
StreamName: "RegisterConfirmationsNtfn",
Handler: _ChainNotifier_RegisterConfirmationsNtfn_Handler,
ServerStreams: true,
},
{
StreamName: "RegisterSpendNtfn",
Handler: _ChainNotifier_RegisterSpendNtfn_Handler,
ServerStreams: true,
},
{
StreamName: "RegisterBlockEpochNtfn",
Handler: _ChainNotifier_RegisterBlockEpochNtfn_Handler,
ServerStreams: true,
},
},
Metadata: "chainrpc/chainnotifier.proto",
}
func init() {
proto.RegisterFile("chainrpc/chainnotifier.proto", fileDescriptor_chainnotifier_7f5a7ad17988064c)
}
var fileDescriptor_chainnotifier_7f5a7ad17988064c = []byte{
// 574 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x54, 0x4f, 0x6f, 0x13, 0x3f,
0x10, 0xed, 0xb6, 0xdd, 0xfc, 0x99, 0x24, 0xbf, 0xb4, 0xfe, 0xa5, 0x51, 0x5a, 0x40, 0x94, 0x3d,
0xd0, 0x48, 0x48, 0x21, 0x0a, 0x1c, 0xb8, 0x21, 0x35, 0x14, 0x25, 0x97, 0x22, 0x6d, 0x7b, 0x5f,
0x6d, 0x37, 0x4e, 0xd6, 0xd0, 0xd8, 0x8b, 0xed, 0x90, 0xbd, 0xf2, 0x69, 0xf9, 0x12, 0x1c, 0x90,
0x67, 0xed, 0x4d, 0x1a, 0x8a, 0x84, 0xb8, 0x79, 0x66, 0xde, 0x3e, 0xbf, 0xf1, 0x7b, 0x09, 0x3c,
0x4d, 0xd2, 0x98, 0x71, 0x99, 0x25, 0xaf, 0xf1, 0xc0, 0x85, 0x66, 0x73, 0x46, 0xe5, 0x20, 0x93,
0x42, 0x0b, 0x52, 0x73, 0xd3, 0x60, 0x0d, 0x8d, 0xb1, 0xe0, 0xf3, 0x90, 0x7e, 0x5d, 0x51, 0xa5,
0x09, 0x81, 0x43, 0x9d, 0xb3, 0x59, 0xcf, 0x3b, 0xf7, 0xfa, 0xcd, 0x10, 0xcf, 0xa4, 0x0b, 0x15,
0x95, 0x48, 0x96, 0xe9, 0xde, 0x3e, 0x76, 0x6d, 0x45, 0x9e, 0x40, 0x9d, 0xaf, 0x96, 0x51, 0x22,
0xf8, 0x5c, 0xf5, 0x0e, 0xce, 0xbd, 0x7e, 0x2b, 0xac, 0xf1, 0xd5, 0xd2, 0xd0, 0x29, 0xf2, 0x1c,
0x1a, 0x29, 0x65, 0x8b, 0x54, 0x47, 0x29, 0xe3, 0xba, 0x77, 0x88, 0x63, 0x28, 0x5a, 0x13, 0xc6,
0x75, 0xf0, 0xdd, 0x2b, 0x6e, 0xfe, 0x40, 0x75, 0xcc, 0xee, 0x15, 0x39, 0x81, 0x8a, 0x8c, 0xd7,
0x91, 0xce, 0xed, 0xdd, 0xbe, 0x8c, 0xd7, 0xb7, 0x39, 0x79, 0x06, 0x70, 0x77, 0x2f, 0x92, 0x2f,
0x51, 0x1a, 0xab, 0xd4, 0x0a, 0xa8, 0x63, 0x67, 0x12, 0xab, 0x94, 0xbc, 0x80, 0xa6, 0x1d, 0x23,
0xb3, 0x95, 0xd1, 0x28, 0x00, 0xd8, 0x22, 0xa7, 0x50, 0xd3, 0x79, 0xc4, 0xf8, 0x8c, 0xe6, 0x56,
0x46, 0x55, 0xe7, 0x53, 0x53, 0x06, 0x55, 0xf0, 0x43, 0x2a, 0xe4, 0x22, 0xf8, 0x0c, 0x75, 0xa3,
0xe5, 0xea, 0x1b, 0xe5, 0x9a, 0xbc, 0x82, 0x43, 0xb3, 0x13, 0xea, 0x68, 0x8c, 0x4e, 0x06, 0xee,
0xad, 0x06, 0x5b, 0x72, 0x27, 0x7b, 0x21, 0x82, 0xc8, 0x05, 0xf8, 0xd2, 0x50, 0xa0, 0xb4, 0xc6,
0xa8, 0xbd, 0x41, 0x23, 0xf3, 0x64, 0x2f, 0x2c, 0xe6, 0x97, 0x55, 0xf0, 0xa9, 0xa1, 0x0f, 0xde,
0x42, 0xed, 0xd3, 0x4a, 0x67, 0x82, 0x71, 0x7c, 0x6e, 0xdc, 0xcb, 0x3e, 0xb7, 0x39, 0x93, 0x0e,
0xf8, 0x85, 0xd8, 0x7d, 0x14, 0x5b, 0x14, 0xc1, 0x1a, 0x9a, 0x37, 0x19, 0xe5, 0x33, 0x67, 0xd4,
0x00, 0x6a, 0xc2, 0xb2, 0x58, 0xa1, 0x64, 0x73, 0xb5, 0xe3, 0x0f, 0x4b, 0xcc, 0x1f, 0x4d, 0xdc,
0xf1, 0xe9, 0xe0, 0x37, 0x9f, 0x7e, 0x7a, 0xf6, 0x66, 0x67, 0xd4, 0x7b, 0x38, 0x56, 0xa6, 0x66,
0x7c, 0x11, 0xfd, 0x85, 0x84, 0x23, 0x07, 0x2e, 0x97, 0x7e, 0x09, 0x6d, 0xe3, 0x74, 0x49, 0xa2,
0x73, 0xab, 0xa9, 0x25, 0xe3, 0xf5, 0x8d, 0xed, 0xde, 0xe6, 0xa4, 0x0f, 0x47, 0x5b, 0x98, 0x22,
0x00, 0x07, 0x08, 0xfc, 0x4f, 0x95, 0x28, 0x4c, 0xc1, 0x10, 0x3a, 0x25, 0x92, 0xf1, 0x6c, 0xa5,
0x1f, 0xd8, 0x4d, 0xdc, 0x6c, 0x6a, 0x46, 0xe8, 0x3c, 0xb9, 0x80, 0x76, 0xf9, 0x85, 0x8d, 0x8e,
0x8f, 0xe0, 0x92, 0xba, 0x48, 0x4f, 0xc0, 0x01, 0x50, 0x52, 0x11, 0x8d, 0x01, 0xf8, 0x38, 0xb7,
0xfb, 0x76, 0x37, 0xfb, 0x6e, 0x3f, 0x91, 0x31, 0x1d, 0x61, 0xff, 0x90, 0x8e, 0x77, 0x00, 0x97,
0x26, 0xbc, 0x57, 0x99, 0x48, 0xd2, 0x47, 0xf3, 0xd1, 0x85, 0x8a, 0x55, 0x5c, 0x04, 0xc4, 0x56,
0xa3, 0x1f, 0x1e, 0xb4, 0xc6, 0x86, 0xfe, 0xda, 0xfe, 0xd6, 0xc9, 0x14, 0x4e, 0x43, 0xba, 0x60,
0x4a, 0x53, 0x69, 0xa2, 0xcb, 0xe4, 0x32, 0xd6, 0x4c, 0x70, 0x75, 0xad, 0xe7, 0x9c, 0xec, 0xe4,
0xda, 0xe6, 0xea, 0xec, 0xff, 0x87, 0x6d, 0x5c, 0x7b, 0xe8, 0x91, 0x31, 0x1c, 0x3b, 0x2a, 0xdc,
0x14, 0x29, 0x76, 0xd7, 0x77, 0x1c, 0x9d, 0x9d, 0xbe, 0x23, 0xf9, 0x08, 0x5d, 0x47, 0xb2, 0xd9,
0x11, 0x99, 0xb6, 0xbe, 0xd8, 0x4c, 0xce, 0x1e, 0xed, 0x0e, 0xbd, 0xbb, 0x0a, 0xfe, 0x89, 0xbd,
0xf9, 0x15, 0x00, 0x00, 0xff, 0xff, 0x65, 0xe6, 0xc2, 0xe4, 0xe4, 0x04, 0x00, 0x00,
}

@ -0,0 +1,177 @@
syntax = "proto3";
package chainrpc;
message ConfRequest {
/*
The transaction hash for which we should request a confirmation notification
for. If set to a hash of all zeros, then the confirmation notification will
be requested for the script instead.
*/
bytes txid = 1;
/*
An output script within a transaction with the hash above which will be used
by light clients to match block filters. If the transaction hash is set to a
hash of all zeros, then a confirmation notification will be requested for
this script instead.
*/
bytes script = 2;
/*
The number of desired confirmations the transaction/output script should
reach before dispatching a confirmation notification.
*/
uint32 num_confs = 3;
/*
The earliest height in the chain for which the transaction/output script
could have been included in a block. This should in most cases be set to the
broadcast height of the transaction/output script.
*/
uint32 height_hint = 4;
}
message ConfDetails {
// The raw bytes of the confirmed transaction.
bytes raw_tx = 1;
// The hash of the block in which the confirmed transaction was included in.
bytes block_hash = 2;
// The height of the block in which the confirmed transaction was included in.
uint32 block_height = 3;
// The index of the confirmed transaction within the transaction.
uint32 tx_index = 4;
}
message Reorg {
// TODO(wilmer): need to know how the client will use this first.
}
message ConfEvent {
oneof event {
/*
An event that includes the confirmation details of the request
(txid/ouput script).
*/
ConfDetails conf = 1;
/*
An event send when the transaction of the request is reorged out of the
chain.
*/
Reorg reorg = 2;
}
}
message Outpoint {
// The hash of the transaction.
bytes hash = 1;
// The index of the output within the transaction.
uint32 index = 2;
}
message SpendRequest {
/*
The outpoint for which we should request a spend notification for. If set to
a zero outpoint, then the spend notification will be requested for the
script instead.
*/
Outpoint outpoint = 1;
/*
The output script for the outpoint above. This will be used by light clients
to match block filters. If the outpoint is set to a zero outpoint, then a
spend notification will be requested for this script instead.
*/
bytes script = 2;
/*
The earliest height in the chain for which the outpoint/output script could
have been spent. This should in most cases be set to the broadcast height of
the outpoint/output script.
*/
uint32 height_hint = 3;
// TODO(wilmer): extend to support num confs on spending tx.
}
message SpendDetails {
// The outpoint was that spent.
Outpoint spending_outpoint = 1;
// The raw bytes of the spending transaction.
bytes raw_spending_tx = 2;
// The hash of the spending transaction.
bytes spending_tx_hash = 3;
// The input of the spending transaction that fulfilled the spend request.
uint32 spending_input_index = 4;
// The height at which the spending transaction was included in a block.
uint32 spending_height = 5;
}
message SpendEvent {
oneof event {
/*
An event that includes the details of the spending transaction of the
request (outpoint/output script).
*/
SpendDetails spend = 1;
/*
An event sent when the spending transaction of the request was
reorged out of the chain.
*/
Reorg reorg = 2;
}
}
message BlockEpoch {
// The hash of the block.
bytes hash = 1;
// The height of the block.
uint32 height = 2;
}
service ChainNotifier {
/*
RegisterConfirmationsNtfn is a synchronous response-streaming RPC that
registers an intent for a client to be notified once a confirmation request
has reached its required number of confirmations on-chain.
A client can specify whether the confirmation request should be for a
particular transaction by its hash or for an output script by specifying a
zero hash.
*/
rpc RegisterConfirmationsNtfn(ConfRequest) returns (stream ConfEvent);
/*
RegisterSpendNtfn is a synchronous response-streaming RPC that registers an
intent for a client to be notification once a spend request has been spent
by a transaction that has confirmed on-chain.
A client can specify whether the spend request should be for a particular
outpoint or for an output script by specifying a zero outpoint.
*/
rpc RegisterSpendNtfn(SpendRequest) returns (stream SpendEvent);
/*
RegisterBlockEpochNtfn is a synchronous response-streaming RPC that
registers an intent for a client to be notified of blocks in the chain. The
stream will return a hash and height tuple of a block for each new/stale
block in the chain. It is the client's responsibility to determine whether
the tuple returned is for a new or stale block in the chain.
A client can also request a historical backlog of blocks from a particular
point. This allows clients to be idempotent by ensuring that they do not
missing processing a single block within the chain.
*/
rpc RegisterBlockEpochNtfn(BlockEpoch) returns (stream BlockEpoch);
}

@ -0,0 +1,34 @@
// +build chainrpc
package chainrpc
import (
"github.com/lightningnetwork/lnd/chainntnfs"
"github.com/lightningnetwork/lnd/macaroons"
)
// Config is the primary configuration struct for the chain notifier RPC server.
// It contains all the items required for the server to carry out its duties.
// The fields with struct tags are meant to be parsed as normal configuration
// options, while if able to be populated, the latter fields MUST also be
// specified.
type Config struct {
// ChainNotifierMacPath is the path for the chain notifier macaroon. If
// unspecified then we assume that the macaroon will be found under the
// network directory, named DefaultChainNotifierMacFilename.
ChainNotifierMacPath string `long:"notifiermacaroonpath" description:"Path to the chain notifier macaroon"`
// NetworkDir is the main network directory wherein the chain notifier
// RPC server will find the macaroon named
// DefaultChainNotifierMacFilename.
NetworkDir string
// MacService is the main macaroon service that we'll use to handle
// authentication for the chain notifier RPC server.
MacService *macaroons.Service
// ChainNotifier is the chain notifier instance that backs the chain
// notifier RPC server. The job of the chain notifier RPC server is
// simply to proxy valid requests to the active chain notifier instance.
ChainNotifier chainntnfs.ChainNotifier
}

@ -0,0 +1,6 @@
// +build !chainrpc
package chainrpc
// Config is empty for non-chainrpc builds.
type Config struct{}

71
lnrpc/chainrpc/driver.go Normal file

@ -0,0 +1,71 @@
// +build chainrpc
package chainrpc
import (
"fmt"
"github.com/lightningnetwork/lnd/lnrpc"
)
// createNewSubServer is a helper method that will create the new chain notifier
// sub server given the main config dispatcher method. If we're unable to find
// the config that is meant for us in the config dispatcher, then we'll exit
// with an error.
func createNewSubServer(configRegistry lnrpc.SubServerConfigDispatcher) (
lnrpc.SubServer, lnrpc.MacaroonPerms, error) {
// We'll attempt to look up the config that we expect, according to our
// subServerName name. If we can't find this, then we'll exit with an
// error, as we're unable to properly initialize ourselves without this
// config.
chainNotifierServerConf, ok := configRegistry.FetchConfig(subServerName)
if !ok {
return nil, nil, fmt.Errorf("unable to find config for "+
"subserver type %s", subServerName)
}
// Now that we've found an object mapping to our service name, we'll
// ensure that it's the type we need.
config, ok := chainNotifierServerConf.(*Config)
if !ok {
return nil, nil, fmt.Errorf("wrong type of config for "+
"subserver %s, expected %T got %T", subServerName,
&Config{}, chainNotifierServerConf)
}
// Before we try to make the new chain notifier service instance, we'll
// perform some sanity checks on the arguments to ensure that they're
// usable.
switch {
// If the macaroon service is set (we should use macaroons), then
// ensure that we know where to look for them, or create them if not
// found.
case config.MacService != nil && config.NetworkDir == "":
return nil, nil, fmt.Errorf("NetworkDir must be set to create " +
"chainrpc")
case config.ChainNotifier == nil:
return nil, nil, fmt.Errorf("ChainNotifier must be set to " +
"create chainrpc")
}
return New(config)
}
func init() {
subServer := &lnrpc.SubServerDriver{
SubServerName: subServerName,
New: func(c lnrpc.SubServerConfigDispatcher) (
lnrpc.SubServer, lnrpc.MacaroonPerms, error) {
return createNewSubServer(c)
},
}
// If the build tag is active, then we'll register ourselves as a
// sub-RPC server within the global lnrpc package namespace.
if err := lnrpc.RegisterSubServer(subServer); err != nil {
panic(fmt.Sprintf("failed to register subserver driver %s: %v",
subServerName, err))
}
}

45
lnrpc/chainrpc/log.go Normal file

@ -0,0 +1,45 @@
package chainrpc
import (
"github.com/btcsuite/btclog"
"github.com/lightningnetwork/lnd/build"
)
// 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() {
UseLogger(build.NewSubLogger("NTFR", nil))
}
// DisableLog disables all library log output. Logging output is disabled
// by default until UseLogger is 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 used to provide a closure over expensive logging operations so
// don't have to be performed when the logging level doesn't warrant it.
type logClosure func() string
// String invokes the underlying function and returns the result.
func (c logClosure) String() string {
return c()
}
// newLogClosure returns a new closure over a function that returns a string
// which itself provides a Stringer interface so that it can be used with the
// logging system.
func newLogClosure(c func() string) logClosure {
return logClosure(c)
}

@ -2362,7 +2362,9 @@ func TestLightningWallet(t *testing.T) {
if err != nil {
t.Fatalf("unable to create height hint cache: %v", err)
}
chainNotifier, err := btcdnotify.New(&rpcConfig, hintCache, hintCache)
chainNotifier, err := btcdnotify.New(
&rpcConfig, netParams, hintCache, hintCache,
)
if err != nil {
t.Fatalf("unable to create notifier: %v", err)
}

5
log.go

@ -10,7 +10,6 @@ import (
"github.com/btcsuite/btclog"
"github.com/jrick/logrotate/rotator"
"github.com/lightninglabs/neutrino"
"github.com/lightningnetwork/lightning-onion"
"github.com/lightningnetwork/lnd/autopilot"
"github.com/lightningnetwork/lnd/build"
@ -21,6 +20,7 @@ import (
"github.com/lightningnetwork/lnd/htlcswitch"
"github.com/lightningnetwork/lnd/invoices"
"github.com/lightningnetwork/lnd/lnrpc/autopilotrpc"
"github.com/lightningnetwork/lnd/lnrpc/chainrpc"
"github.com/lightningnetwork/lnd/lnrpc/signrpc"
"github.com/lightningnetwork/lnd/lnrpc/walletrpc"
"github.com/lightningnetwork/lnd/lnwallet"
@ -77,6 +77,7 @@ var (
invcLog = build.NewSubLogger("INVC", backendLog.Logger)
nannLog = build.NewSubLogger("NANN", backendLog.Logger)
wtwrLog = build.NewSubLogger("WTWR", backendLog.Logger)
ntfrLog = build.NewSubLogger("NTFR", backendLog.Logger)
)
// Initialize package-global logger variables.
@ -100,6 +101,7 @@ func init() {
invoices.UseLogger(invcLog)
netann.UseLogger(nannLog)
watchtower.UseLogger(wtwrLog)
chainrpc.UseLogger(ntfrLog)
}
// subsystemLoggers maps each subsystem identifier to its associated logger.
@ -129,6 +131,7 @@ var subsystemLoggers = map[string]btclog.Logger{
"INVC": invcLog,
"NANN": nannLog,
"WTWR": wtwrLog,
"NTFR": ntfnLog,
}
// initLogRotator initializes the logging rotator to write logs to logFile and

@ -6,6 +6,7 @@ import (
"github.com/lightningnetwork/lnd/autopilot"
"github.com/lightningnetwork/lnd/lnrpc/autopilotrpc"
"github.com/lightningnetwork/lnd/lnrpc/chainrpc"
"github.com/lightningnetwork/lnd/lnrpc/signrpc"
"github.com/lightningnetwork/lnd/lnrpc/walletrpc"
"github.com/lightningnetwork/lnd/macaroons"
@ -31,6 +32,11 @@ type subRPCServerConfigs struct {
// AutopilotRPC is a sub-RPC server that exposes methods on the running
// autopilot as a gRPC service.
AutopilotRPC *autopilotrpc.Config `group:"autopilotrpc" namespace:"autopilotrpc"`
// ChainRPC is a sub-RPC server that exposes functionality allowing a
// client to be notified of certain on-chain events (new blocks,
// confirmations, spends).
ChainRPC *chainrpc.Config `group:"chainrpc" namespace:"chainrpc"`
}
// PopulateDependencies attempts to iterate through all the sub-server configs
@ -106,6 +112,19 @@ func (s *subRPCServerConfigs) PopulateDependencies(cc *chainControl,
reflect.ValueOf(atpl),
)
case *chainrpc.Config:
subCfgValue := extractReflectValue(cfg)
subCfgValue.FieldByName("NetworkDir").Set(
reflect.ValueOf(networkDir),
)
subCfgValue.FieldByName("MacService").Set(
reflect.ValueOf(macService),
)
subCfgValue.FieldByName("ChainNotifier").Set(
reflect.ValueOf(cc.chainNotifier),
)
default:
return fmt.Errorf("unknown field: %v, %T", fieldName,
cfg)