[NOD-1164] Remove the singleton from dbaccess, to enable multiple db connections in same run (#806)

* [NOD-1164] Defined DatabaseContext as the basic object of dbaccess

* [NOD-1164] Update everything to use databaseContext

* [NOD-1164] Fix tests

* [NOD-1164] Add comments

* [NOD-1164] Removed databaseContext from blockNode

* [NOD-1164] Enforce DatabaseContext != nil

* [NOD-1164] Remove redundant and wrong comment line
This commit is contained in:
Svarog 2020-07-21 12:02:44 +03:00 committed by GitHub
parent b797436884
commit 6b5b4bfb2a
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
28 changed files with 303 additions and 293 deletions

View File

@ -36,6 +36,7 @@ type triedBucket [TriedBucketCount]*list.List
// peers on the Kaspa network.
type AddrManager struct {
cfg *config.Config
databaseContext *dbaccess.DatabaseContext
mtx sync.Mutex
lookupFunc func(string) ([]net.IP, error)
rand *rand.Rand
@ -58,6 +59,21 @@ type AddrManager struct {
localSubnetworkID *subnetworkid.SubnetworkID
}
// New returns a new Kaspa address manager.
func New(cfg *config.Config, databaseContext *dbaccess.DatabaseContext) *AddrManager {
addressManager := AddrManager{
cfg: cfg,
databaseContext: databaseContext,
lookupFunc: cfg.Lookup,
rand: rand.New(rand.NewSource(time.Now().UnixNano())),
quit: make(chan struct{}),
localAddresses: make(map[string]*localAddress),
localSubnetworkID: cfg.SubnetworkID,
}
addressManager.reset()
return &addressManager
}
type serializedKnownAddress struct {
Addr string
Src string
@ -453,7 +469,7 @@ func (a *AddrManager) savePeers() error {
return err
}
return dbaccess.StorePeersState(dbaccess.NoTx(), serializedPeersState)
return dbaccess.StorePeersState(a.databaseContext, serializedPeersState)
}
func (a *AddrManager) serializePeersState() ([]byte, error) {
@ -563,7 +579,7 @@ func (a *AddrManager) loadPeers() error {
a.mtx.Lock()
defer a.mtx.Unlock()
serializedPeerState, err := dbaccess.FetchPeersState(dbaccess.NoTx())
serializedPeerState, err := dbaccess.FetchPeersState(a.databaseContext)
if dbaccess.IsNotFoundError(err) {
a.reset()
log.Info("No peers state was found in the database. Created a new one", a.totalNumAddresses())
@ -1356,18 +1372,3 @@ func (a *AddrManager) GetBestLocalAddress(remoteAddr *wire.NetAddress) *wire.Net
return bestAddress
}
// New returns a new Kaspa address manager.
// Use Start to begin processing asynchronous address updates.
func New(cfg *config.Config) *AddrManager {
am := AddrManager{
cfg: cfg,
lookupFunc: cfg.Lookup,
rand: rand.New(rand.NewSource(time.Now().UnixNano())),
quit: make(chan struct{}),
localAddresses: make(map[string]*localAddress),
localSubnetworkID: cfg.SubnetworkID,
}
am.reset()
return &am
}

View File

@ -119,15 +119,15 @@ func newAddrManagerForTest(t *testing.T, testName string,
t.Fatalf("Error creating temporary directory: %s", err)
}
err = dbaccess.Open(dbPath)
databaseContext, err := dbaccess.New(dbPath)
if err != nil {
t.Fatalf("error creating db: %s", err)
}
addressManager = New(cfg)
addressManager = New(cfg, databaseContext)
return addressManager, func() {
err := dbaccess.Close()
err := databaseContext.Close()
if err != nil {
t.Fatalf("error closing the database: %s", err)
}

View File

@ -6,6 +6,7 @@ package blockdag
import (
"fmt"
"github.com/kaspanet/kaspad/dbaccess"
"github.com/kaspanet/kaspad/util"
"github.com/pkg/errors"
@ -17,7 +18,7 @@ func (dag *BlockDAG) addNodeToIndexWithInvalidAncestor(block *util.Block) error
newNode.status = statusInvalidAncestor
dag.index.AddNode(newNode)
dbTx, err := dbaccess.NewTx()
dbTx, err := dag.databaseContext.NewTx()
if err != nil {
return err
}
@ -72,7 +73,7 @@ func (dag *BlockDAG) maybeAcceptBlock(block *util.Block, flags BehaviorFlags) er
// expensive connection logic. It also has some other nice properties
// such as making blocks that never become part of the DAG or
// blocks that fail to connect available for further analysis.
dbTx, err := dbaccess.NewTx()
dbTx, err := dag.databaseContext.NewTx()
if err != nil {
return err
}

View File

@ -6,10 +6,11 @@ package blockdag
import (
"fmt"
"math"
"github.com/kaspanet/kaspad/dagconfig"
"github.com/kaspanet/kaspad/util/mstime"
"github.com/pkg/errors"
"math"
"github.com/kaspanet/kaspad/util/daghash"
"github.com/kaspanet/kaspad/wire"

View File

@ -4,6 +4,8 @@ import (
"bufio"
"bytes"
"encoding/binary"
"io"
"github.com/kaspanet/kaspad/dbaccess"
"github.com/kaspanet/kaspad/util"
"github.com/kaspanet/kaspad/util/coinbasepayload"
@ -12,7 +14,6 @@ import (
"github.com/kaspanet/kaspad/util/txsort"
"github.com/kaspanet/kaspad/wire"
"github.com/pkg/errors"
"io"
)
// compactFeeData is a specialized data type to store a compact list of fees
@ -75,11 +76,11 @@ func (cfr *compactFeeIterator) next() (uint64, error) {
// getBluesFeeData returns the compactFeeData for all nodes's blues,
// used to calculate the fees this blockNode needs to pay
func (node *blockNode) getBluesFeeData(dag *BlockDAG) (map[daghash.Hash]compactFeeData, error) {
func (dag *BlockDAG) getBluesFeeData(node *blockNode) (map[daghash.Hash]compactFeeData, error) {
bluesFeeData := make(map[daghash.Hash]compactFeeData)
for _, blueBlock := range node.blues {
feeData, err := dbaccess.FetchFeeData(dbaccess.NoTx(), blueBlock.hash)
feeData, err := dbaccess.FetchFeeData(dag.databaseContext, blueBlock.hash)
if err != nil {
return nil, err
}
@ -118,7 +119,7 @@ func (node *blockNode) validateCoinbaseTransaction(dag *BlockDAG, block *util.Bl
// expectedCoinbaseTransaction returns the coinbase transaction for the current block
func (node *blockNode) expectedCoinbaseTransaction(dag *BlockDAG, txsAcceptanceData MultiBlockTxsAcceptanceData, scriptPubKey []byte, extraData []byte) (*util.Tx, error) {
bluesFeeData, err := node.getBluesFeeData(dag)
bluesFeeData, err := dag.getBluesFeeData(node)
if err != nil {
return nil, err
}

View File

@ -63,11 +63,12 @@ type BlockDAG struct {
// The following fields are set when the instance is created and can't
// be changed afterwards, so there is no need to protect them with a
// separate mutex.
Params *dagconfig.Params
timeSource TimeSource
sigCache *txscript.SigCache
indexManager IndexManager
genesis *blockNode
Params *dagconfig.Params
databaseContext *dbaccess.DatabaseContext
timeSource TimeSource
sigCache *txscript.SigCache
indexManager IndexManager
genesis *blockNode
// The following fields are calculated based upon the provided DAG
// parameters. They are also set when the instance is created and
@ -162,6 +163,106 @@ type BlockDAG struct {
startTime mstime.Time
}
// New returns a BlockDAG instance using the provided configuration details.
func New(config *Config) (*BlockDAG, error) {
// Enforce required config fields.
if config.DAGParams == nil {
return nil, errors.New("BlockDAG.New DAG parameters nil")
}
if config.TimeSource == nil {
return nil, errors.New("BlockDAG.New timesource is nil")
}
if config.DatabaseContext == nil {
return nil, errors.New("BlockDAG.DatabaseContext timesource is nil")
}
params := config.DAGParams
targetTimePerBlock := int64(params.TargetTimePerBlock / time.Second)
index := newBlockIndex(params)
dag := &BlockDAG{
Params: params,
databaseContext: config.DatabaseContext,
timeSource: config.TimeSource,
sigCache: config.SigCache,
indexManager: config.IndexManager,
targetTimePerBlock: targetTimePerBlock,
difficultyAdjustmentWindowSize: params.DifficultyAdjustmentWindowSize,
TimestampDeviationTolerance: params.TimestampDeviationTolerance,
powMaxBits: util.BigToCompact(params.PowMax),
index: index,
orphans: make(map[daghash.Hash]*orphanBlock),
prevOrphans: make(map[daghash.Hash][]*orphanBlock),
delayedBlocks: make(map[daghash.Hash]*delayedBlock),
delayedBlocksQueue: newDelayedBlocksHeap(),
warningCaches: newThresholdCaches(vbNumBits),
deploymentCaches: newThresholdCaches(dagconfig.DefinedDeployments),
blockCount: 0,
subnetworkID: config.SubnetworkID,
startTime: mstime.Now(),
}
dag.virtual = newVirtualBlock(dag, nil)
dag.utxoDiffStore = newUTXODiffStore(dag)
dag.multisetStore = newMultisetStore(dag)
dag.reachabilityTree = newReachabilityTree(dag)
// Initialize the DAG state from the passed database. When the db
// does not yet contain any DAG state, both it and the DAG state
// will be initialized to contain only the genesis block.
err := dag.initDAGState()
if err != nil {
return nil, err
}
// Initialize and catch up all of the currently active optional indexes
// as needed.
if config.IndexManager != nil {
err = config.IndexManager.Init(dag, dag.databaseContext)
if err != nil {
return nil, err
}
}
genesis, ok := index.LookupNode(params.GenesisHash)
if !ok {
genesisBlock := util.NewBlock(dag.Params.GenesisBlock)
// To prevent the creation of a new err variable unintentionally so the
// defered function above could read err - declare isOrphan and isDelayed explicitly.
var isOrphan, isDelayed bool
isOrphan, isDelayed, err = dag.ProcessBlock(genesisBlock, BFNone)
if err != nil {
return nil, err
}
if isDelayed {
return nil, errors.New("genesis block shouldn't be in the future")
}
if isOrphan {
return nil, errors.New("genesis block is unexpectedly orphan")
}
genesis, ok = index.LookupNode(params.GenesisHash)
if !ok {
return nil, errors.New("genesis is not found in the DAG after it was proccessed")
}
}
// Save a reference to the genesis block.
dag.genesis = genesis
// Initialize rule change threshold state caches.
err = dag.initThresholdCaches()
if err != nil {
return nil, err
}
selectedTip := dag.selectedTip()
log.Infof("DAG state (blue score %d, hash %s)",
selectedTip.blueScore, selectedTip.hash)
return dag, nil
}
// IsKnownBlock returns whether or not the DAG instance has the block represented
// by the passed hash. This includes checking the various places a block can
// be in, like part of the DAG or the orphan pool.
@ -493,7 +594,7 @@ func (dag *BlockDAG) addBlock(node *blockNode,
if errors.As(err, &RuleError{}) {
dag.index.SetStatusFlags(node, statusValidateFailed)
dbTx, err := dbaccess.NewTx()
dbTx, err := dag.databaseContext.NewTx()
if err != nil {
return nil, err
}
@ -686,7 +787,7 @@ func addTxToMultiset(ms *secp256k1.MultiSet, tx *wire.MsgTx, pastUTXO UTXOSet, b
func (dag *BlockDAG) saveChangesFromBlock(block *util.Block, virtualUTXODiff *UTXODiff,
txsAcceptanceData MultiBlockTxsAcceptanceData, feeData compactFeeData) error {
dbTx, err := dbaccess.NewTx()
dbTx, err := dag.databaseContext.NewTx()
if err != nil {
return err
}
@ -788,7 +889,7 @@ func (dag *BlockDAG) validateGasLimit(block *util.Block) error {
if !msgTx.SubnetworkID.IsEqual(currentSubnetworkID) {
currentSubnetworkID = &msgTx.SubnetworkID
currentGasUsage = 0
currentSubnetworkGasLimit, err = GasLimit(currentSubnetworkID)
currentSubnetworkGasLimit, err = dag.GasLimit(currentSubnetworkID)
if err != nil {
return errors.Errorf("Error getting gas limit for subnetworkID '%s': %s", currentSubnetworkID, err)
}
@ -906,7 +1007,7 @@ func (dag *BlockDAG) finalizeNodesBelowFinalityPoint(deleteDiffData bool) {
}
}
if deleteDiffData {
err := dag.utxoDiffStore.removeBlocksDiffData(dbaccess.NoTx(), nodesToDelete)
err := dag.utxoDiffStore.removeBlocksDiffData(dag.databaseContext, nodesToDelete)
if err != nil {
panic(fmt.Sprintf("Error removing diff data from utxoDiffStore: %s", err))
}
@ -1137,10 +1238,10 @@ func genesisPastUTXO(virtual *virtualBlock) UTXOSet {
return genesisPastUTXO
}
func (node *blockNode) fetchBlueBlocks() ([]*util.Block, error) {
func (dag *BlockDAG) fetchBlueBlocks(node *blockNode) ([]*util.Block, error) {
blueBlocks := make([]*util.Block, len(node.blues))
for i, blueBlockNode := range node.blues {
blueBlock, err := fetchBlockByHash(dbaccess.NoTx(), blueBlockNode.hash)
blueBlock, err := dag.fetchBlockByHash(blueBlockNode.hash)
if err != nil {
return nil, err
}
@ -1254,7 +1355,7 @@ func (dag *BlockDAG) pastUTXO(node *blockNode) (
return nil, nil, nil, err
}
blueBlocks, err := node.fetchBlueBlocks()
blueBlocks, err := dag.fetchBlueBlocks(node)
if err != nil {
return nil, nil, nil, err
}
@ -1979,7 +2080,7 @@ func (dag *BlockDAG) peekDelayedBlock() *delayedBlock {
type IndexManager interface {
// Init is invoked during DAG initialize in order to allow the index
// manager to initialize itself and any indexes it is managing.
Init(*BlockDAG) error
Init(*BlockDAG, *dbaccess.DatabaseContext) error
// ConnectBlock is invoked when a new block has been connected to the
// DAG.
@ -2026,102 +2127,10 @@ type Config struct {
//
// This field is required.
SubnetworkID *subnetworkid.SubnetworkID
}
// New returns a BlockDAG instance using the provided configuration details.
func New(config *Config) (*BlockDAG, error) {
// Enforce required config fields.
if config.DAGParams == nil {
return nil, errors.New("BlockDAG.New DAG parameters nil")
}
if config.TimeSource == nil {
return nil, errors.New("BlockDAG.New timesource is nil")
}
params := config.DAGParams
targetTimePerBlock := int64(params.TargetTimePerBlock / time.Second)
index := newBlockIndex(params)
dag := &BlockDAG{
Params: params,
timeSource: config.TimeSource,
sigCache: config.SigCache,
indexManager: config.IndexManager,
targetTimePerBlock: targetTimePerBlock,
difficultyAdjustmentWindowSize: params.DifficultyAdjustmentWindowSize,
TimestampDeviationTolerance: params.TimestampDeviationTolerance,
powMaxBits: util.BigToCompact(params.PowMax),
index: index,
orphans: make(map[daghash.Hash]*orphanBlock),
prevOrphans: make(map[daghash.Hash][]*orphanBlock),
delayedBlocks: make(map[daghash.Hash]*delayedBlock),
delayedBlocksQueue: newDelayedBlocksHeap(),
warningCaches: newThresholdCaches(vbNumBits),
deploymentCaches: newThresholdCaches(dagconfig.DefinedDeployments),
blockCount: 0,
subnetworkID: config.SubnetworkID,
startTime: mstime.Now(),
}
dag.virtual = newVirtualBlock(dag, nil)
dag.utxoDiffStore = newUTXODiffStore(dag)
dag.multisetStore = newMultisetStore(dag)
dag.reachabilityTree = newReachabilityTree(dag)
// Initialize the DAG state from the passed database. When the db
// does not yet contain any DAG state, both it and the DAG state
// will be initialized to contain only the genesis block.
err := dag.initDAGState()
if err != nil {
return nil, err
}
// Initialize and catch up all of the currently active optional indexes
// as needed.
if config.IndexManager != nil {
err = config.IndexManager.Init(dag)
if err != nil {
return nil, err
}
}
genesis, ok := index.LookupNode(params.GenesisHash)
if !ok {
genesisBlock := util.NewBlock(dag.Params.GenesisBlock)
// To prevent the creation of a new err variable unintentionally so the
// defered function above could read err - declare isOrphan and isDelayed explicitly.
var isOrphan, isDelayed bool
isOrphan, isDelayed, err = dag.ProcessBlock(genesisBlock, BFNone)
if err != nil {
return nil, err
}
if isDelayed {
return nil, errors.New("genesis block shouldn't be in the future")
}
if isOrphan {
return nil, errors.New("genesis block is unexpectedly orphan")
}
genesis, ok = index.LookupNode(params.GenesisHash)
if !ok {
return nil, errors.New("genesis is not found in the DAG after it was proccessed")
}
}
// Save a reference to the genesis block.
dag.genesis = genesis
// Initialize rule change threshold state caches.
err = dag.initThresholdCaches()
if err != nil {
return nil, err
}
selectedTip := dag.selectedTip()
log.Infof("DAG state (blue score %d, hash %s)",
selectedTip.blueScore, selectedTip.hash)
return dag, nil
// DatabaseContext is the context in which all database queries related to
// this DAG are going to run.
DatabaseContext *dbaccess.DatabaseContext
}
func (dag *BlockDAG) isKnownDelayedBlock(hash *daghash.Hash) bool {

View File

@ -6,9 +6,6 @@ package blockdag
import (
"fmt"
"github.com/kaspanet/go-secp256k1"
"github.com/kaspanet/kaspad/dbaccess"
"github.com/pkg/errors"
"math"
"os"
"path/filepath"
@ -16,6 +13,10 @@ import (
"testing"
"time"
"github.com/kaspanet/go-secp256k1"
"github.com/kaspanet/kaspad/dbaccess"
"github.com/pkg/errors"
"github.com/kaspanet/kaspad/dagconfig"
"github.com/kaspanet/kaspad/txscript"
"github.com/kaspanet/kaspad/util"
@ -556,18 +557,19 @@ func TestNew(t *testing.T) {
dbPath := filepath.Join(tempDir, "TestNew")
_ = os.RemoveAll(dbPath)
err := dbaccess.Open(dbPath)
databaseContext, err := dbaccess.New(dbPath)
if err != nil {
t.Fatalf("error creating db: %s", err)
}
defer func() {
dbaccess.Close()
databaseContext.Close()
os.RemoveAll(dbPath)
}()
config := &Config{
DAGParams: &dagconfig.SimnetParams,
TimeSource: NewTimeSource(),
SigCache: txscript.NewSigCache(1000),
DatabaseContext: databaseContext,
DAGParams: &dagconfig.SimnetParams,
TimeSource: NewTimeSource(),
SigCache: txscript.NewSigCache(1000),
}
_, err = New(config)
if err != nil {
@ -595,20 +597,21 @@ func TestAcceptingInInit(t *testing.T) {
// Create a test database
dbPath := filepath.Join(tempDir, "TestAcceptingInInit")
_ = os.RemoveAll(dbPath)
err := dbaccess.Open(dbPath)
databaseContext, err := dbaccess.New(dbPath)
if err != nil {
t.Fatalf("error creating db: %s", err)
}
defer func() {
dbaccess.Close()
databaseContext.Close()
os.RemoveAll(dbPath)
}()
// Create a DAG to add the test block into
config := &Config{
DAGParams: &dagconfig.SimnetParams,
TimeSource: NewTimeSource(),
SigCache: txscript.NewSigCache(1000),
DatabaseContext: databaseContext,
DAGParams: &dagconfig.SimnetParams,
TimeSource: NewTimeSource(),
SigCache: txscript.NewSigCache(1000),
}
dag, err := New(config)
if err != nil {
@ -632,7 +635,7 @@ func TestAcceptingInInit(t *testing.T) {
testNode.status = statusDataStored
// Manually add the test block to the database
dbTx, err := dbaccess.NewTx()
dbTx, err := databaseContext.NewTx()
if err != nil {
t.Fatalf("Failed to open database "+
"transaction: %s", err)
@ -924,7 +927,7 @@ func testFinalizeNodesBelowFinalityPoint(t *testing.T, deleteDiffData bool) {
blockTime := dag.genesis.Header().Timestamp
flushUTXODiffStore := func() {
dbTx, err := dbaccess.NewTx()
dbTx, err := dag.databaseContext.NewTx()
if err != nil {
t.Fatalf("Failed to open database transaction: %s", err)
}

View File

@ -164,7 +164,7 @@ func saveDAGState(dbContext dbaccess.Context, state *dagState) error {
// createDAGState initializes the DAG state to the
// genesis block and the node's local subnetwork id.
func (dag *BlockDAG) createDAGState(localSubnetworkID *subnetworkid.SubnetworkID) error {
return saveDAGState(dbaccess.NoTx(), &dagState{
return saveDAGState(dag.databaseContext, &dagState{
TipHashes: []*daghash.Hash{dag.Params.GenesisHash},
LastFinalityPoint: dag.Params.GenesisHash,
LocalSubnetworkID: localSubnetworkID,
@ -177,7 +177,7 @@ func (dag *BlockDAG) createDAGState(localSubnetworkID *subnetworkid.SubnetworkID
func (dag *BlockDAG) initDAGState() error {
// Fetch the stored DAG state from the database. If it doesn't exist,
// it means that kaspad is running for the first time.
serializedDAGState, err := dbaccess.FetchDAGState(dbaccess.NoTx())
serializedDAGState, err := dbaccess.FetchDAGState(dag.databaseContext)
if dbaccess.IsNotFoundError(err) {
// Initialize the database and the DAG state to the genesis block.
return dag.createDAGState(dag.subnetworkID)
@ -209,13 +209,13 @@ func (dag *BlockDAG) initDAGState() error {
}
log.Debugf("Loading reachability data...")
err = dag.reachabilityTree.init(dbaccess.NoTx())
err = dag.reachabilityTree.init(dag.databaseContext)
if err != nil {
return err
}
log.Debugf("Loading multiset data...")
err = dag.multisetStore.init(dbaccess.NoTx())
err = dag.multisetStore.init(dag.databaseContext)
if err != nil {
return err
}
@ -263,7 +263,7 @@ func (dag *BlockDAG) validateLocalSubnetworkID(state *dagState) error {
}
func (dag *BlockDAG) initBlockIndex() (unprocessedBlockNodes []*blockNode, err error) {
blockIndexCursor, err := dbaccess.BlockIndexCursor(dbaccess.NoTx())
blockIndexCursor, err := dbaccess.BlockIndexCursor(dag.databaseContext)
if err != nil {
return nil, err
}
@ -317,7 +317,7 @@ func (dag *BlockDAG) initBlockIndex() (unprocessedBlockNodes []*blockNode, err e
func (dag *BlockDAG) initUTXOSet() (fullUTXOCollection utxoCollection, err error) {
fullUTXOCollection = make(utxoCollection)
cursor, err := dbaccess.UTXOSetCursor(dbaccess.NoTx())
cursor, err := dbaccess.UTXOSetCursor(dag.databaseContext)
if err != nil {
return nil, err
}
@ -368,7 +368,7 @@ func (dag *BlockDAG) processUnprocessedBlockNodes(unprocessedBlockNodes []*block
for _, node := range unprocessedBlockNodes {
// Check to see if the block exists in the block DB. If it
// doesn't, the database has certainly been corrupted.
blockExists, err := dbaccess.HasBlock(dbaccess.NoTx(), node.hash)
blockExists, err := dbaccess.HasBlock(dag.databaseContext, node.hash)
if err != nil {
return errors.Wrapf(err, "HasBlock "+
"for block %s failed: %s", node.hash, err)
@ -379,7 +379,7 @@ func (dag *BlockDAG) processUnprocessedBlockNodes(unprocessedBlockNodes []*block
}
// Attempt to accept the block.
block, err := fetchBlockByHash(dbaccess.NoTx(), node.hash)
block, err := dag.fetchBlockByHash(node.hash)
if err != nil {
return err
}
@ -510,8 +510,8 @@ func (dag *BlockDAG) deserializeBlockNode(blockRow []byte) (*blockNode, error) {
// fetchBlockByHash retrieves the raw block for the provided hash,
// deserializes it, and returns a util.Block of it.
func fetchBlockByHash(dbContext dbaccess.Context, hash *daghash.Hash) (*util.Block, error) {
blockBytes, err := dbaccess.FetchBlock(dbContext, hash)
func (dag *BlockDAG) fetchBlockByHash(hash *daghash.Hash) (*util.Block, error) {
blockBytes, err := dbaccess.FetchBlock(dag.databaseContext, hash)
if err != nil {
return nil, err
}
@ -610,7 +610,7 @@ func (dag *BlockDAG) BlockByHash(hash *daghash.Hash) (*util.Block, error) {
return nil, ErrNotInDAG(str)
}
block, err := fetchBlockByHash(dbaccess.NoTx(), node.hash)
block, err := dag.fetchBlockByHash(node.hash)
if err != nil {
return nil, err
}
@ -639,7 +639,7 @@ func (dag *BlockDAG) BlockHashesFrom(lowHash *daghash.Hash, limit int) ([]*dagha
}
key := blockIndexKey(lowHash, blueScore)
cursor, err := dbaccess.BlockIndexCursorFrom(dbaccess.NoTx(), key)
cursor, err := dbaccess.BlockIndexCursorFrom(dag.databaseContext, key)
if dbaccess.IsNotFoundError(err) {
return nil, errors.Wrapf(err, "block %s not in block index", lowHash)
}

View File

@ -201,7 +201,7 @@ func TestSubnetworkRegistry(t *testing.T) {
if err != nil {
t.Fatalf("could not register network: %s", err)
}
limit, err := blockdag.GasLimit(subnetworkID)
limit, err := dag.GasLimit(subnetworkID)
if err != nil {
t.Fatalf("could not retrieve gas limit: %s", err)
}

View File

@ -2,14 +2,15 @@ package blockdag
import (
"fmt"
"github.com/kaspanet/kaspad/dagconfig"
"github.com/kaspanet/kaspad/dbaccess"
"github.com/kaspanet/kaspad/util"
"github.com/kaspanet/kaspad/util/daghash"
"reflect"
"sort"
"strings"
"testing"
"github.com/kaspanet/kaspad/dagconfig"
"github.com/kaspanet/kaspad/dbaccess"
"github.com/kaspanet/kaspad/util"
"github.com/kaspanet/kaspad/util/daghash"
)
type testBlockData struct {
@ -351,7 +352,7 @@ func TestGHOSTDAGErrors(t *testing.T) {
// Clear the reachability store
dag.reachabilityTree.store.loaded = map[daghash.Hash]*reachabilityData{}
dbTx, err := dbaccess.NewTx()
dbTx, err := dag.databaseContext.NewTx()
if err != nil {
t.Fatalf("NewTx: %s", err)
}

View File

@ -3,6 +3,7 @@ package indexers
import (
"bytes"
"encoding/gob"
"github.com/kaspanet/kaspad/blockdag"
"github.com/kaspanet/kaspad/dbaccess"
"github.com/kaspanet/kaspad/util"
@ -14,7 +15,8 @@ import (
// it stores a mapping between a block's hash and the set of transactions that the
// block accepts among its blue blocks.
type AcceptanceIndex struct {
dag *blockdag.BlockDAG
dag *blockdag.BlockDAG
databaseContext *dbaccess.DatabaseContext
}
// Ensure the AcceptanceIndex type implements the Indexer interface.
@ -31,8 +33,8 @@ func NewAcceptanceIndex() *AcceptanceIndex {
}
// DropAcceptanceIndex drops the acceptance index.
func DropAcceptanceIndex() error {
dbTx, err := dbaccess.NewTx()
func DropAcceptanceIndex(databaseContext *dbaccess.DatabaseContext) error {
dbTx, err := databaseContext.NewTx()
if err != nil {
return err
}
@ -49,8 +51,9 @@ func DropAcceptanceIndex() error {
// Init initializes the hash-based acceptance index.
//
// This is part of the Indexer interface.
func (idx *AcceptanceIndex) Init(dag *blockdag.BlockDAG) error {
func (idx *AcceptanceIndex) Init(dag *blockdag.BlockDAG, databaseContext *dbaccess.DatabaseContext) error {
idx.dag = dag
idx.databaseContext = databaseContext
return idx.recover()
}
@ -60,7 +63,7 @@ func (idx *AcceptanceIndex) Init(dag *blockdag.BlockDAG) error {
// This is part of the Indexer interface.
func (idx *AcceptanceIndex) recover() error {
return idx.dag.ForEachHash(func(hash daghash.Hash) error {
dbTx, err := dbaccess.NewTx()
dbTx, err := idx.databaseContext.NewTx()
if err != nil {
return err
}
@ -102,7 +105,7 @@ func (idx *AcceptanceIndex) ConnectBlock(dbContext *dbaccess.TxContext, blockHas
// TxsAcceptanceData returns the acceptance data of all the transactions that
// were accepted by the block with hash blockHash.
func (idx *AcceptanceIndex) TxsAcceptanceData(blockHash *daghash.Hash) (blockdag.MultiBlockTxsAcceptanceData, error) {
serializedTxsAcceptanceData, err := dbaccess.FetchAcceptanceData(dbaccess.NoTx(), blockHash)
serializedTxsAcceptanceData, err := dbaccess.FetchAcceptanceData(idx.databaseContext, blockHash)
if err != nil {
return nil, err
}

View File

@ -1,13 +1,6 @@
package indexers
import (
"github.com/kaspanet/kaspad/blockdag"
"github.com/kaspanet/kaspad/dagconfig"
"github.com/kaspanet/kaspad/dbaccess"
"github.com/kaspanet/kaspad/util"
"github.com/kaspanet/kaspad/util/daghash"
"github.com/kaspanet/kaspad/wire"
"github.com/pkg/errors"
"io"
"io/ioutil"
"os"
@ -15,6 +8,14 @@ import (
"reflect"
"syscall"
"testing"
"github.com/kaspanet/kaspad/blockdag"
"github.com/kaspanet/kaspad/dagconfig"
"github.com/kaspanet/kaspad/dbaccess"
"github.com/kaspanet/kaspad/util"
"github.com/kaspanet/kaspad/util/daghash"
"github.com/kaspanet/kaspad/wire"
"github.com/pkg/errors"
)
func TestAcceptanceIndexSerializationAndDeserialization(t *testing.T) {
@ -96,14 +97,15 @@ func TestAcceptanceIndexRecover(t *testing.T) {
}
defer os.RemoveAll(db1Path)
err = dbaccess.Open(db1Path)
databaseContext1, err := dbaccess.New(db1Path)
if err != nil {
t.Fatalf("error creating db: %s", err)
}
db1Config := blockdag.Config{
IndexManager: db1IndexManager,
DAGParams: params,
IndexManager: db1IndexManager,
DAGParams: params,
DatabaseContext: databaseContext1,
}
db1DAG, teardown, err := blockdag.DAGSetup("", false, db1Config)
@ -160,17 +162,18 @@ func TestAcceptanceIndexRecover(t *testing.T) {
t.Fatalf("Error fetching acceptance data: %s", err)
}
err = dbaccess.Close()
err = databaseContext1.Close()
if err != nil {
t.Fatalf("Error closing the database: %s", err)
}
err = dbaccess.Open(db2Path)
databaseContext2, err := dbaccess.New(db2Path)
if err != nil {
t.Fatalf("error creating db: %s", err)
}
db2Config := blockdag.Config{
DAGParams: params,
DAGParams: params,
DatabaseContext: databaseContext2,
}
db2DAG, teardown, err := blockdag.DAGSetup("", false, db2Config)
@ -206,11 +209,11 @@ func TestAcceptanceIndexRecover(t *testing.T) {
t.Fatalf("copyDirectory: %s", err)
}
err = dbaccess.Close()
err = databaseContext2.Close()
if err != nil {
t.Fatalf("Error closing the database: %s", err)
}
err = dbaccess.Open(db3Path)
databaseContext3, err := dbaccess.New(db3Path)
if err != nil {
t.Fatalf("error creating db: %s", err)
}
@ -218,8 +221,9 @@ func TestAcceptanceIndexRecover(t *testing.T) {
db3AcceptanceIndex := NewAcceptanceIndex()
db3IndexManager := NewManager([]Indexer{db3AcceptanceIndex})
db3Config := blockdag.Config{
IndexManager: db3IndexManager,
DAGParams: params,
IndexManager: db3IndexManager,
DAGParams: params,
DatabaseContext: databaseContext3,
}
_, teardown, err = blockdag.DAGSetup("", false, db3Config)

View File

@ -18,7 +18,7 @@ import (
type Indexer interface {
// Init is invoked when the index manager is first initializing the
// index.
Init(dag *blockdag.BlockDAG) error
Init(dag *blockdag.BlockDAG, databaseContext *dbaccess.DatabaseContext) error
// ConnectBlock is invoked when the index manager is notified that a new
// block has been connected to the DAG.

View File

@ -22,9 +22,9 @@ var _ blockdag.IndexManager = (*Manager)(nil)
// Init initializes the enabled indexes.
// This is part of the blockdag.IndexManager interface.
func (m *Manager) Init(dag *blockdag.BlockDAG) error {
func (m *Manager) Init(dag *blockdag.BlockDAG, databaseContext *dbaccess.DatabaseContext) error {
for _, indexer := range m.enabledIndexes {
if err := indexer.Init(dag); err != nil {
if err := indexer.Init(dag, databaseContext); err != nil {
return err
}
}

View File

@ -4,6 +4,7 @@ import (
"bytes"
"encoding/binary"
"fmt"
"github.com/kaspanet/kaspad/dbaccess"
"github.com/pkg/errors"
@ -75,8 +76,8 @@ func TxToSubnetworkID(tx *wire.MsgTx) (*subnetworkid.SubnetworkID, error) {
}
// fetchSubnetwork returns a registered subnetwork.
func fetchSubnetwork(subnetworkID *subnetworkid.SubnetworkID) (*subnetwork, error) {
serializedSubnetwork, err := dbaccess.FetchSubnetworkData(dbaccess.NoTx(), subnetworkID)
func (dag *BlockDAG) fetchSubnetwork(subnetworkID *subnetworkid.SubnetworkID) (*subnetwork, error) {
serializedSubnetwork, err := dbaccess.FetchSubnetworkData(dag.databaseContext, subnetworkID)
if err != nil {
return nil, err
}
@ -91,8 +92,8 @@ func fetchSubnetwork(subnetworkID *subnetworkid.SubnetworkID) (*subnetwork, erro
// GasLimit returns the gas limit of a registered subnetwork. If the subnetwork does not
// exist this method returns an error.
func GasLimit(subnetworkID *subnetworkid.SubnetworkID) (uint64, error) {
sNet, err := fetchSubnetwork(subnetworkID)
func (dag *BlockDAG) GasLimit(subnetworkID *subnetworkid.SubnetworkID) (uint64, error) {
sNet, err := dag.fetchSubnetwork(subnetworkID)
if err != nil {
return 0, err
}

View File

@ -5,11 +5,6 @@ package blockdag
import (
"compress/bzip2"
"encoding/binary"
"github.com/kaspanet/kaspad/database/ffldb/ldb"
"github.com/kaspanet/kaspad/dbaccess"
"github.com/kaspanet/kaspad/util"
"github.com/pkg/errors"
"github.com/syndtr/goleveldb/leveldb/opt"
"io"
"io/ioutil"
"os"
@ -19,6 +14,12 @@ import (
"sync"
"testing"
"github.com/kaspanet/kaspad/database/ffldb/ldb"
"github.com/kaspanet/kaspad/dbaccess"
"github.com/kaspanet/kaspad/util"
"github.com/pkg/errors"
"github.com/syndtr/goleveldb/leveldb/opt"
"github.com/kaspanet/kaspad/util/subnetworkid"
"github.com/kaspanet/kaspad/txscript"
@ -75,17 +76,19 @@ func DAGSetup(dbName string, openDb bool, config Config) (*BlockDAG, func(), err
dbPath := filepath.Join(tmpDir, dbName)
_ = os.RemoveAll(dbPath)
err = dbaccess.Open(dbPath)
databaseContext, err := dbaccess.New(dbPath)
if err != nil {
return nil, nil, errors.Errorf("error creating db: %s", err)
}
config.DatabaseContext = databaseContext
// Setup a teardown function for cleaning up. This function is
// returned to the caller to be invoked when it is done testing.
teardown = func() {
spawnWaitGroup.Wait()
spawn = realSpawn
dbaccess.Close()
databaseContext.Close()
ldb.Options = originalLDBOptions
os.RemoveAll(dbPath)
}

View File

@ -2,6 +2,7 @@ package blockdag
import (
"bytes"
"github.com/kaspanet/kaspad/dbaccess"
"github.com/kaspanet/kaspad/util/daghash"
"github.com/kaspanet/kaspad/util/locks"
@ -116,7 +117,7 @@ func (diffStore *utxoDiffStore) diffChildByNode(node *blockNode) (*blockNode, er
}
func (diffStore *utxoDiffStore) diffDataFromDB(hash *daghash.Hash) (*blockUTXODiffData, error) {
serializedBlockDiffData, err := dbaccess.FetchUTXODiffData(dbaccess.NoTx(), hash)
serializedBlockDiffData, err := dbaccess.FetchUTXODiffData(diffStore.dag.databaseContext, hash)
if err != nil {
return nil, err
}

View File

@ -66,7 +66,7 @@ func TestUTXODiffStore(t *testing.T) {
// Flush changes to db, delete them from the dag.utxoDiffStore.loaded
// map, and check if the diff data is re-fetched from the database.
dbTx, err := dbaccess.NewTx()
dbTx, err := dag.databaseContext.NewTx()
if err != nil {
t.Fatalf("Failed to open database transaction: %s", err)
}

View File

@ -185,18 +185,18 @@ func createEmptyAddressManagerForTest(t *testing.T, testName string, appConfig *
"failed: %s", err)
}
err = dbaccess.Open(path)
databaseContext, err := dbaccess.New(path)
if err != nil {
t.Fatalf("error creating db: %s", err)
}
return addrmgr.New(appConfig), func() {
return addrmgr.New(appConfig, databaseContext), func() {
// Wait for the connection manager to finish, so it'll
// have access to the address manager as long as it's
// alive.
time.Sleep(10 * time.Millisecond)
err := dbaccess.Close()
err := databaseContext.Close()
if err != nil {
t.Fatalf("error closing the database: %s", err)
}

View File

@ -1,11 +1,12 @@
package dbaccess
import (
"github.com/kaspanet/kaspad/dagconfig"
"github.com/kaspanet/kaspad/util"
"io/ioutil"
"reflect"
"testing"
"github.com/kaspanet/kaspad/dagconfig"
"github.com/kaspanet/kaspad/util"
)
func TestBlockStoreSanity(t *testing.T) {
@ -15,13 +16,13 @@ func TestBlockStoreSanity(t *testing.T) {
t.Fatalf("TestBlockStoreSanity: TempDir unexpectedly "+
"failed: %s", err)
}
err = Open(path)
databaseContext, err := New(path)
if err != nil {
t.Fatalf("TestBlockStoreSanity: Open unexpectedly "+
"failed: %s", err)
}
defer func() {
err := Close()
err := databaseContext.Close()
if err != nil {
t.Fatalf("TestBlockStoreSanity: Close unexpectedly "+
"failed: %s", err)
@ -36,7 +37,7 @@ func TestBlockStoreSanity(t *testing.T) {
t.Fatalf("TestBlockStoreSanity: util.Block.Bytes unexpectedly "+
"failed: %s", err)
}
dbTx, err := NewTx()
dbTx, err := databaseContext.NewTx()
if err != nil {
t.Fatalf("Failed to open database "+
"transaction: %s", err)
@ -54,7 +55,7 @@ func TestBlockStoreSanity(t *testing.T) {
}
// Make sure the genesis block now exists in the db
exists, err := HasBlock(NoTx(), genesisHash)
exists, err := HasBlock(databaseContext, genesisHash)
if err != nil {
t.Fatalf("TestBlockStoreSanity: HasBlock unexpectedly "+
"failed: %s", err)
@ -66,7 +67,7 @@ func TestBlockStoreSanity(t *testing.T) {
// Fetch the genesis block back from the db and make sure
// that it's equal to the original
fetchedGenesisBytes, err := FetchBlock(NoTx(), genesisHash)
fetchedGenesisBytes, err := FetchBlock(databaseContext, genesisHash)
if err != nil {
t.Fatalf("TestBlockStoreSanity: FetchBlock unexpectedly "+
"failed: %s", err)

View File

@ -11,17 +11,12 @@ type Context interface {
accessor() (database.DataAccessor, error)
}
type noTxContext struct{}
var noTxContextSingleton = &noTxContext{}
func (*noTxContext) accessor() (database.DataAccessor, error) {
return db()
type noTxContext struct {
backend *DatabaseContext
}
// NoTx creates and returns an instance of dbaccess.Context without an attached database transaction
func NoTx() Context {
return noTxContextSingleton
func (ctx *noTxContext) accessor() (database.DataAccessor, error) {
return ctx.backend.db, nil
}
// TxContext represents a database context with an attached database transaction
@ -29,6 +24,15 @@ type TxContext struct {
dbTransaction database.Transaction
}
// NewTx returns an instance of TxContext with a new database transaction
func (ctx *DatabaseContext) NewTx() (*TxContext, error) {
dbTransaction, err := ctx.db.Begin()
if err != nil {
return nil, err
}
return &TxContext{dbTransaction: dbTransaction}, nil
}
func (ctx *TxContext) accessor() (database.DataAccessor, error) {
return ctx.dbTransaction, nil
}
@ -48,16 +52,3 @@ func (ctx *TxContext) Rollback() error {
func (ctx *TxContext) RollbackUnlessClosed() error {
return ctx.dbTransaction.RollbackUnlessClosed()
}
// NewTx returns an instance of TxContext with a new database transaction
func NewTx() (*TxContext, error) {
db, err := db()
if err != nil {
return nil, err
}
dbTransaction, err := db.Begin()
if err != nil {
return nil, err
}
return &TxContext{dbTransaction: dbTransaction}, nil
}

View File

@ -3,41 +3,28 @@ package dbaccess
import (
"github.com/kaspanet/kaspad/database"
"github.com/kaspanet/kaspad/database/ffldb"
"github.com/pkg/errors"
)
// dbSingleton is a handle to an instance of the kaspad database
var dbSingleton database.Database
// db returns a handle to the database
func db() (database.Database, error) {
if dbSingleton == nil {
return nil, errors.New("database is not open")
}
return dbSingleton, nil
// DatabaseContext represents a context in which all database queries run
type DatabaseContext struct {
db database.Database
*noTxContext
}
// Open opens the database for given path
func Open(path string) error {
if dbSingleton != nil {
return errors.New("database is already open")
}
// New creates a new DatabaseContext with database is in the specified `path`
func New(path string) (*DatabaseContext, error) {
db, err := ffldb.Open(path)
if err != nil {
return err
return nil, err
}
dbSingleton = db
return nil
databaseContext := &DatabaseContext{db: db}
databaseContext.noTxContext = &noTxContext{backend: databaseContext}
return databaseContext, nil
}
// Close closes the database, if it's open
func Close() error {
if dbSingleton == nil {
return nil
}
err := dbSingleton.Close()
dbSingleton = nil
return err
// Close closes the DatabaseContext's connection, if it's open
func (ctx *DatabaseContext) Close() error {
return ctx.db.Close()
}

View File

@ -4,6 +4,8 @@ import (
"fmt"
"sync/atomic"
"github.com/kaspanet/kaspad/dbaccess"
"github.com/kaspanet/kaspad/dnsseed"
"github.com/kaspanet/kaspad/wire"
@ -104,13 +106,13 @@ func (k *kaspad) stop() error {
// newKaspad returns a new kaspad instance configured to listen on addr for the
// kaspa network type specified by dagParams. Use start to begin accepting
// connections from peers.
func newKaspad(cfg *config.Config, interrupt <-chan struct{}) (*kaspad, error) {
func newKaspad(cfg *config.Config, databaseContext *dbaccess.DatabaseContext, interrupt <-chan struct{}) (*kaspad, error) {
indexManager, acceptanceIndex := setupIndexes(cfg)
sigCache := txscript.NewSigCache(cfg.SigCacheMaxSize)
// Create a new block DAG instance with the appropriate configuration.
dag, err := setupDAG(cfg, interrupt, sigCache, indexManager)
dag, err := setupDAG(cfg, databaseContext, interrupt, sigCache, indexManager)
if err != nil {
return nil, err
}
@ -121,7 +123,7 @@ func newKaspad(cfg *config.Config, interrupt <-chan struct{}) (*kaspad, error) {
if err != nil {
return nil, err
}
addressManager := addrmgr.New(cfg)
addressManager := addrmgr.New(cfg, databaseContext)
protocolManager, err := protocol.NewManager(cfg, dag, addressManager, txMempool)
if err != nil {
@ -146,14 +148,17 @@ func newKaspad(cfg *config.Config, interrupt <-chan struct{}) (*kaspad, error) {
}, nil
}
func setupDAG(cfg *config.Config, interrupt <-chan struct{}, sigCache *txscript.SigCache, indexManager blockdag.IndexManager) (*blockdag.BlockDAG, error) {
func setupDAG(cfg *config.Config, databaseContext *dbaccess.DatabaseContext, interrupt <-chan struct{},
sigCache *txscript.SigCache, indexManager blockdag.IndexManager) (*blockdag.BlockDAG, error) {
dag, err := blockdag.New(&blockdag.Config{
Interrupt: interrupt,
DAGParams: cfg.NetParams(),
TimeSource: blockdag.NewTimeSource(),
SigCache: sigCache,
IndexManager: indexManager,
SubnetworkID: cfg.SubnetworkID,
Interrupt: interrupt,
DatabaseContext: databaseContext,
DAGParams: cfg.NetParams(),
TimeSource: blockdag.NewTimeSource(),
SigCache: sigCache,
IndexManager: indexManager,
SubnetworkID: cfg.SubnetworkID,
})
return dag, err
}

16
main.go
View File

@ -94,14 +94,14 @@ func kaspadMain(startedChan chan<- struct{}) error {
}
// Open the database
err = openDB(cfg)
databaseContext, err := openDB(cfg)
if err != nil {
log.Errorf("%s", err)
return err
}
defer func() {
log.Infof("Gracefully shutting down the database...")
err := dbaccess.Close()
err := databaseContext.Close()
if err != nil {
log.Errorf("Failed to close the database: %s", err)
}
@ -114,7 +114,7 @@ func kaspadMain(startedChan chan<- struct{}) error {
// Drop indexes and exit if requested.
if cfg.DropAcceptanceIndex {
if err := indexers.DropAcceptanceIndex(); err != nil {
if err := indexers.DropAcceptanceIndex(databaseContext); err != nil {
log.Errorf("%s", err)
return err
}
@ -123,7 +123,7 @@ func kaspadMain(startedChan chan<- struct{}) error {
}
// Create kaspad and start it.
kaspad, err := newKaspad(cfg, interrupt)
kaspad, err := newKaspad(cfg, databaseContext, interrupt)
if err != nil {
log.Errorf("Unable to start kaspad: %+v", err)
return err
@ -175,14 +175,10 @@ func blockDbPath(cfg *config.Config) string {
return dbPath
}
func openDB(cfg *config.Config) error {
func openDB(cfg *config.Config) (*dbaccess.DatabaseContext, error) {
dbPath := filepath.Join(cfg.DataDir, "db")
log.Infof("Loading database from '%s'", dbPath)
err := dbaccess.Open(dbPath)
if err != nil {
return err
}
return nil
return dbaccess.New(dbPath)
}
func main() {

View File

@ -843,7 +843,7 @@ func (mp *TxPool) maybeAcceptTransaction(tx *util.Tx, rejectDupOrphans bool) ([]
// Check that transaction does not overuse GAS
msgTx := tx.MsgTx()
if !msgTx.SubnetworkID.IsEqual(subnetworkid.SubnetworkIDNative) && !msgTx.SubnetworkID.IsEqual(subnetworkid.SubnetworkIDRegistry) {
gasLimit, err := blockdag.GasLimit(&msgTx.SubnetworkID)
gasLimit, err := mp.cfg.DAG.GasLimit(&msgTx.SubnetworkID)
if err != nil {
return nil, nil, err
}

View File

@ -158,7 +158,7 @@ func (g *BlkTmplGenerator) collectCandidatesTxs(sourceTxs []*TxDesc) []*candidat
gasLimit := uint64(0)
if !tx.MsgTx().SubnetworkID.IsEqual(subnetworkid.SubnetworkIDNative) && !tx.MsgTx().SubnetworkID.IsBuiltIn() {
subnetworkID := tx.MsgTx().SubnetworkID
gasLimit, err = blockdag.GasLimit(&subnetworkID)
gasLimit, err = g.dag.GasLimit(&subnetworkID)
if err != nil {
log.Warnf("Skipping tx %s due to error in "+
"GasLimit: %s", tx.ID(), err)
@ -208,7 +208,7 @@ func (g *BlkTmplGenerator) calcTxValue(tx *util.Tx, fee uint64) (float64, error)
}
gas := msgTx.Gas
gasLimit, err := blockdag.GasLimit(&msgTx.SubnetworkID)
gasLimit, err := g.dag.GasLimit(&msgTx.SubnetworkID)
if err != nil {
return 0, err
}

View File

@ -18,6 +18,8 @@ import (
"sync/atomic"
"time"
"github.com/kaspanet/kaspad/dbaccess"
"github.com/pkg/errors"
"github.com/kaspanet/kaspad/util/subnetworkid"
@ -1462,14 +1464,14 @@ out:
// kaspa network type specified by dagParams. Use start to begin accepting
// connections from peers.
func NewServer(cfg *config.Config, listenAddrs []string, dagParams *dagconfig.Params, interrupt <-chan struct{},
notifyNewTransactions func(txns []*mempool.TxDesc)) (*Server, error) {
notifyNewTransactions func(txns []*mempool.TxDesc), databaseContext *dbaccess.DatabaseContext) (*Server, error) {
services := defaultServices
if cfg.NoPeerBloomFilters {
services &^= wire.SFNodeBloom
}
addressManager := addrmgr.New(cfg)
addressManager := addrmgr.New(cfg, databaseContext)
var listeners []net.Listener
var nat serverutils.NAT

View File

@ -1,7 +1,6 @@
package rpc
import (
"github.com/kaspanet/kaspad/blockdag"
"github.com/kaspanet/kaspad/rpcmodel"
"github.com/kaspanet/kaspad/util/subnetworkid"
)
@ -18,7 +17,7 @@ func handleGetSubnetwork(s *Server, cmd interface{}, closeChan <-chan struct{})
var gasLimit *uint64
if !subnetworkID.IsEqual(subnetworkid.SubnetworkIDNative) &&
!subnetworkID.IsBuiltIn() {
limit, err := blockdag.GasLimit(subnetworkID)
limit, err := s.cfg.DAG.GasLimit(subnetworkID)
if err != nil {
return nil, &rpcmodel.RPCError{
Code: rpcmodel.ErrRPCSubnetworkNotFound,