[DEV-307] Create the sub-network registry (#147)

* [DEV-307] Implemented adding pending registry transactions.

* [DEV-307] Implemented a skeleton for the sub-network registry algorithm.

* [DEV-307] Implemented the serialization/deserialization logic for pending sub-network transactions.

* [DEV-307] Implemented marking sub-network registry transactions as successfully registered.

* [DEV-307] Implemented clearing pending sub-network entries and writing registered sub-networks.

* [DEV-307] Added comments for all the dagio functions.

* [DEV-307] Fixed a couple of bugs and added a test for checking serialization/deserialization of pending sub-network transactions.

* [DEV-307] Implemented getting a registered sub-network's gas limit.

* [DEV-307] Updated sub-network announcement transaction validation rules.

* [DEV-307] Specified what is validated.

* [DEV-307] Added initialization for lastSubNetworkID.

* [DEV-307] Renamed extractValidSubNetworkRegistryTxs to validateAndExtractSubNetworkRegistryTxs to better reflect what it does.

* Dev 303 implement block finality (#139)

* [DEV-303] Implement block finality

* [DEV-303] Add finality tests

* [DEV-303] Make finality tests use maybeAcceptBlock

* [DEV-303] Only check finality rules if we are fastAdd mode

* [DEV-303] replace hasBiggerFinalityScoreThan checks with .finalityScore()

* [DEV-307] Forgot to actually create the sub-network buckets.

* [DEV-307] Wrote a full-flow test for sub-network registry and fixed a couple of bugs that it had uncovered.

* [DEV-307] Took firstSubNetworkID out of dagconfig.Params. It's unnecessary there.

* [DEV-307] Added persistance for dag.lastSubNetworkID.

* [DEV-307] Moved sub-network stuff to their own files. Got rid of firstSubNetworkID. Replaced SubNetworkReservedLast with SubNetworkUnreservedFirst. Added ErrSubNetworkRegistry.

* [DEV-307] Added a newline at the end of dag_test.go.

* [DEV-307] Renamed previousFinalityPoint to initialFinalityPoint.

* [DEV-307] Moved sub-network IO stuff to subnetworks.go and subnetworks_test.go.

* [DEV-307] Merged dbPutRegisteredSubNetworkTx and dbRegisterSubNetwork. Fixed a too-big allocation and a block double-processing bug.

* [DEV-307] Simplified the serialized representation of a slice of transactions.

* [DEV-307] Fixed comments.
This commit is contained in:
stasatdaglabs 2019-01-07 17:10:23 +02:00 committed by Svarog
parent ade33a94f3
commit d33d633e77
9 changed files with 662 additions and 53 deletions

View File

@ -96,6 +96,10 @@ type BlockDAG struct {
// virtual tracks the current tips.
virtual *virtualBlock
// lastSubNetworkID holds the last registered sub-network ID in the DAG.
// Note that it is NOT the total amount of registered (or active) sub-networks.
lastSubNetworkID uint64
// These fields are related to handling of orphan blocks. They are
// protected by a combination of the chain lock and the orphan lock.
orphanLock sync.RWMutex
@ -512,8 +516,9 @@ func (dag *BlockDAG) connectBlock(node *blockNode, block *util.Block, fastAdd bo
}
}
var finalityPointCandidate *blockNode
initialFinalityPoint := dag.lastFinalityPoint
var finalityPointCandidate *blockNode
if !fastAdd {
var err error
finalityPointCandidate, err = dag.checkFinalityRulesAndGetFinalityPointCandidate(node)
@ -532,6 +537,14 @@ func (dag *BlockDAG) connectBlock(node *blockNode, block *util.Block, fastAdd bo
dag.lastFinalityPoint = finalityPointCandidate
}
// Scan all accepted transactions and collect any sub-network registry
// transactions into subNetworkRegistryTxs. If any sub-network registry
// transaction is not well-formed, fail the entire block.
subNetworkRegistryTxs, err := validateAndExtractSubNetworkRegistryTxs(acceptedTxsData)
if err != nil {
return err
}
// Write any block status changes to DB before updating the DAG state.
err = dag.index.flushToDB()
if err != nil {
@ -544,6 +557,7 @@ func (dag *BlockDAG) connectBlock(node *blockNode, block *util.Block, fastAdd bo
state := &dagState{
TipHashes: dag.TipHashes(),
LastFinalityPoint: dag.lastFinalityPoint.hash,
LastSubNetworkID: dag.lastSubNetworkID,
}
err := dbPutDAGState(dbTx, state)
if err != nil {
@ -564,6 +578,22 @@ func (dag *BlockDAG) connectBlock(node *blockNode, block *util.Block, fastAdd bo
return err
}
// Add the pending sub-network in this block to the pending sub-networks
// collection.
err = dbPutPendingSubNetworkTxs(dbTx, block.Hash(), subNetworkRegistryTxs)
if err != nil {
return err
}
// Register all pending sub-networks between the initial finality point and
// the new one.
if initialFinalityPoint != dag.lastFinalityPoint {
err = dag.registerPendingSubNetworks(dbTx, initialFinalityPoint, dag.lastFinalityPoint)
if err != nil {
return err
}
}
// Allow the index manager to call each of the currently active
// optional indexes with the block being connected so they can
// update themselves accordingly.
@ -1573,6 +1603,7 @@ func New(config *Config) (*BlockDAG, error) {
warningCaches: newThresholdCaches(vbNumBits),
deploymentCaches: newThresholdCaches(dagconfig.DefinedDeployments),
blockCount: 1,
lastSubNetworkID: wire.SubNetworkUnreservedFirst,
}
// Initialize the chain state from the passed database. When the db

View File

@ -53,6 +53,18 @@ var (
// unspent transaction output set.
utxoSetBucketName = []byte("utxoset")
// pendingSubNetworksBucketName is the name of the db bucket used to store the
// pending sub-networks.
pendingSubNetworksBucketName = []byte("pendingsubnetworks")
// registeredSubNetworkTxsBucketName is the name of the db bucket used to house
// the transactions that have been used to register sub-networks.
registeredSubNetworkTxsBucketName = []byte("registeredsubnetworktxs")
// subNetworksBucketName is the name of the db bucket used to store the
// sub-network registry.
subNetworksBucketName = []byte("subnetworks")
// byteOrder is the preferred byte order used for serializing numeric
// fields for storage in the database.
byteOrder = binary.LittleEndian
@ -567,48 +579,6 @@ func deserializeUTXOEntry(serialized []byte) (*UTXOEntry, error) {
return entry, nil
}
// dbFetchUTXOEntry uses an existing database transaction to fetch the specified
// transaction output from the UTXO set.
//
// When there is no entry for the provided output, nil will be returned for both
// the entry and the error.
func dbFetchUTXOEntry(dbTx database.Tx, outpoint wire.OutPoint) (*UTXOEntry, error) {
// Fetch the unspent transaction output information for the passed
// transaction output. Return now when there is no entry.
key := outpointKey(outpoint)
utxoBucket := dbTx.Metadata().Bucket(utxoSetBucketName)
serializedUTXO := utxoBucket.Get(*key)
recycleOutpointKey(key)
if serializedUTXO == nil {
return nil, nil
}
// A non-nil zero-length entry means there is an entry in the database
// for a spent transaction output which should never be the case.
if len(serializedUTXO) == 0 {
return nil, AssertError(fmt.Sprintf("database contains entry "+
"for spent tx output %v", outpoint))
}
// Deserialize the utxo entry and return it.
entry, err := deserializeUTXOEntry(serializedUTXO)
if err != nil {
// Ensure any deserialization errors are returned as database
// corruption errors.
if isDeserializeErr(err) {
return nil, database.Error{
ErrorCode: database.ErrCorruption,
Description: fmt.Sprintf("corrupt UTXO entry "+
"for %v: %v", outpoint, err),
}
}
return nil, err
}
return entry, nil
}
// dbPutUTXODiff uses an existing database transaction to update the UTXO set
// in the database based on the provided UTXO view contents and state. In
// particular, only the entries that have been marked as modified are written
@ -700,6 +670,7 @@ func dbFetchHeightByHash(dbTx database.Tx, hash *daghash.Hash) (int32, error) {
type dagState struct {
TipHashes []daghash.Hash
LastFinalityPoint daghash.Hash
LastSubNetworkID uint64
}
// serializeDAGState returns the serialization of the DAG state.
@ -805,6 +776,25 @@ func (dag *BlockDAG) createDAGState() error {
return err
}
// Create the bucket that houses the pending sub-networks.
_, err = meta.CreateBucket(pendingSubNetworksBucketName)
if err != nil {
return err
}
// Create the bucket that houses the registered sub-networks to
// their registry transactions index.
_, err = meta.CreateBucket(registeredSubNetworkTxsBucketName)
if err != nil {
return err
}
// Create the bucket that houses the registered sub-networks.
_, err = meta.CreateBucket(subNetworksBucketName)
if err != nil {
return err
}
// Save the genesis block to the block index database.
err = dbStoreBlockNode(dbTx, node)
if err != nil {
@ -1002,6 +992,9 @@ func (dag *BlockDAG) initDAGState() error {
// Set the last finality point
dag.lastFinalityPoint = dag.index.LookupNode(&state.LastFinalityPoint)
// Set the last sub-network ID
dag.lastSubNetworkID = state.LastSubNetworkID
return nil
})
}

View File

@ -545,18 +545,20 @@ func TestDAGStateSerialization(t *testing.T) {
{
name: "genesis",
state: &dagState{
LastFinalityPoint: *newHashFromStr("000000000019d6689c085ae165831e934ff763ae46a2a6c172b3f1b60a8ce26f"),
TipHashes: []daghash.Hash{*newHashFromStr("000000000019d6689c085ae165831e934ff763ae46a2a6c172b3f1b60a8ce26f")},
LastFinalityPoint: *newHashFromStr("000000000019d6689c085ae165831e934ff763ae46a2a6c172b3f1b60a8ce26f"),
LastSubNetworkID: 123456,
},
serialized: []byte("{\"TipHashes\":[[111,226,140,10,182,241,179,114,193,166,162,70,174,99,247,79,147,30,131,101,225,90,8,156,104,214,25,0,0,0,0,0]],\"LastFinalityPoint\":[111,226,140,10,182,241,179,114,193,166,162,70,174,99,247,79,147,30,131,101,225,90,8,156,104,214,25,0,0,0,0,0]}"),
serialized: []byte("{\"TipHashes\":[[111,226,140,10,182,241,179,114,193,166,162,70,174,99,247,79,147,30,131,101,225,90,8,156,104,214,25,0,0,0,0,0]],\"LastFinalityPoint\":[111,226,140,10,182,241,179,114,193,166,162,70,174,99,247,79,147,30,131,101,225,90,8,156,104,214,25,0,0,0,0,0],\"LastSubNetworkID\":123456}"),
},
{
name: "block 1",
state: &dagState{
LastFinalityPoint: *newHashFromStr("000000000019d6689c085ae165831e934ff763ae46a2a6c172b3f1b60a8ce26f"),
TipHashes: []daghash.Hash{*newHashFromStr("00000000839a8e6886ab5951d76f411475428afc90947ee320161bbf18eb6048")},
LastFinalityPoint: *newHashFromStr("000000000019d6689c085ae165831e934ff763ae46a2a6c172b3f1b60a8ce26f"),
LastSubNetworkID: 684321,
},
serialized: []byte("{\"TipHashes\":[[72,96,235,24,191,27,22,32,227,126,148,144,252,138,66,117,20,65,111,215,81,89,171,134,104,142,154,131,0,0,0,0]],\"LastFinalityPoint\":[111,226,140,10,182,241,179,114,193,166,162,70,174,99,247,79,147,30,131,101,225,90,8,156,104,214,25,0,0,0,0,0]}"),
serialized: []byte("{\"TipHashes\":[[72,96,235,24,191,27,22,32,227,126,148,144,252,138,66,117,20,65,111,215,81,89,171,134,104,142,154,131,0,0,0,0]],\"LastFinalityPoint\":[111,226,140,10,182,241,179,114,193,166,162,70,174,99,247,79,147,30,131,101,225,90,8,156,104,214,25,0,0,0,0,0],\"LastSubNetworkID\":684321}"),
},
}

View File

@ -211,9 +211,14 @@ const (
// ErrFinality indicates that a block doesn't adhere to the finality rules
ErrFinality
// ErrTransactionsNotSorted indicates that transactions in block are not
// sorted by subnetwork
// sorted by sub-network
ErrTransactionsNotSorted
// ErrSubNetwork indicates that a block doesn't adhere to the sub-network
// registry rules
ErrSubNetworkRegistry
)
// Map of ErrorCode values back to their constant names for pretty printing.

357
blockdag/subnetworks.go Normal file
View File

@ -0,0 +1,357 @@
package blockdag
import (
"bytes"
"encoding/binary"
"fmt"
"github.com/daglabs/btcd/dagconfig/daghash"
"github.com/daglabs/btcd/database"
"github.com/daglabs/btcd/wire"
)
// validateAndExtractSubNetworkRegistryTxs filters the given input and extracts a list
// of valid sub-network registry transactions.
func validateAndExtractSubNetworkRegistryTxs(txs []*TxWithBlockHash) ([]*wire.MsgTx, error) {
validSubNetworkRegistryTxs := make([]*wire.MsgTx, 0)
for _, txData := range txs {
tx := txData.Tx.MsgTx()
if tx.SubNetworkID == wire.SubNetworkRegistry {
err := validateSubNetworkRegistryTransaction(tx)
if err != nil {
return nil, err
}
validSubNetworkRegistryTxs = append(validSubNetworkRegistryTxs, tx)
}
}
return validSubNetworkRegistryTxs, nil
}
// validateSubNetworkRegistryTransaction makes sure that a given sub-network registry
// transaction is valid. Such a transaction is valid iff:
// - Its entire payload is a uint64 (8 bytes)
func validateSubNetworkRegistryTransaction(tx *wire.MsgTx) error {
if len(tx.Payload) != 8 {
return ruleError(ErrSubNetworkRegistry, fmt.Sprintf("validation failed: subnetwork registry"+
"tx '%s' has an invalid payload", tx.TxHash()))
}
return nil
}
// registerPendingSubNetworks attempts to register all the pending sub-networks that
// had previously been defined between the initial finality point and the new one.
// Note: transactions within newFinalityPoint itself are not registered. Instead, they will
// be registered when the next finality point is chosen; then it will be the
// initialFinalityPoint.
func (dag *BlockDAG) registerPendingSubNetworks(dbTx database.Tx, initialFinalityPoint *blockNode, newFinalityPoint *blockNode) error {
var stack []*blockNode
for currentNode := newFinalityPoint; currentNode != initialFinalityPoint; currentNode = currentNode.selectedParent {
stack = append(stack, currentNode)
}
// Register a pending sub-network for all blues. The block itself is not explicitly
// registered since it will be one of the blues of the next block.
// Note that this means that the very last block in the selected parent chain is not
// registered. This is intentional.
for i := len(stack) - 1; i >= 0; i-- {
currentNode := stack[i]
for _, blue := range currentNode.blues {
err := dag.registerPendingSubNetworksInBlock(dbTx, blue.hash)
if err != nil {
return fmt.Errorf("failed to register pending sub-networks: %s", err)
}
}
}
return nil
}
// registerPendingSubNetworksInBlock attempts to register all the sub-networks
// that had been defined in a given block.
func (dag *BlockDAG) registerPendingSubNetworksInBlock(dbTx database.Tx, blockHash daghash.Hash) error {
pendingSubNetworkTxs, err := dbGetPendingSubNetworkTxs(dbTx, blockHash)
if err != nil {
return fmt.Errorf("failed to retrieve pending sub-network txs in block '%s': %s", blockHash, err)
}
for _, tx := range pendingSubNetworkTxs {
if !dbIsRegisteredSubNetworkTx(dbTx, tx.TxHash()) {
createdSubNetwork := newSubNetwork(tx)
err := dbRegisterSubNetwork(dbTx, dag.lastSubNetworkID, tx.TxHash(), createdSubNetwork)
if err != nil {
return fmt.Errorf("failed registering sub-network"+
"for tx '%s' in block '%s': %s", tx.TxHash(), blockHash, err)
}
dag.lastSubNetworkID++
}
}
err = dbRemovePendingSubNetworkTxs(dbTx, blockHash)
if err != nil {
return fmt.Errorf("failed to remove block '%s'"+
"from pending sub-networks: %s", blockHash, err)
}
return nil
}
// subNetwork returns a registered-and-finalized sub-network. If the sub-network
// does not exist this method returns an error.
func (dag *BlockDAG) subNetwork(subNetworkID uint64) (*subNetwork, error) {
var sNet *subNetwork
var err error
dbErr := dag.db.View(func(dbTx database.Tx) error {
sNet, err = dbGetSubNetwork(dbTx, subNetworkID)
return nil
})
if dbErr != nil {
return nil, fmt.Errorf("could not retrieve sub-network '%d': %s", subNetworkID, dbErr)
}
if err != nil {
return nil, fmt.Errorf("could not retrieve sub-network '%d': %s", subNetworkID, err)
}
return sNet, nil
}
// GasLimit returns the gas limit of a registered-and-finalized sub-network. If
// the sub-network does not exist this method returns an error.
func (dag *BlockDAG) GasLimit(subNetworkID uint64) (uint64, error) {
sNet, err := dag.subNetwork(subNetworkID)
if err != nil {
return 0, err
}
return sNet.gasLimit, nil
}
// -----------------------------------------------------------------------------
// The sub-network registry consists of three buckets:
// a. The pending sub-network bucket
// b. The registered transaction bucket
// c. The sub-network bucket
//
// All newly-discovered sub-network registry transactions are stored in the
// pending sub-network bucket. These transactions are withheld until the
// blocks that contain them become final.
//
// Once the block of a sub-network registry transaction becomes final, all the
// transactions within that block are retrieved and checked for validity.
// Valid transactions are then:
// 1. Assigned a sub-network ID
// 2. Added to the registered transaction bucket
// 3. Added to the sub-network bucket
// -----------------------------------------------------------------------------
// dbPutPendingSubNetworkTxs stores mappings from a block (via its hash) to an
// array of sub-network transactions.
func dbPutPendingSubNetworkTxs(dbTx database.Tx, blockHash *daghash.Hash, subNetworkRegistryTxs []*wire.MsgTx) error {
// Empty blocks are not written
if len(subNetworkRegistryTxs) == 0 {
return nil
}
serializedTxs, err := serializeSubNetworkRegistryTxs(subNetworkRegistryTxs)
if err != nil {
return fmt.Errorf("failed to serialize pending sub-network txs in block '%s': %s", blockHash, err)
}
// Store the serialized transactions
bucket := dbTx.Metadata().Bucket(pendingSubNetworksBucketName)
err = bucket.Put(blockHash[:], serializedTxs)
if err != nil {
return fmt.Errorf("failed to write pending sub-network txs in block '%s': %s", blockHash, err)
}
return nil
}
// dbGetPendingSubNetworkTxs retrieves an array of sub-network transactions,
// associated with a block's hash, that was previously stored with
// dbPutPendingSubNetworkTxs.
// Returns an empty slice if the hash was not previously stored.
func dbGetPendingSubNetworkTxs(dbTx database.Tx, blockHash daghash.Hash) ([]*wire.MsgTx, error) {
bucket := dbTx.Metadata().Bucket(pendingSubNetworksBucketName)
serializedTxsBytes := bucket.Get(blockHash[:])
if serializedTxsBytes == nil {
return []*wire.MsgTx{}, nil
}
txs, err := deserializeSubNetworkRegistryTxs(serializedTxsBytes)
if err != nil {
return nil, fmt.Errorf("failed to deserialize pending sub-network txs for block '%s': %s", blockHash, err)
}
return txs, nil
}
// serializeSubNetworkRegistryTxs serializes a slice of MsgTxs by serializing each transaction
// individually and appending it to one long byte slice.
func serializeSubNetworkRegistryTxs(subNetworkRegistryTxs []*wire.MsgTx) ([]byte, error) {
// Calculate the length in bytes of the serialized transactions
serializedTxsLength := uint64(0)
for _, tx := range subNetworkRegistryTxs {
serializedTxsLength += uint64(tx.SerializeSize())
}
serializedTxs := bytes.NewBuffer(make([]byte, 0, serializedTxsLength))
// Write each transaction in the order it appears in
for _, tx := range subNetworkRegistryTxs {
err := tx.Serialize(serializedTxs)
if err != nil {
return nil, fmt.Errorf("failed to serialize tx '%s': %s", tx.TxHash(), err)
}
}
return serializedTxs.Bytes(), nil
}
// deserializeSubNetworkRegistryTxs deserializes a byte slice into a slice of MsgTxs.
func deserializeSubNetworkRegistryTxs(serializedTxsBytes []byte) ([]*wire.MsgTx, error) {
serializedTxs := bytes.NewBuffer(serializedTxsBytes)
// Read each transaction and store it in txs until the end of the buffer
txs := make([]*wire.MsgTx, 0)
for serializedTxs.Len() > 0 {
var tx wire.MsgTx
err := tx.Deserialize(serializedTxs)
if err != nil {
return nil, fmt.Errorf("failed to deserialize pending sub-network txs: %s", err)
}
txs = append(txs, &tx)
}
return txs, nil
}
// dbRemovePendingSubNetworkTxs deletes an array of sub-network transactions,
// associated with a block's hash, that was previously stored with
// dbPutPendingSubNetworkTxs.
// This function does not return an error if the hash was not previously stored.
func dbRemovePendingSubNetworkTxs(dbTx database.Tx, blockHash daghash.Hash) error {
bucket := dbTx.Metadata().Bucket(pendingSubNetworksBucketName)
err := bucket.Delete(blockHash[:])
if err != nil {
return fmt.Errorf("failed to remove pending sub-network txs in block '%s': %s", blockHash, err)
}
return nil
}
// dbIsRegisteredSubNetworkTx checks whether a sub-network registry transaction
// had successfully registered a sub-network.
func dbIsRegisteredSubNetworkTx(dbTx database.Tx, txHash daghash.Hash) bool {
bucket := dbTx.Metadata().Bucket(registeredSubNetworkTxsBucketName)
subNetworkIDBytes := bucket.Get(txHash[:])
return subNetworkIDBytes != nil
}
// dbRegisterSubNetwork stores mappings:
// a. from the ID of the sub-network to the sub-network data.
// b. from the hash of a sub-network registry transaction to the sub-network ID.
func dbRegisterSubNetwork(dbTx database.Tx, subNetworkID uint64, txHash daghash.Hash, network *subNetwork) error {
// Serialize the sub-network ID
subNetworkIDBytes := make([]byte, 8)
byteOrder.PutUint64(subNetworkIDBytes, subNetworkID)
// Serialize the sub-network
serializedSubNetwork, err := serializeSubNetwork(network)
if err != nil {
return fmt.Errorf("failed to serialize sub-netowrk of tx '%s': %s", network.txHash, err)
}
// Store the sub-network
subNetworksBucket := dbTx.Metadata().Bucket(subNetworksBucketName)
err = subNetworksBucket.Put(subNetworkIDBytes, serializedSubNetwork)
if err != nil {
return fmt.Errorf("failed to write sub-netowrk of tx '%s': %s", network.txHash, err)
}
// Store the mapping between txHash and subNetworkID
registeredSubNetworkTxs := dbTx.Metadata().Bucket(registeredSubNetworkTxsBucketName)
err = registeredSubNetworkTxs.Put(txHash[:], subNetworkIDBytes)
if err != nil {
return fmt.Errorf("failed to put registered sub-networkTx '%s': %s", txHash, err)
}
return nil
}
func dbGetSubNetwork(dbTx database.Tx, subNetworkID uint64) (*subNetwork, error) {
// Serialize the sub-network ID
subNetworkIDBytes := make([]byte, 8)
byteOrder.PutUint64(subNetworkIDBytes, subNetworkID)
// Get the sub-network
bucket := dbTx.Metadata().Bucket(subNetworksBucketName)
serializedSubNetwork := bucket.Get(subNetworkIDBytes)
if serializedSubNetwork == nil {
return nil, fmt.Errorf("sub-network '%d' not found", subNetworkID)
}
return deserializeSubNetwork(serializedSubNetwork)
}
type subNetwork struct {
txHash daghash.Hash
gasLimit uint64
}
func newSubNetwork(tx *wire.MsgTx) *subNetwork {
txHash := tx.TxHash()
gasLimit := binary.LittleEndian.Uint64(tx.Payload[:8])
return &subNetwork{
txHash: txHash,
gasLimit: gasLimit,
}
}
// serializeSubNetwork serializes a subNetwork into the following binary format:
// | txHash (32 bytes) | gasLimit (8 bytes) |
func serializeSubNetwork(sNet *subNetwork) ([]byte, error) {
serializedSNet := bytes.NewBuffer(make([]byte, 0, 40))
// Write the tx hash
err := binary.Write(serializedSNet, byteOrder, sNet.txHash)
if err != nil {
return nil, fmt.Errorf("failed to serialize sub-network for tx '%s': %s", sNet.txHash, err)
}
// Write the gas limit
err = binary.Write(serializedSNet, byteOrder, sNet.gasLimit)
if err != nil {
return nil, fmt.Errorf("failed to serialize sub-network for tx '%s': %s", sNet.txHash, err)
}
return serializedSNet.Bytes(), nil
}
// deserializeSubNetwork deserializes a byte slice into a subNetwork.
// See serializeSubNetwork for the binary format.
func deserializeSubNetwork(serializedSNetBytes []byte) (*subNetwork, error) {
serializedSNet := bytes.NewBuffer(serializedSNetBytes)
// Read the tx hash
var txHash daghash.Hash
err := binary.Read(serializedSNet, byteOrder, &txHash)
if err != nil {
return nil, fmt.Errorf("failed to deserialize sub-network: %s", err)
}
// Read the gas limit
var gasLimit uint64
err = binary.Read(serializedSNet, byteOrder, &gasLimit)
if err != nil {
return nil, fmt.Errorf("failed to deserialize sub-network: %s", err)
}
return &subNetwork{
txHash: txHash,
gasLimit: gasLimit,
}, nil
}

View File

@ -0,0 +1,220 @@
package blockdag
import (
"encoding/binary"
"github.com/daglabs/btcd/dagconfig"
"github.com/daglabs/btcd/util"
"github.com/daglabs/btcd/wire"
"math"
"reflect"
"testing"
"time"
)
// TestSubNetworkRegistry tests the full sub-network registry flow. In this test:
// 1. We create a sub-network registry transaction and add its block to the DAG
// 2. Add 2*finalityInterval so that the sub-network registry transaction becomes final
// 3. Attempt to retrieve the gas limit of the now-registered sub-network
func TestSubNetworkRegistry(t *testing.T) {
params := dagconfig.SimNetParams
params.K = 1
dag, teardownFunc, err := DAGSetup("TestFinality", Config{
DAGParams: &params,
})
if err != nil {
t.Fatalf("Failed to setup DAG instance: %v", err)
}
defer teardownFunc()
blockTime := time.Unix(dag.genesis.timestamp, 0)
extraNonce := int64(0)
buildNextBlock := func(parents blockSet, txs []*wire.MsgTx) (*util.Block, error) {
// We need to change the blockTime to keep all block hashes unique
blockTime = blockTime.Add(time.Second)
// We need to change the extraNonce to keep coinbase hashes unique
extraNonce++
bh := &wire.BlockHeader{
Version: 1,
Bits: dag.genesis.bits,
ParentHashes: parents.hashes(),
Timestamp: blockTime,
}
msgBlock := wire.NewMsgBlock(bh)
blockHeight := parents.maxHeight() + 1
coinbaseTx, err := createCoinbaseTx(blockHeight, 1, extraNonce, dag.dagParams)
if err != nil {
return nil, err
}
_ = msgBlock.AddTransaction(coinbaseTx)
for _, tx := range txs {
_ = msgBlock.AddTransaction(tx)
}
return util.NewBlock(msgBlock), nil
}
addBlockToDAG := func(block *util.Block) (*blockNode, error) {
dag.dagLock.Lock()
defer dag.dagLock.Unlock()
err = dag.maybeAcceptBlock(block, BFNone)
if err != nil {
return nil, err
}
return dag.index.LookupNode(block.Hash()), nil
}
currentNode := dag.genesis
// Create a block with a valid sub-network registry transaction
gasLimit := uint64(12345)
registryTx := wire.NewMsgTx(wire.TxVersion)
registryTx.SubNetworkID = wire.SubNetworkRegistry
registryTx.Payload = make([]byte, 8)
binary.LittleEndian.PutUint64(registryTx.Payload, gasLimit)
// Add it to the DAG
registryBlock, err := buildNextBlock(setFromSlice(currentNode), []*wire.MsgTx{registryTx})
if err != nil {
t.Fatalf("could not build registry block: %s", err)
}
currentNode, err = addBlockToDAG(registryBlock)
if err != nil {
t.Fatalf("could not add registry block to DAG: %s", err)
}
// Add 2*finalityInterval to ensure the registry transaction is finalized
for currentNode.blueScore < 2*finalityInterval {
nextBlock, err := buildNextBlock(setFromSlice(currentNode), []*wire.MsgTx{})
if err != nil {
t.Fatalf("could not create block: %s", err)
}
currentNode, err = addBlockToDAG(nextBlock)
if err != nil {
t.Fatalf("could not add block to DAG: %s", err)
}
}
// Make sure that the sub-network had been successfully registered by trying
// to retrieve its gas limit.
mostRecentlyRegisteredSubNetworkID := dag.lastSubNetworkID - 1
limit, err := dag.GasLimit(mostRecentlyRegisteredSubNetworkID)
if err != nil {
t.Fatalf("could not retrieve gas limit: %s", err)
}
if limit != gasLimit {
t.Fatalf("unexpected gas limit. want: %d, got: %d", gasLimit, limit)
}
}
func TestSerializeSubNetworkRegistryTxs(t *testing.T) {
payload1 := make([]byte, 8)
binary.LittleEndian.PutUint64(payload1, uint64(100))
tx1 := wire.MsgTx{
Version: 1,
SubNetworkID: wire.SubNetworkRegistry,
Payload: payload1,
TxIn: []*wire.TxIn{
{
PreviousOutPoint: wire.OutPoint{
Hash: *newHashFromStr("0437cd7f8525ceed2324359c2d0ba26006d92d856a9c20fa0241106ee5a597c9"),
Index: 0,
},
SignatureScript: hexToBytes("47304402204e45e16932b8af514961a1d3a1a25fdf3f4f7732e9d624c6c61548ab5fb8cd410220181522ec8eca07de4860a4acdd12909d831cc56cbbac4622082221a8768d1d0901"),
Sequence: math.MaxUint64,
},
},
TxOut: []*wire.TxOut{{
Value: 1000000000,
PkScript: hexToBytes("4104ae1a62fe09c5f51b13905f07f06b99a2f7159b2225f374cd378d71302fa28414e7aab37397f554a7df5f142c21c1b7303b8a0626f1baded5c72a704f7e6cd84cac"),
}, {
Value: 4000000000,
PkScript: hexToBytes("410411db93e1dcdb8a016b49840f8c53bc1eb68a382e97b1482ecad7b148a6909a5cb2e0eaddfb84ccf9744464f82e160bfa9b8b64f9d4c03f999b8643f656b412a3ac"),
}},
}
payload2 := make([]byte, 8)
binary.LittleEndian.PutUint64(payload2, uint64(200))
tx2 := wire.MsgTx{
Version: 1,
SubNetworkID: wire.SubNetworkRegistry,
Payload: payload2,
TxIn: []*wire.TxIn{{
PreviousOutPoint: wire.OutPoint{
Hash: *newHashFromStr("0437cd7f8525ceed2324359c2d0ba26006d92d856a9c20fa0241106ee5a597c9"),
Index: 0,
},
SignatureScript: hexToBytes("47304402204e45e16932b8af514961a1d3a1a25fdf3f4f7732e9d624c6c61548ab5fb8cd410220181522ec8eca07de4860a4acdd12909d831cc56cbbac4622082221a8768d1d0901"),
Sequence: math.MaxUint64,
}},
TxOut: []*wire.TxOut{{
Value: 5000000,
PkScript: hexToBytes("76a914f419b8db4ba65f3b6fcc233acb762ca6f51c23d488ac"),
}, {
Value: 34400000000,
PkScript: hexToBytes("76a914cadf4fc336ab3c6a4610b75f31ba0676b7f663d288ac"),
}},
LockTime: 0,
}
tests := []struct {
name string
txs []*wire.MsgTx
}{
{
name: "empty slice",
txs: []*wire.MsgTx{},
},
{
name: "one transaction",
txs: []*wire.MsgTx{&tx1},
},
{
name: "two transactions",
txs: []*wire.MsgTx{&tx2, &tx1},
},
}
for _, test := range tests {
serializedTxs, err := serializeSubNetworkRegistryTxs(test.txs)
if err != nil {
t.Errorf("serialization in test '%s' unexpectedly failed: %s", test.name, err)
continue
}
deserializedTxs, err := deserializeSubNetworkRegistryTxs(serializedTxs)
if err != nil {
t.Errorf("deserialization in test '%s' unexpectedly failed: %s", test.name, err)
continue
}
if !reflect.DeepEqual(test.txs, deserializedTxs) {
t.Errorf("original txs and deserialized txs are not equal in test '%s'", test.name)
}
}
}
func TestSerializeSubNetwork(t *testing.T) {
sNet := &subNetwork{
txHash: *newHashFromStr("0437cd7f8525ceed2324359c2d0ba26006d92d856a9c20fa0241106ee5a597c9"),
gasLimit: 1000,
}
serializedSNet, err := serializeSubNetwork(sNet)
if err != nil {
t.Fatalf("sub-network serialization unexpectedly failed: %s", err)
}
deserializedSNet, err := deserializeSubNetwork(serializedSNet)
if err != nil {
t.Fatalf("sub-network deserialization unexpectedly failed: %s", err)
}
if !reflect.DeepEqual(sNet, deserializedSNet) {
t.Errorf("original sub-network and deserialized sub-network are not equal")
}
}

View File

@ -751,7 +751,7 @@ func loadConfig() (*Config, []string, error) {
cfg.MiningAddrs = append(cfg.MiningAddrs, addr)
}
if cfg.SubNetwork >= wire.SubNetworkReservedFirst && cfg.SubNetwork <= wire.SubNetworkReservedLast {
if cfg.SubNetwork >= wire.SubNetworkReservedFirst && cfg.SubNetwork < wire.SubNetworkUnreservedFirst {
str := "%s: subnetworkID %d is reserved for future use, and nodes can not run in it "
err := fmt.Errorf(str, funcName, cfg.SubNetwork)
fmt.Fprintln(os.Stderr, err)

View File

@ -83,7 +83,7 @@ func TestConstants(t *testing.T) {
t.Errorf("wire.SubNetworkReservedFirst value was changed from 0, therefore you probably need to update the help text for SubNetwork")
}
if wire.SubNetworkReservedLast != 255 {
t.Errorf("wire.SubNetworkReservedLast value was changed from 0, therefore you probably need to update the help text for SubNetwork")
if wire.SubNetworkUnreservedFirst != 256 {
t.Errorf("wire.SubNetworkUnreservedFirst value was changed from 0, therefore you probably need to update the help text for SubNetwork")
}
}

View File

@ -109,8 +109,9 @@ const (
// and are currently un-assigned for anything
// SubNetworkReservedFirst is the first reserved sub-network
SubNetworkReservedFirst = 3
// SubNetworkReservedLast is the last reserved sub-network
SubNetworkReservedLast = 255
// SubNetworkUnreservedFirst is the first unreserved sub-network
SubNetworkUnreservedFirst = 256
)
// scriptFreeList defines a free list of byte slices (up to the maximum number