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,
|
||||
shutDownChan,
|
||||
)
|
||||
protocolManager.SetOnVirtualChange(rpcManager.NotifyVirtualChange)
|
||||
protocolManager.SetOnBlockAddedToDAGHandler(rpcManager.NotifyBlockAddedToDAG)
|
||||
protocolManager.SetOnPruningPointUTXOSetOverrideHandler(rpcManager.NotifyPruningPointUTXOSetOverride)
|
||||
|
||||
|
@ -18,7 +18,7 @@ import (
|
||||
// relays newly unorphaned transactions and possibly rebroadcast
|
||||
// manually added transactions when not in IBD.
|
||||
func (f *FlowContext) OnNewBlock(block *externalapi.DomainBlock,
|
||||
blockInsertionResult *externalapi.BlockInsertionResult) error {
|
||||
virtualChangeSet *externalapi.VirtualChangeSet) error {
|
||||
|
||||
hash := consensushashing.BlockHash(block)
|
||||
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))
|
||||
|
||||
newBlocks := []*externalapi.DomainBlock{block}
|
||||
newBlockInsertionResults := []*externalapi.BlockInsertionResult{blockInsertionResult}
|
||||
newVirtualChangeSets := []*externalapi.VirtualChangeSet{virtualChangeSet}
|
||||
for _, unorphaningResult := range unorphaningResults {
|
||||
newBlocks = append(newBlocks, unorphaningResult.block)
|
||||
newBlockInsertionResults = append(newBlockInsertionResults, unorphaningResult.blockInsertionResult)
|
||||
newVirtualChangeSets = append(newVirtualChangeSets, unorphaningResult.virtualChangeSet)
|
||||
}
|
||||
|
||||
allAcceptedTransactions := make([]*externalapi.DomainTransaction, 0)
|
||||
@ -49,8 +49,8 @@ func (f *FlowContext) OnNewBlock(block *externalapi.DomainBlock,
|
||||
|
||||
if f.onBlockAddedToDAGHandler != nil {
|
||||
log.Debugf("OnNewBlock: calling f.onBlockAddedToDAGHandler for block %s", hash)
|
||||
blockInsertionResult = newBlockInsertionResults[i]
|
||||
err := f.onBlockAddedToDAGHandler(newBlock, blockInsertionResult)
|
||||
virtualChangeSet = newVirtualChangeSets[i]
|
||||
err := f.onBlockAddedToDAGHandler(newBlock, virtualChangeSet)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -60,6 +60,15 @@ func (f *FlowContext) OnNewBlock(block *externalapi.DomainBlock,
|
||||
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
|
||||
// resets due to pruning point change via IBD.
|
||||
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")
|
||||
}
|
||||
|
||||
blockInsertionResult, err := f.Domain().Consensus().ValidateAndInsertBlock(block, true)
|
||||
virtualChangeSet, err := f.Domain().Consensus().ValidateAndInsertBlock(block, true)
|
||||
if err != nil {
|
||||
if errors.As(err, &ruleerrors.RuleError{}) {
|
||||
log.Warnf("Validation failed for block %s: %s", consensushashing.BlockHash(block), err)
|
||||
}
|
||||
return err
|
||||
}
|
||||
err = f.OnNewBlock(block, blockInsertionResult)
|
||||
err = f.OnNewBlock(block, virtualChangeSet)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -22,7 +22,10 @@ import (
|
||||
|
||||
// OnBlockAddedToDAGHandler is a handler function that's triggered
|
||||
// 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
|
||||
// resets due to pruning point change via IBD.
|
||||
@ -43,6 +46,7 @@ type FlowContext struct {
|
||||
|
||||
timeStarted int64
|
||||
|
||||
onVirtualChangeHandler OnVirtualChangeHandler
|
||||
onBlockAddedToDAGHandler OnBlockAddedToDAGHandler
|
||||
onPruningPointUTXOSetOverrideHandler OnPruningPointUTXOSetOverrideHandler
|
||||
onTransactionAddedToMempoolHandler OnTransactionAddedToMempoolHandler
|
||||
@ -100,6 +104,11 @@ func (f *FlowContext) ShutdownChan() <-chan struct{} {
|
||||
return f.shutdownChan
|
||||
}
|
||||
|
||||
// SetOnVirtualChangeHandler sets the onVirtualChangeHandler handler
|
||||
func (f *FlowContext) SetOnVirtualChangeHandler(onVirtualChangeHandler OnVirtualChangeHandler) {
|
||||
f.onVirtualChangeHandler = onVirtualChangeHandler
|
||||
}
|
||||
|
||||
// SetOnBlockAddedToDAGHandler sets the onBlockAddedToDAG handler
|
||||
func (f *FlowContext) SetOnBlockAddedToDAGHandler(onBlockAddedToDAGHandler OnBlockAddedToDAGHandler) {
|
||||
f.onBlockAddedToDAGHandler = onBlockAddedToDAGHandler
|
||||
|
@ -18,7 +18,7 @@ const maxOrphans = 600
|
||||
// UnorphaningResult is the result of unorphaning a block
|
||||
type UnorphaningResult struct {
|
||||
block *externalapi.DomainBlock
|
||||
blockInsertionResult *externalapi.BlockInsertionResult
|
||||
virtualChangeSet *externalapi.VirtualChangeSet
|
||||
}
|
||||
|
||||
// AddOrphan adds the block to the orphan set
|
||||
@ -90,14 +90,14 @@ func (f *FlowContext) UnorphanBlocks(rootBlock *externalapi.DomainBlock) ([]*Uno
|
||||
}
|
||||
}
|
||||
if canBeUnorphaned {
|
||||
blockInsertionResult, unorphaningSucceeded, err := f.unorphanBlock(orphanHash)
|
||||
virtualChangeSet, unorphaningSucceeded, err := f.unorphanBlock(orphanHash)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if unorphaningSucceeded {
|
||||
unorphaningResults = append(unorphaningResults, &UnorphaningResult{
|
||||
block: orphanBlock,
|
||||
blockInsertionResult: blockInsertionResult,
|
||||
virtualChangeSet: virtualChangeSet,
|
||||
})
|
||||
processQueue = f.addChildOrphansToProcessQueue(&orphanHash, processQueue)
|
||||
}
|
||||
@ -143,14 +143,14 @@ func (f *FlowContext) findChildOrphansOfBlock(blockHash *externalapi.DomainHash)
|
||||
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]
|
||||
if !ok {
|
||||
return nil, false, errors.Errorf("attempted to unorphan a non-orphan block %s", 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 errors.As(err, &ruleerrors.RuleError{}) {
|
||||
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)
|
||||
return blockInsertionResult, true, nil
|
||||
return virtualChangeSet, true, nil
|
||||
}
|
||||
|
||||
// GetOrphanRoots returns the roots of the missing ancestors DAG of the given orphan
|
||||
|
@ -23,7 +23,8 @@ var orphanResolutionRange uint32 = 5
|
||||
type RelayInvsContext interface {
|
||||
Domain() domain.Domain
|
||||
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
|
||||
SharedRequestedBlocks() *SharedRequestedBlocks
|
||||
Broadcast(message appmessage.Message) error
|
||||
@ -128,7 +129,7 @@ func (flow *handleRelayInvsFlow) start() error {
|
||||
}
|
||||
|
||||
log.Debugf("Processing block %s", inv.Hash)
|
||||
missingParents, blockInsertionResult, err := flow.processBlock(block)
|
||||
missingParents, virtualChangeSet, err := flow.processBlock(block)
|
||||
if err != nil {
|
||||
if errors.Is(err, ruleerrors.ErrPrunedBlock) {
|
||||
log.Infof("Ignoring pruned block %s", inv.Hash)
|
||||
@ -156,7 +157,7 @@ func (flow *handleRelayInvsFlow) start() error {
|
||||
return err
|
||||
}
|
||||
log.Infof("Accepted block %s via relay", inv.Hash)
|
||||
err = flow.OnNewBlock(block, blockInsertionResult)
|
||||
err = flow.OnNewBlock(block, virtualChangeSet)
|
||||
if err != nil {
|
||||
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)
|
||||
blockInsertionResult, err := flow.Domain().Consensus().ValidateAndInsertBlock(block, true)
|
||||
virtualChangeSet, err := flow.Domain().Consensus().ValidateAndInsertBlock(block, true)
|
||||
if err != nil {
|
||||
if !errors.As(err, &ruleerrors.RuleError{}) {
|
||||
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)
|
||||
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 {
|
||||
|
@ -471,7 +471,7 @@ func (flow *handleRelayInvsFlow) syncMissingBlockBodies(highHash *externalapi.Do
|
||||
return err
|
||||
}
|
||||
|
||||
blockInsertionResult, err := flow.Domain().Consensus().ValidateAndInsertBlock(block, false)
|
||||
virtualChangeSet, err := flow.Domain().Consensus().ValidateAndInsertBlock(block, false)
|
||||
if err != nil {
|
||||
if errors.Is(err, ruleerrors.ErrDuplicateBlock) {
|
||||
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)
|
||||
}
|
||||
err = flow.OnNewBlock(block, blockInsertionResult)
|
||||
err = flow.OnNewBlock(block, virtualChangeSet)
|
||||
if err != nil {
|
||||
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
|
||||
|
@ -84,6 +84,11 @@ func (m *Manager) runFlows(flows []*flow, peer *peerpkg.Peer, errChan <-chan err
|
||||
return <-errChan
|
||||
}
|
||||
|
||||
// SetOnVirtualChange sets the onVirtualChangeHandler handler
|
||||
func (m *Manager) SetOnVirtualChange(onVirtualChangeHandler flowcontext.OnVirtualChangeHandler) {
|
||||
m.context.SetOnVirtualChangeHandler(onVirtualChangeHandler)
|
||||
}
|
||||
|
||||
// SetOnBlockAddedToDAGHandler sets the onBlockAddedToDAG handler
|
||||
func (m *Manager) SetOnBlockAddedToDAGHandler(onBlockAddedToDAGHandler flowcontext.OnBlockAddedToDAGHandler) {
|
||||
m.context.SetOnBlockAddedToDAGHandler(onBlockAddedToDAGHandler)
|
||||
|
@ -48,12 +48,31 @@ func NewManager(
|
||||
}
|
||||
|
||||
// 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")
|
||||
defer onEnd()
|
||||
|
||||
if m.context.Config.UTXOIndex {
|
||||
err := m.notifyUTXOsChanged(blockInsertionResult)
|
||||
err := m.notifyUTXOsChanged(virtualChangeSet)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -69,18 +88,12 @@ func (m *Manager) NotifyBlockAddedToDAG(block *externalapi.DomainBlock, blockIns
|
||||
return err
|
||||
}
|
||||
|
||||
err = m.notifyVirtualSelectedParentChainChanged(blockInsertionResult)
|
||||
err = m.notifyVirtualSelectedParentChainChanged(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)
|
||||
return nil
|
||||
}
|
||||
|
||||
// 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)
|
||||
}
|
||||
|
||||
func (m *Manager) notifyUTXOsChanged(blockInsertionResult *externalapi.BlockInsertionResult) error {
|
||||
func (m *Manager) notifyUTXOsChanged(virtualChangeSet *externalapi.VirtualChangeSet) error {
|
||||
onEnd := logger.LogAndMeasureExecutionTime(log, "RPCManager.NotifyUTXOsChanged")
|
||||
defer onEnd()
|
||||
|
||||
utxoIndexChanges, err := m.context.UTXOIndex.Update(blockInsertionResult)
|
||||
utxoIndexChanges, err := m.context.UTXOIndex.Update(virtualChangeSet)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -171,12 +184,12 @@ func (m *Manager) notifyVirtualDaaScoreChanged() error {
|
||||
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")
|
||||
defer onEnd()
|
||||
|
||||
notification, err := m.context.ConvertVirtualSelectedParentChainChangesToChainChangedNotificationMessage(
|
||||
blockInsertionResult.VirtualSelectedParentChainChanges)
|
||||
virtualChangeSet.VirtualSelectedParentChainChanges)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -163,7 +163,7 @@ func TestMultisig(t *testing.T) {
|
||||
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 {
|
||||
t.Fatalf("AddBlock: %+v", err)
|
||||
}
|
||||
@ -172,7 +172,7 @@ func TestMultisig(t *testing.T) {
|
||||
TransactionID: *consensushashing.TransactionID(extractedSignedTxStep2),
|
||||
Index: 0,
|
||||
}
|
||||
if !insertionResult.VirtualUTXODiff.ToAdd().Contains(addedUTXO) {
|
||||
if !virtualChangeSet.VirtualUTXODiff.ToAdd().Contains(addedUTXO) {
|
||||
t.Fatalf("Transaction wasn't accepted in the DAG")
|
||||
}
|
||||
})
|
||||
@ -294,7 +294,7 @@ func TestP2PK(t *testing.T) {
|
||||
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 {
|
||||
t.Fatalf("AddBlock: %+v", err)
|
||||
}
|
||||
@ -303,7 +303,7 @@ func TestP2PK(t *testing.T) {
|
||||
TransactionID: *consensushashing.TransactionID(tx),
|
||||
Index: 0,
|
||||
}
|
||||
if !insertionResult.VirtualUTXODiff.ToAdd().Contains(addedUTXO) {
|
||||
if !virtualChangeSet.VirtualUTXODiff.ToAdd().Contains(addedUTXO) {
|
||||
t.Fatalf("Transaction wasn't accepted in the DAG")
|
||||
}
|
||||
})
|
||||
|
@ -56,7 +56,7 @@ type consensus struct {
|
||||
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()
|
||||
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
|
||||
// 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()
|
||||
defer s.lock.Unlock()
|
||||
|
||||
@ -720,30 +720,13 @@ func (s *consensus) PopulateMass(transaction *externalapi.DomainTransaction) {
|
||||
s.transactionValidator.PopulateMass(transaction)
|
||||
}
|
||||
|
||||
func (s *consensus) ResolveVirtual() error {
|
||||
// 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.
|
||||
for i := 0; ; i++ {
|
||||
if i%10 == 0 {
|
||||
log.Infof("Resolving virtual. This may take some time...")
|
||||
}
|
||||
var isCompletelyResolved bool
|
||||
var err error
|
||||
func() {
|
||||
func (s *consensus) ResolveVirtual() (*externalapi.VirtualChangeSet, bool, error) {
|
||||
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
|
||||
}
|
||||
}
|
||||
// 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.
|
||||
return s.consensusStateManager.ResolveVirtual(100)
|
||||
}
|
||||
|
||||
func (s *consensus) BuildPruningPointProof() (*externalapi.PruningPointProof, error) {
|
||||
|
@ -4,8 +4,8 @@ package externalapi
|
||||
type Consensus interface {
|
||||
Init(skipAddingGenesis bool) error
|
||||
BuildBlock(coinbaseData *DomainCoinbaseData, transactions []*DomainTransaction) (*DomainBlock, error)
|
||||
ValidateAndInsertBlock(block *DomainBlock, shouldValidateAgainstUTXO bool) (*BlockInsertionResult, error)
|
||||
ValidateAndInsertBlockWithTrustedData(block *BlockWithTrustedData, validateUTXO bool) (*BlockInsertionResult, error)
|
||||
ValidateAndInsertBlock(block *DomainBlock, shouldValidateAgainstUTXO bool) (*VirtualChangeSet, error)
|
||||
ValidateAndInsertBlockWithTrustedData(block *BlockWithTrustedData, validateUTXO bool) (*VirtualChangeSet, error)
|
||||
ValidateTransactionAndPopulateWithConsensusData(transaction *DomainTransaction) error
|
||||
ImportPruningPoints(pruningPoints []BlockHeader) error
|
||||
BuildPruningPointProof() (*PruningPointProof, error)
|
||||
@ -46,5 +46,5 @@ type Consensus interface {
|
||||
Anticone(blockHash *DomainHash) ([]*DomainHash, error)
|
||||
EstimateNetworkHashesPerSecond(startHash *DomainHash, windowSize int) (uint64, error)
|
||||
PopulateMass(transaction *DomainTransaction)
|
||||
ResolveVirtual() error
|
||||
ResolveVirtual() (*VirtualChangeSet, bool, error)
|
||||
}
|
||||
|
@ -1,7 +1,7 @@
|
||||
package externalapi
|
||||
|
||||
// BlockInsertionResult is auxiliary data returned from ValidateAndInsertBlock
|
||||
type BlockInsertionResult struct {
|
||||
// VirtualChangeSet is auxiliary data returned from ValidateAndInsertBlock and ResolveVirtual
|
||||
type VirtualChangeSet struct {
|
||||
VirtualSelectedParentChainChanges *SelectedChainPath
|
||||
VirtualUTXODiff UTXODiff
|
||||
VirtualParents []*DomainHash
|
@ -4,7 +4,7 @@ import "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
|
||||
|
||||
// BlockProcessor is responsible for processing incoming blocks
|
||||
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
|
||||
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)
|
||||
RecoverUTXOIfRequired() 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.
|
||||
// Returns the hash of the added block
|
||||
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,
|
||||
*externalapi.BlockInsertionResult, error)
|
||||
*externalapi.VirtualChangeSet, 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
|
||||
// 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")
|
||||
defer onEnd()
|
||||
|
||||
@ -159,7 +159,7 @@ func (bp *blockProcessor) ValidateAndInsertImportedPruningPoint(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")
|
||||
defer onEnd()
|
||||
|
||||
|
@ -77,7 +77,7 @@ func (bp *blockProcessor) updateVirtualAcceptanceDataAfterImportingPruningPoint(
|
||||
}
|
||||
|
||||
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)
|
||||
err := bp.validateBlock(stagingArea, block, isBlockWithTrustedData)
|
||||
@ -210,7 +210,7 @@ func (bp *blockProcessor) validateAndInsertBlock(stagingArea *model.StagingArea,
|
||||
|
||||
bp.blockLogger.LogBlock(block)
|
||||
|
||||
return &externalapi.BlockInsertionResult{
|
||||
return &externalapi.VirtualChangeSet{
|
||||
VirtualSelectedParentChainChanges: selectedParentChainChanges,
|
||||
VirtualUTXODiff: virtualUTXODiff,
|
||||
VirtualParents: virtualParents,
|
||||
|
@ -8,7 +8,7 @@ import (
|
||||
)
|
||||
|
||||
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)
|
||||
for i, daaBlock := range block.DAAWindow {
|
||||
|
@ -22,12 +22,12 @@ func TestVirtualDiff(t *testing.T) {
|
||||
defer teardown(false)
|
||||
|
||||
// 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 {
|
||||
t.Fatalf("Error adding block A: %+v", err)
|
||||
}
|
||||
|
||||
virtualUTXODiff := blockInsertionResult.VirtualUTXODiff
|
||||
virtualUTXODiff := virtualChangeSet.VirtualUTXODiff
|
||||
if virtualUTXODiff.ToRemove().Len() != 0 {
|
||||
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())
|
||||
}
|
||||
|
||||
blockBHash, blockInsertionResult, err := tc.AddBlock([]*externalapi.DomainHash{blockAHash}, nil, nil)
|
||||
blockBHash, virtualChangeSet, err := tc.AddBlock([]*externalapi.DomainHash{blockAHash}, nil, nil)
|
||||
if err != nil {
|
||||
t.Fatalf("Error adding block A: %+v", err)
|
||||
}
|
||||
@ -47,7 +47,7 @@ func TestVirtualDiff(t *testing.T) {
|
||||
t.Fatalf("Block: %+v", err)
|
||||
}
|
||||
|
||||
virtualUTXODiff = blockInsertionResult.VirtualUTXODiff
|
||||
virtualUTXODiff = virtualChangeSet.VirtualUTXODiff
|
||||
if virtualUTXODiff.ToRemove().Len() != 0 {
|
||||
t.Fatalf("Unexpected length %d for virtualUTXODiff.ToRemove()", virtualUTXODiff.ToRemove().Len())
|
||||
}
|
||||
|
@ -19,11 +19,11 @@ func TestCalculateChainPath(t *testing.T) {
|
||||
defer teardown(false)
|
||||
|
||||
// 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 {
|
||||
t.Fatalf("Error adding block A: %+v", err)
|
||||
}
|
||||
blockASelectedParentChainChanges := blockAInsertionResult.VirtualSelectedParentChainChanges
|
||||
blockASelectedParentChainChanges := blockAVirtualChangeSet.VirtualSelectedParentChainChanges
|
||||
|
||||
// Make sure that the removed slice is empty
|
||||
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
|
||||
// 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 {
|
||||
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
|
||||
// the selected parent
|
||||
@ -92,11 +92,11 @@ func TestCalculateChainPath(t *testing.T) {
|
||||
}
|
||||
|
||||
// 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 {
|
||||
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
|
||||
if len(blockDSelectedParentChainChanges.Added) > 0 {
|
||||
|
@ -8,14 +8,14 @@ import (
|
||||
"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")
|
||||
defer onEnd()
|
||||
|
||||
readStagingArea := model.NewStagingArea()
|
||||
tips, err := csm.consensusStateStore.Tips(readStagingArea, csm.databaseContext)
|
||||
if err != nil {
|
||||
return false, err
|
||||
return nil, false, err
|
||||
}
|
||||
|
||||
var sortErr error
|
||||
@ -29,7 +29,7 @@ func (csm *consensusStateManager) ResolveVirtual(maxBlocksToResolve uint64) (boo
|
||||
return selectedParent.Equal(tips[i])
|
||||
})
|
||||
if sortErr != nil {
|
||||
return false, sortErr
|
||||
return nil, false, sortErr
|
||||
}
|
||||
|
||||
var selectedTip *externalapi.DomainHash
|
||||
@ -39,7 +39,7 @@ func (csm *consensusStateManager) ResolveVirtual(maxBlocksToResolve uint64) (boo
|
||||
resolveStagingArea := model.NewStagingArea()
|
||||
unverifiedBlocks, err := csm.getUnverifiedChainBlocks(resolveStagingArea, tip)
|
||||
if err != nil {
|
||||
return false, err
|
||||
return nil, false, err
|
||||
}
|
||||
|
||||
resolveTip := tip
|
||||
@ -51,7 +51,7 @@ func (csm *consensusStateManager) ResolveVirtual(maxBlocksToResolve uint64) (boo
|
||||
|
||||
blockStatus, reversalData, err := csm.resolveBlockStatus(resolveStagingArea, resolveTip, true)
|
||||
if err != nil {
|
||||
return false, err
|
||||
return nil, false, err
|
||||
}
|
||||
|
||||
if blockStatus == externalapi.StatusUTXOValid {
|
||||
@ -60,13 +60,13 @@ func (csm *consensusStateManager) ResolveVirtual(maxBlocksToResolve uint64) (boo
|
||||
|
||||
err = staging.CommitAllChanges(csm.databaseContext, resolveStagingArea)
|
||||
if err != nil {
|
||||
return false, err
|
||||
return nil, false, err
|
||||
}
|
||||
|
||||
if reversalData != nil {
|
||||
err = csm.ReverseUTXODiffs(resolveTip, reversalData)
|
||||
if err != nil {
|
||||
return false, err
|
||||
return nil, false, err
|
||||
}
|
||||
}
|
||||
break
|
||||
@ -75,19 +75,39 @@ func (csm *consensusStateManager) ResolveVirtual(maxBlocksToResolve uint64) (boo
|
||||
|
||||
if selectedTip == nil {
|
||||
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()
|
||||
_, err = csm.updateVirtualWithParents(updateVirtualStagingArea, []*externalapi.DomainHash{selectedTip})
|
||||
virtualUTXODiff, err := csm.updateVirtualWithParents(updateVirtualStagingArea, []*externalapi.DomainHash{selectedTip})
|
||||
if err != nil {
|
||||
return false, err
|
||||
return nil, false, err
|
||||
}
|
||||
|
||||
err = staging.CommitAllChanges(csm.databaseContext, updateVirtualStagingArea)
|
||||
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
|
||||
}
|
||||
|
||||
_, 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 {
|
||||
t.Fatalf("AddBlock: %+v", err)
|
||||
}
|
||||
@ -375,7 +375,7 @@ func TestSigningTwoInputs(t *testing.T) {
|
||||
TransactionID: *consensushashing.TransactionID(tx),
|
||||
Index: 0,
|
||||
}
|
||||
if !insertionResult.VirtualUTXODiff.ToAdd().Contains(txOutpoint) {
|
||||
if !virtualChangeSet.VirtualUTXODiff.ToAdd().Contains(txOutpoint) {
|
||||
t.Fatalf("tx was not accepted by the DAG")
|
||||
}
|
||||
})
|
||||
@ -492,7 +492,7 @@ func TestSigningTwoInputsECDSA(t *testing.T) {
|
||||
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 {
|
||||
t.Fatalf("AddBlock: %+v", err)
|
||||
}
|
||||
@ -501,7 +501,7 @@ func TestSigningTwoInputsECDSA(t *testing.T) {
|
||||
TransactionID: *consensushashing.TransactionID(tx),
|
||||
Index: 0,
|
||||
}
|
||||
if !insertionResult.VirtualUTXODiff.ToAdd().Contains(txOutpoint) {
|
||||
if !virtualChangeSet.VirtualUTXODiff.ToAdd().Contains(txOutpoint) {
|
||||
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,
|
||||
transactions []*externalapi.DomainTransaction) (*externalapi.DomainHash, *externalapi.BlockInsertionResult, error) {
|
||||
transactions []*externalapi.DomainTransaction) (*externalapi.DomainHash, *externalapi.VirtualChangeSet, error) {
|
||||
|
||||
// Require write lock because BuildBlockWithParents stages temporary data
|
||||
tc.lock.Lock()
|
||||
@ -57,16 +57,16 @@ func (tc *testConsensus) AddBlock(parentHashes []*externalapi.DomainHash, coinba
|
||||
return nil, nil, err
|
||||
}
|
||||
|
||||
blockInsertionResult, err := tc.blockProcessor.ValidateAndInsertBlock(block, true)
|
||||
virtualChangeSet, err := tc.blockProcessor.ValidateAndInsertBlock(block, true)
|
||||
if err != nil {
|
||||
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,
|
||||
*externalapi.BlockInsertionResult, error) {
|
||||
*externalapi.VirtualChangeSet, error) {
|
||||
|
||||
// Require write lock because BuildBlockWithParents stages temporary data
|
||||
tc.lock.Lock()
|
||||
@ -77,7 +77,7 @@ func (tc *testConsensus) AddUTXOInvalidHeader(parentHashes []*externalapi.Domain
|
||||
return nil, nil, err
|
||||
}
|
||||
|
||||
blockInsertionResult, err := tc.blockProcessor.ValidateAndInsertBlock(&externalapi.DomainBlock{
|
||||
virtualChangeSet, err := tc.blockProcessor.ValidateAndInsertBlock(&externalapi.DomainBlock{
|
||||
Header: header,
|
||||
Transactions: nil,
|
||||
}, true)
|
||||
@ -85,11 +85,11 @@ func (tc *testConsensus) AddUTXOInvalidHeader(parentHashes []*externalapi.Domain
|
||||
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,
|
||||
*externalapi.BlockInsertionResult, error) {
|
||||
*externalapi.VirtualChangeSet, error) {
|
||||
|
||||
// Require write lock because BuildBlockWithParents stages temporary data
|
||||
tc.lock.Lock()
|
||||
@ -100,12 +100,12 @@ func (tc *testConsensus) AddUTXOInvalidBlock(parentHashes []*externalapi.DomainH
|
||||
return nil, nil, err
|
||||
}
|
||||
|
||||
blockInsertionResult, err := tc.blockProcessor.ValidateAndInsertBlock(block, true)
|
||||
virtualChangeSet, err := tc.blockProcessor.ValidateAndInsertBlock(block, true)
|
||||
if err != nil {
|
||||
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) {
|
||||
|
@ -94,25 +94,25 @@ func (ui *UTXOIndex) isSynced() (bool, error) {
|
||||
}
|
||||
|
||||
// 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")
|
||||
defer onEnd()
|
||||
|
||||
ui.mutex.Lock()
|
||||
defer ui.mutex.Unlock()
|
||||
|
||||
log.Tracef("Updating UTXO index with VirtualUTXODiff: %+v", blockInsertionResult.VirtualUTXODiff)
|
||||
err := ui.removeUTXOs(blockInsertionResult.VirtualUTXODiff.ToRemove())
|
||||
log.Tracef("Updating UTXO index with VirtualUTXODiff: %+v", virtualChangeSet.VirtualUTXODiff)
|
||||
err := ui.removeUTXOs(virtualChangeSet.VirtualUTXODiff.ToRemove())
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
err = ui.addUTXOs(blockInsertionResult.VirtualUTXODiff.ToAdd())
|
||||
err = ui.addUTXOs(virtualChangeSet.VirtualUTXODiff.ToAdd())
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
ui.store.updateVirtualParents(blockInsertionResult.VirtualParents)
|
||||
ui.store.updateVirtualParents(virtualChangeSet.VirtualParents)
|
||||
|
||||
added, removed, _ := ui.store.stagedData()
|
||||
utxoIndexChanges := &UTXOChanges{
|
||||
|
Loading…
x
Reference in New Issue
Block a user