Merge pull request #1210 from wpaulino/indexing-use-spend-journal
blockchain: modify indexers to use the stxo set for a block rather than the utxo set
This commit is contained in:
commit
991d32e72f
13 changed files with 235 additions and 165 deletions
|
@ -557,7 +557,9 @@ func (b *BlockChain) getReorganizeNodes(node *blockNode) (*list.List, *list.List
|
|||
// it would be inefficient to repeat it.
|
||||
//
|
||||
// This function MUST be called with the chain state lock held (for writes).
|
||||
func (b *BlockChain) connectBlock(node *blockNode, block *btcutil.Block, view *UtxoViewpoint, stxos []spentTxOut) error {
|
||||
func (b *BlockChain) connectBlock(node *blockNode, block *btcutil.Block,
|
||||
view *UtxoViewpoint, stxos []SpentTxOut) error {
|
||||
|
||||
// Make sure it's extending the end of the best chain.
|
||||
prevHash := &block.MsgBlock().Header.PrevBlock
|
||||
if !prevHash.IsEqual(&b.bestChain.Tip().hash) {
|
||||
|
@ -638,7 +640,7 @@ func (b *BlockChain) connectBlock(node *blockNode, block *btcutil.Block, view *U
|
|||
// optional indexes with the block being connected so they can
|
||||
// update themselves accordingly.
|
||||
if b.indexManager != nil {
|
||||
err := b.indexManager.ConnectBlock(dbTx, block, view)
|
||||
err := b.indexManager.ConnectBlock(dbTx, block, stxos)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -739,6 +741,13 @@ func (b *BlockChain) disconnectBlock(node *blockNode, block *btcutil.Block, view
|
|||
return err
|
||||
}
|
||||
|
||||
// Before we delete the spend journal entry for this back,
|
||||
// we'll fetch it as is so the indexers can utilize if needed.
|
||||
stxos, err := dbFetchSpendJournalEntry(dbTx, block)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Update the transaction spend journal by removing the record
|
||||
// that contains all txos spent by the block.
|
||||
err = dbRemoveSpendJournalEntry(dbTx, block.Hash())
|
||||
|
@ -750,7 +759,7 @@ func (b *BlockChain) disconnectBlock(node *blockNode, block *btcutil.Block, view
|
|||
// optional indexes with the block being disconnected so they
|
||||
// can update themselves accordingly.
|
||||
if b.indexManager != nil {
|
||||
err := b.indexManager.DisconnectBlock(dbTx, block, view)
|
||||
err := b.indexManager.DisconnectBlock(dbTx, block, stxos)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -816,7 +825,7 @@ func (b *BlockChain) reorganizeChain(detachNodes, attachNodes *list.List) error
|
|||
// then they are needed again when doing the actual database updates.
|
||||
// Rather than doing two loads, cache the loaded data into these slices.
|
||||
detachBlocks := make([]*btcutil.Block, 0, detachNodes.Len())
|
||||
detachSpentTxOuts := make([][]spentTxOut, 0, detachNodes.Len())
|
||||
detachSpentTxOuts := make([][]SpentTxOut, 0, detachNodes.Len())
|
||||
attachBlocks := make([]*btcutil.Block, 0, attachNodes.Len())
|
||||
|
||||
// Disconnect all of the blocks back to the point of the fork. This
|
||||
|
@ -846,7 +855,7 @@ func (b *BlockChain) reorganizeChain(detachNodes, attachNodes *list.List) error
|
|||
|
||||
// Load all of the spent txos for the block from the spend
|
||||
// journal.
|
||||
var stxos []spentTxOut
|
||||
var stxos []SpentTxOut
|
||||
err = b.db.View(func(dbTx database.Tx) error {
|
||||
stxos, err = dbFetchSpendJournalEntry(dbTx, block)
|
||||
return err
|
||||
|
@ -990,7 +999,7 @@ func (b *BlockChain) reorganizeChain(detachNodes, attachNodes *list.List) error
|
|||
// as spent and add all transactions being created by this block
|
||||
// to it. Also, provide an stxo slice so the spent txout
|
||||
// details are generated.
|
||||
stxos := make([]spentTxOut, 0, countSpentOutputs(block))
|
||||
stxos := make([]SpentTxOut, 0, countSpentOutputs(block))
|
||||
err = view.connectTransactions(block, &stxos)
|
||||
if err != nil {
|
||||
return err
|
||||
|
@ -1044,7 +1053,7 @@ func (b *BlockChain) connectBestChain(node *blockNode, block *btcutil.Block, fla
|
|||
// actually connecting the block.
|
||||
view := NewUtxoViewpoint()
|
||||
view.SetBestHash(parentHash)
|
||||
stxos := make([]spentTxOut, 0, countSpentOutputs(block))
|
||||
stxos := make([]SpentTxOut, 0, countSpentOutputs(block))
|
||||
if !fastAdd {
|
||||
err := b.checkConnectBlock(node, block, view, &stxos)
|
||||
if err == nil {
|
||||
|
@ -1569,12 +1578,16 @@ type IndexManager interface {
|
|||
Init(*BlockChain, <-chan struct{}) error
|
||||
|
||||
// ConnectBlock is invoked when a new block has been connected to the
|
||||
// main chain.
|
||||
ConnectBlock(database.Tx, *btcutil.Block, *UtxoViewpoint) error
|
||||
// main chain. The set of output spent within a block is also passed in
|
||||
// so indexers can access the previous output scripts input spent if
|
||||
// required.
|
||||
ConnectBlock(database.Tx, *btcutil.Block, []SpentTxOut) error
|
||||
|
||||
// DisconnectBlock is invoked when a block has been disconnected from
|
||||
// the main chain.
|
||||
DisconnectBlock(database.Tx, *btcutil.Block, *UtxoViewpoint) error
|
||||
// the main chain. The set of outputs scripts that were spent within
|
||||
// this block is also returned so indexers can clean up the prior index
|
||||
// state for this block.
|
||||
DisconnectBlock(database.Tx, *btcutil.Block, []SpentTxOut) error
|
||||
}
|
||||
|
||||
// Config is a descriptor which specifies the blockchain instance configuration.
|
||||
|
|
|
@ -232,28 +232,59 @@ func dbFetchOrCreateVersion(dbTx database.Tx, key []byte, defaultVersion uint32)
|
|||
// - 0xb2...ec: pubkey hash
|
||||
// -----------------------------------------------------------------------------
|
||||
|
||||
// spentTxOut contains a spent transaction output and potentially additional
|
||||
// SpentTxOut contains a spent transaction output and potentially additional
|
||||
// contextual information such as whether or not it was contained in a coinbase
|
||||
// transaction, the version of the transaction it was contained in, and which
|
||||
// block height the containing transaction was included in. As described in
|
||||
// the comments above, the additional contextual information will only be valid
|
||||
// when this spent txout is spending the last unspent output of the containing
|
||||
// transaction.
|
||||
type spentTxOut struct {
|
||||
amount int64 // The amount of the output.
|
||||
pkScript []byte // The public key script for the output.
|
||||
height int32 // Height of the the block containing the creating tx.
|
||||
isCoinBase bool // Whether creating tx is a coinbase.
|
||||
type SpentTxOut struct {
|
||||
// Amount is the amount of the output.
|
||||
Amount int64
|
||||
|
||||
// PkScipt is the the public key script for the output.
|
||||
PkScript []byte
|
||||
|
||||
// Height is the height of the the block containing the creating tx.
|
||||
Height int32
|
||||
|
||||
// Denotes if the creating tx is a coinbase.
|
||||
IsCoinBase bool
|
||||
}
|
||||
|
||||
// FetchSpendJournal attempts to retrieve the spend journal, or the set of
|
||||
// outputs spent for the target block. This provides a view of all the outputs
|
||||
// that will be consumed once the target block is connected to the end of the
|
||||
// main chain.
|
||||
//
|
||||
// This function is safe for concurrent access.
|
||||
func (b *BlockChain) FetchSpendJournal(targetBlock *btcutil.Block) ([]SpentTxOut, error) {
|
||||
b.chainLock.RLock()
|
||||
defer b.chainLock.RUnlock()
|
||||
|
||||
var spendEntries []SpentTxOut
|
||||
err := b.db.View(func(dbTx database.Tx) error {
|
||||
var err error
|
||||
|
||||
spendEntries, err = dbFetchSpendJournalEntry(dbTx, targetBlock)
|
||||
return err
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return spendEntries, nil
|
||||
}
|
||||
|
||||
// spentTxOutHeaderCode returns the calculated header code to be used when
|
||||
// serializing the provided stxo entry.
|
||||
func spentTxOutHeaderCode(stxo *spentTxOut) uint64 {
|
||||
func spentTxOutHeaderCode(stxo *SpentTxOut) uint64 {
|
||||
// As described in the serialization format comments, the header code
|
||||
// encodes the height shifted over one bit and the coinbase flag in the
|
||||
// lowest bit.
|
||||
headerCode := uint64(stxo.height) << 1
|
||||
if stxo.isCoinBase {
|
||||
headerCode := uint64(stxo.Height) << 1
|
||||
if stxo.IsCoinBase {
|
||||
headerCode |= 0x01
|
||||
}
|
||||
|
||||
|
@ -262,38 +293,38 @@ func spentTxOutHeaderCode(stxo *spentTxOut) uint64 {
|
|||
|
||||
// spentTxOutSerializeSize returns the number of bytes it would take to
|
||||
// serialize the passed stxo according to the format described above.
|
||||
func spentTxOutSerializeSize(stxo *spentTxOut) int {
|
||||
func spentTxOutSerializeSize(stxo *SpentTxOut) int {
|
||||
size := serializeSizeVLQ(spentTxOutHeaderCode(stxo))
|
||||
if stxo.height > 0 {
|
||||
if stxo.Height > 0 {
|
||||
// The legacy v1 spend journal format conditionally tracked the
|
||||
// containing transaction version when the height was non-zero,
|
||||
// so this is required for backwards compat.
|
||||
size += serializeSizeVLQ(0)
|
||||
}
|
||||
return size + compressedTxOutSize(uint64(stxo.amount), stxo.pkScript)
|
||||
return size + compressedTxOutSize(uint64(stxo.Amount), stxo.PkScript)
|
||||
}
|
||||
|
||||
// putSpentTxOut serializes the passed stxo according to the format described
|
||||
// above directly into the passed target byte slice. The target byte slice must
|
||||
// be at least large enough to handle the number of bytes returned by the
|
||||
// spentTxOutSerializeSize function or it will panic.
|
||||
func putSpentTxOut(target []byte, stxo *spentTxOut) int {
|
||||
// SpentTxOutSerializeSize function or it will panic.
|
||||
func putSpentTxOut(target []byte, stxo *SpentTxOut) int {
|
||||
headerCode := spentTxOutHeaderCode(stxo)
|
||||
offset := putVLQ(target, headerCode)
|
||||
if stxo.height > 0 {
|
||||
if stxo.Height > 0 {
|
||||
// The legacy v1 spend journal format conditionally tracked the
|
||||
// containing transaction version when the height was non-zero,
|
||||
// so this is required for backwards compat.
|
||||
offset += putVLQ(target[offset:], 0)
|
||||
}
|
||||
return offset + putCompressedTxOut(target[offset:], uint64(stxo.amount),
|
||||
stxo.pkScript)
|
||||
return offset + putCompressedTxOut(target[offset:], uint64(stxo.Amount),
|
||||
stxo.PkScript)
|
||||
}
|
||||
|
||||
// decodeSpentTxOut decodes the passed serialized stxo entry, possibly followed
|
||||
// by other data, into the passed stxo struct. It returns the number of bytes
|
||||
// read.
|
||||
func decodeSpentTxOut(serialized []byte, stxo *spentTxOut) (int, error) {
|
||||
func decodeSpentTxOut(serialized []byte, stxo *SpentTxOut) (int, error) {
|
||||
// Ensure there are bytes to decode.
|
||||
if len(serialized) == 0 {
|
||||
return 0, errDeserialize("no serialized bytes")
|
||||
|
@ -310,9 +341,9 @@ func decodeSpentTxOut(serialized []byte, stxo *spentTxOut) (int, error) {
|
|||
//
|
||||
// Bit 0 indicates containing transaction is a coinbase.
|
||||
// Bits 1-x encode height of containing transaction.
|
||||
stxo.isCoinBase = code&0x01 != 0
|
||||
stxo.height = int32(code >> 1)
|
||||
if stxo.height > 0 {
|
||||
stxo.IsCoinBase = code&0x01 != 0
|
||||
stxo.Height = int32(code >> 1)
|
||||
if stxo.Height > 0 {
|
||||
// The legacy v1 spend journal format conditionally tracked the
|
||||
// containing transaction version when the height was non-zero,
|
||||
// so this is required for backwards compat.
|
||||
|
@ -332,8 +363,8 @@ func decodeSpentTxOut(serialized []byte, stxo *spentTxOut) (int, error) {
|
|||
return offset, errDeserialize(fmt.Sprintf("unable to decode "+
|
||||
"txout: %v", err))
|
||||
}
|
||||
stxo.amount = int64(amount)
|
||||
stxo.pkScript = pkScript
|
||||
stxo.Amount = int64(amount)
|
||||
stxo.PkScript = pkScript
|
||||
return offset, nil
|
||||
}
|
||||
|
||||
|
@ -343,7 +374,7 @@ func decodeSpentTxOut(serialized []byte, stxo *spentTxOut) (int, error) {
|
|||
// Since the serialization format is not self describing, as noted in the
|
||||
// format comments, this function also requires the transactions that spend the
|
||||
// txouts.
|
||||
func deserializeSpendJournalEntry(serialized []byte, txns []*wire.MsgTx) ([]spentTxOut, error) {
|
||||
func deserializeSpendJournalEntry(serialized []byte, txns []*wire.MsgTx) ([]SpentTxOut, error) {
|
||||
// Calculate the total number of stxos.
|
||||
var numStxos int
|
||||
for _, tx := range txns {
|
||||
|
@ -368,7 +399,7 @@ func deserializeSpendJournalEntry(serialized []byte, txns []*wire.MsgTx) ([]spen
|
|||
// reverse order to match the serialization order.
|
||||
stxoIdx := numStxos - 1
|
||||
offset := 0
|
||||
stxos := make([]spentTxOut, numStxos)
|
||||
stxos := make([]SpentTxOut, numStxos)
|
||||
for txIdx := len(txns) - 1; txIdx > -1; txIdx-- {
|
||||
tx := txns[txIdx]
|
||||
|
||||
|
@ -394,7 +425,7 @@ func deserializeSpendJournalEntry(serialized []byte, txns []*wire.MsgTx) ([]spen
|
|||
|
||||
// serializeSpendJournalEntry serializes all of the passed spent txouts into a
|
||||
// single byte slice according to the format described in detail above.
|
||||
func serializeSpendJournalEntry(stxos []spentTxOut) []byte {
|
||||
func serializeSpendJournalEntry(stxos []SpentTxOut) []byte {
|
||||
if len(stxos) == 0 {
|
||||
return nil
|
||||
}
|
||||
|
@ -422,7 +453,7 @@ func serializeSpendJournalEntry(stxos []spentTxOut) []byte {
|
|||
// NOTE: Legacy entries will not have the coinbase flag or height set unless it
|
||||
// was the final output spend in the containing transaction. It is up to the
|
||||
// caller to handle this properly by looking the information up in the utxo set.
|
||||
func dbFetchSpendJournalEntry(dbTx database.Tx, block *btcutil.Block) ([]spentTxOut, error) {
|
||||
func dbFetchSpendJournalEntry(dbTx database.Tx, block *btcutil.Block) ([]SpentTxOut, error) {
|
||||
// Exclude the coinbase transaction since it can't spend anything.
|
||||
spendBucket := dbTx.Metadata().Bucket(spendJournalBucketName)
|
||||
serialized := spendBucket.Get(block.Hash()[:])
|
||||
|
@ -450,7 +481,7 @@ func dbFetchSpendJournalEntry(dbTx database.Tx, block *btcutil.Block) ([]spentTx
|
|||
// spend journal entry for the given block hash using the provided slice of
|
||||
// spent txouts. The spent txouts slice must contain an entry for every txout
|
||||
// the transactions in the block spend in the order they are spent.
|
||||
func dbPutSpendJournalEntry(dbTx database.Tx, blockHash *chainhash.Hash, stxos []spentTxOut) error {
|
||||
func dbPutSpendJournalEntry(dbTx database.Tx, blockHash *chainhash.Hash, stxos []SpentTxOut) error {
|
||||
spendBucket := dbTx.Metadata().Bucket(spendJournalBucketName)
|
||||
serialized := serializeSpendJournalEntry(stxos)
|
||||
return spendBucket.Put(blockHash[:], serialized)
|
||||
|
|
|
@ -44,37 +44,37 @@ func TestStxoSerialization(t *testing.T) {
|
|||
|
||||
tests := []struct {
|
||||
name string
|
||||
stxo spentTxOut
|
||||
stxo SpentTxOut
|
||||
serialized []byte
|
||||
}{
|
||||
// From block 170 in main blockchain.
|
||||
{
|
||||
name: "Spends last output of coinbase",
|
||||
stxo: spentTxOut{
|
||||
amount: 5000000000,
|
||||
pkScript: hexToBytes("410411db93e1dcdb8a016b49840f8c53bc1eb68a382e97b1482ecad7b148a6909a5cb2e0eaddfb84ccf9744464f82e160bfa9b8b64f9d4c03f999b8643f656b412a3ac"),
|
||||
isCoinBase: true,
|
||||
height: 9,
|
||||
stxo: SpentTxOut{
|
||||
Amount: 5000000000,
|
||||
PkScript: hexToBytes("410411db93e1dcdb8a016b49840f8c53bc1eb68a382e97b1482ecad7b148a6909a5cb2e0eaddfb84ccf9744464f82e160bfa9b8b64f9d4c03f999b8643f656b412a3ac"),
|
||||
IsCoinBase: true,
|
||||
Height: 9,
|
||||
},
|
||||
serialized: hexToBytes("1300320511db93e1dcdb8a016b49840f8c53bc1eb68a382e97b1482ecad7b148a6909a5c"),
|
||||
},
|
||||
// Adapted from block 100025 in main blockchain.
|
||||
{
|
||||
name: "Spends last output of non coinbase",
|
||||
stxo: spentTxOut{
|
||||
amount: 13761000000,
|
||||
pkScript: hexToBytes("76a914b2fb57eadf61e106a100a7445a8c3f67898841ec88ac"),
|
||||
isCoinBase: false,
|
||||
height: 100024,
|
||||
stxo: SpentTxOut{
|
||||
Amount: 13761000000,
|
||||
PkScript: hexToBytes("76a914b2fb57eadf61e106a100a7445a8c3f67898841ec88ac"),
|
||||
IsCoinBase: false,
|
||||
Height: 100024,
|
||||
},
|
||||
serialized: hexToBytes("8b99700086c64700b2fb57eadf61e106a100a7445a8c3f67898841ec"),
|
||||
},
|
||||
// Adapted from block 100025 in main blockchain.
|
||||
{
|
||||
name: "Does not spend last output, legacy format",
|
||||
stxo: spentTxOut{
|
||||
amount: 34405000000,
|
||||
pkScript: hexToBytes("76a9146edbc6c4d31bae9f1ccc38538a114bf42de65e8688ac"),
|
||||
stxo: SpentTxOut{
|
||||
Amount: 34405000000,
|
||||
PkScript: hexToBytes("76a9146edbc6c4d31bae9f1ccc38538a114bf42de65e8688ac"),
|
||||
},
|
||||
serialized: hexToBytes("0091f20f006edbc6c4d31bae9f1ccc38538a114bf42de65e86"),
|
||||
},
|
||||
|
@ -85,7 +85,7 @@ func TestStxoSerialization(t *testing.T) {
|
|||
// actually serializing it is calculated properly.
|
||||
gotSize := spentTxOutSerializeSize(&test.stxo)
|
||||
if gotSize != len(test.serialized) {
|
||||
t.Errorf("spentTxOutSerializeSize (%s): did not get "+
|
||||
t.Errorf("SpentTxOutSerializeSize (%s): did not get "+
|
||||
"expected size - got %d, want %d", test.name,
|
||||
gotSize, len(test.serialized))
|
||||
continue
|
||||
|
@ -110,7 +110,7 @@ func TestStxoSerialization(t *testing.T) {
|
|||
|
||||
// Ensure the serialized bytes are decoded back to the expected
|
||||
// stxo.
|
||||
var gotStxo spentTxOut
|
||||
var gotStxo SpentTxOut
|
||||
gotBytesRead, err := decodeSpentTxOut(test.serialized, &gotStxo)
|
||||
if err != nil {
|
||||
t.Errorf("decodeSpentTxOut (%s): unexpected error: %v",
|
||||
|
@ -138,42 +138,42 @@ func TestStxoDecodeErrors(t *testing.T) {
|
|||
|
||||
tests := []struct {
|
||||
name string
|
||||
stxo spentTxOut
|
||||
stxo SpentTxOut
|
||||
serialized []byte
|
||||
bytesRead int // Expected number of bytes read.
|
||||
errType error
|
||||
}{
|
||||
{
|
||||
name: "nothing serialized",
|
||||
stxo: spentTxOut{},
|
||||
stxo: SpentTxOut{},
|
||||
serialized: hexToBytes(""),
|
||||
errType: errDeserialize(""),
|
||||
bytesRead: 0,
|
||||
},
|
||||
{
|
||||
name: "no data after header code w/o reserved",
|
||||
stxo: spentTxOut{},
|
||||
stxo: SpentTxOut{},
|
||||
serialized: hexToBytes("00"),
|
||||
errType: errDeserialize(""),
|
||||
bytesRead: 1,
|
||||
},
|
||||
{
|
||||
name: "no data after header code with reserved",
|
||||
stxo: spentTxOut{},
|
||||
stxo: SpentTxOut{},
|
||||
serialized: hexToBytes("13"),
|
||||
errType: errDeserialize(""),
|
||||
bytesRead: 1,
|
||||
},
|
||||
{
|
||||
name: "no data after reserved",
|
||||
stxo: spentTxOut{},
|
||||
stxo: SpentTxOut{},
|
||||
serialized: hexToBytes("1300"),
|
||||
errType: errDeserialize(""),
|
||||
bytesRead: 2,
|
||||
},
|
||||
{
|
||||
name: "incomplete compressed txout",
|
||||
stxo: spentTxOut{},
|
||||
stxo: SpentTxOut{},
|
||||
serialized: hexToBytes("1332"),
|
||||
errType: errDeserialize(""),
|
||||
bytesRead: 2,
|
||||
|
@ -208,7 +208,7 @@ func TestSpendJournalSerialization(t *testing.T) {
|
|||
|
||||
tests := []struct {
|
||||
name string
|
||||
entry []spentTxOut
|
||||
entry []SpentTxOut
|
||||
blockTxns []*wire.MsgTx
|
||||
serialized []byte
|
||||
}{
|
||||
|
@ -222,11 +222,11 @@ func TestSpendJournalSerialization(t *testing.T) {
|
|||
// From block 170 in main blockchain.
|
||||
{
|
||||
name: "One tx with one input spends last output of coinbase",
|
||||
entry: []spentTxOut{{
|
||||
amount: 5000000000,
|
||||
pkScript: hexToBytes("410411db93e1dcdb8a016b49840f8c53bc1eb68a382e97b1482ecad7b148a6909a5cb2e0eaddfb84ccf9744464f82e160bfa9b8b64f9d4c03f999b8643f656b412a3ac"),
|
||||
isCoinBase: true,
|
||||
height: 9,
|
||||
entry: []SpentTxOut{{
|
||||
Amount: 5000000000,
|
||||
PkScript: hexToBytes("410411db93e1dcdb8a016b49840f8c53bc1eb68a382e97b1482ecad7b148a6909a5cb2e0eaddfb84ccf9744464f82e160bfa9b8b64f9d4c03f999b8643f656b412a3ac"),
|
||||
IsCoinBase: true,
|
||||
Height: 9,
|
||||
}},
|
||||
blockTxns: []*wire.MsgTx{{ // Coinbase omitted.
|
||||
Version: 1,
|
||||
|
@ -252,16 +252,16 @@ func TestSpendJournalSerialization(t *testing.T) {
|
|||
// Adapted from block 100025 in main blockchain.
|
||||
{
|
||||
name: "Two txns when one spends last output, one doesn't",
|
||||
entry: []spentTxOut{{
|
||||
amount: 34405000000,
|
||||
pkScript: hexToBytes("76a9146edbc6c4d31bae9f1ccc38538a114bf42de65e8688ac"),
|
||||
isCoinBase: false,
|
||||
height: 100024,
|
||||
entry: []SpentTxOut{{
|
||||
Amount: 34405000000,
|
||||
PkScript: hexToBytes("76a9146edbc6c4d31bae9f1ccc38538a114bf42de65e8688ac"),
|
||||
IsCoinBase: false,
|
||||
Height: 100024,
|
||||
}, {
|
||||
amount: 13761000000,
|
||||
pkScript: hexToBytes("76a914b2fb57eadf61e106a100a7445a8c3f67898841ec88ac"),
|
||||
isCoinBase: false,
|
||||
height: 100024,
|
||||
Amount: 13761000000,
|
||||
PkScript: hexToBytes("76a914b2fb57eadf61e106a100a7445a8c3f67898841ec88ac"),
|
||||
IsCoinBase: false,
|
||||
Height: 100024,
|
||||
}},
|
||||
blockTxns: []*wire.MsgTx{{ // Coinbase omitted.
|
||||
Version: 1,
|
||||
|
|
|
@ -692,23 +692,26 @@ func (idx *AddrIndex) indexPkScript(data writeIndexData, pkScript []byte, txIdx
|
|||
// indexBlock extract all of the standard addresses from all of the transactions
|
||||
// in the passed block and maps each of them to the associated transaction using
|
||||
// the passed map.
|
||||
func (idx *AddrIndex) indexBlock(data writeIndexData, block *btcutil.Block, view *blockchain.UtxoViewpoint) {
|
||||
func (idx *AddrIndex) indexBlock(data writeIndexData, block *btcutil.Block,
|
||||
stxos []blockchain.SpentTxOut) {
|
||||
|
||||
stxoIndex := 0
|
||||
for txIdx, tx := range block.Transactions() {
|
||||
// Coinbases do not reference any inputs. Since the block is
|
||||
// required to have already gone through full validation, it has
|
||||
// already been proven on the first transaction in the block is
|
||||
// a coinbase.
|
||||
if txIdx != 0 {
|
||||
for _, txIn := range tx.MsgTx().TxIn {
|
||||
// The view should always have the input since
|
||||
// the index contract requires it, however, be
|
||||
// safe and simply ignore any missing entries.
|
||||
entry := view.LookupEntry(txIn.PreviousOutPoint)
|
||||
if entry == nil {
|
||||
continue
|
||||
}
|
||||
for range tx.MsgTx().TxIn {
|
||||
// We'll access the slice of all the
|
||||
// transactions spent in this block properly
|
||||
// ordered to fetch the previous input script.
|
||||
pkScript := stxos[stxoIndex].PkScript
|
||||
idx.indexPkScript(data, pkScript, txIdx)
|
||||
|
||||
idx.indexPkScript(data, entry.PkScript(), txIdx)
|
||||
// With an input indexed, we'll advance the
|
||||
// stxo coutner.
|
||||
stxoIndex++
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -723,7 +726,9 @@ func (idx *AddrIndex) indexBlock(data writeIndexData, block *btcutil.Block, view
|
|||
// the transactions in the block involve.
|
||||
//
|
||||
// This is part of the Indexer interface.
|
||||
func (idx *AddrIndex) ConnectBlock(dbTx database.Tx, block *btcutil.Block, view *blockchain.UtxoViewpoint) error {
|
||||
func (idx *AddrIndex) ConnectBlock(dbTx database.Tx, block *btcutil.Block,
|
||||
stxos []blockchain.SpentTxOut) error {
|
||||
|
||||
// The offset and length of the transactions within the serialized
|
||||
// block.
|
||||
txLocs, err := block.TxLoc()
|
||||
|
@ -739,7 +744,7 @@ func (idx *AddrIndex) ConnectBlock(dbTx database.Tx, block *btcutil.Block, view
|
|||
|
||||
// Build all of the address to transaction mappings in a local map.
|
||||
addrsToTxns := make(writeIndexData)
|
||||
idx.indexBlock(addrsToTxns, block, view)
|
||||
idx.indexBlock(addrsToTxns, block, stxos)
|
||||
|
||||
// Add all of the index entries for each address.
|
||||
addrIdxBucket := dbTx.Metadata().Bucket(addrIndexKey)
|
||||
|
@ -761,10 +766,12 @@ func (idx *AddrIndex) ConnectBlock(dbTx database.Tx, block *btcutil.Block, view
|
|||
// each transaction in the block involve.
|
||||
//
|
||||
// This is part of the Indexer interface.
|
||||
func (idx *AddrIndex) DisconnectBlock(dbTx database.Tx, block *btcutil.Block, view *blockchain.UtxoViewpoint) error {
|
||||
func (idx *AddrIndex) DisconnectBlock(dbTx database.Tx, block *btcutil.Block,
|
||||
stxos []blockchain.SpentTxOut) error {
|
||||
|
||||
// Build all of the address to transaction mappings in a local map.
|
||||
addrsToTxns := make(writeIndexData)
|
||||
idx.indexBlock(addrsToTxns, block, view)
|
||||
idx.indexBlock(addrsToTxns, block, stxos)
|
||||
|
||||
// Remove all of the index entries for each address.
|
||||
bucket := dbTx.Metadata().Bucket(addrIndexKey)
|
||||
|
|
|
@ -202,7 +202,7 @@ func storeFilter(dbTx database.Tx, block *btcutil.Block, f *gcs.Filter,
|
|||
// connected to the main chain. This indexer adds a hash-to-cf mapping for
|
||||
// every passed block. This is part of the Indexer interface.
|
||||
func (idx *CfIndex) ConnectBlock(dbTx database.Tx, block *btcutil.Block,
|
||||
view *blockchain.UtxoViewpoint) error {
|
||||
stxos []blockchain.SpentTxOut) error {
|
||||
|
||||
f, err := builder.BuildBasicFilter(block.MsgBlock())
|
||||
if err != nil {
|
||||
|
@ -226,7 +226,7 @@ func (idx *CfIndex) ConnectBlock(dbTx database.Tx, block *btcutil.Block,
|
|||
// disconnected from the main chain. This indexer removes the hash-to-cf
|
||||
// mapping for every passed block. This is part of the Indexer interface.
|
||||
func (idx *CfIndex) DisconnectBlock(dbTx database.Tx, block *btcutil.Block,
|
||||
view *blockchain.UtxoViewpoint) error {
|
||||
_ []blockchain.SpentTxOut) error {
|
||||
|
||||
for _, key := range cfIndexKeys {
|
||||
err := dbDeleteFilterIdxEntry(dbTx, key, block.Hash())
|
||||
|
|
|
@ -50,13 +50,17 @@ type Indexer interface {
|
|||
// every load, including the case the index was just created.
|
||||
Init() error
|
||||
|
||||
// ConnectBlock is invoked when the index manager is notified that a new
|
||||
// block has been connected to the main chain.
|
||||
ConnectBlock(dbTx database.Tx, block *btcutil.Block, view *blockchain.UtxoViewpoint) error
|
||||
// ConnectBlock is invoked when a new block has been connected to the
|
||||
// main chain. The set of output spent within a block is also passed in
|
||||
// so indexers can access the pevious output scripts input spent if
|
||||
// required.
|
||||
ConnectBlock(database.Tx, *btcutil.Block, []blockchain.SpentTxOut) error
|
||||
|
||||
// DisconnectBlock is invoked when the index manager is notified that a
|
||||
// block has been disconnected from the main chain.
|
||||
DisconnectBlock(dbTx database.Tx, block *btcutil.Block, view *blockchain.UtxoViewpoint) error
|
||||
// DisconnectBlock is invoked when a block has been disconnected from
|
||||
// the main chain. The set of outputs scripts that were spent within
|
||||
// this block is also returned so indexers can clean up the prior index
|
||||
// state for this block
|
||||
DisconnectBlock(database.Tx, *btcutil.Block, []blockchain.SpentTxOut) error
|
||||
}
|
||||
|
||||
// AssertError identifies an error that indicates an internal code consistency
|
||||
|
|
|
@ -68,7 +68,9 @@ func dbFetchIndexerTip(dbTx database.Tx, idxKey []byte) (*chainhash.Hash, int32,
|
|||
// given block using the provided indexer and updates the tip of the indexer
|
||||
// accordingly. An error will be returned if the current tip for the indexer is
|
||||
// not the previous block for the passed block.
|
||||
func dbIndexConnectBlock(dbTx database.Tx, indexer Indexer, block *btcutil.Block, view *blockchain.UtxoViewpoint) error {
|
||||
func dbIndexConnectBlock(dbTx database.Tx, indexer Indexer, block *btcutil.Block,
|
||||
stxo []blockchain.SpentTxOut) error {
|
||||
|
||||
// Assert that the block being connected properly connects to the
|
||||
// current tip of the index.
|
||||
idxKey := indexer.Key()
|
||||
|
@ -84,7 +86,7 @@ func dbIndexConnectBlock(dbTx database.Tx, indexer Indexer, block *btcutil.Block
|
|||
}
|
||||
|
||||
// Notify the indexer with the connected block so it can index it.
|
||||
if err := indexer.ConnectBlock(dbTx, block, view); err != nil {
|
||||
if err := indexer.ConnectBlock(dbTx, block, stxo); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
|
@ -96,7 +98,9 @@ func dbIndexConnectBlock(dbTx database.Tx, indexer Indexer, block *btcutil.Block
|
|||
// given block using the provided indexer and updates the tip of the indexer
|
||||
// accordingly. An error will be returned if the current tip for the indexer is
|
||||
// not the passed block.
|
||||
func dbIndexDisconnectBlock(dbTx database.Tx, indexer Indexer, block *btcutil.Block, view *blockchain.UtxoViewpoint) error {
|
||||
func dbIndexDisconnectBlock(dbTx database.Tx, indexer Indexer, block *btcutil.Block,
|
||||
stxo []blockchain.SpentTxOut) error {
|
||||
|
||||
// Assert that the block being disconnected is the current tip of the
|
||||
// index.
|
||||
idxKey := indexer.Key()
|
||||
|
@ -113,7 +117,7 @@ func dbIndexDisconnectBlock(dbTx database.Tx, indexer Indexer, block *btcutil.Bl
|
|||
|
||||
// Notify the indexer with the disconnected block so it can remove all
|
||||
// of the appropriate entries.
|
||||
if err := indexer.DisconnectBlock(dbTx, block, view); err != nil {
|
||||
if err := indexer.DisconnectBlock(dbTx, block, stxo); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
|
@ -299,7 +303,8 @@ func (m *Manager) Init(chain *blockchain.BlockChain, interrupt <-chan struct{})
|
|||
// loaded directly since it is no longer in the main
|
||||
// chain and thus the chain.BlockByHash function would
|
||||
// error.
|
||||
err = m.db.Update(func(dbTx database.Tx) error {
|
||||
var block *btcutil.Block
|
||||
err := m.db.View(func(dbTx database.Tx) error {
|
||||
blockBytes, err := dbTx.FetchBlock(hash)
|
||||
if err != nil {
|
||||
return err
|
||||
|
@ -309,24 +314,27 @@ func (m *Manager) Init(chain *blockchain.BlockChain, interrupt <-chan struct{})
|
|||
return err
|
||||
}
|
||||
block.SetHeight(height)
|
||||
|
||||
// When the index requires all of the referenced
|
||||
// txouts they need to be retrieved from the
|
||||
// transaction index.
|
||||
var view *blockchain.UtxoViewpoint
|
||||
if indexNeedsInputs(indexer) {
|
||||
var err error
|
||||
view, err = makeUtxoView(dbTx, block,
|
||||
interrupt)
|
||||
return err
|
||||
})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// We'll also grab the set of outputs spent by this
|
||||
// block so we can remove them from the index.
|
||||
spentTxos, err := chain.FetchSpendJournal(block)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// With the block and stxo set for that block retrieved,
|
||||
// we can now update the index itself.
|
||||
err = m.db.Update(func(dbTx database.Tx) error {
|
||||
// Remove all of the index entries associated
|
||||
// with the block and update the indexer tip.
|
||||
err = dbIndexDisconnectBlock(dbTx, indexer,
|
||||
block, view)
|
||||
err = dbIndexDisconnectBlock(
|
||||
dbTx, indexer, block, spentTxos,
|
||||
)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -407,7 +415,7 @@ func (m *Manager) Init(chain *blockchain.BlockChain, interrupt <-chan struct{})
|
|||
}
|
||||
|
||||
// Connect the block for all indexes that need it.
|
||||
var view *blockchain.UtxoViewpoint
|
||||
var spentTxos []blockchain.SpentTxOut
|
||||
for i, indexer := range m.enabledIndexes {
|
||||
// Skip indexes that don't need to be updated with this
|
||||
// block.
|
||||
|
@ -415,21 +423,20 @@ func (m *Manager) Init(chain *blockchain.BlockChain, interrupt <-chan struct{})
|
|||
continue
|
||||
}
|
||||
|
||||
err := m.db.Update(func(dbTx database.Tx) error {
|
||||
// When the index requires all of the referenced
|
||||
// txouts and they haven't been loaded yet, they
|
||||
// need to be retrieved from the transaction
|
||||
// index.
|
||||
if view == nil && indexNeedsInputs(indexer) {
|
||||
var err error
|
||||
view, err = makeUtxoView(dbTx, block,
|
||||
interrupt)
|
||||
// When the index requires all of the referenced txouts
|
||||
// and they haven't been loaded yet, they need to be
|
||||
// retrieved from the spend journal.
|
||||
if spentTxos == nil && indexNeedsInputs(indexer) {
|
||||
spentTxos, err = chain.FetchSpendJournal(block)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return dbIndexConnectBlock(dbTx, indexer, block,
|
||||
view)
|
||||
|
||||
err := m.db.Update(func(dbTx database.Tx) error {
|
||||
return dbIndexConnectBlock(
|
||||
dbTx, indexer, block, spentTxos,
|
||||
)
|
||||
})
|
||||
if err != nil {
|
||||
return err
|
||||
|
@ -528,11 +535,13 @@ func makeUtxoView(dbTx database.Tx, block *btcutil.Block, interrupt <-chan struc
|
|||
// checks, and invokes each indexer.
|
||||
//
|
||||
// This is part of the blockchain.IndexManager interface.
|
||||
func (m *Manager) ConnectBlock(dbTx database.Tx, block *btcutil.Block, view *blockchain.UtxoViewpoint) error {
|
||||
func (m *Manager) ConnectBlock(dbTx database.Tx, block *btcutil.Block,
|
||||
stxos []blockchain.SpentTxOut) error {
|
||||
|
||||
// Call each of the currently active optional indexes with the block
|
||||
// being connected so they can update accordingly.
|
||||
for _, index := range m.enabledIndexes {
|
||||
err := dbIndexConnectBlock(dbTx, index, block, view)
|
||||
err := dbIndexConnectBlock(dbTx, index, block, stxos)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -546,11 +555,13 @@ func (m *Manager) ConnectBlock(dbTx database.Tx, block *btcutil.Block, view *blo
|
|||
// the index entries associated with the block.
|
||||
//
|
||||
// This is part of the blockchain.IndexManager interface.
|
||||
func (m *Manager) DisconnectBlock(dbTx database.Tx, block *btcutil.Block, view *blockchain.UtxoViewpoint) error {
|
||||
func (m *Manager) DisconnectBlock(dbTx database.Tx, block *btcutil.Block,
|
||||
stxo []blockchain.SpentTxOut) error {
|
||||
|
||||
// Call each of the currently active optional indexes with the block
|
||||
// being disconnected so they can update accordingly.
|
||||
for _, index := range m.enabledIndexes {
|
||||
err := dbIndexDisconnectBlock(dbTx, index, block, view)
|
||||
err := dbIndexDisconnectBlock(dbTx, index, block, stxo)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
|
|
@ -388,7 +388,9 @@ func (idx *TxIndex) Create(dbTx database.Tx) error {
|
|||
// for every transaction in the passed block.
|
||||
//
|
||||
// This is part of the Indexer interface.
|
||||
func (idx *TxIndex) ConnectBlock(dbTx database.Tx, block *btcutil.Block, view *blockchain.UtxoViewpoint) error {
|
||||
func (idx *TxIndex) ConnectBlock(dbTx database.Tx, block *btcutil.Block,
|
||||
stxos []blockchain.SpentTxOut) error {
|
||||
|
||||
// Increment the internal block ID to use for the block being connected
|
||||
// and add all of the transactions in the block to the index.
|
||||
newBlockID := idx.curBlockID + 1
|
||||
|
@ -411,7 +413,9 @@ func (idx *TxIndex) ConnectBlock(dbTx database.Tx, block *btcutil.Block, view *b
|
|||
// hash-to-transaction mapping for every transaction in the block.
|
||||
//
|
||||
// This is part of the Indexer interface.
|
||||
func (idx *TxIndex) DisconnectBlock(dbTx database.Tx, block *btcutil.Block, view *blockchain.UtxoViewpoint) error {
|
||||
func (idx *TxIndex) DisconnectBlock(dbTx database.Tx, block *btcutil.Block,
|
||||
stxos []blockchain.SpentTxOut) error {
|
||||
|
||||
// Remove all of the transactions in the block from the index.
|
||||
if err := dbRemoveTxIndexEntries(dbTx, block); err != nil {
|
||||
return err
|
||||
|
|
|
@ -216,7 +216,7 @@ func (view *UtxoViewpoint) AddTxOuts(tx *btcutil.Tx, blockHeight int32) {
|
|||
// spent. In addition, when the 'stxos' argument is not nil, it will be updated
|
||||
// to append an entry for each spent txout. An error will be returned if the
|
||||
// view does not contain the required utxos.
|
||||
func (view *UtxoViewpoint) connectTransaction(tx *btcutil.Tx, blockHeight int32, stxos *[]spentTxOut) error {
|
||||
func (view *UtxoViewpoint) connectTransaction(tx *btcutil.Tx, blockHeight int32, stxos *[]SpentTxOut) error {
|
||||
// Coinbase transactions don't have any inputs to spend.
|
||||
if IsCoinBase(tx) {
|
||||
// Add the transaction's outputs as available utxos.
|
||||
|
@ -239,11 +239,11 @@ func (view *UtxoViewpoint) connectTransaction(tx *btcutil.Tx, blockHeight int32,
|
|||
// Only create the stxo details if requested.
|
||||
if stxos != nil {
|
||||
// Populate the stxo details using the utxo entry.
|
||||
var stxo = spentTxOut{
|
||||
amount: entry.Amount(),
|
||||
pkScript: entry.PkScript(),
|
||||
height: entry.BlockHeight(),
|
||||
isCoinBase: entry.IsCoinBase(),
|
||||
var stxo = SpentTxOut{
|
||||
Amount: entry.Amount(),
|
||||
PkScript: entry.PkScript(),
|
||||
Height: entry.BlockHeight(),
|
||||
IsCoinBase: entry.IsCoinBase(),
|
||||
}
|
||||
*stxos = append(*stxos, stxo)
|
||||
}
|
||||
|
@ -264,7 +264,7 @@ func (view *UtxoViewpoint) connectTransaction(tx *btcutil.Tx, blockHeight int32,
|
|||
// spend as spent, and setting the best hash for the view to the passed block.
|
||||
// In addition, when the 'stxos' argument is not nil, it will be updated to
|
||||
// append an entry for each spent txout.
|
||||
func (view *UtxoViewpoint) connectTransactions(block *btcutil.Block, stxos *[]spentTxOut) error {
|
||||
func (view *UtxoViewpoint) connectTransactions(block *btcutil.Block, stxos *[]SpentTxOut) error {
|
||||
for _, tx := range block.Transactions() {
|
||||
err := view.connectTransaction(tx, block.Height(), stxos)
|
||||
if err != nil {
|
||||
|
@ -308,7 +308,7 @@ func (view *UtxoViewpoint) fetchEntryByHash(db database.DB, hash *chainhash.Hash
|
|||
// created by the passed block, restoring all utxos the transactions spent by
|
||||
// using the provided spent txo information, and setting the best hash for the
|
||||
// view to the block before the passed block.
|
||||
func (view *UtxoViewpoint) disconnectTransactions(db database.DB, block *btcutil.Block, stxos []spentTxOut) error {
|
||||
func (view *UtxoViewpoint) disconnectTransactions(db database.DB, block *btcutil.Block, stxos []SpentTxOut) error {
|
||||
// Sanity check the correct number of stxos are provided.
|
||||
if len(stxos) != countSpentOutputs(block) {
|
||||
return AssertError("disconnectTransactions called with bad " +
|
||||
|
@ -405,7 +405,7 @@ func (view *UtxoViewpoint) disconnectTransactions(db database.DB, block *btcutil
|
|||
// connected. In the case of a fresh database that has
|
||||
// only ever run with the new v2 format, this code path
|
||||
// will never run.
|
||||
if stxo.height == 0 {
|
||||
if stxo.Height == 0 {
|
||||
utxo, err := view.fetchEntryByHash(db, txHash)
|
||||
if err != nil {
|
||||
return err
|
||||
|
@ -416,17 +416,17 @@ func (view *UtxoViewpoint) disconnectTransactions(db database.DB, block *btcutil
|
|||
*originOut))
|
||||
}
|
||||
|
||||
stxo.height = utxo.BlockHeight()
|
||||
stxo.isCoinBase = utxo.IsCoinBase()
|
||||
stxo.Height = utxo.BlockHeight()
|
||||
stxo.IsCoinBase = utxo.IsCoinBase()
|
||||
}
|
||||
|
||||
// Restore the utxo using the stxo data from the spend
|
||||
// journal and mark it as modified.
|
||||
entry.amount = stxo.amount
|
||||
entry.pkScript = stxo.pkScript
|
||||
entry.blockHeight = stxo.height
|
||||
entry.amount = stxo.Amount
|
||||
entry.pkScript = stxo.PkScript
|
||||
entry.blockHeight = stxo.Height
|
||||
entry.packedFlags = tfModified
|
||||
if stxo.isCoinBase {
|
||||
if stxo.IsCoinBase {
|
||||
entry.packedFlags |= tfCoinBase
|
||||
}
|
||||
}
|
||||
|
|
|
@ -986,7 +986,7 @@ func CheckTransactionInputs(tx *btcutil.Tx, txHeight int32, utxoView *UtxoViewpo
|
|||
// with that node.
|
||||
//
|
||||
// This function MUST be called with the chain state lock held (for writes).
|
||||
func (b *BlockChain) checkConnectBlock(node *blockNode, block *btcutil.Block, view *UtxoViewpoint, stxos *[]spentTxOut) error {
|
||||
func (b *BlockChain) checkConnectBlock(node *blockNode, block *btcutil.Block, view *UtxoViewpoint, stxos *[]SpentTxOut) error {
|
||||
// If the side chain blocks end up in the database, a call to
|
||||
// CheckBlockSanity should be done here in case a previous version
|
||||
// allowed a block that is no longer valid. However, since the
|
||||
|
|
|
@ -1191,8 +1191,8 @@ func TestChainSvrCmdErrors(t *testing.T) {
|
|||
for i, test := range tests {
|
||||
err := json.Unmarshal([]byte(test.marshalled), &test.result)
|
||||
if reflect.TypeOf(err) != reflect.TypeOf(test.err) {
|
||||
t.Errorf("Test #%d (%s) wrong error - got %T (%[2]v), "+
|
||||
"want %T", i, test.name, err, test.err)
|
||||
t.Errorf("Test #%d (%s) wrong error - got %T (%v), "+
|
||||
"want %T", i, test.name, err, err, test.err)
|
||||
continue
|
||||
}
|
||||
|
||||
|
|
|
@ -387,8 +387,8 @@ func TestNewCmdErrors(t *testing.T) {
|
|||
for i, test := range tests {
|
||||
_, err := btcjson.NewCmd(test.method, test.args...)
|
||||
if reflect.TypeOf(err) != reflect.TypeOf(test.err) {
|
||||
t.Errorf("Test #%d (%s) wrong error - got %T (%[2]v), "+
|
||||
"want %T", i, test.name, err, test.err)
|
||||
t.Errorf("Test #%d (%s) wrong error - got %T (%v), "+
|
||||
"want %T", i, test.name, err, err, test.err)
|
||||
continue
|
||||
}
|
||||
gotErrorCode := err.(btcjson.Error).ErrorCode
|
||||
|
@ -435,8 +435,8 @@ func TestMarshalCmdErrors(t *testing.T) {
|
|||
for i, test := range tests {
|
||||
_, err := btcjson.MarshalCmd(test.id, test.cmd)
|
||||
if reflect.TypeOf(err) != reflect.TypeOf(test.err) {
|
||||
t.Errorf("Test #%d (%s) wrong error - got %T (%[2]v), "+
|
||||
"want %T", i, test.name, err, test.err)
|
||||
t.Errorf("Test #%d (%s) wrong error - got %T (%v), "+
|
||||
"want %T", i, test.name, err, err, test.err)
|
||||
continue
|
||||
}
|
||||
gotErrorCode := err.(btcjson.Error).ErrorCode
|
||||
|
@ -504,8 +504,8 @@ func TestUnmarshalCmdErrors(t *testing.T) {
|
|||
for i, test := range tests {
|
||||
_, err := btcjson.UnmarshalCmd(&test.request)
|
||||
if reflect.TypeOf(err) != reflect.TypeOf(test.err) {
|
||||
t.Errorf("Test #%d (%s) wrong error - got %T (%[2]v), "+
|
||||
"want %T", i, test.name, err, test.err)
|
||||
t.Errorf("Test #%d (%s) wrong error - got %T (%v), "+
|
||||
"want %T", i, test.name, err, err, test.err)
|
||||
continue
|
||||
}
|
||||
gotErrorCode := err.(btcjson.Error).ErrorCode
|
||||
|
|
|
@ -699,8 +699,8 @@ func TestGenerateHelpErrors(t *testing.T) {
|
|||
_, err := btcjson.GenerateHelp(test.method, nil,
|
||||
test.resultTypes...)
|
||||
if reflect.TypeOf(err) != reflect.TypeOf(test.err) {
|
||||
t.Errorf("Test #%d (%s) wrong error - got %T (%[2]v), "+
|
||||
"want %T", i, test.name, err, test.err)
|
||||
t.Errorf("Test #%d (%s) wrong error - got %T (%v), "+
|
||||
"want %T", i, test.name, err, err, test.err)
|
||||
continue
|
||||
}
|
||||
gotErrorCode := err.(btcjson.Error).ErrorCode
|
||||
|
|
Loading…
Reference in a new issue