mirror of
https://github.com/kaspanet/kaspad.git
synced 2025-06-06 14:16:43 +00:00
Apply ResolveVirtual diffs to the UTXO index (#1868)
* Apply ResolveVirtual diffs to the UTXO index * Add comments Co-authored-by: Ori Newman <>
This commit is contained in:
parent
11103a36d3
commit
32e8e539ac
@ -152,6 +152,7 @@ func setupRPC(
|
|||||||
utxoIndex,
|
utxoIndex,
|
||||||
shutDownChan,
|
shutDownChan,
|
||||||
)
|
)
|
||||||
|
protocolManager.SetOnVirtualChange(rpcManager.NotifyVirtualChange)
|
||||||
protocolManager.SetOnBlockAddedToDAGHandler(rpcManager.NotifyBlockAddedToDAG)
|
protocolManager.SetOnBlockAddedToDAGHandler(rpcManager.NotifyBlockAddedToDAG)
|
||||||
protocolManager.SetOnPruningPointUTXOSetOverrideHandler(rpcManager.NotifyPruningPointUTXOSetOverride)
|
protocolManager.SetOnPruningPointUTXOSetOverrideHandler(rpcManager.NotifyPruningPointUTXOSetOverride)
|
||||||
|
|
||||||
|
@ -18,7 +18,7 @@ import (
|
|||||||
// relays newly unorphaned transactions and possibly rebroadcast
|
// relays newly unorphaned transactions and possibly rebroadcast
|
||||||
// manually added transactions when not in IBD.
|
// manually added transactions when not in IBD.
|
||||||
func (f *FlowContext) OnNewBlock(block *externalapi.DomainBlock,
|
func (f *FlowContext) OnNewBlock(block *externalapi.DomainBlock,
|
||||||
blockInsertionResult *externalapi.BlockInsertionResult) error {
|
virtualChangeSet *externalapi.VirtualChangeSet) error {
|
||||||
|
|
||||||
hash := consensushashing.BlockHash(block)
|
hash := consensushashing.BlockHash(block)
|
||||||
log.Debugf("OnNewBlock start for block %s", hash)
|
log.Debugf("OnNewBlock start for block %s", hash)
|
||||||
@ -32,10 +32,10 @@ func (f *FlowContext) OnNewBlock(block *externalapi.DomainBlock,
|
|||||||
log.Debugf("OnNewBlock: block %s unorphaned %d blocks", hash, len(unorphaningResults))
|
log.Debugf("OnNewBlock: block %s unorphaned %d blocks", hash, len(unorphaningResults))
|
||||||
|
|
||||||
newBlocks := []*externalapi.DomainBlock{block}
|
newBlocks := []*externalapi.DomainBlock{block}
|
||||||
newBlockInsertionResults := []*externalapi.BlockInsertionResult{blockInsertionResult}
|
newVirtualChangeSets := []*externalapi.VirtualChangeSet{virtualChangeSet}
|
||||||
for _, unorphaningResult := range unorphaningResults {
|
for _, unorphaningResult := range unorphaningResults {
|
||||||
newBlocks = append(newBlocks, unorphaningResult.block)
|
newBlocks = append(newBlocks, unorphaningResult.block)
|
||||||
newBlockInsertionResults = append(newBlockInsertionResults, unorphaningResult.blockInsertionResult)
|
newVirtualChangeSets = append(newVirtualChangeSets, unorphaningResult.virtualChangeSet)
|
||||||
}
|
}
|
||||||
|
|
||||||
allAcceptedTransactions := make([]*externalapi.DomainTransaction, 0)
|
allAcceptedTransactions := make([]*externalapi.DomainTransaction, 0)
|
||||||
@ -49,8 +49,8 @@ func (f *FlowContext) OnNewBlock(block *externalapi.DomainBlock,
|
|||||||
|
|
||||||
if f.onBlockAddedToDAGHandler != nil {
|
if f.onBlockAddedToDAGHandler != nil {
|
||||||
log.Debugf("OnNewBlock: calling f.onBlockAddedToDAGHandler for block %s", hash)
|
log.Debugf("OnNewBlock: calling f.onBlockAddedToDAGHandler for block %s", hash)
|
||||||
blockInsertionResult = newBlockInsertionResults[i]
|
virtualChangeSet = newVirtualChangeSets[i]
|
||||||
err := f.onBlockAddedToDAGHandler(newBlock, blockInsertionResult)
|
err := f.onBlockAddedToDAGHandler(newBlock, virtualChangeSet)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
@ -60,6 +60,15 @@ func (f *FlowContext) OnNewBlock(block *externalapi.DomainBlock,
|
|||||||
return f.broadcastTransactionsAfterBlockAdded(newBlocks, allAcceptedTransactions)
|
return f.broadcastTransactionsAfterBlockAdded(newBlocks, allAcceptedTransactions)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// OnVirtualChange calls the handler function whenever the virtual block changes.
|
||||||
|
func (f *FlowContext) OnVirtualChange(virtualChangeSet *externalapi.VirtualChangeSet) error {
|
||||||
|
if f.onVirtualChangeHandler != nil && virtualChangeSet != nil {
|
||||||
|
return f.onVirtualChangeHandler(virtualChangeSet)
|
||||||
|
}
|
||||||
|
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
// OnPruningPointUTXOSetOverride calls the handler function whenever the UTXO set
|
// OnPruningPointUTXOSetOverride calls the handler function whenever the UTXO set
|
||||||
// resets due to pruning point change via IBD.
|
// resets due to pruning point change via IBD.
|
||||||
func (f *FlowContext) OnPruningPointUTXOSetOverride() error {
|
func (f *FlowContext) OnPruningPointUTXOSetOverride() error {
|
||||||
@ -110,14 +119,14 @@ func (f *FlowContext) AddBlock(block *externalapi.DomainBlock) error {
|
|||||||
return protocolerrors.Errorf(false, "cannot add header only block")
|
return protocolerrors.Errorf(false, "cannot add header only block")
|
||||||
}
|
}
|
||||||
|
|
||||||
blockInsertionResult, err := f.Domain().Consensus().ValidateAndInsertBlock(block, true)
|
virtualChangeSet, err := f.Domain().Consensus().ValidateAndInsertBlock(block, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
if errors.As(err, &ruleerrors.RuleError{}) {
|
if errors.As(err, &ruleerrors.RuleError{}) {
|
||||||
log.Warnf("Validation failed for block %s: %s", consensushashing.BlockHash(block), err)
|
log.Warnf("Validation failed for block %s: %s", consensushashing.BlockHash(block), err)
|
||||||
}
|
}
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
err = f.OnNewBlock(block, blockInsertionResult)
|
err = f.OnNewBlock(block, virtualChangeSet)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
@ -22,7 +22,10 @@ import (
|
|||||||
|
|
||||||
// OnBlockAddedToDAGHandler is a handler function that's triggered
|
// OnBlockAddedToDAGHandler is a handler function that's triggered
|
||||||
// when a block is added to the DAG
|
// when a block is added to the DAG
|
||||||
type OnBlockAddedToDAGHandler func(block *externalapi.DomainBlock, blockInsertionResult *externalapi.BlockInsertionResult) error
|
type OnBlockAddedToDAGHandler func(block *externalapi.DomainBlock, virtualChangeSet *externalapi.VirtualChangeSet) error
|
||||||
|
|
||||||
|
// OnVirtualChangeHandler is a handler function that's triggered when the virtual changes
|
||||||
|
type OnVirtualChangeHandler func(virtualChangeSet *externalapi.VirtualChangeSet) error
|
||||||
|
|
||||||
// OnPruningPointUTXOSetOverrideHandler is a handle function that's triggered whenever the UTXO set
|
// OnPruningPointUTXOSetOverrideHandler is a handle function that's triggered whenever the UTXO set
|
||||||
// resets due to pruning point change via IBD.
|
// resets due to pruning point change via IBD.
|
||||||
@ -43,6 +46,7 @@ type FlowContext struct {
|
|||||||
|
|
||||||
timeStarted int64
|
timeStarted int64
|
||||||
|
|
||||||
|
onVirtualChangeHandler OnVirtualChangeHandler
|
||||||
onBlockAddedToDAGHandler OnBlockAddedToDAGHandler
|
onBlockAddedToDAGHandler OnBlockAddedToDAGHandler
|
||||||
onPruningPointUTXOSetOverrideHandler OnPruningPointUTXOSetOverrideHandler
|
onPruningPointUTXOSetOverrideHandler OnPruningPointUTXOSetOverrideHandler
|
||||||
onTransactionAddedToMempoolHandler OnTransactionAddedToMempoolHandler
|
onTransactionAddedToMempoolHandler OnTransactionAddedToMempoolHandler
|
||||||
@ -100,6 +104,11 @@ func (f *FlowContext) ShutdownChan() <-chan struct{} {
|
|||||||
return f.shutdownChan
|
return f.shutdownChan
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// SetOnVirtualChangeHandler sets the onVirtualChangeHandler handler
|
||||||
|
func (f *FlowContext) SetOnVirtualChangeHandler(onVirtualChangeHandler OnVirtualChangeHandler) {
|
||||||
|
f.onVirtualChangeHandler = onVirtualChangeHandler
|
||||||
|
}
|
||||||
|
|
||||||
// SetOnBlockAddedToDAGHandler sets the onBlockAddedToDAG handler
|
// SetOnBlockAddedToDAGHandler sets the onBlockAddedToDAG handler
|
||||||
func (f *FlowContext) SetOnBlockAddedToDAGHandler(onBlockAddedToDAGHandler OnBlockAddedToDAGHandler) {
|
func (f *FlowContext) SetOnBlockAddedToDAGHandler(onBlockAddedToDAGHandler OnBlockAddedToDAGHandler) {
|
||||||
f.onBlockAddedToDAGHandler = onBlockAddedToDAGHandler
|
f.onBlockAddedToDAGHandler = onBlockAddedToDAGHandler
|
||||||
|
@ -17,8 +17,8 @@ const maxOrphans = 600
|
|||||||
|
|
||||||
// UnorphaningResult is the result of unorphaning a block
|
// UnorphaningResult is the result of unorphaning a block
|
||||||
type UnorphaningResult struct {
|
type UnorphaningResult struct {
|
||||||
block *externalapi.DomainBlock
|
block *externalapi.DomainBlock
|
||||||
blockInsertionResult *externalapi.BlockInsertionResult
|
virtualChangeSet *externalapi.VirtualChangeSet
|
||||||
}
|
}
|
||||||
|
|
||||||
// AddOrphan adds the block to the orphan set
|
// AddOrphan adds the block to the orphan set
|
||||||
@ -90,14 +90,14 @@ func (f *FlowContext) UnorphanBlocks(rootBlock *externalapi.DomainBlock) ([]*Uno
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
if canBeUnorphaned {
|
if canBeUnorphaned {
|
||||||
blockInsertionResult, unorphaningSucceeded, err := f.unorphanBlock(orphanHash)
|
virtualChangeSet, unorphaningSucceeded, err := f.unorphanBlock(orphanHash)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
if unorphaningSucceeded {
|
if unorphaningSucceeded {
|
||||||
unorphaningResults = append(unorphaningResults, &UnorphaningResult{
|
unorphaningResults = append(unorphaningResults, &UnorphaningResult{
|
||||||
block: orphanBlock,
|
block: orphanBlock,
|
||||||
blockInsertionResult: blockInsertionResult,
|
virtualChangeSet: virtualChangeSet,
|
||||||
})
|
})
|
||||||
processQueue = f.addChildOrphansToProcessQueue(&orphanHash, processQueue)
|
processQueue = f.addChildOrphansToProcessQueue(&orphanHash, processQueue)
|
||||||
}
|
}
|
||||||
@ -143,14 +143,14 @@ func (f *FlowContext) findChildOrphansOfBlock(blockHash *externalapi.DomainHash)
|
|||||||
return childOrphans
|
return childOrphans
|
||||||
}
|
}
|
||||||
|
|
||||||
func (f *FlowContext) unorphanBlock(orphanHash externalapi.DomainHash) (*externalapi.BlockInsertionResult, bool, error) {
|
func (f *FlowContext) unorphanBlock(orphanHash externalapi.DomainHash) (*externalapi.VirtualChangeSet, bool, error) {
|
||||||
orphanBlock, ok := f.orphans[orphanHash]
|
orphanBlock, ok := f.orphans[orphanHash]
|
||||||
if !ok {
|
if !ok {
|
||||||
return nil, false, errors.Errorf("attempted to unorphan a non-orphan block %s", orphanHash)
|
return nil, false, errors.Errorf("attempted to unorphan a non-orphan block %s", orphanHash)
|
||||||
}
|
}
|
||||||
delete(f.orphans, orphanHash)
|
delete(f.orphans, orphanHash)
|
||||||
|
|
||||||
blockInsertionResult, err := f.domain.Consensus().ValidateAndInsertBlock(orphanBlock, true)
|
virtualChangeSet, err := f.domain.Consensus().ValidateAndInsertBlock(orphanBlock, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
if errors.As(err, &ruleerrors.RuleError{}) {
|
if errors.As(err, &ruleerrors.RuleError{}) {
|
||||||
log.Warnf("Validation failed for orphan block %s: %s", orphanHash, err)
|
log.Warnf("Validation failed for orphan block %s: %s", orphanHash, err)
|
||||||
@ -160,7 +160,7 @@ func (f *FlowContext) unorphanBlock(orphanHash externalapi.DomainHash) (*externa
|
|||||||
}
|
}
|
||||||
|
|
||||||
log.Infof("Unorphaned block %s", orphanHash)
|
log.Infof("Unorphaned block %s", orphanHash)
|
||||||
return blockInsertionResult, true, nil
|
return virtualChangeSet, true, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// GetOrphanRoots returns the roots of the missing ancestors DAG of the given orphan
|
// GetOrphanRoots returns the roots of the missing ancestors DAG of the given orphan
|
||||||
|
@ -23,7 +23,8 @@ var orphanResolutionRange uint32 = 5
|
|||||||
type RelayInvsContext interface {
|
type RelayInvsContext interface {
|
||||||
Domain() domain.Domain
|
Domain() domain.Domain
|
||||||
Config() *config.Config
|
Config() *config.Config
|
||||||
OnNewBlock(block *externalapi.DomainBlock, blockInsertionResult *externalapi.BlockInsertionResult) error
|
OnNewBlock(block *externalapi.DomainBlock, virtualChangeSet *externalapi.VirtualChangeSet) error
|
||||||
|
OnVirtualChange(virtualChangeSet *externalapi.VirtualChangeSet) error
|
||||||
OnPruningPointUTXOSetOverride() error
|
OnPruningPointUTXOSetOverride() error
|
||||||
SharedRequestedBlocks() *SharedRequestedBlocks
|
SharedRequestedBlocks() *SharedRequestedBlocks
|
||||||
Broadcast(message appmessage.Message) error
|
Broadcast(message appmessage.Message) error
|
||||||
@ -128,7 +129,7 @@ func (flow *handleRelayInvsFlow) start() error {
|
|||||||
}
|
}
|
||||||
|
|
||||||
log.Debugf("Processing block %s", inv.Hash)
|
log.Debugf("Processing block %s", inv.Hash)
|
||||||
missingParents, blockInsertionResult, err := flow.processBlock(block)
|
missingParents, virtualChangeSet, err := flow.processBlock(block)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
if errors.Is(err, ruleerrors.ErrPrunedBlock) {
|
if errors.Is(err, ruleerrors.ErrPrunedBlock) {
|
||||||
log.Infof("Ignoring pruned block %s", inv.Hash)
|
log.Infof("Ignoring pruned block %s", inv.Hash)
|
||||||
@ -156,7 +157,7 @@ func (flow *handleRelayInvsFlow) start() error {
|
|||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
log.Infof("Accepted block %s via relay", inv.Hash)
|
log.Infof("Accepted block %s via relay", inv.Hash)
|
||||||
err = flow.OnNewBlock(block, blockInsertionResult)
|
err = flow.OnNewBlock(block, virtualChangeSet)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
@ -243,9 +244,9 @@ func (flow *handleRelayInvsFlow) readMsgBlock() (msgBlock *appmessage.MsgBlock,
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (flow *handleRelayInvsFlow) processBlock(block *externalapi.DomainBlock) ([]*externalapi.DomainHash, *externalapi.BlockInsertionResult, error) {
|
func (flow *handleRelayInvsFlow) processBlock(block *externalapi.DomainBlock) ([]*externalapi.DomainHash, *externalapi.VirtualChangeSet, error) {
|
||||||
blockHash := consensushashing.BlockHash(block)
|
blockHash := consensushashing.BlockHash(block)
|
||||||
blockInsertionResult, err := flow.Domain().Consensus().ValidateAndInsertBlock(block, true)
|
virtualChangeSet, err := flow.Domain().Consensus().ValidateAndInsertBlock(block, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
if !errors.As(err, &ruleerrors.RuleError{}) {
|
if !errors.As(err, &ruleerrors.RuleError{}) {
|
||||||
return nil, nil, errors.Wrapf(err, "failed to process block %s", blockHash)
|
return nil, nil, errors.Wrapf(err, "failed to process block %s", blockHash)
|
||||||
@ -258,7 +259,7 @@ func (flow *handleRelayInvsFlow) processBlock(block *externalapi.DomainBlock) ([
|
|||||||
log.Warnf("Rejected block %s from %s: %s", blockHash, flow.peer, err)
|
log.Warnf("Rejected block %s from %s: %s", blockHash, flow.peer, err)
|
||||||
return nil, nil, protocolerrors.Wrapf(true, err, "got invalid block %s from relay", blockHash)
|
return nil, nil, protocolerrors.Wrapf(true, err, "got invalid block %s from relay", blockHash)
|
||||||
}
|
}
|
||||||
return nil, blockInsertionResult, nil
|
return nil, virtualChangeSet, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (flow *handleRelayInvsFlow) relayBlock(block *externalapi.DomainBlock) error {
|
func (flow *handleRelayInvsFlow) relayBlock(block *externalapi.DomainBlock) error {
|
||||||
|
@ -471,7 +471,7 @@ func (flow *handleRelayInvsFlow) syncMissingBlockBodies(highHash *externalapi.Do
|
|||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
blockInsertionResult, err := flow.Domain().Consensus().ValidateAndInsertBlock(block, false)
|
virtualChangeSet, err := flow.Domain().Consensus().ValidateAndInsertBlock(block, false)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
if errors.Is(err, ruleerrors.ErrDuplicateBlock) {
|
if errors.Is(err, ruleerrors.ErrDuplicateBlock) {
|
||||||
log.Debugf("Skipping IBD Block %s as it has already been added to the DAG", blockHash)
|
log.Debugf("Skipping IBD Block %s as it has already been added to the DAG", blockHash)
|
||||||
@ -479,14 +479,36 @@ func (flow *handleRelayInvsFlow) syncMissingBlockBodies(highHash *externalapi.Do
|
|||||||
}
|
}
|
||||||
return protocolerrors.ConvertToBanningProtocolErrorIfRuleError(err, "invalid block %s", blockHash)
|
return protocolerrors.ConvertToBanningProtocolErrorIfRuleError(err, "invalid block %s", blockHash)
|
||||||
}
|
}
|
||||||
err = flow.OnNewBlock(block, blockInsertionResult)
|
err = flow.OnNewBlock(block, virtualChangeSet)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
return flow.Domain().Consensus().ResolveVirtual()
|
return flow.resolveVirtual()
|
||||||
|
}
|
||||||
|
|
||||||
|
func (flow *handleRelayInvsFlow) resolveVirtual() error {
|
||||||
|
for i := 0; ; i++ {
|
||||||
|
if i%10 == 0 {
|
||||||
|
log.Infof("Resolving virtual. This may take some time...")
|
||||||
|
}
|
||||||
|
virtualChangeSet, isCompletelyResolved, err := flow.Domain().Consensus().ResolveVirtual()
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
|
err = flow.OnVirtualChange(virtualChangeSet)
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
|
if isCompletelyResolved {
|
||||||
|
log.Infof("Resolved virtual")
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// dequeueIncomingMessageAndSkipInvs is a convenience method to be used during
|
// dequeueIncomingMessageAndSkipInvs is a convenience method to be used during
|
||||||
|
@ -84,6 +84,11 @@ func (m *Manager) runFlows(flows []*flow, peer *peerpkg.Peer, errChan <-chan err
|
|||||||
return <-errChan
|
return <-errChan
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// SetOnVirtualChange sets the onVirtualChangeHandler handler
|
||||||
|
func (m *Manager) SetOnVirtualChange(onVirtualChangeHandler flowcontext.OnVirtualChangeHandler) {
|
||||||
|
m.context.SetOnVirtualChangeHandler(onVirtualChangeHandler)
|
||||||
|
}
|
||||||
|
|
||||||
// SetOnBlockAddedToDAGHandler sets the onBlockAddedToDAG handler
|
// SetOnBlockAddedToDAGHandler sets the onBlockAddedToDAG handler
|
||||||
func (m *Manager) SetOnBlockAddedToDAGHandler(onBlockAddedToDAGHandler flowcontext.OnBlockAddedToDAGHandler) {
|
func (m *Manager) SetOnBlockAddedToDAGHandler(onBlockAddedToDAGHandler flowcontext.OnBlockAddedToDAGHandler) {
|
||||||
m.context.SetOnBlockAddedToDAGHandler(onBlockAddedToDAGHandler)
|
m.context.SetOnBlockAddedToDAGHandler(onBlockAddedToDAGHandler)
|
||||||
|
@ -48,12 +48,31 @@ func NewManager(
|
|||||||
}
|
}
|
||||||
|
|
||||||
// NotifyBlockAddedToDAG notifies the manager that a block has been added to the DAG
|
// NotifyBlockAddedToDAG notifies the manager that a block has been added to the DAG
|
||||||
func (m *Manager) NotifyBlockAddedToDAG(block *externalapi.DomainBlock, blockInsertionResult *externalapi.BlockInsertionResult) error {
|
func (m *Manager) NotifyBlockAddedToDAG(block *externalapi.DomainBlock, virtualChangeSet *externalapi.VirtualChangeSet) error {
|
||||||
|
onEnd := logger.LogAndMeasureExecutionTime(log, "RPCManager.NotifyBlockAddedToDAG")
|
||||||
|
defer onEnd()
|
||||||
|
|
||||||
|
err := m.NotifyVirtualChange(virtualChangeSet)
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
|
rpcBlock := appmessage.DomainBlockToRPCBlock(block)
|
||||||
|
err = m.context.PopulateBlockWithVerboseData(rpcBlock, block.Header, block, false)
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
blockAddedNotification := appmessage.NewBlockAddedNotificationMessage(rpcBlock)
|
||||||
|
return m.context.NotificationManager.NotifyBlockAdded(blockAddedNotification)
|
||||||
|
}
|
||||||
|
|
||||||
|
// NotifyVirtualChange notifies the manager that the virtual block has been changed.
|
||||||
|
func (m *Manager) NotifyVirtualChange(virtualChangeSet *externalapi.VirtualChangeSet) error {
|
||||||
onEnd := logger.LogAndMeasureExecutionTime(log, "RPCManager.NotifyBlockAddedToDAG")
|
onEnd := logger.LogAndMeasureExecutionTime(log, "RPCManager.NotifyBlockAddedToDAG")
|
||||||
defer onEnd()
|
defer onEnd()
|
||||||
|
|
||||||
if m.context.Config.UTXOIndex {
|
if m.context.Config.UTXOIndex {
|
||||||
err := m.notifyUTXOsChanged(blockInsertionResult)
|
err := m.notifyUTXOsChanged(virtualChangeSet)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
@ -69,18 +88,12 @@ func (m *Manager) NotifyBlockAddedToDAG(block *externalapi.DomainBlock, blockIns
|
|||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
err = m.notifyVirtualSelectedParentChainChanged(blockInsertionResult)
|
err = m.notifyVirtualSelectedParentChainChanged(virtualChangeSet)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
rpcBlock := appmessage.DomainBlockToRPCBlock(block)
|
return nil
|
||||||
err = m.context.PopulateBlockWithVerboseData(rpcBlock, block.Header, block, false)
|
|
||||||
if err != nil {
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
blockAddedNotification := appmessage.NewBlockAddedNotificationMessage(rpcBlock)
|
|
||||||
return m.context.NotificationManager.NotifyBlockAdded(blockAddedNotification)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// NotifyPruningPointUTXOSetOverride notifies the manager whenever the UTXO index
|
// NotifyPruningPointUTXOSetOverride notifies the manager whenever the UTXO index
|
||||||
@ -117,11 +130,11 @@ func (m *Manager) NotifyFinalityConflictResolved(finalityBlockHash string) error
|
|||||||
return m.context.NotificationManager.NotifyFinalityConflictResolved(notification)
|
return m.context.NotificationManager.NotifyFinalityConflictResolved(notification)
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *Manager) notifyUTXOsChanged(blockInsertionResult *externalapi.BlockInsertionResult) error {
|
func (m *Manager) notifyUTXOsChanged(virtualChangeSet *externalapi.VirtualChangeSet) error {
|
||||||
onEnd := logger.LogAndMeasureExecutionTime(log, "RPCManager.NotifyUTXOsChanged")
|
onEnd := logger.LogAndMeasureExecutionTime(log, "RPCManager.NotifyUTXOsChanged")
|
||||||
defer onEnd()
|
defer onEnd()
|
||||||
|
|
||||||
utxoIndexChanges, err := m.context.UTXOIndex.Update(blockInsertionResult)
|
utxoIndexChanges, err := m.context.UTXOIndex.Update(virtualChangeSet)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
@ -171,12 +184,12 @@ func (m *Manager) notifyVirtualDaaScoreChanged() error {
|
|||||||
return m.context.NotificationManager.NotifyVirtualDaaScoreChanged(notification)
|
return m.context.NotificationManager.NotifyVirtualDaaScoreChanged(notification)
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *Manager) notifyVirtualSelectedParentChainChanged(blockInsertionResult *externalapi.BlockInsertionResult) error {
|
func (m *Manager) notifyVirtualSelectedParentChainChanged(virtualChangeSet *externalapi.VirtualChangeSet) error {
|
||||||
onEnd := logger.LogAndMeasureExecutionTime(log, "RPCManager.NotifyVirtualSelectedParentChainChanged")
|
onEnd := logger.LogAndMeasureExecutionTime(log, "RPCManager.NotifyVirtualSelectedParentChainChanged")
|
||||||
defer onEnd()
|
defer onEnd()
|
||||||
|
|
||||||
notification, err := m.context.ConvertVirtualSelectedParentChainChangesToChainChangedNotificationMessage(
|
notification, err := m.context.ConvertVirtualSelectedParentChainChangesToChainChangedNotificationMessage(
|
||||||
blockInsertionResult.VirtualSelectedParentChainChanges)
|
virtualChangeSet.VirtualSelectedParentChainChanges)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
@ -163,7 +163,7 @@ func TestMultisig(t *testing.T) {
|
|||||||
t.Fatalf("Expected extractedSignedTxOneStep and extractedSignedTxStep2 IDs to be equal")
|
t.Fatalf("Expected extractedSignedTxOneStep and extractedSignedTxStep2 IDs to be equal")
|
||||||
}
|
}
|
||||||
|
|
||||||
_, insertionResult, err := tc.AddBlock([]*externalapi.DomainHash{block1Hash}, nil, []*externalapi.DomainTransaction{extractedSignedTxStep2})
|
_, virtualChangeSet, err := tc.AddBlock([]*externalapi.DomainHash{block1Hash}, nil, []*externalapi.DomainTransaction{extractedSignedTxStep2})
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("AddBlock: %+v", err)
|
t.Fatalf("AddBlock: %+v", err)
|
||||||
}
|
}
|
||||||
@ -172,7 +172,7 @@ func TestMultisig(t *testing.T) {
|
|||||||
TransactionID: *consensushashing.TransactionID(extractedSignedTxStep2),
|
TransactionID: *consensushashing.TransactionID(extractedSignedTxStep2),
|
||||||
Index: 0,
|
Index: 0,
|
||||||
}
|
}
|
||||||
if !insertionResult.VirtualUTXODiff.ToAdd().Contains(addedUTXO) {
|
if !virtualChangeSet.VirtualUTXODiff.ToAdd().Contains(addedUTXO) {
|
||||||
t.Fatalf("Transaction wasn't accepted in the DAG")
|
t.Fatalf("Transaction wasn't accepted in the DAG")
|
||||||
}
|
}
|
||||||
})
|
})
|
||||||
@ -294,7 +294,7 @@ func TestP2PK(t *testing.T) {
|
|||||||
t.Fatalf("ExtractTransaction: %+v", err)
|
t.Fatalf("ExtractTransaction: %+v", err)
|
||||||
}
|
}
|
||||||
|
|
||||||
_, insertionResult, err := tc.AddBlock([]*externalapi.DomainHash{block1Hash}, nil, []*externalapi.DomainTransaction{tx})
|
_, virtualChangeSet, err := tc.AddBlock([]*externalapi.DomainHash{block1Hash}, nil, []*externalapi.DomainTransaction{tx})
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("AddBlock: %+v", err)
|
t.Fatalf("AddBlock: %+v", err)
|
||||||
}
|
}
|
||||||
@ -303,7 +303,7 @@ func TestP2PK(t *testing.T) {
|
|||||||
TransactionID: *consensushashing.TransactionID(tx),
|
TransactionID: *consensushashing.TransactionID(tx),
|
||||||
Index: 0,
|
Index: 0,
|
||||||
}
|
}
|
||||||
if !insertionResult.VirtualUTXODiff.ToAdd().Contains(addedUTXO) {
|
if !virtualChangeSet.VirtualUTXODiff.ToAdd().Contains(addedUTXO) {
|
||||||
t.Fatalf("Transaction wasn't accepted in the DAG")
|
t.Fatalf("Transaction wasn't accepted in the DAG")
|
||||||
}
|
}
|
||||||
})
|
})
|
||||||
|
@ -56,7 +56,7 @@ type consensus struct {
|
|||||||
daaBlocksStore model.DAABlocksStore
|
daaBlocksStore model.DAABlocksStore
|
||||||
}
|
}
|
||||||
|
|
||||||
func (s *consensus) ValidateAndInsertBlockWithTrustedData(block *externalapi.BlockWithTrustedData, validateUTXO bool) (*externalapi.BlockInsertionResult, error) {
|
func (s *consensus) ValidateAndInsertBlockWithTrustedData(block *externalapi.BlockWithTrustedData, validateUTXO bool) (*externalapi.VirtualChangeSet, error) {
|
||||||
s.lock.Lock()
|
s.lock.Lock()
|
||||||
defer s.lock.Unlock()
|
defer s.lock.Unlock()
|
||||||
|
|
||||||
@ -164,7 +164,7 @@ func (s *consensus) BuildBlock(coinbaseData *externalapi.DomainCoinbaseData,
|
|||||||
|
|
||||||
// ValidateAndInsertBlock validates the given block and, if valid, applies it
|
// ValidateAndInsertBlock validates the given block and, if valid, applies it
|
||||||
// to the current state
|
// to the current state
|
||||||
func (s *consensus) ValidateAndInsertBlock(block *externalapi.DomainBlock, shouldValidateAgainstUTXO bool) (*externalapi.BlockInsertionResult, error) {
|
func (s *consensus) ValidateAndInsertBlock(block *externalapi.DomainBlock, shouldValidateAgainstUTXO bool) (*externalapi.VirtualChangeSet, error) {
|
||||||
s.lock.Lock()
|
s.lock.Lock()
|
||||||
defer s.lock.Unlock()
|
defer s.lock.Unlock()
|
||||||
|
|
||||||
@ -720,30 +720,13 @@ func (s *consensus) PopulateMass(transaction *externalapi.DomainTransaction) {
|
|||||||
s.transactionValidator.PopulateMass(transaction)
|
s.transactionValidator.PopulateMass(transaction)
|
||||||
}
|
}
|
||||||
|
|
||||||
func (s *consensus) ResolveVirtual() error {
|
func (s *consensus) ResolveVirtual() (*externalapi.VirtualChangeSet, bool, error) {
|
||||||
|
s.lock.Lock()
|
||||||
|
defer s.lock.Unlock()
|
||||||
|
|
||||||
// In order to prevent a situation that the consensus lock is held for too much time, we
|
// In order to prevent a situation that the consensus lock is held for too much time, we
|
||||||
// release the lock each time resolve 100 blocks.
|
// release the lock each time resolve 100 blocks.
|
||||||
for i := 0; ; i++ {
|
return s.consensusStateManager.ResolveVirtual(100)
|
||||||
if i%10 == 0 {
|
|
||||||
log.Infof("Resolving virtual. This may take some time...")
|
|
||||||
}
|
|
||||||
var isCompletelyResolved bool
|
|
||||||
var err error
|
|
||||||
func() {
|
|
||||||
s.lock.Lock()
|
|
||||||
defer s.lock.Unlock()
|
|
||||||
|
|
||||||
isCompletelyResolved, err = s.consensusStateManager.ResolveVirtual(100)
|
|
||||||
}()
|
|
||||||
if err != nil {
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
|
|
||||||
if isCompletelyResolved {
|
|
||||||
log.Infof("Resolved virtual")
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
func (s *consensus) BuildPruningPointProof() (*externalapi.PruningPointProof, error) {
|
func (s *consensus) BuildPruningPointProof() (*externalapi.PruningPointProof, error) {
|
||||||
|
@ -4,8 +4,8 @@ package externalapi
|
|||||||
type Consensus interface {
|
type Consensus interface {
|
||||||
Init(skipAddingGenesis bool) error
|
Init(skipAddingGenesis bool) error
|
||||||
BuildBlock(coinbaseData *DomainCoinbaseData, transactions []*DomainTransaction) (*DomainBlock, error)
|
BuildBlock(coinbaseData *DomainCoinbaseData, transactions []*DomainTransaction) (*DomainBlock, error)
|
||||||
ValidateAndInsertBlock(block *DomainBlock, shouldValidateAgainstUTXO bool) (*BlockInsertionResult, error)
|
ValidateAndInsertBlock(block *DomainBlock, shouldValidateAgainstUTXO bool) (*VirtualChangeSet, error)
|
||||||
ValidateAndInsertBlockWithTrustedData(block *BlockWithTrustedData, validateUTXO bool) (*BlockInsertionResult, error)
|
ValidateAndInsertBlockWithTrustedData(block *BlockWithTrustedData, validateUTXO bool) (*VirtualChangeSet, error)
|
||||||
ValidateTransactionAndPopulateWithConsensusData(transaction *DomainTransaction) error
|
ValidateTransactionAndPopulateWithConsensusData(transaction *DomainTransaction) error
|
||||||
ImportPruningPoints(pruningPoints []BlockHeader) error
|
ImportPruningPoints(pruningPoints []BlockHeader) error
|
||||||
BuildPruningPointProof() (*PruningPointProof, error)
|
BuildPruningPointProof() (*PruningPointProof, error)
|
||||||
@ -46,5 +46,5 @@ type Consensus interface {
|
|||||||
Anticone(blockHash *DomainHash) ([]*DomainHash, error)
|
Anticone(blockHash *DomainHash) ([]*DomainHash, error)
|
||||||
EstimateNetworkHashesPerSecond(startHash *DomainHash, windowSize int) (uint64, error)
|
EstimateNetworkHashesPerSecond(startHash *DomainHash, windowSize int) (uint64, error)
|
||||||
PopulateMass(transaction *DomainTransaction)
|
PopulateMass(transaction *DomainTransaction)
|
||||||
ResolveVirtual() error
|
ResolveVirtual() (*VirtualChangeSet, bool, error)
|
||||||
}
|
}
|
||||||
|
@ -1,7 +1,7 @@
|
|||||||
package externalapi
|
package externalapi
|
||||||
|
|
||||||
// BlockInsertionResult is auxiliary data returned from ValidateAndInsertBlock
|
// VirtualChangeSet is auxiliary data returned from ValidateAndInsertBlock and ResolveVirtual
|
||||||
type BlockInsertionResult struct {
|
type VirtualChangeSet struct {
|
||||||
VirtualSelectedParentChainChanges *SelectedChainPath
|
VirtualSelectedParentChainChanges *SelectedChainPath
|
||||||
VirtualUTXODiff UTXODiff
|
VirtualUTXODiff UTXODiff
|
||||||
VirtualParents []*DomainHash
|
VirtualParents []*DomainHash
|
@ -4,7 +4,7 @@ import "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
|
|||||||
|
|
||||||
// BlockProcessor is responsible for processing incoming blocks
|
// BlockProcessor is responsible for processing incoming blocks
|
||||||
type BlockProcessor interface {
|
type BlockProcessor interface {
|
||||||
ValidateAndInsertBlock(block *externalapi.DomainBlock, shouldValidateAgainstUTXO bool) (*externalapi.BlockInsertionResult, error)
|
ValidateAndInsertBlock(block *externalapi.DomainBlock, shouldValidateAgainstUTXO bool) (*externalapi.VirtualChangeSet, error)
|
||||||
ValidateAndInsertImportedPruningPoint(newPruningPoint *externalapi.DomainHash) error
|
ValidateAndInsertImportedPruningPoint(newPruningPoint *externalapi.DomainHash) error
|
||||||
ValidateAndInsertBlockWithTrustedData(block *externalapi.BlockWithTrustedData, validateUTXO bool) (*externalapi.BlockInsertionResult, error)
|
ValidateAndInsertBlockWithTrustedData(block *externalapi.BlockWithTrustedData, validateUTXO bool) (*externalapi.VirtualChangeSet, error)
|
||||||
}
|
}
|
||||||
|
@ -13,5 +13,5 @@ type ConsensusStateManager interface {
|
|||||||
GetVirtualSelectedParentChainFromBlock(stagingArea *StagingArea, blockHash *externalapi.DomainHash) (*externalapi.SelectedChainPath, error)
|
GetVirtualSelectedParentChainFromBlock(stagingArea *StagingArea, blockHash *externalapi.DomainHash) (*externalapi.SelectedChainPath, error)
|
||||||
RecoverUTXOIfRequired() error
|
RecoverUTXOIfRequired() error
|
||||||
ReverseUTXODiffs(tipHash *externalapi.DomainHash, reversalData *UTXODiffReversalData) error
|
ReverseUTXODiffs(tipHash *externalapi.DomainHash, reversalData *UTXODiffReversalData) error
|
||||||
ResolveVirtual(maxBlocksToResolve uint64) (bool, error)
|
ResolveVirtual(maxBlocksToResolve uint64) (*externalapi.VirtualChangeSet, bool, error)
|
||||||
}
|
}
|
||||||
|
@ -41,12 +41,12 @@ type TestConsensus interface {
|
|||||||
// AddBlock builds a block with given information, solves it, and adds to the DAG.
|
// AddBlock builds a block with given information, solves it, and adds to the DAG.
|
||||||
// Returns the hash of the added block
|
// Returns the hash of the added block
|
||||||
AddBlock(parentHashes []*externalapi.DomainHash, coinbaseData *externalapi.DomainCoinbaseData,
|
AddBlock(parentHashes []*externalapi.DomainHash, coinbaseData *externalapi.DomainCoinbaseData,
|
||||||
transactions []*externalapi.DomainTransaction) (*externalapi.DomainHash, *externalapi.BlockInsertionResult, error)
|
transactions []*externalapi.DomainTransaction) (*externalapi.DomainHash, *externalapi.VirtualChangeSet, error)
|
||||||
|
|
||||||
AddUTXOInvalidHeader(parentHashes []*externalapi.DomainHash) (*externalapi.DomainHash, *externalapi.BlockInsertionResult, error)
|
AddUTXOInvalidHeader(parentHashes []*externalapi.DomainHash) (*externalapi.DomainHash, *externalapi.VirtualChangeSet, error)
|
||||||
|
|
||||||
AddUTXOInvalidBlock(parentHashes []*externalapi.DomainHash) (*externalapi.DomainHash,
|
AddUTXOInvalidBlock(parentHashes []*externalapi.DomainHash) (*externalapi.DomainHash,
|
||||||
*externalapi.BlockInsertionResult, error)
|
*externalapi.VirtualChangeSet, error)
|
||||||
|
|
||||||
MineJSON(r io.Reader, blockType MineJSONBlockType) (tips []*externalapi.DomainHash, err error)
|
MineJSON(r io.Reader, blockType MineJSONBlockType) (tips []*externalapi.DomainHash, err error)
|
||||||
|
|
||||||
|
@ -143,7 +143,7 @@ func New(
|
|||||||
|
|
||||||
// ValidateAndInsertBlock validates the given block and, if valid, applies it
|
// ValidateAndInsertBlock validates the given block and, if valid, applies it
|
||||||
// to the current state
|
// to the current state
|
||||||
func (bp *blockProcessor) ValidateAndInsertBlock(block *externalapi.DomainBlock, shouldValidateAgainstUTXO bool) (*externalapi.BlockInsertionResult, error) {
|
func (bp *blockProcessor) ValidateAndInsertBlock(block *externalapi.DomainBlock, shouldValidateAgainstUTXO bool) (*externalapi.VirtualChangeSet, error) {
|
||||||
onEnd := logger.LogAndMeasureExecutionTime(log, "ValidateAndInsertBlock")
|
onEnd := logger.LogAndMeasureExecutionTime(log, "ValidateAndInsertBlock")
|
||||||
defer onEnd()
|
defer onEnd()
|
||||||
|
|
||||||
@ -159,7 +159,7 @@ func (bp *blockProcessor) ValidateAndInsertImportedPruningPoint(newPruningPoint
|
|||||||
return bp.validateAndInsertImportedPruningPoint(stagingArea, newPruningPoint)
|
return bp.validateAndInsertImportedPruningPoint(stagingArea, newPruningPoint)
|
||||||
}
|
}
|
||||||
|
|
||||||
func (bp *blockProcessor) ValidateAndInsertBlockWithTrustedData(block *externalapi.BlockWithTrustedData, shouldValidateAgainstUTXO bool) (*externalapi.BlockInsertionResult, error) {
|
func (bp *blockProcessor) ValidateAndInsertBlockWithTrustedData(block *externalapi.BlockWithTrustedData, shouldValidateAgainstUTXO bool) (*externalapi.VirtualChangeSet, error) {
|
||||||
onEnd := logger.LogAndMeasureExecutionTime(log, "ValidateAndInsertBlockWithTrustedData")
|
onEnd := logger.LogAndMeasureExecutionTime(log, "ValidateAndInsertBlockWithTrustedData")
|
||||||
defer onEnd()
|
defer onEnd()
|
||||||
|
|
||||||
|
@ -77,7 +77,7 @@ func (bp *blockProcessor) updateVirtualAcceptanceDataAfterImportingPruningPoint(
|
|||||||
}
|
}
|
||||||
|
|
||||||
func (bp *blockProcessor) validateAndInsertBlock(stagingArea *model.StagingArea, block *externalapi.DomainBlock,
|
func (bp *blockProcessor) validateAndInsertBlock(stagingArea *model.StagingArea, block *externalapi.DomainBlock,
|
||||||
isPruningPoint bool, shouldValidateAgainstUTXO bool, isBlockWithTrustedData bool) (*externalapi.BlockInsertionResult, error) {
|
isPruningPoint bool, shouldValidateAgainstUTXO bool, isBlockWithTrustedData bool) (*externalapi.VirtualChangeSet, error) {
|
||||||
|
|
||||||
blockHash := consensushashing.HeaderHash(block.Header)
|
blockHash := consensushashing.HeaderHash(block.Header)
|
||||||
err := bp.validateBlock(stagingArea, block, isBlockWithTrustedData)
|
err := bp.validateBlock(stagingArea, block, isBlockWithTrustedData)
|
||||||
@ -210,7 +210,7 @@ func (bp *blockProcessor) validateAndInsertBlock(stagingArea *model.StagingArea,
|
|||||||
|
|
||||||
bp.blockLogger.LogBlock(block)
|
bp.blockLogger.LogBlock(block)
|
||||||
|
|
||||||
return &externalapi.BlockInsertionResult{
|
return &externalapi.VirtualChangeSet{
|
||||||
VirtualSelectedParentChainChanges: selectedParentChainChanges,
|
VirtualSelectedParentChainChanges: selectedParentChainChanges,
|
||||||
VirtualUTXODiff: virtualUTXODiff,
|
VirtualUTXODiff: virtualUTXODiff,
|
||||||
VirtualParents: virtualParents,
|
VirtualParents: virtualParents,
|
||||||
|
@ -8,7 +8,7 @@ import (
|
|||||||
)
|
)
|
||||||
|
|
||||||
func (bp *blockProcessor) validateAndInsertBlockWithTrustedData(stagingArea *model.StagingArea,
|
func (bp *blockProcessor) validateAndInsertBlockWithTrustedData(stagingArea *model.StagingArea,
|
||||||
block *externalapi.BlockWithTrustedData, validateUTXO bool) (*externalapi.BlockInsertionResult, error) {
|
block *externalapi.BlockWithTrustedData, validateUTXO bool) (*externalapi.VirtualChangeSet, error) {
|
||||||
|
|
||||||
blockHash := consensushashing.BlockHash(block.Block)
|
blockHash := consensushashing.BlockHash(block.Block)
|
||||||
for i, daaBlock := range block.DAAWindow {
|
for i, daaBlock := range block.DAAWindow {
|
||||||
|
@ -22,12 +22,12 @@ func TestVirtualDiff(t *testing.T) {
|
|||||||
defer teardown(false)
|
defer teardown(false)
|
||||||
|
|
||||||
// Add block A over the genesis
|
// Add block A over the genesis
|
||||||
blockAHash, blockInsertionResult, err := tc.AddBlock([]*externalapi.DomainHash{consensusConfig.GenesisHash}, nil, nil)
|
blockAHash, virtualChangeSet, err := tc.AddBlock([]*externalapi.DomainHash{consensusConfig.GenesisHash}, nil, nil)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("Error adding block A: %+v", err)
|
t.Fatalf("Error adding block A: %+v", err)
|
||||||
}
|
}
|
||||||
|
|
||||||
virtualUTXODiff := blockInsertionResult.VirtualUTXODiff
|
virtualUTXODiff := virtualChangeSet.VirtualUTXODiff
|
||||||
if virtualUTXODiff.ToRemove().Len() != 0 {
|
if virtualUTXODiff.ToRemove().Len() != 0 {
|
||||||
t.Fatalf("Unexpected length %d for virtualUTXODiff.ToRemove()", virtualUTXODiff.ToRemove().Len())
|
t.Fatalf("Unexpected length %d for virtualUTXODiff.ToRemove()", virtualUTXODiff.ToRemove().Len())
|
||||||
}
|
}
|
||||||
@ -37,7 +37,7 @@ func TestVirtualDiff(t *testing.T) {
|
|||||||
t.Fatalf("Unexpected length %d for virtualUTXODiff.ToAdd()", virtualUTXODiff.ToAdd().Len())
|
t.Fatalf("Unexpected length %d for virtualUTXODiff.ToAdd()", virtualUTXODiff.ToAdd().Len())
|
||||||
}
|
}
|
||||||
|
|
||||||
blockBHash, blockInsertionResult, err := tc.AddBlock([]*externalapi.DomainHash{blockAHash}, nil, nil)
|
blockBHash, virtualChangeSet, err := tc.AddBlock([]*externalapi.DomainHash{blockAHash}, nil, nil)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("Error adding block A: %+v", err)
|
t.Fatalf("Error adding block A: %+v", err)
|
||||||
}
|
}
|
||||||
@ -47,7 +47,7 @@ func TestVirtualDiff(t *testing.T) {
|
|||||||
t.Fatalf("Block: %+v", err)
|
t.Fatalf("Block: %+v", err)
|
||||||
}
|
}
|
||||||
|
|
||||||
virtualUTXODiff = blockInsertionResult.VirtualUTXODiff
|
virtualUTXODiff = virtualChangeSet.VirtualUTXODiff
|
||||||
if virtualUTXODiff.ToRemove().Len() != 0 {
|
if virtualUTXODiff.ToRemove().Len() != 0 {
|
||||||
t.Fatalf("Unexpected length %d for virtualUTXODiff.ToRemove()", virtualUTXODiff.ToRemove().Len())
|
t.Fatalf("Unexpected length %d for virtualUTXODiff.ToRemove()", virtualUTXODiff.ToRemove().Len())
|
||||||
}
|
}
|
||||||
|
@ -19,11 +19,11 @@ func TestCalculateChainPath(t *testing.T) {
|
|||||||
defer teardown(false)
|
defer teardown(false)
|
||||||
|
|
||||||
// Add block A over the genesis
|
// Add block A over the genesis
|
||||||
blockAHash, blockAInsertionResult, err := consensus.AddBlock([]*externalapi.DomainHash{consensusConfig.GenesisHash}, nil, nil)
|
blockAHash, blockAVirtualChangeSet, err := consensus.AddBlock([]*externalapi.DomainHash{consensusConfig.GenesisHash}, nil, nil)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("Error adding block A: %+v", err)
|
t.Fatalf("Error adding block A: %+v", err)
|
||||||
}
|
}
|
||||||
blockASelectedParentChainChanges := blockAInsertionResult.VirtualSelectedParentChainChanges
|
blockASelectedParentChainChanges := blockAVirtualChangeSet.VirtualSelectedParentChainChanges
|
||||||
|
|
||||||
// Make sure that the removed slice is empty
|
// Make sure that the removed slice is empty
|
||||||
if len(blockASelectedParentChainChanges.Removed) > 0 {
|
if len(blockASelectedParentChainChanges.Removed) > 0 {
|
||||||
@ -59,11 +59,11 @@ func TestCalculateChainPath(t *testing.T) {
|
|||||||
|
|
||||||
// Add block C over the block that isn't the current virtual's selected parent
|
// Add block C over the block that isn't the current virtual's selected parent
|
||||||
// We expect this to cause a reorg
|
// We expect this to cause a reorg
|
||||||
blockCHash, blockCInsertionResult, err := consensus.AddBlock([]*externalapi.DomainHash{notVirtualSelectedParent}, nil, nil)
|
blockCHash, blockCVirtualChangeSet, err := consensus.AddBlock([]*externalapi.DomainHash{notVirtualSelectedParent}, nil, nil)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("Error adding block C: %+v", err)
|
t.Fatalf("Error adding block C: %+v", err)
|
||||||
}
|
}
|
||||||
blockCSelectedParentChainChanges := blockCInsertionResult.VirtualSelectedParentChainChanges
|
blockCSelectedParentChainChanges := blockCVirtualChangeSet.VirtualSelectedParentChainChanges
|
||||||
|
|
||||||
// Make sure that the removed slice contains only the block that was previously
|
// Make sure that the removed slice contains only the block that was previously
|
||||||
// the selected parent
|
// the selected parent
|
||||||
@ -92,11 +92,11 @@ func TestCalculateChainPath(t *testing.T) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// Add block D over the genesis
|
// Add block D over the genesis
|
||||||
_, blockDInsertionResult, err := consensus.AddBlock([]*externalapi.DomainHash{consensusConfig.GenesisHash}, nil, nil)
|
_, blockDVirtualChangeSet, err := consensus.AddBlock([]*externalapi.DomainHash{consensusConfig.GenesisHash}, nil, nil)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("Error adding block D: %+v", err)
|
t.Fatalf("Error adding block D: %+v", err)
|
||||||
}
|
}
|
||||||
blockDSelectedParentChainChanges := blockDInsertionResult.VirtualSelectedParentChainChanges
|
blockDSelectedParentChainChanges := blockDVirtualChangeSet.VirtualSelectedParentChainChanges
|
||||||
|
|
||||||
// Make sure that both the added and the removed slices are empty
|
// Make sure that both the added and the removed slices are empty
|
||||||
if len(blockDSelectedParentChainChanges.Added) > 0 {
|
if len(blockDSelectedParentChainChanges.Added) > 0 {
|
||||||
|
@ -8,14 +8,14 @@ import (
|
|||||||
"sort"
|
"sort"
|
||||||
)
|
)
|
||||||
|
|
||||||
func (csm *consensusStateManager) ResolveVirtual(maxBlocksToResolve uint64) (bool, error) {
|
func (csm *consensusStateManager) ResolveVirtual(maxBlocksToResolve uint64) (*externalapi.VirtualChangeSet, bool, error) {
|
||||||
onEnd := logger.LogAndMeasureExecutionTime(log, "csm.ResolveVirtual")
|
onEnd := logger.LogAndMeasureExecutionTime(log, "csm.ResolveVirtual")
|
||||||
defer onEnd()
|
defer onEnd()
|
||||||
|
|
||||||
readStagingArea := model.NewStagingArea()
|
readStagingArea := model.NewStagingArea()
|
||||||
tips, err := csm.consensusStateStore.Tips(readStagingArea, csm.databaseContext)
|
tips, err := csm.consensusStateStore.Tips(readStagingArea, csm.databaseContext)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return false, err
|
return nil, false, err
|
||||||
}
|
}
|
||||||
|
|
||||||
var sortErr error
|
var sortErr error
|
||||||
@ -29,7 +29,7 @@ func (csm *consensusStateManager) ResolveVirtual(maxBlocksToResolve uint64) (boo
|
|||||||
return selectedParent.Equal(tips[i])
|
return selectedParent.Equal(tips[i])
|
||||||
})
|
})
|
||||||
if sortErr != nil {
|
if sortErr != nil {
|
||||||
return false, sortErr
|
return nil, false, sortErr
|
||||||
}
|
}
|
||||||
|
|
||||||
var selectedTip *externalapi.DomainHash
|
var selectedTip *externalapi.DomainHash
|
||||||
@ -39,7 +39,7 @@ func (csm *consensusStateManager) ResolveVirtual(maxBlocksToResolve uint64) (boo
|
|||||||
resolveStagingArea := model.NewStagingArea()
|
resolveStagingArea := model.NewStagingArea()
|
||||||
unverifiedBlocks, err := csm.getUnverifiedChainBlocks(resolveStagingArea, tip)
|
unverifiedBlocks, err := csm.getUnverifiedChainBlocks(resolveStagingArea, tip)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return false, err
|
return nil, false, err
|
||||||
}
|
}
|
||||||
|
|
||||||
resolveTip := tip
|
resolveTip := tip
|
||||||
@ -51,7 +51,7 @@ func (csm *consensusStateManager) ResolveVirtual(maxBlocksToResolve uint64) (boo
|
|||||||
|
|
||||||
blockStatus, reversalData, err := csm.resolveBlockStatus(resolveStagingArea, resolveTip, true)
|
blockStatus, reversalData, err := csm.resolveBlockStatus(resolveStagingArea, resolveTip, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return false, err
|
return nil, false, err
|
||||||
}
|
}
|
||||||
|
|
||||||
if blockStatus == externalapi.StatusUTXOValid {
|
if blockStatus == externalapi.StatusUTXOValid {
|
||||||
@ -60,13 +60,13 @@ func (csm *consensusStateManager) ResolveVirtual(maxBlocksToResolve uint64) (boo
|
|||||||
|
|
||||||
err = staging.CommitAllChanges(csm.databaseContext, resolveStagingArea)
|
err = staging.CommitAllChanges(csm.databaseContext, resolveStagingArea)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return false, err
|
return nil, false, err
|
||||||
}
|
}
|
||||||
|
|
||||||
if reversalData != nil {
|
if reversalData != nil {
|
||||||
err = csm.ReverseUTXODiffs(resolveTip, reversalData)
|
err = csm.ReverseUTXODiffs(resolveTip, reversalData)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return false, err
|
return nil, false, err
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
break
|
break
|
||||||
@ -75,19 +75,39 @@ func (csm *consensusStateManager) ResolveVirtual(maxBlocksToResolve uint64) (boo
|
|||||||
|
|
||||||
if selectedTip == nil {
|
if selectedTip == nil {
|
||||||
log.Warnf("Non of the DAG tips are valid")
|
log.Warnf("Non of the DAG tips are valid")
|
||||||
return true, nil
|
return nil, true, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
oldVirtualGHOSTDAGData, err := csm.ghostdagDataStore.Get(csm.databaseContext, readStagingArea, model.VirtualBlockHash, false)
|
||||||
|
if err != nil {
|
||||||
|
return nil, false, err
|
||||||
}
|
}
|
||||||
|
|
||||||
updateVirtualStagingArea := model.NewStagingArea()
|
updateVirtualStagingArea := model.NewStagingArea()
|
||||||
_, err = csm.updateVirtualWithParents(updateVirtualStagingArea, []*externalapi.DomainHash{selectedTip})
|
virtualUTXODiff, err := csm.updateVirtualWithParents(updateVirtualStagingArea, []*externalapi.DomainHash{selectedTip})
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return false, err
|
return nil, false, err
|
||||||
}
|
}
|
||||||
|
|
||||||
err = staging.CommitAllChanges(csm.databaseContext, updateVirtualStagingArea)
|
err = staging.CommitAllChanges(csm.databaseContext, updateVirtualStagingArea)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return false, err
|
return nil, false, err
|
||||||
}
|
}
|
||||||
|
|
||||||
return isCompletelyResolved, nil
|
selectedParentChainChanges, err := csm.dagTraversalManager.
|
||||||
|
CalculateChainPath(readStagingArea, oldVirtualGHOSTDAGData.SelectedParent(), selectedTip)
|
||||||
|
if err != nil {
|
||||||
|
return nil, false, err
|
||||||
|
}
|
||||||
|
|
||||||
|
virtualParents, err := csm.dagTopologyManager.Parents(readStagingArea, model.VirtualBlockHash)
|
||||||
|
if err != nil {
|
||||||
|
return nil, false, err
|
||||||
|
}
|
||||||
|
|
||||||
|
return &externalapi.VirtualChangeSet{
|
||||||
|
VirtualSelectedParentChainChanges: selectedParentChainChanges,
|
||||||
|
VirtualUTXODiff: virtualUTXODiff,
|
||||||
|
VirtualParents: virtualParents,
|
||||||
|
}, isCompletelyResolved, nil
|
||||||
}
|
}
|
||||||
|
@ -366,7 +366,7 @@ func TestSigningTwoInputs(t *testing.T) {
|
|||||||
input.SignatureScript = signatureScript
|
input.SignatureScript = signatureScript
|
||||||
}
|
}
|
||||||
|
|
||||||
_, insertionResult, err := tc.AddBlock([]*externalapi.DomainHash{block3Hash}, nil, []*externalapi.DomainTransaction{tx})
|
_, virtualChangeSet, err := tc.AddBlock([]*externalapi.DomainHash{block3Hash}, nil, []*externalapi.DomainTransaction{tx})
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("AddBlock: %+v", err)
|
t.Fatalf("AddBlock: %+v", err)
|
||||||
}
|
}
|
||||||
@ -375,7 +375,7 @@ func TestSigningTwoInputs(t *testing.T) {
|
|||||||
TransactionID: *consensushashing.TransactionID(tx),
|
TransactionID: *consensushashing.TransactionID(tx),
|
||||||
Index: 0,
|
Index: 0,
|
||||||
}
|
}
|
||||||
if !insertionResult.VirtualUTXODiff.ToAdd().Contains(txOutpoint) {
|
if !virtualChangeSet.VirtualUTXODiff.ToAdd().Contains(txOutpoint) {
|
||||||
t.Fatalf("tx was not accepted by the DAG")
|
t.Fatalf("tx was not accepted by the DAG")
|
||||||
}
|
}
|
||||||
})
|
})
|
||||||
@ -492,7 +492,7 @@ func TestSigningTwoInputsECDSA(t *testing.T) {
|
|||||||
input.SignatureScript = signatureScript
|
input.SignatureScript = signatureScript
|
||||||
}
|
}
|
||||||
|
|
||||||
_, insertionResult, err := tc.AddBlock([]*externalapi.DomainHash{block3Hash}, nil, []*externalapi.DomainTransaction{tx})
|
_, virtualChangeSet, err := tc.AddBlock([]*externalapi.DomainHash{block3Hash}, nil, []*externalapi.DomainTransaction{tx})
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("AddBlock: %+v", err)
|
t.Fatalf("AddBlock: %+v", err)
|
||||||
}
|
}
|
||||||
@ -501,7 +501,7 @@ func TestSigningTwoInputsECDSA(t *testing.T) {
|
|||||||
TransactionID: *consensushashing.TransactionID(tx),
|
TransactionID: *consensushashing.TransactionID(tx),
|
||||||
Index: 0,
|
Index: 0,
|
||||||
}
|
}
|
||||||
if !insertionResult.VirtualUTXODiff.ToAdd().Contains(txOutpoint) {
|
if !virtualChangeSet.VirtualUTXODiff.ToAdd().Contains(txOutpoint) {
|
||||||
t.Fatalf("tx was not accepted by the DAG")
|
t.Fatalf("tx was not accepted by the DAG")
|
||||||
}
|
}
|
||||||
})
|
})
|
||||||
|
@ -46,7 +46,7 @@ func (tc *testConsensus) BuildBlockWithParents(parentHashes []*externalapi.Domai
|
|||||||
}
|
}
|
||||||
|
|
||||||
func (tc *testConsensus) AddBlock(parentHashes []*externalapi.DomainHash, coinbaseData *externalapi.DomainCoinbaseData,
|
func (tc *testConsensus) AddBlock(parentHashes []*externalapi.DomainHash, coinbaseData *externalapi.DomainCoinbaseData,
|
||||||
transactions []*externalapi.DomainTransaction) (*externalapi.DomainHash, *externalapi.BlockInsertionResult, error) {
|
transactions []*externalapi.DomainTransaction) (*externalapi.DomainHash, *externalapi.VirtualChangeSet, error) {
|
||||||
|
|
||||||
// Require write lock because BuildBlockWithParents stages temporary data
|
// Require write lock because BuildBlockWithParents stages temporary data
|
||||||
tc.lock.Lock()
|
tc.lock.Lock()
|
||||||
@ -57,16 +57,16 @@ func (tc *testConsensus) AddBlock(parentHashes []*externalapi.DomainHash, coinba
|
|||||||
return nil, nil, err
|
return nil, nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
blockInsertionResult, err := tc.blockProcessor.ValidateAndInsertBlock(block, true)
|
virtualChangeSet, err := tc.blockProcessor.ValidateAndInsertBlock(block, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, nil, err
|
return nil, nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
return consensushashing.BlockHash(block), blockInsertionResult, nil
|
return consensushashing.BlockHash(block), virtualChangeSet, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (tc *testConsensus) AddUTXOInvalidHeader(parentHashes []*externalapi.DomainHash) (*externalapi.DomainHash,
|
func (tc *testConsensus) AddUTXOInvalidHeader(parentHashes []*externalapi.DomainHash) (*externalapi.DomainHash,
|
||||||
*externalapi.BlockInsertionResult, error) {
|
*externalapi.VirtualChangeSet, error) {
|
||||||
|
|
||||||
// Require write lock because BuildBlockWithParents stages temporary data
|
// Require write lock because BuildBlockWithParents stages temporary data
|
||||||
tc.lock.Lock()
|
tc.lock.Lock()
|
||||||
@ -77,7 +77,7 @@ func (tc *testConsensus) AddUTXOInvalidHeader(parentHashes []*externalapi.Domain
|
|||||||
return nil, nil, err
|
return nil, nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
blockInsertionResult, err := tc.blockProcessor.ValidateAndInsertBlock(&externalapi.DomainBlock{
|
virtualChangeSet, err := tc.blockProcessor.ValidateAndInsertBlock(&externalapi.DomainBlock{
|
||||||
Header: header,
|
Header: header,
|
||||||
Transactions: nil,
|
Transactions: nil,
|
||||||
}, true)
|
}, true)
|
||||||
@ -85,11 +85,11 @@ func (tc *testConsensus) AddUTXOInvalidHeader(parentHashes []*externalapi.Domain
|
|||||||
return nil, nil, err
|
return nil, nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
return consensushashing.HeaderHash(header), blockInsertionResult, nil
|
return consensushashing.HeaderHash(header), virtualChangeSet, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (tc *testConsensus) AddUTXOInvalidBlock(parentHashes []*externalapi.DomainHash) (*externalapi.DomainHash,
|
func (tc *testConsensus) AddUTXOInvalidBlock(parentHashes []*externalapi.DomainHash) (*externalapi.DomainHash,
|
||||||
*externalapi.BlockInsertionResult, error) {
|
*externalapi.VirtualChangeSet, error) {
|
||||||
|
|
||||||
// Require write lock because BuildBlockWithParents stages temporary data
|
// Require write lock because BuildBlockWithParents stages temporary data
|
||||||
tc.lock.Lock()
|
tc.lock.Lock()
|
||||||
@ -100,12 +100,12 @@ func (tc *testConsensus) AddUTXOInvalidBlock(parentHashes []*externalapi.DomainH
|
|||||||
return nil, nil, err
|
return nil, nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
blockInsertionResult, err := tc.blockProcessor.ValidateAndInsertBlock(block, true)
|
virtualChangeSet, err := tc.blockProcessor.ValidateAndInsertBlock(block, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, nil, err
|
return nil, nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
return consensushashing.BlockHash(block), blockInsertionResult, nil
|
return consensushashing.BlockHash(block), virtualChangeSet, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (tc *testConsensus) MineJSON(r io.Reader, blockType testapi.MineJSONBlockType) (tips []*externalapi.DomainHash, err error) {
|
func (tc *testConsensus) MineJSON(r io.Reader, blockType testapi.MineJSONBlockType) (tips []*externalapi.DomainHash, err error) {
|
||||||
|
@ -94,25 +94,25 @@ func (ui *UTXOIndex) isSynced() (bool, error) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// Update updates the UTXO index with the given DAG selected parent chain changes
|
// Update updates the UTXO index with the given DAG selected parent chain changes
|
||||||
func (ui *UTXOIndex) Update(blockInsertionResult *externalapi.BlockInsertionResult) (*UTXOChanges, error) {
|
func (ui *UTXOIndex) Update(virtualChangeSet *externalapi.VirtualChangeSet) (*UTXOChanges, error) {
|
||||||
onEnd := logger.LogAndMeasureExecutionTime(log, "UTXOIndex.Update")
|
onEnd := logger.LogAndMeasureExecutionTime(log, "UTXOIndex.Update")
|
||||||
defer onEnd()
|
defer onEnd()
|
||||||
|
|
||||||
ui.mutex.Lock()
|
ui.mutex.Lock()
|
||||||
defer ui.mutex.Unlock()
|
defer ui.mutex.Unlock()
|
||||||
|
|
||||||
log.Tracef("Updating UTXO index with VirtualUTXODiff: %+v", blockInsertionResult.VirtualUTXODiff)
|
log.Tracef("Updating UTXO index with VirtualUTXODiff: %+v", virtualChangeSet.VirtualUTXODiff)
|
||||||
err := ui.removeUTXOs(blockInsertionResult.VirtualUTXODiff.ToRemove())
|
err := ui.removeUTXOs(virtualChangeSet.VirtualUTXODiff.ToRemove())
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
err = ui.addUTXOs(blockInsertionResult.VirtualUTXODiff.ToAdd())
|
err = ui.addUTXOs(virtualChangeSet.VirtualUTXODiff.ToAdd())
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
ui.store.updateVirtualParents(blockInsertionResult.VirtualParents)
|
ui.store.updateVirtualParents(virtualChangeSet.VirtualParents)
|
||||||
|
|
||||||
added, removed, _ := ui.store.stagedData()
|
added, removed, _ := ui.store.stagedData()
|
||||||
utxoIndexChanges := &UTXOChanges{
|
utxoIndexChanges := &UTXOChanges{
|
||||||
|
Loading…
x
Reference in New Issue
Block a user