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:
Ori Newman 2021-12-05 13:22:48 +02:00 committed by GitHub
parent 11103a36d3
commit 32e8e539ac
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
24 changed files with 186 additions and 123 deletions

View File

@ -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)

View File

@ -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
} }

View File

@ -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

View File

@ -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

View File

@ -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 {

View File

@ -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

View File

@ -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)

View File

@ -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
} }

View File

@ -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")
} }
}) })

View File

@ -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) {

View File

@ -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)
} }

View File

@ -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

View File

@ -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)
} }

View File

@ -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)
} }

View File

@ -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)

View File

@ -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()

View File

@ -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,

View File

@ -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 {

View File

@ -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())
} }

View File

@ -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 {

View File

@ -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
} }

View File

@ -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")
} }
}) })

View File

@ -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) {

View File

@ -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{