Use separate depth than finality depth for merge set calculations after HF (#2013)

* Use separate than finality depth for merge set calculations after HF

* Add comments and edit error messages

* Fix TestValidateTransactionInContextAndPopulateFee

* Don't disconnect from node if isViolatingBoundedMergeDepth

* Use new merge root for virtual pick parents; apply HF1 daa score split for validation only

* Use `blue work` heuristic to skip irrelevant relay blocks

* Minor

* Make sure virtual's merge depth root is a real block

* For ghostdag data we always use the non-trusted data

* Fix TestBoundedMergeDepth and in IBD use VirtualMergeDepthRoot instead of MergeDepthRoot

* Update HF1DAAScore

* Make sure merge root and finality are called + avoid calculating virtual root twice

* Update block version to 1 after HF

* Update to v0.12.0

Co-authored-by: msutton <mikisiton2@gmail.com>
This commit is contained in:
Ori Newman 2022-04-12 00:26:44 +03:00 committed by GitHub
parent ada559f007
commit 2b395e34b1
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
43 changed files with 1222 additions and 338 deletions

View File

@ -7,6 +7,7 @@ import (
peerpkg "github.com/kaspanet/kaspad/app/protocol/peer"
"github.com/kaspanet/kaspad/app/protocol/protocolerrors"
"github.com/kaspanet/kaspad/domain"
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/kaspanet/kaspad/domain/consensus/ruleerrors"
"github.com/kaspanet/kaspad/domain/consensus/utils/consensushashing"
@ -139,6 +140,26 @@ func (flow *handleRelayInvsFlow) start() error {
continue
}
// Test bounded merge depth to avoid requesting irrelevant data which cannot be merged under virtual
virtualMergeDepthRoot, err := flow.Domain().Consensus().VirtualMergeDepthRoot()
if err != nil {
return err
}
if !virtualMergeDepthRoot.Equal(model.VirtualGenesisBlockHash) {
mergeDepthRootHeader, err := flow.Domain().Consensus().GetBlockHeader(virtualMergeDepthRoot)
if err != nil {
return err
}
// Since `BlueWork` respects topology, this condition means that the relay
// block is not in the future of virtual's merge depth root, and thus cannot be merged unless
// other valid blocks Kosherize it, in which case it will be obtained once the merger is relayed
if block.Header.BlueWork().Cmp(mergeDepthRootHeader.BlueWork()) <= 0 {
log.Debugf("Block %s has lower blue work than virtual's merge root %s (%d <= %d), hence we are skipping it",
inv.Hash, virtualMergeDepthRoot, block.Header.BlueWork(), mergeDepthRootHeader.BlueWork())
continue
}
}
log.Debugf("Processing block %s", inv.Hash)
oldVirtualInfo, err := flow.Domain().Consensus().GetVirtualInfo()
if err != nil {

View File

@ -2,6 +2,10 @@ package libkaspawallet_test
import (
"fmt"
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/ruleerrors"
"github.com/kaspanet/kaspad/domain/consensus/utils/constants"
"github.com/pkg/errors"
"strings"
"testing"
@ -310,3 +314,239 @@ func TestP2PK(t *testing.T) {
})
})
}
func TestMaxSompi(t *testing.T) {
testutils.ForAllNets(t, true, func(t *testing.T, consensusConfig *consensus.Config) {
params := &consensusConfig.Params
cfg := *consensusConfig
cfg.BlockCoinbaseMaturity = 0
cfg.PreDeflationaryPhaseBaseSubsidy = 20e6 * constants.SompiPerKaspa
cfg.HF1DAAScore = cfg.GenesisBlock.Header.DAAScore() + 10
tc, teardown, err := consensus.NewFactory().NewTestConsensus(&cfg, "TestMaxSompi")
if err != nil {
t.Fatalf("Error setting up tc: %+v", err)
}
defer teardown(false)
const numKeys = 1
mnemonics := make([]string, numKeys)
publicKeys := make([]string, numKeys)
for i := 0; i < numKeys; i++ {
var err error
mnemonics[i], err = libkaspawallet.CreateMnemonic()
if err != nil {
t.Fatalf("CreateMnemonic: %+v", err)
}
publicKeys[i], err = libkaspawallet.MasterPublicKeyFromMnemonic(&cfg.Params, mnemonics[i], false)
if err != nil {
t.Fatalf("MasterPublicKeyFromMnemonic: %+v", err)
}
}
const minimumSignatures = 1
path := "m/1/2/3"
address, err := libkaspawallet.Address(params, publicKeys, minimumSignatures, path, false)
if err != nil {
t.Fatalf("Address: %+v", err)
}
scriptPublicKey, err := txscript.PayToAddrScript(address)
if err != nil {
t.Fatalf("PayToAddrScript: %+v", err)
}
coinbaseData := &externalapi.DomainCoinbaseData{
ScriptPublicKey: scriptPublicKey,
ExtraData: nil,
}
fundingBlock1Hash, _, err := tc.AddBlock([]*externalapi.DomainHash{cfg.GenesisHash}, coinbaseData, nil)
if err != nil {
t.Fatalf("AddBlock: %+v", err)
}
fundingBlock2Hash, _, err := tc.AddBlock([]*externalapi.DomainHash{fundingBlock1Hash}, coinbaseData, nil)
if err != nil {
t.Fatalf("AddBlock: %+v", err)
}
fundingBlock3Hash, _, err := tc.AddBlock([]*externalapi.DomainHash{fundingBlock2Hash}, coinbaseData, nil)
if err != nil {
t.Fatalf("AddBlock: %+v", err)
}
fundingBlock4Hash, _, err := tc.AddBlock([]*externalapi.DomainHash{fundingBlock3Hash}, coinbaseData, nil)
if err != nil {
t.Fatalf("AddBlock: %+v", err)
}
fundingBlock2, err := tc.GetBlock(fundingBlock2Hash)
if err != nil {
t.Fatalf("GetBlock: %+v", err)
}
fundingBlock3, err := tc.GetBlock(fundingBlock3Hash)
if err != nil {
t.Fatalf("GetBlock: %+v", err)
}
fundingBlock4, err := tc.GetBlock(fundingBlock4Hash)
if err != nil {
t.Fatalf("GetBlock: %+v", err)
}
block1Hash, _, err := tc.AddBlock([]*externalapi.DomainHash{fundingBlock4Hash}, nil, nil)
if err != nil {
t.Fatalf("AddBlock: %+v", err)
}
block1, err := tc.GetBlock(block1Hash)
if err != nil {
t.Fatalf("GetBlock: %+v", err)
}
txOut1 := fundingBlock2.Transactions[0].Outputs[0]
txOut2 := fundingBlock3.Transactions[0].Outputs[0]
txOut3 := fundingBlock4.Transactions[0].Outputs[0]
txOut4 := block1.Transactions[0].Outputs[0]
selectedUTXOsForTxWithLargeInputAmount := []*libkaspawallet.UTXO{
{
Outpoint: &externalapi.DomainOutpoint{
TransactionID: *consensushashing.TransactionID(fundingBlock2.Transactions[0]),
Index: 0,
},
UTXOEntry: utxo.NewUTXOEntry(txOut1.Value, txOut1.ScriptPublicKey, true, 0),
DerivationPath: path,
},
{
Outpoint: &externalapi.DomainOutpoint{
TransactionID: *consensushashing.TransactionID(fundingBlock3.Transactions[0]),
Index: 0,
},
UTXOEntry: utxo.NewUTXOEntry(txOut2.Value, txOut2.ScriptPublicKey, true, 0),
DerivationPath: path,
},
}
unsignedTxWithLargeInputAmount, err := libkaspawallet.CreateUnsignedTransaction(publicKeys, minimumSignatures,
[]*libkaspawallet.Payment{{
Address: address,
Amount: 10,
}}, selectedUTXOsForTxWithLargeInputAmount)
if err != nil {
t.Fatalf("CreateUnsignedTransactions: %+v", err)
}
signedTxWithLargeInputAmount, err := libkaspawallet.Sign(params, mnemonics, unsignedTxWithLargeInputAmount, false)
if err != nil {
t.Fatalf("Sign: %+v", err)
}
txWithLargeInputAmount, err := libkaspawallet.ExtractTransaction(signedTxWithLargeInputAmount, false)
if err != nil {
t.Fatalf("ExtractTransaction: %+v", err)
}
_, virtualChangeSet, err := tc.AddBlock([]*externalapi.DomainHash{block1Hash}, nil, []*externalapi.DomainTransaction{txWithLargeInputAmount})
if err != nil {
t.Fatalf("AddBlock: %+v", err)
}
addedUTXO1 := &externalapi.DomainOutpoint{
TransactionID: *consensushashing.TransactionID(txWithLargeInputAmount),
Index: 0,
}
if virtualChangeSet.VirtualUTXODiff.ToAdd().Contains(addedUTXO1) {
t.Fatalf("Transaction was accepted in the DAG")
}
selectedUTXOsForTxWithLargeInputAndOutputAmount := []*libkaspawallet.UTXO{
{
Outpoint: &externalapi.DomainOutpoint{
TransactionID: *consensushashing.TransactionID(fundingBlock4.Transactions[0]),
Index: 0,
},
UTXOEntry: utxo.NewUTXOEntry(txOut3.Value, txOut3.ScriptPublicKey, true, 0),
DerivationPath: path,
},
{
Outpoint: &externalapi.DomainOutpoint{
TransactionID: *consensushashing.TransactionID(block1.Transactions[0]),
Index: 0,
},
UTXOEntry: utxo.NewUTXOEntry(txOut4.Value, txOut4.ScriptPublicKey, true, 0),
DerivationPath: path,
},
}
unsignedTxWithLargeInputAndOutputAmount, err := libkaspawallet.CreateUnsignedTransaction(publicKeys, minimumSignatures,
[]*libkaspawallet.Payment{{
Address: address,
Amount: 22e6 * constants.SompiPerKaspa,
}}, selectedUTXOsForTxWithLargeInputAndOutputAmount)
if err != nil {
t.Fatalf("CreateUnsignedTransactions: %+v", err)
}
signedTxWithLargeInputAndOutputAmount, err := libkaspawallet.Sign(params, mnemonics, unsignedTxWithLargeInputAndOutputAmount, false)
if err != nil {
t.Fatalf("Sign: %+v", err)
}
txWithLargeInputAndOutputAmount, err := libkaspawallet.ExtractTransaction(signedTxWithLargeInputAndOutputAmount, false)
if err != nil {
t.Fatalf("ExtractTransaction: %+v", err)
}
_, _, err = tc.AddBlock([]*externalapi.DomainHash{block1Hash}, nil, []*externalapi.DomainTransaction{txWithLargeInputAndOutputAmount})
if !errors.Is(err, ruleerrors.ErrBadTxOutValue) {
t.Fatalf("AddBlock: %+v", err)
}
tip := block1Hash
for {
tip, _, err = tc.AddBlock([]*externalapi.DomainHash{tip}, nil, nil)
if err != nil {
t.Fatalf("AddBlock: %+v", err)
}
selectedTip, err := tc.GetVirtualSelectedParent()
if err != nil {
t.Fatalf("GetVirtualDAAScore: %+v", err)
}
daaScore, err := tc.DAABlocksStore().DAAScore(tc.DatabaseContext(), model.NewStagingArea(), selectedTip)
if err != nil {
t.Fatalf("DAAScore: %+v", err)
}
if daaScore >= cfg.HF1DAAScore {
break
}
}
tip, virtualChangeSet, err = tc.AddBlock([]*externalapi.DomainHash{tip}, nil, []*externalapi.DomainTransaction{txWithLargeInputAndOutputAmount})
if err != nil {
t.Fatalf("AddBlock: %+v", err)
}
addedUTXO2 := &externalapi.DomainOutpoint{
TransactionID: *consensushashing.TransactionID(txWithLargeInputAndOutputAmount),
Index: 0,
}
if !virtualChangeSet.VirtualUTXODiff.ToAdd().Contains(addedUTXO2) {
t.Fatalf("txWithLargeInputAndOutputAmount weren't accepted in the DAG")
}
_, virtualChangeSet, err = tc.AddBlock([]*externalapi.DomainHash{tip}, nil, []*externalapi.DomainTransaction{txWithLargeInputAmount})
if err != nil {
t.Fatalf("AddBlock: %+v", err)
}
if !virtualChangeSet.VirtualUTXODiff.ToAdd().Contains(addedUTXO1) {
t.Fatalf("txWithLargeInputAmount wasn't accepted in the DAG")
}
})
}

View File

