mirror of
https://github.com/kaspanet/kaspad.git
synced 2025-06-05 13:46:42 +00:00
[DEV-343] Change SubNetworkRegistry to use SubNetworkID as Hash160 (#156)
* [DEV-343] Made sub-network registry use subNetworkIDs. * [DEV-343] Removed an unnecessary clone. * [DEV-343] Renamed buildSubNetworkID to txToSubNetworkID. Broke out of a loop when it was known that no further processing is required. Handled error cases from dbGetNetwork separately from the "not-found" case/ * [DEV-343] Added an error case in GasLimit() for where the sub-network is nil. * [DEV-343] Fixed return nil instead of err. Used a better way to check whether we should continue checking accepted transactions.
This commit is contained in:
parent
0553f7e2e7
commit
5171d26bba
@ -516,8 +516,6 @@ func (dag *BlockDAG) connectBlock(node *blockNode, block *util.Block, fastAdd bo
|
||||
}
|
||||
}
|
||||
|
||||
initialFinalityPoint := dag.lastFinalityPoint
|
||||
|
||||
var finalityPointCandidate *blockNode
|
||||
if !fastAdd {
|
||||
var err error
|
||||
@ -537,14 +535,6 @@ 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 {
|
||||
@ -557,7 +547,6 @@ 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 {
|
||||
@ -578,22 +567,14 @@ 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)
|
||||
// Scan all accepted transactions and register any sub-network registry
|
||||
// transaction. If any sub-network registry transaction is not well-formed,
|
||||
// fail the entire block.
|
||||
err = registerSubNetworks(dbTx, acceptedTxsData)
|
||||
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.
|
||||
@ -1603,7 +1584,6 @@ 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
|
||||
|
@ -992,9 +992,6 @@ 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
|
||||
})
|
||||
}
|
||||
|
@ -4,29 +4,57 @@ import (
|
||||
"bytes"
|
||||
"encoding/binary"
|
||||
"fmt"
|
||||
"github.com/daglabs/btcd/util"
|
||||
|
||||
"github.com/daglabs/btcd/dagconfig/daghash"
|
||||
"github.com/daglabs/btcd/database"
|
||||
"github.com/daglabs/btcd/util/subnetworkid"
|
||||
"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) {
|
||||
// registerSubNetworks scans a list of accepted transactions, singles out
|
||||
// sub-network registry transactions, validates them, and registers a new
|
||||
// sub-network based on it.
|
||||
// This function returns an error if one or more transactions are invalid
|
||||
func registerSubNetworks(dbTx database.Tx, txs []*TxWithBlockHash) 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
|
||||
return err
|
||||
}
|
||||
validSubNetworkRegistryTxs = append(validSubNetworkRegistryTxs, tx)
|
||||
}
|
||||
|
||||
if subnetworkid.Less(&wire.SubNetworkRegistry, &tx.SubNetworkID) {
|
||||
// Transactions are ordered by sub-network, so we can safely assume
|
||||
// that the rest of the transactions will not be sub-network registry
|
||||
// transactions.
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
return validSubNetworkRegistryTxs, nil
|
||||
for _, registryTx := range validSubNetworkRegistryTxs {
|
||||
subNetworkID, err := txToSubNetworkID(registryTx)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
sNet, err := dbGetSubNetwork(dbTx, subNetworkID)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if sNet == nil {
|
||||
createdSubNetwork := newSubNetwork(registryTx)
|
||||
err := dbRegisterSubNetwork(dbTx, subNetworkID, createdSubNetwork)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed registering sub-network"+
|
||||
"for tx '%s': %s", registryTx.TxHash(), err)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// validateSubNetworkRegistryTransaction makes sure that a given sub-network registry
|
||||
@ -41,66 +69,15 @@ func validateSubNetworkRegistryTransaction(tx *wire.MsgTx) error {
|
||||
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
|
||||
// txToSubNetworkID creates a sub-network ID from a sub-network registry transaction
|
||||
func txToSubNetworkID(tx *wire.MsgTx) (*subnetworkid.SubNetworkID, error) {
|
||||
txHash := tx.TxHash()
|
||||
return subnetworkid.New(util.Hash160(txHash[:]))
|
||||
}
|
||||
|
||||
// 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 subnetworkid.SubNetworkID) (*subNetwork, error) {
|
||||
// subNetwork returns a registered sub-network. If the sub-network does not exist
|
||||
// this method returns an error.
|
||||
func (dag *BlockDAG) subNetwork(subNetworkID *subnetworkid.SubNetworkID) (*subNetwork, error) {
|
||||
var sNet *subNetwork
|
||||
var err error
|
||||
dbErr := dag.db.View(func(dbTx database.Tx) error {
|
||||
@ -117,213 +94,70 @@ func (dag *BlockDAG) subNetwork(subNetworkID subnetworkid.SubNetworkID) (*subNet
|
||||
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 subnetworkid.SubNetworkID) (uint64, error) {
|
||||
// GasLimit returns the gas limit of a registered sub-network. If the sub-network does not
|
||||
// exist this method returns an error.
|
||||
func (dag *BlockDAG) GasLimit(subNetworkID *subnetworkid.SubNetworkID) (uint64, error) {
|
||||
sNet, err := dag.subNetwork(subNetworkID)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
if sNet == nil {
|
||||
return 0, fmt.Errorf("sub-network '%s' not found", subNetworkID)
|
||||
}
|
||||
|
||||
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)
|
||||
|
||||
// dbRegisterSubNetwork stores mappings from ID of the sub-network to the sub-network data.
|
||||
func dbRegisterSubNetwork(dbTx database.Tx, subNetworkID *subnetworkid.SubNetworkID, network *subNetwork) error {
|
||||
// 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)
|
||||
return fmt.Errorf("failed to serialize sub-netowrk '%s': %s", subNetworkID, err)
|
||||
}
|
||||
|
||||
// Store the sub-network
|
||||
subNetworksBucket := dbTx.Metadata().Bucket(subNetworksBucketName)
|
||||
err = subNetworksBucket.Put(subNetworkIDBytes, serializedSubNetwork)
|
||||
err = subNetworksBucket.Put(subNetworkID[:], 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 fmt.Errorf("failed to write sub-netowrk '%s': %s", subNetworkID, err)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func dbGetSubNetwork(dbTx database.Tx, subNetworkID subnetworkid.SubNetworkID) (*subNetwork, error) {
|
||||
// Get the sub-network
|
||||
// dbGetSubNetwork returns the sub-network associated with subNetworkID or nil if the sub-network was not found.
|
||||
func dbGetSubNetwork(dbTx database.Tx, subNetworkID *subnetworkid.SubNetworkID) (*subNetwork, error) {
|
||||
bucket := dbTx.Metadata().Bucket(subNetworksBucketName)
|
||||
serializedSubNetwork := bucket.Get(subNetworkID[:])
|
||||
if serializedSubNetwork == nil {
|
||||
return nil, fmt.Errorf("sub-network '%d' not found", subNetworkID)
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
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) |
|
||||
// | 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)
|
||||
}
|
||||
serializedSNet := bytes.NewBuffer(make([]byte, 0, 8))
|
||||
|
||||
// Write the gas limit
|
||||
err = binary.Write(serializedSNet, byteOrder, sNet.gasLimit)
|
||||
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 nil, fmt.Errorf("failed to serialize sub-network: %s", err)
|
||||
}
|
||||
|
||||
return serializedSNet.Bytes(), nil
|
||||
@ -334,22 +168,14 @@ func serializeSubNetwork(sNet *subNetwork) ([]byte, error) {
|
||||
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)
|
||||
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
|
||||
}
|
||||
|
@ -1,23 +1,17 @@
|
||||
package blockdag
|
||||
|
||||
import (
|
||||
"encoding/binary"
|
||||
"math"
|
||||
"reflect"
|
||||
"testing"
|
||||
|
||||
"github.com/daglabs/btcd/dagconfig"
|
||||
"github.com/daglabs/btcd/wire"
|
||||
)
|
||||
|
||||
// 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
|
||||
// TestSubNetworkRegistry tests the full sub-network registry flow
|
||||
func TestSubNetworkRegistry(t *testing.T) {
|
||||
params := dagconfig.SimNetParams
|
||||
params.K = 1
|
||||
dag, teardownFunc, err := DAGSetup("TestFinality", Config{
|
||||
dag, teardownFunc, err := DAGSetup("TestSubNetworkRegistry", Config{
|
||||
DAGParams: ¶ms,
|
||||
})
|
||||
if err != nil {
|
||||
@ -25,103 +19,22 @@ func TestSubNetworkRegistry(t *testing.T) {
|
||||
}
|
||||
defer teardownFunc()
|
||||
|
||||
_, err = RegisterSubnetworkForTest(dag, 12345)
|
||||
|
||||
gasLimit := uint64(12345)
|
||||
subNetworkID, err := RegisterSubNetworkForTest(dag, gasLimit)
|
||||
if err != nil {
|
||||
t.Fatalf("could not register network: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
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"),
|
||||
}},
|
||||
limit, err := dag.GasLimit(subNetworkID)
|
||||
if err != nil {
|
||||
t.Fatalf("could not retrieve gas limit: %s", err)
|
||||
}
|
||||
|
||||
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)
|
||||
}
|
||||
if limit != gasLimit {
|
||||
t.Fatalf("unexpected gas limit. want: %d, got: %d", gasLimit, limit)
|
||||
}
|
||||
}
|
||||
|
||||
func TestSerializeSubNetwork(t *testing.T) {
|
||||
sNet := &subNetwork{
|
||||
txHash: *newHashFromStr("0437cd7f8525ceed2324359c2d0ba26006d92d856a9c20fa0241106ee5a597c9"),
|
||||
gasLimit: 1000,
|
||||
}
|
||||
|
||||
|
@ -3,6 +3,7 @@ package blockdag
|
||||
import (
|
||||
"encoding/binary"
|
||||
"fmt"
|
||||
"github.com/daglabs/btcd/util/subnetworkid"
|
||||
"os"
|
||||
"path/filepath"
|
||||
"time"
|
||||
@ -143,9 +144,9 @@ func createCoinbaseTxForTest(blockHeight int32, numOutputs uint32, extraNonce in
|
||||
return tx, nil
|
||||
}
|
||||
|
||||
// RegisterSubnetworkForTest is used to register network on DAG with specified GAS limit.
|
||||
func RegisterSubnetworkForTest(dag *BlockDAG, gasLimit uint64) (subNetworkID uint64, err error) {
|
||||
blockTime := time.Unix(dag.genesis.timestamp, 0)
|
||||
// RegisterSubNetworkForTest is used to register network on DAG with specified gas limit
|
||||
func RegisterSubNetworkForTest(dag *BlockDAG, gasLimit uint64) (*subnetworkid.SubNetworkID, error) {
|
||||
blockTime := time.Unix(dag.selectedTip().timestamp, 0)
|
||||
extraNonce := int64(0)
|
||||
|
||||
buildNextBlock := func(parents blockSet, txs []*wire.MsgTx) (*util.Block, error) {
|
||||
@ -180,7 +181,7 @@ func RegisterSubnetworkForTest(dag *BlockDAG, gasLimit uint64) (subNetworkID uin
|
||||
dag.dagLock.Lock()
|
||||
defer dag.dagLock.Unlock()
|
||||
|
||||
err = dag.maybeAcceptBlock(block, BFNone)
|
||||
err := dag.maybeAcceptBlock(block, BFNone)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@ -188,7 +189,7 @@ func RegisterSubnetworkForTest(dag *BlockDAG, gasLimit uint64) (subNetworkID uin
|
||||
return dag.index.LookupNode(block.Hash()), nil
|
||||
}
|
||||
|
||||
currentNode := dag.genesis
|
||||
currentNode := dag.selectedTip()
|
||||
|
||||
// Create a block with a valid sub-network registry transaction
|
||||
registryTx := wire.NewMsgTx(wire.TxVersion)
|
||||
@ -199,34 +200,17 @@ func RegisterSubnetworkForTest(dag *BlockDAG, gasLimit uint64) (subNetworkID uin
|
||||
// Add it to the DAG
|
||||
registryBlock, err := buildNextBlock(setFromSlice(currentNode), []*wire.MsgTx{registryTx})
|
||||
if err != nil {
|
||||
return 0, fmt.Errorf("could not build registry block: %s", err)
|
||||
return nil, fmt.Errorf("could not build registry block: %s", err)
|
||||
}
|
||||
currentNode, err = addBlockToDAG(registryBlock)
|
||||
if err != nil {
|
||||
return 0, fmt.Errorf("could not add registry block to DAG: %s", err)
|
||||
return nil, fmt.Errorf("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 {
|
||||
return 0, fmt.Errorf("could not create block: %s", err)
|
||||
}
|
||||
currentNode, err = addBlockToDAG(nextBlock)
|
||||
if err != nil {
|
||||
return 0, fmt.Errorf("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)
|
||||
// Build a sub-network ID from the registry transaction
|
||||
subNetworkID, err := txToSubNetworkID(registryTx)
|
||||
if err != nil {
|
||||
return 0, fmt.Errorf("could not retrieve gas limit: %s", err)
|
||||
return nil, fmt.Errorf("could not build sub-network ID: %s", err)
|
||||
}
|
||||
if limit != gasLimit {
|
||||
return 0, fmt.Errorf("unexpected gas limit. want: %d, got: %d", gasLimit, limit)
|
||||
}
|
||||
return mostRecentlyRegisteredSubNetworkID, nil
|
||||
return subNetworkID, nil
|
||||
}
|
||||
|
@ -754,7 +754,7 @@ func loadConfig() (*Config, []string, error) {
|
||||
}
|
||||
|
||||
if cfg.configFlags.SubNetwork != "" {
|
||||
cfg.SubNetwork, err = subnetworkid.NewIDFromStr(cfg.configFlags.SubNetwork)
|
||||
cfg.SubNetwork, err = subnetworkid.NewFromStr(cfg.configFlags.SubNetwork)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
|
@ -659,10 +659,10 @@ func (mp *TxPool) maybeAcceptTransaction(tx *util.Tx, isNew, rateLimit, rejectDu
|
||||
|
||||
// Check that transaction does not overuse GAS
|
||||
msgTx := tx.MsgTx()
|
||||
if msgTx.SubNetworkID == 0 {
|
||||
if msgTx.SubNetworkID == wire.SubNetworkSupportsAll {
|
||||
return nil, nil, txRuleError(wire.RejectInvalid, "Subnetwork 0 is not permited in transaction")
|
||||
} else if msgTx.SubNetworkID != wire.SubNetworkDAGCoin {
|
||||
gasLimit, err := mp.cfg.DAG.GasLimit(msgTx.SubNetworkID)
|
||||
gasLimit, err := mp.cfg.DAG.GasLimit(&msgTx.SubNetworkID)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
|
@ -9,6 +9,7 @@ import (
|
||||
"encoding/hex"
|
||||
"errors"
|
||||
"fmt"
|
||||
"github.com/daglabs/btcd/util/subnetworkid"
|
||||
"math"
|
||||
"reflect"
|
||||
"runtime"
|
||||
@ -155,11 +156,11 @@ func (p *poolHarness) CreateCoinbaseTx(blockHeight int32, numOutputs uint32) (*u
|
||||
return util.NewTx(tx), nil
|
||||
}
|
||||
|
||||
// CreateSignedTxForSubnetwork creates a new signed transaction that consumes the provided
|
||||
// CreateSignedTxForSubNetwork creates a new signed transaction that consumes the provided
|
||||
// inputs and generates the provided number of outputs by evenly splitting the
|
||||
// total input amount. All outputs will be to the payment script associated
|
||||
// with the harness and all inputs are assumed to do the same.
|
||||
func (p *poolHarness) CreateSignedTxForSubnetwork(inputs []spendableOutpoint, numOutputs uint32, subnetworkID uint64, gas uint64) (*util.Tx, error) {
|
||||
func (p *poolHarness) CreateSignedTxForSubNetwork(inputs []spendableOutpoint, numOutputs uint32, subNetworkID *subnetworkid.SubNetworkID, gas uint64) (*util.Tx, error) {
|
||||
// Calculate the total input amount and split it amongst the requested
|
||||
// number of outputs.
|
||||
var totalInput util.Amount
|
||||
@ -170,7 +171,7 @@ func (p *poolHarness) CreateSignedTxForSubnetwork(inputs []spendableOutpoint, nu
|
||||
remainder := uint64(totalInput) - amountPerOutput*uint64(numOutputs)
|
||||
|
||||
tx := wire.NewMsgTx(wire.TxVersion)
|
||||
tx.SubNetworkID = subnetworkID
|
||||
tx.SubNetworkID = *subNetworkID
|
||||
tx.Gas = gas
|
||||
for _, input := range inputs {
|
||||
tx.AddTxIn(&wire.TxIn{
|
||||
@ -210,7 +211,7 @@ func (p *poolHarness) CreateSignedTxForSubnetwork(inputs []spendableOutpoint, nu
|
||||
// total input amount. All outputs will be to the payment script associated
|
||||
// with the harness and all inputs are assumed to do the same.
|
||||
func (p *poolHarness) CreateSignedTx(inputs []spendableOutpoint, numOutputs uint32) (*util.Tx, error) {
|
||||
return p.CreateSignedTxForSubnetwork(inputs, numOutputs, wire.SubNetworkDAGCoin, 0)
|
||||
return p.CreateSignedTxForSubNetwork(inputs, numOutputs, &wire.SubNetworkDAGCoin, 0)
|
||||
}
|
||||
|
||||
// CreateTxChain creates a chain of zero-fee transactions (each subsequent
|
||||
@ -1843,13 +1844,13 @@ func TestTransactionGas(t *testing.T) {
|
||||
// tc := &testContext{t, harness}
|
||||
|
||||
const gasLimit = 10000
|
||||
subnetworkID, err := blockdag.RegisterSubnetworkForTest(harness.txPool.cfg.DAG, gasLimit)
|
||||
subNetworkID, err := blockdag.RegisterSubNetworkForTest(harness.txPool.cfg.DAG, gasLimit)
|
||||
if err != nil {
|
||||
t.Fatalf("unable to register network: %v", err)
|
||||
}
|
||||
|
||||
// Create valid transaction
|
||||
tx, err := harness.CreateSignedTxForSubnetwork(spendableOuts[:1], 1, subnetworkID, gasLimit)
|
||||
tx, err := harness.CreateSignedTxForSubNetwork(spendableOuts[:1], 1, subNetworkID, gasLimit)
|
||||
if err != nil {
|
||||
t.Fatalf("unable to create transaction: %v", err)
|
||||
}
|
||||
@ -1859,7 +1860,7 @@ func TestTransactionGas(t *testing.T) {
|
||||
}
|
||||
|
||||
// Create invalid transaction
|
||||
tx, err = harness.CreateSignedTxForSubnetwork(spendableOuts[1:], 1, subnetworkID, gasLimit+1)
|
||||
tx, err = harness.CreateSignedTxForSubNetwork(spendableOuts[1:], 1, subNetworkID, gasLimit+1)
|
||||
if err != nil {
|
||||
t.Fatalf("unable to create transaction: %v", err)
|
||||
}
|
||||
|
@ -484,23 +484,23 @@ func (g *BlkTmplGenerator) NewBlockTemplate(payToAddress util.Address) (*BlockTe
|
||||
tx := prioItem.tx
|
||||
|
||||
if tx.MsgTx().SubNetworkID != wire.SubNetworkDAGCoin {
|
||||
subnetwork := tx.MsgTx().SubNetworkID
|
||||
gasUsage, ok := gasUsageMap[subnetwork]
|
||||
subNetwork := tx.MsgTx().SubNetworkID
|
||||
gasUsage, ok := gasUsageMap[subNetwork]
|
||||
if !ok {
|
||||
gasUsage = 0
|
||||
}
|
||||
gasLimit, err := g.dag.GasLimit(subnetwork)
|
||||
gasLimit, err := g.dag.GasLimit(&subNetwork)
|
||||
if err != nil {
|
||||
log.Errorf("Cannot get GAS limit for subnetwork %v", subnetwork)
|
||||
log.Errorf("Cannot get GAS limit for subNetwork %v", subNetwork)
|
||||
continue
|
||||
}
|
||||
txGas := tx.MsgTx().Gas
|
||||
if gasLimit-gasUsage < txGas {
|
||||
log.Tracef("Transaction %v (GAS=%v) ignored because gas overusage (GASUsage=%v) in subnetwork %v (GASLimit=%v)",
|
||||
tx.MsgTx().TxHash, txGas, gasUsage, subnetwork, gasLimit)
|
||||
log.Tracef("Transaction %v (GAS=%v) ignored because gas overusage (GASUsage=%v) in subNetwork %v (GASLimit=%v)",
|
||||
tx.MsgTx().TxHash, txGas, gasUsage, subNetwork, gasLimit)
|
||||
continue
|
||||
}
|
||||
gasUsageMap[subnetwork] = gasUsage + txGas
|
||||
gasUsageMap[subNetwork] = gasUsage + txGas
|
||||
}
|
||||
|
||||
// Enforce maximum block size. Also check for overflow.
|
||||
|
@ -96,14 +96,6 @@ const (
|
||||
// peers. Thus, the peak usage of the free list is 12,500 * 512 =
|
||||
// 6,400,000 bytes.
|
||||
freeListMaxItems = 12500
|
||||
|
||||
// SubNetworkReservedFirst and SubnetworkReservedLast mark the range of sub-networks that are reserved for future use
|
||||
// and are currently un-assigned for anything
|
||||
// SubNetworkReservedFirst is the first reserved sub-network
|
||||
SubNetworkReservedFirst = 3
|
||||
|
||||
// SubNetworkUnreservedFirst is the first unreserved sub-network
|
||||
SubNetworkUnreservedFirst = 256
|
||||
)
|
||||
|
||||
var (
|
||||
|
Loading…
x
Reference in New Issue
Block a user