From b20abce609992c0f20f7266e7dbfdc76e68b37d6 Mon Sep 17 00:00:00 2001 From: Dave Collins Date: Fri, 16 Feb 2018 16:31:34 -0600 Subject: [PATCH 1/3] blockchain: Infrastructure to manage block index. This adds infrastructure necessary for blockchain to independently manage and persist a full block index in a bucket. Note that the new infrastructure is not used yet as it will be integrated in future commits after migrating existing databases has been done. The purpose of this infrastructure is to significantly speed up startup time which currently has to load a bunch of the most recent blocks in order to rebuild the stake state and also to pave the way towards being able to have the full block index in memory allows for a lot of optimizations and greatly simplified code. This also includes a full suite of tests to ensure proper serialization and deserialization functionality. --- blockchain/blockindex.go | 24 +- blockchain/chain.go | 14 - blockchain/chainio.go | 252 ++++++++++++++ blockchain/chainio_test.go | 310 ++++++++++++++++++ .../internal/dbnamespace/dbnamespace.go | 5 + 5 files changed, 583 insertions(+), 22 deletions(-) diff --git a/blockchain/blockindex.go b/blockchain/blockindex.go index 49f4b087fa..a9b643e8be 100644 --- a/blockchain/blockindex.go +++ b/blockchain/blockindex.go @@ -90,14 +90,15 @@ type blockNode struct { votes []stake.VoteVersionTuple } -// newBlockNode returns a new block node for the given block header and parent -// node. The workSum is calculated based on the parent, or, in the case no -// parent is provided, it will just be the work for the passed block. -func newBlockNode(blockHeader *wire.BlockHeader, parent *blockNode) *blockNode { - // Make a copy of the hash so the node doesn't keep a reference to part - // of the full block/block header preventing it from being garbage - // collected. - node := blockNode{ +// initBlockNode initializes a block node from the given header, initialization +// vector for the ticket lottery, and parent node. The workSum is calculated +// based on the parent, or, in the case no parent is provided, it will just be +// the work for the passed block. +// +// This function is NOT safe for concurrent access. It must only be called when +// initially creating a node. +func initBlockNode(node *blockNode, blockHeader *wire.BlockHeader, parent *blockNode) { + *node = blockNode{ hash: blockHeader.BlockHash(), parentHash: blockHeader.PrevBlock, workSum: CalcWork(blockHeader.Bits), @@ -123,7 +124,14 @@ func newBlockNode(blockHeader *wire.BlockHeader, parent *blockNode) *blockNode { node.parent = parent node.workSum = node.workSum.Add(parent.workSum, node.workSum) } +} +// newBlockNode returns a new block node for the given block header and parent +// node. The workSum is calculated based on the parent, or, in the case no +// parent is provided, it will just be the work for the passed block. +func newBlockNode(blockHeader *wire.BlockHeader, parent *blockNode) *blockNode { + var node blockNode + initBlockNode(&node, blockHeader, parent) return &node } diff --git a/blockchain/chain.go b/blockchain/chain.go index 73eb39dd4f..21f264a1af 100644 --- a/blockchain/chain.go +++ b/blockchain/chain.go @@ -799,20 +799,6 @@ func (b *BlockChain) pushMainChainBlockCache(block *dcrutil.Block) { b.mainchainBlockCacheLock.Unlock() } -// dbMaybeStoreBlock stores the provided block in the database if it's not -// already there. -func dbMaybeStoreBlock(dbTx database.Tx, block *dcrutil.Block) error { - hasBlock, err := dbTx.HasBlock(block.Hash()) - if err != nil { - return err - } - if hasBlock { - return nil - } - - return dbTx.StoreBlock(block) -} - // connectBlock handles connecting the passed node/block to the end of the main // (best) chain. // diff --git a/blockchain/chainio.go b/blockchain/chainio.go index 31dfb612d9..5118e4baac 100644 --- a/blockchain/chainio.go +++ b/blockchain/chainio.go @@ -32,6 +32,11 @@ const ( // currentDatabaseVersion indicates what the current database // version is. currentDatabaseVersion = 2 + + // blockHdrSize is the size of a block header. This is simply the + // constant from wire and is only provided here for convenience since + // wire.MaxBlockHeaderPayload is quite long. + blockHdrSize = wire.MaxBlockHeaderPayload ) // errNotInMainChain signifies that a block hash or height that is not in the @@ -167,6 +172,253 @@ func ConvertUtxosToMinimalOutputs(entry *UtxoEntry) []*stake.MinimalOutput { return minOuts } +// ----------------------------------------------------------------------------- +// The block index consists of an entry for every known block. It consists of +// information such as the block header and hashes of tickets voted and revoked. +// +// The serialized key format is: +// +// +// +// Field Type Size +// block height uint32 4 bytes +// block hash chainhash.Hash chainhash.HashSize +// +// The serialized value format is: +// +// +// +// Field Type Size +// block header wire.BlockHeader 180 bytes +// num votes VLQ variable +// vote info +// ticket hash chainhash.Hash chainhash.HashSize +// vote version VLQ variable +// vote bits VLQ variable +// num revoked VLQ variable +// revoked tickets +// ticket hash chainhash.Hash chainhash.HashSize +// ----------------------------------------------------------------------------- + +// blockIndexKey generates the binary key for an entry in the block index +// bucket. The key is composed of the block height encoded as a big-endian +// 32-bit unsigned int followed by the 32 byte block hash. Big endian is used +// here so the entries can easily be iterated by height. +func blockIndexKey(blockHash *chainhash.Hash, blockHeight uint32) []byte { + indexKey := make([]byte, chainhash.HashSize+4) + binary.BigEndian.PutUint32(indexKey[0:4], blockHeight) + copy(indexKey[4:chainhash.HashSize+4], blockHash[:]) + return indexKey +} + +// blockNodeSerializeSize returns the number of bytes it would take to serialize +// the passed block node according to the format described above. +func blockNodeSerializeSize(node *blockNode) int { + voteInfoSize := 0 + for i := range node.votes { + voteInfoSize += chainhash.HashSize + + serializeSizeVLQ(uint64(node.votes[i].Version)) + + serializeSizeVLQ(uint64(node.votes[i].Bits)) + } + + return blockHdrSize + serializeSizeVLQ(uint64(len(node.votes))) + + voteInfoSize + serializeSizeVLQ(uint64(len(node.ticketsRevoked))) + + chainhash.HashSize*len(node.ticketsRevoked) +} + +// putBlockNode serializes the passed block node 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 blockNodeSerializeSize function or it will panic. +func putBlockNode(target []byte, node *blockNode) (int, error) { + if len(node.ticketsVoted) != len(node.votes) { + return 0, AssertError("putBlockNode called with a block node " + + "that has a mismatched number of tickets voted and " + + "votes") + } + + // Serialize the entire block header. + w := bytes.NewBuffer(target[0:0]) + header := node.Header() + if err := header.Serialize(w); err != nil { + return 0, err + } + + // Serialize the number of votes and associated vote information. + offset := blockHdrSize + offset += putVLQ(target[offset:], uint64(len(node.votes))) + for i := range node.votes { + offset += copy(target[offset:], node.ticketsVoted[i][:]) + offset += putVLQ(target[offset:], uint64(node.votes[i].Version)) + offset += putVLQ(target[offset:], uint64(node.votes[i].Bits)) + } + + // Serialize the number of revocations and associated revocation + // information. + offset += putVLQ(target[offset:], uint64(len(node.ticketsRevoked))) + for i := range node.ticketsRevoked { + offset += copy(target[offset:], node.ticketsRevoked[i][:]) + } + + return offset, nil +} + +// serializeBlockNode serializes the passed block node into a single byte slice +// according to the format described in detail above. +func serializeBlockNode(node *blockNode) ([]byte, error) { + serialized := make([]byte, blockNodeSerializeSize(node)) + _, err := putBlockNode(serialized, node) + return serialized, err +} + +// decodeBlockNode decodes the passed serialized block node into the passed +// struct according to the format described above. It returns the number of +// bytes read. +func decodeBlockNode(serialized []byte, node *blockNode) (int, error) { + // Ensure there are enough bytes to decode header. + if len(serialized) < blockHdrSize { + return 0, errDeserialize("unexpected end of data while " + + "reading block header") + } + hB := serialized[0:blockHdrSize] + + // Deserialize the header. + var header wire.BlockHeader + if err := header.Deserialize(bytes.NewReader(hB)); err != nil { + return 0, err + } + offset := blockHdrSize + + // Deserialize the number of tickets spent. + var ticketsVoted []chainhash.Hash + var votes []stake.VoteVersionTuple + numVotes, bytesRead := deserializeVLQ(serialized[offset:]) + if bytesRead == 0 { + return offset, errDeserialize("unexpected end of data while " + + "reading num votes") + } + offset += bytesRead + if numVotes > 0 { + ticketsVoted = make([]chainhash.Hash, numVotes) + votes = make([]stake.VoteVersionTuple, numVotes) + for i := uint64(0); i < numVotes; i++ { + // Deserialize the ticket hash associated with the vote. + if offset+chainhash.HashSize > len(serialized) { + return offset, errDeserialize(fmt.Sprintf("unexpected "+ + "end of data while reading vote #%d hash", + i)) + } + copy(ticketsVoted[i][:], serialized[offset:]) + offset += chainhash.HashSize + + // Deserialize the vote version. + version, bytesRead := deserializeVLQ(serialized[offset:]) + if bytesRead == 0 { + return offset, errDeserialize(fmt.Sprintf("unexpected "+ + "end of data while reading vote #%d version", + i)) + } + offset += bytesRead + + // Deserialize the vote bits. + voteBits, bytesRead := deserializeVLQ(serialized[offset:]) + if bytesRead == 0 { + return offset, errDeserialize(fmt.Sprintf("unexpected "+ + "end of data while reading vote #%d bits", + i)) + } + offset += bytesRead + + votes[i].Version = uint32(version) + votes[i].Bits = uint16(voteBits) + } + } + + // Deserialize the number of tickets revoked. + var ticketsRevoked []chainhash.Hash + numTicketsRevoked, bytesRead := deserializeVLQ(serialized[offset:]) + if bytesRead == 0 { + return offset, errDeserialize("unexpected end of data while " + + "reading num tickets revoked") + } + offset += bytesRead + if numTicketsRevoked > 0 { + ticketsRevoked = make([]chainhash.Hash, numTicketsRevoked) + for i := uint64(0); i < numTicketsRevoked; i++ { + // Deserialize the ticket hash associated with the + // revocation. + if offset+chainhash.HashSize > len(serialized) { + return offset, errDeserialize(fmt.Sprintf("unexpected "+ + "end of data while reading revocation "+ + "#%d", i)) + } + copy(ticketsRevoked[i][:], serialized[offset:]) + offset += chainhash.HashSize + } + } + + initBlockNode(node, &header, nil) + node.populateTicketInfo(&stake.SpentTicketsInBlock{ + VotedTickets: ticketsVoted, + RevokedTickets: ticketsRevoked, + Votes: votes, + }) + + return offset, nil +} + +// deserializeBlockNode decodes the passed serialized byte slice into a block +// node according to the format described above. +func deserializeBlockNode(serialized []byte) (*blockNode, error) { + var node blockNode + if _, err := decodeBlockNode(serialized, &node); err != nil { + return nil, err + } + return &node, nil +} + +// dbPutBlockNode stores the information needed to reconstruct the provided +// block node in the block index according to the format described above. +func dbPutBlockNode(dbTx database.Tx, node *blockNode) error { + serialized, err := serializeBlockNode(node) + if err != nil { + return err + } + + bucket := dbTx.Metadata().Bucket(dbnamespace.BlockIndexBucketName) + key := blockIndexKey(&node.hash, uint32(node.height)) + return bucket.Put(key, serialized) +} + +// dbFetchBlockNode fetches the block node for the passed hash and height from +// the block index. +func dbFetchBlockNode(dbTx database.Tx, hash *chainhash.Hash, height uint32) (*blockNode, error) { + bucket := dbTx.Metadata().Bucket(dbnamespace.BlockIndexBucketName) + key := blockIndexKey(hash, height) + serialized := bucket.Get(key) + if serialized == nil { + return nil, AssertError(fmt.Sprintf("missing block node %s "+ + "(height %d)", hash, height)) + } + + return deserializeBlockNode(serialized) +} + +// dbMaybeStoreBlock stores the provided block in the database if it's not +// already there. +func dbMaybeStoreBlock(dbTx database.Tx, block *dcrutil.Block) error { + // Store the block in ffldb if not already done. + hasBlock, err := dbTx.HasBlock(block.Hash()) + if err != nil { + return err + } + if hasBlock { + return nil + } + + return dbTx.StoreBlock(block) +} + // ----------------------------------------------------------------------------- // The transaction spend journal consists of an entry for each block connected // to the main chain which contains the transaction outputs the block spends diff --git a/blockchain/chainio_test.go b/blockchain/chainio_test.go index 8d0ccb3947..537b47db44 100644 --- a/blockchain/chainio_test.go +++ b/blockchain/chainio_test.go @@ -7,12 +7,14 @@ package blockchain import ( "bytes" + "encoding/hex" "errors" "fmt" "math/big" "reflect" "testing" + "github.com/decred/dcrd/blockchain/stake" "github.com/decred/dcrd/chaincfg/chainhash" "github.com/decred/dcrd/database" "github.com/decred/dcrd/wire" @@ -27,6 +29,42 @@ func newHashFromStr(hexStr string) *chainhash.Hash { return hash } +// hexToFinalState converts the passed hex string into an array of 6 bytes and +// will panic if there is an error. This is only provided for the hard-coded +// constants so errors in the source code can be detected. It will only (and +// must only) be called with hard-coded values. +func hexToFinalState(s string) [6]byte { + b, err := hex.DecodeString(s) + if err != nil { + panic("invalid hex in source file: " + s) + } + + var finalState [6]byte + if len(b) != len(finalState) { + panic("invalid hex in source file: " + s) + } + copy(finalState[:], b) + return finalState +} + +// hexToExtraData converts the passed hex string into an array of 32 bytes and +// will panic if there is an error. This is only provided for the hard-coded +// constants so errors in the source code can be detected. It will only (and +// must only) be called with hard-coded values. +func hexToExtraData(s string) [32]byte { + b, err := hex.DecodeString(s) + if err != nil { + panic("invalid hex in source file: " + s) + } + + var extraData [32]byte + if len(b) != len(extraData) { + panic("invalid hex in source file: " + s) + } + copy(extraData[:], b) + return extraData +} + // DoStxoTest does a test on a simulated blockchain to ensure that the data // stored in the STXO buckets is not corrupt. func (b *BlockChain) DoStxoTest() error { @@ -82,6 +120,278 @@ func TestErrNotInMainChain(t *testing.T) { } } +// TestBlockNodeSerialization ensures serializing and deserializing block nodes +// works as expected. +func TestBlockNodeSerialization(t *testing.T) { + t.Parallel() + + // baseNode is based on block 150287 on mainnet and serves as a template + // for the various tests below. + baseNode := blockNode{ + hash: *newHashFromStr("000000000000005d9820c21dc15d859cc18d660171aaf74fe835d736eaaffc26"), + blockVersion: 4, + parentHash: *newHashFromStr("000000000000016916671ae225343a5ee131c999d5cadb6348805db25737731f"), + merkleRoot: *newHashFromStr("5ef2bb79795d7503c0ccc5cb6e0d4731992fc8c8c5b332c1c0e2c687d864c666"), + stakeRoot: *newHashFromStr("022965059b7527dc2bc18daaa533f806eda1f96fd0b04bbda2381f5552d7c2de"), + voteBits: 0x0001, + finalState: hexToFinalState("313e16e64c0b"), + voters: 4, + freshStake: 3, + revocations: 2, + poolSize: 41332, + bits: 0x1a016f98, + sbits: 7473162478, + height: 150287, + blockSize: 11295, + timestamp: 1499907127, + nonce: 4116576260, + extraData: hexToExtraData("8f01ed92645e0a6b11ee3b3c0000000000000000000000000000000000000000"), + stakeVersion: 4, + ticketsVoted: []chainhash.Hash{ + *newHashFromStr("8b62a877544753ea80a822142a48ec066170e9381d21a9e8a84bc7373f0f9b2e"), + *newHashFromStr("4427a003a7aceb1404ffd9072e9aff1e128a24333a543332030e91668a389db7"), + *newHashFromStr("4415b88ac74881d7b6b15d41df465257cd1cc92d55e95f1b648434aef3a2110b"), + *newHashFromStr("9d2621b57352088809d3a069b04b76c832f30a76da14e56aece72208b3e5b87a"), + }, + ticketsRevoked: []chainhash.Hash{ + *newHashFromStr("8146f01b8ffca8008ebc80293d2978d63b1dffa5c456a73e7b39a9b1e695e8eb"), + *newHashFromStr("2292ff2461e725c58cc6e2051eac2a10e6ee6d1f62327ed676b7a196fb94be0c"), + }, + votes: []stake.VoteVersionTuple{ + {Version: 4, Bits: 0x0001}, + {Version: 4, Bits: 0x0015}, + {Version: 4, Bits: 0x0015}, + {Version: 4, Bits: 0x0001}, + }, + } + baseNode.workSum = CalcWork(baseNode.bits) + + tests := []struct { + name string + node blockNode + serialized []byte + }{ + { + name: "no votes, no revokes", + node: func() blockNode { + node := baseNode + node.ticketsVoted = nil + node.votes = nil + node.ticketsRevoked = nil + return node + }(), + serialized: hexToBytes("040000001f733757b25d804863dbcad599c931e15e3a3425e21a6716690100000000000066c664d887c6e2c0c132b3c5c8c82f9931470d6ecbc5ccc003755d7979bbf25edec2d752551f38a2bd4bb0d06ff9a1ed06f833a5aa8dc12bdc27759b056529020100313e16e64c0b0400030274a10000986f011aee686fbd010000000f4b02001f2c000037c4665904f85df58f01ed92645e0a6b11ee3b3c0000000000000000000000000000000000000000040000000000"), + }, + { + name: "1 vote, no revokes", + node: func() blockNode { + node := baseNode + node.ticketsVoted = node.ticketsVoted[:1] + node.votes = node.votes[:1] + node.ticketsRevoked = nil + return node + }(), + serialized: hexToBytes("040000001f733757b25d804863dbcad599c931e15e3a3425e21a6716690100000000000066c664d887c6e2c0c132b3c5c8c82f9931470d6ecbc5ccc003755d7979bbf25edec2d752551f38a2bd4bb0d06ff9a1ed06f833a5aa8dc12bdc27759b056529020100313e16e64c0b0400030274a10000986f011aee686fbd010000000f4b02001f2c000037c4665904f85df58f01ed92645e0a6b11ee3b3c000000000000000000000000000000000000000004000000012e9b0f3f37c74ba8e8a9211d38e9706106ec482a1422a880ea53475477a8628b040100"), + }, + { + name: "no votes, 1 revoke", + node: func() blockNode { + node := baseNode + node.ticketsVoted = nil + node.ticketsRevoked = node.ticketsRevoked[:1] + node.votes = nil + return node + }(), + serialized: hexToBytes("040000001f733757b25d804863dbcad599c931e15e3a3425e21a6716690100000000000066c664d887c6e2c0c132b3c5c8c82f9931470d6ecbc5ccc003755d7979bbf25edec2d752551f38a2bd4bb0d06ff9a1ed06f833a5aa8dc12bdc27759b056529020100313e16e64c0b0400030274a10000986f011aee686fbd010000000f4b02001f2c000037c4665904f85df58f01ed92645e0a6b11ee3b3c0000000000000000000000000000000000000000040000000001ebe895e6b1a9397b3ea756c4a5ff1d3bd678293d2980bc8e00a8fc8f1bf04681"), + }, + { + name: "4 votes, same vote versions, different vote bits, 2 revokes", + node: baseNode, + serialized: hexToBytes("040000001f733757b25d804863dbcad599c931e15e3a3425e21a6716690100000000000066c664d887c6e2c0c132b3c5c8c82f9931470d6ecbc5ccc003755d7979bbf25edec2d752551f38a2bd4bb0d06ff9a1ed06f833a5aa8dc12bdc27759b056529020100313e16e64c0b0400030274a10000986f011aee686fbd010000000f4b02001f2c000037c4665904f85df58f01ed92645e0a6b11ee3b3c000000000000000000000000000000000000000004000000042e9b0f3f37c74ba8e8a9211d38e9706106ec482a1422a880ea53475477a8628b0401b79d388a66910e033233543a33248a121eff9a2e07d9ff0414ebaca703a0274404150b11a2f3ae3484641b5fe9552dc91ccd575246df415db1b6d78148c78ab8154404157ab8e5b30822e7ec6ae514da760af332c8764bb069a0d30988085273b521269d040102ebe895e6b1a9397b3ea756c4a5ff1d3bd678293d2980bc8e00a8fc8f1bf046810cbe94fb96a1b776d67e32621f6deee6102aac1e05e2c68cc525e76124ff9222"), + }, + } + + for _, test := range tests { + // Ensure the function to calculate the serialized size without + // actually serializing it is calculated properly. + gotSize := blockNodeSerializeSize(&test.node) + if gotSize != len(test.serialized) { + t.Errorf("blockNodeSerializeSize (%s): did not get "+ + "expected size - got %d, want %d", test.name, + gotSize, len(test.serialized)) + } + + // Ensure the block node serializes to the expected value. + gotSerialized, err := serializeBlockNode(&test.node) + if err != nil { + t.Errorf("serializeBlockNode (%s): unexpected error: %v", + test.name, err) + continue + } + if !bytes.Equal(gotSerialized, test.serialized) { + t.Errorf("serializeBlockNode (%s): did not get expected "+ + "bytes - got %x, want %x", test.name, + gotSerialized, test.serialized) + continue + } + + // Ensure the block node serializes to the expected value and + // produces the expected number of bytes written via a direct + // put. + gotSerialized2 := make([]byte, gotSize) + gotBytesWritten, err := putBlockNode(gotSerialized2, &test.node) + if err != nil { + t.Errorf("putBlockNode (%s): unexpected error: %v", + test.name, err) + continue + } + if !bytes.Equal(gotSerialized2, test.serialized) { + t.Errorf("putBlockNode (%s): did not get expected "+ + "bytes - got %x, want %x", test.name, + gotSerialized2, test.serialized) + continue + } + if gotBytesWritten != len(test.serialized) { + t.Errorf("putBlockNode (%s): did not get expected "+ + "number of bytes written - got %d, want %d", + test.name, gotBytesWritten, + len(test.serialized)) + continue + } + + // Ensure the serialized bytes are decoded back to the expected + // block node. + gotNode, err := deserializeBlockNode(test.serialized) + if err != nil { + t.Errorf("deserializeBlockNode (%s): unexpected error: %v", + test.name, err) + continue + } + if !reflect.DeepEqual(*gotNode, test.node) { + t.Errorf("deserializeBlockNode (%s): mismatched entries\ngot "+ + "%+v\nwant %+v", test.name, gotNode, test.node) + continue + } + + // Ensure the serialized bytes are decoded back to the expected + // block node. + var gotNode2 blockNode + bytesRead, err := decodeBlockNode(test.serialized, &gotNode2) + if err != nil { + t.Errorf("decodeBlockNode (%s): unexpected error: %v", + test.name, err) + continue + } + if !reflect.DeepEqual(gotNode2, test.node) { + t.Errorf("decodeBlockNode (%s): mismatched entries\ngot "+ + "%+v\nwant %+v", test.name, gotNode2, test.node) + continue + } + if bytesRead != len(test.serialized) { + t.Errorf("decodeBlockNode (%s): did not get expected "+ + "number of bytes read - got %d, want %d", + test.name, bytesRead, len(test.serialized)) + continue + } + } +} + +// TestBlockNodeDecodeErrors performs negative tests against decoding block +// nodes to ensure error paths work as expected. +func TestBlockNodeDecodeErrors(t *testing.T) { + t.Parallel() + tests := []struct { + name string + node blockNode + serialized []byte + bytesRead int // Expected number of bytes read. + errType error + }{ + { + name: "nothing serialized", + node: blockNode{}, + serialized: hexToBytes(""), + errType: errDeserialize(""), + bytesRead: 0, + }, + { + name: "no data after block header", + node: blockNode{}, + serialized: hexToBytes("040000001f733757b25d804863dbcad599c931e15e3a3425e21a6716690100000000000066c664d887c6e2c0c132b3c5c8c82f9931470d6ecbc5ccc003755d7979bbf25edec2d752551f38a2bd4bb0d06ff9a1ed06f833a5aa8dc12bdc27759b056529020100313e16e64c0b0400030274a10000986f011aee686fbd010000000f4b02001f2c000037c4665904f85df58f01ed92645e0a6b11ee3b3c000000000000000000000000000000000000000004000000"), + errType: errDeserialize(""), + bytesRead: 180, + }, + { + name: "no data after num votes with no votes", + node: blockNode{}, + serialized: hexToBytes("040000001f733757b25d804863dbcad599c931e15e3a3425e21a6716690100000000000066c664d887c6e2c0c132b3c5c8c82f9931470d6ecbc5ccc003755d7979bbf25edec2d752551f38a2bd4bb0d06ff9a1ed06f833a5aa8dc12bdc27759b056529020100313e16e64c0b0400030274a10000986f011aee686fbd010000000f4b02001f2c000037c4665904f85df58f01ed92645e0a6b11ee3b3c00000000000000000000000000000000000000000400000000"), + errType: errDeserialize(""), + bytesRead: 181, + }, + { + name: "no data after num votes with votes", + node: blockNode{}, + serialized: hexToBytes("040000001f733757b25d804863dbcad599c931e15e3a3425e21a6716690100000000000066c664d887c6e2c0c132b3c5c8c82f9931470d6ecbc5ccc003755d7979bbf25edec2d752551f38a2bd4bb0d06ff9a1ed06f833a5aa8dc12bdc27759b056529020100313e16e64c0b0400030274a10000986f011aee686fbd010000000f4b02001f2c000037c4665904f85df58f01ed92645e0a6b11ee3b3c00000000000000000000000000000000000000000400000001"), + errType: errDeserialize(""), + bytesRead: 181, + }, + { + name: "short data in vote ticket hash", + node: blockNode{}, + serialized: hexToBytes("040000001f733757b25d804863dbcad599c931e15e3a3425e21a6716690100000000000066c664d887c6e2c0c132b3c5c8c82f9931470d6ecbc5ccc003755d7979bbf25edec2d752551f38a2bd4bb0d06ff9a1ed06f833a5aa8dc12bdc27759b056529020100313e16e64c0b0400030274a10000986f011aee686fbd010000000f4b02001f2c000037c4665904f85df58f01ed92645e0a6b11ee3b3c000000000000000000000000000000000000000004000000012e9b0f3f37c74ba8e8a9211d38e9706106ec482a1422a880ea53475477a862"), + errType: errDeserialize(""), + bytesRead: 181, + }, + { + name: "no data after vote ticket hash", + node: blockNode{}, + serialized: hexToBytes("040000001f733757b25d804863dbcad599c931e15e3a3425e21a6716690100000000000066c664d887c6e2c0c132b3c5c8c82f9931470d6ecbc5ccc003755d7979bbf25edec2d752551f38a2bd4bb0d06ff9a1ed06f833a5aa8dc12bdc27759b056529020100313e16e64c0b0400030274a10000986f011aee686fbd010000000f4b02001f2c000037c4665904f85df58f01ed92645e0a6b11ee3b3c000000000000000000000000000000000000000004000000012e9b0f3f37c74ba8e8a9211d38e9706106ec482a1422a880ea53475477a8628b"), + errType: errDeserialize(""), + bytesRead: 213, + }, + { + name: "no data after vote version", + node: blockNode{}, + serialized: hexToBytes("040000001f733757b25d804863dbcad599c931e15e3a3425e21a6716690100000000000066c664d887c6e2c0c132b3c5c8c82f9931470d6ecbc5ccc003755d7979bbf25edec2d752551f38a2bd4bb0d06ff9a1ed06f833a5aa8dc12bdc27759b056529020100313e16e64c0b0400030274a10000986f011aee686fbd010000000f4b02001f2c000037c4665904f85df58f01ed92645e0a6b11ee3b3c000000000000000000000000000000000000000004000000012e9b0f3f37c74ba8e8a9211d38e9706106ec482a1422a880ea53475477a8628b04"), + errType: errDeserialize(""), + bytesRead: 214, + }, + { + name: "no data after votes", + node: blockNode{}, + serialized: hexToBytes("040000001f733757b25d804863dbcad599c931e15e3a3425e21a6716690100000000000066c664d887c6e2c0c132b3c5c8c82f9931470d6ecbc5ccc003755d7979bbf25edec2d752551f38a2bd4bb0d06ff9a1ed06f833a5aa8dc12bdc27759b056529020100313e16e64c0b0400030274a10000986f011aee686fbd010000000f4b02001f2c000037c4665904f85df58f01ed92645e0a6b11ee3b3c000000000000000000000000000000000000000004000000012e9b0f3f37c74ba8e8a9211d38e9706106ec482a1422a880ea53475477a8628b0401"), + errType: errDeserialize(""), + bytesRead: 215, + }, + { + name: "no data after num revokes with revokes", + node: blockNode{}, + serialized: hexToBytes("040000001f733757b25d804863dbcad599c931e15e3a3425e21a6716690100000000000066c664d887c6e2c0c132b3c5c8c82f9931470d6ecbc5ccc003755d7979bbf25edec2d752551f38a2bd4bb0d06ff9a1ed06f833a5aa8dc12bdc27759b056529020100313e16e64c0b0400030274a10000986f011aee686fbd010000000f4b02001f2c000037c4665904f85df58f01ed92645e0a6b11ee3b3c0000000000000000000000000000000000000000040000000001"), + errType: errDeserialize(""), + bytesRead: 182, + }, + } + + for _, test := range tests { + // Ensure the expected error type is returned. + gotBytesRead, err := decodeBlockNode(test.serialized, &test.node) + if reflect.TypeOf(err) != reflect.TypeOf(test.errType) { + t.Errorf("decodeBlockNode (%s): expected error type "+ + "does not match - got %T, want %T", test.name, + err, test.errType) + continue + } + + // Ensure the expected number of bytes read is returned. + if gotBytesRead != test.bytesRead { + t.Errorf("decodeBlockNode (%s): unexpected number of "+ + "bytes read - got %d, want %d", test.name, + gotBytesRead, test.bytesRead) + continue + } + } +} + // TestStxoSerialization ensures serializing and deserializing spent transaction // output entries works as expected. func TestStxoSerialization(t *testing.T) { diff --git a/blockchain/internal/dbnamespace/dbnamespace.go b/blockchain/internal/dbnamespace/dbnamespace.go index 1307fba3af..de17960af5 100644 --- a/blockchain/internal/dbnamespace/dbnamespace.go +++ b/blockchain/internal/dbnamespace/dbnamespace.go @@ -50,4 +50,9 @@ var ( // UtxoSetBucketName is the name of the db bucket used to house the // unspent transaction output set. UtxoSetBucketName = []byte("utxoset") + + // BlockIndexBucketName is the name of the db bucket used to house the + // block index which consists of metadata for all known blocks both in + // the main chain and on side chains. + BlockIndexBucketName = []byte("blockidx") ) From 132eb4b15055d7375ca3aa994a0dfdd6a84e31ec Mon Sep 17 00:00:00 2001 From: Dave Collins Date: Wed, 21 Feb 2018 19:33:19 -0600 Subject: [PATCH 2/3] blockchain: Add block validation status to index. This adds a new field to the block node struct named status that consists of bit flags for keeping tracking of the validation state of each block. The status is only stored in memory as of this commit, but it will be stored as a part of the block index in the upcoming block index migration code. Since the field will be updated after node creation, this also introduces some new functions for interacting with the status field in a concurrent-safe fashion. This is largely based on upstream commits 2492af0 and fb0d13c; however, it does not include some of the logic in the reorganization paths those commits include as that approach doesn't match the intended direction this package is moving towards. --- blockchain/accept.go | 10 +++++ blockchain/blockindex.go | 82 ++++++++++++++++++++++++++++++++++++++ blockchain/chain.go | 14 +++++++ blockchain/chainio.go | 22 ++++++++-- blockchain/chainio_test.go | 44 +++++++++++--------- blockchain/error.go | 5 +++ blockchain/error_test.go | 1 + 7 files changed, 157 insertions(+), 21 deletions(-) diff --git a/blockchain/accept.go b/blockchain/accept.go index 772c0d882b..0327f496cc 100644 --- a/blockchain/accept.go +++ b/blockchain/accept.go @@ -126,6 +126,15 @@ func (b *BlockChain) maybeAcceptBlock(block *dcrutil.Block, flags BehaviorFlags) return false, ruleError(ErrMissingParent, str) } + // There is no need to validate the block if an ancestor is already + // known to be invalid. + if b.index.NodeStatus(prevNode).KnownInvalid() { + prevHash := &block.MsgBlock().Header.PrevBlock + str := fmt.Sprintf("previous block %s is known to be invalid", + prevHash) + return false, ruleError(ErrInvalidAncestorBlock, str) + } + // The block must pass all of the validation rules which depend on the // position of the block within the block chain. err = b.checkBlockContext(block, prevNode, flags) @@ -160,6 +169,7 @@ func (b *BlockChain) maybeAcceptBlock(block *dcrutil.Block, flags BehaviorFlags) blockHeader := &block.MsgBlock().Header newNode := newBlockNode(blockHeader, prevNode) newNode.populateTicketInfo(stake.FindSpentTicketsInBlock(block.MsgBlock())) + newNode.status = statusDataStored b.index.AddNode(newNode) // Remove the node from the block index and disconnect it from the diff --git a/blockchain/blockindex.go b/blockchain/blockindex.go index a9b643e8be..632b746574 100644 --- a/blockchain/blockindex.go +++ b/blockchain/blockindex.go @@ -21,6 +21,50 @@ import ( "github.com/decred/dcrd/wire" ) +// blockStatus is a bit field representing the validation state of the block. +type blockStatus byte + +// The following constants specify possible status bit flags for a block. +// +// NOTE: This section specifically does not use iota since the block status is +// serialized and must be stable for long-term storage. +const ( + // statusNone indicates that the block has no validation state flags set. + statusNone blockStatus = 0 + + // statusDataStored indicates that the block's payload is stored on disk. + statusDataStored blockStatus = 1 << 0 + + // statusValid indicates that the block has been fully validated. + statusValid blockStatus = 1 << 1 + + // statusValidateFailed indicates that the block has failed validation. + statusValidateFailed blockStatus = 1 << 2 + + // statusInvalidAncestor indicates that one of the ancestors of the block + // has failed validation, thus the block is also invalid. + statusInvalidAncestor = 1 << 3 +) + +// HaveData returns whether the full block data is stored in the database. This +// will return false for a block node where only the header is downloaded or +// stored. +func (status blockStatus) HaveData() bool { + return status&statusDataStored != 0 +} + +// KnownValid returns whether the block is known to be valid. This will return +// false for a valid block that has not been fully validated yet. +func (status blockStatus) KnownValid() bool { + return status&statusValid != 0 +} + +// KnownInvalid returns whether the block is known to be invalid. This will +// return false for invalid blocks that have not been proven invalid yet. +func (status blockStatus) KnownInvalid() bool { + return status&(statusValidateFailed|statusInvalidAncestor) != 0 +} + // blockNode represents a block within the block chain and is primarily used to // aid in selecting the best chain to be the main chain. The main chain is // stored into the block database. @@ -75,6 +119,13 @@ type blockNode struct { extraData [32]byte stakeVersion uint32 + // status is a bitfield representing the validation state of the block. + // This field, unlike the other fields, may be changed after the block + // node is created, so it must only be accessed or updated using the + // concurrent-safe NodeStatus, SetStatusFlags, and UnsetStatusFlags + // methods on blockIndex once the node has been added to the index. + status blockStatus + // stakeNode contains all the consensus information required for the // staking system. The node also caches information required to add or // remove stake nodes, so that the stake node itself may be pruneable @@ -273,6 +324,7 @@ func (bi *blockIndex) loadBlockNode(dbTx database.Tx, hash *chainhash.Hash) (*bl blockHeader := block.MsgBlock().Header node := newBlockNode(&blockHeader, nil) node.populateTicketInfo(stake.FindSpentTicketsInBlock(block.MsgBlock())) + node.status = statusDataStored | statusValid node.inMainChain = true // Add the node to the chain. @@ -482,6 +534,36 @@ func (bi *blockIndex) LookupNode(hash *chainhash.Hash) *blockNode { return node } +// NodeStatus returns the status associated with the provided node. +// +// This function is safe for concurrent access. +func (bi *blockIndex) NodeStatus(node *blockNode) blockStatus { + bi.RLock() + status := node.status + bi.RUnlock() + return status +} + +// SetStatusFlags sets the provided status flags for the given block node +// regardless of their previous state. It does not unset any flags. +// +// This function is safe for concurrent access. +func (bi *blockIndex) SetStatusFlags(node *blockNode, flags blockStatus) { + bi.Lock() + node.status |= flags + bi.Unlock() +} + +// UnsetStatusFlags unsets the provided status flags for the given block node +// regardless of their previous state. +// +// This function is safe for concurrent access. +func (bi *blockIndex) UnsetStatusFlags(node *blockNode, flags blockStatus) { + bi.Lock() + node.status &^= flags + bi.Unlock() +} + // CalcPastMedianTime calculates the median time of the previous few blocks // prior to, and including, the passed block node. // diff --git a/blockchain/chain.go b/blockchain/chain.go index 21f264a1af..9b7eafbde7 100644 --- a/blockchain/chain.go +++ b/blockchain/chain.go @@ -745,6 +745,12 @@ func (b *BlockChain) getReorganizeNodes(node *blockNode) (*list.List, *list.List return detachNodes, attachNodes, nil } + // Don't allow a reorganize to a descendant of an known invalid block. + if b.index.NodeStatus(node.parent).KnownInvalid() { + b.index.SetStatusFlags(node, statusInvalidAncestor) + return detachNodes, attachNodes, nil + } + // Find the fork point (if any) adding each block to the list of nodes // to attach to the main tree. Push them onto the list in reverse order // so they are attached in the appropriate order when iterating the list @@ -1602,6 +1608,10 @@ func (b *BlockChain) connectBestChain(node *blockNode, block, parent *dcrutil.Bl // We are extending the main (best) chain with a new block. This is the // most common case. if node.parentHash == b.bestNode.hash { + // Skip expensive checks if the block has already been fully + // validated. + fastAdd = fastAdd || b.index.NodeStatus(node).KnownValid() + // Perform several checks to verify the block can be connected // to the main chain without violating any rules and without // actually connecting the block. @@ -1613,8 +1623,12 @@ func (b *BlockChain) connectBestChain(node *blockNode, block, parent *dcrutil.Bl err := b.checkConnectBlock(node, block, parent, view, &stxos) if err != nil { + if _, ok := err.(RuleError); ok { + b.index.SetStatusFlags(node, statusValidateFailed) + } return false, err } + b.index.SetStatusFlags(node, statusValid) } // Don't connect the block if performing a dry run. diff --git a/blockchain/chainio.go b/blockchain/chainio.go index 5118e4baac..95698c4384 100644 --- a/blockchain/chainio.go +++ b/blockchain/chainio.go @@ -186,10 +186,11 @@ func ConvertUtxosToMinimalOutputs(entry *UtxoEntry) []*stake.MinimalOutput { // // The serialized value format is: // -// +// // // Field Type Size // block header wire.BlockHeader 180 bytes +// status blockStatus 1 byte // num votes VLQ variable // vote info // ticket hash chainhash.Hash chainhash.HashSize @@ -221,7 +222,7 @@ func blockNodeSerializeSize(node *blockNode) int { serializeSizeVLQ(uint64(node.votes[i].Bits)) } - return blockHdrSize + serializeSizeVLQ(uint64(len(node.votes))) + + return blockHdrSize + 1 + serializeSizeVLQ(uint64(len(node.votes))) + voteInfoSize + serializeSizeVLQ(uint64(len(node.ticketsRevoked))) + chainhash.HashSize*len(node.ticketsRevoked) } @@ -244,8 +245,12 @@ func putBlockNode(target []byte, node *blockNode) (int, error) { return 0, err } - // Serialize the number of votes and associated vote information. + // Serialize the status. offset := blockHdrSize + target[offset] = byte(node.status) + offset++ + + // Serialize the number of votes and associated vote information. offset += putVLQ(target[offset:], uint64(len(node.votes))) for i := range node.votes { offset += copy(target[offset:], node.ticketsVoted[i][:]) @@ -289,6 +294,14 @@ func decodeBlockNode(serialized []byte, node *blockNode) (int, error) { } offset := blockHdrSize + // Deserialize the status. + if offset+1 > len(serialized) { + return offset, errDeserialize("unexpected end of data while " + + "reading status") + } + status := blockStatus(serialized[offset]) + offset++ + // Deserialize the number of tickets spent. var ticketsVoted []chainhash.Hash var votes []stake.VoteVersionTuple @@ -358,6 +371,7 @@ func decodeBlockNode(serialized []byte, node *blockNode) (int, error) { } initBlockNode(node, &header, nil) + node.status = status node.populateTicketInfo(&stake.SpentTicketsInBlock{ VotedTickets: ticketsVoted, RevokedTickets: ticketsRevoked, @@ -1521,6 +1535,7 @@ func (b *BlockChain) createChainState() error { genesisBlock := dcrutil.NewBlock(b.chainParams.GenesisBlock) header := &genesisBlock.MsgBlock().Header node := newBlockNode(header, nil) + node.status = statusDataStored | statusValid node.inMainChain = true // Initialize the state related to the best block. Since it is the @@ -1730,6 +1745,7 @@ func (b *BlockChain) initChainState(interrupt <-chan struct{}) error { header := &block.Header node := newBlockNode(header, nil) node.populateTicketInfo(stake.FindSpentTicketsInBlock(&block)) + node.status = statusDataStored | statusValid node.inMainChain = true node.workSum = state.workSum diff --git a/blockchain/chainio_test.go b/blockchain/chainio_test.go index 537b47db44..5fccaeb881 100644 --- a/blockchain/chainio_test.go +++ b/blockchain/chainio_test.go @@ -147,6 +147,7 @@ func TestBlockNodeSerialization(t *testing.T) { nonce: 4116576260, extraData: hexToExtraData("8f01ed92645e0a6b11ee3b3c0000000000000000000000000000000000000000"), stakeVersion: 4, + status: statusDataStored | statusValid, ticketsVoted: []chainhash.Hash{ *newHashFromStr("8b62a877544753ea80a822142a48ec066170e9381d21a9e8a84bc7373f0f9b2e"), *newHashFromStr("4427a003a7aceb1404ffd9072e9aff1e128a24333a543332030e91668a389db7"), @@ -180,7 +181,7 @@ func TestBlockNodeSerialization(t *testing.T) { node.ticketsRevoked = nil return node }(), - serialized: hexToBytes("040000001f733757b25d804863dbcad599c931e15e3a3425e21a6716690100000000000066c664d887c6e2c0c132b3c5c8c82f9931470d6ecbc5ccc003755d7979bbf25edec2d752551f38a2bd4bb0d06ff9a1ed06f833a5aa8dc12bdc27759b056529020100313e16e64c0b0400030274a10000986f011aee686fbd010000000f4b02001f2c000037c4665904f85df58f01ed92645e0a6b11ee3b3c0000000000000000000000000000000000000000040000000000"), + serialized: hexToBytes("040000001f733757b25d804863dbcad599c931e15e3a3425e21a6716690100000000000066c664d887c6e2c0c132b3c5c8c82f9931470d6ecbc5ccc003755d7979bbf25edec2d752551f38a2bd4bb0d06ff9a1ed06f833a5aa8dc12bdc27759b056529020100313e16e64c0b0400030274a10000986f011aee686fbd010000000f4b02001f2c000037c4665904f85df58f01ed92645e0a6b11ee3b3c000000000000000000000000000000000000000004000000030000"), }, { name: "1 vote, no revokes", @@ -191,7 +192,7 @@ func TestBlockNodeSerialization(t *testing.T) { node.ticketsRevoked = nil return node }(), - serialized: hexToBytes("040000001f733757b25d804863dbcad599c931e15e3a3425e21a6716690100000000000066c664d887c6e2c0c132b3c5c8c82f9931470d6ecbc5ccc003755d7979bbf25edec2d752551f38a2bd4bb0d06ff9a1ed06f833a5aa8dc12bdc27759b056529020100313e16e64c0b0400030274a10000986f011aee686fbd010000000f4b02001f2c000037c4665904f85df58f01ed92645e0a6b11ee3b3c000000000000000000000000000000000000000004000000012e9b0f3f37c74ba8e8a9211d38e9706106ec482a1422a880ea53475477a8628b040100"), + serialized: hexToBytes("040000001f733757b25d804863dbcad599c931e15e3a3425e21a6716690100000000000066c664d887c6e2c0c132b3c5c8c82f9931470d6ecbc5ccc003755d7979bbf25edec2d752551f38a2bd4bb0d06ff9a1ed06f833a5aa8dc12bdc27759b056529020100313e16e64c0b0400030274a10000986f011aee686fbd010000000f4b02001f2c000037c4665904f85df58f01ed92645e0a6b11ee3b3c00000000000000000000000000000000000000000400000003012e9b0f3f37c74ba8e8a9211d38e9706106ec482a1422a880ea53475477a8628b040100"), }, { name: "no votes, 1 revoke", @@ -202,12 +203,12 @@ func TestBlockNodeSerialization(t *testing.T) { node.votes = nil return node }(), - serialized: hexToBytes("040000001f733757b25d804863dbcad599c931e15e3a3425e21a6716690100000000000066c664d887c6e2c0c132b3c5c8c82f9931470d6ecbc5ccc003755d7979bbf25edec2d752551f38a2bd4bb0d06ff9a1ed06f833a5aa8dc12bdc27759b056529020100313e16e64c0b0400030274a10000986f011aee686fbd010000000f4b02001f2c000037c4665904f85df58f01ed92645e0a6b11ee3b3c0000000000000000000000000000000000000000040000000001ebe895e6b1a9397b3ea756c4a5ff1d3bd678293d2980bc8e00a8fc8f1bf04681"), + serialized: hexToBytes("040000001f733757b25d804863dbcad599c931e15e3a3425e21a6716690100000000000066c664d887c6e2c0c132b3c5c8c82f9931470d6ecbc5ccc003755d7979bbf25edec2d752551f38a2bd4bb0d06ff9a1ed06f833a5aa8dc12bdc27759b056529020100313e16e64c0b0400030274a10000986f011aee686fbd010000000f4b02001f2c000037c4665904f85df58f01ed92645e0a6b11ee3b3c000000000000000000000000000000000000000004000000030001ebe895e6b1a9397b3ea756c4a5ff1d3bd678293d2980bc8e00a8fc8f1bf04681"), }, { name: "4 votes, same vote versions, different vote bits, 2 revokes", node: baseNode, - serialized: hexToBytes("040000001f733757b25d804863dbcad599c931e15e3a3425e21a6716690100000000000066c664d887c6e2c0c132b3c5c8c82f9931470d6ecbc5ccc003755d7979bbf25edec2d752551f38a2bd4bb0d06ff9a1ed06f833a5aa8dc12bdc27759b056529020100313e16e64c0b0400030274a10000986f011aee686fbd010000000f4b02001f2c000037c4665904f85df58f01ed92645e0a6b11ee3b3c000000000000000000000000000000000000000004000000042e9b0f3f37c74ba8e8a9211d38e9706106ec482a1422a880ea53475477a8628b0401b79d388a66910e033233543a33248a121eff9a2e07d9ff0414ebaca703a0274404150b11a2f3ae3484641b5fe9552dc91ccd575246df415db1b6d78148c78ab8154404157ab8e5b30822e7ec6ae514da760af332c8764bb069a0d30988085273b521269d040102ebe895e6b1a9397b3ea756c4a5ff1d3bd678293d2980bc8e00a8fc8f1bf046810cbe94fb96a1b776d67e32621f6deee6102aac1e05e2c68cc525e76124ff9222"), + serialized: hexToBytes("040000001f733757b25d804863dbcad599c931e15e3a3425e21a6716690100000000000066c664d887c6e2c0c132b3c5c8c82f9931470d6ecbc5ccc003755d7979bbf25edec2d752551f38a2bd4bb0d06ff9a1ed06f833a5aa8dc12bdc27759b056529020100313e16e64c0b0400030274a10000986f011aee686fbd010000000f4b02001f2c000037c4665904f85df58f01ed92645e0a6b11ee3b3c00000000000000000000000000000000000000000400000003042e9b0f3f37c74ba8e8a9211d38e9706106ec482a1422a880ea53475477a8628b0401b79d388a66910e033233543a33248a121eff9a2e07d9ff0414ebaca703a0274404150b11a2f3ae3484641b5fe9552dc91ccd575246df415db1b6d78148c78ab8154404157ab8e5b30822e7ec6ae514da760af332c8764bb069a0d30988085273b521269d040102ebe895e6b1a9397b3ea756c4a5ff1d3bd678293d2980bc8e00a8fc8f1bf046810cbe94fb96a1b776d67e32621f6deee6102aac1e05e2c68cc525e76124ff9222"), }, } @@ -322,53 +323,60 @@ func TestBlockNodeDecodeErrors(t *testing.T) { bytesRead: 180, }, { - name: "no data after num votes with no votes", + name: "no data after status", node: blockNode{}, - serialized: hexToBytes("040000001f733757b25d804863dbcad599c931e15e3a3425e21a6716690100000000000066c664d887c6e2c0c132b3c5c8c82f9931470d6ecbc5ccc003755d7979bbf25edec2d752551f38a2bd4bb0d06ff9a1ed06f833a5aa8dc12bdc27759b056529020100313e16e64c0b0400030274a10000986f011aee686fbd010000000f4b02001f2c000037c4665904f85df58f01ed92645e0a6b11ee3b3c00000000000000000000000000000000000000000400000000"), + serialized: hexToBytes("040000001f733757b25d804863dbcad599c931e15e3a3425e21a6716690100000000000066c664d887c6e2c0c132b3c5c8c82f9931470d6ecbc5ccc003755d7979bbf25edec2d752551f38a2bd4bb0d06ff9a1ed06f833a5aa8dc12bdc27759b056529020100313e16e64c0b0400030274a10000986f011aee686fbd010000000f4b02001f2c000037c4665904f85df58f01ed92645e0a6b11ee3b3c00000000000000000000000000000000000000000400000003"), errType: errDeserialize(""), bytesRead: 181, }, + { + name: "no data after num votes with no votes", + node: blockNode{}, + serialized: hexToBytes("040000001f733757b25d804863dbcad599c931e15e3a3425e21a6716690100000000000066c664d887c6e2c0c132b3c5c8c82f9931470d6ecbc5ccc003755d7979bbf25edec2d752551f38a2bd4bb0d06ff9a1ed06f833a5aa8dc12bdc27759b056529020100313e16e64c0b0400030274a10000986f011aee686fbd010000000f4b02001f2c000037c4665904f85df58f01ed92645e0a6b11ee3b3c0000000000000000000000000000000000000000040000000300"), + errType: errDeserialize(""), + bytesRead: 182, + }, { name: "no data after num votes with votes", node: blockNode{}, - serialized: hexToBytes("040000001f733757b25d804863dbcad599c931e15e3a3425e21a6716690100000000000066c664d887c6e2c0c132b3c5c8c82f9931470d6ecbc5ccc003755d7979bbf25edec2d752551f38a2bd4bb0d06ff9a1ed06f833a5aa8dc12bdc27759b056529020100313e16e64c0b0400030274a10000986f011aee686fbd010000000f4b02001f2c000037c4665904f85df58f01ed92645e0a6b11ee3b3c00000000000000000000000000000000000000000400000001"), + serialized: hexToBytes("040000001f733757b25d804863dbcad599c931e15e3a3425e21a6716690100000000000066c664d887c6e2c0c132b3c5c8c82f9931470d6ecbc5ccc003755d7979bbf25edec2d752551f38a2bd4bb0d06ff9a1ed06f833a5aa8dc12bdc27759b056529020100313e16e64c0b0400030274a10000986f011aee686fbd010000000f4b02001f2c000037c4665904f85df58f01ed92645e0a6b11ee3b3c0000000000000000000000000000000000000000040000000301"), errType: errDeserialize(""), - bytesRead: 181, + bytesRead: 182, }, { name: "short data in vote ticket hash", node: blockNode{}, - serialized: hexToBytes("040000001f733757b25d804863dbcad599c931e15e3a3425e21a6716690100000000000066c664d887c6e2c0c132b3c5c8c82f9931470d6ecbc5ccc003755d7979bbf25edec2d752551f38a2bd4bb0d06ff9a1ed06f833a5aa8dc12bdc27759b056529020100313e16e64c0b0400030274a10000986f011aee686fbd010000000f4b02001f2c000037c4665904f85df58f01ed92645e0a6b11ee3b3c000000000000000000000000000000000000000004000000012e9b0f3f37c74ba8e8a9211d38e9706106ec482a1422a880ea53475477a862"), + serialized: hexToBytes("040000001f733757b25d804863dbcad599c931e15e3a3425e21a6716690100000000000066c664d887c6e2c0c132b3c5c8c82f9931470d6ecbc5ccc003755d7979bbf25edec2d752551f38a2bd4bb0d06ff9a1ed06f833a5aa8dc12bdc27759b056529020100313e16e64c0b0400030274a10000986f011aee686fbd010000000f4b02001f2c000037c4665904f85df58f01ed92645e0a6b11ee3b3c00000000000000000000000000000000000000000400000003012e9b0f3f37c74ba8e8a9211d38e9706106ec482a1422a880ea53475477a862"), errType: errDeserialize(""), - bytesRead: 181, + bytesRead: 182, }, { name: "no data after vote ticket hash", node: blockNode{}, - serialized: hexToBytes("040000001f733757b25d804863dbcad599c931e15e3a3425e21a6716690100000000000066c664d887c6e2c0c132b3c5c8c82f9931470d6ecbc5ccc003755d7979bbf25edec2d752551f38a2bd4bb0d06ff9a1ed06f833a5aa8dc12bdc27759b056529020100313e16e64c0b0400030274a10000986f011aee686fbd010000000f4b02001f2c000037c4665904f85df58f01ed92645e0a6b11ee3b3c000000000000000000000000000000000000000004000000012e9b0f3f37c74ba8e8a9211d38e9706106ec482a1422a880ea53475477a8628b"), + serialized: hexToBytes("040000001f733757b25d804863dbcad599c931e15e3a3425e21a6716690100000000000066c664d887c6e2c0c132b3c5c8c82f9931470d6ecbc5ccc003755d7979bbf25edec2d752551f38a2bd4bb0d06ff9a1ed06f833a5aa8dc12bdc27759b056529020100313e16e64c0b0400030274a10000986f011aee686fbd010000000f4b02001f2c000037c4665904f85df58f01ed92645e0a6b11ee3b3c00000000000000000000000000000000000000000400000003012e9b0f3f37c74ba8e8a9211d38e9706106ec482a1422a880ea53475477a8628b"), errType: errDeserialize(""), - bytesRead: 213, + bytesRead: 214, }, { name: "no data after vote version", node: blockNode{}, - serialized: hexToBytes("040000001f733757b25d804863dbcad599c931e15e3a3425e21a6716690100000000000066c664d887c6e2c0c132b3c5c8c82f9931470d6ecbc5ccc003755d7979bbf25edec2d752551f38a2bd4bb0d06ff9a1ed06f833a5aa8dc12bdc27759b056529020100313e16e64c0b0400030274a10000986f011aee686fbd010000000f4b02001f2c000037c4665904f85df58f01ed92645e0a6b11ee3b3c000000000000000000000000000000000000000004000000012e9b0f3f37c74ba8e8a9211d38e9706106ec482a1422a880ea53475477a8628b04"), + serialized: hexToBytes("040000001f733757b25d804863dbcad599c931e15e3a3425e21a6716690100000000000066c664d887c6e2c0c132b3c5c8c82f9931470d6ecbc5ccc003755d7979bbf25edec2d752551f38a2bd4bb0d06ff9a1ed06f833a5aa8dc12bdc27759b056529020100313e16e64c0b0400030274a10000986f011aee686fbd010000000f4b02001f2c000037c4665904f85df58f01ed92645e0a6b11ee3b3c00000000000000000000000000000000000000000400000003012e9b0f3f37c74ba8e8a9211d38e9706106ec482a1422a880ea53475477a8628b04"), errType: errDeserialize(""), - bytesRead: 214, + bytesRead: 215, }, { name: "no data after votes", node: blockNode{}, - serialized: hexToBytes("040000001f733757b25d804863dbcad599c931e15e3a3425e21a6716690100000000000066c664d887c6e2c0c132b3c5c8c82f9931470d6ecbc5ccc003755d7979bbf25edec2d752551f38a2bd4bb0d06ff9a1ed06f833a5aa8dc12bdc27759b056529020100313e16e64c0b0400030274a10000986f011aee686fbd010000000f4b02001f2c000037c4665904f85df58f01ed92645e0a6b11ee3b3c000000000000000000000000000000000000000004000000012e9b0f3f37c74ba8e8a9211d38e9706106ec482a1422a880ea53475477a8628b0401"), + serialized: hexToBytes("040000001f733757b25d804863dbcad599c931e15e3a3425e21a6716690100000000000066c664d887c6e2c0c132b3c5c8c82f9931470d6ecbc5ccc003755d7979bbf25edec2d752551f38a2bd4bb0d06ff9a1ed06f833a5aa8dc12bdc27759b056529020100313e16e64c0b0400030274a10000986f011aee686fbd010000000f4b02001f2c000037c4665904f85df58f01ed92645e0a6b11ee3b3c00000000000000000000000000000000000000000400000003012e9b0f3f37c74ba8e8a9211d38e9706106ec482a1422a880ea53475477a8628b0401"), errType: errDeserialize(""), - bytesRead: 215, + bytesRead: 216, }, { name: "no data after num revokes with revokes", node: blockNode{}, - serialized: hexToBytes("040000001f733757b25d804863dbcad599c931e15e3a3425e21a6716690100000000000066c664d887c6e2c0c132b3c5c8c82f9931470d6ecbc5ccc003755d7979bbf25edec2d752551f38a2bd4bb0d06ff9a1ed06f833a5aa8dc12bdc27759b056529020100313e16e64c0b0400030274a10000986f011aee686fbd010000000f4b02001f2c000037c4665904f85df58f01ed92645e0a6b11ee3b3c0000000000000000000000000000000000000000040000000001"), + serialized: hexToBytes("040000001f733757b25d804863dbcad599c931e15e3a3425e21a6716690100000000000066c664d887c6e2c0c132b3c5c8c82f9931470d6ecbc5ccc003755d7979bbf25edec2d752551f38a2bd4bb0d06ff9a1ed06f833a5aa8dc12bdc27759b056529020100313e16e64c0b0400030274a10000986f011aee686fbd010000000f4b02001f2c000037c4665904f85df58f01ed92645e0a6b11ee3b3c000000000000000000000000000000000000000004000000030001"), errType: errDeserialize(""), - bytesRead: 182, + bytesRead: 183, }, } diff --git a/blockchain/error.go b/blockchain/error.go index 410a2d68d5..8622cc2427 100644 --- a/blockchain/error.go +++ b/blockchain/error.go @@ -449,6 +449,10 @@ const ( // height had a non-zero final state. ErrInvalidEarlyFinalState + // ErrInvalidAncestorBlock indicates that an ancestor of this block has + // failed validation. + ErrInvalidAncestorBlock + // numErrorCodes is the maximum error code number used in tests. numErrorCodes ) @@ -550,6 +554,7 @@ var errorCodeStrings = map[ErrorCode]string{ ErrZeroValueOutputSpend: "ErrZeroValueOutputSpend", ErrInvalidEarlyVoteBits: "ErrInvalidEarlyVoteBits", ErrInvalidEarlyFinalState: "ErrInvalidEarlyFinalState", + ErrInvalidAncestorBlock: "ErrInvalidAncestorBlock", } // String returns the ErrorCode as a human-readable name. diff --git a/blockchain/error_test.go b/blockchain/error_test.go index 1bcd864d8e..dfb59fdb4c 100644 --- a/blockchain/error_test.go +++ b/blockchain/error_test.go @@ -110,6 +110,7 @@ func TestErrorCodeStringer(t *testing.T) { {ErrZeroValueOutputSpend, "ErrZeroValueOutputSpend"}, {ErrInvalidEarlyVoteBits, "ErrInvalidEarlyVoteBits"}, {ErrInvalidEarlyFinalState, "ErrInvalidEarlyFinalState"}, + {ErrInvalidAncestorBlock, "ErrInvalidAncestorBlock"}, {0xffff, "Unknown ErrorCode (65535)"}, } From 7056c678356c4bd5f378736db1d77903ea6de92d Mon Sep 17 00:00:00 2001 From: Dave Collins Date: Fri, 16 Feb 2018 22:46:07 -0600 Subject: [PATCH 3/3] blockchain: Migrate to new block index and use it. This adds code to migrate the existing block index in ffldb to the new format managed by the blockchain package and updates the code to use the new infrastructure. --- blockchain/accept.go | 44 ++-- blockchain/blockindex.go | 17 +- blockchain/chain.go | 17 +- blockchain/chainio.go | 46 +++- .../internal/dbnamespace/dbnamespace.go | 5 + blockchain/upgrade.go | 230 ++++++++++++++++++ 6 files changed, 331 insertions(+), 28 deletions(-) diff --git a/blockchain/accept.go b/blockchain/accept.go index 0327f496cc..a569e3be60 100644 --- a/blockchain/accept.go +++ b/blockchain/accept.go @@ -142,6 +142,21 @@ func (b *BlockChain) maybeAcceptBlock(block *dcrutil.Block, flags BehaviorFlags) return false, err } + // Prune stake nodes which are no longer needed before creating a new + // node. + if !dryRun { + b.pruner.pruneChainIfNeeded() + } + + // Create a new block node for the block and add it to the block index. + // The block could either be on a side chain or the main chain, but it + // starts off as a side chain regardless. + blockHeader := &block.MsgBlock().Header + newNode := newBlockNode(blockHeader, prevNode) + newNode.populateTicketInfo(stake.FindSpentTicketsInBlock(block.MsgBlock())) + newNode.status = statusDataStored + b.index.AddNode(newNode) + // Insert the block into the database if it's not already there. Even // though it is possible the block will ultimately fail to connect, it // has already passed all proof-of-work and validity tests which means @@ -151,27 +166,26 @@ func (b *BlockChain) maybeAcceptBlock(block *dcrutil.Block, flags BehaviorFlags) // expensive connection logic. It also has some other nice properties // such as making blocks that never become part of the main chain or // blocks that fail to connect available for further analysis. + // + // Also, store the associated block index entry when not running in dry + // run mode. err = b.db.Update(func(dbTx database.Tx) error { - return dbMaybeStoreBlock(dbTx, block) + if err := dbMaybeStoreBlock(dbTx, block); err != nil { + return err + } + + if !dryRun { + if err := dbPutBlockNode(dbTx, newNode); err != nil { + return err + } + } + + return nil }) if err != nil { return false, err } - // Prune stake nodes which are no longer needed before creating a new - // node. - if !dryRun { - b.pruner.pruneChainIfNeeded() - } - - // Create a new block node for the block and add it to the in-memory - // block chain (could be either a side chain or the main chain). - blockHeader := &block.MsgBlock().Header - newNode := newBlockNode(blockHeader, prevNode) - newNode.populateTicketInfo(stake.FindSpentTicketsInBlock(block.MsgBlock())) - newNode.status = statusDataStored - b.index.AddNode(newNode) - // Remove the node from the block index and disconnect it from the // parent node when running in dry run mode. if dryRun { diff --git a/blockchain/blockindex.go b/blockchain/blockindex.go index 632b746574..7d9b17f258 100644 --- a/blockchain/blockindex.go +++ b/blockchain/blockindex.go @@ -314,17 +314,18 @@ func (bi *blockIndex) HaveBlock(hash *chainhash.Hash) bool { // This function MUST be called with the block index lock held (for writes). // The database transaction may be read-only. func (bi *blockIndex) loadBlockNode(dbTx database.Tx, hash *chainhash.Hash) (*blockNode, error) { - // Load the block from the db. - block, err := dbFetchBlockByHash(dbTx, hash) + // Try to look up the height for passed block hash in the main chain. + height, err := dbFetchHeightByHash(dbTx, hash) if err != nil { return nil, err } - // Create the new block node for the block. - blockHeader := block.MsgBlock().Header - node := newBlockNode(&blockHeader, nil) - node.populateTicketInfo(stake.FindSpentTicketsInBlock(block.MsgBlock())) - node.status = statusDataStored | statusValid + // Load the block node for the provided hash and height from the + // database. + node, err := dbFetchBlockNode(dbTx, hash, uint32(height)) + if err != nil { + return nil, err + } node.inMainChain = true // Add the node to the chain. @@ -333,7 +334,7 @@ func (bi *blockIndex) loadBlockNode(dbTx database.Tx, hash *chainhash.Hash) (*bl // 2) This node is the parent of one or more nodes // 3) Neither 1 or 2 is true which implies it's an orphan block and // therefore is an error to insert into the chain - prevHash := &blockHeader.PrevBlock + prevHash := &node.parentHash if parentNode, ok := bi.index[*prevHash]; ok { // Case 1 -- This node is a child of an existing block node. // Update the node's work sum with the sum of the parent node's diff --git a/blockchain/chain.go b/blockchain/chain.go index 9b7eafbde7..045b783fcf 100644 --- a/blockchain/chain.go +++ b/blockchain/chain.go @@ -745,7 +745,7 @@ func (b *BlockChain) getReorganizeNodes(node *blockNode) (*list.List, *list.List return detachNodes, attachNodes, nil } - // Don't allow a reorganize to a descendant of an known invalid block. + // Don't allow a reorganize to a descendant of a known invalid block. if b.index.NodeStatus(node.parent).KnownInvalid() { b.index.SetStatusFlags(node, statusInvalidAncestor) return detachNodes, attachNodes, nil @@ -871,6 +871,16 @@ func (b *BlockChain) connectBlock(node *blockNode, block, parent *dcrutil.Block, return err } + // Add the block to the block index. Ultimately the block index + // should track modified nodes and persist all of them prior + // this point as opposed to unconditionally peristing the node + // again. However, this is needed for now in lieu of that to + // ensure the updated status is written to the database. + err = dbPutBlockNode(dbTx, node) + if err != nil { + return err + } + // Add the block hash and height to the main chain index. err = dbPutMainChainIndex(dbTx, block.Hash(), node.height) if err != nil { @@ -1994,8 +2004,9 @@ func New(config *Config) (*BlockChain, error) { b.subsidyCache = NewSubsidyCache(b.bestNode.height, b.chainParams) b.pruner = newChainPruner(&b) - log.Infof("Blockchain database version %v loaded", - b.dbInfo.version) + log.Infof("Blockchain database version info: chain: %d, compression: "+ + "%d, block index: %d", b.dbInfo.version, b.dbInfo.compVer, + b.dbInfo.bidxVer) log.Infof("Chain state: height %d, hash %v, total transactions %d, "+ "work %v, stake version %v", b.bestNode.height, b.bestNode.hash, diff --git a/blockchain/chainio.go b/blockchain/chainio.go index 95698c4384..74fa1a6da3 100644 --- a/blockchain/chainio.go +++ b/blockchain/chainio.go @@ -31,7 +31,11 @@ var ( const ( // currentDatabaseVersion indicates what the current database // version is. - currentDatabaseVersion = 2 + currentDatabaseVersion = 3 + + // currentBlockIndexVersion indicates what the current block index + // database version. + currentBlockIndexVersion = 2 // blockHdrSize is the size of a block header. This is simply the // constant from wire and is only provided here for convenience since @@ -1319,6 +1323,7 @@ func dbFetchHashByHeight(dbTx database.Tx, height int64) (*chainhash.Hash, error // Key Value Size Description // version uint32 4 bytes The version of the database // compver uint32 4 bytes The script compression version of the database +// bidxver uint32 4 bytes The block index version of the database // created uint64 8 bytes The date of the creation of the database // ----------------------------------------------------------------------------- @@ -1326,6 +1331,7 @@ func dbFetchHashByHeight(dbTx database.Tx, height int64) (*chainhash.Hash, error type databaseInfo struct { version uint32 compVer uint32 + bidxVer uint32 created time.Time } @@ -1364,6 +1370,13 @@ func dbPutDatabaseInfo(dbTx database.Tx, dbi *databaseInfo) error { return err } + // Store the block index version. + err = bucket.Put(dbnamespace.BCDBInfoBlockIndexVersionKeyName, + uint32Bytes(dbi.bidxVer)) + if err != nil { + return err + } + // Store the database creation date. return bucket.Put(dbnamespace.BCDBInfoCreatedKeyName, uint64Bytes(uint64(dbi.created.Unix()))) @@ -1394,6 +1407,13 @@ func dbFetchDatabaseInfo(dbTx database.Tx) (*databaseInfo, error) { compVer = dbnamespace.ByteOrder.Uint32(compVerBytes) } + // Load the database block index version. + var bidxVer uint32 + bidxVerBytes := bucket.Get(dbnamespace.BCDBInfoBlockIndexVersionKeyName) + if bidxVerBytes != nil { + bidxVer = dbnamespace.ByteOrder.Uint32(bidxVerBytes) + } + // Load the database creation date. var created time.Time createdBytes := bucket.Get(dbnamespace.BCDBInfoCreatedKeyName) @@ -1405,6 +1425,7 @@ func dbFetchDatabaseInfo(dbTx database.Tx) (*databaseInfo, error) { return &databaseInfo{ version: version, compVer: compVer, + bidxVer: bidxVer, created: created, }, nil } @@ -1560,6 +1581,7 @@ func (b *BlockChain) createChainState() error { b.dbInfo = &databaseInfo{ version: currentDatabaseVersion, compVer: currentCompressionVersion, + bidxVer: currentBlockIndexVersion, created: time.Now(), } err = dbPutDatabaseInfo(dbTx, b.dbInfo) @@ -1567,6 +1589,12 @@ func (b *BlockChain) createChainState() error { return err } + // Create the bucket that houses the block index data. + _, err = meta.CreateBucket(dbnamespace.BlockIndexBucketName) + if err != nil { + return err + } + // Create the bucket that houses the chain block hash to height // index. _, err = meta.CreateBucket(dbnamespace.HashIndexBucketName) @@ -1595,6 +1623,12 @@ func (b *BlockChain) createChainState() error { return err } + // Add the genesis block to the block index. + err = dbPutBlockNode(dbTx, node) + if err != nil { + return err + } + // Add the genesis block hash to height and height to hash // mappings to the index. err = dbPutMainChainIndex(dbTx, &node.hash, node.height) @@ -1691,8 +1725,16 @@ func (b *BlockChain) initChainState(interrupt <-chan struct{}) error { "version of the software (%d > %d)", dbInfo.compVer, currentCompressionVersion) } - b.dbInfo = dbInfo + // Don't allow downgrades of the block index. + if dbInfo.bidxVer > currentBlockIndexVersion { + return fmt.Errorf("the current database block index "+ + "version is no longer compatible with this "+ + "version of the software (%d > %d)", + dbInfo.bidxVer, currentBlockIndexVersion) + } + + b.dbInfo = dbInfo isStateInitialized = true return nil }) diff --git a/blockchain/internal/dbnamespace/dbnamespace.go b/blockchain/internal/dbnamespace/dbnamespace.go index de17960af5..6ade9c58bc 100644 --- a/blockchain/internal/dbnamespace/dbnamespace.go +++ b/blockchain/internal/dbnamespace/dbnamespace.go @@ -26,6 +26,11 @@ var ( // the BCDBInfoBucketName bucket. BCDBInfoCompressionVersionKeyName = []byte("compver") + // BCDBInfoBlockIndexVersionKeyName is the name of the database key + // used to house the database block index version. It is itself under + // the BCDBInfoBucketName bucket. + BCDBInfoBlockIndexVersionKeyName = []byte("bidxver") + // BCDBInfoCreatedKeyName is the name of the database key used to house // date the database was created. It is itself under the // BCDBInfoBucketName bucket. diff --git a/blockchain/upgrade.go b/blockchain/upgrade.go index 38514d2f2a..f226063464 100644 --- a/blockchain/upgrade.go +++ b/blockchain/upgrade.go @@ -6,7 +6,10 @@ package blockchain import ( + "bytes" "encoding/binary" + "errors" + "fmt" "time" "github.com/decred/dcrd/blockchain/internal/progresslog" @@ -15,8 +18,30 @@ import ( "github.com/decred/dcrd/chaincfg/chainhash" "github.com/decred/dcrd/database" "github.com/decred/dcrd/dcrutil" + "github.com/decred/dcrd/wire" ) +// errInterruptRequested indicates that an operation was cancelled due +// to a user-requested interrupt. +var errInterruptRequested = errors.New("interrupt requested") + +// errBatchFinished indicates that a foreach database loop was exited due to +// reaching the maximum batch size. +var errBatchFinished = errors.New("batch finished") + +// interruptRequested returns true when the provided channel has been closed. +// This simplifies early shutdown slightly since the caller can just use an if +// statement instead of a select. +func interruptRequested(interrupted <-chan struct{}) bool { + select { + case <-interrupted: + return true + default: + } + + return false +} + // deserializeDatabaseInfoV2 deserializes a database information struct from the // passed serialized byte slice according to the legacy version 2 format. // @@ -172,6 +197,201 @@ func upgradeToVersion2(db database.DB, chainParams *chaincfg.Params, dbInfo *dat return nil } +// migrateBlockIndex migrates all block entries from the v1 block index bucket +// manged by ffldb to the v2 bucket managed by this package. The v1 bucket +// stored all block entries keyed by block hash, whereas the v2 bucket stores +// them keyed by block height + hash. Also, the old block index only stored the +// header, while the new one stores all info needed to recreate block nodes. +// +// The new block index is guaranteed to be fully updated if this returns without +// failure. +func migrateBlockIndex(db database.DB, interrupt <-chan struct{}) error { + // blkHdrOffset defines the offsets into a v1 block index row for the block + // header. + // + // The serialized block index row format is: + // + const blkHdrOffset = 12 + + // blkHdrHeightStart is the offset of the height in the serialized block + // header bytes as it existed at the time of this migration. It is hard + // coded here so potential future changes do not affect old upgrades. + const blkHdrHeightStart = 128 + + // Hardcoded bucket names so updates to the global values do not affect old + // upgrades. + v1BucketName := []byte("ffldb-blockidx") + v2BucketName := []byte("blockidx") + hashIdxBucketName := []byte("hashidx") + + log.Info("Reindexing block information in the database. This will take " + + "a while...") + start := time.Now() + + // Create the new block index bucket as needed. + err := db.Update(func(dbTx database.Tx) error { + _, err := dbTx.Metadata().CreateBucketIfNotExists(v2BucketName) + return err + }) + if err != nil { + return err + } + + // doBatch contains the primary logic for upgrading the block index from + // version 1 to 2 in batches. This is done because attempting to migrate in + // a single database transaction could result in massive memory usage and + // could potentially crash on many systems due to ulimits. + // + // It returns the number of entries processed. + const maxEntries = 20000 + var resumeOffset uint32 + doBatch := func(dbTx database.Tx) (uint32, error) { + meta := dbTx.Metadata() + v1BlockIdxBucket := meta.Bucket(v1BucketName) + if v1BlockIdxBucket == nil { + return 0, fmt.Errorf("bucket %s does not exist", v1BucketName) + } + + v2BlockIdxBucket := meta.Bucket(v2BucketName) + if v2BlockIdxBucket == nil { + return 0, fmt.Errorf("bucket %s does not exist", v2BucketName) + } + + hashIdxBucket := meta.Bucket(hashIdxBucketName) + if hashIdxBucket == nil { + return 0, fmt.Errorf("bucket %s does not exist", hashIdxBucketName) + } + + // Migrate block index entries so long as the max number of entries for + // this batch has not been exceeded. + var numMigrated, numIterated uint32 + err := v1BlockIdxBucket.ForEach(func(hashBytes, blockRow []byte) error { + if numMigrated >= maxEntries { + return errBatchFinished + } + + // Skip entries that have already been migrated in previous batches. + numIterated++ + if numIterated-1 < resumeOffset { + return nil + } + resumeOffset++ + + // Skip entries that have already been migrated in previous + // interrupted upgrades. + var blockHash chainhash.Hash + copy(blockHash[:], hashBytes) + endOffset := blkHdrOffset + blockHdrSize + headerBytes := blockRow[blkHdrOffset:endOffset:endOffset] + heightBytes := headerBytes[blkHdrHeightStart : blkHdrHeightStart+4] + height := binary.LittleEndian.Uint32(heightBytes) + key := blockIndexKey(&blockHash, height) + if v2BlockIdxBucket.Get(key) != nil { + return nil + } + + // Load the raw full block from the database. + blockBytes, err := dbTx.FetchBlock(&blockHash) + if err != nil { + return err + } + + // Deserialize the block bytes. + var block wire.MsgBlock + err = block.Deserialize(bytes.NewReader(blockBytes)) + if err != nil { + return err + } + + // Construct a block node from the block. + blockNode := newBlockNode(&block.Header, nil) + blockNode.populateTicketInfo(stake.FindSpentTicketsInBlock(&block)) + blockNode.status = statusDataStored + + // Mark the block as valid if it's part of the main chain. While it + // is possible side chain blocks were validated too, there was + // previously no tracking of that information, so there is no way to + // know for sure. It's better to be safe and just assume side chain + // blocks were never validated. + if hashIdxBucket.Get(blockHash[:]) != nil { + blockNode.status |= statusValid + } + + // Write the serialized block node to the new bucket keyed by its + // hash and height. + serialized, err := serializeBlockNode(blockNode) + if err != nil { + return err + } + err = v2BlockIdxBucket.Put(key, serialized) + if err != nil { + return err + } + + numMigrated++ + + if interruptRequested(interrupt) { + return errInterruptRequested + } + + return nil + }) + return numMigrated, err + } + + // Migrate all entries in batches for the reasons mentioned above. + var totalMigrated uint64 + for { + var numMigrated uint32 + err := db.Update(func(dbTx database.Tx) error { + var err error + numMigrated, err = doBatch(dbTx) + if err == errInterruptRequested || err == errBatchFinished { + // No error here so the database transaction is not cancelled + // and therefore outstanding work is written to disk. The + // outer function will exit with an interrupted error below due + // to another interrupted check. + err = nil + } + return err + }) + if err != nil { + return err + } + + if interruptRequested(interrupt) { + return errInterruptRequested + } + + if numMigrated == 0 { + break + } + + totalMigrated += uint64(numMigrated) + log.Infof("Migrated %d entries (%d total)", numMigrated, totalMigrated) + } + + seconds := int64(time.Since(start) / time.Second) + log.Infof("Done upgrading block index. Total entries: %d in %d seconds", + totalMigrated, seconds) + return nil +} + +// upgradeToVersion3 upgrades a version 2 blockchain to version 3 along with +// upgrading the block index to version 2. +func upgradeToVersion3(db database.DB, dbInfo *databaseInfo, interrupt <-chan struct{}) error { + if err := migrateBlockIndex(db, interrupt); err != nil { + return err + } + + // Update and persist the updated database versions. + dbInfo.version = 3 + dbInfo.bidxVer = 2 + return db.Update(func(dbTx database.Tx) error { + return dbPutDatabaseInfo(dbTx, dbInfo) + }) +} + // upgradeDB upgrades old database versions to the newest version by applying // all possible upgrades iteratively. // @@ -182,5 +402,15 @@ func upgradeDB(db database.DB, chainParams *chaincfg.Params, dbInfo *databaseInf return err } } + + // Migrate to the new v2 block index format if needed. That database + // version was bumped because prior versions of the software did not have + // a block index version. + if dbInfo.version == 2 && dbInfo.bidxVer < 2 { + if err := upgradeToVersion3(db, dbInfo, interrupt); err != nil { + return err + } + } + return nil }