From 57b1653383ee20f9cddeaac0d88de1a8d7d389c4 Mon Sep 17 00:00:00 2001 From: stasatdaglabs <39559713+stasatdaglabs@users.noreply.github.com> Date: Sun, 28 Jun 2020 14:27:01 +0300 Subject: [PATCH] [NOD-1063] Optimize deep reachability tree insertions (#773) * [NOD-1055] Give higher priority for requesting missing ancestors when sending a getdata message (#767) * [NOD-1063] Remove the remainingInterval field. * [NOD-1063] Add helper functions to reachabilityTreeNode. * [NOD-1063] Add reachabilityReindexRoot. * [NOD-1063] Start implementing findNextReachabilityReindexRoot. * [NOD-1063] Implement findCommonAncestor. * [NOD-1063] Implement findReachabilityTreeAncestorInChildren. * [NOD-1063] Add reachabilityReindexWindow. * [NOD-1063] Fix findReachabilityTreeAncestorInChildren. * [NOD-1063] Remove BlockDAG reference in findReachabilityTreeAncestorInChildren. * [NOD-1063] Extract updateReachabilityReindexRoot to a separate function. * [NOD-1063] Add reachabilityReindexSlack. * [NOD-1063] Implement splitReindexRootChildrenAroundChosen. * [NOD-1063] Implement calcReachabilityTreeNodeSizes. * [NOD-1063] Implement propagateChildIntervals. * [NOD-1063] Extract tightenReachabilityTreeIntervalsBeforeChosenReindexRootChild and tightenReachabilityTreeIntervalsAfterChosenReindexRootChild to separate functions. * [NOD-1063] Implement expandReachabilityTreeIntervalInChosenReindexRootChild. * [NOD-1063] Finished implementing concentrateReachabilityTreeIntervalAroundReindexRootChild. * [NOD-1063] Begin implementing reindexIntervalsBeforeReindexRoot. * [NOD-1063] Implement top-level logic of reindexIntervalsBeforeReindexRoot. * [NOD-1063] Implement reclaimIntervalBeforeChosenChild. * [NOD-1063] Add a debug log for reindexIntervalsBeforeReindexRoot. * [NOD-1063] Rename reindexIntervalsBeforeReindexRoot to reindexIntervalsEarlierThanReindexRoot. * [NOD-1063] Implement reclaimIntervalAfterChosenChild. * [NOD-1063] Add a debug log for updateReachabilityReindexRoot. * [NOD-1063] Convert modifiedTreeNodes from slices to sets. * [NOD-1063] Fix findCommonAncestor. * [NOD-1063] Fix reindexIntervalsEarlierThanReindexRoot.` * [NOD-1063] Remove redundant nil conditions. * [NOD-1063] Make map[*reachabilityTreeNode]struct{} into a type alias with a copyAllFrom method. * [NOD-1063] Remove setInterval. * [NOD-1063] Create a new struct to hold reachability stuff called reachabilityTree. * [NOD-1063] Rename functions under reachabilityTree. * [NOD-1063] Move reachabilityStore into reachabilityTree. * [NOD-1063] Move the rest of the functions in reachability.go into the reachabilityTree struct. * [NOD-1063] Update newReachabilityTree to take an instance of reachabilityStore. * [NOD-1063] Fix merge errors. * [NOD-1063] Fix merge errors. * [NOD-1063] Pass a reference to the dag into reachabilityTree. * [NOD-1063] Use Wrapf instead of Errorf. * [NOD-1063] Merge assignments. * [NOD-1063] Disambiguate a varaible name. * [NOD-1063] Add a test case for intervalBefore. * [NOD-1063] Simplify splitChildrenAroundChosenChild. * [NOD-1063] Fold temporary variables into newReachabilityInterval. * [NOD-1063] Fold more temporary variables into newReachabilityInterval. * [NOD-1063] Fix a bug in expandIntervalInReindexRootChosenChild. * [NOD-1063] Remove blockNode from futureCoveringBlock. * [NOD-1063] Get rid of futureCoveringBlock. * [NOD-1063] Use findIndex directly in findAncestorAmongChildren. * [NOD-1063] Make findIndex a bit nicer to use. Also rename it to findAncestorIndexOfNode. * [NOD-1063] Rename childIntervalAllocationRange to intervalRangeForChildAllocation. * [NOD-1063] Optimize findCommonAncestor. * [NOD-1063] In reindexIntervalsBeforeChosenChild, use chosenChild.interval.start - 1 instead of childrenBeforeChosen[len(childrenBeforeChosen)-1].interval.end + 1. * [NOD-1063] Rename reindexIntervalsBeforeChosenChild to reindexIntervalsBeforeNode. * [NOD-1063] Add a comment explain what "the chosen child" is. * [NOD-1063] In concentrateIntervalAroundReindexRootChosenChild, rename modifiedTreeNodes to allModifiedTreeNodes. * [NOD-1063] Extract propagateIntervals to a function. * [NOD-1063] Extract interval "contains" logic to a separate function. * [NOD-1063] Simplify "looping up" logic in reclaimIntervalXXXChosenChild. * [NOD-1063] Add comments to reclaimIntervalXXXChosenChild. * [NOD-1063] Rename copyAllFrom to addAll. * [NOD-1063] Rename reachabilityStore (the variable) to just store. * [NOD-1063] Fix an error message. * [NOD-1063] Reword a comment. * [NOD-1063] Don't return -1 from findAncestorIndexOfNode. * [NOD-1063] Extract slackReachabilityIntervalForReclaiming to a constant. * [NOD-1063] Add a missing condition. * [NOD-1063] Call isAncestorOf directly in insertNode. * [NOD-1063] Rename chosenReindexRootChild to reindexRootChosenChild. * [NOD-1063] Rename treeNodeSet to orderedTreeNodeSet. * [NOD-1063] Add a disclaimer to orderedTreeNodeSet. * [NOD-1063] Implement StoreReachabilityReindexRoot and FetchReachabilityReindexRoot. * [NOD-1063] Move storing the reindex root to within reachabilityTree. * [NOD-1063] Remove isAncestorOf from reachabilityInterval. * [NOD-1063] Add a comment about graph theory conventions. * [NOD-1063] Fix tests. * [NOD-1063] Change inclusion in isAncestorOf functions. * [NOD-1063] Rename a test. * [NOD-1063] Implement TestIsInFuture. * [NOD-1063] Fix error messages in TestIsInFuture. * [NOD-1063] Fix error messages in TestIsInFuture. * [NOD-1063] Rename isInSelectedParentChain to isInSelectedParentChainOf. * [NOD-1063] Rename isInFuture to isInPast. * [NOD-1063] Expand on a comment. * [NOD-1063] Rename modifiedTreeNodes. * [NOD-1063] Implement test: TestReindexIntervalsEarlierThanReindexRoot. * [NOD-1063] Implement test: TestUpdateReindexRoot. * [NOD-1063] Explain a check. * [NOD-1063] Use a method instead of calling reachabilityStore.loaded directly. * [NOD-1063] Lowercasified an error message. * [NOD-1063] Fix failing test. Co-authored-by: Ori Newman --- blockdag/dag.go | 44 +- blockdag/dagio.go | 5 +- blockdag/ghostdag.go | 6 +- blockdag/ghostdag_test.go | 4 +- blockdag/reachability.go | 886 +++++++++++++++++++++++++++++----- blockdag/reachability_test.go | 539 +++++++++++++++------ blockdag/reachabilitystore.go | 56 +-- blockdag/validate.go | 2 +- dbaccess/reachability.go | 24 + 9 files changed, 1226 insertions(+), 340 deletions(-) diff --git a/blockdag/dag.go b/blockdag/dag.go index ab3022b9d..9280d254d 100644 --- a/blockdag/dag.go +++ b/blockdag/dag.go @@ -151,9 +151,10 @@ type BlockDAG struct { lastFinalityPoint *blockNode - utxoDiffStore *utxoDiffStore - reachabilityStore *reachabilityStore - multisetStore *multisetStore + utxoDiffStore *utxoDiffStore + multisetStore *multisetStore + + reachabilityTree *reachabilityTree recentBlockProcessingTimestamps []time.Time startTime time.Time @@ -700,7 +701,7 @@ func (dag *BlockDAG) saveChangesFromBlock(block *util.Block, virtualUTXODiff *UT return err } - err = dag.reachabilityStore.flushToDB(dbTx) + err = dag.reachabilityTree.storeState(dbTx) if err != nil { return err } @@ -760,7 +761,7 @@ func (dag *BlockDAG) saveChangesFromBlock(block *util.Block, virtualUTXODiff *UT dag.index.clearDirtyEntries() dag.utxoDiffStore.clearDirtyEntries() dag.utxoDiffStore.clearOldEntries() - dag.reachabilityStore.clearDirtyEntries() + dag.reachabilityTree.store.clearDirtyEntries() dag.multisetStore.clearNewEntries() return nil @@ -816,19 +817,14 @@ func (dag *BlockDAG) LastFinalityPointHash() *daghash.Hash { return dag.lastFinalityPoint.hash } -// isInSelectedParentChain returns whether aNode is in the selected parent chain of bNode. -func (dag *BlockDAG) isInSelectedParentChain(aNode, bNode *blockNode) (bool, error) { - aTreeNode, err := dag.reachabilityStore.treeNodeByBlockNode(aNode) - if err != nil { - return false, err +// isInSelectedParentChainOf returns whether `node` is in the selected parent chain of `other`. +func (dag *BlockDAG) isInSelectedParentChainOf(node *blockNode, other *blockNode) (bool, error) { + // By definition, a node is not in the selected parent chain of itself. + if node == other { + return false, nil } - bTreeNode, err := dag.reachabilityStore.treeNodeByBlockNode(bNode) - if err != nil { - return false, err - } - - return aTreeNode.interval.isAncestorOf(bTreeNode.interval), nil + return dag.reachabilityTree.isReachabilityTreeAncestorOf(node, other) } // checkFinalityViolation checks the new block does not violate the finality rules @@ -848,7 +844,7 @@ func (dag *BlockDAG) checkFinalityViolation(newNode *blockNode) error { return nil } - isInSelectedChain, err := dag.isInSelectedParentChain(dag.lastFinalityPoint, newNode.selectedParent) + isInSelectedChain, err := dag.isInSelectedParentChainOf(dag.lastFinalityPoint, newNode.selectedParent) if err != nil { return err } @@ -995,10 +991,10 @@ func (dag *BlockDAG) applyDAGChanges(node *blockNode, newBlockPastUTXO UTXOSet, newBlockMultiset *secp256k1.MultiSet, selectedParentAnticone []*blockNode) ( virtualUTXODiff *UTXODiff, chainUpdates *chainUpdates, err error) { - // Add the block to the reachability structures - err = dag.updateReachability(node, selectedParentAnticone) + // Add the block to the reachability tree + err = dag.reachabilityTree.addBlock(node, selectedParentAnticone) if err != nil { - return nil, nil, errors.Wrap(err, "failed updating reachability") + return nil, nil, errors.Wrap(err, "failed adding block to the reachability tree") } dag.multisetStore.setMultiset(node, newBlockMultiset) @@ -1785,7 +1781,7 @@ func (dag *BlockDAG) antiPastBetween(lowHash, highHash *daghash.Hash, maxEntries continue } visited.add(current) - isCurrentAncestorOfLowNode, err := dag.isAncestorOf(current, lowNode) + isCurrentAncestorOfLowNode, err := dag.isInPast(current, lowNode) if err != nil { return nil, err } @@ -1811,6 +1807,10 @@ func (dag *BlockDAG) antiPastBetween(lowHash, highHash *daghash.Hash, maxEntries return nodes, nil } +func (dag *BlockDAG) isInPast(this *blockNode, other *blockNode) (bool, error) { + return dag.reachabilityTree.isInPast(this, other) +} + // AntiPastHashesBetween returns the hashes of the blocks between the // lowHash's antiPast and highHash's antiPast, or up to the provided // max number of block hashes. @@ -2064,8 +2064,8 @@ func New(config *Config) (*BlockDAG, error) { dag.virtual = newVirtualBlock(dag, nil) dag.utxoDiffStore = newUTXODiffStore(dag) - dag.reachabilityStore = newReachabilityStore(dag) dag.multisetStore = newMultisetStore(dag) + dag.reachabilityTree = newReachabilityTree(dag) // Initialize the DAG state from the passed database. When the db // does not yet contain any DAG state, both it and the DAG state diff --git a/blockdag/dagio.go b/blockdag/dagio.go index 32c01877e..9300ecaf9 100644 --- a/blockdag/dagio.go +++ b/blockdag/dagio.go @@ -209,7 +209,7 @@ func (dag *BlockDAG) initDAGState() error { } log.Debugf("Loading reachability data...") - err = dag.reachabilityStore.init(dbaccess.NoTx()) + err = dag.reachabilityTree.init(dbaccess.NoTx()) if err != nil { return err } @@ -236,7 +236,8 @@ func (dag *BlockDAG) initDAGState() error { var ok bool dag.lastFinalityPoint, ok = dag.index.LookupNode(dagState.LastFinalityPoint) if !ok { - return errors.Errorf("block %s does not exist in the DAG", dagState.LastFinalityPoint) + return errors.Errorf("finality point block %s "+ + "does not exist in the DAG", dagState.LastFinalityPoint) } dag.finalizeNodesBelowFinalityPoint(false) diff --git a/blockdag/ghostdag.go b/blockdag/ghostdag.go index 6f8f488fc..8c2a5e240 100644 --- a/blockdag/ghostdag.go +++ b/blockdag/ghostdag.go @@ -57,7 +57,7 @@ func (dag *BlockDAG) ghostdag(newNode *blockNode) (selectedParentAnticone []*blo // newNode is always in the future of blueCandidate, so there's // no point in checking it. if chainBlock != newNode { - if isAncestorOfBlueCandidate, err := dag.isAncestorOf(chainBlock, blueCandidate); err != nil { + if isAncestorOfBlueCandidate, err := dag.isInPast(chainBlock, blueCandidate); err != nil { return nil, err } else if isAncestorOfBlueCandidate { break @@ -66,7 +66,7 @@ func (dag *BlockDAG) ghostdag(newNode *blockNode) (selectedParentAnticone []*blo for _, block := range chainBlock.blues { // Skip blocks that exist in the past of blueCandidate. - if isAncestorOfBlueCandidate, err := dag.isAncestorOf(block, blueCandidate); err != nil { + if isAncestorOfBlueCandidate, err := dag.isInPast(block, blueCandidate); err != nil { return nil, err } else if isAncestorOfBlueCandidate { continue @@ -148,7 +148,7 @@ func (dag *BlockDAG) selectedParentAnticone(node *blockNode) ([]*blockNode, erro if anticoneSet.contains(parent) || selectedParentPast.contains(parent) { continue } - isAncestorOfSelectedParent, err := dag.isAncestorOf(parent, node.selectedParent) + isAncestorOfSelectedParent, err := dag.isInPast(parent, node.selectedParent) if err != nil { return nil, err } diff --git a/blockdag/ghostdag_test.go b/blockdag/ghostdag_test.go index 1556db239..29686d99c 100644 --- a/blockdag/ghostdag_test.go +++ b/blockdag/ghostdag_test.go @@ -349,7 +349,7 @@ func TestGHOSTDAGErrors(t *testing.T) { block3 := prepareAndProcessBlockByParentMsgBlocks(t, dag, block1, block2) // Clear the reachability store - dag.reachabilityStore.loaded = map[daghash.Hash]*reachabilityData{} + dag.reachabilityTree.store.loaded = map[daghash.Hash]*reachabilityData{} dbTx, err := dbaccess.NewTx() if err != nil { @@ -377,7 +377,7 @@ func TestGHOSTDAGErrors(t *testing.T) { if err == nil { t.Fatalf("TestGHOSTDAGErrors: ghostdag unexpectedly succeeded") } - expectedErrSubstring := "Couldn't find reachability data" + expectedErrSubstring := "couldn't find reachability data" if !strings.Contains(err.Error(), expectedErrSubstring) { t.Fatalf("TestGHOSTDAGErrors: ghostdag returned wrong error. "+ "Want: %s, got: %s", expectedErrSubstring, err) diff --git a/blockdag/reachability.go b/blockdag/reachability.go index e16544826..fa5a1ec2b 100644 --- a/blockdag/reachability.go +++ b/blockdag/reachability.go @@ -2,12 +2,52 @@ package blockdag import ( "fmt" + "github.com/kaspanet/kaspad/dbaccess" "github.com/pkg/errors" "math" "strings" "time" ) +var ( + // reachabilityReindexWindow is the target window size for reachability + // reindexes. Note that this is not a constant for testing purposes. + reachabilityReindexWindow uint64 = 200 + + // reachabilityReindexSlack is the slack interval given to reachability + // tree nodes not in the selected parent chain. Note that this is not + // a constant for testing purposes. + reachabilityReindexSlack uint64 = 1 << 12 + + // slackReachabilityIntervalForReclaiming is the slack interval to + // reclaim during reachability reindexes earlier than the reindex root. + // See reclaimIntervalBeforeChosenChild for further details. Note that + // this is not a constant for testing purposes. + slackReachabilityIntervalForReclaiming uint64 = 1 +) + +// modifiedTreeNodes are a set of reachabilityTreeNodes that's bubbled up +// from any function that modifies them, so that the original caller may +// update the database accordingly. This is a set rather than a slice due +// to frequent duplicate treeNodes between operations. +type modifiedTreeNodes map[*reachabilityTreeNode]struct{} + +func newModifiedTreeNodes(nodes ...*reachabilityTreeNode) modifiedTreeNodes { + modifiedNodes := make(modifiedTreeNodes) + for _, node := range nodes { + modifiedNodes[node] = struct{}{} + } + return modifiedNodes +} + +// addAll adds all the reachabilityTreeNodes in `other` +// into `mtn`. Note that `other` is not affected. +func (mtn modifiedTreeNodes) addAll(other modifiedTreeNodes) { + for node := range other { + mtn[node] = struct{}{} + } +} + // reachabilityInterval represents an interval to be used within the // tree reachability algorithm. See reachabilityTreeNode for further // details. @@ -150,15 +190,8 @@ func exponentialFractions(sizes []uint64) []float64 { return fractions } -// isAncestorOf checks if this interval's node is a reachability tree -// ancestor of the other interval's node. The condition below is relying on the -// property of reachability intervals that intervals are either completely disjoint, -// or one strictly contains the other. -func (ri *reachabilityInterval) isAncestorOf(other *reachabilityInterval) bool { - // An interval is not an ancestor of itself. - if ri.start == other.start && ri.end == other.end { - return false - } +// contains returns true if ri contains other. +func (ri *reachabilityInterval) contains(other *reachabilityInterval) bool { return ri.start <= other.start && other.end <= ri.end } @@ -191,33 +224,78 @@ type reachabilityTreeNode struct { // interval is the index interval containing all intervals of // blocks in this node's subtree interval *reachabilityInterval - - // remainingInterval is the not-yet allocated interval (within - // this node's interval) awaiting new children - remainingInterval *reachabilityInterval } func newReachabilityTreeNode(blockNode *blockNode) *reachabilityTreeNode { // Please see the comment above reachabilityTreeNode to understand why // we use these initial values. interval := newReachabilityInterval(1, math.MaxUint64-1) - // We subtract 1 from the end of the remaining interval to prevent the node from allocating + return &reachabilityTreeNode{blockNode: blockNode, interval: interval} +} + +func (rtn *reachabilityTreeNode) intervalRangeForChildAllocation() *reachabilityInterval { + // We subtract 1 from the end of the range to prevent the node from allocating // the entire interval to its child, so its interval would *strictly* contain the interval of its child. - remainingInterval := newReachabilityInterval(interval.start, interval.end-1) - return &reachabilityTreeNode{blockNode: blockNode, interval: interval, remainingInterval: remainingInterval} + return newReachabilityInterval(rtn.interval.start, rtn.interval.end-1) +} + +func (rtn *reachabilityTreeNode) remainingIntervalBefore() *reachabilityInterval { + childRange := rtn.intervalRangeForChildAllocation() + if len(rtn.children) == 0 { + return childRange + } + return newReachabilityInterval(childRange.start, rtn.children[0].interval.start-1) +} + +func (rtn *reachabilityTreeNode) remainingIntervalAfter() *reachabilityInterval { + childRange := rtn.intervalRangeForChildAllocation() + if len(rtn.children) == 0 { + return childRange + } + return newReachabilityInterval(rtn.children[len(rtn.children)-1].interval.end+1, childRange.end) +} + +func (rtn *reachabilityTreeNode) hasSlackIntervalBefore() bool { + return rtn.remainingIntervalBefore().size() > 0 +} + +func (rtn *reachabilityTreeNode) hasSlackIntervalAfter() bool { + return rtn.remainingIntervalAfter().size() > 0 } // addChild adds child to this tree node. If this node has no // remaining interval to allocate, a reindexing is triggered. // This method returns a list of reachabilityTreeNodes modified // by it. -func (rtn *reachabilityTreeNode) addChild(child *reachabilityTreeNode) ([]*reachabilityTreeNode, error) { +func (rtn *reachabilityTreeNode) addChild(child *reachabilityTreeNode, reindexRoot *reachabilityTreeNode) ( + modifiedTreeNodes, error) { + + remaining := rtn.remainingIntervalAfter() + // Set the parent-child relationship rtn.children = append(rtn.children, child) child.parent = rtn + // Handle rtn not being a descendant of the reindex root. + // Note that we check rtn here instead of child because + // at this point we don't yet know child's interval. + if !reindexRoot.isAncestorOf(rtn) { + reindexStartTime := time.Now() + modifiedNodes, err := rtn.reindexIntervalsEarlierThanReindexRoot(reindexRoot) + if err != nil { + return nil, err + } + reindexTimeElapsed := time.Since(reindexStartTime) + log.Debugf("Reachability reindex triggered for "+ + "block %s. This block is not a child of the current "+ + "reindex root %s. Modified %d tree nodes and took %dms.", + rtn.blockNode.hash, reindexRoot.blockNode.hash, + len(modifiedNodes), reindexTimeElapsed.Milliseconds()) + return modifiedNodes, nil + } + // No allocation space left -- reindex - if rtn.remainingInterval.size() == 0 { + if remaining.size() == 0 { reindexStartTime := time.Now() modifiedNodes, err := rtn.reindexIntervals() if err != nil { @@ -231,23 +309,12 @@ func (rtn *reachabilityTreeNode) addChild(child *reachabilityTreeNode) ([]*reach } // Allocate from the remaining space - allocated, remaining, err := rtn.remainingInterval.splitInHalf() + allocated, _, err := remaining.splitInHalf() if err != nil { return nil, err } - child.setInterval(allocated) - rtn.remainingInterval = remaining - return []*reachabilityTreeNode{rtn, child}, nil -} - -// setInterval sets the reachability interval for this node. -func (rtn *reachabilityTreeNode) setInterval(interval *reachabilityInterval) { - rtn.interval = interval - - // Reserve a single interval index for the current node. This - // is necessary to ensure that ancestor intervals are strictly - // supersets of any descendant intervals and not equal - rtn.remainingInterval = newReachabilityInterval(interval.start, interval.end-1) + child.interval = allocated + return newModifiedTreeNodes(rtn, child), nil } // reindexIntervals traverses the reachability subtree that's @@ -257,7 +324,7 @@ func (rtn *reachabilityTreeNode) setInterval(interval *reachabilityInterval) { // tree until it finds a node with a subreeSize that's greater than // its interval size. See propagateInterval for further details. // This method returns a list of reachabilityTreeNodes modified by it. -func (rtn *reachabilityTreeNode) reindexIntervals() ([]*reachabilityTreeNode, error) { +func (rtn *reachabilityTreeNode) reindexIntervals() (modifiedTreeNodes, error) { current := rtn // Initial interval and subtree sizes @@ -348,11 +415,11 @@ func (rtn *reachabilityTreeNode) countSubtrees(subTreeSizeMap map[*reachabilityT // Subtree intervals are recursively allocated according to subtree sizes and // the allocation rule in splitWithExponentialBias. This method returns // a list of reachabilityTreeNodes modified by it. -func (rtn *reachabilityTreeNode) propagateInterval(subTreeSizeMap map[*reachabilityTreeNode]uint64) ([]*reachabilityTreeNode, error) { - // We set the interval to reset its remainingInterval, so we could reallocate it while reindexing. - rtn.setInterval(rtn.interval) +func (rtn *reachabilityTreeNode) propagateInterval(subTreeSizeMap map[*reachabilityTreeNode]uint64) ( + modifiedTreeNodes, error) { + + allModifiedTreeNodes := newModifiedTreeNodes() queue := []*reachabilityTreeNode{rtn} - var modifiedNodes []*reachabilityTreeNode for len(queue) > 0 { var current *reachabilityTreeNode current, queue = queue[0], queue[1:] @@ -361,29 +428,245 @@ func (rtn *reachabilityTreeNode) propagateInterval(subTreeSizeMap map[*reachabil for i, child := range current.children { sizes[i] = subTreeSizeMap[child] } - intervals, err := current.remainingInterval.splitWithExponentialBias(sizes) + intervals, err := current.intervalRangeForChildAllocation().splitWithExponentialBias(sizes) if err != nil { return nil, err } for i, child := range current.children { childInterval := intervals[i] - child.setInterval(childInterval) + child.interval = childInterval queue = append(queue, child) } - - // Empty up remaining interval - current.remainingInterval.start = current.remainingInterval.end + 1 } - modifiedNodes = append(modifiedNodes, current) + allModifiedTreeNodes[current] = struct{}{} } - return modifiedNodes, nil + return allModifiedTreeNodes, nil +} + +func (rtn *reachabilityTreeNode) reindexIntervalsEarlierThanReindexRoot( + reindexRoot *reachabilityTreeNode) (modifiedTreeNodes, error) { + + // Find the common ancestor for both rtn and the reindex root + commonAncestor := rtn.findCommonAncestorWithReindexRoot(reindexRoot) + + // The chosen child is: + // a. A reachability tree child of `commonAncestor` + // b. A reachability tree ancestor of `reindexRoot` + commonAncestorChosenChild, err := commonAncestor.findAncestorAmongChildren(reindexRoot) + if err != nil { + return nil, err + } + + if rtn.interval.end < commonAncestorChosenChild.interval.start { + // rtn is in the subtree before the chosen child + return rtn.reclaimIntervalBeforeChosenChild(commonAncestor, commonAncestorChosenChild, reindexRoot) + } + if commonAncestorChosenChild.interval.end < rtn.interval.start { + // rtn is in the subtree after the chosen child + return rtn.reclaimIntervalAfterChosenChild(commonAncestor, commonAncestorChosenChild, reindexRoot) + } + return nil, errors.Errorf("rtn is in the chosen child's subtree") +} + +func (rtn *reachabilityTreeNode) reclaimIntervalBeforeChosenChild( + commonAncestor *reachabilityTreeNode, commonAncestorChosenChild *reachabilityTreeNode, reindexRoot *reachabilityTreeNode) ( + modifiedTreeNodes, error) { + + allModifiedTreeNodes := newModifiedTreeNodes() + + current := commonAncestorChosenChild + if !commonAncestorChosenChild.hasSlackIntervalBefore() { + // The common ancestor ran out of slack before its chosen child. + // Climb up the reachability tree toward the reindex root until + // we find a node that has enough slack. + for !current.hasSlackIntervalBefore() && current != reindexRoot { + var err error + current, err = current.findAncestorAmongChildren(reindexRoot) + if err != nil { + return nil, err + } + } + + if current == reindexRoot { + // "Deallocate" an interval of slackReachabilityIntervalForReclaiming + // from this node. This is the interval that we'll use for the new + // node. + originalInterval := current.interval + current.interval = newReachabilityInterval( + current.interval.start+slackReachabilityIntervalForReclaiming, + current.interval.end, + ) + modifiedNodes, err := current.countSubtreesAndPropagateInterval() + if err != nil { + return nil, err + } + allModifiedTreeNodes.addAll(modifiedNodes) + current.interval = originalInterval + } + } + + // Go down the reachability tree towards the common ancestor. + // On every hop we reindex the reachability subtree before the + // current node with an interval that is smaller by + // slackReachabilityIntervalForReclaiming. This is to make room + // for the new node. + for current != commonAncestor { + current.interval = newReachabilityInterval( + current.interval.start+slackReachabilityIntervalForReclaiming, + current.interval.end, + ) + modifiedNodes, err := current.parent.reindexIntervalsBeforeNode(current) + if err != nil { + return nil, err + } + allModifiedTreeNodes.addAll(modifiedNodes) + current = current.parent + } + + return allModifiedTreeNodes, nil +} + +// reindexIntervalsBeforeNode applies a tight interval to the reachability +// subtree before `node`. Note that `node` itself is unaffected. +func (rtn *reachabilityTreeNode) reindexIntervalsBeforeNode(node *reachabilityTreeNode) ( + modifiedTreeNodes, error) { + + childrenBeforeNode, _, err := rtn.splitChildrenAroundChild(node) + if err != nil { + return nil, err + } + + childrenBeforeNodeSizes, childrenBeforeNodeSubtreeSizeMaps, childrenBeforeNodeSizesSum := + calcReachabilityTreeNodeSizes(childrenBeforeNode) + + // Apply a tight interval + newIntervalEnd := node.interval.start - 1 + newInterval := newReachabilityInterval(newIntervalEnd-childrenBeforeNodeSizesSum+1, newIntervalEnd) + intervals, err := newInterval.splitExact(childrenBeforeNodeSizes) + if err != nil { + return nil, err + } + return orderedTreeNodeSet(childrenBeforeNode).propagateIntervals(intervals, childrenBeforeNodeSubtreeSizeMaps) +} + +func (rtn *reachabilityTreeNode) reclaimIntervalAfterChosenChild( + commonAncestor *reachabilityTreeNode, commonAncestorChosenChild *reachabilityTreeNode, reindexRoot *reachabilityTreeNode) ( + modifiedTreeNodes, error) { + + allModifiedTreeNodes := newModifiedTreeNodes() + + current := commonAncestorChosenChild + if !commonAncestorChosenChild.hasSlackIntervalAfter() { + // The common ancestor ran out of slack after its chosen child. + // Climb up the reachability tree toward the reindex root until + // we find a node that has enough slack. + for !current.hasSlackIntervalAfter() && current != reindexRoot { + var err error + current, err = current.findAncestorAmongChildren(reindexRoot) + if err != nil { + return nil, err + } + } + + if current == reindexRoot { + // "Deallocate" an interval of slackReachabilityIntervalForReclaiming + // from this node. This is the interval that we'll use for the new + // node. + originalInterval := current.interval + current.interval = newReachabilityInterval( + current.interval.start, + current.interval.end-slackReachabilityIntervalForReclaiming, + ) + modifiedNodes, err := current.countSubtreesAndPropagateInterval() + if err != nil { + return nil, err + } + allModifiedTreeNodes.addAll(modifiedNodes) + current.interval = originalInterval + } + } + + // Go down the reachability tree towards the common ancestor. + // On every hop we reindex the reachability subtree after the + // current node with an interval that is smaller by + // slackReachabilityIntervalForReclaiming. This is to make room + // for the new node. + for current != commonAncestor { + current.interval = newReachabilityInterval( + current.interval.start, + current.interval.end-slackReachabilityIntervalForReclaiming, + ) + modifiedNodes, err := current.parent.reindexIntervalsAfterNode(current) + if err != nil { + return nil, err + } + allModifiedTreeNodes.addAll(modifiedNodes) + current = current.parent + } + + return allModifiedTreeNodes, nil +} + +// reindexIntervalsAfterNode applies a tight interval to the reachability +// subtree after `node`. Note that `node` itself is unaffected. +func (rtn *reachabilityTreeNode) reindexIntervalsAfterNode(node *reachabilityTreeNode) ( + modifiedTreeNodes, error) { + + _, childrenAfterNode, err := rtn.splitChildrenAroundChild(node) + if err != nil { + return nil, err + } + + childrenAfterNodeSizes, childrenAfterNodeSubtreeSizeMaps, childrenAfterNodeSizesSum := + calcReachabilityTreeNodeSizes(childrenAfterNode) + + // Apply a tight interval + newIntervalStart := node.interval.end + 1 + newInterval := newReachabilityInterval(newIntervalStart, newIntervalStart+childrenAfterNodeSizesSum-1) + intervals, err := newInterval.splitExact(childrenAfterNodeSizes) + if err != nil { + return nil, err + } + return orderedTreeNodeSet(childrenAfterNode).propagateIntervals(intervals, childrenAfterNodeSubtreeSizeMaps) +} + +func (tns orderedTreeNodeSet) propagateIntervals(intervals []*reachabilityInterval, + subtreeSizeMaps []map[*reachabilityTreeNode]uint64) (modifiedTreeNodes, error) { + + allModifiedTreeNodes := newModifiedTreeNodes() + for i, node := range tns { + node.interval = intervals[i] + subtreeSizeMap := subtreeSizeMaps[i] + modifiedNodes, err := node.propagateInterval(subtreeSizeMap) + if err != nil { + return nil, err + } + allModifiedTreeNodes.addAll(modifiedNodes) + } + return allModifiedTreeNodes, nil } // isAncestorOf checks if this node is a reachability tree ancestor -// of the other node. +// of the other node. Note that we use the graph theory convention +// here which defines that rtn is also an ancestor of itself. func (rtn *reachabilityTreeNode) isAncestorOf(other *reachabilityTreeNode) bool { - return rtn.interval.isAncestorOf(other.interval) + return rtn.interval.contains(other.interval) +} + +// findCommonAncestorWithReindexRoot finds the most recent reachability +// tree ancestor common to both rtn and the given reindex root. Note +// that we assume that almost always the chain between the reindex root +// and the common ancestor is longer than the chain between rtn and the +// common ancestor. +func (rtn *reachabilityTreeNode) findCommonAncestorWithReindexRoot(reindexRoot *reachabilityTreeNode) *reachabilityTreeNode { + currentThis := rtn + for { + if currentThis.isAncestorOf(reindexRoot) { + return currentThis + } + currentThis = currentThis.parent + } } // String returns a string representation of a reachability tree node @@ -408,174 +691,527 @@ func (rtn *reachabilityTreeNode) String() string { return strings.Join(lines, "\n") } -// futureCoveringBlockSet represents a collection of blocks in the future of +// orderedTreeNodeSet is an ordered set of reachabilityTreeNodes +// Note that this type does not validate order validity. It's the +// responsibility of the caller to construct instances of this +// type properly. +type orderedTreeNodeSet []*reachabilityTreeNode + +// futureCoveringTreeNodeSet represents a collection of blocks in the future of // a certain block. Once a block B is added to the DAG, every block A_i in -// B's selected parent anticone must register B in its futureCoveringBlockSet. This allows -// to relatively quickly (O(log(|futureCoveringBlockSet|))) query whether B +// B's selected parent anticone must register B in its futureCoveringTreeNodeSet. This allows +// to relatively quickly (O(log(|futureCoveringTreeNodeSet|))) query whether B // is a descendent (is in the "future") of any block that previously // registered it. // -// Note that futureCoveringBlockSet is meant to be queried only if B is not +// Note that futureCoveringTreeNodeSet is meant to be queried only if B is not // a reachability tree descendant of the block in question, as reachability // tree queries are always O(1). // -// See insertBlock, isInFuture, and dag.isAncestorOf for further details. -type futureCoveringBlockSet []*futureCoveringBlock +// See insertNode, hasAncestorOf, and reachabilityTree.isInPast for further +// details. +type futureCoveringTreeNodeSet orderedTreeNodeSet -// futureCoveringBlock represents a block in the future of some other block. -type futureCoveringBlock struct { - blockNode *blockNode - treeNode *reachabilityTreeNode -} - -// insertBlock inserts the given block into this futureCoveringBlockSet -// while keeping futureCoveringBlockSet ordered by interval. -// If a block B ∈ futureCoveringBlockSet exists such that its interval +// insertNode inserts the given block into this futureCoveringTreeNodeSet +// while keeping futureCoveringTreeNodeSet ordered by interval. +// If a block B ∈ futureCoveringTreeNodeSet exists such that its interval // contains block's interval, block need not be added. If block's // interval contains B's interval, it replaces it. // // Notes: // * Intervals never intersect unless one contains the other // (this follows from the tree structure and the indexing rule). -// * Since futureCoveringBlockSet is kept ordered, a binary search can be +// * Since futureCoveringTreeNodeSet is kept ordered, a binary search can be // used for insertion/queries. // * Although reindexing may change a block's interval, the // is-superset relation will by definition // be always preserved. -func (fb *futureCoveringBlockSet) insertBlock(block *futureCoveringBlock) { - blockInterval := block.treeNode.interval - i := fb.findIndex(block) - if i > 0 { - candidate := (*fb)[i-1] - candidateInterval := candidate.treeNode.interval - if candidateInterval.isAncestorOf(blockInterval) { - // candidate is an ancestor of block, no need to insert - return - } - if blockInterval.isAncestorOf(candidateInterval) { - // block is an ancestor of candidate, and can thus replace it - (*fb)[i-1] = block - return - } +func (fb *futureCoveringTreeNodeSet) insertNode(node *reachabilityTreeNode) { + ancestorIndex, ok := orderedTreeNodeSet(*fb).findAncestorIndexOfNode(node) + if !ok { + *fb = append([]*reachabilityTreeNode{node}, *fb...) + return } - // Insert block in the correct index to maintain futureCoveringBlockSet as + candidate := (*fb)[ancestorIndex] + if candidate.isAncestorOf(node) { + // candidate is an ancestor of node, no need to insert + return + } + if node.isAncestorOf(candidate) { + // node is an ancestor of candidate, and can thus replace it + (*fb)[ancestorIndex] = node + return + } + + // Insert node in the correct index to maintain futureCoveringTreeNodeSet as // a sorted-by-interval list. - // Note that i might be equal to len(futureCoveringBlockSet) - left := (*fb)[:i] - right := append([]*futureCoveringBlock{block}, (*fb)[i:]...) + // Note that ancestorIndex might be equal to len(futureCoveringTreeNodeSet) + left := (*fb)[:ancestorIndex+1] + right := append([]*reachabilityTreeNode{node}, (*fb)[ancestorIndex+1:]...) *fb = append(left, right...) } -// isInFuture resolves whether the given block is in the subtree of -// any block in this futureCoveringBlockSet. -// See insertBlock method for the complementary insertion behavior. +// hasAncestorOf resolves whether the given node is in the subtree of +// any node in this futureCoveringTreeNodeSet. +// See insertNode method for the complementary insertion behavior. // // Like the insert method, this method also relies on the fact that -// futureCoveringBlockSet is kept ordered by interval to efficiently perform a -// binary search over futureCoveringBlockSet and answer the query in -// O(log(|futureCoveringBlockSet|)). -func (fb futureCoveringBlockSet) isInFuture(block *futureCoveringBlock) bool { - i := fb.findIndex(block) - if i == 0 { - // No candidate to contain block +// futureCoveringTreeNodeSet is kept ordered by interval to efficiently perform a +// binary search over futureCoveringTreeNodeSet and answer the query in +// O(log(|futureCoveringTreeNodeSet|)). +func (fb futureCoveringTreeNodeSet) hasAncestorOf(node *reachabilityTreeNode) bool { + ancestorIndex, ok := orderedTreeNodeSet(fb).findAncestorIndexOfNode(node) + if !ok { + // No candidate to contain node return false } - candidate := fb[i-1] - return candidate.treeNode.isAncestorOf(block.treeNode) + candidate := fb[ancestorIndex] + return candidate.isAncestorOf(node) } -// findIndex finds the index of the block with the maximum start that is below -// the given block. -func (fb futureCoveringBlockSet) findIndex(block *futureCoveringBlock) int { - blockInterval := block.treeNode.interval +// findAncestorOfNode finds the reachability tree ancestor of `node` +// among the nodes in `tns`. +func (tns orderedTreeNodeSet) findAncestorOfNode(node *reachabilityTreeNode) (*reachabilityTreeNode, bool) { + ancestorIndex, ok := tns.findAncestorIndexOfNode(node) + if !ok { + return nil, false + } + return tns[ancestorIndex], true +} + +// findAncestorIndexOfNode finds the index of the reachability tree +// ancestor of `node` among the nodes in `tns`. It does so by finding +// the index of the block with the maximum start that is below the +// given block. +func (tns orderedTreeNodeSet) findAncestorIndexOfNode(node *reachabilityTreeNode) (int, bool) { + blockInterval := node.interval end := blockInterval.end low := 0 - high := len(fb) + high := len(tns) for low < high { middle := (low + high) / 2 - middleInterval := fb[middle].treeNode.interval + middleInterval := tns[middle].interval if end < middleInterval.start { high = middle } else { low = middle + 1 } } - return low + + if low == 0 { + return 0, false + } + return low - 1, true } -// String returns a string representation of the intervals in this futureCoveringBlockSet. -func (fb futureCoveringBlockSet) String() string { +// String returns a string representation of the intervals in this futureCoveringTreeNodeSet. +func (fb futureCoveringTreeNodeSet) String() string { intervalsString := "" - for _, block := range fb { - intervalsString += block.treeNode.interval.String() + for _, node := range fb { + intervalsString += node.interval.String() } return intervalsString } -func (dag *BlockDAG) updateReachability(node *blockNode, selectedParentAnticone []*blockNode) error { +func (rt *reachabilityTree) addBlock(node *blockNode, selectedParentAnticone []*blockNode) error { // Allocate a new reachability tree node newTreeNode := newReachabilityTreeNode(node) // If this is the genesis node, simply initialize it and return if node.isGenesis() { - dag.reachabilityStore.setTreeNode(newTreeNode) + rt.store.setTreeNode(newTreeNode) + rt.reindexRoot = newTreeNode return nil } // Insert the node into the selected parent's reachability tree - selectedParentTreeNode, err := dag.reachabilityStore.treeNodeByBlockNode(node.selectedParent) + selectedParentTreeNode, err := rt.store.treeNodeByBlockNode(node.selectedParent) if err != nil { return err } - modifiedTreeNodes, err := selectedParentTreeNode.addChild(newTreeNode) + modifiedNodes, err := selectedParentTreeNode.addChild(newTreeNode, rt.reindexRoot) if err != nil { return err } - for _, modifiedTreeNode := range modifiedTreeNodes { - dag.reachabilityStore.setTreeNode(modifiedTreeNode) + for modifiedNode := range modifiedNodes { + rt.store.setTreeNode(modifiedNode) } // Add the block to the futureCoveringSets of all the blocks // in the selected parent's anticone for _, current := range selectedParentAnticone { - currentFutureCoveringSet, err := dag.reachabilityStore.futureCoveringSetByBlockNode(current) + currentFutureCoveringSet, err := rt.store.futureCoveringSetByBlockNode(current) if err != nil { return err } - currentFutureCoveringSet.insertBlock(&futureCoveringBlock{blockNode: node, treeNode: newTreeNode}) - err = dag.reachabilityStore.setFutureCoveringSet(current, currentFutureCoveringSet) + currentFutureCoveringSet.insertNode(newTreeNode) + err = rt.store.setFutureCoveringSet(current, currentFutureCoveringSet) if err != nil { return err } } + + // Update the reindex root. + // Note that we check for blue score here in order to find out + // whether the new node is going to be the virtual's selected + // parent. We don't check node == virtual.selectedParent because + // at this stage the virtual had not yet been updated. + if node.blueScore > rt.dag.SelectedTipBlueScore() { + updateStartTime := time.Now() + modifiedNodes, err := rt.updateReindexRoot(newTreeNode) + if err != nil { + return err + } + if len(modifiedNodes) > 0 { + updateTimeElapsed := time.Since(updateStartTime) + log.Debugf("Reachability reindex root updated to %s. "+ + "Modified %d tree nodes and took %dms.", + rt.reindexRoot.blockNode.hash, + len(modifiedNodes), updateTimeElapsed.Milliseconds()) + for modifiedNode := range modifiedNodes { + rt.store.setTreeNode(modifiedNode) + } + } + } + return nil } -// isAncestorOf returns true if this node is in the past of the other node -// in the DAG. The complexity of this method is O(log(|this.futureCoveringBlockSet|)) -func (dag *BlockDAG) isAncestorOf(this *blockNode, other *blockNode) (bool, error) { - // First, check if this node is a reachability tree ancestor of the +type reachabilityTree struct { + dag *BlockDAG + store *reachabilityStore + reindexRoot *reachabilityTreeNode +} + +func newReachabilityTree(dag *BlockDAG) *reachabilityTree { + store := newReachabilityStore(dag) + return &reachabilityTree{ + dag: dag, + store: store, + reindexRoot: nil, + } +} + +func (rt *reachabilityTree) init(dbContext dbaccess.Context) error { + // Init the store + err := rt.store.init(dbContext) + if err != nil { + return err + } + + // Fetch the reindex root hash. If missing, use the genesis hash + reindexRootHash, err := dbaccess.FetchReachabilityReindexRoot(dbContext) + if err != nil { + if !dbaccess.IsNotFoundError(err) { + return err + } + reindexRootHash = rt.dag.dagParams.GenesisHash + } + + // Init the reindex root + reachabilityReindexRootNode, ok := rt.dag.index.LookupNode(reindexRootHash) + if !ok { + return errors.Errorf("reachability reindex root block %s "+ + "does not exist in the DAG", reindexRootHash) + } + rt.reindexRoot, err = rt.store.treeNodeByBlockNode(reachabilityReindexRootNode) + if err != nil { + return errors.Wrapf(err, "cannot set reachability reindex root") + } + + return nil +} + +func (rt *reachabilityTree) storeState(dbTx *dbaccess.TxContext) error { + // Flush the store + err := rt.dag.reachabilityTree.store.flushToDB(dbTx) + if err != nil { + return err + } + + // Store the reindex root + err = dbaccess.StoreReachabilityReindexRoot(dbTx, rt.reindexRoot.blockNode.hash) + if err != nil { + return err + } + + return nil +} + +func (rt *reachabilityTree) updateReindexRoot(newTreeNode *reachabilityTreeNode) (modifiedTreeNodes, error) { + allModifiedTreeNodes := newModifiedTreeNodes() + + nextReindexRoot := rt.reindexRoot + for { + candidateReindexRoot, modifiedNodes, found, err := rt.maybeMoveReindexRoot(nextReindexRoot, newTreeNode) + if err != nil { + return nil, err + } + if !found { + break + } + allModifiedTreeNodes.addAll(modifiedNodes) + nextReindexRoot = candidateReindexRoot + } + + rt.reindexRoot = nextReindexRoot + return allModifiedTreeNodes, nil +} + +func (rt *reachabilityTree) maybeMoveReindexRoot( + reindexRoot *reachabilityTreeNode, newTreeNode *reachabilityTreeNode) ( + newReindexRoot *reachabilityTreeNode, modifiedNodes modifiedTreeNodes, found bool, err error) { + + if !reindexRoot.isAncestorOf(newTreeNode) { + commonAncestor := newTreeNode.findCommonAncestorWithReindexRoot(reindexRoot) + return commonAncestor, nil, true, nil + } + + reindexRootChosenChild, err := reindexRoot.findAncestorAmongChildren(newTreeNode) + if err != nil { + return nil, nil, false, err + } + if newTreeNode.blockNode.blueScore-reindexRootChosenChild.blockNode.blueScore < reachabilityReindexWindow { + return nil, nil, false, nil + } + modifiedNodes, err = rt.concentrateIntervalAroundReindexRootChosenChild(reindexRoot, reindexRootChosenChild) + if err != nil { + return nil, nil, false, err + } + + return reindexRootChosenChild, modifiedNodes, true, nil +} + +// findAncestorAmongChildren finds the reachability tree child +// of rtn that is the ancestor of node. +func (rtn *reachabilityTreeNode) findAncestorAmongChildren(node *reachabilityTreeNode) (*reachabilityTreeNode, error) { + ancestor, ok := orderedTreeNodeSet(rtn.children).findAncestorOfNode(node) + if !ok { + return nil, errors.Errorf("rtn is not an ancestor of node") + } + + return ancestor, nil +} + +func (rt *reachabilityTree) concentrateIntervalAroundReindexRootChosenChild( + reindexRoot *reachabilityTreeNode, reindexRootChosenChild *reachabilityTreeNode) ( + modifiedTreeNodes, error) { + + allModifiedTreeNodes := newModifiedTreeNodes() + + reindexRootChildNodesBeforeChosen, reindexRootChildNodesAfterChosen, err := + reindexRoot.splitChildrenAroundChild(reindexRootChosenChild) + if err != nil { + return nil, err + } + + reindexRootChildNodesBeforeChosenSizesSum, modifiedNodesBeforeChosen, err := + rt.tightenIntervalsBeforeReindexRootChosenChild(reindexRoot, reindexRootChildNodesBeforeChosen) + if err != nil { + return nil, err + } + allModifiedTreeNodes.addAll(modifiedNodesBeforeChosen) + + reindexRootChildNodesAfterChosenSizesSum, modifiedNodesAfterChosen, err := + rt.tightenIntervalsAfterReindexRootChosenChild(reindexRoot, reindexRootChildNodesAfterChosen) + if err != nil { + return nil, err + } + allModifiedTreeNodes.addAll(modifiedNodesAfterChosen) + + modifiedNodesForReindexRootExpansion, err := rt.expandIntervalInReindexRootChosenChild( + reindexRoot, reindexRootChosenChild, reindexRootChildNodesBeforeChosenSizesSum, reindexRootChildNodesAfterChosenSizesSum) + if err != nil { + return nil, err + } + allModifiedTreeNodes.addAll(modifiedNodesForReindexRootExpansion) + + return allModifiedTreeNodes, nil +} + +// splitChildrenAroundChild splits `rtn` into two slices: the nodes that are before +// `child` and the nodes that are after. +func (rtn *reachabilityTreeNode) splitChildrenAroundChild(child *reachabilityTreeNode) ( + nodesBeforeChild []*reachabilityTreeNode, nodesAfterChild []*reachabilityTreeNode, err error) { + + for i, candidateChild := range rtn.children { + if candidateChild == child { + return rtn.children[:i], rtn.children[i+1:], nil + } + } + return nil, nil, errors.Errorf("child not a child of rtn") +} + +func (rt *reachabilityTree) tightenIntervalsBeforeReindexRootChosenChild( + reindexRoot *reachabilityTreeNode, reindexRootChildNodesBeforeChosen []*reachabilityTreeNode) ( + reindexRootChildNodesBeforeChosenSizesSum uint64, modifiedNodes modifiedTreeNodes, err error) { + + reindexRootChildNodesBeforeChosenSizes, reindexRootChildNodesBeforeChosenSubtreeSizeMaps, reindexRootChildNodesBeforeChosenSizesSum := + calcReachabilityTreeNodeSizes(reindexRootChildNodesBeforeChosen) + + intervalBeforeReindexRootStart := newReachabilityInterval( + reindexRoot.interval.start+reachabilityReindexSlack, + reindexRoot.interval.start+reachabilityReindexSlack+reindexRootChildNodesBeforeChosenSizesSum-1, + ) + + modifiedNodes, err = rt.propagateChildIntervals(intervalBeforeReindexRootStart, reindexRootChildNodesBeforeChosen, + reindexRootChildNodesBeforeChosenSizes, reindexRootChildNodesBeforeChosenSubtreeSizeMaps) + if err != nil { + return 0, nil, err + } + return reindexRootChildNodesBeforeChosenSizesSum, modifiedNodes, nil +} + +func (rt *reachabilityTree) tightenIntervalsAfterReindexRootChosenChild( + reindexRoot *reachabilityTreeNode, reindexRootChildNodesAfterChosen []*reachabilityTreeNode) ( + reindexRootChildNodesAfterChosenSizesSum uint64, modifiedNodes modifiedTreeNodes, err error) { + + reindexRootChildNodesAfterChosenSizes, reindexRootChildNodesAfterChosenSubtreeSizeMaps, reindexRootChildNodesAfterChosenSizesSum := + calcReachabilityTreeNodeSizes(reindexRootChildNodesAfterChosen) + + intervalAfterReindexRootEnd := newReachabilityInterval( + reindexRoot.interval.end-reachabilityReindexSlack-reindexRootChildNodesAfterChosenSizesSum, + reindexRoot.interval.end-reachabilityReindexSlack-1, + ) + + modifiedNodes, err = rt.propagateChildIntervals(intervalAfterReindexRootEnd, reindexRootChildNodesAfterChosen, + reindexRootChildNodesAfterChosenSizes, reindexRootChildNodesAfterChosenSubtreeSizeMaps) + if err != nil { + return 0, nil, err + } + return reindexRootChildNodesAfterChosenSizesSum, modifiedNodes, nil +} + +func (rt *reachabilityTree) expandIntervalInReindexRootChosenChild(reindexRoot *reachabilityTreeNode, + reindexRootChosenChild *reachabilityTreeNode, reindexRootChildNodesBeforeChosenSizesSum uint64, + reindexRootChildNodesAfterChosenSizesSum uint64) (modifiedTreeNodes, error) { + + allModifiedTreeNodes := newModifiedTreeNodes() + + newReindexRootChildInterval := newReachabilityInterval( + reindexRoot.interval.start+reindexRootChildNodesBeforeChosenSizesSum+reachabilityReindexSlack, + reindexRoot.interval.end-reindexRootChildNodesAfterChosenSizesSum-reachabilityReindexSlack-1, + ) + + if !newReindexRootChildInterval.contains(reindexRootChosenChild.interval) { + // New interval doesn't contain the previous one, propagation is required + + // We assign slack on both sides as an optimization. Were we to + // assign a tight interval, the next time the reindex root moves we + // would need to propagate intervals again. That is to say, When we + // DO allocate slack, next time + // expandIntervalInReindexRootChosenChild is called (next time the + // reindex root moves), newReindexRootChildInterval is likely to + // contain reindexRootChosenChild.interval. + reindexRootChosenChild.interval = newReachabilityInterval( + newReindexRootChildInterval.start+reachabilityReindexSlack, + newReindexRootChildInterval.end-reachabilityReindexSlack, + ) + modifiedNodes, err := reindexRootChosenChild.countSubtreesAndPropagateInterval() + if err != nil { + return nil, err + } + allModifiedTreeNodes.addAll(modifiedNodes) + } + + reindexRootChosenChild.interval = newReindexRootChildInterval + allModifiedTreeNodes[reindexRootChosenChild] = struct{}{} + return allModifiedTreeNodes, nil +} + +func (rtn *reachabilityTreeNode) countSubtreesAndPropagateInterval() (modifiedTreeNodes, error) { + subtreeSizeMap := make(map[*reachabilityTreeNode]uint64) + rtn.countSubtrees(subtreeSizeMap) + return rtn.propagateInterval(subtreeSizeMap) +} + +func calcReachabilityTreeNodeSizes(treeNodes []*reachabilityTreeNode) ( + sizes []uint64, subtreeSizeMaps []map[*reachabilityTreeNode]uint64, sum uint64) { + + sizes = make([]uint64, len(treeNodes)) + subtreeSizeMaps = make([]map[*reachabilityTreeNode]uint64, len(treeNodes)) + sum = 0 + for i, node := range treeNodes { + subtreeSizeMap := make(map[*reachabilityTreeNode]uint64) + node.countSubtrees(subtreeSizeMap) + subtreeSize := subtreeSizeMap[node] + sizes[i] = subtreeSize + subtreeSizeMaps[i] = subtreeSizeMap + sum += subtreeSize + } + return sizes, subtreeSizeMaps, sum +} + +func (rt *reachabilityTree) propagateChildIntervals(interval *reachabilityInterval, + childNodes []*reachabilityTreeNode, sizes []uint64, subtreeSizeMaps []map[*reachabilityTreeNode]uint64) ( + modifiedTreeNodes, error) { + + allModifiedTreeNodes := newModifiedTreeNodes() + + childIntervalSizes, err := interval.splitExact(sizes) + if err != nil { + return nil, err + } + + for i, child := range childNodes { + childInterval := childIntervalSizes[i] + child.interval = childInterval + + childSubtreeSizeMap := subtreeSizeMaps[i] + modifiedNodes, err := child.propagateInterval(childSubtreeSizeMap) + if err != nil { + return nil, err + } + allModifiedTreeNodes.addAll(modifiedNodes) + } + + return allModifiedTreeNodes, nil +} + +// isInPast returns true if `this` is in the past (exclusive) of `other` +// in the DAG. +// The complexity of this method is O(log(|this.futureCoveringTreeNodeSet|)) +func (rt *reachabilityTree) isInPast(this *blockNode, other *blockNode) (bool, error) { + // By definition, a node is not in the past of itself. + if this == other { + return false, nil + } + + // Check if this node is a reachability tree ancestor of the // other node - thisTreeNode, err := dag.reachabilityStore.treeNodeByBlockNode(this) + isReachabilityTreeAncestor, err := rt.isReachabilityTreeAncestorOf(this, other) if err != nil { return false, err } - otherTreeNode, err := dag.reachabilityStore.treeNodeByBlockNode(other) - if err != nil { - return false, err - } - if thisTreeNode.isAncestorOf(otherTreeNode) { + if isReachabilityTreeAncestor { return true, nil } // Otherwise, use previously registered future blocks to complete the // reachability test - thisFutureCoveringSet, err := dag.reachabilityStore.futureCoveringSetByBlockNode(this) + thisFutureCoveringSet, err := rt.store.futureCoveringSetByBlockNode(this) if err != nil { return false, err } - return thisFutureCoveringSet.isInFuture(&futureCoveringBlock{blockNode: other, treeNode: otherTreeNode}), nil + otherTreeNode, err := rt.store.treeNodeByBlockNode(other) + if err != nil { + return false, err + } + return thisFutureCoveringSet.hasAncestorOf(otherTreeNode), nil +} + +// isReachabilityTreeAncestorOf returns whether `this` is in the selected parent chain of `other`. +func (rt *reachabilityTree) isReachabilityTreeAncestorOf(this *blockNode, other *blockNode) (bool, error) { + thisTreeNode, err := rt.store.treeNodeByBlockNode(this) + if err != nil { + return false, err + } + otherTreeNode, err := rt.store.treeNodeByBlockNode(other) + if err != nil { + return false, err + } + return thisTreeNode.isAncestorOf(otherTreeNode), nil } diff --git a/blockdag/reachability_test.go b/blockdag/reachability_test.go index e7ab60219..47ad98bcc 100644 --- a/blockdag/reachability_test.go +++ b/blockdag/reachability_test.go @@ -1,6 +1,8 @@ package blockdag import ( + "github.com/kaspanet/kaspad/dagconfig" + "github.com/kaspanet/kaspad/util/daghash" "reflect" "strings" "testing" @@ -11,19 +13,19 @@ func TestAddChild(t *testing.T) { // root -> a -> b -> c... // Create the root node of a new reachability tree root := newReachabilityTreeNode(&blockNode{}) - root.setInterval(newReachabilityInterval(1, 100)) + root.interval = newReachabilityInterval(1, 100) // Add a chain of child nodes just before a reindex occurs (2^6=64 < 100) currentTip := root for i := 0; i < 6; i++ { node := newReachabilityTreeNode(&blockNode{}) - modifiedNodes, err := currentTip.addChild(node) + modifiedNodes, err := currentTip.addChild(node, root) if err != nil { t.Fatalf("TestAddChild: addChild failed: %s", err) } // Expect only the node and its parent to be affected - expectedModifiedNodes := []*reachabilityTreeNode{currentTip, node} + expectedModifiedNodes := newModifiedTreeNodes(currentTip, node) if !reflect.DeepEqual(modifiedNodes, expectedModifiedNodes) { t.Fatalf("TestAddChild: unexpected modifiedNodes. "+ "want: %s, got: %s", expectedModifiedNodes, modifiedNodes) @@ -34,7 +36,7 @@ func TestAddChild(t *testing.T) { // Add another node to the tip of the chain to trigger a reindex (100 < 2^7=128) lastChild := newReachabilityTreeNode(&blockNode{}) - modifiedNodes, err := currentTip.addChild(lastChild) + modifiedNodes, err := currentTip.addChild(lastChild, root) if err != nil { t.Fatalf("TestAddChild: addChild failed: %s", err) } @@ -45,14 +47,18 @@ func TestAddChild(t *testing.T) { t.Fatalf("TestAddChild: unexpected amount of modifiedNodes.") } - // Expect the tip to have an interval of 1 and remaining interval of 0 + // Expect the tip to have an interval of 1 and remaining interval of 0 both before and after tipInterval := lastChild.interval.size() if tipInterval != 1 { t.Fatalf("TestAddChild: unexpected tip interval size: want: 1, got: %d", tipInterval) } - tipRemainingInterval := lastChild.remainingInterval.size() - if tipRemainingInterval != 0 { - t.Fatalf("TestAddChild: unexpected tip interval size: want: 0, got: %d", tipRemainingInterval) + tipRemainingIntervalBefore := lastChild.remainingIntervalBefore().size() + if tipRemainingIntervalBefore != 0 { + t.Fatalf("TestAddChild: unexpected tip interval before size: want: 0, got: %d", tipRemainingIntervalBefore) + } + tipRemainingIntervalAfter := lastChild.remainingIntervalAfter().size() + if tipRemainingIntervalAfter != 0 { + t.Fatalf("TestAddChild: unexpected tip interval after size: want: 0, got: %d", tipRemainingIntervalAfter) } // Expect all nodes to be descendant nodes of root @@ -68,19 +74,19 @@ func TestAddChild(t *testing.T) { // root -> a, b, c... // Create the root node of a new reachability tree root = newReachabilityTreeNode(&blockNode{}) - root.setInterval(newReachabilityInterval(1, 100)) + root.interval = newReachabilityInterval(1, 100) // Add child nodes to root just before a reindex occurs (2^6=64 < 100) childNodes := make([]*reachabilityTreeNode, 6) for i := 0; i < len(childNodes); i++ { childNodes[i] = newReachabilityTreeNode(&blockNode{}) - modifiedNodes, err := root.addChild(childNodes[i]) + modifiedNodes, err := root.addChild(childNodes[i], root) if err != nil { t.Fatalf("TestAddChild: addChild failed: %s", err) } // Expect only the node and the root to be affected - expectedModifiedNodes := []*reachabilityTreeNode{root, childNodes[i]} + expectedModifiedNodes := newModifiedTreeNodes(root, childNodes[i]) if !reflect.DeepEqual(modifiedNodes, expectedModifiedNodes) { t.Fatalf("TestAddChild: unexpected modifiedNodes. "+ "want: %s, got: %s", expectedModifiedNodes, modifiedNodes) @@ -89,7 +95,7 @@ func TestAddChild(t *testing.T) { // Add another node to the root to trigger a reindex (100 < 2^7=128) lastChild = newReachabilityTreeNode(&blockNode{}) - modifiedNodes, err = root.addChild(lastChild) + modifiedNodes, err = root.addChild(lastChild, root) if err != nil { t.Fatalf("TestAddChild: addChild failed: %s", err) } @@ -100,14 +106,18 @@ func TestAddChild(t *testing.T) { t.Fatalf("TestAddChild: unexpected amount of modifiedNodes.") } - // Expect the last-added child to have an interval of 1 and remaining interval of 0 + // Expect the last-added child to have an interval of 1 and remaining interval of 0 both before and after lastChildInterval := lastChild.interval.size() if lastChildInterval != 1 { t.Fatalf("TestAddChild: unexpected lastChild interval size: want: 1, got: %d", lastChildInterval) } - lastChildRemainingInterval := lastChild.remainingInterval.size() - if lastChildRemainingInterval != 0 { - t.Fatalf("TestAddChild: unexpected lastChild interval size: want: 0, got: %d", lastChildRemainingInterval) + lastChildRemainingIntervalBefore := lastChild.remainingIntervalBefore().size() + if lastChildRemainingIntervalBefore != 0 { + t.Fatalf("TestAddChild: unexpected lastChild interval before size: want: 0, got: %d", lastChildRemainingIntervalBefore) + } + lastChildRemainingIntervalAfter := lastChild.remainingIntervalAfter().size() + if lastChildRemainingIntervalAfter != 0 { + t.Fatalf("TestAddChild: unexpected lastChild interval after size: want: 0, got: %d", lastChildRemainingIntervalAfter) } // Expect all nodes to be descendant nodes of root @@ -125,7 +135,7 @@ func TestReachabilityTreeNodeIsAncestorOf(t *testing.T) { descendants := make([]*reachabilityTreeNode, numberOfDescendants) for i := 0; i < numberOfDescendants; i++ { node := newReachabilityTreeNode(&blockNode{}) - _, err := currentTip.addChild(node) + _, err := currentTip.addChild(node, root) if err != nil { t.Fatalf("TestReachabilityTreeNodeIsAncestorOf: addChild failed: %s", err) } @@ -140,65 +150,65 @@ func TestReachabilityTreeNodeIsAncestorOf(t *testing.T) { } } - if root.isAncestorOf(root) { - t.Fatalf("TestReachabilityTreeNodeIsAncestorOf: root is not expected to be a descendant of root") + if !root.isAncestorOf(root) { + t.Fatalf("TestReachabilityTreeNodeIsAncestorOf: root is expected to be an ancestor of root") } } -func TestIntervalIsAncestorOf(t *testing.T) { +func TestIntervalContains(t *testing.T) { tests := []struct { - name string - this, other *reachabilityInterval - isThisAncestorOfOther bool + name string + this, other *reachabilityInterval + thisContainsOther bool }{ { - name: "this == other", - this: newReachabilityInterval(10, 100), - other: newReachabilityInterval(10, 100), - isThisAncestorOfOther: false, + name: "this == other", + this: newReachabilityInterval(10, 100), + other: newReachabilityInterval(10, 100), + thisContainsOther: true, }, { - name: "this.start == other.start && this.end < other.end", - this: newReachabilityInterval(10, 90), - other: newReachabilityInterval(10, 100), - isThisAncestorOfOther: false, + name: "this.start == other.start && this.end < other.end", + this: newReachabilityInterval(10, 90), + other: newReachabilityInterval(10, 100), + thisContainsOther: false, }, { - name: "this.start == other.start && this.end > other.end", - this: newReachabilityInterval(10, 100), - other: newReachabilityInterval(10, 90), - isThisAncestorOfOther: true, + name: "this.start == other.start && this.end > other.end", + this: newReachabilityInterval(10, 100), + other: newReachabilityInterval(10, 90), + thisContainsOther: true, }, { - name: "this.start > other.start && this.end == other.end", - this: newReachabilityInterval(20, 100), - other: newReachabilityInterval(10, 100), - isThisAncestorOfOther: false, + name: "this.start > other.start && this.end == other.end", + this: newReachabilityInterval(20, 100), + other: newReachabilityInterval(10, 100), + thisContainsOther: false, }, { - name: "this.start < other.start && this.end == other.end", - this: newReachabilityInterval(10, 100), - other: newReachabilityInterval(20, 100), - isThisAncestorOfOther: true, + name: "this.start < other.start && this.end == other.end", + this: newReachabilityInterval(10, 100), + other: newReachabilityInterval(20, 100), + thisContainsOther: true, }, { - name: "this.start > other.start && this.end < other.end", - this: newReachabilityInterval(20, 90), - other: newReachabilityInterval(10, 100), - isThisAncestorOfOther: false, + name: "this.start > other.start && this.end < other.end", + this: newReachabilityInterval(20, 90), + other: newReachabilityInterval(10, 100), + thisContainsOther: false, }, { - name: "this.start < other.start && this.end > other.end", - this: newReachabilityInterval(10, 100), - other: newReachabilityInterval(20, 90), - isThisAncestorOfOther: true, + name: "this.start < other.start && this.end > other.end", + this: newReachabilityInterval(10, 100), + other: newReachabilityInterval(20, 90), + thisContainsOther: true, }, } for _, test := range tests { - if isAncestorOf := test.this.isAncestorOf(test.other); isAncestorOf != test.isThisAncestorOfOther { - t.Errorf("test.this.isAncestorOf(test.other) is expected to be %t but got %t", - test.isThisAncestorOfOther, isAncestorOf) + if thisContainsOther := test.this.contains(test.other); thisContainsOther != test.thisContainsOther { + t.Errorf("test.this.contains(test.other) is expected to be %t but got %t", + test.thisContainsOther, thisContainsOther) } } } @@ -431,140 +441,140 @@ func TestSplitWithExponentialBias(t *testing.T) { } } -func TestIsInFuture(t *testing.T) { - blocks := futureCoveringBlockSet{ - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(2, 3)}}, - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(4, 67)}}, - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(67, 77)}}, - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(657, 789)}}, - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(1000, 1000)}}, - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(1920, 1921)}}, +func TestHasAncestorOf(t *testing.T) { + treeNodes := futureCoveringTreeNodeSet{ + &reachabilityTreeNode{interval: newReachabilityInterval(2, 3)}, + &reachabilityTreeNode{interval: newReachabilityInterval(4, 67)}, + &reachabilityTreeNode{interval: newReachabilityInterval(67, 77)}, + &reachabilityTreeNode{interval: newReachabilityInterval(657, 789)}, + &reachabilityTreeNode{interval: newReachabilityInterval(1000, 1000)}, + &reachabilityTreeNode{interval: newReachabilityInterval(1920, 1921)}, } tests := []struct { - block *futureCoveringBlock + treeNode *reachabilityTreeNode expectedResult bool }{ { - block: &futureCoveringBlock{treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(1, 1)}}, + treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(1, 1)}, expectedResult: false, }, { - block: &futureCoveringBlock{treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(5, 7)}}, + treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(5, 7)}, expectedResult: true, }, { - block: &futureCoveringBlock{treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(67, 76)}}, + treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(67, 76)}, expectedResult: true, }, { - block: &futureCoveringBlock{treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(78, 100)}}, + treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(78, 100)}, expectedResult: false, }, { - block: &futureCoveringBlock{treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(1980, 2000)}}, + treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(1980, 2000)}, expectedResult: false, }, { - block: &futureCoveringBlock{treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(1920, 1920)}}, + treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(1920, 1920)}, expectedResult: true, }, } for i, test := range tests { - result := blocks.isInFuture(test.block) + result := treeNodes.hasAncestorOf(test.treeNode) if result != test.expectedResult { - t.Errorf("TestIsInFuture: unexpected result in test #%d. Want: %t, got: %t", + t.Errorf("TestHasAncestorOf: unexpected result in test #%d. Want: %t, got: %t", i, test.expectedResult, result) } } } -func TestInsertBlock(t *testing.T) { - blocks := futureCoveringBlockSet{ - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(1, 3)}}, - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(4, 67)}}, - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(67, 77)}}, - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(657, 789)}}, - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(1000, 1000)}}, - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(1920, 1921)}}, +func TestInsertNode(t *testing.T) { + treeNodes := futureCoveringTreeNodeSet{ + &reachabilityTreeNode{interval: newReachabilityInterval(1, 3)}, + &reachabilityTreeNode{interval: newReachabilityInterval(4, 67)}, + &reachabilityTreeNode{interval: newReachabilityInterval(67, 77)}, + &reachabilityTreeNode{interval: newReachabilityInterval(657, 789)}, + &reachabilityTreeNode{interval: newReachabilityInterval(1000, 1000)}, + &reachabilityTreeNode{interval: newReachabilityInterval(1920, 1921)}, } tests := []struct { - toInsert []*futureCoveringBlock - expectedResult futureCoveringBlockSet + toInsert []*reachabilityTreeNode + expectedResult futureCoveringTreeNodeSet }{ { - toInsert: []*futureCoveringBlock{ - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(5, 7)}}, + toInsert: []*reachabilityTreeNode{ + {interval: newReachabilityInterval(5, 7)}, }, - expectedResult: futureCoveringBlockSet{ - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(1, 3)}}, - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(4, 67)}}, - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(67, 77)}}, - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(657, 789)}}, - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(1000, 1000)}}, - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(1920, 1921)}}, + expectedResult: futureCoveringTreeNodeSet{ + &reachabilityTreeNode{interval: newReachabilityInterval(1, 3)}, + &reachabilityTreeNode{interval: newReachabilityInterval(4, 67)}, + &reachabilityTreeNode{interval: newReachabilityInterval(67, 77)}, + &reachabilityTreeNode{interval: newReachabilityInterval(657, 789)}, + &reachabilityTreeNode{interval: newReachabilityInterval(1000, 1000)}, + &reachabilityTreeNode{interval: newReachabilityInterval(1920, 1921)}, }, }, { - toInsert: []*futureCoveringBlock{ - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(65, 78)}}, + toInsert: []*reachabilityTreeNode{ + {interval: newReachabilityInterval(65, 78)}, }, - expectedResult: futureCoveringBlockSet{ - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(1, 3)}}, - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(4, 67)}}, - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(65, 78)}}, - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(657, 789)}}, - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(1000, 1000)}}, - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(1920, 1921)}}, + expectedResult: futureCoveringTreeNodeSet{ + &reachabilityTreeNode{interval: newReachabilityInterval(1, 3)}, + &reachabilityTreeNode{interval: newReachabilityInterval(4, 67)}, + &reachabilityTreeNode{interval: newReachabilityInterval(65, 78)}, + &reachabilityTreeNode{interval: newReachabilityInterval(657, 789)}, + &reachabilityTreeNode{interval: newReachabilityInterval(1000, 1000)}, + &reachabilityTreeNode{interval: newReachabilityInterval(1920, 1921)}, }, }, { - toInsert: []*futureCoveringBlock{ - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(88, 97)}}, + toInsert: []*reachabilityTreeNode{ + {interval: newReachabilityInterval(88, 97)}, }, - expectedResult: futureCoveringBlockSet{ - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(1, 3)}}, - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(4, 67)}}, - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(67, 77)}}, - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(88, 97)}}, - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(657, 789)}}, - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(1000, 1000)}}, - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(1920, 1921)}}, + expectedResult: futureCoveringTreeNodeSet{ + &reachabilityTreeNode{interval: newReachabilityInterval(1, 3)}, + &reachabilityTreeNode{interval: newReachabilityInterval(4, 67)}, + &reachabilityTreeNode{interval: newReachabilityInterval(67, 77)}, + &reachabilityTreeNode{interval: newReachabilityInterval(88, 97)}, + &reachabilityTreeNode{interval: newReachabilityInterval(657, 789)}, + &reachabilityTreeNode{interval: newReachabilityInterval(1000, 1000)}, + &reachabilityTreeNode{interval: newReachabilityInterval(1920, 1921)}, }, }, { - toInsert: []*futureCoveringBlock{ - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(88, 97)}}, - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(3000, 3010)}}, + toInsert: []*reachabilityTreeNode{ + {interval: newReachabilityInterval(88, 97)}, + {interval: newReachabilityInterval(3000, 3010)}, }, - expectedResult: futureCoveringBlockSet{ - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(1, 3)}}, - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(4, 67)}}, - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(67, 77)}}, - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(88, 97)}}, - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(657, 789)}}, - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(1000, 1000)}}, - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(1920, 1921)}}, - {treeNode: &reachabilityTreeNode{interval: newReachabilityInterval(3000, 3010)}}, + expectedResult: futureCoveringTreeNodeSet{ + &reachabilityTreeNode{interval: newReachabilityInterval(1, 3)}, + &reachabilityTreeNode{interval: newReachabilityInterval(4, 67)}, + &reachabilityTreeNode{interval: newReachabilityInterval(67, 77)}, + &reachabilityTreeNode{interval: newReachabilityInterval(88, 97)}, + &reachabilityTreeNode{interval: newReachabilityInterval(657, 789)}, + &reachabilityTreeNode{interval: newReachabilityInterval(1000, 1000)}, + &reachabilityTreeNode{interval: newReachabilityInterval(1920, 1921)}, + &reachabilityTreeNode{interval: newReachabilityInterval(3000, 3010)}, }, }, } for i, test := range tests { - // Create a clone of blocks so that we have a clean start for every test - blocksClone := make(futureCoveringBlockSet, len(blocks)) - for i, block := range blocks { - blocksClone[i] = block + // Create a clone of treeNodes so that we have a clean start for every test + treeNodesClone := make(futureCoveringTreeNodeSet, len(treeNodes)) + for i, treeNode := range treeNodes { + treeNodesClone[i] = treeNode } - for _, block := range test.toInsert { - blocksClone.insertBlock(block) + for _, treeNode := range test.toInsert { + treeNodesClone.insertNode(treeNode) } - if !reflect.DeepEqual(blocksClone, test.expectedResult) { - t.Errorf("TestInsertBlock: unexpected result in test #%d. Want: %s, got: %s", - i, test.expectedResult, blocksClone) + if !reflect.DeepEqual(treeNodesClone, test.expectedResult) { + t.Errorf("TestInsertNode: unexpected result in test #%d. Want: %s, got: %s", + i, test.expectedResult, treeNodesClone) } } } @@ -665,14 +675,14 @@ func TestSplitWithExponentialBiasErrors(t *testing.T) { func TestReindexIntervalErrors(t *testing.T) { // Create a treeNode and give it size = 100 treeNode := newReachabilityTreeNode(&blockNode{}) - treeNode.setInterval(newReachabilityInterval(0, 99)) + treeNode.interval = newReachabilityInterval(0, 99) // Add a chain of 100 child treeNodes to treeNode var err error currentTreeNode := treeNode for i := 0; i < 100; i++ { childTreeNode := newReachabilityTreeNode(&blockNode{}) - _, err = currentTreeNode.addChild(childTreeNode) + _, err = currentTreeNode.addChild(childTreeNode, treeNode) if err != nil { break } @@ -704,12 +714,12 @@ func BenchmarkReindexInterval(b *testing.B) { // its first child gets half of the interval, so a reindex // from the root should happen after adding subTreeSize // nodes. - root.setInterval(newReachabilityInterval(0, subTreeSize*2)) + root.interval = newReachabilityInterval(0, subTreeSize*2) currentTreeNode := root for i := 0; i < subTreeSize; i++ { childTreeNode := newReachabilityTreeNode(&blockNode{}) - _, err := currentTreeNode.addChild(childTreeNode) + _, err := currentTreeNode.addChild(childTreeNode, root) if err != nil { b.Fatalf("addChild: %s", err) } @@ -717,50 +727,47 @@ func BenchmarkReindexInterval(b *testing.B) { currentTreeNode = childTreeNode } - remainingIntervalBefore := *root.remainingInterval + originalRemainingInterval := *root.remainingIntervalAfter() // After we added subTreeSize nodes, adding the next // node should lead to a reindex from root. fullReindexTriggeringNode := newReachabilityTreeNode(&blockNode{}) b.StartTimer() - _, err := currentTreeNode.addChild(fullReindexTriggeringNode) + _, err := currentTreeNode.addChild(fullReindexTriggeringNode, root) b.StopTimer() if err != nil { b.Fatalf("addChild: %s", err) } - if *root.remainingInterval == remainingIntervalBefore { + if *root.remainingIntervalAfter() == originalRemainingInterval { b.Fatal("Expected a reindex from root, but it didn't happen") } } } -func TestFutureCoveringBlockSetString(t *testing.T) { +func TestFutureCoveringTreeNodeSetString(t *testing.T) { treeNodeA := newReachabilityTreeNode(&blockNode{}) - treeNodeA.setInterval(newReachabilityInterval(123, 456)) + treeNodeA.interval = newReachabilityInterval(123, 456) treeNodeB := newReachabilityTreeNode(&blockNode{}) - treeNodeB.setInterval(newReachabilityInterval(457, 789)) - futureCoveringSet := futureCoveringBlockSet{ - &futureCoveringBlock{treeNode: treeNodeA}, - &futureCoveringBlock{treeNode: treeNodeB}, - } + treeNodeB.interval = newReachabilityInterval(457, 789) + futureCoveringSet := futureCoveringTreeNodeSet{treeNodeA, treeNodeB} str := futureCoveringSet.String() expectedStr := "[123,456][457,789]" if str != expectedStr { - t.Fatalf("TestFutureCoveringBlockSetString: unexpected "+ + t.Fatalf("TestFutureCoveringTreeNodeSetString: unexpected "+ "string. Want: %s, got: %s", expectedStr, str) } } func TestReachabilityTreeNodeString(t *testing.T) { treeNodeA := newReachabilityTreeNode(&blockNode{}) - treeNodeA.setInterval(newReachabilityInterval(100, 199)) + treeNodeA.interval = newReachabilityInterval(100, 199) treeNodeB1 := newReachabilityTreeNode(&blockNode{}) - treeNodeB1.setInterval(newReachabilityInterval(100, 150)) + treeNodeB1.interval = newReachabilityInterval(100, 150) treeNodeB2 := newReachabilityTreeNode(&blockNode{}) - treeNodeB2.setInterval(newReachabilityInterval(150, 199)) + treeNodeB2.interval = newReachabilityInterval(150, 199) treeNodeC := newReachabilityTreeNode(&blockNode{}) - treeNodeC.setInterval(newReachabilityInterval(100, 149)) + treeNodeC.interval = newReachabilityInterval(100, 149) treeNodeA.children = []*reachabilityTreeNode{treeNodeB1, treeNodeB2} treeNodeB2.children = []*reachabilityTreeNode{treeNodeC} @@ -771,3 +778,237 @@ func TestReachabilityTreeNodeString(t *testing.T) { "string. Want: %s, got: %s", expectedStr, str) } } + +func TestIsInPast(t *testing.T) { + // Create a new database and DAG instance to run tests against. + dag, teardownFunc, err := DAGSetup("TestIsInPast", true, Config{ + DAGParams: &dagconfig.SimnetParams, + }) + if err != nil { + t.Fatalf("TestIsInPast: Failed to setup DAG instance: %v", err) + } + defer teardownFunc() + + // Add a chain of two blocks above the genesis. This will be the + // selected parent chain. + blockA := PrepareAndProcessBlockForTest(t, dag, []*daghash.Hash{dag.genesis.hash}, nil) + blockB := PrepareAndProcessBlockForTest(t, dag, []*daghash.Hash{blockA.BlockHash()}, nil) + + // Add another block above the genesis + blockC := PrepareAndProcessBlockForTest(t, dag, []*daghash.Hash{dag.genesis.hash}, nil) + nodeC, ok := dag.index.LookupNode(blockC.BlockHash()) + if !ok { + t.Fatalf("TestIsInPast: block C is not in the block index") + } + + // Add a block whose parents are the two tips + blockD := PrepareAndProcessBlockForTest(t, dag, []*daghash.Hash{blockB.BlockHash(), blockC.BlockHash()}, nil) + nodeD, ok := dag.index.LookupNode(blockD.BlockHash()) + if !ok { + t.Fatalf("TestIsInPast: block C is not in the block index") + } + + // Make sure that node C is in the past of node D + isInFuture, err := dag.reachabilityTree.isInPast(nodeC, nodeD) + if err != nil { + t.Fatalf("TestIsInPast: isInPast unexpectedly failed: %s", err) + } + if !isInFuture { + t.Fatalf("TestIsInPast: node C is unexpectedly not the past of node D") + } +} + +func TestUpdateReindexRoot(t *testing.T) { + // Create a new database and DAG instance to run tests against. + dag, teardownFunc, err := DAGSetup("TestUpdateReindexRoot", true, Config{ + DAGParams: &dagconfig.SimnetParams, + }) + if err != nil { + t.Fatalf("Failed to setup DAG instance: %v", err) + } + defer teardownFunc() + + // Set the reindex window to a low number to make this test run fast + originalReachabilityReindexWindow := reachabilityReindexWindow + reachabilityReindexWindow = 10 + defer func() { + reachabilityReindexWindow = originalReachabilityReindexWindow + }() + + // Add two blocks on top of the genesis block + chain1RootBlock := PrepareAndProcessBlockForTest(t, dag, []*daghash.Hash{dag.genesis.hash}, nil) + chain2RootBlock := PrepareAndProcessBlockForTest(t, dag, []*daghash.Hash{dag.genesis.hash}, nil) + + // Add chain of reachabilityReindexWindow - 1 blocks above chain1RootBlock and + // chain2RootBlock, respectively. This should not move the reindex root + chain1RootBlockTipHash := chain1RootBlock.BlockHash() + chain2RootBlockTipHash := chain2RootBlock.BlockHash() + genesisTreeNode, err := dag.reachabilityTree.store.treeNodeByBlockHash(dag.genesis.hash) + if err != nil { + t.Fatalf("failed to get tree node: %s", err) + } + for i := uint64(0); i < reachabilityReindexWindow-1; i++ { + chain1Block := PrepareAndProcessBlockForTest(t, dag, []*daghash.Hash{chain1RootBlockTipHash}, nil) + chain1RootBlockTipHash = chain1Block.BlockHash() + + chain2Block := PrepareAndProcessBlockForTest(t, dag, []*daghash.Hash{chain2RootBlockTipHash}, nil) + chain2RootBlockTipHash = chain2Block.BlockHash() + + if dag.reachabilityTree.reindexRoot != genesisTreeNode { + t.Fatalf("reindex root unexpectedly moved") + } + } + + // Add another block over chain1. This will move the reindex root to chain1RootBlock + PrepareAndProcessBlockForTest(t, dag, []*daghash.Hash{chain1RootBlockTipHash}, nil) + + // Make sure that chain1RootBlock is now the reindex root + chain1RootTreeNode, err := dag.reachabilityTree.store.treeNodeByBlockHash(chain1RootBlock.BlockHash()) + if err != nil { + t.Fatalf("failed to get tree node: %s", err) + } + if dag.reachabilityTree.reindexRoot != chain1RootTreeNode { + t.Fatalf("chain1RootBlock is not the reindex root after reindex") + } + + // Make sure that tight intervals have been applied to chain2. Since + // we added reachabilityReindexWindow-1 blocks to chain2, the size + // of the interval at its root should be equal to reachabilityReindexWindow + chain2RootTreeNode, err := dag.reachabilityTree.store.treeNodeByBlockHash(chain2RootBlock.BlockHash()) + if err != nil { + t.Fatalf("failed to get tree node: %s", err) + } + if chain2RootTreeNode.interval.size() != reachabilityReindexWindow { + t.Fatalf("got unexpected chain2RootNode interval. Want: %d, got: %d", + chain2RootTreeNode.interval.size(), reachabilityReindexWindow) + } + + // Make sure that the rest of the interval has been allocated to + // chain1RootNode, minus slack from both sides + expectedChain1RootIntervalSize := genesisTreeNode.interval.size() - 1 - + chain2RootTreeNode.interval.size() - 2*reachabilityReindexSlack + if chain1RootTreeNode.interval.size() != expectedChain1RootIntervalSize { + t.Fatalf("got unexpected chain1RootNode interval. Want: %d, got: %d", + chain1RootTreeNode.interval.size(), expectedChain1RootIntervalSize) + } +} + +func TestReindexIntervalsEarlierThanReindexRoot(t *testing.T) { + // Create a new database and DAG instance to run tests against. + dag, teardownFunc, err := DAGSetup("TestReindexIntervalsEarlierThanReindexRoot", true, Config{ + DAGParams: &dagconfig.SimnetParams, + }) + if err != nil { + t.Fatalf("Failed to setup DAG instance: %v", err) + } + defer teardownFunc() + + // Set the reindex window and slack to low numbers to make this test + // run fast + originalReachabilityReindexWindow := reachabilityReindexWindow + originalReachabilityReindexSlack := reachabilityReindexSlack + reachabilityReindexWindow = 10 + reachabilityReindexSlack = 5 + defer func() { + reachabilityReindexWindow = originalReachabilityReindexWindow + reachabilityReindexSlack = originalReachabilityReindexSlack + }() + + // Add three children to the genesis: leftBlock, centerBlock, rightBlock + leftBlock := PrepareAndProcessBlockForTest(t, dag, []*daghash.Hash{dag.genesis.hash}, nil) + centerBlock := PrepareAndProcessBlockForTest(t, dag, []*daghash.Hash{dag.genesis.hash}, nil) + rightBlock := PrepareAndProcessBlockForTest(t, dag, []*daghash.Hash{dag.genesis.hash}, nil) + + // Add a chain of reachabilityReindexWindow blocks above centerBlock. + // This will move the reindex root to centerBlock + centerTipHash := centerBlock.BlockHash() + for i := uint64(0); i < reachabilityReindexWindow; i++ { + block := PrepareAndProcessBlockForTest(t, dag, []*daghash.Hash{centerTipHash}, nil) + centerTipHash = block.BlockHash() + } + + // Make sure that centerBlock is now the reindex root + centerTreeNode, err := dag.reachabilityTree.store.treeNodeByBlockHash(centerBlock.BlockHash()) + if err != nil { + t.Fatalf("failed to get tree node: %s", err) + } + if dag.reachabilityTree.reindexRoot != centerTreeNode { + t.Fatalf("centerBlock is not the reindex root after reindex") + } + + // Get the current interval for leftBlock. The reindex should have + // resulted in a tight interval there + leftTreeNode, err := dag.reachabilityTree.store.treeNodeByBlockHash(leftBlock.BlockHash()) + if err != nil { + t.Fatalf("failed to get tree node: %s", err) + } + if leftTreeNode.interval.size() != 1 { + t.Fatalf("leftBlock interval not tight after reindex") + } + + // Get the current interval for rightBlock. The reindex should have + // resulted in a tight interval there + rightTreeNode, err := dag.reachabilityTree.store.treeNodeByBlockHash(rightBlock.BlockHash()) + if err != nil { + t.Fatalf("failed to get tree node: %s", err) + } + if rightTreeNode.interval.size() != 1 { + t.Fatalf("rightBlock interval not tight after reindex") + } + + // Get the current interval for centerBlock. Its interval should be: + // genesisInterval - 1 - leftInterval - leftSlack - rightInterval - rightSlack + genesisTreeNode, err := dag.reachabilityTree.store.treeNodeByBlockHash(dag.genesis.hash) + if err != nil { + t.Fatalf("failed to get tree node: %s", err) + } + expectedCenterInterval := genesisTreeNode.interval.size() - 1 - + leftTreeNode.interval.size() - reachabilityReindexSlack - + rightTreeNode.interval.size() - reachabilityReindexSlack + if centerTreeNode.interval.size() != expectedCenterInterval { + t.Fatalf("unexpected centerBlock interval. Want: %d, got: %d", + expectedCenterInterval, centerTreeNode.interval.size()) + } + + // Add a chain of reachabilityReindexWindow - 1 blocks above leftBlock. + // Each addition will trigger a low-than-reindex-root reindex. We + // expect the centerInterval to shrink by 1 each time, but its child + // to remain unaffected + treeChildOfCenterBlock := centerTreeNode.children[0] + treeChildOfCenterBlockOriginalIntervalSize := treeChildOfCenterBlock.interval.size() + leftTipHash := leftBlock.BlockHash() + for i := uint64(0); i < reachabilityReindexWindow-1; i++ { + block := PrepareAndProcessBlockForTest(t, dag, []*daghash.Hash{leftTipHash}, nil) + leftTipHash = block.BlockHash() + + expectedCenterInterval-- + if centerTreeNode.interval.size() != expectedCenterInterval { + t.Fatalf("unexpected centerBlock interval. Want: %d, got: %d", + expectedCenterInterval, centerTreeNode.interval.size()) + } + + if treeChildOfCenterBlock.interval.size() != treeChildOfCenterBlockOriginalIntervalSize { + t.Fatalf("the interval of centerBlock's child unexpectedly changed") + } + } + + // Add a chain of reachabilityReindexWindow - 1 blocks above rightBlock. + // Each addition will trigger a low-than-reindex-root reindex. We + // expect the centerInterval to shrink by 1 each time, but its child + // to remain unaffected + rightTipHash := rightBlock.BlockHash() + for i := uint64(0); i < reachabilityReindexWindow-1; i++ { + block := PrepareAndProcessBlockForTest(t, dag, []*daghash.Hash{rightTipHash}, nil) + rightTipHash = block.BlockHash() + + expectedCenterInterval-- + if centerTreeNode.interval.size() != expectedCenterInterval { + t.Fatalf("unexpected centerBlock interval. Want: %d, got: %d", + expectedCenterInterval, centerTreeNode.interval.size()) + } + + if treeChildOfCenterBlock.interval.size() != treeChildOfCenterBlockOriginalIntervalSize { + t.Fatalf("the interval of centerBlock's child unexpectedly changed") + } + } +} diff --git a/blockdag/reachabilitystore.go b/blockdag/reachabilitystore.go index dea254784..064e82d6b 100644 --- a/blockdag/reachabilitystore.go +++ b/blockdag/reachabilitystore.go @@ -12,7 +12,7 @@ import ( type reachabilityData struct { treeNode *reachabilityTreeNode - futureCoveringSet futureCoveringBlockSet + futureCoveringSet futureCoveringTreeNodeSet } type reachabilityStore struct { @@ -41,11 +41,11 @@ func (store *reachabilityStore) setTreeNode(treeNode *reachabilityTreeNode) { store.setBlockAsDirty(node.hash) } -func (store *reachabilityStore) setFutureCoveringSet(node *blockNode, futureCoveringSet futureCoveringBlockSet) error { +func (store *reachabilityStore) setFutureCoveringSet(node *blockNode, futureCoveringSet futureCoveringTreeNodeSet) error { // load the reachability data from DB to store.loaded _, exists := store.reachabilityDataByHash(node.hash) if !exists { - return reachabilityNotFoundError(node) + return reachabilityNotFoundError(node.hash) } store.loaded[*node.hash].futureCoveringSet = futureCoveringSet @@ -57,22 +57,26 @@ func (store *reachabilityStore) setBlockAsDirty(blockHash *daghash.Hash) { store.dirty[*blockHash] = struct{}{} } -func reachabilityNotFoundError(node *blockNode) error { - return errors.Errorf("Couldn't find reachability data for block %s", node.hash) +func reachabilityNotFoundError(hash *daghash.Hash) error { + return errors.Errorf("couldn't find reachability data for block %s", hash) } -func (store *reachabilityStore) treeNodeByBlockNode(node *blockNode) (*reachabilityTreeNode, error) { - reachabilityData, exists := store.reachabilityDataByHash(node.hash) +func (store *reachabilityStore) treeNodeByBlockHash(hash *daghash.Hash) (*reachabilityTreeNode, error) { + reachabilityData, exists := store.reachabilityDataByHash(hash) if !exists { - return nil, reachabilityNotFoundError(node) + return nil, reachabilityNotFoundError(hash) } return reachabilityData.treeNode, nil } -func (store *reachabilityStore) futureCoveringSetByBlockNode(node *blockNode) (futureCoveringBlockSet, error) { +func (store *reachabilityStore) treeNodeByBlockNode(node *blockNode) (*reachabilityTreeNode, error) { + return store.treeNodeByBlockHash(node.hash) +} + +func (store *reachabilityStore) futureCoveringSetByBlockNode(node *blockNode) (futureCoveringTreeNodeSet, error) { reachabilityData, exists := store.reachabilityDataByHash(node.hash) if !exists { - return nil, reachabilityNotFoundError(node) + return nil, reachabilityNotFoundError(node.hash) } return reachabilityData.futureCoveringSet, nil } @@ -215,12 +219,6 @@ func (store *reachabilityStore) serializeTreeNode(w io.Writer, treeNode *reachab return err } - // Serialize the remaining interval - err = store.serializeReachabilityInterval(w, treeNode.remainingInterval) - if err != nil { - return err - } - // Serialize the parent // If this is the genesis block, write the zero hash instead parentHash := &daghash.ZeroHash @@ -265,16 +263,16 @@ func (store *reachabilityStore) serializeReachabilityInterval(w io.Writer, inter return nil } -func (store *reachabilityStore) serializeFutureCoveringSet(w io.Writer, futureCoveringSet futureCoveringBlockSet) error { +func (store *reachabilityStore) serializeFutureCoveringSet(w io.Writer, futureCoveringSet futureCoveringTreeNodeSet) error { // Serialize the set size err := wire.WriteVarInt(w, uint64(len(futureCoveringSet))) if err != nil { return err } - // Serialize each block in the set - for _, block := range futureCoveringSet { - err = wire.WriteElement(w, block.blockNode.hash) + // Serialize each node in the set + for _, node := range futureCoveringSet { + err = wire.WriteElement(w, node.blockNode.hash) if err != nil { return err } @@ -311,13 +309,6 @@ func (store *reachabilityStore) deserializeTreeNode(r io.Reader, destination *re } destination.treeNode.interval = interval - // Deserialize the remaining interval - remainingInterval, err := store.deserializeReachabilityInterval(r) - if err != nil { - return err - } - destination.treeNode.remainingInterval = remainingInterval - // Deserialize the parent // If this is the zero hash, this node is the genesis and as such doesn't have a parent parentHash := &daghash.Hash{} @@ -388,25 +379,18 @@ func (store *reachabilityStore) deserializeFutureCoveringSet(r io.Reader, destin } // Deserialize each block in the set - futureCoveringSet := make(futureCoveringBlockSet, setSize) + futureCoveringSet := make(futureCoveringTreeNodeSet, setSize) for i := uint64(0); i < setSize; i++ { blockHash := &daghash.Hash{} err = wire.ReadElement(r, blockHash) if err != nil { return err } - blockNode, ok := store.dag.index.LookupNode(blockHash) - if !ok { - return errors.Errorf("blockNode not found for hash %s", blockHash) - } blockReachabilityData, ok := store.reachabilityDataByHash(blockHash) if !ok { return errors.Errorf("block reachability data not found for hash: %s", blockHash) } - futureCoveringSet[i] = &futureCoveringBlock{ - blockNode: blockNode, - treeNode: blockReachabilityData.treeNode, - } + futureCoveringSet[i] = blockReachabilityData.treeNode } destination.futureCoveringSet = futureCoveringSet diff --git a/blockdag/validate.go b/blockdag/validate.go index 7103986b1..18534a12e 100644 --- a/blockdag/validate.go +++ b/blockdag/validate.go @@ -709,7 +709,7 @@ func (dag *BlockDAG) validateParents(blockHeader *wire.BlockHeader, parents bloc continue } - isAncestorOf, err := dag.isAncestorOf(parentA, parentB) + isAncestorOf, err := dag.isInPast(parentA, parentB) if err != nil { return err } diff --git a/dbaccess/reachability.go b/dbaccess/reachability.go index 68401448e..dada75cfc 100644 --- a/dbaccess/reachability.go +++ b/dbaccess/reachability.go @@ -6,6 +6,7 @@ import ( ) var reachabilityDataBucket = database.MakeBucket([]byte("reachability")) +var reachabilityReindexKey = database.MakeBucket().Key([]byte("reachability-reindex-root")) func reachabilityKey(hash *daghash.Hash) *database.Key { return reachabilityDataBucket.Key(hash[:]) @@ -38,3 +39,26 @@ func StoreReachabilityData(context Context, blockHash *daghash.Hash, reachabilit func ClearReachabilityData(dbTx *TxContext) error { return clearBucket(dbTx, reachabilityDataBucket) } + +// StoreReachabilityReindexRoot stores the reachability reindex root in the database. +func StoreReachabilityReindexRoot(context Context, reachabilityReindexRoot *daghash.Hash) error { + accessor, err := context.accessor() + if err != nil { + return err + } + return accessor.Put(reachabilityReindexKey, reachabilityReindexRoot[:]) +} + +// FetchReachabilityReindexRoot retrieves the reachability reindex root from the database. +// Returns ErrNotFound if the state is missing from the database. +func FetchReachabilityReindexRoot(context Context) (*daghash.Hash, error) { + accessor, err := context.accessor() + if err != nil { + return nil, err + } + bytes, err := accessor.Get(reachabilityReindexKey) + if err != nil { + return nil, err + } + return daghash.NewHash(bytes) +}