Merge pull request #2314 from wpaulino/chainnotifier-subserver
chainntnfs+lnrpc/chainrpc: add ChainNotifier RPC subserver
This commit is contained in:
commit
375be936ce
@ -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, ¶ms)
|
||||
_, addrs, _, err := txscript.ExtractPkScriptAddrs(
|
||||
confRequest.PkScript.Script(), ¶ms,
|
||||
)
|
||||
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
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
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=
|
||||
|
459
lnrpc/chainrpc/chainnotifer_server.go
Normal file
459
lnrpc/chainrpc/chainnotifer_server.go
Normal file
@ -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
|
||||
}
|
||||
}
|
||||
}
|
1050
lnrpc/chainrpc/chainnotifier.pb.go
Normal file
1050
lnrpc/chainrpc/chainnotifier.pb.go
Normal file
@ -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,
|
||||
}
|
177
lnrpc/chainrpc/chainnotifier.proto
Normal file
177
lnrpc/chainrpc/chainnotifier.proto
Normal file
@ -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);
|
||||
}
|
34
lnrpc/chainrpc/config_active.go
Normal file
34
lnrpc/chainrpc/config_active.go
Normal file
@ -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
|
||||
}
|
6
lnrpc/chainrpc/config_default.go
Normal file
6
lnrpc/chainrpc/config_default.go
Normal file
@ -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
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
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
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)
|
||||
|
Loading…
Reference in New Issue
Block a user