Adding DAA score (#1596)

* Save DAA score and DAA added blocks for each block

* Add test

* Add pruning support

* Replace 8 with uint64Length

* Separate DAABlocksStore cache size to DAA score and daaAddedBlocks
This commit is contained in:
Ori Newman 2021-03-14 09:44:44 +02:00 committed by GitHub
parent 3d668cc1bd
commit ec446ac511
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
18 changed files with 470 additions and 34 deletions

View File

@ -46,6 +46,7 @@ type consensus struct {
utxoDiffStore model.UTXODiffStore
finalityStore model.FinalityStore
headersSelectedChainStore model.HeadersSelectedChainStore
daaBlocksStore model.DAABlocksStore
}
// BuildBlock builds a block over the current state, with the transactions

View File

@ -13,7 +13,7 @@ type GHOSTDAGManagerConstructor func(model.DBReader, model.DAGTopologyManager,
// DifficultyManagerConstructor is the function signature for a constructor of a type implementing model.DifficultyManager
type DifficultyManagerConstructor func(model.DBReader, model.GHOSTDAGManager, model.GHOSTDAGDataStore,
model.BlockHeaderStore, model.DAGTopologyManager, model.DAGTraversalManager, *big.Int, int, bool, time.Duration,
model.BlockHeaderStore, model.DAABlocksStore, model.DAGTopologyManager, model.DAGTraversalManager, *big.Int, int, bool, time.Duration,
*externalapi.DomainHash) model.DifficultyManager
// PastMedianTimeManagerConstructor is the function signature for a constructor of a type implementing model.PastMedianTimeManager

View File

@ -0,0 +1,24 @@
package binaryserialization
import (
"encoding/binary"
"github.com/pkg/errors"
)
const uint64Length = 8
// SerializeUint64 serializes a uint64
func SerializeUint64(value uint64) []byte {
var keyBytes [uint64Length]byte
binary.LittleEndian.PutUint64(keyBytes[:], value)
return keyBytes[:]
}
// DeserializeUint64 deserializes bytes to uint64
func DeserializeUint64(valueBytes []byte) (uint64, error) {
if len(valueBytes) != uint64Length {
return 0, errors.Errorf("the given value is %d bytes so it cannot be deserialized into uint64",
len(valueBytes))
}
return binary.LittleEndian.Uint64(valueBytes), nil
}

View File

@ -1,13 +1,47 @@
package binaryserialization
import "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
import (
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/pkg/errors"
)
// SerializeHash serializes hash to a slice of bytes
func SerializeHash(hash *externalapi.DomainHash) []byte {
return hash.ByteSlice()
}
// DeserializeHash a slice of bytes to a hash
// DeserializeHash deserializes a slice of bytes to a hash
func DeserializeHash(hashBytes []byte) (*externalapi.DomainHash, error) {
return externalapi.NewDomainHashFromByteSlice(hashBytes)
}
// SerializeHashes serializes a slice of hashes to a slice of bytes
func SerializeHashes(hashes []*externalapi.DomainHash) []byte {
buff := make([]byte, len(hashes)*externalapi.DomainHashSize)
for i, hash := range hashes {
copy(buff[externalapi.DomainHashSize*i:], hash.ByteSlice())
}
return buff
}
// DeserializeHashes deserializes a slice of bytes to a slice of hashes
func DeserializeHashes(hashesBytes []byte) ([]*externalapi.DomainHash, error) {
if len(hashesBytes)%externalapi.DomainHashSize != 0 {
return nil, errors.Errorf("The length of hashBytes is not divisible by externalapi.DomainHashSize (%d)",
externalapi.DomainHashSize)
}
numHashes := len(hashesBytes) / externalapi.DomainHashSize
hashes := make([]*externalapi.DomainHash, numHashes)
for i := 0; i < numHashes; i++ {
var err error
start := i * externalapi.DomainHashSize
end := i*externalapi.DomainHashSize + externalapi.DomainHashSize
hashes[i], err = externalapi.NewDomainHashFromByteSlice(hashesBytes[start:end])
if err != nil {
return nil, err
}
}
return hashes, nil
}

View File

@ -1,15 +0,0 @@
package binaryserialization
import "encoding/binary"
// SerializeChainBlockIndex serializes chain block index
func SerializeChainBlockIndex(index uint64) []byte {
var keyBytes [8]byte
binary.LittleEndian.PutUint64(keyBytes[:], index)
return keyBytes[:]
}
// DeserializeChainBlockIndex deserializes chain block index to uint64
func DeserializeChainBlockIndex(indexBytes []byte) uint64 {
return binary.LittleEndian.Uint64(indexBytes)
}

View File

@ -0,0 +1,160 @@
package daablocksstore
import (
"github.com/kaspanet/kaspad/domain/consensus/database"
"github.com/kaspanet/kaspad/domain/consensus/database/binaryserialization"
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/kaspanet/kaspad/domain/consensus/utils/lrucache"
)
var daaScoreBucket = database.MakeBucket([]byte("daa-score"))
var daaAddedBlocksBucket = database.MakeBucket([]byte("daa-added-blocks"))
// daaBlocksStore represents a store of DAABlocksStore
type daaBlocksStore struct {
daaScoreStaging map[externalapi.DomainHash]uint64
daaAddedBlocksStaging map[externalapi.DomainHash][]*externalapi.DomainHash
daaScoreToDelete map[externalapi.DomainHash]struct{}
daaAddedBlocksToDelete map[externalapi.DomainHash]struct{}
daaScoreLRUCache *lrucache.LRUCache
daaAddedBlocksLRUCache *lrucache.LRUCache
}
// New instantiates a new DAABlocksStore
func New(daaScoreCacheSize int, daaAddedBlocksCacheSize int, preallocate bool) model.DAABlocksStore {
return &daaBlocksStore{
daaScoreStaging: make(map[externalapi.DomainHash]uint64),
daaAddedBlocksStaging: make(map[externalapi.DomainHash][]*externalapi.DomainHash),
daaScoreLRUCache: lrucache.New(daaScoreCacheSize, preallocate),
daaAddedBlocksLRUCache: lrucache.New(daaAddedBlocksCacheSize, preallocate),
}
}
func (daas *daaBlocksStore) StageDAAScore(blockHash *externalapi.DomainHash, daaScore uint64) {
daas.daaScoreStaging[*blockHash] = daaScore
}
func (daas *daaBlocksStore) StageBlockDAAAddedBlocks(blockHash *externalapi.DomainHash,
addedBlocks []*externalapi.DomainHash) {
daas.daaAddedBlocksStaging[*blockHash] = externalapi.CloneHashes(addedBlocks)
}
func (daas *daaBlocksStore) IsAnythingStaged() bool {
return len(daas.daaScoreStaging) != 0 ||
len(daas.daaAddedBlocksStaging) != 0 ||
len(daas.daaScoreToDelete) != 0 ||
len(daas.daaAddedBlocksToDelete) != 0
}
func (daas *daaBlocksStore) Discard() {
daas.daaScoreStaging = make(map[externalapi.DomainHash]uint64)
daas.daaAddedBlocksStaging = make(map[externalapi.DomainHash][]*externalapi.DomainHash)
daas.daaScoreToDelete = make(map[externalapi.DomainHash]struct{})
daas.daaAddedBlocksToDelete = make(map[externalapi.DomainHash]struct{})
}
func (daas *daaBlocksStore) Commit(dbTx model.DBTransaction) error {
for hash, daaScore := range daas.daaScoreStaging {
daaScoreBytes := binaryserialization.SerializeUint64(daaScore)
err := dbTx.Put(daas.daaScoreHashAsKey(&hash), daaScoreBytes)
if err != nil {
return err
}
daas.daaScoreLRUCache.Add(&hash, daaScore)
}
for hash, addedBlocks := range daas.daaAddedBlocksStaging {
addedBlocksBytes := binaryserialization.SerializeHashes(addedBlocks)
err := dbTx.Put(daas.daaAddedBlocksHashAsKey(&hash), addedBlocksBytes)
if err != nil {
return err
}
daas.daaAddedBlocksLRUCache.Add(&hash, addedBlocks)
}
for hash := range daas.daaScoreToDelete {
err := dbTx.Delete(daas.daaScoreHashAsKey(&hash))
if err != nil {
return err
}
daas.daaScoreLRUCache.Remove(&hash)
}
for hash := range daas.daaAddedBlocksToDelete {
err := dbTx.Delete(daas.daaAddedBlocksHashAsKey(&hash))
if err != nil {
return err
}
daas.daaAddedBlocksLRUCache.Remove(&hash)
}
daas.Discard()
return nil
}
func (daas *daaBlocksStore) DAAScore(dbContext model.DBReader, blockHash *externalapi.DomainHash) (uint64, error) {
if daaScore, ok := daas.daaScoreStaging[*blockHash]; ok {
return daaScore, nil
}
if daaScore, ok := daas.daaScoreLRUCache.Get(blockHash); ok {
return daaScore.(uint64), nil
}
daaScoreBytes, err := dbContext.Get(daas.daaScoreHashAsKey(blockHash))
if err != nil {
return 0, err
}
daaScore, err := binaryserialization.DeserializeUint64(daaScoreBytes)
if err != nil {
return 0, err
}
daas.daaScoreLRUCache.Add(blockHash, daaScore)
return daaScore, nil
}
func (daas *daaBlocksStore) DAAAddedBlocks(dbContext model.DBReader, blockHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error) {
if addedBlocks, ok := daas.daaAddedBlocksStaging[*blockHash]; ok {
return externalapi.CloneHashes(addedBlocks), nil
}
if addedBlocks, ok := daas.daaAddedBlocksLRUCache.Get(blockHash); ok {
return externalapi.CloneHashes(addedBlocks.([]*externalapi.DomainHash)), nil
}
addedBlocksBytes, err := dbContext.Get(daas.daaAddedBlocksHashAsKey(blockHash))
if err != nil {
return nil, err
}
addedBlocks, err := binaryserialization.DeserializeHashes(addedBlocksBytes)
if err != nil {
return nil, err
}
daas.daaAddedBlocksLRUCache.Add(blockHash, addedBlocks)
return externalapi.CloneHashes(addedBlocks), nil
}
func (daas *daaBlocksStore) daaScoreHashAsKey(hash *externalapi.DomainHash) model.DBKey {
return daaScoreBucket.Key(hash.ByteSlice())
}
func (daas *daaBlocksStore) daaAddedBlocksHashAsKey(hash *externalapi.DomainHash) model.DBKey {
return daaAddedBlocksBucket.Key(hash.ByteSlice())
}
func (daas *daaBlocksStore) Delete(blockHash *externalapi.DomainHash) {
if _, ok := daas.daaScoreStaging[*blockHash]; ok {
delete(daas.daaScoreStaging, *blockHash)
} else {
daas.daaAddedBlocksToDelete[*blockHash] = struct{}{}
}
if _, ok := daas.daaAddedBlocksStaging[*blockHash]; ok {
delete(daas.daaAddedBlocksStaging, *blockHash)
} else {
daas.daaAddedBlocksToDelete[*blockHash] = struct{}{}
}
}

View File

@ -161,7 +161,11 @@ func (hscs *headersSelectedChainStore) GetIndexByHash(dbContext model.DBReader,
return 0, err
}
index := hscs.deserializeIndex(indexBytes)
index, err := hscs.deserializeIndex(indexBytes)
if err != nil {
return 0, err
}
hscs.cacheByHash.Add(blockHash, index)
return index, nil
}
@ -193,11 +197,11 @@ func (hscs *headersSelectedChainStore) GetHashByIndex(dbContext model.DBReader,
}
func (hscs *headersSelectedChainStore) serializeIndex(index uint64) []byte {
return binaryserialization.SerializeChainBlockIndex(index)
return binaryserialization.SerializeUint64(index)
}
func (hscs *headersSelectedChainStore) deserializeIndex(indexBytes []byte) uint64 {
return binaryserialization.DeserializeChainBlockIndex(indexBytes)
func (hscs *headersSelectedChainStore) deserializeIndex(indexBytes []byte) (uint64, error) {
return binaryserialization.DeserializeUint64(indexBytes)
}
func (hscs *headersSelectedChainStore) hashAsKey(hash *externalapi.DomainHash) model.DBKey {
@ -223,7 +227,11 @@ func (hscs *headersSelectedChainStore) highestChainBlockIndex(dbContext model.DB
return 0, false, err
}
index := hscs.deserializeIndex(indexBytes)
index, err := hscs.deserializeIndex(indexBytes)
if err != nil {
return 0, false, err
}
hscs.cacheHighestChainBlockIndex = index
return index, true, nil
}

View File

@ -1,6 +1,7 @@
package consensus
import (
daablocksstore "github.com/kaspanet/kaspad/domain/consensus/datastructures/daablocksstore"
"io/ioutil"
"os"
"sync"
@ -126,6 +127,7 @@ func (f *factory) NewConsensus(dagParams *dagconfig.Params, db infrastructuredat
headersSelectedTipStore := headersselectedtipstore.New()
finalityStore := finalitystore.New(200, preallocateCaches)
headersSelectedChainStore := headersselectedchainstore.New(pruningWindowSizeForCaches, preallocateCaches)
daaBlocksStore := daablocksstore.New(pruningWindowSizeForCaches, int(dagParams.FinalityDepth()), preallocateCaches)
// Processes
reachabilityManager := reachabilitymanager.New(
@ -172,6 +174,7 @@ func (f *factory) NewConsensus(dagParams *dagconfig.Params, db infrastructuredat
ghostdagManager,
ghostdagDataStore,
blockHeaderStore,
daaBlocksStore,
dagTopologyManager,
dagTraversalManager,
dagParams.PowMax,
@ -281,6 +284,7 @@ func (f *factory) NewConsensus(dagParams *dagconfig.Params, db infrastructuredat
blockStore,
blockHeaderStore,
utxoDiffStore,
daaBlocksStore,
isArchivalNode,
genesisHash,
dagParams.FinalityDepth(),
@ -345,7 +349,8 @@ func (f *factory) NewConsensus(dagParams *dagconfig.Params, db infrastructuredat
blockHeaderStore,
headersSelectedTipStore,
finalityStore,
headersSelectedChainStore)
headersSelectedChainStore,
daaBlocksStore)
c := &consensus{
lock: &sync.Mutex{},
@ -383,6 +388,7 @@ func (f *factory) NewConsensus(dagParams *dagconfig.Params, db infrastructuredat
utxoDiffStore: utxoDiffStore,
finalityStore: finalityStore,
headersSelectedChainStore: headersSelectedChainStore,
daaBlocksStore: daaBlocksStore,
}
genesisInfo, err := c.GetBlockInfo(genesisHash)

View File

@ -0,0 +1,14 @@
package model
import "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
// DAABlocksStore represents a store of ???
type DAABlocksStore interface {
Store
StageDAAScore(blockHash *externalapi.DomainHash, daaScore uint64)
StageBlockDAAAddedBlocks(blockHash *externalapi.DomainHash, addedBlocks []*externalapi.DomainHash)
IsAnythingStaged() bool
DAAAddedBlocks(dbContext DBReader, blockHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error)
DAAScore(dbContext DBReader, blockHash *externalapi.DomainHash) (uint64, error)
Delete(blockHash *externalapi.DomainHash)
}

View File

@ -66,6 +66,7 @@ type TestConsensus interface {
ReachabilityDataStore() model.ReachabilityDataStore
UTXODiffStore() model.UTXODiffStore
HeadersSelectedChainStore() model.HeadersSelectedChainStore
DAABlocksStore() model.DAABlocksStore
BlockBuilder() TestBlockBuilder
BlockProcessor() model.BlockProcessor

View File

@ -42,6 +42,7 @@ type blockProcessor struct {
headersSelectedTipStore model.HeaderSelectedTipStore
finalityStore model.FinalityStore
headersSelectedChainStore model.HeadersSelectedChainStore
daaBlocksStore model.DAABlocksStore
stores []model.Store
}
@ -78,6 +79,7 @@ func New(
headersSelectedTipStore model.HeaderSelectedTipStore,
finalityStore model.FinalityStore,
headersSelectedChainStore model.HeadersSelectedChainStore,
daaBlocksStore model.DAABlocksStore,
) model.BlockProcessor {
return &blockProcessor{
@ -111,6 +113,7 @@ func New(
headersSelectedTipStore: headersSelectedTipStore,
finalityStore: finalityStore,
headersSelectedChainStore: headersSelectedChainStore,
daaBlocksStore: daaBlocksStore,
stores: []model.Store{
consensusStateStore,
@ -128,6 +131,7 @@ func New(
headersSelectedTipStore,
finalityStore,
headersSelectedChainStore,
daaBlocksStore,
},
}
}

View File

@ -254,7 +254,7 @@ func TestValidateDifficulty(t *testing.T) {
factory := consensus.NewFactory()
mocDifficulty := &mocDifficultyManager{}
factory.SetTestDifficultyManager(func(model.DBReader, model.GHOSTDAGManager, model.GHOSTDAGDataStore,
model.BlockHeaderStore, model.DAGTopologyManager, model.DAGTraversalManager, *big.Int, int, bool, time.Duration,
model.BlockHeaderStore, model.DAABlocksStore, model.DAGTopologyManager, model.DAGTraversalManager, *big.Int, int, bool, time.Duration,
*externalapi.DomainHash) model.DifficultyManager {
return mocDifficulty
})

View File

@ -29,21 +29,23 @@ func (dm *difficultyManager) getDifficultyBlock(blockHash *externalapi.DomainHas
// blocks in the past of startindNode, the sorting is unspecified.
// If the number of blocks in the past of startingNode is less then windowSize,
// the window will be padded by genesis blocks to achieve a size of windowSize.
func (dm *difficultyManager) blockWindow(startingNode *externalapi.DomainHash, windowSize int) (blockWindow, error) {
func (dm *difficultyManager) blockWindow(startingNode *externalapi.DomainHash, windowSize int) (blockWindow,
[]*externalapi.DomainHash, error) {
window := make(blockWindow, 0, windowSize)
windowHashes, err := dm.dagTraversalManager.BlockWindow(startingNode, windowSize)
if err != nil {
return nil, err
return nil, nil, err
}
for _, hash := range windowHashes {
block, err := dm.getDifficultyBlock(hash)
if err != nil {
return nil, err
return nil, nil, err
}
window = append(window, block)
}
return window, nil
return window, windowHashes, nil
}
func (window blockWindow) minMaxTimestamps() (min, max int64, minIndex, maxIndex int) {

View File

@ -1,6 +1,7 @@
package difficultymanager
import (
"github.com/kaspanet/kaspad/infrastructure/logger"
"github.com/kaspanet/kaspad/util/math"
"math/big"
"time"
@ -18,6 +19,7 @@ type difficultyManager struct {
ghostdagManager model.GHOSTDAGManager
ghostdagStore model.GHOSTDAGDataStore
headerStore model.BlockHeaderStore
daaBlocksStore model.DAABlocksStore
dagTopologyManager model.DAGTopologyManager
dagTraversalManager model.DAGTraversalManager
genesisHash *externalapi.DomainHash
@ -32,6 +34,7 @@ func New(databaseContext model.DBReader,
ghostdagManager model.GHOSTDAGManager,
ghostdagStore model.GHOSTDAGDataStore,
headerStore model.BlockHeaderStore,
daaBlocksStore model.DAABlocksStore,
dagTopologyManager model.DAGTopologyManager,
dagTraversalManager model.DAGTraversalManager,
powMax *big.Int,
@ -44,6 +47,7 @@ func New(databaseContext model.DBReader,
ghostdagManager: ghostdagManager,
ghostdagStore: ghostdagStore,
headerStore: headerStore,
daaBlocksStore: daaBlocksStore,
dagTopologyManager: dagTopologyManager,
dagTraversalManager: dagTraversalManager,
powMax: powMax,
@ -65,14 +69,21 @@ func (dm *difficultyManager) genesisBits() (uint32, error) {
// RequiredDifficulty returns the difficulty required for some block
func (dm *difficultyManager) RequiredDifficulty(blockHash *externalapi.DomainHash) (uint32, error) {
if dm.disableDifficultyAdjustment {
return dm.genesisBits()
}
// Fetch window of dag.difficultyAdjustmentWindowSize + 1 so we can have dag.difficultyAdjustmentWindowSize block intervals
targetsWindow, err := dm.blockWindow(blockHash, dm.difficultyAdjustmentWindowSize+1)
targetsWindow, windowHashes, err := dm.blockWindow(blockHash, dm.difficultyAdjustmentWindowSize+1)
if err != nil {
return 0, err
}
err = dm.updateDaaScoreAndAddedBlocks(blockHash, windowHashes)
if err != nil {
return 0, err
}
if dm.disableDifficultyAdjustment {
return dm.genesisBits()
}
// We need at least 2 blocks to get a timestamp interval
// We could instead clamp the timestamp difference to `targetTimePerBlock`,
// but then everything will cancel out and we'll get the target from the last block, which will be the same as genesis.
@ -100,3 +111,56 @@ func (dm *difficultyManager) RequiredDifficulty(blockHash *externalapi.DomainHas
newTargetBits := difficulty.BigToCompact(newTarget)
return newTargetBits, nil
}
func (dm *difficultyManager) updateDaaScoreAndAddedBlocks(blockHash *externalapi.DomainHash,
windowHashes []*externalapi.DomainHash) error {
onEnd := logger.LogAndMeasureExecutionTime(log, "updateDaaScoreAndAddedBlocks")
defer onEnd()
daaScore, addedBlocks, err := dm.calculateDaaScoreAndAddedBlocks(blockHash, windowHashes)
if err != nil {
return err
}
dm.daaBlocksStore.StageDAAScore(blockHash, daaScore)
dm.daaBlocksStore.StageBlockDAAAddedBlocks(blockHash, addedBlocks)
return nil
}
func (dm *difficultyManager) calculateDaaScoreAndAddedBlocks(blockHash *externalapi.DomainHash,
windowHashes []*externalapi.DomainHash) (uint64, []*externalapi.DomainHash, error) {
if blockHash.Equal(dm.genesisHash) {
return 0, nil, nil
}
ghostdagData, err := dm.ghostdagStore.Get(dm.databaseContext, blockHash)
if err != nil {
return 0, nil, err
}
mergeSet := make(map[externalapi.DomainHash]struct{}, len(ghostdagData.MergeSet()))
for _, hash := range ghostdagData.MergeSet() {
mergeSet[*hash] = struct{}{}
}
// TODO: Consider optimizing by breaking the loop once you arrive to the
// window block with blue work higher than all non-added merge set blocks.
daaAddedBlocks := make([]*externalapi.DomainHash, 0, len(mergeSet))
for _, hash := range windowHashes {
if _, exists := mergeSet[*hash]; exists {
daaAddedBlocks = append(daaAddedBlocks, hash)
if len(daaAddedBlocks) == len(mergeSet) {
break
}
}
}
selectedParentDAAScore, err := dm.daaBlocksStore.DAAScore(dm.databaseContext, ghostdagData.SelectedParent())
if err != nil {
return 0, nil, err
}
return selectedParentDAAScore + uint64(len(daaAddedBlocks)), daaAddedBlocks, nil
}

View File

@ -228,6 +228,124 @@ func TestDifficulty(t *testing.T) {
})
}
func TestDAAScore(t *testing.T) {
testutils.ForAllNets(t, true, func(t *testing.T, params *dagconfig.Params) {
params.DifficultyAdjustmentWindowSize = 264
factory := consensus.NewFactory()
tc, teardown, err := factory.NewTestConsensus(params, false, "TestDifficulty")
if err != nil {
t.Fatalf("Error setting up consensus: %+v", err)
}
defer teardown(false)
// We create a small DAG in order to skip from block with blue score of 1 directly to 3
split1Hash, _, err := tc.AddBlock([]*externalapi.DomainHash{params.GenesisHash}, nil, nil)
if err != nil {
t.Fatalf("AddBlock: %+v", err)
}
block, _, err := tc.AddBlock([]*externalapi.DomainHash{params.GenesisHash}, nil, nil)
if err != nil {
t.Fatalf("AddBlock: %+v", err)
}
blockBlueScore3, _, err := tc.AddBlock([]*externalapi.DomainHash{split1Hash, block}, nil, nil)
if err != nil {
t.Fatalf("AddBlock: %+v", err)
}
tipHash := blockBlueScore3
blockBlueScore3DAAScore, err := tc.DAABlocksStore().DAAScore(tc.DatabaseContext(), tipHash)
if err != nil {
t.Fatalf("DAAScore: %+v", err)
}
blockBlueScore3ExpectedDAAScore := uint64(2)
if blockBlueScore3DAAScore != blockBlueScore3ExpectedDAAScore {
t.Fatalf("DAA score is expected to be %d but got %d", blockBlueScore3ExpectedDAAScore, blockBlueScore3ExpectedDAAScore)
}
tipDAAScore := blockBlueScore3ExpectedDAAScore
for i := uint64(0); i < 10; i++ {
tipHash, _, err = tc.AddBlock([]*externalapi.DomainHash{tipHash}, nil, nil)
if err != nil {
t.Fatalf("AddBlock: %+v", err)
}
tipDAAScore, err = tc.DAABlocksStore().DAAScore(tc.DatabaseContext(), tipHash)
if err != nil {
t.Fatalf("DAAScore: %+v", err)
}
expectedDAAScore := blockBlueScore3ExpectedDAAScore + i + 1
if tipDAAScore != expectedDAAScore {
t.Fatalf("DAA score is expected to be %d but got %d", expectedDAAScore, tipDAAScore)
}
}
split2Hash := tipHash
split2DAAScore := tipDAAScore
for i := uint64(0); i < uint64(params.DifficultyAdjustmentWindowSize)-1; i++ {
tipHash, _, err = tc.AddBlock([]*externalapi.DomainHash{tipHash}, nil, nil)
if err != nil {
t.Fatalf("AddBlock: %+v", err)
}
tipDAAScore, err = tc.DAABlocksStore().DAAScore(tc.DatabaseContext(), tipHash)
if err != nil {
t.Fatalf("DAAScore: %+v", err)
}
expectedDAAScore := split2DAAScore + i + 1
if tipDAAScore != expectedDAAScore {
t.Fatalf("DAA score is expected to be %d but got %d", expectedDAAScore, split2DAAScore)
}
}
// This block should have blue score of 2 so it shouldn't be added to the DAA window of a merging block
blockAboveSplit1, _, err := tc.AddBlock([]*externalapi.DomainHash{split1Hash}, nil, nil)
if err != nil {
t.Fatalf("AddBlock: %+v", err)
}
// This block is in the anticone of params.DifficultyAdjustmentWindowSize-1 blocks, so it must be part
// of the DAA window of a merging block
blockAboveSplit2, _, err := tc.AddBlock([]*externalapi.DomainHash{split2Hash}, nil, nil)
if err != nil {
t.Fatalf("AddBlock: %+v", err)
}
currentSelectedTipDAAScore := tipDAAScore
currentSelectedTip := tipHash
tipHash, _, err = tc.AddBlock([]*externalapi.DomainHash{blockAboveSplit1, blockAboveSplit2, tipHash}, nil, nil)
if err != nil {
t.Fatalf("AddBlock: %+v", err)
}
tipDAAScore, err = tc.DAABlocksStore().DAAScore(tc.DatabaseContext(), tipHash)
if err != nil {
t.Fatalf("DAAScore: %+v", err)
}
// The DAA score should be increased only by 2, because 1 of the 3 merged blocks
// is not in the DAA window
expectedDAAScore := currentSelectedTipDAAScore + 2
if tipDAAScore != expectedDAAScore {
t.Fatalf("DAA score is expected to be %d but got %d", expectedDAAScore, tipDAAScore)
}
tipDAAAddedBlocks, err := tc.DAABlocksStore().DAAAddedBlocks(tc.DatabaseContext(), tipHash)
if err != nil {
t.Fatalf("DAAScore: %+v", err)
}
// blockAboveSplit2 should be excluded from the DAA added blocks because it's not in the tip's
// DAA window.
expectedDAABlocks := []*externalapi.DomainHash{blockAboveSplit2, currentSelectedTip}
if !externalapi.HashesEqual(tipDAAAddedBlocks, expectedDAABlocks) {
t.Fatalf("DAA added blocks are expected to be %s but got %s", expectedDAABlocks, tipDAAAddedBlocks)
}
})
}
func compareBits(a uint32, b uint32) int {
aTarget := difficulty.CompactToBig(a)
bTarget := difficulty.CompactToBig(b)

View File

@ -0,0 +1,7 @@
package difficultymanager
import (
"github.com/kaspanet/kaspad/infrastructure/logger"
)
var log = logger.RegisterSubSystem("DAA")

View File

@ -28,6 +28,7 @@ type pruningManager struct {
blocksStore model.BlockStore
blockHeaderStore model.BlockHeaderStore
utxoDiffStore model.UTXODiffStore
daaBlocksStore model.DAABlocksStore
isArchivalNode bool
genesisHash *externalapi.DomainHash
@ -42,17 +43,18 @@ func New(
dagTraversalManager model.DAGTraversalManager,
dagTopologyManager model.DAGTopologyManager,
consensusStateManager model.ConsensusStateManager,
consensusStateStore model.ConsensusStateStore,
ghostdagDataStore model.GHOSTDAGDataStore,
pruningStore model.PruningStore,
blockStatusStore model.BlockStatusStore,
headerSelectedTipStore model.HeaderSelectedTipStore,
multiSetStore model.MultisetStore,
acceptanceDataStore model.AcceptanceDataStore,
blocksStore model.BlockStore,
blockHeaderStore model.BlockHeaderStore,
utxoDiffStore model.UTXODiffStore,
daaBlocksStore model.DAABlocksStore,
isArchivalNode bool,
genesisHash *externalapi.DomainHash,
@ -75,6 +77,7 @@ func New(
blockHeaderStore: blockHeaderStore,
utxoDiffStore: utxoDiffStore,
headerSelectedTipStore: headerSelectedTipStore,
daaBlocksStore: daaBlocksStore,
isArchivalNode: isArchivalNode,
genesisHash: genesisHash,
pruningDepth: pruningDepth,
@ -348,6 +351,7 @@ func (pm *pruningManager) deleteBlock(blockHash *externalapi.DomainHash) (alread
pm.acceptanceDataStore.Delete(blockHash)
pm.blocksStore.Delete(blockHash)
pm.utxoDiffStore.Delete(blockHash)
pm.daaBlocksStore.Delete(blockHash)
return false, nil
}

View File

@ -137,3 +137,7 @@ func (tc *testConsensus) FinalityStore() model.FinalityStore {
func (tc *testConsensus) HeadersSelectedChainStore() model.HeadersSelectedChainStore {
return tc.headersSelectedChainStore
}
func (tc *testConsensus) DAABlocksStore() model.DAABlocksStore {
return tc.daaBlocksStore
}