From a61191e52eb8e17f5ab7b35b9100e2c9f394edfc Mon Sep 17 00:00:00 2001 From: Jingguo Yao Date: Sat, 16 Jul 2016 15:08:54 +0800 Subject: [PATCH] perf: replace LLRB-tree with btree in interval.Tree Fixes #6465 --- pkg/ccl/sqlccl/backup.go | 2 +- pkg/storage/command_queue.go | 4 +- pkg/util/cache/cache.go | 2 +- pkg/util/interval/btree_based_interval.go | 918 ++++++++++++++++++ .../interval/btree_based_interval_test.go | 591 +++++++++++ pkg/util/interval/bu23.go | 2 +- pkg/util/interval/interval.go | 752 ++------------ pkg/util/interval/llrb_based_interval.go | 679 +++++++++++++ pkg/util/interval/range_group.go | 20 +- pkg/util/interval/range_group_test.go | 6 +- pkg/util/interval/td234.go | 2 +- 11 files changed, 2277 insertions(+), 701 deletions(-) create mode 100644 pkg/util/interval/btree_based_interval.go create mode 100644 pkg/util/interval/btree_based_interval_test.go create mode 100644 pkg/util/interval/llrb_based_interval.go diff --git a/pkg/ccl/sqlccl/backup.go b/pkg/ccl/sqlccl/backup.go index 3c332334caae..1be7fa8a6a0a 100644 --- a/pkg/ccl/sqlccl/backup.go +++ b/pkg/ccl/sqlccl/backup.go @@ -138,7 +138,7 @@ func allRangeDescriptors(ctx context.Context, txn *client.Txn) ([]roachpb.RangeD // spansForAllTableIndexes returns non-overlapping spans for every index and // table passed in. They would normally overlap if any of them are interleaved. func spansForAllTableIndexes(tables []*sqlbase.TableDescriptor) []roachpb.Span { - sstIntervalTree := interval.Tree{Overlapper: interval.Range.OverlapExclusive} + sstIntervalTree := interval.Tree{Overlapper: interval.ExclusiveOverlapper} for _, table := range tables { for _, index := range table.AllNonDropIndexes() { if err := sstIntervalTree.Insert(intervalSpan(table.IndexSpan(index.ID)), false); err != nil { diff --git a/pkg/storage/command_queue.go b/pkg/storage/command_queue.go index 0763e93c5042..26e533f9f975 100644 --- a/pkg/storage/command_queue.go +++ b/pkg/storage/command_queue.go @@ -139,8 +139,8 @@ func (c *cmd) String() string { // typically contain many spans, but are spatially disjoint. func NewCommandQueue(coveringOptimization bool) *CommandQueue { cq := &CommandQueue{ - reads: interval.Tree{Overlapper: interval.Range.OverlapExclusive}, - writes: interval.Tree{Overlapper: interval.Range.OverlapExclusive}, + reads: interval.NewTree(interval.ExclusiveOverlapper), + writes: interval.NewTree(interval.ExclusiveOverlapper), wRg: interval.NewRangeTree(), rwRg: interval.NewRangeTree(), coveringOptimization: coveringOptimization, diff --git a/pkg/util/cache/cache.go b/pkg/util/cache/cache.go index ea85ffc217ad..a526ac6a44b7 100644 --- a/pkg/util/cache/cache.go +++ b/pkg/util/cache/cache.go @@ -496,7 +496,7 @@ func (ic *IntervalCache) MakeKey(start, end []byte) IntervalKey { // Implementation of cacheStore interface. func (ic *IntervalCache) init() { - ic.tree = interval.Tree{Overlapper: interval.Range.OverlapExclusive} + ic.tree = interval.NewTree(interval.ExclusiveOverlapper) } func (ic *IntervalCache) get(key interface{}) *Entry { diff --git a/pkg/util/interval/btree_based_interval.go b/pkg/util/interval/btree_based_interval.go new file mode 100644 index 000000000000..b6c45427cd18 --- /dev/null +++ b/pkg/util/interval/btree_based_interval.go @@ -0,0 +1,918 @@ +// Copyright 2016 The Cockroach Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. See the License for the specific language governing +// permissions and limitations under the License. +// +// This code is based on: https://github.com/google/btree +// +// Author: Jingguo Yao (yaojingguo@gmail.com) + +// +build interval_btree + +// Package interval implements an interval tree based on an augmented B-tree. +// For all the functions which have a fast argment, fast being true means a fast +// operation which does not adjust node ranges. If fast is false, node ranges +// are adjusted. +package interval + +import ( + "errors" + "sort" +) + +const ( + // DefaultBTreeMinimumDegree is the default B-tree minimum degree. Benchmarks + // show that the interval tree performs best with this minimum degree. + DefaultBTreeMinimumDegree = 32 +) + +// NewTree creates a new interval tree with the given overlapper function and +// the default B-tree minimum degree. +func NewTree(overlapper Overlapper) Tree { + return NewTreeWithDegree(overlapper, DefaultBTreeMinimumDegree) +} + +// NewTreeWithDegree creates a new interval tree with the given overlapper +// function and the given minimum degree. A minimum degree less than 2 will +// cause a panic. +// +// NewTreeWithDegree(overlapper, 2), for example, will create a 2-3-4 tree (each +// node contains 1-3 Interfaces and 2-4 children). +func NewTreeWithDegree(overlapper Overlapper, minimumDegree int) Tree { + if minimumDegree < 2 { + panic("bad minimum degree") + } + return Tree{ + MinimumDegree: minimumDegree, + Overlapper: overlapper, + } +} + +func isValidInterface(a Interface) error { + if a == nil { + return errors.New("nil interface") + } + r := a.Range() + return rangeError(r) +} + +// interfaces stores Interfaces sorted by Range().End in a node. +type items []Interface + +// insertAt inserts a value into the given index, pushing all subsequent values +// forward. +func (s *items) insertAt(index int, e Interface) { + oldLen := len(*s) + *s = append(*s, nil) + if index < oldLen { + copy((*s)[index+1:], (*s)[index:]) + } + (*s)[index] = e +} + +// removeAt removes a value at a given index, pulling all subsequent values +// back. +func (s *items) removeAt(index int) Interface { + e := (*s)[index] + (*s)[index] = nil + copy((*s)[index:], (*s)[index+1:]) + *s = (*s)[:len(*s)-1] + return e +} + +// pop removes and returns the last element in the list. +func (s *items) pop() (out Interface) { + index := len(*s) - 1 + out = (*s)[index] + (*s)[index] = nil + *s = (*s)[:index] + return +} + +// find returns the index where the given Interface should be inserted into this +// list. 'found' is true if the interface already exists in the list at the +// given index. +func (s items) find(e Interface) (index int, found bool) { + i := sort.Search(len(s), func(i int) bool { + return Compare(e, s[i]) < 0 + }) + if i > 0 && Equal(s[i-1], e) { + return i - 1, true + } + return i, false +} + +// children stores child nodes sorted by Range.End in a node. +type children []*node + +// insertAt inserts a value into the given index, pushing all subsequent values +// forward. +func (s *children) insertAt(index int, n *node) { + oldLen := len(*s) + *s = append(*s, nil) + if index < oldLen { + copy((*s)[index+1:], (*s)[index:]) + } + (*s)[index] = n +} + +// removeAt removes a value at a given index, pulling all subsequent values +// back. +func (s *children) removeAt(index int) *node { + n := (*s)[index] + (*s)[index] = nil + copy((*s)[index:], (*s)[index+1:]) + *s = (*s)[:len(*s)-1] + return n +} + +// pop removes and returns the last element in the list. +func (s *children) pop() (out *node) { + index := len(*s) - 1 + out = (*s)[index] + (*s)[index] = nil + *s = (*s)[:index] + return +} + +// node is an internal node in a tree. +// +// It must at all times maintain the invariant that either +// * len(children) == 0, len(interfaces) unconstrained +// * len(children) == len(interfaces) + 1 +type node struct { + // Range is the node range which covers all the ranges in the subtree rooted + // at the node. Range.Start is the leftmost position. Range.End is the + // rightmost position. Here we follow the approach employed by + // https://github.com/biogo/store/tree/master/interval since it make it easy + // to analyze the traversal of intervals which overlaps with a given interval. + // CLRS only uses Range.End. + Range Range + items items + children children + t *Tree +} + +// split splits the given node at the given index. The current node shrinks, and +// this function returns the Interface that existed at that index and a new node +// containing all interfaces/children after it. Before splitting: +// +// +-----------+ +// | x y z | +// ---/-/-\-\--+ +// +// After splitting: +// +// +-----------+ +// | y | +// -----/-\----+ +// / \ +// v v +// +-----------+ +-----------+ +// | x | | z | +// +-----------+ +-----------+ +// +func (n *node) split(i int, fast bool) (Interface, *node) { + e := n.items[i] + second := n.t.newNode() + second.items = make(items, n.t.minItems()) + copy(second.items, n.items[i+1:]) + n.items = n.items[:i] + if len(n.children) > 0 { + second.children = make(children, n.t.minItems()+1) + copy(second.children, n.children[i+1:]) + n.children = n.children[:i+1] + } + if !fast { + // adjust range for the first split part + oldRangeEnd := n.Range.End + n.Range.End = n.rangeEnd() + + // adjust ragne for the second split part + second.Range.Start = second.rangeStart() + if n.Range.End.Equal(oldRangeEnd) || e.Range().End.Equal(oldRangeEnd) { + second.Range.End = second.rangeEnd() + } else { + second.Range.End = oldRangeEnd + } + } + return e, second +} + +// maybeSplitChild checks if a child should be split, and if so splits it. +// Returns whether or not a split occurred. +func (n *node) maybeSplitChild(i int, fast bool) bool { + maxItems := n.t.maxItems() + if len(n.children[i].items) < maxItems { + return false + } + first := n.children[i] + e, second := first.split(maxItems/2, fast) + n.items.insertAt(i, e) + n.children.insertAt(i+1, second) + return true +} + +// insert inserts an Interface into the subtree rooted at this node, making sure +// no nodes in the subtree exceed maxItems Interfaces. +func (n *node) insert(e Interface, fast bool) (out Interface, extended bool) { + i, found := n.items.find(e) + if found { + out = n.items[i] + n.items[i] = e + return + } + if len(n.children) == 0 { + n.items.insertAt(i, e) + out = nil + if !fast { + if i == 0 { + extended = true + n.Range.Start = n.items[0].Range().Start + } + if n.items[i].Range().End.Compare(n.Range.End) > 0 { + extended = true + n.Range.End = n.items[i].Range().End + } + } + return + } + if n.maybeSplitChild(i, fast) { + inTree := n.items[i] + switch Compare(e, inTree) { + case -1: + // no change, we want first split node + case 1: + i++ // we want second split node + default: + out = n.items[i] + n.items[i] = e + return + } + } + out, extended = n.children[i].insert(e, fast) + if !fast && extended { + extended = false + if i == 0 && n.children[0].Range.Start.Compare(n.Range.Start) < 0 { + extended = true + n.Range.Start = n.children[0].Range.Start + } + if n.children[i].Range.End.Compare(n.Range.End) > 0 { + extended = true + n.Range.End = n.children[i].Range.End + } + } + return +} + +func (t *Tree) isEmpty() bool { + return t.root == nil || len(t.root.items) == 0 +} + +// Get returns a slice of Interfaces that overlap r in the tree. The slice is +// sorted nondecreasingly by interval start. +func (t *Tree) Get(r Range) (o []Interface) { + return t.GetWithOverlapper(r, t.Overlapper) +} + +// GetWithOverlapper returns a slice of Interfaces that overlap r in the tree +// using the provided overlapper function. The slice is sorted nondecreasingly +// by interval start. +func (t *Tree) GetWithOverlapper(r Range, overlapper Overlapper) (o []Interface) { + if err := rangeError(r); err != nil { + return + } + if !t.overlappable(r) { + return + } + t.root.doMatch(func(e Interface) (done bool) { o = append(o, e); return }, r, overlapper) + return +} + +// DoMatching performs fn on all intervals stored in the tree that overlaps r. +// The traversal is done in the nondecreasing order of interval start. A boolean +// is returned indicating whether the traversal was interrupted by an Operation +// returning true. If fn alters stored intervals' sort relationships, future +// tree operation behaviors are undefined. +func (t *Tree) DoMatching(fn Operation, r Range) bool { + if err := rangeError(r); err != nil { + return false + } + if !t.overlappable(r) { + return false + } + return t.root.doMatch(fn, r, t.Overlapper) +} + +func (t *Tree) overlappable(r Range) bool { + if t.isEmpty() || !t.Overlapper.Overlap(r, t.root.Range) { + return false + } + return true +} + +// benchmarks show that if Comparable.Compare is invoked directly instead of +// through an indirection with Overlapper, Insert, Delete and a traversal to +// visit overlapped intervals have a noticeable speed-up. So two versions of +// doMatch are created. One is for InclusiveOverlapper. The other is for +// ExclusiveOverlapper. +func (n *node) doMatch(fn Operation, r Range, overlapper Overlapper) (done bool) { + if overlapper == InclusiveOverlapper { + return n.inclusiveDoMatch(fn, r, overlapper) + } + return n.exclusiveDoMatch(fn, r, overlapper) +} + +// doMatch for InclusiveOverlapper. +func (n *node) inclusiveDoMatch(fn Operation, r Range, overlapper Overlapper) (done bool) { + length := sort.Search(len(n.items), func(i int) bool { + return n.items[i].Range().Start.Compare(r.End) > 0 + }) + + if len(n.children) == 0 { + for _, e := range n.items[:length] { + if r.Start.Compare(e.Range().End) <= 0 { + if done = fn(e); done { + return + } + } + } + return + } + + for i := 0; i < length; i++ { + c := n.children[i] + if r.Start.Compare(c.Range.End) <= 0 { + if done = c.inclusiveDoMatch(fn, r, overlapper); done { + return + } + } + e := n.items[i] + if r.Start.Compare(e.Range().End) <= 0 { + if done = fn(e); done { + return + } + } + } + + if overlapper.Overlap(r, n.children[length].Range) { + done = n.children[length].inclusiveDoMatch(fn, r, overlapper) + } + return +} + +// doMatch for ExclusiveOverlapper. +func (n *node) exclusiveDoMatch(fn Operation, r Range, overlapper Overlapper) (done bool) { + length := sort.Search(len(n.items), func(i int) bool { + return n.items[i].Range().Start.Compare(r.End) >= 0 + }) + + if len(n.children) == 0 { + for _, e := range n.items[:length] { + if r.Start.Compare(e.Range().End) < 0 { + if done = fn(e); done { + return + } + } + } + return + } + + for i := 0; i < length; i++ { + c := n.children[i] + if r.Start.Compare(c.Range.End) < 0 { + if done = c.exclusiveDoMatch(fn, r, overlapper); done { + return + } + } + e := n.items[i] + if r.Start.Compare(e.Range().End) < 0 { + if done = fn(e); done { + return + } + } + } + + if overlapper.Overlap(r, n.children[length].Range) { + done = n.children[length].exclusiveDoMatch(fn, r, overlapper) + } + return +} + +// Do performs fn on all intervals stored in the tree. The traversal is done in +// the nondecreasing order of interval start. A boolean is returned indicating +// whether the traversal was interrupted by an Operation returning true. If fn +// alters stored intervals' sort relationships, future tree operation behaviors +// are undefined. +func (t *Tree) Do(fn Operation) bool { + if t.root == nil { + return false + } + return t.root.do(fn) +} + +func (n *node) do(fn Operation) (done bool) { + cLen := len(n.children) + if cLen == 0 { + for _, e := range n.items { + if done = fn(e); done { + return + } + } + return + } + + for i := 0; i < cLen-1; i++ { + c := n.children[i] + if done = c.do(fn); done { + return + } + e := n.items[i] + if done = fn(e); done { + return + } + } + done = n.children[cLen-1].do(fn) + return +} + +// toRemove details what interface to remove in a node.remove call. +type toRemove int + +const ( + removeItem toRemove = iota // removes the given interface + removeMin // removes smallest interface in the subtree + removeMax // removes largest interface in the subtree +) + +// remove removes an interface from the subtree rooted at this node. +func (n *node) remove( + e Interface, minItems int, typ toRemove, fast bool, +) (out Interface, shrunk bool) { + var i int + var found bool + switch typ { + case removeMax: + if len(n.children) == 0 { + return n.removeFromLeaf(len(n.items)-1, fast) + } + i = len(n.items) + case removeMin: + if len(n.children) == 0 { + return n.removeFromLeaf(0, fast) + } + i = 0 + case removeItem: + i, found = n.items.find(e) + if len(n.children) == 0 { + if found { + return n.removeFromLeaf(i, fast) + } + return + } + default: + panic("invalid remove type") + } + // If we get to here, we have children. + child := n.children[i] + if len(child.items) <= minItems { + out, shrunk = n.growChildAndRemove(i, e, minItems, typ, fast) + return + } + // Either we had enough interfaces to begin with, or we've done some + // merging/stealing, because we've got enough now and we're ready to return + // stuff. + if found { + // The interface exists at index 'i', and the child we've selected can give + // us a predecessor, since if we've gotten here it's got > minItems + // interfaces in it. + out = n.items[i] + // We use our special-case 'remove' call with typ=removeMax to pull the + // predecessor of interface i (the rightmost leaf of our immediate left + // child) and set it into where we pulled the interface from. + n.items[i], _ = child.remove(nil, minItems, removeMax, fast) + if !fast { + shrunk = n.adjustRangeEndForRemoval(out, nil) + } + return + } + // Final recursive call. Once we're here, we know that the interface isn't in + // this node and that the child is big enough to remove from. + out, shrunk = child.remove(e, minItems, typ, fast) + if !fast && shrunk { + shrunkOnStart := false + if i == 0 { + if n.Range.Start.Compare(child.Range.Start) < 0 { + shrunkOnStart = true + n.Range.Start = child.Range.Start + } + } + shrunkOnEnd := n.adjustRangeEndForRemoval(out, nil) + shrunk = shrunkOnStart || shrunkOnEnd + } + return +} + +// adjustRangeEndForRemoval adjusts Range.End for the node after an interface +// and/or a child is removed. +func (n *node) adjustRangeEndForRemoval(e Interface, c *node) (decreased bool) { + if (e != nil && e.Range().End.Equal(n.Range.End)) || (c != nil && c.Range.End.Equal(n.Range.End)) { + newEnd := n.rangeEnd() + if n.Range.End.Compare(newEnd) > 0 { + decreased = true + n.Range.End = newEnd + } + } + return +} + +// removeFromLeaf removes children[i] from the leaf node. +func (n *node) removeFromLeaf(i int, fast bool) (out Interface, shrunk bool) { + if i == len(n.items)-1 { + out = n.items.pop() + } else { + out = n.items.removeAt(i) + } + if !fast && len(n.items) > 0 { + shrunkOnStart := false + if i == 0 { + oldStart := n.Range.Start + n.Range.Start = n.items[0].Range().Start + if !n.Range.Start.Equal(oldStart) { + shrunkOnStart = true + } + } + shrunkOnEnd := n.adjustRangeEndForRemoval(out, nil) + shrunk = shrunkOnStart || shrunkOnEnd + } + return +} + +// growChildAndRemove grows child 'i' to make sure it's possible to remove an +// Interface from it while keeping it at minItems, then calls remove to +// actually remove it. +// +// Most documentation says we have to do two sets of special casing: +// 1) interface is in this node +// 2) interface is in child +// In both cases, we need to handle the two subcases: +// A) node has enough values that it can spare one +// B) node doesn't have enough values +// For the latter, we have to check: +// a) left sibling has node to spare +// b) right sibling has node to spare +// c) we must merge +// To simplify our code here, we handle cases #1 and #2 the same: +// If a node doesn't have enough Interfaces, we make sure it does (using a,b,c). +// We then simply redo our remove call, and the second time (regardless of +// whether we're in case 1 or 2), we'll have enough Interfaces and can guarantee +// that we hit case A. +func (n *node) growChildAndRemove( + i int, e Interface, minItems int, typ toRemove, fast bool, +) (out Interface, shrunk bool) { + if i > 0 && len(n.children[i-1].items) > minItems { + n.stealFromLeftChild(i, fast) + } else if i < len(n.items) && len(n.children[i+1].items) > minItems { + n.stealFromRightChild(i, fast) + } else { + if i >= len(n.items) { + i-- + } + n.mergeWithRightChild(i, fast) + } + return n.remove(e, minItems, typ, fast) +} + +// Steal from left child. Before stealing: +// +// +-----------+ +// | y | +// -----/-\----+ +// / \ +// v v +// +-----------+ +-----------+ +// | x | | | +// +----------\+ +-----------+ +// \ +// v +// a +// +// After stealing: +// +// +-----------+ +// | x | +// -----/-\----+ +// / \ +// v v +// +-----------+ +-----------+ +// | | | y | +// +-----------+ +/----------+ +// / +// v +// a +// +func (n *node) stealFromLeftChild(i int, fast bool) { + // steal + stealTo := n.children[i] + stealFrom := n.children[i-1] + x := stealFrom.items.pop() + y := n.items[i-1] + stealTo.items.insertAt(0, y) + n.items[i-1] = x + var a *node + if len(stealFrom.children) > 0 { + a = stealFrom.children.pop() + stealTo.children.insertAt(0, a) + } + + if !fast { + // adjust range for stealFrom + stealFrom.adjustRangeEndForRemoval(x, a) + + // adjust range for stealTo + stealTo.Range.Start = stealTo.rangeStart() + if y.Range().End.Compare(stealTo.Range.End) > 0 { + stealTo.Range.End = y.Range().End + } + if a != nil && a.Range.End.Compare(stealTo.Range.End) > 0 { + stealTo.Range.End = a.Range.End + } + } +} + +// Steal from right child. Before stealing: +// +// +-----------+ +// | y | +// -----/-\----+ +// / \ +// v v +// +-----------+ +-----------+ +// | | | x | +// +---------- + +/----------+ +// / +// v +// a +// +// After stealing: +// +// +-----------+ +// | x | +// -----/-\----+ +// / \ +// v v +// +-----------+ +-----------+ +// | y | | | +// +----------\+ +-----------+ +// \ +// v +// a +// +func (n *node) stealFromRightChild(i int, fast bool) { + // steal + stealTo := n.children[i] + stealFrom := n.children[i+1] + x := stealFrom.items.removeAt(0) + y := n.items[i] + stealTo.items = append(stealTo.items, y) + n.items[i] = x + var a *node + if len(stealFrom.children) > 0 { + a = stealFrom.children.removeAt(0) + stealTo.children = append(stealTo.children, a) + } + + if !fast { + // adjust range for stealFrom + stealFrom.Range.Start = stealFrom.rangeStart() + stealFrom.adjustRangeEndForRemoval(x, a) + + // adjust range for stealTo + if y.Range().End.Compare(stealTo.Range.End) > 0 { + stealTo.Range.End = y.Range().End + } + if a != nil && a.Range.End.Compare(stealTo.Range.End) > 0 { + stealTo.Range.End = a.Range.End + } + } +} + +// Merge with right child. Before merging: +// +// +-----------+ +// | u y v | +// -----/-\----+ +// / \ +// v v +// +-----------+ +-----------+ +// | x | | z | +// +---------- + +-----------+ +// +// After merging: +// +// +-----------+ +// | u v | +// ------|-----+ +// | +// v +// +-----------+ +// | x y z | +// +---------- + +// +func (n *node) mergeWithRightChild(i int, fast bool) { + // merge + y := n.items.removeAt(i) + child := n.children[i] + mergeChild := n.children.removeAt(i + 1) + child.items = append(child.items, y) + child.items = append(child.items, mergeChild.items...) + child.children = append(child.children, mergeChild.children...) + + if !fast { + if y.Range().End.Compare(child.Range.End) > 0 { + child.Range.End = y.Range().End + } + if mergeChild.Range.End.Compare(child.Range.End) > 0 { + child.Range.End = mergeChild.Range.End + } + } +} + +// Tree is a B-tree based interval tree. +// +// Tree stores Instances in an ordered structure, allowing easy insertion, +// removal, and iteration. +// +// Write operations are not safe for concurrent mutation by multiple +// goroutines, but Read operations are. +type Tree struct { + root *node + length int + Overlapper Overlapper + MinimumDegree int +} + +// adjustRange sets the Range to the maximum extent of the childrens' Range +// spans and its range spans. +func (n *node) adjustRange() { + n.Range.Start = n.rangeStart() + n.Range.End = n.rangeEnd() +} + +// rangeStart returns the leftmost position for the node range, assuming that +// its children have correct range extents. +func (n *node) rangeStart() Comparable { + minStart := n.items[0].Range().Start + if len(n.children) > 0 { + minStart = n.children[0].Range.Start + } + return minStart +} + +// rangeEnd returns the rightmost position for the node range, assuming that its +// children have correct range extents. +func (n *node) rangeEnd() Comparable { + if len(n.items) == 0 { + maxEnd := n.children[0].Range.End + for _, c := range n.children[1:] { + if end := c.Range.End; maxEnd.Compare(end) < 0 { + maxEnd = end + } + } + return maxEnd + } + maxEnd := n.items[0].Range().End + for _, e := range n.items[1:] { + if end := e.Range().End; maxEnd.Compare(end) < 0 { + maxEnd = end + } + } + for _, c := range n.children { + if end := c.Range.End; maxEnd.Compare(end) < 0 { + maxEnd = end + } + } + return maxEnd +} + +// AdjustRanges fixes range fields for all nodes in the tree. This must be +// called before Get, Do or DoMatching* is used if fast insertion or deletion +// has been performed. +func (t *Tree) AdjustRanges() { + if t.isEmpty() { + return + } + t.root.adjustRanges() +} + +func (n *node) adjustRanges() { + for _, c := range n.children { + c.adjustRanges() + } + n.adjustRange() +} + +// maxItems returns the max number of Interfaces to allow per node. +func (t *Tree) maxItems() int { + return t.MinimumDegree*2 - 1 +} + +// minItems returns the min number of Interfaces to allow per node (ignored +// for the root node). +func (t *Tree) minItems() int { + return t.MinimumDegree - 1 +} + +func (t *Tree) newNode() (n *node) { + n = &node{t: t} + return +} + +// Insert inserts the Interface e into the tree. Insertions may replace an +// existing Interface which is equal to the Interface e. +func (t *Tree) Insert(e Interface, fast bool) (err error) { + // t.metrics("Insert") + if err = isValidInterface(e); err != nil { + return + } + + if t.root == nil { + t.root = t.newNode() + t.root.items = append(t.root.items, e) + t.length++ + if !fast { + t.root.Range.Start = e.Range().Start + t.root.Range.End = e.Range().End + } + return nil + } else if len(t.root.items) >= t.maxItems() { + oldroot := t.root + t.root = t.newNode() + if !fast { + t.root.Range.Start = oldroot.Range.Start + t.root.Range.End = oldroot.Range.End + } + e2, second := oldroot.split(t.maxItems()/2, fast) + t.root.items = append(t.root.items, e2) + t.root.children = append(t.root.children, oldroot, second) + } + out, _ := t.root.insert(e, fast) + if out == nil { + t.length++ + } + return +} + +// Delete deletes the Interface e if it exists in the B-tree. The deleted +// Interface is equal to the Interface e. +func (t *Tree) Delete(e Interface, fast bool) (err error) { + // t.metrics("Delete") + if err = isValidInterface(e); err != nil { + return + } + if !t.overlappable(e.Range()) { + return + } + t.delete(e, removeItem, fast) + return +} + +func (t *Tree) delete(e Interface, typ toRemove, fast bool) Interface { + out, _ := t.root.remove(e, t.minItems(), typ, fast) + if len(t.root.items) == 0 && len(t.root.children) > 0 { + t.root = t.root.children[0] + } + if out != nil { + t.length-- + } + return out +} + +// Len returns the number of Interfaces currently in the tree. +func (t *Tree) Len() int { + return t.length +} + +// TreeIterator iterates over all intervals stored in the Tree, in-order. +type TreeIterator struct { +} + +// Next iterates over the items stored in the tree, in-order. +func (ti *TreeIterator) Next() (i Interface, ok bool) { + panic("TODO") +} + +// Iterator creates an iterator to iterate over all intervals stored in the +// tree, in-order. +func (t *Tree) Iterator() TreeIterator { + panic("TODO") +} diff --git a/pkg/util/interval/btree_based_interval_test.go b/pkg/util/interval/btree_based_interval_test.go new file mode 100644 index 000000000000..33be63766a6e --- /dev/null +++ b/pkg/util/interval/btree_based_interval_test.go @@ -0,0 +1,591 @@ +// Copyright 2016 The Cockroach Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. See the License for the specific language governing +// permissions and limitations under the License. +// +// Author: Jingguo Yao (yaojingguo@gmail.com) + +// +build interval_btree + +package interval + +import ( + "bytes" + "encoding/binary" + "flag" + "fmt" + "math/rand" + "reflect" + "sort" + "testing" + + "github.com/cockroachdb/cockroach/pkg/util/timeutil" +) + +var btreeMinDegree = flag.Int("btree_min_degree", 32, "B-Tree minimum degree") + +func init() { + seed := timeutil.Now().Unix() + rand.Seed(seed) +} + +// perm returns a random permutation of intervals whose range start is in the +// range [0, n). +func perm(n uint32) (out items) { + for _, i := range rand.Perm(int(n)) { + u := uint32(i) + iv := makeMultiByteInterval(u, u+1, u) + out = append(out, iv) + } + return +} + +// rang returns an ordered list of intervals in the range [m, n]. +func rang(m, n uint32) (out items) { + for i := m; i <= n; i++ { + u := uint32(i) + iv := makeMultiByteInterval(u, u+1, u) + out = append(out, iv) + } + return +} + +func makeMultiByteInterval(start, end, id uint32) *Interval { + return &Interval{Range{toBytes(start), toBytes(end)}, uintptr(id)} +} + +func toBytes(n uint32) Comparable { + buf := new(bytes.Buffer) + if err := binary.Write(buf, binary.BigEndian, n); err != nil { + panic(fmt.Sprintf("binary.Write error: %s", err)) + } + return Comparable(buf.Bytes()) +} + +type Interval struct { + r Range + id uintptr +} + +func (iv *Interval) Range() Range { + return iv.r +} + +func (iv *Interval) ID() uintptr { + return iv.id +} + +func (iv *Interval) String() string { + return fmt.Sprintf("%v-%d", iv.Range(), iv.ID()) +} + +func (items items) Len() int { + return len(items) +} + +func (items items) Less(i, j int) bool { + return Compare(items[i], items[j]) <= 0 +} + +func (items items) Swap(i, j int) { + items[i], items[j] = items[j], items[i] +} + +func (children children) Len() int { + return len(children) +} + +func (children children) Less(i, j int) bool { + return children[i].Range.Start.Compare(children[j].Range.Start) <= 0 +} + +func (children children) Swap(i, j int) { + children[i], children[j] = children[j], children[i] +} + +// describe returns a string description of the tree. The format is similar to +// https://en.wikipedia.org/wiki/Newick_format +func (tree *Tree) describe() string { + if tree.isEmpty() { + return ";" + } + return tree.root.String() +} + +func (n node) String() string { + var buf bytes.Buffer + n.describe(&buf) + return buf.String() +} + +func (n *node) describe(buf *bytes.Buffer) { + if len(n.children) == 0 { + for idx, i := range n.items { + if idx != 0 { + buf.WriteString(",") + } + buf.WriteString(i.(*Interval).String()) + } + } + for i, c := range n.children { + buf.WriteString("(") + c.describe(buf) + buf.WriteString(fmt.Sprintf(":%s", c.Range)) + buf.WriteString(")") + if i < len(n.children)-1 { + buf.WriteString(n.items[i].(*Interval).String()) + } + } +} + +func (n *node) isKeyInRange(t *testing.T, min, max Comparable) bool { + t.Logf("%v, min: %v, max: %v", n, min, max) + for _, i := range n.items { + start := i.Range().Start + t.Log(i.Range()) + if min != nil && start.Compare(min) < 0 { + return false + } + if max != nil && start.Compare(max) > 0 { + return false + } + } + oldMin, oldMax := min, max + for i, c := range n.children { + min, max := oldMin, oldMax + if i != 0 { + min = n.items[i-1].Range().Start + } + if i != len(n.children)-1 { + max = n.items[i].Range().Start + } + if !c.isKeyInRange(t, min, max) { + return false + } + } + return true +} + +func (n *node) isSorted(t *testing.T) bool { + for _, c := range n.children { + if !c.isSorted(t) { + return false + } + } + if !sort.IsSorted(n.items) { + return false + } + if !sort.IsSorted(n.children) { + return false + } + return true +} + +func (tree *Tree) computeHeight() (h int) { + h = -1 + for node := tree.root; ; { + h++ + if len(node.children) == 0 { + break + } + node = node.children[0] + } + return +} + +func (n *node) isDepthEqualToHeight(t *testing.T, depth, height int) bool { + if len(n.children) == 0 { + return depth == height + } + for _, c := range n.children { + if !c.isDepthEqualToHeight(t, depth+1, height) { + return false + } + } + return true +} + +func (n *node) isCountAllowed(t *testing.T, minItems, maxItems int, root bool) bool { + iLen := len(n.items) + cLen := len(n.children) + if !root { + iAllowed := minItems <= iLen && iLen <= maxItems + if !iAllowed { + return false + } + } + if cLen > 0 { + cAllowed := cLen == iLen+1 + if !cAllowed { + return false + } + for _, c := range n.children { + allowed := c.isCountAllowed(t, minItems, maxItems, false) + if !allowed { + return false + } + } + } + return true +} + +func (n *node) isIntervalInRange(t *testing.T) bool { + for _, c := range n.children { + if !c.isIntervalInRange(t) { + return false + } + } + r := n.bound() + if !n.Range.Equal(r) { + t.Errorf("%v expected range %v, got %v", n, r, n.Range) + return false + } + return true +} + +func (r *Range) combine(other Range) { + if r.Start.Compare(other.Start) > 0 { + r.Start = other.Start + } + if r.End.Compare(other.End) < 0 { + r.End = other.End + } +} + +func (n *node) bound() Range { + r := n.items[0].Range() + ptr := &r + for _, e := range n.items[1:] { + ptr.combine(e.Range()) + } + for _, c := range n.children { + ptr.combine(c.Range) + } + return r +} + +func checkWithLen(t *testing.T, tree *Tree, l int) { + if tree.Len() != l { + t.Errorf("expected tree length %d, got %d", l, tree.Len()) + } + check(t, tree) +} + +func check(t *testing.T, tree *Tree) { + t.Logf("tree: %s", tree.describe()) + if !tree.isLeafSameDepth(t) { + t.Error("Not all the leaves have the same depth as the tree height") + } + if !tree.isCountAllowed(t) { + t.Error("Not all the nodes have allowed key count and child node count") + } + if !tree.isIntervalInRange(t) { + t.Error("Not all the nodes bound all the intervals in its subtree with its Range field") + } + if !tree.isSorted(t) { + t.Error("Not all the nodes have its items and children fields sorted") + } + if !tree.isKeyInRange(t) { + t.Error("not all the nodes keep node keys (range.start) in range") + } +} + +func (tree *Tree) isLeafSameDepth(t *testing.T) bool { + if tree.isEmpty() { + return true + } + h := tree.computeHeight() + t.Logf("tree height: %d", h) + return tree.root.isDepthEqualToHeight(t, 0, h) +} + +func (tree *Tree) isCountAllowed(t *testing.T) bool { + if tree.isEmpty() { + return true + } + return tree.root.isCountAllowed(t, tree.minItems(), tree.maxItems(), true) +} + +// Does every node correctly annotate the range of its children. +func (tree *Tree) isIntervalInRange(t *testing.T) bool { + if tree.isEmpty() { + return true + } + return tree.root.isIntervalInRange(t) +} + +func (tree *Tree) isSorted(t *testing.T) bool { + if tree.isEmpty() { + return true + } + return tree.root.isSorted(t) +} + +func (tree *Tree) isKeyInRange(t *testing.T) bool { + if tree.isEmpty() { + return true + } + return tree.root.isKeyInRange(t, nil, nil) +} + +func checkEqualIntervals(t *testing.T, actual, expected items) { + for i := 0; i < len(actual)-1; i++ { + if actual[i].Range().Start.Compare(actual[i+1].Range().Start) > 0 { + t.Fatalf("interval slice is not sorted: %v", actual) + break + } + } + sort.Sort(expected) + if !reflect.DeepEqual(actual, expected) { + t.Errorf("expected intervals %v, got %v", expected, actual) + } +} + +func checkTraversal(t *testing.T, tree *Tree, ivs items) { + // Get, GetWithOverlapper + r := Range{Comparable{0x0}, Comparable{0x1}} + expectedIntervals := items{ivs[0], ivs[2], ivs[4]} + checkEqualIntervals(t, tree.Get(r), expectedIntervals) + checkEqualIntervals(t, tree.GetWithOverlapper(r, ExclusiveOverlapper), items{ivs[0]}) + + // DoMatching + var overlapped items + tree.DoMatching(func(e Interface) bool { + overlapped = append(overlapped, e) + return false + }, r) + checkEqualIntervals(t, overlapped, expectedIntervals) + + // Do + var all items + tree.Do(func(e Interface) bool { + all = append(all, e) + return false + }) + checkEqualIntervals(t, all, ivs) +} + +func checkFastDelete(t *testing.T, tree *Tree, ivs items, deleteCount int) { + for i, iv := range ivs[:deleteCount] { + if err := tree.Delete(iv, true); err != nil { + t.Fatalf("delete error: %s", err) + } + // Unlike fast insert, AdjustRanges must be called after each fast delete. + // Otherwise, the following fast deletes may go wrong. + tree.AdjustRanges() + checkWithLen(t, tree, len(ivs)-i-1) + } +} + +func makeIntervals() items { + ivs := items{ + &Interval{Range{Comparable{0}, Comparable{2}}, 0}, + &Interval{Range{Comparable{2}, Comparable{4}}, 0}, + &Interval{Range{Comparable{1}, Comparable{6}}, 0}, + &Interval{Range{Comparable{3}, Comparable{4}}, 0}, + &Interval{Range{Comparable{1}, Comparable{3}}, 0}, + &Interval{Range{Comparable{4}, Comparable{6}}, 0}, + &Interval{Range{Comparable{5}, Comparable{8}}, 0}, + &Interval{Range{Comparable{6}, Comparable{8}}, 0}, + &Interval{Range{Comparable{5}, Comparable{9}}, 0}, + &Interval{Range{Comparable{0x11}, Comparable{0x13}}, 0}, + &Interval{Range{Comparable{0x14}, Comparable{0x16}}, 0}, + &Interval{Range{Comparable{0x15}, Comparable{0x18}}, 0}, + &Interval{Range{Comparable{0x10}, Comparable{0x12}}, 0}, + &Interval{Range{Comparable{0x20}, Comparable{0x62}}, 0}, + &Interval{Range{Comparable{0x24}, Comparable{0xA0}}, 0}, + &Interval{Range{Comparable{0x31}, Comparable{0x63}}, 0}, + &Interval{Range{Comparable{0x44}, Comparable{0x56}}, 0}, + &Interval{Range{Comparable{0x45}, Comparable{0x68}}, 0}, + &Interval{Range{Comparable{0x30}, Comparable{0x72}}, 0}, + &Interval{Range{Comparable{0x30}, Comparable{0x52}}, 0}, + &Interval{Range{Comparable{0x44}, Comparable{0xB0}}, 0}, + } + for i, iv := range ivs { + iv.(*Interval).id = uintptr(i) + } + return ivs +} + +// TestBTree is based on https://github.com/google/btree/blob/master/btree_test.go. +func TestBTree(t *testing.T) { + tree := NewTreeWithDegree(InclusiveOverlapper, *btreeMinDegree) + const treeSize = 10000 + for i := 0; i < 10; i++ { + for _, iv := range perm(treeSize) { + if x := tree.Insert(iv, false); x != nil { + t.Fatalf("insert found interval %v", x) + } + } + + for _, iv := range perm(treeSize) { + if x := tree.Insert(iv, false); x != nil { + t.Fatalf("insert didn't find interval %v", x) + } + } + + var all items + tree.DoMatching(func(e Interface) bool { + all = append(all, e) + return false + }, Range{toBytes(0), toBytes(treeSize)}) + if expected := rang(0, treeSize-1); !reflect.DeepEqual(all, expected) { + t.Fatalf("expected intervals %v, got %v", expected, all) + } + + var slice items + min := uint32(10) + max := uint32(20) + tree.DoMatching(func(e Interface) bool { + slice = append(slice, e) + return false + }, Range{toBytes(min + 1), toBytes(max)}) + if expected := rang(min, max); !reflect.DeepEqual(slice, expected) { + t.Fatalf("expected intervals %v, got %v", expected, slice) + } + + var halfSlice items + half := uint32(15) + tree.DoMatching(func(e Interface) bool { + if e.Range().Start.Compare(toBytes(half)) > 0 { + return true + } + halfSlice = append(halfSlice, e) + return false + }, Range{toBytes(min + 1), toBytes(max)}) + if expected := rang(min, half); !reflect.DeepEqual(halfSlice, expected) { + t.Fatalf("expected intervals %v, got %v", expected, halfSlice) + } + + for i, item := range perm(treeSize) { + t.Logf("i: %d", i) + if err := tree.Delete(item, false); err != nil { + t.Fatalf("delete error: %s", err) + } + } + + if len := tree.Len(); len > 0 { + t.Fatalf("expected 0 item, got %d itemes", len) + } + } +} + +// TestDeleteAfterRootNodeMerge verifies that delete from a leaf node works +// correctly after a merge which involves the root node. During the delete of a +// Interface from a leaf node, if the root node has only one Interface and takes +// part of a merge, the root does have any Interface after the merge. The +// subsequent adjustment of node range should take this into account. +func TestDeleteAfterRootNodeMerge(t *testing.T) { + tree := NewTreeWithDegree(InclusiveOverlapper, 2) + ivs := items{ + &Interval{Range{Comparable{1}, Comparable{8}}, 0}, + &Interval{Range{Comparable{2}, Comparable{3}}, 1}, + &Interval{Range{Comparable{3}, Comparable{4}}, 2}, + &Interval{Range{Comparable{4}, Comparable{5}}, 3}, + } + + // + // +------+ + // | id-1 | + // +------+ + // / \ + // v v + // +------+ +-----------+ + // | id-0 | | id-2 id-3 | + // +------+ +-----------+ + // + for i := 0; i < len(ivs); i++ { + if err := tree.Insert(ivs[i], false); err != nil { + t.Fatalf("insert error: %s", err) + } + } + + // + // +------+ + // | id-1 | + // +------+ + // / \ + // v v + // +------+ +------+ + // | id-0 | | id-3 | + // +------+ +------+ + // + if err := tree.Delete(ivs[2], false); err != nil { + t.Fatalf("delete error: %s", err) + } + + // Delete id-0 + if err := tree.Delete(ivs[0], false); err != nil { + t.Fatalf("delete error: %s", err) + } +} + +func TestSmallTree(t *testing.T) { + tree := NewTreeWithDegree(InclusiveOverlapper, 2) + ivs := makeIntervals() + + // Insert + for i, iv := range ivs { + if err := tree.Insert(iv, false); err != nil { + t.Fatalf("insert error: %s", err) + } + checkWithLen(t, &tree, i+1) + } + + checkTraversal(t, &tree, ivs) + + // Delete + l := tree.Len() + for i, iv := range ivs { + if err := tree.Delete(iv, false); err != nil { + t.Fatalf("delete error: %s", err) + } + checkWithLen(t, &tree, l-i-1) + } +} + +func TestSmallTreeWithFastOperations(t *testing.T) { + tree := NewTreeWithDegree(InclusiveOverlapper, 2) + ivs := makeIntervals() + + // Fast insert + for _, iv := range ivs { + if err := tree.Insert(iv, true); err != nil { + t.Fatalf("insert error: %s", err) + } + } + tree.AdjustRanges() + checkWithLen(t, &tree, len(ivs)) + + checkTraversal(t, &tree, ivs) + checkFastDelete(t, &tree, ivs, tree.Len()) +} + +func TestLargeTree(t *testing.T) { + var ivs items + + const treeSize = 40000 + for i := uint32(0); i < treeSize; i++ { + iv := makeMultiByteInterval(i, i+1, i) + ivs = append(ivs, iv) + } + + tree := NewTreeWithDegree(ExclusiveOverlapper, *btreeMinDegree) + for _, iv := range ivs { + if err := tree.Insert(iv, true); err != nil { + t.Fatalf("fast insert error: %s", err) + } + } + tree.AdjustRanges() + checkWithLen(t, &tree, treeSize) + checkFastDelete(t, &tree, ivs, 10) +} diff --git a/pkg/util/interval/bu23.go b/pkg/util/interval/bu23.go index d16136e95021..ad346fdef5f8 100644 --- a/pkg/util/interval/bu23.go +++ b/pkg/util/interval/bu23.go @@ -2,7 +2,7 @@ // Use of this source code is governed by a BSD-style // license that can be found in the LICENSE file. -// +build !td234 +// +build !td234,!interval_btree package interval diff --git a/pkg/util/interval/interval.go b/pkg/util/interval/interval.go index 8f59b4773655..9eda1aacae8c 100644 --- a/pkg/util/interval/interval.go +++ b/pkg/util/interval/interval.go @@ -2,30 +2,14 @@ // Use of this source code is governed by a BSD-style // license that can be found in the LICENSE file. -// Package interval implements an interval tree based on an augmented -// Left-Leaning Red Black tree. package interval import ( "bytes" "errors" "fmt" - - "github.com/biogo/store/llrb" -) - -// Operation mode of the underlying LLRB tree. -const ( - TD234 = iota - BU23 ) -func init() { - if Mode != TD234 && Mode != BU23 { - panic("interval: unknown mode") - } -} - // ErrInvertedRange is returned if an interval is used where the start value is greater // than the end value. var ErrInvertedRange = errors.New("interval: inverted range") @@ -50,33 +34,41 @@ type Range struct { Start, End Comparable } -// OverlapInclusive returns whether the two provided ranges overlap. It -// defines overlapping as a pair of ranges that share a segment, with both -// start and end keys treated as inclusive values. -func (r Range) OverlapInclusive(other Range) bool { - return r.End.Compare(other.Start) >= 0 && r.Start.Compare(other.End) <= 0 +// String implements the Stringer interface. +func (r Range) String() string { + return fmt.Sprintf("{%x-%x}", r.Start, r.End) } -// OverlapExclusive returns whether the two provided ranges overlap. It defines -// overlapping as a pair of ranges that share a segment of the keyspace, with the -// start keys treated as inclusive and the end keys treated as exclusive. -func (r Range) OverlapExclusive(other Range) bool { - return r.End.Compare(other.Start) > 0 && r.Start.Compare(other.End) < 0 +// Overlapper specifies the overlapping relationship. +type Overlapper interface { + // Overlap checks whether two ranges overlap. + Overlap(Range, Range) bool } -var _ = Range{}.Equal +type inclusiveOverlapper struct{} -// Equal returns whether the two ranges are equal. -func (r Range) Equal(other Range) bool { - return r.End.Equal(other.Start) && r.Start.Equal(other.End) +// Overlap checks where a and b overlap in the inclusive way. +func (overlapper inclusiveOverlapper) Overlap(a Range, b Range) bool { + return a.Start.Compare(b.End) <= 0 && b.Start.Compare(a.End) <= 0 } -// String implements the Stringer interface. -func (r Range) String() string { - return fmt.Sprintf("[%x-%x)", r.Start, r.End) +// InclusiveOverlapper defines overlapping as a pair of ranges that share a segment of the keyspace +// in the inclusive way. "inclusive" means that both start and end keys treated as inclusive values. +var InclusiveOverlapper = inclusiveOverlapper{} + +type exclusiveOverlapper struct{} + +// Overlap checks where a and b overlap in the exclusive way. +func (overlapper exclusiveOverlapper) Overlap(a Range, b Range) bool { + return a.Start.Compare(b.End) < 0 && b.Start.Compare(a.End) < 0 } -// An Interface is a type that can be inserted into a Tree. +// ExclusiveOverlapper defines overlapping as a pair of ranges that share a segment of the keyspace +// in the exclusive. "exclusive" means that the start keys are treated as inclusive and the end keys +// are treated as exclusive. +var ExclusiveOverlapper = exclusiveOverlapper{} + +// An Interface is a type that can be inserted into an interval tree. type Interface interface { Range() Range // Returns a unique ID for the element. @@ -84,6 +76,46 @@ type Interface interface { ID() uintptr } +var _ = Compare + +// Compare returns a value indicating the sort order relationship between a and b. The comparison is +// performed lexicographically on (a.Range().Start, a.ID()) and (b.Range().Start, b.ID()) tuples +// where Range().Start is more significant that ID(). +// +// Given c = Compare(a, b): +// +// c == -1 if (a.Range().Start, a.ID()) < (b.Range().Start, b.ID()); +// c == 0 if (a.Range().Start, a.ID()) == (b.Range().Start, b.ID()); and +// c == 1 if (a.Range().Start, a.ID()) > (b.Range().Start, b.ID()). +// +// "c == 0" is equivalent to "Equal(a, b) == true". +func Compare(a, b Interface) int { + startCmp := a.Range().Start.Compare(b.Range().Start) + if startCmp != 0 { + return startCmp + } + aID := a.ID() + bID := b.ID() + if aID < bID { + return -1 + } else if aID > bID { + return 1 + } else { + return 0 + } +} + +var _ = Equal + +// Equal returns a boolean indicating whethter the given Interfaces are equal to each other. If +// "Equal(a, b) == true", "a.Range().End == b.Range().End" must hold. Otherwise, the interval tree +// behavior is undefined. "Equal(a, b) == true" is equivalent to "Compare(a, b) == 0". But the +// former has measurably better performance than the latter. So Equal should be used when only +// equality state is needed. +func Equal(a, b Interface) bool { + return a.Range().Start.Equal(b.Range().Start) && a.ID() == b.ID() +} + // A Comparable is a type that describes the ends of a Range. type Comparable []byte @@ -91,9 +123,9 @@ type Comparable []byte // receiver and the parameter. // // Given c = a.Compare(b): -// c < 0 if a < b; +// c == -1 if a < b; // c == 0 if a == b; and -// c > 0 if a > b. +// c == 1 if a > b. // func (c Comparable) Compare(o Comparable) int { return bytes.Compare(c, o) @@ -108,651 +140,7 @@ func (c Comparable) Equal(o Comparable) bool { return bytes.Equal(c, o) } -// A Node represents a node in a Tree. -type Node struct { - Elem Interface - Range Range - Left, Right *Node - Color llrb.Color -} - -// A Tree manages the root node of an interval tree. Public methods are exposed through this type. -type Tree struct { - Root *Node // root node of the tree. - Count int // number of elements stored. - Overlapper func(Range, Range) bool // determines how to define Range overlap. -} - -// Helper methods - -// color returns the effect color of a Node. A nil node returns black. -func (n *Node) color() llrb.Color { - if n == nil { - return llrb.Black - } - return n.Color -} - -// maxRange returns the furthest right position held by the subtree -// rooted at root, assuming that the left and right nodes have correct -// range extents. -func maxRange(root, left, right *Node) Comparable { - end := root.Elem.Range().End - if left != nil && left.Range.End.Compare(end) > 0 { - end = left.Range.End - } - if right != nil && right.Range.End.Compare(end) > 0 { - end = right.Range.End - } - return end -} - -// (a,c)b -rotL-> ((a,)b,)c -func (n *Node) rotateLeft() (root *Node) { - // Assumes: n has a right child. - root = n.Right - n.Right = root.Left - root.Left = n - root.Color = n.Color - n.Color = llrb.Red - - root.Left.Range.End = maxRange(root.Left, root.Left.Left, root.Left.Right) - if root.Left == nil { - root.Range.Start = root.Elem.Range().Start - } else { - root.Range.Start = root.Left.Range.Start - } - root.Range.End = maxRange(root, root.Left, root.Right) - - return -} - -// (a,c)b -rotR-> (,(,c)b)a -func (n *Node) rotateRight() (root *Node) { - // Assumes: n has a left child. - root = n.Left - n.Left = root.Right - root.Right = n - root.Color = n.Color - n.Color = llrb.Red - - if root.Right.Left == nil { - root.Right.Range.Start = root.Right.Elem.Range().Start - } else { - root.Right.Range.Start = root.Right.Left.Range.Start - } - root.Right.Range.End = maxRange(root.Right, root.Right.Left, root.Right.Right) - root.Range.End = maxRange(root, root.Left, root.Right) - - return -} - -// (aR,cR)bB -flipC-> (aB,cB)bR | (aB,cB)bR -flipC-> (aR,cR)bB -func (n *Node) flipColors() { - // Assumes: n has two children. - n.Color = !n.Color - n.Left.Color = !n.Left.Color - n.Right.Color = !n.Right.Color -} - -// fixUp ensures that black link balance is correct, that red nodes lean left, -// and that 4 nodes are split in the case of BU23 and properly balanced in TD234. -func (n *Node) fixUp(fast bool) *Node { - if !fast { - n.adjustRange() - } - if n.Right.color() == llrb.Red { - if Mode == TD234 && n.Right.Left.color() == llrb.Red { - n.Right = n.Right.rotateRight() - } - n = n.rotateLeft() - } - if n.Left.color() == llrb.Red && n.Left.Left.color() == llrb.Red { - n = n.rotateRight() - } - if Mode == BU23 && n.Left.color() == llrb.Red && n.Right.color() == llrb.Red { - n.flipColors() - } - - return n -} - -// adjustRange sets the Range to the maximum extent of the children's Range -// spans and the node's Elem span. -func (n *Node) adjustRange() { - if n.Left == nil { - n.Range.Start = n.Elem.Range().Start - } else { - n.Range.Start = n.Left.Range.Start - } - n.Range.End = maxRange(n, n.Left, n.Right) -} - -func (n *Node) moveRedLeft() *Node { - n.flipColors() - if n.Right.Left.color() == llrb.Red { - n.Right = n.Right.rotateRight() - n = n.rotateLeft() - n.flipColors() - if Mode == TD234 && n.Right.Right.color() == llrb.Red { - n.Right = n.Right.rotateLeft() - } - } - return n -} - -func (n *Node) moveRedRight() *Node { - n.flipColors() - if n.Left.Left.color() == llrb.Red { - n = n.rotateRight() - n.flipColors() - } - return n -} - -// Len returns the number of intervals stored in the Tree. -func (t *Tree) Len() int { - return t.Count -} - -// Get returns a slice of Interfaces that overlap r in the Tree. -func (t *Tree) Get(r Range) (o []Interface) { - return t.GetWithOverlapper(r, t.Overlapper) -} - -// GetWithOverlapper returns a slice of Interfaces that overlap r -// in the Tree using the provided overlapper function. -func (t *Tree) GetWithOverlapper(r Range, overlapper func(Range, Range) bool) (o []Interface) { - if t.Root != nil && overlapper(r, t.Root.Range) { - t.Root.doMatch(func(e Interface) (done bool) { o = append(o, e); return }, r, overlapper) - } - return -} - -// AdjustRanges fixes range fields for all Nodes in the Tree. This must be called -// before Get or DoMatching* is used if fast insertion or deletion has been performed. -func (t *Tree) AdjustRanges() { - if t.Root == nil { - return - } - t.Root.adjustRanges() -} - -func (n *Node) adjustRanges() { - if n.Left != nil { - n.Left.adjustRanges() - } - if n.Right != nil { - n.Right.adjustRanges() - } - n.adjustRange() -} - -// Insert inserts the Interface e into the Tree. Insertions may replace -// existing stored intervals. -func (t *Tree) Insert(e Interface, fast bool) (err error) { - r := e.Range() - if err := rangeError(r); err != nil { - return err - } - var d int - t.Root, d = t.Root.insert(e, r.Start, e.ID(), fast) - t.Count += d - t.Root.Color = llrb.Black - return -} - -func (n *Node) insert(e Interface, min Comparable, id uintptr, fast bool) (root *Node, d int) { - if n == nil { - return &Node{Elem: e, Range: e.Range()}, 1 - } else if n.Elem == nil { - n.Elem = e - if !fast { - n.adjustRange() - } - return n, 1 - } - - if Mode == TD234 { - if n.Left.color() == llrb.Red && n.Right.color() == llrb.Red { - n.flipColors() - } - } - - switch c := min.Compare(n.Elem.Range().Start); { - case c == 0: - switch eid := n.Elem.ID(); { - case id == eid: - n.Elem = e - if !fast { - n.Range.End = e.Range().End - } - case id < eid: - n.Left, d = n.Left.insert(e, min, id, fast) - default: - n.Right, d = n.Right.insert(e, min, id, fast) - } - case c < 0: - n.Left, d = n.Left.insert(e, min, id, fast) - default: - n.Right, d = n.Right.insert(e, min, id, fast) - } - - if n.Right.color() == llrb.Red && n.Left.color() == llrb.Black { - n = n.rotateLeft() - } - if n.Left.color() == llrb.Red && n.Left.Left.color() == llrb.Red { - n = n.rotateRight() - } - - if Mode == BU23 { - if n.Left.color() == llrb.Red && n.Right.color() == llrb.Red { - n.flipColors() - } - } - - if !fast { - n.adjustRange() - } - root = n - - return -} - -var _ = (*Tree)(nil).DeleteMin - -// DeleteMin deletes the left-most interval. -func (t *Tree) DeleteMin(fast bool) { - if t.Root == nil { - return - } - var d int - t.Root, d = t.Root.deleteMin(fast) - t.Count += d - if t.Root == nil { - return - } - t.Root.Color = llrb.Black -} - -func (n *Node) deleteMin(fast bool) (root *Node, d int) { - if n.Left == nil { - return nil, -1 - } - if n.Left.color() == llrb.Black && n.Left.Left.color() == llrb.Black { - n = n.moveRedLeft() - } - n.Left, d = n.Left.deleteMin(fast) - if n.Left == nil { - n.Range.Start = n.Elem.Range().Start - } - - root = n.fixUp(fast) - - return -} - -var _ = (*Tree)(nil).DeleteMax - -// DeleteMax deletes the right-most interval. -func (t *Tree) DeleteMax(fast bool) { - if t.Root == nil { - return - } - var d int - t.Root, d = t.Root.deleteMax(fast) - t.Count += d - if t.Root == nil { - return - } - t.Root.Color = llrb.Black -} - -func (n *Node) deleteMax(fast bool) (root *Node, d int) { - if n.Left != nil && n.Left.color() == llrb.Red { - n = n.rotateRight() - } - if n.Right == nil { - return nil, -1 - } - if n.Right.color() == llrb.Black && n.Right.Left.color() == llrb.Black { - n = n.moveRedRight() - } - n.Right, d = n.Right.deleteMax(fast) - if n.Right == nil { - n.Range.End = n.Elem.Range().End - } - - root = n.fixUp(fast) - - return -} - -// Delete deletes the element e if it exists in the Tree. -func (t *Tree) Delete(e Interface, fast bool) (err error) { - r := e.Range() - if err := rangeError(r); err != nil { - return err - } - if t.Root == nil || !t.Overlapper(r, t.Root.Range) { - return - } - var d int - t.Root, d = t.Root.delete(r.Start, e.ID(), fast) - t.Count += d - if t.Root == nil { - return - } - t.Root.Color = llrb.Black - return -} - -func (n *Node) delete(min Comparable, id uintptr, fast bool) (root *Node, d int) { - if p := min.Compare(n.Elem.Range().Start); p < 0 || (p == 0 && id < n.Elem.ID()) { - if n.Left != nil { - if n.Left.color() == llrb.Black && n.Left.Left.color() == llrb.Black { - n = n.moveRedLeft() - } - n.Left, d = n.Left.delete(min, id, fast) - if n.Left == nil { - n.Range.Start = n.Elem.Range().Start - } - } - } else { - if n.Left.color() == llrb.Red { - n = n.rotateRight() - } - if n.Right == nil && id == n.Elem.ID() { - return nil, -1 - } - if n.Right != nil { - if n.Right.color() == llrb.Black && n.Right.Left.color() == llrb.Black { - n = n.moveRedRight() - } - if id == n.Elem.ID() { - n.Elem = n.Right.min().Elem - n.Right, d = n.Right.deleteMin(fast) - } else { - n.Right, d = n.Right.delete(min, id, fast) - } - if n.Right == nil { - n.Range.End = n.Elem.Range().End - } - } - } - - root = n.fixUp(fast) - - return -} - -var _ = (*Tree)(nil).Min - -// Min returns the left-most interval stored in the tree. -func (t *Tree) Min() Interface { - if t.Root == nil { - return nil - } - return t.Root.min().Elem -} - -func (n *Node) min() *Node { - for ; n.Left != nil; n = n.Left { - } - return n -} - -var _ = (*Tree)(nil).Max - -// Max returns the right-most interval stored in the tree. -func (t *Tree) Max() Interface { - if t.Root == nil { - return nil - } - return t.Root.max().Elem -} - -func (n *Node) max() *Node { - for ; n.Right != nil; n = n.Right { - } - return n -} - -var _ = (*Tree)(nil).Floor - -// Floor returns the largest value equal to or less than the query q according to -// q.Start.Compare(), with ties broken by comparison of ID() values. -func (t *Tree) Floor(q Interface) (o Interface, err error) { - if t.Root == nil { - return - } - n := t.Root.floor(q.Range().Start, q.ID()) - if n == nil { - return - } - return n.Elem, nil -} - -func (n *Node) floor(m Comparable, id uintptr) *Node { - if n == nil { - return nil - } - switch c := m.Compare(n.Elem.Range().Start); { - case c == 0: - switch eid := n.Elem.ID(); { - case id == eid: - return n - case id < eid: - return n.Left.floor(m, id) - default: - if r := n.Right.floor(m, id); r != nil { - return r - } - } - case c < 0: - return n.Left.floor(m, id) - default: - if r := n.Right.floor(m, id); r != nil { - return r - } - } - return n -} - -var _ = (*Tree)(nil).Ceil - -// Ceil returns the smallest value equal to or greater than the query q according to -// q.Start.Compare(), with ties broken by comparison of ID() values. -func (t *Tree) Ceil(q Interface) (o Interface, err error) { - if t.Root == nil { - return - } - n := t.Root.ceil(q.Range().Start, q.ID()) - if n == nil { - return - } - return n.Elem, nil -} - -func (n *Node) ceil(m Comparable, id uintptr) *Node { - if n == nil { - return nil - } - switch c := m.Compare(n.Elem.Range().Start); { - case c == 0: - switch eid := n.Elem.ID(); { - case id == eid: - return n - case id > eid: - return n.Right.ceil(m, id) - default: - if l := n.Left.ceil(m, id); l != nil { - return l - } - } - case c > 0: - return n.Right.ceil(m, id) - default: - if l := n.Left.ceil(m, id); l != nil { - return l - } - } - return n -} - // An Operation is a function that operates on an Interface. If done is returned true, the -// Operation is indicating that no further work needs to be done and so the Do function should -// traverse no further. +// Operation is indicating that no further work needs to be done and so the DoMatching function +// should traverse no further. type Operation func(Interface) (done bool) - -// Do performs fn on all intervals stored in the tree. A boolean is returned indicating whether the -// Do traversal was interrupted by an Operation returning true. If fn alters stored intervals' sort -// relationships, future tree operation behaviors are undefined. -func (t *Tree) Do(fn Operation) bool { - if t.Root == nil { - return false - } - return t.Root.do(fn) -} - -func (n *Node) do(fn Operation) (done bool) { - if n.Left != nil { - done = n.Left.do(fn) - if done { - return - } - } - done = fn(n.Elem) - if done { - return - } - if n.Right != nil { - done = n.Right.do(fn) - } - return -} - -var _ = (*Tree)(nil).DoReverse - -// DoReverse performs fn on all intervals stored in the tree, but in reverse of sort order. A boolean -// is returned indicating whether the Do traversal was interrupted by an Operation returning true. -// If fn alters stored intervals' sort relationships, future tree operation behaviors are undefined. -func (t *Tree) DoReverse(fn Operation) bool { - if t.Root == nil { - return false - } - return t.Root.doReverse(fn) -} - -func (n *Node) doReverse(fn Operation) (done bool) { - if n.Right != nil { - done = n.Right.doReverse(fn) - if done { - return - } - } - done = fn(n.Elem) - if done { - return - } - if n.Left != nil { - done = n.Left.doReverse(fn) - } - return -} - -// DoMatching performs fn on all intervals stored in the tree that match r according to -// t.Overlapper, with Overlapper() used to guide tree traversal, so DoMatching() will -// outperform Do() with a called conditional function if the condition is based on sort -// order, but can not be reliably used if the condition is independent of sort order. A -// boolean is returned indicating whether the Do traversal was interrupted by an Operation -// returning true. If fn alters stored intervals' sort relationships, future tree operation -// behaviors are undefined. -func (t *Tree) DoMatching(fn Operation, r Range) bool { - if t.Root != nil && t.Overlapper(r, t.Root.Range) { - return t.Root.doMatch(fn, r, t.Overlapper) - } - return false -} - -func (n *Node) doMatch(fn Operation, r Range, overlaps func(Range, Range) bool) (done bool) { - if n.Left != nil && overlaps(r, n.Left.Range) { - done = n.Left.doMatch(fn, r, overlaps) - if done { - return - } - } - if overlaps(r, n.Elem.Range()) { - done = fn(n.Elem) - if done { - return - } - } - if n.Right != nil && overlaps(r, n.Right.Range) { - done = n.Right.doMatch(fn, r, overlaps) - } - return -} - -var _ = (*Tree)(nil).DoMatchingReverse - -// DoMatchingReverse performs fn on all intervals stored in the tree that match r according to -// t.Overlapper, with Overlapper() used to guide tree traversal, so DoMatching() will outperform -// Do() with a called conditional function if the condition is based on sort order, but can not -// be reliably used if the condition is independent of sort order. A boolean is returned indicating -// whether the Do traversal was interrupted by an Operation returning true. If fn alters stored -// intervals' sort relationships, future tree operation behaviors are undefined. -func (t *Tree) DoMatchingReverse(fn Operation, r Range) bool { - if t.Root != nil && t.Overlapper(r, t.Root.Range) { - return t.Root.doMatchReverse(fn, r, t.Overlapper) - } - return false -} - -func (n *Node) doMatchReverse(fn Operation, r Range, overlaps func(Range, Range) bool) (done bool) { - if n.Right != nil && overlaps(r, n.Right.Range) { - done = n.Right.doMatchReverse(fn, r, overlaps) - if done { - return - } - } - if overlaps(r, n.Elem.Range()) { - done = fn(n.Elem) - if done { - return - } - } - if n.Left != nil && overlaps(r, n.Left.Range) { - done = n.Left.doMatchReverse(fn, r, overlaps) - } - return -} - -// TreeIterator iterates over all intervals stored in the Tree, in-order. -type TreeIterator struct { - stack []*Node -} - -// Next moves the iterator to the next Node in the Tree and returns the node's -// Elem. The method returns false if no Nodes remain in the Tree. -func (ti *TreeIterator) Next() (i Interface, ok bool) { - if len(ti.stack) == 0 { - return nil, false - } - n := ti.stack[len(ti.stack)-1] - ti.stack = ti.stack[:len(ti.stack)-1] - for r := n.Right; r != nil; r = r.Left { - ti.stack = append(ti.stack, r) - } - return n.Elem, true -} - -// Iterator creates an iterator to iterate over all intervals stored in the -// tree, in-order. -func (t *Tree) Iterator() TreeIterator { - var ti TreeIterator - for n := t.Root; n != nil; n = n.Left { - ti.stack = append(ti.stack, n) - } - return ti -} diff --git a/pkg/util/interval/llrb_based_interval.go b/pkg/util/interval/llrb_based_interval.go new file mode 100644 index 000000000000..a2a15d3b55f3 --- /dev/null +++ b/pkg/util/interval/llrb_based_interval.go @@ -0,0 +1,679 @@ +// Copyright ©2012 The bíogo Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// +build !interval_btree + +// Package interval implements an interval tree based on an augmented +// Left-Leaning Red Black tree. +package interval + +import ( + "github.com/biogo/store/llrb" +) + +// Operation mode of the underlying LLRB tree. +const ( + TD234 = iota + BU23 +) + +func init() { + if Mode != TD234 && Mode != BU23 { + panic("interval: unknown mode") + } +} + +// A Node represents a node in a tree. +type Node struct { + Elem Interface + Range Range + Left, Right *Node + Color llrb.Color +} + +// Tree manages the root node of an interval tree. Public methods are exposed through this type. +type Tree struct { + Root *Node // root node of the tree. + Count int // number of elements stored. + Overlapper Overlapper +} + +// NewTree creates a new interval tree with the given overlapper function. +func NewTree(overlapper Overlapper) Tree { + return Tree{Overlapper: overlapper} +} + +// Helper methods + +// color returns the effect color of a Node. A nil node returns black. +func (n *Node) color() llrb.Color { + if n == nil { + return llrb.Black + } + return n.Color +} + +// maxRange returns the furthest right position held by the subtree +// rooted at root, assuming that the left and right nodes have correct +// range extents. +func maxRange(root, left, right *Node) Comparable { + end := root.Elem.Range().End + if left != nil && left.Range.End.Compare(end) > 0 { + end = left.Range.End + } + if right != nil && right.Range.End.Compare(end) > 0 { + end = right.Range.End + } + return end +} + +// (a,c)b -rotL-> ((a,)b,)c +func (n *Node) rotateLeft() (root *Node) { + // Assumes: n has a right child. + root = n.Right + n.Right = root.Left + root.Left = n + root.Color = n.Color + n.Color = llrb.Red + + root.Left.Range.End = maxRange(root.Left, root.Left.Left, root.Left.Right) + if root.Left == nil { + root.Range.Start = root.Elem.Range().Start + } else { + root.Range.Start = root.Left.Range.Start + } + root.Range.End = maxRange(root, root.Left, root.Right) + + return +} + +// (a,c)b -rotR-> (,(,c)b)a +func (n *Node) rotateRight() (root *Node) { + // Assumes: n has a left child. + root = n.Left + n.Left = root.Right + root.Right = n + root.Color = n.Color + n.Color = llrb.Red + + if root.Right.Left == nil { + root.Right.Range.Start = root.Right.Elem.Range().Start + } else { + root.Right.Range.Start = root.Right.Left.Range.Start + } + root.Right.Range.End = maxRange(root.Right, root.Right.Left, root.Right.Right) + root.Range.End = maxRange(root, root.Left, root.Right) + + return +} + +// (aR,cR)bB -flipC-> (aB,cB)bR | (aB,cB)bR -flipC-> (aR,cR)bB +func (n *Node) flipColors() { + // Assumes: n has two children. + n.Color = !n.Color + n.Left.Color = !n.Left.Color + n.Right.Color = !n.Right.Color +} + +// fixUp ensures that black link balance is correct, that red nodes lean left, +// and that 4 nodes are split in the case of BU23 and properly balanced in TD234. +func (n *Node) fixUp(fast bool) *Node { + if !fast { + n.adjustRange() + } + if n.Right.color() == llrb.Red { + if Mode == TD234 && n.Right.Left.color() == llrb.Red { + n.Right = n.Right.rotateRight() + } + n = n.rotateLeft() + } + if n.Left.color() == llrb.Red && n.Left.Left.color() == llrb.Red { + n = n.rotateRight() + } + if Mode == BU23 && n.Left.color() == llrb.Red && n.Right.color() == llrb.Red { + n.flipColors() + } + + return n +} + +// adjustRange sets the Range to the maximum extent of the children's Range +// spans and the node's Elem span. +func (n *Node) adjustRange() { + if n.Left == nil { + n.Range.Start = n.Elem.Range().Start + } else { + n.Range.Start = n.Left.Range.Start + } + n.Range.End = maxRange(n, n.Left, n.Right) +} + +func (n *Node) moveRedLeft() *Node { + n.flipColors() + if n.Right.Left.color() == llrb.Red { + n.Right = n.Right.rotateRight() + n = n.rotateLeft() + n.flipColors() + if Mode == TD234 && n.Right.Right.color() == llrb.Red { + n.Right = n.Right.rotateLeft() + } + } + return n +} + +func (n *Node) moveRedRight() *Node { + n.flipColors() + if n.Left.Left.color() == llrb.Red { + n = n.rotateRight() + n.flipColors() + } + return n +} + +// Len returns the number of intervals stored in the Tree. +func (t *Tree) Len() int { + return t.Count +} + +// Get returns a slice of Interfaces that overlap r in the Tree. +func (t *Tree) Get(r Range) (o []Interface) { + return t.GetWithOverlapper(r, t.Overlapper) +} + +// GetWithOverlapper returns a slice of Interfaces that overlap r in the Tree +// using the provided overlapper function. +func (t *Tree) GetWithOverlapper(r Range, overlapper Overlapper) (o []Interface) { + if t.Root != nil && overlapper.Overlap(r, t.Root.Range) { + t.Root.doMatch(func(e Interface) (done bool) { o = append(o, e); return }, r, overlapper.Overlap) + } + return +} + +// AdjustRanges fixes range fields for all Nodes in the Tree. This must be +// called before Get or DoMatching* is used if fast insertion or deletion has +// been performed. +func (t *Tree) AdjustRanges() { + if t.Root == nil { + return + } + t.Root.adjustRanges() +} + +func (n *Node) adjustRanges() { + if n.Left != nil { + n.Left.adjustRanges() + } + if n.Right != nil { + n.Right.adjustRanges() + } + n.adjustRange() +} + +// Insert inserts the Interface e into the Tree. Insertions may replace existing +// stored intervals. +func (t *Tree) Insert(e Interface, fast bool) (err error) { + r := e.Range() + if err := rangeError(r); err != nil { + return err + } + var d int + t.Root, d = t.Root.insert(e, r.Start, e.ID(), fast) + t.Count += d + t.Root.Color = llrb.Black + return +} + +func (n *Node) insert(e Interface, min Comparable, id uintptr, fast bool) (root *Node, d int) { + if n == nil { + return &Node{Elem: e, Range: e.Range()}, 1 + } else if n.Elem == nil { + n.Elem = e + if !fast { + n.adjustRange() + } + return n, 1 + } + + if Mode == TD234 { + if n.Left.color() == llrb.Red && n.Right.color() == llrb.Red { + n.flipColors() + } + } + + switch c := min.Compare(n.Elem.Range().Start); { + case c == 0: + switch eid := n.Elem.ID(); { + case id == eid: + n.Elem = e + if !fast { + n.Range.End = e.Range().End + } + case id < eid: + n.Left, d = n.Left.insert(e, min, id, fast) + default: + n.Right, d = n.Right.insert(e, min, id, fast) + } + case c < 0: + n.Left, d = n.Left.insert(e, min, id, fast) + default: + n.Right, d = n.Right.insert(e, min, id, fast) + } + + if n.Right.color() == llrb.Red && n.Left.color() == llrb.Black { + n = n.rotateLeft() + } + if n.Left.color() == llrb.Red && n.Left.Left.color() == llrb.Red { + n = n.rotateRight() + } + + if Mode == BU23 { + if n.Left.color() == llrb.Red && n.Right.color() == llrb.Red { + n.flipColors() + } + } + + if !fast { + n.adjustRange() + } + root = n + + return +} + +var _ = (*Tree)(nil).DeleteMin + +// DeleteMin deletes the leftmost interval. +func (t *Tree) DeleteMin(fast bool) { + if t.Root == nil { + return + } + var d int + t.Root, d = t.Root.deleteMin(fast) + t.Count += d + if t.Root == nil { + return + } + t.Root.Color = llrb.Black +} + +func (n *Node) deleteMin(fast bool) (root *Node, d int) { + if n.Left == nil { + return nil, -1 + } + if n.Left.color() == llrb.Black && n.Left.Left.color() == llrb.Black { + n = n.moveRedLeft() + } + n.Left, d = n.Left.deleteMin(fast) + if n.Left == nil { + n.Range.Start = n.Elem.Range().Start + } + + root = n.fixUp(fast) + + return +} + +var _ = (*Tree)(nil).DeleteMax + +// DeleteMax deletes the rightmost interval. +func (t *Tree) DeleteMax(fast bool) { + if t.Root == nil { + return + } + var d int + t.Root, d = t.Root.deleteMax(fast) + t.Count += d + if t.Root == nil { + return + } + t.Root.Color = llrb.Black +} + +func (n *Node) deleteMax(fast bool) (root *Node, d int) { + if n.Left != nil && n.Left.color() == llrb.Red { + n = n.rotateRight() + } + if n.Right == nil { + return nil, -1 + } + if n.Right.color() == llrb.Black && n.Right.Left.color() == llrb.Black { + n = n.moveRedRight() + } + n.Right, d = n.Right.deleteMax(fast) + if n.Right == nil { + n.Range.End = n.Elem.Range().End + } + + root = n.fixUp(fast) + + return +} + +// Delete deletes the element e if it exists in the Tree. +func (t *Tree) Delete(e Interface, fast bool) (err error) { + r := e.Range() + if err := rangeError(r); err != nil { + return err + } + if t.Root == nil || !t.Overlapper.Overlap(r, t.Root.Range) { + return + } + var d int + t.Root, d = t.Root.delete(r.Start, e.ID(), fast) + t.Count += d + if t.Root == nil { + return + } + t.Root.Color = llrb.Black + return +} + +func (n *Node) delete(min Comparable, id uintptr, fast bool) (root *Node, d int) { + if p := min.Compare(n.Elem.Range().Start); p < 0 || (p == 0 && id < n.Elem.ID()) { + if n.Left != nil { + if n.Left.color() == llrb.Black && n.Left.Left.color() == llrb.Black { + n = n.moveRedLeft() + } + n.Left, d = n.Left.delete(min, id, fast) + if n.Left == nil { + n.Range.Start = n.Elem.Range().Start + } + } + } else { + if n.Left.color() == llrb.Red { + n = n.rotateRight() + } + if n.Right == nil && id == n.Elem.ID() { + return nil, -1 + } + if n.Right != nil { + if n.Right.color() == llrb.Black && n.Right.Left.color() == llrb.Black { + n = n.moveRedRight() + } + if id == n.Elem.ID() { + n.Elem = n.Right.min().Elem + n.Right, d = n.Right.deleteMin(fast) + } else { + n.Right, d = n.Right.delete(min, id, fast) + } + if n.Right == nil { + n.Range.End = n.Elem.Range().End + } + } + } + + root = n.fixUp(fast) + + return +} + +var _ = (*Tree)(nil).Min + +// Min returns the leftmost interval stored in the tree. +func (t *Tree) Min() Interface { + if t.Root == nil { + return nil + } + return t.Root.min().Elem +} + +func (n *Node) min() *Node { + for ; n.Left != nil; n = n.Left { + } + return n +} + +var _ = (*Tree)(nil).Max + +// Max returns the rightmost interval stored in the tree. +func (t *Tree) Max() Interface { + if t.Root == nil { + return nil + } + return t.Root.max().Elem +} + +func (n *Node) max() *Node { + for ; n.Right != nil; n = n.Right { + } + return n +} + +var _ = (*Tree)(nil).Floor + +// Floor returns the largest value equal to or less than the query q according to +// q.Start.Compare(), with ties broken by comparison of ID() values. +func (t *Tree) Floor(q Interface) (o Interface, err error) { + if t.Root == nil { + return + } + n := t.Root.floor(q.Range().Start, q.ID()) + if n == nil { + return + } + return n.Elem, nil +} + +func (n *Node) floor(m Comparable, id uintptr) *Node { + if n == nil { + return nil + } + switch c := m.Compare(n.Elem.Range().Start); { + case c == 0: + switch eid := n.Elem.ID(); { + case id == eid: + return n + case id < eid: + return n.Left.floor(m, id) + default: + if r := n.Right.floor(m, id); r != nil { + return r + } + } + case c < 0: + return n.Left.floor(m, id) + default: + if r := n.Right.floor(m, id); r != nil { + return r + } + } + return n +} + +var _ = (*Tree)(nil).Ceil + +// Ceil returns the smallest value equal to or greater than the query q according to +// q.Start.Compare(), with ties broken by comparison of ID() values. +func (t *Tree) Ceil(q Interface) (o Interface, err error) { + if t.Root == nil { + return + } + n := t.Root.ceil(q.Range().Start, q.ID()) + if n == nil { + return + } + return n.Elem, nil +} + +func (n *Node) ceil(m Comparable, id uintptr) *Node { + if n == nil { + return nil + } + switch c := m.Compare(n.Elem.Range().Start); { + case c == 0: + switch eid := n.Elem.ID(); { + case id == eid: + return n + case id > eid: + return n.Right.ceil(m, id) + default: + if l := n.Left.ceil(m, id); l != nil { + return l + } + } + case c > 0: + return n.Right.ceil(m, id) + default: + if l := n.Left.ceil(m, id); l != nil { + return l + } + } + return n +} + +// Do performs fn on all intervals stored in the tree. A boolean is returned +// indicating whether the Do traversal was interrupted by an Operation returning +// true. If fn alters stored intervals' sort relationships, future tree +// operation behaviors are undefined. +func (t *Tree) Do(fn Operation) bool { + if t.Root == nil { + return false + } + return t.Root.do(fn) +} + +func (n *Node) do(fn Operation) (done bool) { + if n.Left != nil { + done = n.Left.do(fn) + if done { + return + } + } + done = fn(n.Elem) + if done { + return + } + if n.Right != nil { + done = n.Right.do(fn) + } + return +} + +var _ = (*Tree)(nil).DoReverse + +// DoReverse performs fn on all intervals stored in the tree, but in reverse of sort order. A boolean +// is returned indicating whether the Do traversal was interrupted by an Operation returning true. +// If fn alters stored intervals' sort relationships, future tree operation behaviors are undefined. +func (t *Tree) DoReverse(fn Operation) bool { + if t.Root == nil { + return false + } + return t.Root.doReverse(fn) +} + +func (n *Node) doReverse(fn Operation) (done bool) { + if n.Right != nil { + done = n.Right.doReverse(fn) + if done { + return + } + } + done = fn(n.Elem) + if done { + return + } + if n.Left != nil { + done = n.Left.doReverse(fn) + } + return +} + +var _ = (*Tree)(nil).DoMatchingReverse + +// DoMatching performs fn on all intervals stored in the tree that match r +// according to t.Overlapper, with Overlapper() used to guide tree traversal, so +// DoMatching() will outperform Do() with a called conditional function if the +// condition is based on sort order, but can not be reliably used if the +// condition is independent of sort order. A boolean is returned indicating +// whether the Do traversal was interrupted by an Operation returning true. If +// fn alters stored intervals' sort relationships, future tree operation +// behaviors are undefined. +func (t *Tree) DoMatching(fn Operation, r Range) bool { + if t.Root != nil && t.Overlapper.Overlap(r, t.Root.Range) { + return t.Root.doMatch(fn, r, t.Overlapper.Overlap) + } + return false +} + +func (n *Node) doMatch(fn Operation, r Range, overlaps func(Range, Range) bool) (done bool) { + if n.Left != nil && overlaps(r, n.Left.Range) { + done = n.Left.doMatch(fn, r, overlaps) + if done { + return + } + } + if overlaps(r, n.Elem.Range()) { + done = fn(n.Elem) + if done { + return + } + } + if n.Right != nil && overlaps(r, n.Right.Range) { + done = n.Right.doMatch(fn, r, overlaps) + } + return +} + +var _ = (*Tree)(nil).DoMatchingReverse + +// DoMatchingReverse performs fn on all intervals stored in the tree that match r according to +// t.Overlapper, with Overlapper() used to guide tree traversal, so DoMatching() will outperform +// Do() with a called conditional function if the condition is based on sort order, but can not +// be reliably used if the condition is independent of sort order. A boolean is returned indicating +// whether the Do traversal was interrupted by an Operation returning true. If fn alters stored +// intervals' sort relationships, future tree operation behaviors are undefined. +func (t *Tree) DoMatchingReverse(fn Operation, r Range) bool { + if t.Root != nil && t.Overlapper.Overlap(r, t.Root.Range) { + return t.Root.doMatchReverse(fn, r, t.Overlapper.Overlap) + } + return false +} + +func (n *Node) doMatchReverse(fn Operation, r Range, overlaps func(Range, Range) bool) (done bool) { + if n.Right != nil && overlaps(r, n.Right.Range) { + done = n.Right.doMatchReverse(fn, r, overlaps) + if done { + return + } + } + if overlaps(r, n.Elem.Range()) { + done = fn(n.Elem) + if done { + return + } + } + if n.Left != nil && overlaps(r, n.Left.Range) { + done = n.Left.doMatchReverse(fn, r, overlaps) + } + return +} + +// TreeIterator iterates over all intervals stored in the Tree, in-order. +type TreeIterator struct { + stack []*Node +} + +// Next moves the iterator to the next Node in the Tree and returns the node's +// Elem. The method returns false if no Nodes remain in the Tree. +func (ti *TreeIterator) Next() (i Interface, ok bool) { + if len(ti.stack) == 0 { + return nil, false + } + n := ti.stack[len(ti.stack)-1] + ti.stack = ti.stack[:len(ti.stack)-1] + for r := n.Right; r != nil; r = r.Left { + ti.stack = append(ti.stack, r) + } + return n.Elem, true +} + +// Iterator creates an iterator to iterate over all intervals stored in the +// tree, in-order. +func (t *Tree) Iterator() TreeIterator { + var ti TreeIterator + for n := t.Root; n != nil; n = n.Left { + ti.stack = append(ti.stack, n) + } + return ti +} diff --git a/pkg/util/interval/range_group.go b/pkg/util/interval/range_group.go index be28fadd5486..37e0624aef42 100644 --- a/pkg/util/interval/range_group.go +++ b/pkg/util/interval/range_group.go @@ -113,7 +113,7 @@ func (rl *rangeList) Add(r Range) bool { for e := rl.ll.Front(); e != nil; e = e.Next() { er := e.Value.(Range) switch { - case er.OverlapInclusive(r): + case InclusiveOverlapper.Overlap(er, r): // If a current range fully contains the new range, no // need to add it. if contains(er, r) { @@ -124,7 +124,7 @@ func (rl *rangeList) Add(r Range) bool { newR := merge(er, r) for p := e.Next(); p != nil; { pr := p.Value.(Range) - if newR.OverlapInclusive(pr) { + if InclusiveOverlapper.Overlap(newR, pr) { newR = merge(newR, pr) nextP := p.Next() @@ -161,7 +161,7 @@ func (rl *rangeList) Sub(r Range) bool { for e := rl.ll.Front(); e != nil; { er := e.Value.(Range) switch { - case er.OverlapExclusive(r): + case ExclusiveOverlapper.Overlap(er, r): sCmp := er.Start.Compare(r.Start) eCmp := er.End.Compare(r.End) @@ -225,7 +225,7 @@ func (rl *rangeList) Overlaps(r Range) bool { for e := rl.ll.Front(); e != nil; e = e.Next() { er := e.Value.(Range) switch { - case er.OverlapExclusive(r): + case ExclusiveOverlapper.Overlap(er, r): return true case r.End.Compare(er.Start) <= 0: // Past where exclusive overlapping ranges would be. @@ -310,7 +310,7 @@ type rangeTree struct { // NewRangeTree constructs an interval tree backed RangeGroup. func NewRangeTree() RangeGroup { return &rangeTree{ - t: Tree{Overlapper: Range.OverlapInclusive}, + t: NewTree(InclusiveOverlapper), } } @@ -398,7 +398,7 @@ func (rt *rangeTree) Sub(r Range) bool { if err := rangeError(r); err != nil { panic(err) } - overlaps := rt.t.GetWithOverlapper(r, Range.OverlapExclusive) + overlaps := rt.t.GetWithOverlapper(r, ExclusiveOverlapper) if len(overlaps) == 0 { return false } @@ -441,7 +441,7 @@ func (rt *rangeTree) Sub(r Range) bool { // Clear implements RangeGroup. It clears all rangeKeys from the rangeTree. func (rt *rangeTree) Clear() { - rt.t = Tree{Overlapper: Range.OverlapInclusive} + rt.t = NewTree(InclusiveOverlapper) } // Overlaps implements RangeGroup. It returns whether the provided @@ -450,7 +450,7 @@ func (rt *rangeTree) Overlaps(r Range) bool { if err := rangeError(r); err != nil { panic(err) } - overlaps := rt.t.GetWithOverlapper(r, Range.OverlapExclusive) + overlaps := rt.t.GetWithOverlapper(r, ExclusiveOverlapper) return len(overlaps) > 0 } @@ -460,7 +460,7 @@ func (rt *rangeTree) Encloses(r Range) bool { if err := rangeError(r); err != nil { panic(err) } - overlaps := rt.t.GetWithOverlapper(r, Range.OverlapExclusive) + overlaps := rt.t.GetWithOverlapper(r, ExclusiveOverlapper) if len(overlaps) != 1 { return false } @@ -561,7 +561,7 @@ func RangeGroupsOverlap(rg1, rg2 RangeGroup) bool { } for { // Check if the current pair of Ranges overlap. - if r1.OverlapExclusive(r2) { + if ExclusiveOverlapper.Overlap(r1, r2) { return true } diff --git a/pkg/util/interval/range_group_test.go b/pkg/util/interval/range_group_test.go index 417807757e93..bf6c5bae25ea 100644 --- a/pkg/util/interval/range_group_test.go +++ b/pkg/util/interval/range_group_test.go @@ -697,14 +697,14 @@ func testRangeGroupStringer(t *testing.T, rg RangeGroup) { }, { rngs: []Range{{Start: []byte{0x01}, End: []byte{0x05}}}, - str: "[[01-05)]", + str: "[{01-05}]", }, { rngs: []Range{ {Start: []byte{0x01}, End: []byte{0x05}}, {Start: []byte{0x09}, End: []byte{0xff}}, }, - str: "[[01-05) [09-ff)]", + str: "[{01-05} {09-ff}]", }, { rngs: []Range{ @@ -712,7 +712,7 @@ func testRangeGroupStringer(t *testing.T, rg RangeGroup) { {Start: []byte{0x09}, End: []byte{0xf0}}, {Start: []byte{0xf1}, End: []byte{0xff}}, }, - str: "[[01-05) [09-f0) [f1-ff)]", + str: "[{01-05} {09-f0} {f1-ff}]", }, } diff --git a/pkg/util/interval/td234.go b/pkg/util/interval/td234.go index a54a718cca86..163fb799cdb9 100644 --- a/pkg/util/interval/td234.go +++ b/pkg/util/interval/td234.go @@ -2,7 +2,7 @@ // Use of this source code is governed by a BSD-style // license that can be found in the LICENSE file. -// +build td234 +// +build td234,!interval_btree package interval