@ -182,7 +182,12 @@ func (s *consensus) ValidateTransactionAndPopulateWithConsensusData(transaction
stagingArea := model.NewStagingArea()
err := s.transactionValidator.ValidateTransactionInIsolation(transaction)
daaScore, err := s.daaBlocksStore.DAAScore(s.databaseContext, stagingArea, model.VirtualBlockHash)
if err != nil {
return err
}
err = s.transactionValidator.ValidateTransactionInIsolation(transaction, daaScore)
if err != nil {
return err
}
@ -881,3 +886,11 @@ func (s *consensus) IsChainBlock(blockHash *externalapi.DomainHash) (bool, error
return s.dagTopologyManagers[0].IsInSelectedParentChainOf(stagingArea, blockHash, virtualGHOSTDAGData.SelectedParent())
}
func (s *consensus) VirtualMergeDepthRoot() (*externalapi.DomainHash, error) {
s.lock.Lock()
defer s.lock.Unlock()
stagingArea := model.NewStagingArea()
return s.mergeDepthManager.VirtualMergeDepthRoot(stagingArea)
}

View File

@ -0,0 +1,36 @@
package mergedepthrootstore
import (
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
)
type mergeDepthRootStagingShard struct {
store *mergeDepthRootStore
toAdd map[externalapi.DomainHash]*externalapi.DomainHash
}
func (mdrs *mergeDepthRootStore) stagingShard(stagingArea *model.StagingArea) *mergeDepthRootStagingShard {
return stagingArea.GetOrCreateShard(mdrs.shardID, func() model.StagingShard {
return &mergeDepthRootStagingShard{
store: mdrs,
toAdd: make(map[externalapi.DomainHash]*externalapi.DomainHash),
}
}).(*mergeDepthRootStagingShard)
}
func (mdrss *mergeDepthRootStagingShard) Commit(dbTx model.DBTransaction) error {
for hash, mergeDepthRoot := range mdrss.toAdd {
err := dbTx.Put(mdrss.store.hashAsKey(&hash), mergeDepthRoot.ByteSlice())
if err != nil {
return err
}
mdrss.store.cache.Add(&hash, mergeDepthRoot)
}
return nil
}
func (mdrss *mergeDepthRootStagingShard) isStaged() bool {
return len(mdrss.toAdd) == 0
}

View File

@ -0,0 +1,63 @@
package mergedepthrootstore
import (
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/kaspanet/kaspad/domain/consensus/utils/lrucache"
"github.com/kaspanet/kaspad/util/staging"
)
var bucketName = []byte("merge-depth-roots")
type mergeDepthRootStore struct {
shardID model.StagingShardID
cache *lrucache.LRUCache
bucket model.DBBucket
}
// New instantiates a new MergeDepthRootStore
func New(prefixBucket model.DBBucket, cacheSize int, preallocate bool) model.MergeDepthRootStore {
return &mergeDepthRootStore{
shardID: staging.GenerateShardingID(),
cache: lrucache.New(cacheSize, preallocate),
bucket: prefixBucket.Bucket(bucketName),
}
}
func (mdrs *mergeDepthRootStore) StageMergeDepthRoot(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash, root *externalapi.DomainHash) {
stagingShard := mdrs.stagingShard(stagingArea)
stagingShard.toAdd[*blockHash] = root
}
func (mdrs *mergeDepthRootStore) MergeDepthRoot(dbContext model.DBReader, stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) (*externalapi.DomainHash, error) {
stagingShard := mdrs.stagingShard(stagingArea)
if root, ok := stagingShard.toAdd[*blockHash]; ok {
return root, nil
}
if root, ok := mdrs.cache.Get(blockHash); ok {
return root.(*externalapi.DomainHash), nil
}
rootBytes, err := dbContext.Get(mdrs.hashAsKey(blockHash))
if err != nil {
return nil, err
}
root, err := externalapi.NewDomainHashFromByteSlice(rootBytes)
if err != nil {
return nil, err
}
mdrs.cache.Add(blockHash, root)
return root, nil
}
func (mdrs *mergeDepthRootStore) IsStaged(stagingArea *model.StagingArea) bool {
return mdrs.stagingShard(stagingArea).isStaged()
}
func (mdrs *mergeDepthRootStore) hashAsKey(hash *externalapi.DomainHash) model.DBKey {
return mdrs.bucket.Key(hash.ByteSlice())
}

View File

@ -3,6 +3,7 @@ package consensus
import (
"github.com/kaspanet/kaspad/domain/consensus/datastructures/blockwindowheapslicestore"
"github.com/kaspanet/kaspad/domain/consensus/datastructures/daawindowstore"
"github.com/kaspanet/kaspad/domain/consensus/datastructures/mergedepthrootstore"
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/processes/blockparentbuilder"
parentssanager "github.com/kaspanet/kaspad/domain/consensus/processes/parentsmanager"
@ -127,6 +128,7 @@ func (f *factory) NewConsensus(config *Config, db infrastructuredatabase.Databas
pruningWindowSizePlusFinalityDepthForCache := int(config.PruningDepth() + config.FinalityDepth())
// Data Structures
mergeDepthRootStore := mergedepthrootstore.New(prefixBucket, 200, preallocateCaches)
daaWindowStore := daawindowstore.New(prefixBucket, 10_000, preallocateCaches)
acceptanceDataStore := acceptancedatastore.New(prefixBucket, 200, preallocateCaches)
blockStore, err := blockstore.New(dbManager, prefixBucket, 200, preallocateCaches)
@ -215,7 +217,8 @@ func (f *factory) NewConsensus(config *Config, db infrastructuredatabase.Databas
pastMedianTimeManager,
ghostdagDataStore,
daaBlocksStore,
txMassCalculator)
txMassCalculator,
config.HF1DAAScore)
difficultyManager := f.difficultyConstructor(
dbManager,
ghostdagManager,
@ -261,7 +264,14 @@ func (f *factory) NewConsensus(config *Config, db infrastructuredatabase.Databas
dagTopologyManager,
dagTraversalManager,
finalityManager,
ghostdagDataStore)
genesisHash,
config.MergeDepth,
config.HF1DAAScore,
ghostdagDataStore,
mergeDepthRootStore,
daaBlocksStore,
pruningStore,
finalityStore)
consensusStateManager, err := consensusstatemanager.New(
dbManager,
config.MaxBlockParents,
@ -334,7 +344,7 @@ func (f *factory) NewConsensus(config *Config, db infrastructuredatabase.Databas
config.MaxBlockParents,
config.TimestampDeviationTolerance,
config.TargetTimePerBlock,
config.IgnoreHeaderMass,
config.HF1DAAScore,
config.MaxBlockLevel,
dbManager,
@ -383,6 +393,7 @@ func (f *factory) NewConsensus(config *Config, db infrastructuredatabase.Databas
blockBuilder := blockbuilder.New(
dbManager,
genesisHash,
config.HF1DAAScore,
difficultyManager,
pastMedianTimeManager,

View File

@ -3,8 +3,6 @@ package consensus_test
import (
"fmt"
"github.com/kaspanet/kaspad/domain/consensus"
"testing"
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/kaspanet/kaspad/domain/consensus/model/testapi"
@ -12,6 +10,9 @@ import (
"github.com/kaspanet/kaspad/domain/consensus/utils/consensushashing"
"github.com/kaspanet/kaspad/domain/consensus/utils/testutils"
"github.com/pkg/errors"
"math"
"math/rand"
"testing"
)
func TestFinality(t *testing.T) {
@ -179,15 +180,21 @@ func TestFinality(t *testing.T) {
func TestBoundedMergeDepth(t *testing.T) {
testutils.ForAllNets(t, true, func(t *testing.T, consensusConfig *consensus.Config) {
rd := rand.New(rand.NewSource(0))
// Set finalityInterval to 50 blocks, so that test runs quickly
consensusConfig.K = 5
consensusConfig.FinalityDuration = 7 * consensusConfig.TargetTimePerBlock
finalityInterval := int(consensusConfig.FinalityDepth())
consensusConfig.MergeDepth = 7
consensusConfig.FinalityDuration = 20 * consensusConfig.TargetTimePerBlock
consensusConfig.HF1DAAScore = consensusConfig.GenesisBlock.Header.DAAScore() + 200
if int(consensusConfig.K) >= finalityInterval {
if uint64(consensusConfig.K) >= consensusConfig.FinalityDepth() {
t.Fatal("K must be smaller than finality duration for this test to run")
}
if uint64(consensusConfig.K) >= consensusConfig.MergeDepth {
t.Fatal("K must be smaller than merge depth for this test to run")
}
checkViolatingMergeDepth := func(consensus testapi.TestConsensus, parents []*externalapi.DomainHash) (*externalapi.DomainBlock, bool) {
block, _, err := consensus.BuildBlockWithParents(parents, nil, nil)
if err != nil {
@ -217,7 +224,7 @@ func TestBoundedMergeDepth(t *testing.T) {
buildAndInsertBlock := func(consensus testapi.TestConsensus, parentHashes []*externalapi.DomainHash) *externalapi.DomainBlock {
block, _, err := consensus.BuildBlockWithParents(parentHashes, nil, nil)
if err != nil {
t.Fatalf("TestBoundedMergeDepth: Failed building block: %v", err)
t.Fatalf("TestBoundedMergeDepth: Failed building block: %+v", err)
}
_, err = consensus.ValidateAndInsertBlock(block, true)
if err != nil {
@ -236,194 +243,296 @@ func TestBoundedMergeDepth(t *testing.T) {
return blockInfo.BlockStatus
}
factory := consensus.NewFactory()
consensusBuild, teardownFunc1, err := factory.NewTestConsensus(consensusConfig, "TestBoundedMergeTestBuild")
if err != nil {
t.Fatalf("TestBoundedMergeDepth: Error setting up consensus: %+v", err)
syncConsensuses := func(tcSyncer, tcSyncee testapi.TestConsensus) {
syncerVirtualSelectedParent, err := tcSyncer.GetVirtualSelectedParent()
if err != nil {
t.Fatalf("GetVirtualSelectedParent: %+v", err)
}
missingHeaderHashes, _, err := tcSyncer.GetHashesBetween(consensusConfig.GenesisHash, syncerVirtualSelectedParent, math.MaxUint64)
if err != nil {
t.Fatalf("GetHashesBetween: %+v", err)
}
for i, blocksHash := range missingHeaderHashes {
blockInfo, err := tcSyncee.GetBlockInfo(blocksHash)
if err != nil {
t.Fatalf("GetBlockInfo: %+v", err)
}
if blockInfo.Exists {
continue
}
block, err := tcSyncer.GetBlock(blocksHash)
if err != nil {
t.Fatalf("GetBlockHeader: %+v", err)
}
_, err = tcSyncee.ValidateAndInsertBlock(block, true)
if err != nil {
t.Fatalf("ValidateAndInsertBlock %d: %+v", i, err)
}
}
synceeVirtualSelectedParent, err := tcSyncee.GetVirtualSelectedParent()
if err != nil {
t.Fatalf("Tips: %+v", err)
}
if !syncerVirtualSelectedParent.Equal(synceeVirtualSelectedParent) {
t.Fatalf("Syncee's selected tip is %s while syncer's is %s", synceeVirtualSelectedParent, syncerVirtualSelectedParent)
}
}
factory := consensus.NewFactory()
consensusReal, teardownFunc2, err := factory.NewTestConsensus(consensusConfig, "TestBoundedMergeTestReal")
if err != nil {
t.Fatalf("TestBoundedMergeDepth: Error setting up consensus: %+v", err)
}
defer teardownFunc2(false)
// Create a block on top on genesis
block1 := buildAndInsertBlock(consensusBuild, []*externalapi.DomainHash{consensusConfig.GenesisHash})
test := func(depth uint64, root *externalapi.DomainHash, checkVirtual, isRealDepth bool) {
consensusBuild, teardownFunc1, err := factory.NewTestConsensus(consensusConfig, "TestBoundedMergeTestBuild")
if err != nil {
t.Fatalf("TestBoundedMergeDepth: Error setting up consensus: %+v", err)
}
defer teardownFunc1(false)
consensusBuild.BlockBuilder().SetNonceCounter(rd.Uint64())
// Create a chain
selectedChain := make([]*externalapi.DomainBlock, 0, finalityInterval+1)
parent := consensushashing.BlockHash(block1)
// Make sure this is always bigger than `blocksChain2` so it will stay the selected chain
for i := 0; i < finalityInterval+2; i++ {
block := buildAndInsertBlock(consensusBuild, []*externalapi.DomainHash{parent})
selectedChain = append(selectedChain, block)
parent = consensushashing.BlockHash(block)
}
syncConsensuses(consensusReal, consensusBuild)
// Create a block on top on genesis
block1 := buildAndInsertBlock(consensusBuild, []*externalapi.DomainHash{root})
// Create another chain
blocksChain2 := make([]*externalapi.DomainBlock, 0, finalityInterval+1)
parent = consensushashing.BlockHash(block1)
for i := 0; i < finalityInterval+1; i++ {
block := buildAndInsertBlock(consensusBuild, []*externalapi.DomainHash{parent})
blocksChain2 = append(blocksChain2, block)
parent = consensushashing.BlockHash(block)
}
// Create a chain
selectedChain := make([]*externalapi.DomainBlock, 0, depth+1)
parent := consensushashing.BlockHash(block1)
// Make sure this is always bigger than `blocksChain2` so it will stay the selected chain
for i := uint64(0); i < depth+2; i++ {
block := buildAndInsertBlock(consensusBuild, []*externalapi.DomainHash{parent})
selectedChain = append(selectedChain, block)
parent = consensushashing.BlockHash(block)
}
// Teardown and assign nil to make sure we use the right DAG from here on.
teardownFunc1(false)
consensusBuild = nil
// Create another chain
blocksChain2 := make([]*externalapi.DomainBlock, 0, depth+1)
parent = consensushashing.BlockHash(block1)
for i := uint64(0); i < depth+1; i++ {
block := buildAndInsertBlock(consensusBuild, []*externalapi.DomainHash{parent})
blocksChain2 = append(blocksChain2, block)
parent = consensushashing.BlockHash(block)
}
// Now test against the real DAG
// submit block1
processBlock(consensusReal, block1, "block1")
// Now test against the real DAG
// submit block1
processBlock(consensusReal, block1, "block1")
// submit chain1
for i, block := range selectedChain {
processBlock(consensusReal, block, fmt.Sprintf("selectedChain block No %d", i))
}
// submit chain1
for i, block := range selectedChain {
processBlock(consensusReal, block, fmt.Sprintf("selectedChain block No %d", i))
}
// submit chain2
for i, block := range blocksChain2 {
processBlock(consensusReal, block, fmt.Sprintf("blocksChain2 block No %d", i))
}
// submit chain2
for i, block := range blocksChain2 {
processBlock(consensusReal, block, fmt.Sprintf("blocksChain2 block No %d", i))
}
// submit a block pointing at tip(chain1) and on first block in chain2 directly
mergeDepthViolatingBlockBottom, isViolatingMergeDepth := checkViolatingMergeDepth(consensusReal, []*externalapi.DomainHash{consensushashing.BlockHash(blocksChain2[0]), consensushashing.BlockHash(selectedChain[len(selectedChain)-1])})
if !isViolatingMergeDepth {
t.Fatalf("TestBoundedMergeDepth: Expected mergeDepthViolatingBlockBottom to violate merge depth")
}
// submit a block pointing at tip(chain1) and on first block in chain2 directly
mergeDepthViolatingBlockBottom, isViolatingMergeDepth := checkViolatingMergeDepth(consensusReal, []*externalapi.DomainHash{consensushashing.BlockHash(blocksChain2[0]), consensushashing.BlockHash(selectedChain[len(selectedChain)-1])})
if isViolatingMergeDepth != isRealDepth {
t.Fatalf("TestBoundedMergeDepth: Expects isViolatingMergeDepth to be %t", isRealDepth)
}
// submit a block pointing at tip(chain1) and tip(chain2) should also obviously violate merge depth (this points at first block in chain2 indirectly)
mergeDepthViolatingTop, isViolatingMergeDepth := checkViolatingMergeDepth(consensusReal, []*externalapi.DomainHash{consensushashing.BlockHash(blocksChain2[len(blocksChain2)-1]), consensushashing.BlockHash(selectedChain[len(selectedChain)-1])})
if !isViolatingMergeDepth {
t.Fatalf("TestBoundedMergeDepth: Expected mergeDepthViolatingTop to violate merge depth")
}
// submit a block pointing at tip(chain1) and tip(chain2) should also obviously violate merge depth (this points at first block in chain2 indirectly)
mergeDepthViolatingTop, isViolatingMergeDepth := checkViolatingMergeDepth(consensusReal, []*externalapi.DomainHash{consensushashing.BlockHash(blocksChain2[len(blocksChain2)-1]), consensushashing.BlockHash(selectedChain[len(selectedChain)-1])})
if isViolatingMergeDepth != isRealDepth {
t.Fatalf("TestBoundedMergeDepth: Expects isViolatingMergeDepth to be %t", isRealDepth)
}
// the location of the parents in the slices need to be both `-X` so the `selectedChain` one will have higher blueScore (it's a chain longer by 1)
kosherizingBlock, isViolatingMergeDepth := checkViolatingMergeDepth(consensusReal, []*externalapi.DomainHash{consensushashing.BlockHash(blocksChain2[len(blocksChain2)-3]), consensushashing.BlockHash(selectedChain[len(selectedChain)-3])})
kosherizingBlockHash := consensushashing.BlockHash(kosherizingBlock)
if isViolatingMergeDepth {
t.Fatalf("TestBoundedMergeDepth: Expected blueKosherizingBlock to not violate merge depth")
}
// the location of the parents in the slices need to be both `-X` so the `selectedChain` one will have higher blueScore (it's a chain longer by 1)
kosherizingBlock, isViolatingMergeDepth := checkViolatingMergeDepth(consensusReal, []*externalapi.DomainHash{consensushashing.BlockHash(blocksChain2[len(blocksChain2)-3]), consensushashing.BlockHash(selectedChain[len(selectedChain)-3])})
kosherizingBlockHash := consensushashing.BlockHash(kosherizingBlock)
if isViolatingMergeDepth {
t.Fatalf("TestBoundedMergeDepth: Expected blueKosherizingBlock to not violate merge depth")
}
stagingArea := model.NewStagingArea()
virtualGhotDagData, err := consensusReal.GHOSTDAGDataStore().Get(consensusReal.DatabaseContext(),
stagingArea, model.VirtualBlockHash, false)
if err != nil {
t.Fatalf("TestBoundedMergeDepth: Failed getting the ghostdag data of the virtual: %v", err)
}
// Make sure it's actually blue
found := false
for _, blue := range virtualGhotDagData.MergeSetBlues() {
if blue.Equal(kosherizingBlockHash) {
found = true
break
if checkVirtual {
stagingArea := model.NewStagingArea()
virtualGhotDagData, err := consensusReal.GHOSTDAGDataStore().Get(consensusReal.DatabaseContext(),
stagingArea, model.VirtualBlockHash, false)
if err != nil {
t.Fatalf("TestBoundedMergeDepth: Failed getting the ghostdag data of the virtual: %v", err)
}
// Make sure it's actually blue
found := false
for _, blue := range virtualGhotDagData.MergeSetBlues() {
if blue.Equal(kosherizingBlockHash) {
found = true
break
}
}
if !found {
t.Fatalf("TestBoundedMergeDepth: Expected kosherizingBlock to be blue by the virtual")
}
}
pointAtBlueKosherizing, isViolatingMergeDepth := checkViolatingMergeDepth(consensusReal, []*externalapi.DomainHash{kosherizingBlockHash, consensushashing.BlockHash(selectedChain[len(selectedChain)-1])})
if isViolatingMergeDepth {
t.Fatalf("TestBoundedMergeDepth: Expected selectedTip to not violate merge depth")
}
if checkVirtual {
virtualSelectedParent, err := consensusReal.GetVirtualSelectedParent()
if err != nil {
t.Fatalf("TestBoundedMergeDepth: Failed getting the virtual selected parent %v", err)
}
if !virtualSelectedParent.Equal(consensushashing.BlockHash(pointAtBlueKosherizing)) {
t.Fatalf("TestBoundedMergeDepth: Expected %s to be the selectedTip but found %s instead", consensushashing.BlockHash(pointAtBlueKosherizing), virtualSelectedParent)
}
}
// Now let's make the kosherizing block red and try to merge again
tip := consensushashing.BlockHash(selectedChain[len(selectedChain)-1])
// we use k-1 because `kosherizingBlock` points at tip-2, so 2+k-1 = k+1 anticone.
for i := 0; i < int(consensusConfig.K)-1; i++ {
block := buildAndInsertBlock(consensusReal, []*externalapi.DomainHash{tip})
tip = consensushashing.BlockHash(block)
}
if checkVirtual {
virtualSelectedParent, err := consensusReal.GetVirtualSelectedParent()
if err != nil {
t.Fatalf("TestBoundedMergeDepth: Failed getting the virtual selected parent %v", err)
}
if !virtualSelectedParent.Equal(tip) {
t.Fatalf("TestBoundedMergeDepth: Expected %s to be the selectedTip but found %s instead", tip, virtualSelectedParent)
}
virtualGhotDagData, err := consensusReal.GHOSTDAGDataStore().Get(
consensusReal.DatabaseContext(), model.NewStagingArea(), model.VirtualBlockHash, false)
if err != nil {
t.Fatalf("TestBoundedMergeDepth: Failed getting the ghostdag data of the virtual: %v", err)
}
// Make sure it's actually blue
found := false
for _, blue := range virtualGhotDagData.MergeSetBlues() {
if blue.Equal(kosherizingBlockHash) {
found = true
break
}
}
if found {
t.Fatalf("expected kosherizingBlock to be red by the virtual")
}
}
pointAtRedKosherizing, isViolatingMergeDepth := checkViolatingMergeDepth(consensusReal, []*externalapi.DomainHash{kosherizingBlockHash, tip})
if isViolatingMergeDepth != isRealDepth {
t.Fatalf("TestBoundedMergeDepth: Expects isViolatingMergeDepth to be %t", isRealDepth)
}
// Now `pointAtBlueKosherizing` itself is actually still blue, so we can still point at that even though we can't point at kosherizing directly anymore
transitiveBlueKosherizing, isViolatingMergeDepth :=
checkViolatingMergeDepth(consensusReal, []*externalapi.DomainHash{consensushashing.BlockHash(pointAtBlueKosherizing), tip})
if isViolatingMergeDepth {
t.Fatalf("TestBoundedMergeDepth: Expected transitiveBlueKosherizing to not violate merge depth")
}
if checkVirtual {
virtualSelectedParent, err := consensusReal.GetVirtualSelectedParent()
if err != nil {
t.Fatalf("TestBoundedMergeDepth: Failed getting the virtual selected parent %v", err)
}
if !virtualSelectedParent.Equal(consensushashing.BlockHash(transitiveBlueKosherizing)) {
t.Fatalf("TestBoundedMergeDepth: Expected %s to be the selectedTip but found %s instead", consensushashing.BlockHash(transitiveBlueKosherizing), virtualSelectedParent)
}
// Lets validate the status of all the interesting blocks
if getStatus(consensusReal, pointAtBlueKosherizing) != externalapi.StatusUTXOValid {
t.Fatalf("TestBoundedMergeDepth: pointAtBlueKosherizing expected status '%s' but got '%s'", externalapi.StatusUTXOValid, getStatus(consensusReal, pointAtBlueKosherizing))
}
if getStatus(consensusReal, pointAtRedKosherizing) != externalapi.StatusInvalid {
t.Fatalf("TestBoundedMergeDepth: pointAtRedKosherizing expected status '%s' but got '%s'", externalapi.StatusInvalid, getStatus(consensusReal, pointAtRedKosherizing))
}
if getStatus(consensusReal, transitiveBlueKosherizing) != externalapi.StatusUTXOValid {
t.Fatalf("TestBoundedMergeDepth: transitiveBlueKosherizing expected status '%s' but got '%s'", externalapi.StatusUTXOValid, getStatus(consensusReal, transitiveBlueKosherizing))
}
if getStatus(consensusReal, mergeDepthViolatingBlockBottom) != externalapi.StatusInvalid {
t.Fatalf("TestBoundedMergeDepth: mergeDepthViolatingBlockBottom expected status '%s' but got '%s'", externalapi.StatusInvalid, getStatus(consensusReal, mergeDepthViolatingBlockBottom))
}
if getStatus(consensusReal, mergeDepthViolatingTop) != externalapi.StatusInvalid {
t.Fatalf("TestBoundedMergeDepth: mergeDepthViolatingTop expected status '%s' but got '%s'", externalapi.StatusInvalid, getStatus(consensusReal, mergeDepthViolatingTop))
}
if getStatus(consensusReal, kosherizingBlock) != externalapi.StatusUTXOPendingVerification {
t.Fatalf("kosherizingBlock expected status '%s' but got '%s'", externalapi.StatusUTXOPendingVerification, getStatus(consensusReal, kosherizingBlock))
}
for i, b := range blocksChain2 {
if getStatus(consensusReal, b) != externalapi.StatusUTXOPendingVerification {
t.Fatalf("blocksChain2[%d] expected status '%s' but got '%s'", i, externalapi.StatusUTXOPendingVerification, getStatus(consensusReal, b))
}
}
for i, b := range selectedChain {
if getStatus(consensusReal, b) != externalapi.StatusUTXOValid {
t.Fatalf("selectedChain[%d] expected status '%s' but got '%s'", i, externalapi.StatusUTXOValid, getStatus(consensusReal, b))
}
}
}
}
if !found {
t.Fatalf("TestBoundedMergeDepth: Expected kosherizingBlock to be blue by the virtual")
}
pointAtBlueKosherizing, isViolatingMergeDepth := checkViolatingMergeDepth(consensusReal, []*externalapi.DomainHash{kosherizingBlockHash, consensushashing.BlockHash(selectedChain[len(selectedChain)-1])})
if isViolatingMergeDepth {
t.Fatalf("TestBoundedMergeDepth: Expected selectedTip to not violate merge depth")
}
virtualSelectedParent, err := consensusReal.GetVirtualSelectedParent()
test(consensusConfig.FinalityDepth(), consensusConfig.GenesisHash, true, true)
virtualDAAScore, err := consensusReal.GetVirtualDAAScore()
if err != nil {
t.Fatalf("TestBoundedMergeDepth: Failed getting the virtual selected parent %v", err)
t.Fatalf("GetVirtualDAAScore: %+v", err)
}
if !virtualSelectedParent.Equal(consensushashing.BlockHash(pointAtBlueKosherizing)) {
t.Fatalf("TestBoundedMergeDepth: Expected %s to be the selectedTip but found %s instead", consensushashing.BlockHash(pointAtBlueKosherizing), virtualSelectedParent)
if virtualDAAScore > consensusConfig.HF1DAAScore {
t.Fatalf("Hard fork is already activated")
}
// Now let's make the kosherizing block red and try to merge again
tip := consensushashing.BlockHash(selectedChain[len(selectedChain)-1])
// we use k-1 because `kosherizingBlock` points at tip-2, so 2+k-1 = k+1 anticone.
for i := 0; i < int(consensusConfig.K)-1; i++ {
block := buildAndInsertBlock(consensusReal, []*externalapi.DomainHash{tip})
tip = consensushashing.BlockHash(block)
}
virtualSelectedParent, err = consensusReal.GetVirtualSelectedParent()
tipBeforeHFActivated, err := consensusReal.GetVirtualSelectedParent()
if err != nil {
t.Fatalf("TestBoundedMergeDepth: Failed getting the virtual selected parent %v", err)
t.Fatalf("GetVirtualSelectedParent: %+v", err)
}
if !virtualSelectedParent.Equal(tip) {
t.Fatalf("TestBoundedMergeDepth: Expected %s to be the selectedTip but found %s instead", tip, virtualSelectedParent)
}
tip := tipBeforeHFActivated
for {
tip, _, err = consensusReal.AddBlock([]*externalapi.DomainHash{tip}, nil, nil)
if err != nil {
t.Fatalf("Failed adding block: %+v", err)
}
virtualGhotDagData, err = consensusReal.GHOSTDAGDataStore().Get(
consensusReal.DatabaseContext(), stagingArea, model.VirtualBlockHash, false)
if err != nil {
t.Fatalf("TestBoundedMergeDepth: Failed getting the ghostdag data of the virtual: %v", err)
}
// Make sure it's actually blue
found = false
for _, blue := range virtualGhotDagData.MergeSetBlues() {
if blue.Equal(kosherizingBlockHash) {
found = true
break
daaScore, err := consensusReal.DAABlocksStore().DAAScore(consensusReal.DatabaseContext(), model.NewStagingArea(), tip)
if err != nil {
t.Fatalf("Failed adding block: %+v", err)
}
// We check what happens when on the transition between rules: if we merge a chain that
// started before the HF was activated, but the HF was already activated for the merging
// block, the HF rules should apply.
// The checked block in `test` is going to have a DAA score of `tip.DAAScore + depth + 5`,
// so this is why we started to test from this depth.
if daaScore == consensusConfig.HF1DAAScore-consensusConfig.MergeDepth-5 {
test(consensusConfig.MergeDepth, tip, true, true)
}
if daaScore > consensusConfig.HF1DAAScore {
virtualSelectedParent, err := consensusReal.GetVirtualSelectedParent()
if err != nil {
t.Fatalf("GetVirtualSelectedParent: %+v", err)
}
if virtualSelectedParent.Equal(tip) {
break
}
}
}
if found {
t.Fatalf("expected kosherizingBlock to be red by the virtual")
}
pointAtRedKosherizing, isViolatingMergeDepth := checkViolatingMergeDepth(consensusReal, []*externalapi.DomainHash{kosherizingBlockHash, tip})
if !isViolatingMergeDepth {
t.Fatalf("TestBoundedMergeDepth: Expected selectedTipRedKosherize to violate merge depth")
}
// Now `pointAtBlueKosherizing` itself is actually still blue, so we can still point at that even though we can't point at kosherizing directly anymore
transitiveBlueKosherizing, isViolatingMergeDepth :=
checkViolatingMergeDepth(consensusReal, []*externalapi.DomainHash{consensushashing.BlockHash(pointAtBlueKosherizing), tip})
if isViolatingMergeDepth {
t.Fatalf("TestBoundedMergeDepth: Expected transitiveBlueKosherizing to not violate merge depth")
}
virtualSelectedParent, err = consensusReal.GetVirtualSelectedParent()
if err != nil {
t.Fatalf("TestBoundedMergeDepth: Failed getting the virtual selected parent %v", err)
}
if !virtualSelectedParent.Equal(consensushashing.BlockHash(transitiveBlueKosherizing)) {
t.Fatalf("TestBoundedMergeDepth: Expected %s to be the selectedTip but found %s instead", consensushashing.BlockHash(transitiveBlueKosherizing), virtualSelectedParent)
}
// Lets validate the status of all the interesting blocks
if getStatus(consensusReal, pointAtBlueKosherizing) != externalapi.StatusUTXOValid {
t.Fatalf("TestBoundedMergeDepth: pointAtBlueKosherizing expected status '%s' but got '%s'", externalapi.StatusUTXOValid, getStatus(consensusReal, pointAtBlueKosherizing))
}
if getStatus(consensusReal, pointAtRedKosherizing) != externalapi.StatusInvalid {
t.Fatalf("TestBoundedMergeDepth: pointAtRedKosherizing expected status '%s' but got '%s'", externalapi.StatusInvalid, getStatus(consensusReal, pointAtRedKosherizing))
}
if getStatus(consensusReal, transitiveBlueKosherizing) != externalapi.StatusUTXOValid {
t.Fatalf("TestBoundedMergeDepth: transitiveBlueKosherizing expected status '%s' but got '%s'", externalapi.StatusUTXOValid, getStatus(consensusReal, transitiveBlueKosherizing))
}
if getStatus(consensusReal, mergeDepthViolatingBlockBottom) != externalapi.StatusInvalid {
t.Fatalf("TestBoundedMergeDepth: mergeDepthViolatingBlockBottom expected status '%s' but got '%s'", externalapi.StatusInvalid, getStatus(consensusReal, mergeDepthViolatingBlockBottom))
}
if getStatus(consensusReal, mergeDepthViolatingTop) != externalapi.StatusInvalid {
t.Fatalf("TestBoundedMergeDepth: mergeDepthViolatingTop expected status '%s' but got '%s'", externalapi.StatusInvalid, getStatus(consensusReal, mergeDepthViolatingTop))
}
if getStatus(consensusReal, kosherizingBlock) != externalapi.StatusUTXOPendingVerification {
t.Fatalf("kosherizingBlock expected status '%s' but got '%s'", externalapi.StatusUTXOPendingVerification, getStatus(consensusReal, kosherizingBlock))
}
for i, b := range blocksChain2 {
if getStatus(consensusReal, b) != externalapi.StatusUTXOPendingVerification {
t.Fatalf("blocksChain2[%d] expected status '%s' but got '%s'", i, externalapi.StatusUTXOPendingVerification, getStatus(consensusReal, b))
}
}
for i, b := range selectedChain {
if getStatus(consensusReal, b) != externalapi.StatusUTXOValid {
t.Fatalf("selectedChain[%d] expected status '%s' but got '%s'", i, externalapi.StatusUTXOValid, getStatus(consensusReal, b))
}
}
test(consensusConfig.MergeDepth, tip, true, true)
test(consensusConfig.FinalityDepth(), tipBeforeHFActivated, false, true)
test(consensusConfig.MergeDepth, tipBeforeHFActivated, false, false)
})
}

View File

@ -53,4 +53,5 @@ type Consensus interface {
TrustedBlockAssociatedGHOSTDAGDataBlockHashes(blockHash *DomainHash) ([]*DomainHash, error)
TrustedGHOSTDAGData(blockHash *DomainHash) (*BlockGHOSTDAGData, error)
IsChainBlock(blockHash *DomainHash) (bool, error)
VirtualMergeDepthRoot() (*DomainHash, error)
}

View File

@ -0,0 +1,13 @@
package model
import (
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
)
// MergeDepthRootStore represents a store for merge depth roots
type MergeDepthRootStore interface {
Store
IsStaged(stagingArea *StagingArea) bool
StageMergeDepthRoot(stagingArea *StagingArea, blockHash *externalapi.DomainHash, root *externalapi.DomainHash)
MergeDepthRoot(dbContext DBReader, stagingArea *StagingArea, blockHash *externalapi.DomainHash) (*externalapi.DomainHash, error)
}

View File

@ -5,5 +5,7 @@ import "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
// MergeDepthManager is used to validate mergeDepth for blocks
type MergeDepthManager interface {
CheckBoundedMergeDepth(stagingArea *StagingArea, blockHash *externalapi.DomainHash, isBlockWithTrustedData bool) error
NonBoundedMergeDepthViolatingBlues(stagingArea *StagingArea, blockHash *externalapi.DomainHash, isBlockWithTrustedData bool) ([]*externalapi.DomainHash, error)
NonBoundedMergeDepthViolatingBlues(stagingArea *StagingArea, blockHash, mergeDepthRoot *externalapi.DomainHash) ([]*externalapi.DomainHash, error)
VirtualMergeDepthRoot(stagingArea *StagingArea) (*externalapi.DomainHash, error)
MergeDepthRoot(stagingArea *StagingArea, blockHash *externalapi.DomainHash, isBlockWithTrustedData bool) (*externalapi.DomainHash, error)
}

View File

@ -7,7 +7,7 @@ import (
// TransactionValidator exposes a set of validation classes, after which
// it's possible to determine whether a transaction is valid
type TransactionValidator interface {
ValidateTransactionInIsolation(transaction *externalapi.DomainTransaction) error
ValidateTransactionInIsolation(transaction *externalapi.DomainTransaction, povDAAScore uint64) error
ValidateTransactionInContextIgnoringUTXO(stagingArea *StagingArea, tx *externalapi.DomainTransaction,
povBlockHash *externalapi.DomainHash, povBlockPastMedianTime int64) error
ValidateTransactionInContextAndPopulateFee(stagingArea *StagingArea,

View File

@ -18,4 +18,6 @@ type TestBlockBuilder interface {
BuildUTXOInvalidBlock(parentHashes []*externalapi.DomainHash) (*externalapi.DomainBlock,
error)
SetNonceCounter(nonceCounter uint64)
}

View File

@ -20,6 +20,7 @@ import (
type blockBuilder struct {
databaseContext model.DBManager
genesisHash *externalapi.DomainHash
hf1DAAScore uint64
difficultyManager model.DifficultyManager
pastMedianTimeManager model.PastMedianTimeManager
@ -42,6 +43,7 @@ type blockBuilder struct {
func New(
databaseContext model.DBManager,
genesisHash *externalapi.DomainHash,
hf1DAAScore uint64,
difficultyManager model.DifficultyManager,
pastMedianTimeManager model.PastMedianTimeManager,
@ -63,6 +65,7 @@ func New(
return &blockBuilder{
databaseContext: databaseContext,
genesisHash: genesisHash,
hf1DAAScore: hf1DAAScore,
difficultyManager: difficultyManager,
pastMedianTimeManager: pastMedianTimeManager,
@ -224,8 +227,13 @@ func (bb *blockBuilder) buildHeader(stagingArea *model.StagingArea, transactions
return nil, err
}
version := constants.BlockVersionBeforeHF1
if daaScore >= bb.hf1DAAScore {
version = constants.BlockVersionAfterHF1
}
return blockheader.NewImmutableBlockHeader(
constants.MaxBlockVersion,
version,
parents,
hashMerkleRoot,
acceptedIDMerkleRoot,

View File

@ -94,9 +94,14 @@ func (bb *testBlockBuilder) buildUTXOInvalidHeader(stagingArea *model.StagingAre
})
}
version := constants.BlockVersionBeforeHF1
if daaScore >= bb.hf1DAAScore {
version = constants.BlockVersionAfterHF1
}
bb.nonceCounter++
return blockheader.NewImmutableBlockHeader(
constants.MaxBlockVersion,
version,
parents,
hashMerkleRoot,
&externalapi.DomainHash{},
@ -283,3 +288,7 @@ func (bb *testBlockBuilder) BuildUTXOInvalidBlock(parentHashes []*externalapi.Do
Transactions: transactions,
}, nil
}
func (bb *testBlockBuilder) SetNonceCounter(nonceCounter uint64) {
bb.nonceCounter = nonceCounter
}

View File

@ -28,6 +28,7 @@ type blockProcessor struct {
coinbaseManager model.CoinbaseManager
headerTipsManager model.HeadersSelectedTipManager
syncManager model.SyncManager
finalityManager model.FinalityManager
acceptanceDataStore model.AcceptanceDataStore
blockStore model.BlockStore

View File

@ -140,7 +140,7 @@ func (v *blockValidator) checkBlockTransactionOrder(block *externalapi.DomainBlo
func (v *blockValidator) checkTransactionsInIsolation(block *externalapi.DomainBlock) error {
for _, tx := range block.Transactions {
err := v.transactionValidator.ValidateTransactionInIsolation(tx)
err := v.transactionValidator.ValidateTransactionInIsolation(tx, block.Header.DAAScore())
if err != nil {
return errors.Wrapf(err, "transaction %s failed isolation "+
"check", consensushashing.TransactionID(tx))
@ -220,7 +220,7 @@ func (v *blockValidator) validateGasLimit(block *externalapi.DomainBlock) error
func (v *blockValidator) checkBlockMass(block *externalapi.DomainBlock) error {
mass := uint64(0)
if !v.ignoreHeaderMass {
if block.Header.DAAScore() < v.hf1DAAScore {
mass += v.headerEstimatedSerializedSize(block.Header)
}

View File

@ -1307,7 +1307,7 @@ func initBlockWithFirstTransactionDifferentThanCoinbase(consensusConfig *consens
return &externalapi.DomainBlock{
Header: blockheader.NewImmutableBlockHeader(
constants.MaxBlockVersion,
constants.BlockVersionBeforeHF1,
[]externalapi.BlockLevelParents{[]*externalapi.DomainHash{consensusConfig.GenesisHash}},
merkle.CalculateHashMerkleRoot([]*externalapi.DomainTransaction{tx}),
&externalapi.DomainHash{},

View File

@ -2,6 +2,7 @@ package blockvalidator_test
import (
"errors"
"github.com/kaspanet/kaspad/domain/consensus/utils/constants"
"math/big"
"testing"
@ -107,9 +108,14 @@ func TestCheckParentsIncest(t *testing.T) {
t.Fatalf("AddBlock: %+v", err)
}
version := constants.BlockVersionBeforeHF1
if consensusConfig.HF1DAAScore == 0 {
version = constants.BlockVersionAfterHF1
}
directParentsRelationBlock := &externalapi.DomainBlock{
Header: blockheader.NewImmutableBlockHeader(
0,
version,
[]externalapi.BlockLevelParents{[]*externalapi.DomainHash{a, b}},
&externalapi.DomainHash{},
&externalapi.DomainHash{},
@ -132,7 +138,7 @@ func TestCheckParentsIncest(t *testing.T) {
indirectParentsRelationBlock := &externalapi.DomainBlock{
Header: blockheader.NewImmutableBlockHeader(
0,
version,
[]externalapi.BlockLevelParents{[]*externalapi.DomainHash{consensusConfig.GenesisHash, b}},
&externalapi.DomainHash{},
&externalapi.DomainHash{},

View File

@ -22,9 +22,11 @@ func (v *blockValidator) ValidateHeaderInIsolation(stagingArea *model.StagingAre
return err
}
err = v.checkBlockVersion(header)
if err != nil {
return err
if !blockHash.Equal(v.genesisHash) {
err = v.checkBlockVersion(header)
if err != nil {
return err
}
}
err = v.checkBlockTimestampInIsolation(header)
@ -54,9 +56,16 @@ func (v *blockValidator) checkParentsLimit(header externalapi.BlockHeader) error
}
func (v *blockValidator) checkBlockVersion(header externalapi.BlockHeader) error {
if header.Version() > constants.MaxBlockVersion {
return errors.Wrapf(
ruleerrors.ErrBlockVersionIsUnknown, "The block version is unknown.")
if header.DAAScore() >= v.hf1DAAScore {
if header.Version() != constants.BlockVersionAfterHF1 {
return errors.Wrapf(
ruleerrors.ErrWrongBlockVersion, "After HF1 the block version should be %d", constants.BlockVersionAfterHF1)
}
} else {
if header.Version() != constants.BlockVersionBeforeHF1 {
return errors.Wrapf(
ruleerrors.ErrWrongBlockVersion, "Beofre HF1 the block version should be %d", constants.BlockVersionBeforeHF1)
}
}
return nil
}

View File

@ -62,8 +62,13 @@ func CheckBlockVersion(t *testing.T, tc testapi.TestConsensus, consensusConfig *
t.Fatalf("BuildBlockWithParents: %+v", err)
}
expectedVersion := constants.BlockVersionBeforeHF1
if consensusConfig.HF1DAAScore == 0 {
expectedVersion = constants.BlockVersionAfterHF1
}
block.Header = blockheader.NewImmutableBlockHeader(
constants.MaxBlockVersion+1,
expectedVersion+1,
block.Header.Parents(),
block.Header.HashMerkleRoot(),
block.Header.AcceptedIDMerkleRoot(),
@ -78,7 +83,7 @@ func CheckBlockVersion(t *testing.T, tc testapi.TestConsensus, consensusConfig *
)
_, err = tc.ValidateAndInsertBlock(block, true)
if !errors.Is(err, ruleerrors.ErrBlockVersionIsUnknown) {
if !errors.Is(err, ruleerrors.ErrWrongBlockVersion) {
t.Fatalf("Unexpected error: %+v", err)
}
}

View File

@ -24,7 +24,7 @@ type blockValidator struct {
maxBlockParents externalapi.KType
timestampDeviationTolerance int
targetTimePerBlock time.Duration
ignoreHeaderMass bool
hf1DAAScore uint64
maxBlockLevel int
databaseContext model.DBReader
@ -64,7 +64,7 @@ func New(powMax *big.Int,
maxBlockParents externalapi.KType,
timestampDeviationTolerance int,
targetTimePerBlock time.Duration,
ignoreHeaderMass bool,
hf1DAAScore uint64,
maxBlockLevel int,
databaseContext model.DBReader,
@ -104,7 +104,7 @@ func New(powMax *big.Int,
maxBlockMass: maxBlockMass,
mergeSetSizeLimit: mergeSetSizeLimit,
maxBlockParents: maxBlockParents,
ignoreHeaderMass: ignoreHeaderMass,
hf1DAAScore: hf1DAAScore,
maxBlockLevel: maxBlockLevel,
timestampDeviationTolerance: timestampDeviationTolerance,

View File

@ -282,19 +282,19 @@ func (csm *consensusStateManager) boundedMergeBreakingParents(stagingArea *model
return nil, err
}
virtualMergeDepthRoot, err := csm.mergeDepthManager.VirtualMergeDepthRoot(stagingArea)
if err != nil {
return nil, err
}
log.Debugf("The merge depth root of virtual is: %s", virtualMergeDepthRoot)
potentiallyKosherizingBlocks, err :=
csm.mergeDepthManager.NonBoundedMergeDepthViolatingBlues(stagingArea, model.VirtualBlockHash, false)
csm.mergeDepthManager.NonBoundedMergeDepthViolatingBlues(stagingArea, model.VirtualBlockHash, virtualMergeDepthRoot)
if err != nil {
return nil, err
}
log.Debugf("The potentially kosherizing blocks are: %s", potentiallyKosherizingBlocks)
virtualFinalityPoint, err := csm.finalityManager.VirtualFinalityPoint(stagingArea)
if err != nil {
return nil, err
}
log.Debugf("The finality point of the virtual is: %s", virtualFinalityPoint)
var badReds []*externalapi.DomainHash
virtualGHOSTDAGData, err := csm.ghostdagDataStore.Get(csm.databaseContext, stagingArea, model.VirtualBlockHash, false)
@ -303,13 +303,13 @@ func (csm *consensusStateManager) boundedMergeBreakingParents(stagingArea *model
}
for _, redBlock := range virtualGHOSTDAGData.MergeSetReds() {
log.Debugf("Check whether red block %s is kosherized", redBlock)
isFinalityPointInPast, err := csm.dagTopologyManager.IsAncestorOf(stagingArea, virtualFinalityPoint, redBlock)
isMergeDepthRootInPast, err := csm.dagTopologyManager.IsAncestorOf(stagingArea, virtualMergeDepthRoot, redBlock)
if err != nil {
return nil, err
}
if isFinalityPointInPast {
if isMergeDepthRootInPast {
log.Debugf("Skipping red block %s because it has the virtual's"+
" finality point in its past", redBlock)
" merge depth root in its past", redBlock)
continue
}

View File

@ -185,12 +185,12 @@ func TestBlockWindow(t *testing.T) {
{
parents: []string{"C", "D"},
id: "E",
expectedWindow: []string{"D", "C", "B"},
expectedWindow: []string{"C", "D", "B"},
},
{
parents: []string{"C", "D"},
id: "F",
expectedWindow: []string{"D", "C", "B"},
expectedWindow: []string{"C", "D", "B"},
},
{
parents: []string{"A"},
@ -205,37 +205,37 @@ func TestBlockWindow(t *testing.T) {
{
parents: []string{"H", "F"},
id: "I",
expectedWindow: []string{"F", "D", "H", "C", "G", "B"},
expectedWindow: []string{"F", "C", "D", "H", "G", "B"},
},
{
parents: []string{"I"},
id: "J",
expectedWindow: []string{"I", "F", "D", "H", "C", "G", "B"},
expectedWindow: []string{"I", "F", "C", "D", "H", "G", "B"},
},
{
parents: []string{"J"},
id: "K",
expectedWindow: []string{"J", "I", "F", "D", "H", "C", "G", "B"},
expectedWindow: []string{"J", "I", "F", "C", "D", "H", "G", "B"},
},
{
parents: []string{"K"},
id: "L",
expectedWindow: []string{"K", "J", "I", "F", "D", "H", "C", "G", "B"},
expectedWindow: []string{"K", "J", "I", "F", "C", "D", "H", "G", "B"},
},
{
parents: []string{"L"},
id: "M",
expectedWindow: []string{"L", "K", "J", "I", "F", "D", "H", "C", "G", "B"},
expectedWindow: []string{"L", "K", "J", "I", "F", "C", "D", "H", "G", "B"},
},
{
parents: []string{"M"},
id: "N",
expectedWindow: []string{"M", "L", "K", "J", "I", "F", "D", "H", "C", "G"},
expectedWindow: []string{"M", "L", "K", "J", "I", "F", "C", "D", "H", "G"},
},
{
parents: []string{"N"},
id: "O",
expectedWindow: []string{"N", "M", "L", "K", "J", "I", "F", "D", "H", "C"},
expectedWindow: []string{"N", "M", "L", "K", "J", "I", "F", "C", "D", "H"},
},
},
dagconfig.SimnetParams.Name: {
@ -255,14 +255,14 @@ func TestBlockWindow(t *testing.T) {
expectedWindow: []string{"B"},
},
{
parents: []string{"D", "C"},
parents: []string{"C", "D"},
id: "E",
expectedWindow: []string{"D", "C", "B"},
expectedWindow: []string{"C", "D", "B"},
},
{
parents: []string{"D", "C"},
parents: []string{"C", "D"},
id: "F",
expectedWindow: []string{"D", "C", "B"},
expectedWindow: []string{"C", "D", "B"},
},
{
parents: []string{"A"},
@ -277,37 +277,37 @@ func TestBlockWindow(t *testing.T) {
{
parents: []string{"H", "F"},
id: "I",
expectedWindow: []string{"F", "H", "D", "C", "B", "G"},
expectedWindow: []string{"F", "C", "H", "D", "B", "G"},
},
{
parents: []string{"I"},
id: "J",
expectedWindow: []string{"I", "F", "H", "D", "C", "B", "G"},
expectedWindow: []string{"I", "F", "C", "H", "D", "B", "G"},
},
{
parents: []string{"J"},
id: "K",
expectedWindow: []string{"J", "I", "F", "H", "D", "C", "B", "G"},
expectedWindow: []string{"J", "I", "F", "C", "H", "D", "B", "G"},
},
{
parents: []string{"K"},
id: "L",
expectedWindow: []string{"K", "J", "I", "F", "H", "D", "C", "B", "G"},
expectedWindow: []string{"K", "J", "I", "F", "C", "H", "D", "B", "G"},
},
{
parents: []string{"L"},
id: "M",
expectedWindow: []string{"L", "K", "J", "I", "F", "H", "D", "C", "B", "G"},
expectedWindow: []string{"L", "K", "J", "I", "F", "C", "H", "D", "B", "G"},
},
{
parents: []string{"M"},
id: "N",
expectedWindow: []string{"M", "L", "K", "J", "I", "F", "H", "D", "C", "B"},
expectedWindow: []string{"M", "L", "K", "J", "I", "F", "C", "H", "D", "B"},
},
{
parents: []string{"N"},
id: "O",
expectedWindow: []string{"N", "M", "L", "K", "J", "I", "F", "H", "D", "C"},
expectedWindow: []string{"N", "M", "L", "K", "J", "I", "F", "C", "H", "D"},
},
},
}

View File

@ -111,7 +111,7 @@ func TestGHOSTDAG(t *testing.T) {
blockID := StringToDomainHash(testBlockData.ID)
dagTopology.parentsMap[*blockID] = StringToDomainHashSlice(testBlockData.Parents)
blockHeadersStore.dagMap[*blockID] = blockheader.NewImmutableBlockHeader(
constants.MaxBlockVersion,
constants.BlockVersionBeforeHF1,
[]externalapi.BlockLevelParents{StringToDomainHashSlice(testBlockData.Parents)},
nil,
nil,

View File

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

View File

@ -4,6 +4,7 @@ import (
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/kaspanet/kaspad/domain/consensus/ruleerrors"
"github.com/kaspanet/kaspad/infrastructure/db/database"
"github.com/pkg/errors"
)
@ -13,7 +14,15 @@ type mergeDepthManager struct {
dagTraversalManager model.DAGTraversalManager
finalityManager model.FinalityManager
ghostdagDataStore model.GHOSTDAGDataStore
genesisHash *externalapi.DomainHash
mergeDepth uint64
hf1DAAScore uint64
ghostdagDataStore model.GHOSTDAGDataStore
mergeDepthRootStore model.MergeDepthRootStore
daaBlocksStore model.DAABlocksStore
pruningStore model.PruningStore
finalityStore model.FinalityStore
}
// New instantiates a new MergeDepthManager
@ -22,24 +31,36 @@ func New(
dagTopologyManager model.DAGTopologyManager,
dagTraversalManager model.DAGTraversalManager,
finalityManager model.FinalityManager,
ghostdagDataStore model.GHOSTDAGDataStore) model.MergeDepthManager {
genesisHash *externalapi.DomainHash,
mergeDepth uint64,
hf1DAAScore uint64,
ghostdagDataStore model.GHOSTDAGDataStore,
mergeDepthRootStore model.MergeDepthRootStore,
daaBlocksStore model.DAABlocksStore,
pruningStore model.PruningStore,
finalityStore model.FinalityStore) model.MergeDepthManager {
return &mergeDepthManager{
databaseContext: databaseContext,
dagTopologyManager: dagTopologyManager,
dagTraversalManager: dagTraversalManager,
finalityManager: finalityManager,
genesisHash: genesisHash,
mergeDepth: mergeDepth,
hf1DAAScore: hf1DAAScore,
ghostdagDataStore: ghostdagDataStore,
mergeDepthRootStore: mergeDepthRootStore,
daaBlocksStore: daaBlocksStore,
pruningStore: pruningStore,
finalityStore: finalityStore,
}
}
// CheckBoundedMergeDepth is used for validation, so must follow the HF1 DAA score for determining the correct depth to verify
func (mdm *mergeDepthManager) CheckBoundedMergeDepth(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash, isBlockWithTrustedData bool) error {
nonBoundedMergeDepthViolatingBlues, err := mdm.NonBoundedMergeDepthViolatingBlues(stagingArea, blockHash, isBlockWithTrustedData)
if err != nil {
return err
}
ghostdagData, err := mdm.ghostdagDataStore.Get(mdm.databaseContext, stagingArea, blockHash, false)
if err != nil {
return err
@ -50,28 +71,34 @@ func (mdm *mergeDepthManager) CheckBoundedMergeDepth(stagingArea *model.StagingA
return nil
}
finalityPoint, err := mdm.finalityManager.FinalityPoint(stagingArea, blockHash, isBlockWithTrustedData)
// For validation, we must follow the HF1 DAA score in order to determine the correct depth to verify
mergeDepthRootByHF1, err := mdm.mergeDepthRootByHF1DAAScoreForValidationOnly(stagingArea, blockHash, isBlockWithTrustedData)
if err != nil {
return err
}
nonBoundedMergeDepthViolatingBlues, err := mdm.NonBoundedMergeDepthViolatingBlues(stagingArea, blockHash, mergeDepthRootByHF1)
if err != nil {
return err
}
for _, red := range ghostdagData.MergeSetReds() {
doesRedHaveFinalityPointInPast, err := mdm.dagTopologyManager.IsAncestorOf(stagingArea, finalityPoint, red)
doesRedHaveMergeRootInPast, err := mdm.dagTopologyManager.IsAncestorOf(stagingArea, mergeDepthRootByHF1, red)
if err != nil {
return err
}
if doesRedHaveFinalityPointInPast {
if doesRedHaveMergeRootInPast {
continue
}
isRedInPastOfAnyNonFinalityViolatingBlue, err :=
isRedInPastOfAnyNonMergeDepthViolatingBlue, err :=
mdm.dagTopologyManager.IsAncestorOfAny(stagingArea, red, nonBoundedMergeDepthViolatingBlues)
if err != nil {
return err
}
if !isRedInPastOfAnyNonFinalityViolatingBlue {
if !isRedInPastOfAnyNonMergeDepthViolatingBlue {
return errors.Wrapf(ruleerrors.ErrViolatingBoundedMergeDepth, "block is violating bounded merge depth")
}
}
@ -79,7 +106,9 @@ func (mdm *mergeDepthManager) CheckBoundedMergeDepth(stagingArea *model.StagingA
return nil
}
func (mdm *mergeDepthManager) NonBoundedMergeDepthViolatingBlues(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash, isBlockWithTrustedData bool) ([]*externalapi.DomainHash, error) {
func (mdm *mergeDepthManager) NonBoundedMergeDepthViolatingBlues(
stagingArea *model.StagingArea, blockHash, mergeDepthRoot *externalapi.DomainHash) ([]*externalapi.DomainHash, error) {
ghostdagData, err := mdm.ghostdagDataStore.Get(mdm.databaseContext, stagingArea, blockHash, false)
if err != nil {
return nil, err
@ -87,20 +116,173 @@ func (mdm *mergeDepthManager) NonBoundedMergeDepthViolatingBlues(stagingArea *mo
nonBoundedMergeDepthViolatingBlues := make([]*externalapi.DomainHash, 0, len(ghostdagData.MergeSetBlues()))
finalityPoint, err := mdm.finalityManager.FinalityPoint(stagingArea, blockHash, isBlockWithTrustedData)
if err != nil {
return nil, err
}
for _, blue := range ghostdagData.MergeSetBlues() {
notViolatingFinality, err := mdm.dagTopologyManager.IsInSelectedParentChainOf(stagingArea, finalityPoint, blue)
isMergeDepthRootInSelectedChainOfBlue, err := mdm.dagTopologyManager.IsInSelectedParentChainOf(stagingArea, mergeDepthRoot, blue)
if err != nil {
return nil, err
}
if notViolatingFinality {
if isMergeDepthRootInSelectedChainOfBlue {
nonBoundedMergeDepthViolatingBlues = append(nonBoundedMergeDepthViolatingBlues, blue)
}
}
return nonBoundedMergeDepthViolatingBlues, nil
}
func (mdm *mergeDepthManager) mergeDepthRootByHF1DAAScoreForValidationOnly(
stagingArea *model.StagingArea, blockHash *externalapi.DomainHash, isBlockWithTrustedData bool) (*externalapi.DomainHash, error) {
daaScore, err := mdm.daaBlocksStore.DAAScore(mdm.databaseContext, stagingArea, blockHash)
if err != nil {
return nil, err
}
// We call both, merge depth root and finality, in order to trigger storage persistency for both,
// although only one of them is used below
mergeDepthRoot, err := mdm.MergeDepthRoot(stagingArea, blockHash, isBlockWithTrustedData)
if err != nil {
return nil, err
}
// As noted above, this line should not be removed following the HF, unless we validate that storage of
// finality point is not needed any more
finalityPoint, err := mdm.finalityManager.FinalityPoint(stagingArea, blockHash, isBlockWithTrustedData)
if err != nil {
return nil, err
}
if daaScore >= mdm.hf1DAAScore {
return mergeDepthRoot, nil
}
// We fall back to the merge depth root before the HF, which was the finality point
return finalityPoint, nil
}
func (mdm *mergeDepthManager) VirtualMergeDepthRoot(stagingArea *model.StagingArea) (*externalapi.DomainHash, error) {
log.Tracef("VirtualMergeDepthRoot start")
defer log.Tracef("VirtualMergeDepthRoot end")
virtualMergeDepthRoot, err := mdm.calculateMergeDepthRoot(stagingArea, model.VirtualBlockHash, false)
if err != nil {
return nil, err
}
log.Debugf("The current virtual merge depth root is: %s", virtualMergeDepthRoot)
return virtualMergeDepthRoot, nil
}
func (mdm *mergeDepthManager) MergeDepthRoot(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash, isBlockWithTrustedData bool) (*externalapi.DomainHash, error) {
log.Tracef("MergeDepthRoot start")
defer log.Tracef("MergeDepthRoot end")
if blockHash.Equal(model.VirtualBlockHash) {
return mdm.VirtualMergeDepthRoot(stagingArea)
}
root, err := mdm.mergeDepthRootStore.MergeDepthRoot(mdm.databaseContext, stagingArea, blockHash)
if err != nil {
log.Debugf("%s merge depth root not found in store - calculating", blockHash)
if errors.Is(err, database.ErrNotFound) {
return mdm.calculateAndStageMergeDepthRoot(stagingArea, blockHash, isBlockWithTrustedData)
}
return nil, err
}
return root, nil
}
func (mdm *mergeDepthManager) calculateAndStageMergeDepthRoot(
stagingArea *model.StagingArea, blockHash *externalapi.DomainHash, isBlockWithTrustedData bool) (*externalapi.DomainHash, error) {
root, err := mdm.calculateMergeDepthRoot(stagingArea, blockHash, isBlockWithTrustedData)
if err != nil {
return nil, err
}
mdm.mergeDepthRootStore.StageMergeDepthRoot(stagingArea, blockHash, root)
return root, nil
}
func (mdm *mergeDepthManager) calculateMergeDepthRoot(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash, isBlockWithTrustedData bool) (
*externalapi.DomainHash, error) {
log.Tracef("calculateMergeDepthRoot start")
defer log.Tracef("calculateMergeDepthRoot end")
if isBlockWithTrustedData {
return model.VirtualGenesisBlockHash, nil
}
ghostdagData, err := mdm.ghostdagDataStore.Get(mdm.databaseContext, stagingArea, blockHash, false)
if err != nil {
return nil, err
}
if ghostdagData.BlueScore() < mdm.mergeDepth {
log.Debugf("%s blue score lower then merge depth - returning genesis as merge depth root", blockHash)
return mdm.genesisHash, nil
}
pruningPoint, err := mdm.pruningStore.PruningPoint(mdm.databaseContext, stagingArea)
if err != nil {
return nil, err
}
pruningPointGhostdagData, err := mdm.ghostdagDataStore.Get(mdm.databaseContext, stagingArea, pruningPoint, false)
if err != nil {
return nil, err
}
if ghostdagData.BlueScore() < pruningPointGhostdagData.BlueScore()+mdm.mergeDepth {
log.Debugf("%s blue score less than merge depth over pruning point - returning virtual genesis as merge depth root", blockHash)
return model.VirtualGenesisBlockHash, nil
}
isPruningPointOnChain, err := mdm.dagTopologyManager.IsInSelectedParentChainOf(stagingArea, pruningPoint, blockHash)
if err != nil {
return nil, err
}
if !isPruningPointOnChain {
log.Debugf("pruning point not in selected chain of %s - returning virtual genesis as merge depth root", blockHash)
return model.VirtualGenesisBlockHash, nil
}
selectedParent := ghostdagData.SelectedParent()
if selectedParent.Equal(mdm.genesisHash) {
return mdm.genesisHash, nil
}
current, err := mdm.mergeDepthRootStore.MergeDepthRoot(mdm.databaseContext, stagingArea, ghostdagData.SelectedParent())
if database.IsNotFoundError(err) {
// This should only occur for a few blocks following the upgrade
log.Debugf("merge point root not in store for %s, falling back to finality point", ghostdagData.SelectedParent())
current, err = mdm.finalityStore.FinalityPoint(mdm.databaseContext, stagingArea, ghostdagData.SelectedParent())
if err != nil {
return nil, err
}
} else if err != nil {
return nil, err
}
// In this case we expect the pruning point or a block above it to be the merge depth root.
// Note that above we already verified the chain and distance conditions for this
if current.Equal(model.VirtualGenesisBlockHash) {
current = pruningPoint
}
requiredBlueScore := ghostdagData.BlueScore() - mdm.mergeDepth
log.Debugf("%s's merge depth root is the one having the highest blue score lower then %d", blockHash, requiredBlueScore)
var next *externalapi.DomainHash
for {
next, err = mdm.dagTopologyManager.ChildInSelectedParentChainOf(stagingArea, current, blockHash)
if err != nil {
return nil, err
}
nextGHOSTDAGData, err := mdm.ghostdagDataStore.Get(mdm.databaseContext, stagingArea, next, false)
if err != nil {
return nil, err
}
if nextGHOSTDAGData.BlueScore() >= requiredBlueScore {
log.Debugf("%s's merge depth root is %s", blockHash, current)
return current, nil
}
current = next
}
}

View File

@ -38,7 +38,7 @@ func TestPruning(t *testing.T) {
"dag-for-test-pruning.json": {
dagconfig.MainnetParams.Name: "503",
dagconfig.TestnetParams.Name: "502",
dagconfig.DevnetParams.Name: "502",
dagconfig.DevnetParams.Name: "503",
dagconfig.SimnetParams.Name: "503",
},
}

View File

@ -58,6 +58,14 @@ func (v *transactionValidator) ValidateTransactionInContextIgnoringUTXO(stagingA
return errors.Wrapf(ruleerrors.ErrUnfinalizedTx, "unfinalized transaction %v", tx)
}
// TODO: Remove checkTransactionAmountRanges from here once HF was activated. It's only temporary
// because in the HF transition period checkTransactionAmountRanges is not context free.
isHF1Activated := povBlockDAAScore >= v.hf1DAAScore
err = v.checkTransactionAmountRanges(tx, isHF1Activated)
if err != nil {
return err
}
return nil
}
@ -73,7 +81,13 @@ func (v *transactionValidator) ValidateTransactionInContextAndPopulateFee(stagin
return err
}
totalSompiIn, err := v.checkTransactionInputAmounts(tx)
daaScore, err := v.daaBlocksStore.DAAScore(v.databaseContext, stagingArea, povBlockHash)
if err != nil {
return err
}
isHF1Activated := daaScore >= v.hf1DAAScore
totalSompiIn, err := v.checkTransactionInputAmounts(tx, isHF1Activated)
if err != nil {
return err
}
@ -135,7 +149,7 @@ func (v *transactionValidator) checkTransactionCoinbaseMaturity(stagingArea *mod
return nil
}
func (v *transactionValidator) checkTransactionInputAmounts(tx *externalapi.DomainTransaction) (totalSompiIn uint64, err error) {
func (v *transactionValidator) checkTransactionInputAmounts(tx *externalapi.DomainTransaction, isHF1Activated bool) (totalSompiIn uint64, err error) {
totalSompiIn = 0
var missingOutpoints []*externalapi.DomainOutpoint
@ -152,7 +166,7 @@ func (v *transactionValidator) checkTransactionInputAmounts(tx *externalapi.Doma
// a transaction are in a unit value known as a sompi. One
// kaspa is a quantity of sompi as defined by the
// SompiPerKaspa constant.
totalSompiIn, err = v.checkEntryAmounts(utxoEntry, totalSompiIn)
totalSompiIn, err = v.checkEntryAmounts(utxoEntry, totalSompiIn, isHF1Activated)
if err != nil {
return 0, err
}
@ -165,18 +179,24 @@ func (v *transactionValidator) checkTransactionInputAmounts(tx *externalapi.Doma
return totalSompiIn, nil
}
func (v *transactionValidator) checkEntryAmounts(entry externalapi.UTXOEntry, totalSompiInBefore uint64) (totalSompiInAfter uint64, err error) {
func (v *transactionValidator) checkEntryAmounts(entry externalapi.UTXOEntry, totalSompiInBefore uint64, isHF1Activated bool) (totalSompiInAfter uint64, err error) {
// The total of all outputs must not be more than the max
// allowed per transaction. Also, we could potentially overflow
// the accumulator so check for overflow.
maxSompi := constants.MaxSompiBeforeHF1
if isHF1Activated {
maxSompi = constants.MaxSompiAfterHF1
}
originTxSompi := entry.Amount()
totalSompiInAfter = totalSompiInBefore + originTxSompi
if totalSompiInAfter < totalSompiInBefore ||
totalSompiInAfter > constants.MaxSompi {
totalSompiInAfter > maxSompi {
return 0, errors.Wrapf(ruleerrors.ErrBadTxOutValue, "total value of all transaction "+
"inputs is %d which is higher than max "+
"allowed value of %d", totalSompiInBefore,
constants.MaxSompi)
maxSompi)
}
return totalSompiInAfter, nil
}

View File

@ -10,12 +10,13 @@ import (
)
// ValidateTransactionInIsolation validates the parts of the transaction that can be validated context-free
func (v *transactionValidator) ValidateTransactionInIsolation(tx *externalapi.DomainTransaction) error {
func (v *transactionValidator) ValidateTransactionInIsolation(tx *externalapi.DomainTransaction, povDAAScore uint64) error {
isHF1Activated := povDAAScore >= v.hf1DAAScore
err := v.checkTransactionInputCount(tx)
if err != nil {
return err
}
err = v.checkTransactionAmountRanges(tx)
err = v.checkTransactionAmountRanges(tx, isHF1Activated)
if err != nil {
return err
}
@ -62,13 +63,18 @@ func (v *transactionValidator) checkTransactionInputCount(tx *externalapi.Domain
return nil
}
func (v *transactionValidator) checkTransactionAmountRanges(tx *externalapi.DomainTransaction) error {
func (v *transactionValidator) checkTransactionAmountRanges(tx *externalapi.DomainTransaction, isHF1Activated bool) error {
// Ensure the transaction amounts are in range. Each transaction
// output must not be negative or more than the max allowed per
// transaction. Also, the total of all outputs must abide by the same
// restrictions. All amounts in a transaction are in a unit value known
// as a sompi. One kaspa is a quantity of sompi as defined by the
// sompiPerKaspa constant.
maxSompi := constants.MaxSompiBeforeHF1
if isHF1Activated {
maxSompi = constants.MaxSompiAfterHF1
}
var totalSompi uint64
for _, txOut := range tx.Outputs {
sompi := txOut.Value
@ -76,9 +82,9 @@ func (v *transactionValidator) checkTransactionAmountRanges(tx *externalapi.Doma
return errors.Wrap(ruleerrors.ErrTxOutValueZero, "zero value outputs are forbidden")
}
if sompi > constants.MaxSompi {
if sompi > maxSompi {
return errors.Wrapf(ruleerrors.ErrBadTxOutValue, "transaction output value of %d is "+
"higher than max allowed value of %d", sompi, constants.MaxSompi)
"higher than max allowed value of %d", sompi, maxSompi)
}
// Binary arithmetic guarantees that any overflow is detected and reported.
@ -88,14 +94,14 @@ func (v *transactionValidator) checkTransactionAmountRanges(tx *externalapi.Doma
if newTotalSompi < totalSompi {
return errors.Wrapf(ruleerrors.ErrBadTxOutValue, "total value of all transaction "+
"outputs exceeds max allowed value of %d",
constants.MaxSompi)
maxSompi)
}
totalSompi = newTotalSompi
if totalSompi > constants.MaxSompi {
if totalSompi > maxSompi {
return errors.Wrapf(ruleerrors.ErrBadTxOutValue, "total value of all transaction "+
"outputs is %d which is higher than max "+
"allowed value of %d", totalSompi,
constants.MaxSompi)
maxSompi)
}
}

View File

@ -21,8 +21,11 @@ type txSubnetworkData struct {
func TestValidateTransactionInIsolationAndPopulateMass(t *testing.T) {
testutils.ForAllNets(t, true, func(t *testing.T, consensusConfig *consensus.Config) {
cfg := *consensusConfig
cfg.HF1DAAScore = 20
factory := consensus.NewFactory()
tc, teardown, err := factory.NewTestConsensus(consensusConfig, "TestValidateTransactionInIsolationAndPopulateMass")
tc, teardown, err := factory.NewTestConsensus(&cfg, "TestValidateTransactionInIsolationAndPopulateMass")
if err != nil {
t.Fatalf("Error setting up consensus: %+v", err)
}
@ -37,25 +40,36 @@ func TestValidateTransactionInIsolationAndPopulateMass(t *testing.T) {
txSubnetworkData *txSubnetworkData
extraModificationsFunc func(*externalapi.DomainTransaction)
expectedErr error
daaScore uint64
}{
{"good one", 1, 1, 1, subnetworks.SubnetworkIDNative, nil, nil, nil},
{"no inputs", 0, 1, 1, subnetworks.SubnetworkIDNative, nil, nil, ruleerrors.ErrNoTxInputs},
{"no outputs", 1, 0, 1, subnetworks.SubnetworkIDNative, nil, nil, nil},
{"too much sompi in one output", 1, 1, constants.MaxSompi + 1,
{"good one", 1, 1, 1, subnetworks.SubnetworkIDNative, nil, nil, nil, 0},
{"no inputs", 0, 1, 1, subnetworks.SubnetworkIDNative, nil, nil, ruleerrors.ErrNoTxInputs, 0},
{"no outputs", 1, 0, 1, subnetworks.SubnetworkIDNative, nil, nil, nil, 0},
{"too much sompi in one output", 1, 1, constants.MaxSompiBeforeHF1 + 1,
subnetworks.SubnetworkIDNative,
nil,
nil,
ruleerrors.ErrBadTxOutValue},
{"too much sompi in total outputs", 1, 2, constants.MaxSompi - 1,
ruleerrors.ErrBadTxOutValue, 0},
{"too much sompi before- valid now", 1, 1, constants.MaxSompiBeforeHF1 + 1,
subnetworks.SubnetworkIDNative,
nil,
nil,
ruleerrors.ErrBadTxOutValue},
nil, cfg.HF1DAAScore},
{"too much sompi in one output - after hf", 1, 1, constants.MaxSompiAfterHF1 + 1,
subnetworks.SubnetworkIDNative,
nil,
nil,
ruleerrors.ErrBadTxOutValue, cfg.HF1DAAScore},
{"too much sompi in one output", 1, 1, constants.MaxSompiAfterHF1 + 1,
subnetworks.SubnetworkIDNative,
nil,
nil,
ruleerrors.ErrBadTxOutValue, 0},
{"duplicate inputs", 2, 1, 1,
subnetworks.SubnetworkIDNative,
nil,
func(tx *externalapi.DomainTransaction) { tx.Inputs[1].PreviousOutpoint.Index = 0 },
ruleerrors.ErrDuplicateTxInputs},
ruleerrors.ErrDuplicateTxInputs, 0},
{"1 input coinbase",
1,
1,
@ -63,7 +77,7 @@ func TestValidateTransactionInIsolationAndPopulateMass(t *testing.T) {
subnetworks.SubnetworkIDNative,
&txSubnetworkData{subnetworks.SubnetworkIDCoinbase, 0, nil},
nil,
nil},
nil, 0},
{"no inputs coinbase",
0,
1,
@ -71,7 +85,7 @@ func TestValidateTransactionInIsolationAndPopulateMass(t *testing.T) {
subnetworks.SubnetworkIDNative,
&txSubnetworkData{subnetworks.SubnetworkIDCoinbase, 0, nil},
nil,
nil},
nil, 0},
{"too long payload coinbase",
1,
1,
@ -79,26 +93,26 @@ func TestValidateTransactionInIsolationAndPopulateMass(t *testing.T) {
subnetworks.SubnetworkIDNative,
&txSubnetworkData{subnetworks.SubnetworkIDCoinbase, 0, make([]byte, consensusConfig.MaxCoinbasePayloadLength+1)},
nil,
ruleerrors.ErrBadCoinbasePayloadLen},
ruleerrors.ErrBadCoinbasePayloadLen, 0},
{"non-zero gas in Kaspa", 1, 1, 1,
subnetworks.SubnetworkIDNative,
nil,
func(tx *externalapi.DomainTransaction) {
tx.Gas = 1
},
ruleerrors.ErrInvalidGas},
ruleerrors.ErrInvalidGas, 0},
{"non-zero gas in subnetwork registry", 1, 1, 1,
subnetworks.SubnetworkIDRegistry,
&txSubnetworkData{subnetworks.SubnetworkIDRegistry, 1, []byte{}},
nil,
ruleerrors.ErrInvalidGas},
ruleerrors.ErrInvalidGas, 0},
{"non-zero payload in Kaspa", 1, 1, 1,
subnetworks.SubnetworkIDNative,
nil,
func(tx *externalapi.DomainTransaction) {
tx.Payload = []byte{1}
},
ruleerrors.ErrInvalidPayload},
ruleerrors.ErrInvalidPayload, 0},
}
for _, test := range tests {
@ -108,7 +122,7 @@ func TestValidateTransactionInIsolationAndPopulateMass(t *testing.T) {
test.extraModificationsFunc(tx)
}
err := tc.TransactionValidator().ValidateTransactionInIsolation(tx)
err := tc.TransactionValidator().ValidateTransactionInIsolation(tx, test.daaScore)
if !errors.Is(err, test.expectedErr) {
t.Errorf("TestValidateTransactionInIsolationAndPopulateMass: '%s': unexpected error %+v", test.name, err)
}

View File

@ -21,6 +21,7 @@ type transactionValidator struct {
sigCache *txscript.SigCache
sigCacheECDSA *txscript.SigCacheECDSA
txMassCalculator *txmass.Calculator
hf1DAAScore uint64
}
// New instantiates a new TransactionValidator
@ -31,7 +32,8 @@ func New(blockCoinbaseMaturity uint64,
pastMedianTimeManager model.PastMedianTimeManager,
ghostdagDataStore model.GHOSTDAGDataStore,
daaBlocksStore model.DAABlocksStore,
txMassCalculator *txmass.Calculator) model.TransactionValidator {
txMassCalculator *txmass.Calculator,
hf1DAAScore uint64) model.TransactionValidator {
return &transactionValidator{
blockCoinbaseMaturity: blockCoinbaseMaturity,
@ -44,5 +46,6 @@ func New(blockCoinbaseMaturity uint64,
sigCache: txscript.NewSigCache(sigCacheSize),
sigCacheECDSA: txscript.NewSigCacheECDSA(sigCacheSize),
txMassCalculator: txMassCalculator,
hf1DAAScore: hf1DAAScore,
}
}

View File

@ -21,7 +21,7 @@ import (
func TestValidateTransactionInContextAndPopulateFee(t *testing.T) {
testutils.ForAllNets(t, true, func(t *testing.T, consensusConfig *consensus.Config) {
consensusConfig.HF1DAAScore = consensusConfig.GenesisBlock.Header.DAAScore() + 1000
factory := consensus.NewFactory()
tc, tearDown, err := factory.NewTestConsensus(consensusConfig,
"TestValidateTransactionInContextAndPopulateFee")
@ -85,17 +85,31 @@ func TestValidateTransactionInContextAndPopulateFee(t *testing.T) {
true,
uint64(6)),
}
txInputWithLargeAmount := externalapi.DomainTransactionInput{
PreviousOutpoint: prevOutPoint,
SignatureScript: []byte{},
Sequence: constants.MaxTxInSequenceNum,
SigOpCount: 1,
UTXOEntry: utxo.NewUTXOEntry(
constants.MaxSompi,
constants.MaxSompiBeforeHF1+1,
scriptPublicKey,
true,
uint64(5)),
false,
0),
}
txInputWithLargeAmountAfterHF := externalapi.DomainTransactionInput{
PreviousOutpoint: prevOutPoint,
SignatureScript: []byte{},
Sequence: constants.MaxTxInSequenceNum,
SigOpCount: 1,
UTXOEntry: utxo.NewUTXOEntry(
constants.MaxSompiAfterHF1+1,
scriptPublicKey,
false,
0),
}
txInputWithBadSigOpCount := externalapi.DomainTransactionInput{
PreviousOutpoint: prevOutPoint,
SignatureScript: []byte{},
@ -141,13 +155,31 @@ func TestValidateTransactionInContextAndPopulateFee(t *testing.T) {
SubnetworkID: subnetworks.SubnetworkIDRegistry,
Gas: 0,
LockTime: 0}
txWithLargeAmount := externalapi.DomainTransaction{
txWithLargeAmountBeforeHF := externalapi.DomainTransaction{
Version: constants.MaxTransactionVersion,
Inputs: []*externalapi.DomainTransactionInput{&txInput, &txInputWithLargeAmount},
Inputs: []*externalapi.DomainTransactionInput{&txInputWithLargeAmount},
Outputs: []*externalapi.DomainTransactionOutput{&txOutput},
SubnetworkID: subnetworks.SubnetworkIDRegistry,
Gas: 0,
LockTime: 0}
txWithLargeAmountAfterHF := externalapi.DomainTransaction{
Version: constants.MaxTransactionVersion,
Inputs: []*externalapi.DomainTransactionInput{&txInputWithLargeAmountAfterHF},
Outputs: []*externalapi.DomainTransactionOutput{&txOutput},
SubnetworkID: subnetworks.SubnetworkIDRegistry,
Gas: 0,
LockTime: 0}
for i, input := range txWithLargeAmountBeforeHF.Inputs {
signatureScript, err := txscript.SignatureScript(&txWithLargeAmountBeforeHF, i, consensushashing.SigHashAll, privateKey,
&consensushashing.SighashReusedValues{})
if err != nil {
t.Fatalf("Failed to create a sigScript: %v", err)
}
input.SignatureScript = signatureScript
}
txWithBigValue := externalapi.DomainTransaction{
Version: constants.MaxTransactionVersion,
Inputs: []*externalapi.DomainTransactionInput{&txInput},
@ -175,6 +207,9 @@ func TestValidateTransactionInContextAndPopulateFee(t *testing.T) {
povBlockHash := externalapi.NewDomainHashFromByteArray(&[32]byte{0x01})
tc.DAABlocksStore().StageDAAScore(stagingArea, povBlockHash, consensusConfig.BlockCoinbaseMaturity+txInput.UTXOEntry.BlockDAAScore())
povAfterHFBlockHash := externalapi.NewDomainHashFromByteArray(&[32]byte{0x02})
tc.DAABlocksStore().StageDAAScore(stagingArea, povAfterHFBlockHash, consensusConfig.HF1DAAScore)
// Just use some stub ghostdag data
tc.GHOSTDAGDataStore().Stage(stagingArea, povBlockHash, externalapi.NewBlockGHOSTDAGData(
0,
@ -207,13 +242,27 @@ func TestValidateTransactionInContextAndPopulateFee(t *testing.T) {
isValid: false,
expectedError: ruleerrors.ErrImmatureSpend,
},
{ // The total inputs amount is bigger than the allowed maximum (constants.MaxSompi)
name: "checkTransactionInputAmounts",
tx: &txWithLargeAmount,
{ // The total inputs amount is bigger than the allowed maximum (constants.MaxSompiBeforeHF1)
name: "checkTransactionInputAmounts - invalid - before HF",
tx: &txWithLargeAmountBeforeHF,
povBlockHash: povBlockHash,
isValid: false,
expectedError: ruleerrors.ErrBadTxOutValue,
},
{ // The total inputs amount is bigger than the allowed maximum (constants.MaxSompiBeforeHF1)
name: "checkTransactionInputAmounts - valid - after HF",
tx: &txWithLargeAmountBeforeHF,
povBlockHash: povAfterHFBlockHash,
isValid: true,
expectedError: nil,
},
{ // The total inputs amount is bigger than the allowed maximum (constants.MaxSompiBeforeHF1)
name: "checkTransactionInputAmounts - invalid - after HF",
tx: &txWithLargeAmountAfterHF,
povBlockHash: povAfterHFBlockHash,
isValid: false,
expectedError: ruleerrors.ErrBadTxOutValue,
},
{ // The total SompiIn (sum of inputs amount) is smaller than the total SompiOut (sum of outputs value) and hence invalid.
name: "checkTransactionOutputAmounts",
tx: &txWithBigValue,

View File

@ -240,6 +240,7 @@ var (
ErrPruningProofMissesBlocksBelowPruningPoint = newRuleError("ErrPruningProofMissesBlocksBelowPruningPoint")
ErrPruningProofEmpty = newRuleError("ErrPruningProofEmpty")
ErrWrongCoinbaseSubsidy = newRuleError("ErrWrongCoinbaseSubsidy")
ErrWrongBlockVersion = newRuleError("ErrWrongBlockVersion")
)
// RuleError identifies a rule violation. It is used to indicate that

View File

@ -3,9 +3,11 @@ package constants
import "math"
const (
// MaxBlockVersion represents the current version of blocks mined and the maximum block version
// this node is able to validate
MaxBlockVersion uint16 = 0
// BlockVersionBeforeHF1 represents the current version of blocks mined before HF1
BlockVersionBeforeHF1 uint16 = 0
// BlockVersionAfterHF1 represents the current version of blocks mined after HF1
BlockVersionAfterHF1 uint16 = 1
// MaxTransactionVersion is the current latest supported transaction version.
MaxTransactionVersion uint16 = 0
@ -16,8 +18,11 @@ const (
// SompiPerKaspa is the number of sompi in one kaspa (1 KAS).
SompiPerKaspa = 100_000_000
// MaxSompi is the maximum transaction amount allowed in sompi.
MaxSompi = 21_000_000 * SompiPerKaspa
// MaxSompiBeforeHF1 is the maximum transaction amount allowed in sompi before the HF1 hard fork is activated.
MaxSompiBeforeHF1 = uint64(21_000_000 * SompiPerKaspa)
// MaxSompiAfterHF1 is the maximum transaction amount allowed in sompi after the HF1 hard fork is activated.
MaxSompiAfterHF1 = uint64(29_000_000_000 * SompiPerKaspa)
// MaxTxInSequenceNum is the maximum sequence number the sequence field
// of a transaction input can be.

View File

@ -82,4 +82,6 @@ const (
// The network was down for three days shortly after launch
// Three days in seconds = 3 * 24 * 60 * 60 = 259200
defaultDeflationaryPhaseDaaScore = 15778800 - 259200
defaultMergeDepth = 3600
)

View File

@ -188,6 +188,9 @@ type Params struct {
// MaxBlockLevel is the maximum possible block level.
MaxBlockLevel int
MergeDepth uint64
HF1DAAScore uint64
}
// NormalizeRPCServerAddress returns addr with the current network default
@ -286,12 +289,14 @@ var MainnetParams = Params{
// This is technically 255, but we clamped it at 256 - block level of mainnet genesis
// This means that any block that has a level lower or equal to genesis will be level 0.
MaxBlockLevel: 225,
MergeDepth: defaultMergeDepth,
HF1DAAScore: 14360917,
}
// TestnetParams defines the network parameters for the test Kaspa network.
var TestnetParams = Params{
K: defaultGHOSTDAGK,
Name: "kaspa-testnet-9",
Name: "kaspa-testnet-10",
Net: appmessage.Testnet,
RPCPort: "16210",
DefaultPort: "16211",
@ -348,6 +353,8 @@ var TestnetParams = Params{
IgnoreHeaderMass: true,
MaxBlockLevel: 250,
MergeDepth: defaultMergeDepth,
HF1DAAScore: 172800,
}
// SimnetParams defines the network parameters for the simulation test Kaspa
@ -413,6 +420,7 @@ var SimnetParams = Params{
DeflationaryPhaseDaaScore: defaultDeflationaryPhaseDaaScore,
MaxBlockLevel: 250,
MergeDepth: defaultMergeDepth,
}
// DevnetParams defines the network parameters for the development Kaspa network.
@ -475,6 +483,7 @@ var DevnetParams = Params{
IgnoreHeaderMass: true,
MaxBlockLevel: 250,
MergeDepth: defaultMergeDepth,
}
// ErrDuplicateNet describes an error where the parameters for a Kaspa

View File

@ -198,8 +198,9 @@ func (mp *mempool) minimumRequiredTransactionRelayFee(mass uint64) uint64 {
// Set the minimum fee to the maximum possible value if the calculated
// fee is not in the valid range for monetary amounts.
if minimumFee > constants.MaxSompi {
minimumFee = constants.MaxSompi
// TODO: Replace it with constants.MaxSompiAfterHF1 once HF is activated
if minimumFee > constants.MaxSompiBeforeHF1 {
minimumFee = constants.MaxSompiBeforeHF1
}
return minimumFee

View File

@ -53,8 +53,8 @@ func TestCalcMinRequiredTxRelayFee(t *testing.T) {
{
"max standard tx size with max sompi relay fee",
MaximumStandardTransactionMass,
constants.MaxSompi,
constants.MaxSompi,
util.Amount(constants.MaxSompiBeforeHF1),
constants.MaxSompiBeforeHF1,
},
{
"1500 bytes with 5000 relay fee",
@ -156,8 +156,8 @@ func TestIsTransactionOutputDust(t *testing.T) {
{
// Maximum allowed value is never dust.
"max sompi amount is never dust",
externalapi.DomainTransactionOutput{Value: constants.MaxSompi, ScriptPublicKey: scriptPublicKey},
constants.MaxSompi,
externalapi.DomainTransactionOutput{Value: constants.MaxSompiBeforeHF1, ScriptPublicKey: scriptPublicKey},
util.Amount(constants.MaxSompiBeforeHF1),
false,
},
{

View File

@ -31,50 +31,3 @@ func TestAddressExchange(t *testing.T) {
t.Errorf("Didn't find testAddress in list of addresses of appHarness3")
}
func TestAddressExchangeV3V4(t *testing.T) {
harnesses, teardown := setupHarnesses(t, []*harnessParams{
{
p2pAddress: p2pAddress1,
rpcAddress: rpcAddress1,
miningAddress: miningAddress1,
miningAddressPrivateKey: miningAddress1PrivateKey,
},
{
p2pAddress: p2pAddress2,
rpcAddress: rpcAddress2,
miningAddress: miningAddress2,
miningAddressPrivateKey: miningAddress2PrivateKey,
}, {
p2pAddress: p2pAddress3,
rpcAddress: rpcAddress3,
miningAddress: miningAddress3,
miningAddressPrivateKey: miningAddress3PrivateKey,
},
})
defer teardown()
appHarness1, appHarness2, appHarness3 := harnesses[0], harnesses[1], harnesses[2]
testAddress := "1.2.3.4:6789"
err := addressmanager.AddAddressByIP(appHarness1.app.AddressManager(), testAddress, nil)
if err != nil {
t.Fatalf("Error adding address to addressManager: %+v", err)
}
connect(t, appHarness1, appHarness2)
connect(t, appHarness2, appHarness3)
peerAddresses, err := appHarness3.rpcClient.GetPeerAddresses()
if err != nil {
t.Fatalf("Error getting peer addresses: %+v", err)
}
for _, peerAddress := range peerAddresses.Addresses {
if peerAddress.Addr == testAddress {
return
}
}
t.Errorf("Didn't find testAddress in list of addresses of appHarness3")
}

View File

@ -238,3 +238,84 @@ func mineNextBlockWithMockTimestamps(t *testing.T, harness *appHarness, rd *rand
return block
}
func TestBoundedMergeDepth(t *testing.T) {
overrideDAGParams := dagconfig.SimnetParams
overrideDAGParams.MergeDepth = 50
harnesses, teardown := setupHarnesses(t, []*harnessParams{
{
p2pAddress: p2pAddress1,
rpcAddress: rpcAddress1,
miningAddress: miningAddress1,
miningAddressPrivateKey: miningAddress1PrivateKey,
overrideDAGParams: &overrideDAGParams,
},
{
p2pAddress: p2pAddress2,
rpcAddress: rpcAddress2,
miningAddress: miningAddress2,
miningAddressPrivateKey: miningAddress2PrivateKey,
overrideDAGParams: &overrideDAGParams,
utxoIndex: true,
},
{
p2pAddress: p2pAddress3,
rpcAddress: rpcAddress3,
miningAddress: miningAddress3,
miningAddressPrivateKey: miningAddress3PrivateKey,
overrideDAGParams: &overrideDAGParams,
utxoIndex: true,
},
{
p2pAddress: p2pAddress4,
rpcAddress: rpcAddress4,
miningAddress: miningAddress3,
miningAddressPrivateKey: miningAddress3PrivateKey,
overrideDAGParams: &overrideDAGParams,
utxoIndex: true,
},
})
defer teardown()
test := func(syncer, syncee *appHarness, depth uint64, shouldSync bool) {
const ibdTriggerRange = 32
if depth <= ibdTriggerRange {
t.Fatalf("Depth is too small")
}
for i := uint64(0); i < depth+ibdTriggerRange+1; i++ {
mineNextBlock(t, syncee)
}
for i := uint64(0); i < ibdTriggerRange+1; i++ {
mineNextBlock(t, syncer)
}
countBefore, err := syncee.rpcClient.GetBlockCount()
if err != nil {
t.Fatalf("GetBlockCount: %+v", err)
}
connect(t, syncee, syncer)
time.Sleep(5 * time.Second)
countAfter, err := syncee.rpcClient.GetBlockCount()
if err != nil {
t.Fatalf("GetBlockCount: %+v", err)
}
if (countBefore.HeaderCount != countAfter.HeaderCount) != shouldSync {
t.Fatalf("countBefore.HeaderCount: %d, countAfter.HeaderCount: %d", countBefore.HeaderCount, countAfter.HeaderCount)
}
}
t.Run("mergeDepth", func(t *testing.T) {
test(harnesses[0], harnesses[1], overrideDAGParams.MergeDepth, false)
})
t.Run("mergeDepth-1", func(t *testing.T) {
test(harnesses[2], harnesses[3], overrideDAGParams.MergeDepth-1, true)
})
}

View File

@ -73,6 +73,8 @@ func round(f float64) Amount {
// NewAmount is for specifically for converting KAS to Sompi.
// For creating a new Amount with an int64 value which denotes a quantity of Sompi,
// do a simple type conversion from type int64 to Amount.
// TODO: Refactor NewAmount. When amounts are more than 1e9 KAS, the precision
// can be higher than one sompi (1e9 and 1e9+1e-8 will result as the same number)
func NewAmount(f float64) (Amount, error) {
// The amount is only considered invalid if it cannot be represented
// as an integer type. This may happen if f is NaN or +-Infinity.

View File

@ -30,13 +30,13 @@ func TestAmountCreation(t *testing.T) {
name: "max producible",
amount: 21e6,
valid: true,
expected: constants.MaxSompi,
expected: Amount(constants.MaxSompiBeforeHF1),
},
{
name: "exceeds max producible",
amount: 21e6 + 1e-8,
valid: true,
expected: constants.MaxSompi + 1,
expected: Amount(constants.MaxSompiBeforeHF1) + 1,
},
{
name: "one hundred",
@ -109,7 +109,7 @@ func TestAmountUnitConversions(t *testing.T) {
}{
{
name: "MKAS",
amount: constants.MaxSompi,
amount: Amount(constants.MaxSompiBeforeHF1),
unit: AmountMegaKAS,
converted: 21,
s: "21 MKAS",

View File

@ -10,8 +10,8 @@ const validCharacters = "0123456789ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrs
const (
appMajor uint = 0
appMinor uint = 11
appPatch uint = 17
appMinor uint = 12
appPatch uint = 0
)
// appBuild is defined as a variable so it can be overridden during the build