Commit 2ab4be93 authored by Kirill Smelkov's avatar Kirill Smelkov

wcfs: xbtree: ΔBtail

ΔBtail provides BTree-level history tail that WCFS - via ΔFtail - will
use to compute which blocks of a ZBigFile need to be invalidated in OS
file cache given raw ZODB changes on ZODB invalidation message.

It also will be used by WCFS to implement isolation protocol, where on
every FUSE READ request WCFS will query ΔBtail - again via ΔFtail - to
find out revision of corresponding file block.

Quoting ΔBtail documentation:

---- 8< ----

ΔBtail provides BTree-level history tail.

It translates ZODB object-level changes to information about which keys of
which BTree were modified, and provides service to query that information.

ΔBtail class documentation
~~~~~~~~~~~~~~~~~~~~~~~~~~

ΔBtail represents tail of revisional changes to BTrees.

It semantically consists of

    []δB			; rev ∈ (tail, head]

where δB represents a change in BTrees space

    δB:
    	.rev↑
    	{} root -> {}(key, δvalue)

It covers only changes to keys from tracked subset of BTrees parts.
In particular a key that was not explicitly requested to be tracked, even if
it was changed in δZ, is not guaranteed to be present in δB.

ΔBtail provides the following operations:

  .Track(path)	- start tracking tree nodes and keys; root=path[0], keys=path[-1].(lo,hi]

  .Update(δZ) -> δB				- update BTree δ tail given raw ZODB changes
  .ForgetPast(revCut)			- forget changes ≤ revCut
  .SliceByRev(lo, hi) -> []δB		- query for all trees changes with rev ∈ (lo, hi]
  .SliceByRootRev(root, lo, hi) -> []δT	- query for changes of a tree with rev ∈ (lo, hi]
  .GetAt(root, key, at) -> (value, rev)	- get root[key] @at assuming root[key] ∈ tracked

where δT represents a change to one tree

    δT:
    	.rev↑
    	{}(key, δvalue)

An example for tracked set is a set of visited BTree paths.
There is no requirement that tracked set belongs to only one single BTree.

See also zodb.ΔTail and zdata.ΔFtail

Concurrency

ΔBtail is safe to use in single-writer / multiple-readers mode. That is at
any time there should be either only sole writer, or, potentially several
simultaneous readers. The table below classifies operations:

    Writers:  Update, ForgetPast
    Readers:  Track + all queries (SliceByRev, SliceByRootRev, GetAt)

Note that, in particular, it is correct to run multiple Track and queries
requests simultaneously.

ΔBtail organization
~~~~~~~~~~~~~~~~~~~

ΔBtail keeps raw ZODB history in ΔZtail and uses BTree-diff algorithm(*) to
turn δZ into BTree-level diff. For each tracked BTree a separate ΔTtail is
maintained with tree-level history in ΔTtail.vδT .

Because it is very computationally expensive(+) to find out for an object to
which BTree it belongs, ΔBtail cannot provide full BTree-level history given
just ΔZtail with δZ changes. Due to this ΔBtail requires help from
users, which are expected to call ΔBtail.Track(treepath) to let ΔBtail know
that such and such ZODB objects constitute a path from root of a tree to some
of its leaf. After Track call the objects from the path and tree keys, that
are covered by leaf node, become tracked: from now-on ΔBtail will detect
and provide BTree-level changes caused by any change of tracked tree objects
or tracked keys. This guarantee can be provided because ΔBtail now knows
that such and such objects belong to a particular tree.

To manage knowledge which tree part is tracked ΔBtail uses PPTreeSubSet.
This data-structure represents so-called PP-connected set of tree nodes:
simply speaking it builds on some leafs and then includes parent(leaf),
parent(parent(leaf)), etc. In other words it's a "parent"-closure of the
leafs. The property of being PP-connected means that starting from any node
from such set, it is always possible to reach root node by traversing
.parent links, and that every intermediate node went-through during
traversal also belongs to the set.

A new Track request potentially grows tracked keys coverage. Due to this,
on a query, ΔBtail needs to recompute potentially whole vδT of the affected
tree. This recomputation is managed by "vδTSnapForTracked*" and "_rebuild"
functions and uses the same treediff algorithm, that Update is using, but
modulo PPTreeSubSet corresponding to δ key coverage. Update also potentially
needs to rebuild whole vδT history, not only append new δT, because a
change to tracked tree nodes can result in growth of tracked key coverage.

Queries are relatively straightforward code that work on vδT snapshot. The
main complexity, besides BTree-diff algorithm, lies in recomputing vδT when
set of tracked keys changes, and in handling that recomputation in such a way
that multiple Track and queries requests could be all served in parallel.

Concurrency

In order to allow multiple Track and queries requests to be served in
parallel ΔBtail employs special organization of vδT rebuild process where
complexity of concurrency is reduced to math on merging updates to vδT and
trackSet, and on key range lookup:

1. vδT is managed under read-copy-update (RCU) discipline: before making
   any vδT change the mutator atomically clones whole vδT and applies its
   change to the clone. This way a query, once it retrieves vδT snapshot,
   does not need to further synchronize with vδT mutators, and can rely on
   that retrieved vδT snapshot will remain immutable.

2. a Track request goes through 3 states: "new", "handle-in-progress" and
   "handled". At each state keys/nodes of the Track are maintained in:

   - ΔTtail.ktrackNew and .trackNew       for "new",
   - ΔTtail.krebuildJobs                  for "handle-in-progress", and
   - ΔBtail.trackSet                      for "handled".

   trackSet keeps nodes, and implicitly keys, from all handled Track
   requests. For all keys, covered by trackSet, vδT is fully computed.

   a new Track(keycov, path) is remembered in ktrackNew and trackNew to be
   further processed when a query should need keys from keycov. vδT is not
   yet providing data for keycov keys.

   when a Track request starts to be processed, its keys and nodes are moved
   from ktrackNew/trackNew into krebuildJobs. vδT is not yet providing data
   for requested-to-be-tracked keys.

   all trackSet, trackNew/ktrackNew and krebuildJobs are completely disjoint:

    trackSet ^ trackNew     = ø
    trackSet ^ krebuildJobs = ø
    trackNew ^ krebuildJobs = ø

3. when a query is served, it needs to retrieve vδT snapshot that takes
   related previous Track requests into account. Retrieving such snapshots
   is implemented in vδTSnapForTracked*() family of functions: there it
   checks ktrackNew/trackNew, and if those sets overlap with query's keys
   of interest, run vδT rebuild for keys queued in ktrackNew.

   the main part of that rebuild can be run without any locks, because it
   does not use nor modify any ΔBtail data, and for δ(vδT) it just computes
   a fresh full vδT build modulo retrieved ktrackNew. Only after that
   computation is complete, ΔBtail is locked again to quickly merge in
   δ(vδT) update back into vδT.

   This organization is based on the fact that

    vδT/(T₁∪T₂) = vδT/T₁ | vδT/T₂

     ( i.e. vδT computed for tracked set being union of T₁ and T₂ is the
       same as merge of vδT computed for tracked set T₁ and vδT computed
      for tracked set T₂ )

   and that

    trackSet | (δPP₁|δPP₂) = (trackSet|δPP₁) | (trackSet|δPP₂)

    ( i.e. tracking set updated for union of δPP₁ and δPP₂ is the same
      as union of tracking set updated with δPP₁ and tracking set updated
      with δPP₂ )

   these merge properties allow to run computation for δ(vδT) and δ(trackSet)
   independently and with ΔBtail unlocked, which in turn enables running
   several Track/queries in parallel.

4. while vδT rebuild is being run, krebuildJobs keeps corresponding keycov
   entry to indicate in-progress rebuild. Should a query need vδT for keys
   from that job, it first waits for corresponding job(s) to complete.

Explained rebuild organization allows non-overlapping queries/track-requests
to run simultaneously. (This property is essential to WCFS because otherwise
WCFS would not be able to serve several non-overlapping READ requests to one
file in parallel.)

--------

(*) implemented in treediff.go
(+) full database scan

~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~

Some preliminary history:

877e64a9    X wcfs: Fix tests to pass again
c32055fc    X wcfs/xbtree: ΔBtail tests += ø -> Tree; Tree -> ø
78f2f88b    X wcfs/xbtree: Fix treediff(a, ø)
5324547c    X wcfs/xbtree: root(a) must stay in trackSet even after treediff(a,ø)
f65f775b    X wcfs/xbtree: treediff(ø, b)
c75b1c6f    X wcfs/xbtree: Start killing holeIdx
0fa06cbd    X kadj must be taken into account as kadj^δZ
ef5e5183    X treediff ret += δtkeycov
f30826a6    X another bug in δtkeyconv computation
0917380e    X wcfs: assert that keycov only grow
502e05c2    X found why TestΔBTailAllStructs was not effective to find δtkeycov bugs
450ba707    X Fix rebuild with ø @at2
f60528c9    X ΔBtail.Clone had bug that it was aliasing klon and orig data
9d20f8e8    X treediff: Fix BUG while computing AB coverage
ddb28043    X rebuild: Don't return nil for empty ΔPPTreeSubSet - that leads to SIGSEGV
324241eb    X rebuild: tests: Don't reflect.DeepEqual in inner loop
8f6e2b1e    X rebuild: tests: Don't access ZODB in XGetδKV
2c0b4793    X rebuild: tests: Don't access ZODB in xtrackKeys
8f0e37f2    X rebuild: tests: Precompute kadj10·kadj21
271d953d    X rebuild: tests: Move ΔBtail.Clone test out of hot inner loop into separate test
a87cc6de    X rebuild: tests: Don't recompute trackSet(keys1R2) several times
01433e96    X rebuild: tests: Don't compute keyCover in trackSet
7371f9c5    X rebuild: tests: Inline _assertTrack
3e9164b3    X rebuild: tests: Don't exercise keys from keys2 that already became tracked after Track(keys1) + Update
e9c4b619    X rebuild: tests: Random testing
d0fe680a    X δbtail += ForgetPast
210e9b07    X Fix ΔBtail.SliceByRootRev (lo,hi] handling
855ab4b8    X ΔBtail: Goodbye .KVAtTail
2f5582e6    X ΔBtail: Tweak tests to run faster in normal mode
cf352737    X random testing found another failing test for rebuild...
7f7e34e0    X wcfs/xbtree: Fix update not to add duplicate extra point if rebuild  - called by Update - already added it
6ad0052c    X ΔBtail.Track: No need to return error
aafcacdf    X xbtree: GetAt test
784a6761    X xbtree: Fix KAdj definition after treediff was reworked this summer to base decisions on node keycoverage instead of particular node keys
0bb1c22e    X xbtree: Verify that ForgetPast clones vδT on trim
a8945cbf    X Start reworking rebuild routines not to modify data inplace
b74dda09    X Start switching Track from Track(key) to Track(keycov)
dea85e87    X Switch GetAt to vδTSnapForTrackedKey
aa0288ce    X Switch SliceByRootRev to vδTSnapForTracked
c4366b14    X xbtree: tests: Also verify state of ΔTtail.ktrackNew
b98706ad    X Track should be nop if keycov/path is already in krebuildJobs
e141848a    X test.go  ↑ timeout  10m -> 20m
423f77be    X wcfs: Goodby holeIdx
37c2e806    X wcfs: Teach treediff to compute not only δtrack (set of nodes), but also δ for track-key coverage
52c72dbb    X ΔBtail.rebuild started to work draftly
c9f13fc7    X Get rebuild tests to run in a sane time; Add proper random-based testing for rebuild
c7f1e3c9    X xbtree: Factor testing infrastructure bits into xbtree/xbtreetest
7602c1f4    ΔBtail concurrency
parent 80153aa5
...@@ -186,7 +186,9 @@ test.py.drd: bigfile/_bigfile.so wcfs/wcfs ...@@ -186,7 +186,9 @@ test.py.drd: bigfile/_bigfile.so wcfs/wcfs
# run go tests # run go tests
test.go : test.go :
cd wcfs && $(GO) test -count=1 ./... # -count=1 disables tests caching # -timeout=20m: xbtree tests are computationally expensive, and
# sometime cross 10 minutes border when testnodes are otherwise loaded.
cd wcfs && $(GO) test -timeout=20m -count=1 ./... # -count=1 disables tests caching
# test pagefault for double/real faults - it should crash # test pagefault for double/real faults - it should crash
......
...@@ -20,7 +20,7 @@ ...@@ -20,7 +20,7 @@
"""Package xbtree provides utilities for inspecting/manipulating internal """Package xbtree provides utilities for inspecting/manipulating internal
structure of integer-keyed BTrees. structure of integer-keyed BTrees.
It will be primarily used to help verify ΔBTail in WCFS. It is primarily used to help verify ΔBTail in WCFS.
- `Tree` represents a tree node. - `Tree` represents a tree node.
- `Bucket` represents a bucket node. - `Bucket` represents a bucket node.
......
...@@ -56,7 +56,9 @@ const KeyMin = blib.KeyMin ...@@ -56,7 +56,9 @@ const KeyMin = blib.KeyMin
type Value = zodb.Oid type Value = zodb.Oid
const VDEL = zodb.InvalidOid const VDEL = zodb.InvalidOid
type setKey = set.I64
type setOid = set.Oid type setOid = set.Oid
type setTid = set.Tid
// pathEqual returns whether two paths are the same. // pathEqual returns whether two paths are the same.
...@@ -99,6 +101,10 @@ func zgetNodeOrNil(ctx context.Context, zconn *zodb.Connection, oid zodb.Oid) (n ...@@ -99,6 +101,10 @@ func zgetNodeOrNil(ctx context.Context, zconn *zodb.Connection, oid zodb.Oid) (n
} }
func kstr(key Key) string {
return blib.KStr(key)
}
func panicf(format string, argv ...interface{}) { func panicf(format string, argv ...interface{}) {
panic(fmt.Sprintf(format, argv...)) panic(fmt.Sprintf(format, argv...))
} }
// Copyright (C) 2020-2021 Nexedi SA and Contributors.
// Kirill Smelkov <kirr@nexedi.com>
//
// This program is free software: you can Use, Study, Modify and Redistribute
// it under the terms of the GNU General Public License version 3, or (at your
// option) any later version, as published by the Free Software Foundation.
//
// You can also Link and Combine this program with other software covered by
// the terms of any of the Free Software licenses or any of the Open Source
// Initiative approved licenses and Convey the resulting work. Corresponding
// source of such a combination shall include the source code for all other
// software used.
//
// This program is distributed WITHOUT ANY WARRANTY; without even the implied
// warranty of MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.
//
// See COPYING file for full licensing terms.
// See https://www.nexedi.com/licensing for rationale and options.
package xbtreetest
// testing-related support
import (
"flag"
"math/rand"
"testing"
"time"
)
var (
verylongFlag = flag.Bool("verylong", false, `switch tests to run in "very long" mode`)
randseedFlag = flag.Int64("randseed", -1, `seed for random number generator`)
)
// VeryLong returns whether -verylong flag is in effect.
func VeryLong() bool {
// -short takes priority over -verylong
if testing.Short() {
return false
}
return *verylongFlag
}
// N returns short, medium, or long depending on whether tests were ran with
// -short, -verylong, or normally.
func N(short, medium, long int) int {
// -short
if testing.Short() {
return short
}
// -verylong
if *verylongFlag {
return long
}
// default
return medium
}
// NewRand returns new random-number generator and seed that was used to initialize it.
//
// The seed can be controlled via -randseed option.
func NewRand() (rng *rand.Rand, seed int64) {
seed = *randseedFlag
if seed == -1 {
seed = time.Now().UnixNano()
}
rng = rand.New(rand.NewSource(seed))
return rng, seed
}
...@@ -20,7 +20,7 @@ ...@@ -20,7 +20,7 @@
# See https://www.nexedi.com/licensing for rationale and options. # See https://www.nexedi.com/licensing for rationale and options.
"""Program treegen provides infrastructure to generate ZODB BTree states. """Program treegen provides infrastructure to generate ZODB BTree states.
It will be used as helper for ΔBtail and ΔFtail tests. It is used as helper for ΔBtail and ΔFtail tests.
The following subcommands are provided: The following subcommands are provided:
......
// Code generated by gen-rangemap _RangedMap_RebuildJob *_RebuildJob; DO NOT EDIT.
// Copyright (C) 2021 Nexedi SA and Contributors.
// Kirill Smelkov <kirr@nexedi.com>
//
// This program is free software: you can Use, Study, Modify and Redistribute
// it under the terms of the GNU General Public License version 3, or (at your
// option) any later version, as published by the Free Software Foundation.
//
// You can also Link and Combine this program with other software covered by
// the terms of any of the Free Software licenses or any of the Open Source
// Initiative approved licenses and Convey the resulting work. Corresponding
// source of such a combination shall include the source code for all other
// software used.
//
// This program is distributed WITHOUT ANY WARRANTY; without even the implied
// warranty of MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.
//
// See COPYING file for full licensing terms.
// See https://www.nexedi.com/licensing for rationale and options.
package xbtree
import "lab.nexedi.com/nexedi/wendelin.core/wcfs/internal/xbtree/blib"
// map [lo,hi) Key ranges to values.
import (
"fmt"
"sort"
)
const trace_RangedMap_RebuildJob = false
const debug_RangedMap_RebuildJob = false
// _RangedMap_RebuildJob is Key->*_RebuildJob map with adjacent keys mapped to the same value coalesced into Ranges.
//
// Zero value represents empty map.
type _RangedMap_RebuildJob struct {
// TODO rework to use BTree lo->hi_ instead if in practice in treediff,
// and other usage places, N(ranges) turns out to be not small
// (i.e. performance turns out to be not acceptable)
entryv []_RangedMap_RebuildJobEntry // lo↑
}
// _RangedMap_RebuildJobEntry represents one entry in _RangedMap_RebuildJob.
type _RangedMap_RebuildJobEntry struct {
Value *_RebuildJob
blib.KeyRange
}
// Get returns value associated with key k.
//
// blib.KeyRange indicates all keys adjacent to k, that are too mapped to the same value.
func (M *_RangedMap_RebuildJob) Get(k Key) (*_RebuildJob, blib.KeyRange) {
v, r, _ := M.Get_(k)
return v, r
}
// Set changes M to map key k to value v.
func (M *_RangedMap_RebuildJob) Set(k Key, v *_RebuildJob) {
M.SetRange(blib.KeyRange{Lo: k, Hi_: k}, v)
}
// Del removes key k.
func (M *_RangedMap_RebuildJob) Del(k Key) {
M.DelRange(blib.KeyRange{Lo: k, Hi_: k})
}
// Has returns whether key k is present in the map.
func (M *_RangedMap_RebuildJob) Has(k Key) bool {
_, _, ok := M.Get_(k)
return ok
}
// Get_ is comma-ok version of Get.
func (M *_RangedMap_RebuildJob) Get_(k Key) (v *_RebuildJob, r blib.KeyRange, ok bool) {
r = blib.KeyRange{0,-1} // zero value represents non-empty [0,1)
if trace_RangedMap_RebuildJob {
fmt.Printf("\n\nGet_:\n")
fmt.Printf(" M: %s\n", M)
fmt.Printf(" k: %s\n", blib.KStr(k))
defer func() {
fmt.Printf("->·: %v%s, %t\n", v, r, ok)
}()
}
M.verify()
// find first ilo: k < [ilo].hi
l := len(M.entryv)
ilo := sort.Search(l, func(i int) bool {
return k <= M.entryv[i].Hi_
})
debugf_RangedMap_RebuildJob("\tilo: %d\n", ilo)
if ilo == l { // not found
return
}
e := M.entryv[ilo]
if !(e.Lo <= k) { // not found
return
}
// found
return e.Value, e.KeyRange, true
}
// SetRange changes M to map key range r to value v.
func (M *_RangedMap_RebuildJob) SetRange(r blib.KeyRange, v *_RebuildJob) {
e := _RangedMap_RebuildJobEntry{v,r}
if trace_RangedMap_RebuildJob {
fmt.Printf("\n\nSetRange:\n")
fmt.Printf(" M: %s\n", M)
fmt.Printf(" e: %s\n", e)
defer fmt.Printf("->·: %s\n", M)
}
M.verify()
defer M.verify()
if r.Empty() {
return
}
// clear range for r and insert new entry
// TODO optimize for same-value/set case (just merge all covered
// entries into one - see commented AddRange from set vvv)
i := M.delRange(r)
vInsert__RangedMap_RebuildJob(&M.entryv, i, e)
debugf_RangedMap_RebuildJob("\tinsert %s\t-> %s\n", e, M)
// check if we should merge inserted entry with right/left neighbours
if i+1 < len(M.entryv) { // right
x := M.entryv[i]
right := M.entryv[i+1]
if (x.Hi_+1 == right.Lo) && (v == right.Value) {
vReplaceSlice__RangedMap_RebuildJob(&M.entryv, i,i+2,
_RangedMap_RebuildJobEntry{v, blib.KeyRange{x.Lo, right.Hi_}})
debugf_RangedMap_RebuildJob("\tmerge right\t-> %s\n", M)
}
}
if i > 0 { // left
left := M.entryv[i-1]
x := M.entryv[i]
if (left.Hi_+1 == x.Lo) && (left.Value == v) {
vReplaceSlice__RangedMap_RebuildJob(&M.entryv, i-1,i+1,
_RangedMap_RebuildJobEntry{v, blib.KeyRange{left.Lo, x.Hi_}})
debugf_RangedMap_RebuildJob("\tmerge left\t-> %s\n", M)
}
}
// done
/* how it was for just set:
// find first ilo: r.Lo < [ilo].hi
l := len(S.rangev)
ilo := sort.Search(l, func(i int) bool {
return r.Lo <= S.rangev[i].Hi_
})
debugfRSet("\tilo: %d\n", ilo)
if ilo == l { // not found
S.rangev = append(S.rangev, r)
l++
debugfRSet("\tappend %s\t-> %s\n", r, S)
}
// find last jhi: [jhi].Lo < r.hi
jhi := ilo
for ;; jhi++ {
if jhi == l {
break
}
if S.rangev[jhi].Lo <= r.Hi_ {
continue
}
break
}
debugfRSet("\tjhi: %d\n", jhi)
// entries in [ilo:jhi) ∈ [r.Lo,r.hi) and should be merged into one
if (jhi - ilo) > 1 {
lo := S.rangev[ilo].Lo
hi_ := S.rangev[jhi-1].Hi_
vReplaceSlice__RangedMap_RebuildJob(&S.rangev, ilo,jhi, blib.KeyRange{lo,hi_})
debugfRSet("\tmerge S[%d:%d]\t-> %s\n", ilo, jhi, S)
}
jhi = -1 // no longer valid
// if [r.lo,r.hi) was outside of any entry - create new entry
if r.Hi_ < S.rangev[ilo].Lo {
vInsert__RangedMap_RebuildJob(&S.rangev, ilo, r)
debugfRSet("\tinsert %s\t-> %s\n", r, S)
}
// now we have covered entries merged as needed into [ilo]
// extend this entry if r coverage is wider
if r.Lo < S.rangev[ilo].Lo {
S.rangev[ilo].Lo = r.Lo
debugfRSet("\textend left\t-> %s\n", S)
}
if r.Hi_ > S.rangev[ilo].Hi_ {
S.rangev[ilo].Hi_ = r.Hi_
debugfRSet("\textend right\t-> %s\n", S)
}
// and check if we should merge it with right/left neighbours
if ilo+1 < len(S.rangev) { // right
if S.rangev[ilo].Hi_+1 == S.rangev[ilo+1].Lo {
vReplaceSlice__RangedMap_RebuildJob(&S.rangev, ilo,ilo+2,
blib.KeyRange{S.rangev[ilo].Lo, S.rangev[ilo+1].Hi_})
debugfRSet("\tmerge right\t-> %s\n", S)
}
}
if ilo > 0 { // left
if S.rangev[ilo-1].Hi_+1 == S.rangev[ilo].Lo {
vReplaceSlice__RangedMap_RebuildJob(&S.rangev, ilo-1,ilo+1,
blib.KeyRange{S.rangev[ilo-1].Lo, S.rangev[ilo].Hi_})
debugfRSet("\tmerge left\t-> %s\n", S)
}
}
// done
*/
}
// DelRange removes range r from the map.
func (M *_RangedMap_RebuildJob) DelRange(r blib.KeyRange) {
if trace_RangedMap_RebuildJob {
fmt.Printf("\n\nDelRange:\n")
fmt.Printf(" M: %s\n", M)
fmt.Printf(" r: %s\n", r)
defer fmt.Printf("->·: %s\n", M)
}
M.verify()
defer M.verify()
if r.Empty() {
return
}
M.delRange(r)
}
// delRange deletes range r from the map and returns .entryv index where r
// should be inserted/appended if needed.
//
// r must be !empty.
func (M *_RangedMap_RebuildJob) delRange(r blib.KeyRange) (i int) {
// find first ilo: r.Lo < [ilo].hi
l := len(M.entryv)
ilo := sort.Search(l, func(i int) bool {
return r.Lo <= M.entryv[i].Hi_
})
debugf_RangedMap_RebuildJob("\tilo: %d\n", ilo)
if ilo == l { // not found
debugf_RangedMap_RebuildJob("\tnon-overlap right\n")
return l
}
// find last jhi: [jhi].Lo < r.hi
jhi := ilo
for ;; jhi++ {
if jhi == l {
break
}
if M.entryv[jhi].Lo <= r.Hi_ {
continue
}
break
}
debugf_RangedMap_RebuildJob("\tjhi: %d\n", jhi)
if jhi == 0 {
debugf_RangedMap_RebuildJob("\tnon-overlap left\n")
return 0
}
// [ilo+1:jhi-1] should be deleted
// [ilo] and [jhi-1] overlap with [r.lo,r.hi) - they should be deleted, or shrinked,
// or split+shrinked if ilo==jhi-1 and r is inside [ilo]
if jhi-ilo == 1 && M.entryv[ilo].Lo < r.Lo && r.Hi_ < M.entryv[ilo].Hi_ {
x := M.entryv[ilo]
vInsert__RangedMap_RebuildJob(&M.entryv, ilo, x)
jhi++
debugf_RangedMap_RebuildJob("\tpresplit copy %s\t-> %s\n", x, M)
}
if M.entryv[ilo].Lo < r.Lo { // shrink left
M.entryv[ilo].Hi_ = r.Lo-1
debugf_RangedMap_RebuildJob("\tshrink [%d] left \t-> %s\n", ilo, M)
ilo++
}
if r.Hi_ < M.entryv[jhi-1].Hi_ { // shrink right
M.entryv[jhi-1].Lo = r.Hi_+1
debugf_RangedMap_RebuildJob("\tshrink [%d] right\t-> %s\n", jhi-1, M)
jhi--
}
if (jhi - ilo) > 0 {
vDeleteSlice__RangedMap_RebuildJob(&M.entryv, ilo,jhi)
debugf_RangedMap_RebuildJob("\tdelete M[%d:%d]\t-> %s\n", ilo, jhi, M)
}
// done
return ilo
}
// HasRange returns whether all keys from range r belong to the map.
func (M *_RangedMap_RebuildJob) HasRange(r blib.KeyRange) (yes bool) {
if trace_RangedMap_RebuildJob {
fmt.Printf("\n\nHasRange:\n")
fmt.Printf(" M: %s\n", M)
fmt.Printf(" r: %s\n", r)
defer func() {
fmt.Printf("->·: %v\n", yes)
}()
}
M.verify()
if r.Empty() {
return true
}
// find first ilo: r.lo < [ilo].hi
l := len(M.entryv)
ilo := sort.Search(l, func(i int) bool {
return r.Lo <= M.entryv[i].Hi_
})
debugf_RangedMap_RebuildJob("\tilo: %d\n", ilo)
if ilo == l { // not found
return false
}
// scan right and verify that whole r is covered
lo := r.Lo
for {
e := M.entryv[ilo]
debugf_RangedMap_RebuildJob("\te: %s\ttocheck: %s\n", e, blib.KeyRange{lo, r.Hi_})
if lo < e.Lo {
return false // hole in coverage
}
if r.Hi_ <= e.Hi_ {
return true // reached full coverage
}
lo = e.Hi_
if lo < KeyMax {
lo++
}
ilo++
if ilo == l {
return false // r's right not fully covered
}
}
}
// IntersectsRange returns whether some keys from range r belong to the map.
func (M *_RangedMap_RebuildJob) IntersectsRange(r blib.KeyRange) (yes bool) {
if trace_RangedMap_RebuildJob {
fmt.Printf("\n\nIntersectsRange:\n")
fmt.Printf(" M: %s\n", M)
fmt.Printf(" r: %s\n", r)
defer func() {
fmt.Printf("->·: %v\n", yes)
}()
}
M.verify()
if r.Empty() {
return false
}
// find first ilo: r.lo < [ilo].hi
l := len(M.entryv)
ilo := sort.Search(l, func(i int) bool {
return r.Lo <= M.entryv[i].Hi_
})
debugf_RangedMap_RebuildJob("\tilo: %d\n", ilo)
if ilo == l { // not found
return false
}
// [ilo].hi may be either inside r (≤ r.hi), or > r.hi
// - if it is inside -> overlap is there,
// - if it is > r.hi -> overlap is there if [ilo].lo < r.hi
// => in any case overlap is there if [ilo].lo < r.hi
return M.entryv[ilo].Lo <= r.Hi_
}
// --------
// verify checks _RangedMap_RebuildJob for internal consistency:
// - ranges must be not overlapping and ↑
// - adjacent ranges must map to different values
func (M *_RangedMap_RebuildJob) verify() {
// TODO !debug -> return
var badv []string
badf := func(format string, argv ...interface{}) {
badv = append(badv, fmt.Sprintf(format, argv...))
}
defer func() {
if badv != nil {
emsg := "M.verify: fail:\n\n"
for _, bad := range badv {
emsg += fmt.Sprintf("- %s\n", bad)
}
emsg += fmt.Sprintf("\nM: %s\n", M)
panic(emsg)
}
}()
hi_Prev := KeyMin
var v_Prev *_RebuildJob
for i, e := range M.entryv {
hiPrev := hi_Prev + 1
if i > 0 {
if (e.Value == v_Prev) {
if !(hiPrev < e.Lo) { // NOTE not ≤ - adjacent ranges must be merged
badf("[%d]: same value: !(hiPrev < e.lo)", i)
}
} else {
if !(hi_Prev <= e.Lo) {
badf("[%d]: different value: !(hiPrev ≤ e.lo)", i)
}
}
}
if !(e.Lo <= e.Hi_) {
badf("[%d]: !(e.lo ≤ e.hi_)", i)
}
hi_Prev = e.Hi_
v_Prev = e.Value
}
}
// Clone returns copy of the map.
//
// NOTE values are _not_ cloned.
func (orig *_RangedMap_RebuildJob) Clone() *_RangedMap_RebuildJob {
klon := &_RangedMap_RebuildJob{}
klon.entryv = append(klon.entryv, orig.entryv...)
return klon
}
// Empty returns whether the map is empty.
func (M *_RangedMap_RebuildJob) Empty() bool {
return len(M.entryv) == 0
}
// Equal returns whether A == B.
func (A *_RangedMap_RebuildJob) Equal(B *_RangedMap_RebuildJob) bool {
if len(A.entryv) != len(B.entryv) {
return false
}
for i, ea := range A.entryv {
eb := B.entryv[i]
if ea != eb {
return false
}
}
return true
}
// Clear removes all elements from the map.
func (M *_RangedMap_RebuildJob) Clear() {
M.entryv = nil
}
// AllRanges returns slice of all key ranges in the set.
//
// TODO -> iter?
func (M *_RangedMap_RebuildJob) AllRanges() /*readonly*/[]_RangedMap_RebuildJobEntry {
return M.entryv
}
func (M _RangedMap_RebuildJob) String() string {
s := "{"
for i, e := range M.entryv {
if i > 0 {
s += " "
}
s += e.String()
}
s += "}"
return s
}
func (e _RangedMap_RebuildJobEntry) String() string {
s := e.KeyRange.String()
v := fmt.Sprintf("%v", e.Value)
if v != "" { // omit ":<v>" in the case of set
s += ":" + v
}
return s
}
func debugf_RangedMap_RebuildJob(format string, argv ...interface{}) {
if !debug_RangedMap_RebuildJob {
return
}
fmt.Printf(format, argv...)
}
// ---- slice ops ----
// vInsert__RangedMap_RebuildJob inserts e into *pv[i].
func vInsert__RangedMap_RebuildJob(pv *[]_RangedMap_RebuildJobEntry, i int, e _RangedMap_RebuildJobEntry) {
v := *pv
v = append(v, _RangedMap_RebuildJobEntry{})
copy(v[i+1:], v[i:])
v[i] = e
*pv = v
}
// vDeleteSlice__RangedMap_RebuildJob deletes *pv[lo:hi].
func vDeleteSlice__RangedMap_RebuildJob(pv *[]_RangedMap_RebuildJobEntry, lo,hi int) {
v := *pv
n := copy(v[lo:], v[hi:])
v = v[:lo+n]
*pv = v
}
// vReplaceSlice__RangedMap_RebuildJob replaces *pv[lo:hi] with e.
func vReplaceSlice__RangedMap_RebuildJob(pv *[]_RangedMap_RebuildJobEntry, lo,hi int, e _RangedMap_RebuildJobEntry) {
v := *pv
n := copy(v[lo+1:], v[hi:])
v[lo] = e
v = v[:lo+1+n]
*pv = v
}
// Copyright (C) 2018-2021 Nexedi SA and Contributors.
// Kirill Smelkov <kirr@nexedi.com>
//
// This program is free software: you can Use, Study, Modify and Redistribute
// it under the terms of the GNU General Public License version 3, or (at your
// option) any later version, as published by the Free Software Foundation.
//
// You can also Link and Combine this program with other software covered by
// the terms of any of the Free Software licenses or any of the Open Source
// Initiative approved licenses and Convey the resulting work. Corresponding
// source of such a combination shall include the source code for all other
// software used.
//
// This program is distributed WITHOUT ANY WARRANTY; without even the implied
// warranty of MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.
//
// See COPYING file for full licensing terms.
// See https://www.nexedi.com/licensing for rationale and options.
package xbtree
// ΔBtail provides BTree-level history tail.
//
// It translates ZODB object-level changes to information about which keys of
// which BTree were modified, and provides service to query that information.
// See ΔBtail class documentation for usage details.
//
//
// ΔBtail organization
//
// ΔBtail keeps raw ZODB history in ΔZtail and uses BTree-diff algorithm(*) to
// turn δZ into BTree-level diff. For each tracked BTree a separate ΔTtail is
// maintained with tree-level history in ΔTtail.vδT .
//
// Because it is very computationally expensive(+) to find out for an object to
// which BTree it belongs, ΔBtail cannot provide full BTree-level history given
// just ΔZtail with δZ changes. Due to this ΔBtail requires help from
// users, which are expected to call ΔBtail.Track(treepath) to let ΔBtail know
// that such and such ZODB objects constitute a path from root of a tree to some
// of its leaf. After Track call the objects from the path and tree keys, that
// are covered by leaf node, become tracked: from now-on ΔBtail will detect
// and provide BTree-level changes caused by any change of tracked tree objects
// or tracked keys. This guarantee can be provided because ΔBtail now knows
// that such and such objects belong to a particular tree.
//
// To manage knowledge which tree part is tracked ΔBtail uses PPTreeSubSet.
// This data-structure represents so-called PP-connected set of tree nodes:
// simply speaking it builds on some leafs and then includes parent(leaf),
// parent(parent(leaf)), etc. In other words it's a "parent"-closure of the
// leafs. The property of being PP-connected means that starting from any node
// from such set, it is always possible to reach root node by traversing
// .parent links, and that every intermediate node went-through during
// traversal also belongs to the set.
//
// A new Track request potentially grows tracked keys coverage. Due to this,
// on a query, ΔBtail needs to recompute potentially whole vδT of the affected
// tree. This recomputation is managed by "vδTSnapForTracked*" and "_rebuild"
// functions and uses the same treediff algorithm, that Update is using, but
// modulo PPTreeSubSet corresponding to δ key coverage. Update also potentially
// needs to rebuild whole vδT history, not only append new δT, because a
// change to tracked tree nodes can result in growth of tracked key coverage.
//
// Queries are relatively straightforward code that work on vδT snapshot. The
// main complexity, besides BTree-diff algorithm, lies in recomputing vδT when
// set of tracked keys changes, and in handling that recomputation in such a way
// that multiple Track and queries requests could be all served in parallel.
//
//
// Concurrency
//
// In order to allow multiple Track and queries requests to be served in
// parallel ΔBtail employs special organization of vδT rebuild process where
// complexity of concurrency is reduced to math on merging updates to vδT and
// trackSet, and on key range lookup:
//
// 1. vδT is managed under read-copy-update (RCU) discipline: before making
// any vδT change the mutator atomically clones whole vδT and applies its
// change to the clone. This way a query, once it retrieves vδT snapshot,
// does not need to further synchronize with vδT mutators, and can rely on
// that retrieved vδT snapshot will remain immutable.
//
// 2. a Track request goes through 3 states: "new", "handle-in-progress" and
// "handled". At each state keys/nodes of the Track are maintained in:
//
// - ΔTtail.ktrackNew and .trackNew for "new",
// - ΔTtail.krebuildJobs for "handle-in-progress", and
// - ΔBtail.trackSet for "handled".
//
// trackSet keeps nodes, and implicitly keys, from all handled Track
// requests. For all keys, covered by trackSet, vδT is fully computed.
//
// a new Track(keycov, path) is remembered in ktrackNew and trackNew to be
// further processed when a query should need keys from keycov. vδT is not
// yet providing data for keycov keys.
//
// when a Track request starts to be processed, its keys and nodes are moved
// from ktrackNew/trackNew into krebuildJobs. vδT is not yet providing data
// for requested-to-be-tracked keys.
//
// all trackSet, trackNew/ktrackNew and krebuildJobs are completely disjoint:
//
// trackSet ^ trackNew = ø
// trackSet ^ krebuildJobs = ø
// trackNew ^ krebuildJobs = ø
//
// 3. when a query is served, it needs to retrieve vδT snapshot that takes
// related previous Track requests into account. Retrieving such snapshots
// is implemented in vδTSnapForTracked*() family of functions: there it
// checks ktrackNew/trackNew, and if those sets overlap with query's keys
// of interest, run vδT rebuild for keys queued in ktrackNew.
//
// the main part of that rebuild can be run without any locks, because it
// does not use nor modify any ΔBtail data, and for δ(vδT) it just computes
// a fresh full vδT build modulo retrieved ktrackNew. Only after that
// computation is complete, ΔBtail is locked again to quickly merge in
// δ(vδT) update back into vδT.
//
// This organization is based on the fact that
//
// vδT/(T₁∪T₂) = vδT/T₁ | vδT/T₂
//
// ( i.e. vδT computed for tracked set being union of T₁ and T₂ is the
// same as merge of vδT computed for tracked set T₁ and vδT computed
// for tracked set T₂ )
//
// and that
//
// trackSet | (δPP₁|δPP₂) = (trackSet|δPP₁) | (trackSet|δPP₂)
//
// ( i.e. tracking set updated for union of δPP₁ and δPP₂ is the same
// as union of tracking set updated with δPP₁ and tracking set updated
// with δPP₂ )
//
// these merge properties allow to run computation for δ(vδT) and δ(trackSet)
// independently and with ΔBtail unlocked, which in turn enables running
// several Track/queries in parallel.
//
// 4. while vδT rebuild is being run, krebuildJobs keeps corresponding keycov
// entry to indicate in-progress rebuild. Should a query need vδT for keys
// from that job, it first waits for corresponding job(s) to complete.
//
// Explained rebuild organization allows non-overlapping queries/track-requests
// to run simultaneously.
//
// --------
//
// (*) implemented in treediff.go
// (+) full database scan
//go:generate ./blib/gen-rangemap _RangedMap_RebuildJob *_RebuildJob zrangemap_rebuildjob.go
import (
"context"
"fmt"
"sort"
"strings"
"sync"
"lab.nexedi.com/kirr/go123/xerr"
"lab.nexedi.com/kirr/go123/xsync"
"lab.nexedi.com/kirr/neo/go/transaction"
"lab.nexedi.com/kirr/neo/go/zodb"
"lab.nexedi.com/nexedi/wendelin.core/wcfs/internal/xbtree/blib"
"lab.nexedi.com/nexedi/wendelin.core/wcfs/internal/xtail"
)
const traceΔBtail = false
const debugΔBtail = false
// ΔBtail represents tail of revisional changes to BTrees.
//
// It semantically consists of
//
// []δB ; rev ∈ (tail, head]
//
// where δB represents a change in BTrees space
//
// δB:
// .rev↑
// {} root -> {}(key, δvalue)
//
// It covers only changes to keys from tracked subset of BTrees parts.
// In particular a key that was not explicitly requested to be tracked, even if
// it was changed in δZ, is not guaranteed to be present in δB.
//
// ΔBtail provides the following operations:
//
// .Track(path) - start tracking tree nodes and keys; root=path[0], keys=path[-1].(lo,hi]
//
// .Update(δZ) -> δB - update BTree δ tail given raw ZODB changes
// .ForgetPast(revCut) - forget changes ≤ revCut
// .SliceByRev(lo, hi) -> []δB - query for all trees changes with rev ∈ (lo, hi]
// .SliceByRootRev(root, lo, hi) -> []δT - query for changes of a tree with rev ∈ (lo, hi]
// .GetAt(root, key, at) -> (value, rev) - get root[key] @at assuming root[key] ∈ tracked
//
// where δT represents a change to one tree
//
// δT:
// .rev↑
// {}(key, δvalue)
//
// An example for tracked set is a set of visited BTree paths.
// There is no requirement that tracked set belongs to only one single BTree.
//
// See also zodb.ΔTail and zdata.ΔFtail
//
//
// Concurrency
//
// ΔBtail is safe to use in single-writer / multiple-readers mode. That is at
// any time there should be either only sole writer, or, potentially several
// simultaneous readers. The table below classifies operations:
//
// Writers: Update, ForgetPast
// Readers: Track + all queries (SliceByRev, SliceByRootRev, GetAt)
//
// Note that, in particular, it is correct to run multiple Track and queries
// requests simultaneously.
type ΔBtail struct {
// raw ZODB changes; Kept to rebuild .byRoot after new Track.
// includes all changed objects, not only tracked ones.
δZtail *zodb.ΔTail
// handle to make connections to access database.
// TODO allow client to optionally provide zconnOld/zconnNew on e.g. Update()
db *zodb.DB // to open connections to load new/old tree|buckets
// mu protects ΔBtail data _and_ all _ΔTtail data for all roots.
//
// NOTE: even though this lock is global, since _ΔTtail.vδT is updated
// via RCU, working with retrieved vδT snapshot does not need to hold the lock.
mu sync.Mutex
vδBroots []_ΔBroots // [] (rev↑, roots changed in this rev)
byRoot map[zodb.Oid]*_ΔTtail // {} root -> [] k/v change history; only for keys ∈ tracked subset
// set of tracked nodes as of @head state.
// For this set all vδT are fully computed.
// The set of keys(nodes) that were requested to be tracked, but were
// not yet taken into account, is kept in _ΔTtail.ktrackNew & co.
trackSet blib.PPTreeSubSet
// set of trees for which _ΔTtail.ktrackNew is non-empty
trackNewRoots setOid
}
// _ΔTtail represent tail of revisional changes to one BTree.
//
// See ΔBtail documentation for details.
type _ΔTtail struct {
// changes to tree keys; rev↑. covers keys ∈ tracked subset
// Note: changes to vδT go through RCU - see "Concurrency" in overview.
vδT []ΔTree
// set of keys that were requested to be tracked in this tree,
// but for which vδT rebuild was not yet started as of @head
ktrackNew blib.RangedKeySet // {keycov}
// set of nodes corresponding to ktrackNew as of @head
trackNew blib.PPTreeSubSet // PP{nodes}
// set of keys(nodes) for which rebuild is in progress
krebuildJobs _RangedMap_RebuildJob // {} keycov -> job
}
// _RebuildJob represents currently in-progress vδT rebuilding job.
type _RebuildJob struct {
trackNew blib.PPTreeSubSet // rebuilding for this trackNew
ready chan struct{} // closed when job completes
err error
}
// _ΔBroots represents roots-only part of ΔB.
//
// It describes which trees were changed, but does not provide δkv details for changed trees.
type _ΔBroots struct {
Rev zodb.Tid
Roots setOid // which roots changed in this revision
}
// ΔB represents a change in BTrees space.
type ΔB struct {
Rev zodb.Tid
ByRoot map[zodb.Oid]map[Key]ΔValue // {} root -> {}(key, δvalue)
}
// ΔTree describes changes to one BTree in one revision.
type ΔTree struct {
Rev zodb.Tid
KV map[Key]ΔValue
}
// NewΔBtail creates new empty ΔBtail object.
//
// Initial tracked set is empty.
// Initial coverage is (at₀, at₀].
//
// db will be used by ΔBtail to open database connections to load data from
// ZODB when needed.
func NewΔBtail(at0 zodb.Tid, db *zodb.DB) *ΔBtail {
return &ΔBtail{
δZtail: zodb.NewΔTail(at0),
vδBroots: nil,
byRoot: map[zodb.Oid]*_ΔTtail{},
trackSet: blib.PPTreeSubSet{},
trackNewRoots: setOid{},
db: db,
}
}
// newΔTtail creates new empty _ΔTtail object.
func newΔTtail() *_ΔTtail {
return &_ΔTtail{
trackNew: blib.PPTreeSubSet{},
}
}
// Clone returns copy of ΔBtail.
func (orig *ΔBtail) Clone() *ΔBtail {
klon := &ΔBtail{db: orig.db}
// δZtail
klon.δZtail = zodb.NewΔTail(orig.Tail())
for _, δZ := range orig.δZtail.Data() {
klon.δZtail.Append(δZ.Rev, δZ.Changev)
}
// vδBroots
klon.vδBroots = make([]_ΔBroots, 0, len(orig.vδBroots))
for _, origδBroots := range orig.vδBroots {
klonδBroots := _ΔBroots{
Rev: origδBroots.Rev,
Roots: origδBroots.Roots.Clone(),
}
klon.vδBroots = append(klon.vδBroots, klonδBroots)
}
// byRoot
klon.byRoot = make(map[zodb.Oid]*_ΔTtail, len(orig.byRoot))
for root, origΔTtail := range orig.byRoot {
klon.byRoot[root] = origΔTtail.Clone()
}
// trackSet, trackNewRoots
klon.trackSet = orig.trackSet.Clone()
klon.trackNewRoots = orig.trackNewRoots.Clone()
return klon
}
// Clone returns copy of _ΔTtail.
func (orig *_ΔTtail) Clone() *_ΔTtail {
klon := &_ΔTtail{}
klon.vδT = vδTClone(orig.vδT)
klon.trackNew = orig.trackNew.Clone()
return klon
}
// vδTClone returns deep copy of []ΔTree.
func vδTClone(orig []ΔTree) []ΔTree {
if orig == nil {
return nil
}
klon := make([]ΔTree, 0, len(orig))
for _, origδT := range orig {
klonδT := ΔTree{
Rev: origδT.Rev,
KV: make(map[Key]ΔValue, len(origδT.KV)),
}
for k, δv := range origδT.KV {
klonδT.KV[k] = δv
}
klon = append(klon, klonδT)
}
return klon
}
// (tail, head] coverage
func (δBtail *ΔBtail) Head() zodb.Tid { return δBtail.δZtail.Head() }
func (δBtail *ΔBtail) Tail() zodb.Tid { return δBtail.δZtail.Tail() }
// ---- Track/snapshot+rebuild/Update/Forget ----
// Track adds tree path to tracked set.
//
// path[0] signifies tree root.
// path[-1] signifies leaf node.
// keycov should be key range covered by the leaf node.
//
// ΔBtail will start tracking provided tree nodes and keys ∈ keycov.
//
// All path elements must be Tree except last one which, for non-empty tree, must be Bucket.
//
// Objects in the path must be with .PJar().At() == .Head()
func (δBtail *ΔBtail) Track(nodePath []Node, keycov KeyRange) {
head := δBtail.Head()
for _, node := range nodePath {
nodeAt := node.PJar().At()
if nodeAt != head {
panicf("node.at (@%s) != δBtail.head (@%s)", nodeAt, head)
}
}
path := nodePathToPath(nodePath)
δBtail.track(path, keycov)
}
// nodePathToPath converts path from []Node to []Oid.
func nodePathToPath(nodePath []Node) (path []zodb.Oid) {
// assert nodePath = Tree Tree ... Tree Bucket
l := len(nodePath)
switch {
case l == 0:
panic("empty path")
case l == 1:
// must be empty Tree
_ = nodePath[0].(*Tree)
default:
// must be Tree Tree ... Tree Bucket
for _, node := range nodePath[:l-1] {
_ = node.(*Tree)
}
_ = nodePath[l-1].(*Bucket)
}
path = make([]zodb.Oid, l)
for i, node := range nodePath {
path[i] = node.POid()
}
return path
}
func (δBtail *ΔBtail) track(path []zodb.Oid, keycov KeyRange) {
δBtail.mu.Lock() // TODO verify that there is no in-progress writers
defer δBtail.mu.Unlock()
if traceΔBtail {
pathv := []string{}
for _, node := range path { pathv = append(pathv, node.String()) }
tracefΔBtail("\nTrack %s %s\n", keycov, strings.Join(pathv, " -> "))
tracefΔBtail("trackSet: %s\n", δBtail.trackSet)
}
// first normalize path: remove embedded bucket and check if it was an
// empty artificial tree. We need to do the normalization because we
// later check whether leaf path[-1] ∈ trackSet and without
// normalization path[-1] can be InvalidOid.
path = blib.NormPath(path)
if len(path) == 0 {
return // empty tree
}
root := path[0]
leaf := path[len(path)-1]
// assertSamePathToLeaf asserts that T.Path(leaf) == path.
assertSamePathToLeaf := func(T blib.PPTreeSubSet, Tname string) {
path_ := T.Path(leaf)
if !pathEqual(path, path_) {
panicf("BUG: keycov %s is already in %s via path=%v\ntrack requests path=%v", keycov, Tname, path_, path)
}
}
// nothing to do if keycov is already tracked
if δBtail.trackSet.Has(leaf) {
tracefΔBtail("->T: nop (already in trackSet)\n")
assertSamePathToLeaf(δBtail.trackSet, "trackSet")
return
}
δTtail, ok := δBtail.byRoot[root]
if !ok {
δTtail = newΔTtail()
δBtail.byRoot[root] = δTtail
}
// nothing to do if keycov is already queued to be tracked in trackNew or krebuildJobs
if δTtail.krebuildJobs.IntersectsRange(keycov) {
tracefΔBtail("->T: nop (already in krebuildJobs)\n")
job, r, ok := δTtail.krebuildJobs.Get_(keycov.Lo)
if !(ok && r == keycov) {
panicf("BUG: keycov is already present in krebuildJobs, but only partly\nkeycov: %s\nkrebuildJobs: %v",
keycov, δTtail.krebuildJobs)
}
assertSamePathToLeaf(job.trackNew, "job.trackNew")
return
}
if δTtail.trackNew.Has(leaf) {
tracefΔBtail("->T: nop (already in trackNew)\n")
assertSamePathToLeaf(δTtail.trackNew, "trackNew")
return
}
// keycov not in trackSet/trackNew/krebuildJobs -> queue it into trackNew
δBtail.trackNewRoots.Add(root)
δTtail.trackNew.AddPath(path)
δTtail.ktrackNew.AddRange(keycov)
tracefΔBtail("->T: [%s].trackNew -> %s\n", root, δTtail.trackNew)
tracefΔBtail("->T: [%s].ktrackNew -> %s\n", root, δTtail.ktrackNew)
}
// vδTSnapForTrackedKey returns vδT snapshot for root that takes into account
// at least all previous Track requests related to key.
//
// vδT is rebuilt if there are such not-yet-handled Track requests.
func (δBtail *ΔBtail) vδTSnapForTrackedKey(root zodb.Oid, key Key) (vδT []ΔTree, err error) {
δBtail.mu.Lock() // TODO verify that there is no in-progress writers
δTtail := δBtail.byRoot[root] // must be there
if δTtail == nil {
δBtail.mu.Unlock()
panicf("δBtail: root<%s> not tracked", root)
}
// TODO key not tracked -> panic (check key ∈ lastRevOf)
if !δTtail.ktrackNew.Has(key) {
// key ∉ ktrackNew
job, _, inJobs := δTtail.krebuildJobs.Get_(key)
if !inJobs {
// key ∉ krebuildJobs -> it should be already in trackSet
vδT = δTtail.vδT
δBtail.mu.Unlock()
return vδT, nil
}
// rebuild for root[key] is in progress -> wait for corresponding job to complete
δBtail.mu.Unlock()
<-job.ready
if job.err == nil {
δBtail.mu.Lock()
vδT = δTtail.vδT
δBtail.mu.Unlock()
}
return vδT, job.err
}
// key ∈ ktrackNew -> this goroutine becomes responsible to rebuild vδT for it
// run rebuild job for all keys queued in ktrackNew so far
err = δTtail._rebuild(root, δBtail)
if err == nil {
vδT = δTtail.vδT
}
δBtail.mu.Unlock()
return vδT, err
}
// vδTSnapForTracked returns vδT snapshot for root that takes into account all
// previous Track requests.
//
// vδT is rebuilt if there are such not-yet-handled Track requests.
func (δBtail *ΔBtail) vδTSnapForTracked(root zodb.Oid) (vδT []ΔTree, err error) {
δBtail.mu.Lock() // TODO verify that there is no in-progress writers
δTtail := δBtail.byRoot[root] // must be there
if δTtail == nil {
δBtail.mu.Unlock()
panicf("δBtail: root<%s> not tracked", root)
}
// prepare to wait for all already running jobs, if any
wg := xsync.NewWorkGroup(context.Background())
for _, e := range δTtail.krebuildJobs.AllRanges() {
job := e.Value
wg.Go(func(ctx context.Context) error {
select {
case <-ctx.Done():
return ctx.Err()
case <-job.ready:
return job.err
}
})
}
// run new rebuild job if there are not-yet-handled Track requests
var errJob error
if !δTtail.ktrackNew.Empty() {
errJob = δTtail._rebuild(root, δBtail)
}
// wait for previous jobs to complete as well
δBtail.mu.Unlock()
errWait := wg.Wait()
err = xerr.First(errJob, errWait)
if err != nil {
return nil, err
}
// now it is ok to take the snapshot
δBtail.mu.Lock()
vδT = δTtail.vδT
δBtail.mu.Unlock()
return vδT, nil
}
// _rebuild runs rebuild job for current .ktrackNew/.trackNew
//
// must be called with δBtail.mu locked.
// returns with δBtail.mu locked.
func (δTtail *_ΔTtail) _rebuild(root zodb.Oid, δBtail *ΔBtail) (err error) {
return δTtail.__rebuild(root, δBtail, /*releaseLock=*/true)
}
func (δTtail *_ΔTtail) __rebuild(root zodb.Oid, δBtail *ΔBtail, releaseLock bool) (err error) {
defer xerr.Contextf(&err, "ΔBtail._rebuild root<%s>", root)
trackNew := δTtail.trackNew
ktrackNew := δTtail.ktrackNew
δTtail.trackNew = blib.PPTreeSubSet{}
δTtail.ktrackNew = blib.RangedKeySet{}
job := &_RebuildJob{trackNew: trackNew, ready: make(chan struct{})}
// krebuildJobs += ktrackNew
for _, r := range ktrackNew.AllRanges() {
// assert krebuildJobs ^ r = ø
if δTtail.krebuildJobs.IntersectsRange(r) {
panicf("BUG: rebuild: prologue: " +
"krebuildJobs ^ ktrackNew != ø:\nkrebuildJobs: %s\nktrackNew: %s",
δTtail.krebuildJobs, ktrackNew)
}
δTtail.krebuildJobs.SetRange(r, job)
}
delete(δBtail.trackNewRoots, root)
// build δ(vδT) without the lock
if releaseLock {
δBtail.mu.Unlock()
}
vδTnew, δtrackSet, err := vδTBuild(root, trackNew, δBtail.δZtail, δBtail.db)
if releaseLock {
δBtail.mu.Lock()
}
// krebuildJobs -= ktrackNew
for _, r := range ktrackNew.AllRanges() {
// assert krebuildJobs[r] = job
job_, r_ := δTtail.krebuildJobs.Get(r.Lo)
if !(job_ == job && r_ == r) {
panicf("BUG: rebuild: epilogue: " +
"krebuildJobs entry mutated:\nset in prologue [%s]=%p\ngot in epilogue: [%s]=%p",
r, job, r_, job_)
}
δTtail.krebuildJobs.DelRange(r)
}
// merge rebuild result
if err == nil {
// vδT <- vδTnew RCU; trackSet += δtrackSet
δTtail.vδT = vδTClone(δTtail.vδT)
δrevSet := vδTMergeInplace(&δTtail.vδT, vδTnew)
δBtail.trackSet.UnionInplace(δtrackSet)
δBtail._vδBroots_Update(root, δrevSet)
} else {
// reinstate trackNew and ktrackNew back, so that data for those
// keys are tried to be rebuilt next time, not silently remain
// missing in vδT, i.e. corrupted.
δTtail.trackNew.UnionInplace(trackNew)
δTtail.ktrackNew.UnionInplace(&ktrackNew)
δBtail.trackNewRoots.Add(root)
}
// we are done
job.err = err
close(job.ready)
return err
}
// Update updates δB with object-level ZODB changes.
//
// Only those objects from δZ that belong to tracked set are guaranteed to be
// taken into account. In other words a tree history will assuredly include
// only those keys, that correspond to tracked subset of δZ.
//
// δZ must include all objects changed by ZODB transaction.
//
// TODO optionally accept zconnOld/zconnNew from client
func (δBtail *ΔBtail) Update(δZ *zodb.EventCommit) (_ ΔB, err error) {
headOld := δBtail.Head()
defer xerr.Contextf(&err, "ΔBtail.Update %s -> %s", headOld, δZ.Tid)
δBtail.mu.Lock()
defer δBtail.mu.Unlock()
// TODO verify that there is no in-progress readers/writers
δB1, err := δBtail._Update1(δZ)
δB := ΔB{Rev: δZ.Tid, ByRoot: make(map[zodb.Oid]map[Key]ΔValue)}
for root, δT1 := range δB1.ByRoot {
δTtail := δBtail.byRoot[root] // must succeed
// δtkeycov1 != ø -> rebuild δTtail with trackNew ~= δtkeycov1
if !δT1.δtkeycov1.Empty() && δBtail.δZtail.Len() > 1 {
trackNew := blib.PPTreeSubSet{}
err := widenTrackNew(trackNew, δT1.δtkeycov1, root, δBtail.Head(), δBtail.db)
if err != nil {
return ΔB{}, err
}
// NOTE we cannot skip computing diff for HEAD~..HEAD
// even after _Update1 because _Update1 was working with different trackNew.
vδTnew, δtrackSet, err := vδTBuild(root, trackNew, δBtail.δZtail, δBtail.db)
if err != nil {
return ΔB{}, err
}
// vδT <- vδTnew RCU; trackSet += δtrackSet
δTtail.vδT = vδTClone(δTtail.vδT)
δrevSet := vδTMergeInplace(&δTtail.vδT, vδTnew)
δBtail.trackSet.UnionInplace(δtrackSet)
δBtail._vδBroots_Update(root, δrevSet)
}
// build δB. Even if δT=ø after _Update1, but δtkeycov1 != ø, above
// rebuild could result in head δT becoming != ø. Detect that δTtail head
// is anew by comparing to δZ.Rev.
l := len(δTtail.vδT)
if l > 0 {
δT := δTtail.vδT[l-1] // δT head
if δT.Rev == δZ.Tid {
δB.ByRoot[root] = δT.KV
}
}
}
// vδBroots += δB (δB.Rev could be already there added by ^^^ rebuild)
for root := range δB.ByRoot {
δBtail._vδBroots_Update1(root, δB.Rev)
}
return δB, err
}
// _Update1 serves Update and performs direct update of δTtail head elements from δZ.
// On key coverage growth rebuilding tail of the history is done by Update itself.
//
// _Update1 is also used in tests to verify δtkeycov return from treediff.
type _ΔBUpdate1 struct {
ByRoot map[zodb.Oid]*_ΔTUpdate1
}
type _ΔTUpdate1 struct {
δtkeycov1 *blib.RangedKeySet // {} root -> δtrackedKeys after first treediff (always grow)
}
func (δBtail *ΔBtail) _Update1(δZ *zodb.EventCommit) (δB1 _ΔBUpdate1, err error) {
headOld := δBtail.Head()
defer xerr.Contextf(&err, "ΔBtail.update1 %s -> %s", headOld, δZ.Tid)
tracefΔBtail("\nUpdate @%s -> @%s δZ: %v\n", δBtail.Head(), δZ.Tid, δZ.Changev)
tracefΔBtail("trackSet: %v\n", δBtail.trackSet)
for _, root := range δBtail.trackNewRoots.SortedElements() {
δTtail := δBtail.byRoot[root]
tracefΔBtail("[%s].trackNew: %v\n", root, δTtail.trackNew)
tracefΔBtail("[%s].ktrackNew: %v\n", root, δTtail.ktrackNew)
}
δB1 = _ΔBUpdate1{ByRoot: make(map[zodb.Oid]*_ΔTUpdate1)}
// update .trackSet and vδB from .trackNew
err = δBtail._rebuildAll()
if err != nil {
return δB1, err
}
δBtail.δZtail.Append(δZ.Tid, δZ.Changev)
// NOTE: keep vvv in sync with vδTBuild1
δZTC, δtopsByRoot := δZConnectTracked(δZ.Changev, δBtail.trackSet)
// skip opening DB connections if there is no change to any tree node
if len(δtopsByRoot) == 0 {
return δB1, nil
}
// open ZODB connections corresponding to "old" and "new" states
// TODO caller should provide one of those (probably new) as usually it has it
txn, ctx := transaction.New(context.TODO()) // TODO - merge in cancel via ctx arg
defer txn.Abort()
zconnOld, err := δBtail.db.Open(ctx, &zodb.ConnOptions{At: headOld})
if err != nil {
return δB1, err
}
zconnNew, err := δBtail.db.Open(ctx, &zodb.ConnOptions{At: δZ.Tid})
if err != nil {
return δB1, err
}
for root, δtops := range δtopsByRoot {
δT, δtrack, δtkeycov, err := treediff(ctx, root, δtops, δZTC, δBtail.trackSet, zconnOld, zconnNew)
if err != nil {
return δB1, err
}
tracefΔBtail("\n-> root<%s> δkv: %v δtrack: %v δtkeycov: %v\n", root, δT, δtrack, δtkeycov)
δTtail := δBtail.byRoot[root] // must be there
if len(δT) > 0 { // an object might be resaved without change
// NOTE no need to clone .vδT here because we only append to it:
// Even though queries return vδT aliases, append
// cannot change any slice returned by query to users.
δTtail.vδT = append(δTtail.vδT, ΔTree{Rev: δZ.Tid, KV: δT})
}
δBtail.trackSet.ApplyΔ(δtrack)
δB1.ByRoot[root] = &_ΔTUpdate1{δtkeycov1: δtkeycov}
}
return δB1, nil
}
// _rebuildAll rebuilds ΔBtail taking all trackNew requests into account.
func (δBtail *ΔBtail) _rebuildAll() (err error) {
defer xerr.Context(&err, "ΔBtail._rebuildAll")
tracefΔBtail("\nRebuildAll @%s..@%s trackNewRoots: %s\n", δBtail.Tail(), δBtail.Head(), δBtail.trackNewRoots)
for root := range δBtail.trackNewRoots {
δTtail := δBtail.byRoot[root] // must be there
err = δTtail.__rebuild(root, δBtail, /*releaseLock=*/false)
if err != nil {
return err
}
}
return nil
}
// _vδBroots_Update updates .vδBroots to remember that _ΔTtail for root has
// changed entries with δrevSet revisions.
//
// must be called with δBtail.mu locked.
func (δBtail *ΔBtail) _vδBroots_Update(root zodb.Oid, δrevSet setTid) {
// TODO δrevSet -> []rev↑ and merge them in one go
for rev := range δrevSet {
δBtail._vδBroots_Update1(root, rev)
}
}
func (δBtail *ΔBtail) _vδBroots_Update1(root zodb.Oid, rev zodb.Tid) {
l := len(δBtail.vδBroots)
j := sort.Search(l, func(k int) bool {
return rev <= δBtail.vδBroots[k].Rev
})
if j == l || rev != δBtail.vδBroots[j].Rev {
δBroots := _ΔBroots{Rev: rev, Roots: setOid{}}
// insert(@j, δBroots)
δBtail.vδBroots = append(δBtail.vδBroots[:j],
append([]_ΔBroots{δBroots},
δBtail.vδBroots[j:]...)...)
}
δBroots := δBtail.vδBroots[j]
δBroots.Roots.Add(root)
}
// ForgetPast forgets history entries with revision ≤ revCut.
func (δBtail *ΔBtail) ForgetPast(revCut zodb.Tid) {
δBtail.mu.Lock()
defer δBtail.mu.Unlock()
// TODO verify that there is no in-progress readers/writers
δBtail.δZtail.ForgetPast(revCut)
// go through vδBroots till revcut -> find which trees to trim -> trim ΔTtails.
totrim := setOid{} // roots whose _ΔTtail has changes ≤ revCut
icut := 0
for ; icut < len(δBtail.vδBroots); icut++ {
δBroots := δBtail.vδBroots[icut]
if δBroots.Rev > revCut {
break
}
totrim.Update(δBroots.Roots)
}
// vδBroots[:icut] should be forgotten
δBtail.vδBroots = append([]_ΔBroots(nil), δBtail.vδBroots[icut:]...)
// trim roots
for root := range totrim {
δTtail := δBtail.byRoot[root] // must be present
δTtail._forgetPast(revCut)
}
}
func (δTtail *_ΔTtail) _forgetPast(revCut zodb.Tid) {
icut := 0
for ; icut < len(δTtail.vδT); icut++ {
if δTtail.vδT[icut].Rev > revCut {
break
}
}
// vδT[:icut] should be forgotten
// NOTE clones vδT because queries return vδT aliases
δTtail.vδT = append([]ΔTree(nil), δTtail.vδT[icut:]...)
}
// ---- queries ----
// GetAt tries to retrieve root[key]@at from δBtail data.
//
// If δBtail has δB entry that covers root[key]@at, corresponding value
// (VDEL means deletion) and valueExact=true are returned. If δBtail data
// allows to determine revision of root[key]@at value, corresponding revision
// and revExact=true are returned. If revision of root[key]@at cannot be
// determined (rev=δBtail.Tail, revExact=false) are returned.
//
// In particular:
//
// If δBtail has no δB entry that covers root[key]@at, return is
//
// value: VDEL,
// valueExact: false,
// rev: δBtail.Tail,
// revExact: false
//
// If δBtail has δB entry that covers root[key]@at, return is
//
// value: δB.δvalue.New,
// valueExact: true,
// rev: δB.rev,
// revExact: true
//
// If δBtail has δB entry that covers value for root[key]@at via
// δB.δvalue.Old, but not entry that covers root[key]@at fully, return is:
//
// value: δB.δvalue.Old,
// valueExact: true,
// rev: δBtail.Tail,
// revExact: false
//
// key must be tracked
// at must ∈ (tail, head]
func (δBtail *ΔBtail) GetAt(root zodb.Oid, key Key, at zodb.Tid) (value Value, rev zodb.Tid, valueExact, revExact bool, err error) {
defer xerr.Contextf(&err, "ΔBtail: root<%s>: get %d @%s", root, key, at)
if traceΔBtail {
tracefΔBtail("\nGet root<%s>[%s] @%s\n", root, kstr(key), at)
defer func() {
vexact := ""
rexact := ""
if !valueExact {
vexact = "~"
}
if !revExact {
rexact = "~"
}
tracefΔBtail("-> value: %s%s rev: @%s%s\n", value, vexact, rev, rexact)
}()
}
tail := δBtail.Tail()
head := δBtail.Head()
if !(tail < at && at <= head) {
panicf("at out of bounds: at: @%s, (tail, head] = (@%s, @%s]", at, tail, head)
}
value = VDEL
valueExact = false
rev = tail
revExact = false
// retrieve vδT snapshot that is rebuilt to take Track(key) requests into account
vδT, err := δBtail.vδTSnapForTrackedKey(root, key)
if err != nil {
return value, rev, valueExact, revExact, err
}
debugfΔBtail(" vδT: %v\n", vδT)
// TODO key not tracked -> panic (check key ∈ lastRevOf -- see vvv)
// TODO -> index lastRevOf(key) | linear scan ↓ looking for change ≤ at
for i := len(vδT)-1; i >= 0; i-- {
δT := vδT[i]
δvalue, ok_ := δT.KV[key]
if ok_ {
valueExact = true
if δT.Rev > at {
value = δvalue.Old
} else {
value = δvalue.New
rev = δT.Rev
revExact = true
break
}
}
}
return value, rev, valueExact, revExact, nil
}
// TODO if needed
// func (δBtail *ΔBtail) SliceByRev(lo, hi zodb.Tid) /*readonly*/ []ΔB
// SliceByRootRev returns history of a tree changes in (lo, hi] range.
//
// it must be called with the following condition:
//
// tail ≤ lo ≤ hi ≤ head
//
// the caller must not modify returned slice.
//
// Only tracked keys are guaranteed to be present.
//
// Note: contrary to regular go slicing, low is exclusive while high is inclusive.
func (δBtail *ΔBtail) SliceByRootRev(root zodb.Oid, lo, hi zodb.Tid) (/*readonly*/vδT []ΔTree, err error) {
xtail.AssertSlice(δBtail, lo, hi)
if traceΔBtail {
tracefΔBtail("\nSlice root<%s> (@%s,@%s]\n", root, lo, hi)
defer func() {
tracefΔBtail("-> vδT(lo,hi]: %v\n", vδT)
}()
}
// retrieve vδT snapshot that is rebuilt to take all previous Track requests into account
vδT, err = δBtail.vδTSnapForTracked(root)
if err != nil {
return nil, err
}
debugfΔBtail(" vδT: %v\n", vδT)
l := len(vδT)
if l == 0 {
return nil, nil
}
// find max j : [j].rev ≤ hi linear scan -> TODO binary search
j := l - 1
for ; j >= 0 && vδT[j].Rev > hi; j-- {}
if j < 0 {
return nil, nil // ø
}
// find max i : [i].rev > lo linear scan -> TODO binary search
i := j
for ; i >= 0 && vδT[i].Rev > lo; i-- {}
i++
// NOTE: no need to duplicate returned vδT slice because vδT is
// modified via RCU: i.e. _ΔTtail.rebuild clones vδT before modifying it.
// This way the data we return to caller will stay unchanged even if
// rebuild is running simultaneously.
return vδT[i:j+1], nil
}
// ---- vδTBuild/vδTMerge (rebuild core) ----
// vδTBuild builds vδT from vδZ for root/tracked=trackNew.
//
// It returns:
//
// - vδT,
// - trackNew* - a superset of trackNew accounting that potentially more keys
// become tracked during the build process.
//
// NOTE ΔBtail calls vδTBuild(root, trackNew) to compute update for ΔTtail.vδT.
func vδTBuild(root zodb.Oid, trackNew blib.PPTreeSubSet, δZtail *zodb.ΔTail, db *zodb.DB) (vδT []ΔTree, trackNew_ blib.PPTreeSubSet, err error) {
defer xerr.Contextf(&err, "root<%s>: build vδT", root)
tracefΔBtail("\nvδTBuild %s @%s .. @%s\n", root, δZtail.Tail(), δZtail.Head())
tracefΔBtail("trackNew: %v\n", trackNew)
if len(trackNew) == 0 {
return nil, nil, nil
}
trackNew = trackNew.Clone() // it will become trackNew*
// go backwards and compute vδT <- treediff(lo..hi/trackNew)
vδZ := δZtail.Data()
for {
δtkeycov := &blib.RangedKeySet{} // all keys coming into tracking set during this lo<-hi scan
trackNewCur := trackNew.Clone() // trackNew adjusted as of when going to i<- entry
for i := len(vδZ)-1; i>=0; i-- {
δZ := vδZ[i]
var atPrev zodb.Tid
if i > 0 {
atPrev = vδZ[i-1].Rev
} else {
atPrev = δZtail.Tail()
}
δkv, δtrackNew, δtkeycov_, err := vδTBuild1(atPrev, δZ, trackNewCur, db)
if err != nil {
return nil, nil, err
}
if len(δkv) > 0 {
δT := ΔTree{Rev: δZ.Rev, KV: δkv}
vδTMerge1Inplace(&vδT, δT)
}
trackNewCur.ApplyΔ(δtrackNew)
δtkeycov.UnionInplace(δtkeycov_)
}
// an iteration closer to tail may turn out to add a key to the tracking set.
// We have to recheck all entries newer that revision for changes to that key,
// for example:
//
// 8 5*
// / \ <- / \
// 2 8 2* 7
//
// here initial tracked set is 5*-2*. Going to earlier revision
// 2'th keycov range is widen from [-∞,5) to [-∞,7), so 5*-7 in
// later revision have to be rechecked because 7 was added into
// tracking set.
//
// Implement this via restarting from head and cycling until
// set of tracked keys does not grow anymore.
if δtkeycov.Empty() {
break
}
err := widenTrackNew(trackNew, δtkeycov, root, δZtail.Head(), db)
if err != nil {
return nil, nil, err
}
}
tracefΔBtail("-> vδT: %v\n", vδT)
tracefΔBtail("-> trackNew*: %v\n", trackNew)
return vδT, trackNew, nil
}
// vδTBuild1 builds δT for single δZ.
//
// δtrackNew/δtkeycov represents how trackNew changes when going through `atPrev <- δZ.Rev` .
func vδTBuild1(atPrev zodb.Tid, δZ zodb.ΔRevEntry, trackNew blib.PPTreeSubSet, db *zodb.DB) (δT map[Key]ΔValue, δtrackNew *blib.ΔPPTreeSubSet, δtkeycov *blib.RangedKeySet, err error) {
defer xerr.Contextf(&err, "build1 %s<-%s", atPrev, δZ.Rev)
debugfΔBtail("\n build1 @%s <- @%s\n", atPrev, δZ.Rev)
debugfΔBtail(" δZ:\t%v\n", δZ.Changev)
debugfΔBtail(" trackNew: %v\n", trackNew)
defer func() {
debugfΔBtail("-> δT: %v\n", δT)
debugfΔBtail("-> δtrackNew: %v\n", δtrackNew)
debugfΔBtail("-> δtkeycov: %v\n", δtkeycov)
debugfΔBtail("\n\n")
}()
// NOTE: keep vvv in sync with ΔBtail._Update1
δZTC, δtopsByRoot := δZConnectTracked(δZ.Changev, trackNew)
// skip opening DB connections if there is no change to this tree
if len(δtopsByRoot) == 0 {
return nil, blib.NewΔPPTreeSubSet(), &blib.RangedKeySet{}, nil
}
if len(δtopsByRoot) != 1 {
panicf("BUG: δtopsByRoot has > 1 entries: %v\ntrackNew: %v\nδZ: %v", δtopsByRoot, trackNew, δZ)
}
var root zodb.Oid
var δtops setOid
for root_, δtops_ := range δtopsByRoot {
root = root_
δtops = δtops_
}
// open ZODB connection corresponding to "current" and "prev" states
txn, ctx := transaction.New(context.TODO()) // TODO - merge in cancel via ctx arg
defer txn.Abort()
zconnPrev, err := db.Open(ctx, &zodb.ConnOptions{At: atPrev})
if err != nil {
return nil, nil, nil, err
}
zconnCurr, err := db.Open(ctx, &zodb.ConnOptions{At: δZ.Rev})
if err != nil {
return nil, nil, nil, err
}
// diff backwards curr -> prev
δT, δtrack, δtkeycov, err := treediff(ctx, root, δtops, δZTC, trackNew, zconnCurr, zconnPrev)
if err != nil {
return nil, nil, nil, err
}
debugfΔBtail(" -> root<%s> δkv*: %v δtrack*: %v δtkeycov*: %v\n", root, δT, δtrack, δtkeycov)
for k, δv := range δT {
// the diff was backward; vδT entries are with diff forward
δv.New, δv.Old = δv.Old, δv.New
δT[k] = δv
}
return δT, δtrack, δtkeycov, nil
}
// vδTMergeInplace merges vδTnew into vδT.
//
// δrevSet indicates set of new revisions created in vδT.
// vδT is modified inplace.
func vδTMergeInplace(pvδT *[]ΔTree, vδTnew []ΔTree) (δrevSet setTid) {
// TODO if needed: optimize to go through vδT and vδTnew sequentially
δrevSet = setTid{}
for _, δT := range vδTnew {
newRevEntry := vδTMerge1Inplace(pvδT, δT)
if newRevEntry {
δrevSet.Add(δT.Rev)
}
}
return δrevSet
}
// vδTMerge1Inplace merges one δT entry into vδT.
//
// newRevEntry indicates whether δT.Rev was not there before in vδT.
// vδT is modified inplace.
func vδTMerge1Inplace(pvδT *[]ΔTree, δT ΔTree) (newRevEntry bool) {
if len(δT.KV) == 0 {
return false // δT has no change
}
vδT := *pvδT
l := len(vδT)
j := sort.Search(l, func(k int) bool {
return δT.Rev <= vδT[k].Rev
})
if j == l || vδT[j].Rev != δT.Rev {
newRevEntry = true
δTcurr := ΔTree{Rev: δT.Rev, KV: map[Key]ΔValue{}}
// insert(@j, δTcurr)
vδT = append(vδT[:j],
append([]ΔTree{δTcurr},
vδT[j:]...)...)
}
δTcurr := vδT[j]
for k, δv := range δT.KV {
δv_, already := δTcurr.KV[k]
if already {
if δv != δv_ {
// TODO: return "conflict"
panicf("[%v] inconsistent δv:\nδTcurr: %v\nδT: %v", k, δTcurr, δT)
}
} else {
δTcurr.KV[k] = δv
}
}
*pvδT = vδT
return newRevEntry
}
// widenTrackNew widens trackNew to cover δtkeycov.
func widenTrackNew(trackNew blib.PPTreeSubSet, δtkeycov *blib.RangedKeySet, root zodb.Oid, at zodb.Tid, db *zodb.DB) (err error) {
defer xerr.Contextf(&err, "widenTrackNew root<%s> @%s +%s", root, at, δtkeycov)
debugfΔBtail("\n widenTrackNew %s @%s +%s", root, at, δtkeycov)
txn, ctx := transaction.New(context.TODO()) // TODO - merge in cancel via ctx arg
defer txn.Abort()
zhead, err := db.Open(ctx, &zodb.ConnOptions{At: at}); /*X*/ if err != nil { return err }
xtree, err := zgetNodeOrNil(ctx, zhead, root); /*X*/ if err != nil { return err }
if xtree == nil {
// root deleted -> root node covers [-∞,∞)
trackNew.AddPath([]zodb.Oid{root})
return nil
}
tree := xtree.(*Tree) // must succeed
top := &nodeInRange{prefix: nil, keycov: blib.KeyRange{KeyMin, KeyMax}, node: tree}
V := rangeSplit{top}
for _, r := range δtkeycov.AllRanges() {
lo := r.Lo
for {
b, err := V.GetToLeaf(ctx, lo); /*X*/ if err != nil { return err }
trackNew.AddPath(b.Path())
// continue with next right bucket until r coverage is complete
if r.Hi_ <= b.keycov.Hi_ {
break
}
lo = b.keycov.Hi_ + 1
}
}
return nil
}
// ----------------------------------------
// ΔZtail returns raw ZODB changes accumulated in δBtail so far.
//
// the caller must not modify returned δZtail.
func (δBtail *ΔBtail) ΔZtail() /*readonly*/*zodb.ΔTail {
return δBtail.δZtail
}
// DB returns database handle that δBtail is using to access ZODB.
func (δBtail *ΔBtail) DB() *zodb.DB {
return δBtail.db
}
func tracefΔBtail(format string, argv ...interface{}) {
if traceΔBtail {
fmt.Printf(format, argv...)
}
}
func debugfΔBtail(format string, argv ...interface{}) {
if debugΔBtail {
fmt.Printf(format, argv...)
}
}
// Copyright (C) 2020-2021 Nexedi SA and Contributors.
// Kirill Smelkov <kirr@nexedi.com>
//
// This program is free software: you can Use, Study, Modify and Redistribute
// it under the terms of the GNU General Public License version 3, or (at your
// option) any later version, as published by the Free Software Foundation.
//
// You can also Link and Combine this program with other software covered by
// the terms of any of the Free Software licenses or any of the Open Source
// Initiative approved licenses and Convey the resulting work. Corresponding
// source of such a combination shall include the source code for all other
// software used.
//
// This program is distributed WITHOUT ANY WARRANTY; without even the implied
// warranty of MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.
//
// See COPYING file for full licensing terms.
// See https://www.nexedi.com/licensing for rationale and options.
package xbtree
// tests for δbtail.go / treediff.go
//
// This are the main tests for ΔBtail functionality. There are two primary testing concerns:
//
// 1) to verify treediff algorithm, and
// 2) to verify how ΔTtail rebuilds its history entries when set of tracked keys
// grows upon either new Track requests, or upon Update that turned out to
// trigger such growth of set of tracked keys.
//
// TestΔBTail*/Update and TestΔBTail*/rebuild exercise points "1" and "2" correspondingly.
//
// There are 2 testing approaches:
//
// a) transition a BTree in ZODB through particular tricky tree topologies
// and feed ΔBtail through created database transactions.
// b) transition a BTree in ZODB through random tree topologies
// and feed ΔBtail through created database transactions.
//
// TestΔBTail and TestΔBTailRandom implement approaches "a" and "b" correspondingly.
import (
"fmt"
"math"
"reflect"
"sort"
"strings"
"testing"
"lab.nexedi.com/kirr/go123/exc"
"lab.nexedi.com/kirr/go123/xerr"
"lab.nexedi.com/kirr/neo/go/zodb"
"lab.nexedi.com/nexedi/wendelin.core/wcfs/internal/xbtree/blib"
"lab.nexedi.com/nexedi/wendelin.core/wcfs/internal/xbtree/xbtreetest"
)
type Δstring = xbtreetest.Δstring
// KAdjMatrix is adjacency matrix that describes how set of tracked keys
// changes (always grow) when tree topology is updated from A to B.
//
// Adjacency matrix
//
// A, B - topologies ex T3/B1,2-B3 T3/B1-B3,4
// Av, Bv - topologies with values ex T3/B1:a,2:b-B3:c T3/B1:d-B3:c,4:e
//
// δ(Av, Bv) - full diff {k -> v} for changed keys; DEL = k -> ø
// ex δ(T3/B1:a,2:b-B3:c, T3/B1:d-B3:c,4:e) = {-1:a +1:d -2:b +4:e}
//
// Δ(T, Av, Bv) - subset of δ(Av, Bv) corresponding to initial tracking set T
// ex Δ({1}, T3/B1:a,2:b-B3:c, T3/B1:d-B3:c,4:e) = {-1:a +1:d -2:b} (no +4:e)
//
// kadj(A,B) {} k -> {k'}: - adjacency matrix
// ∃v1,v2: k'∈ Δ({k}, Av1, Bv2)
//
// ex kadj(T3/B1:a,2:b-B3:c, T3/B1:d-B3:c,4:e) =
// = {1:{1,2} 2:{1,2} 3:{3,4,∞} 4:{3,4,∞} ∞:{3,4,∞}} k ∈ A+B+{∞}
// + {0:{0,1,2} 5:{5,3,4,∞} + ... all possible keys}
//
// Δ(T, Av, Bv) = δ(Av, Bv)/kadj(A,B)[T]
//
// i.e. = δ(Av, Bv) for k: k ∈ U kadj(A,B)[·]
// ·∈T
//
// Use:
//
// - KAdj(A,B) to build adjacency matrix for A -> B transition.
// - kadj.Map(keys) to compute kadj·keys.
// - kadj1.Mul(kadj2) to compute kadj1·kadj2.
//
// Note: adjacency matrix is symmetric (KAdj verifies this at runtime):
//
// KAdj(A,B) == KAdj(B,A)
type KAdjMatrix map[Key]setKey
// ΔBTestEntry represents one entry in ΔBTail tests.
type ΔBTestEntry struct {
tree string // next tree topology
kadjOK KAdjMatrix // adjacency matrix against previous case (optional)
flags ΔBTestFlags
}
type ΔBTestFlags int
const ΔBTest_SkipUpdate ΔBTestFlags = 1 // skip verifying Update for this test entry
const ΔBTest_SkipRebuild ΔBTestFlags = 2 // skip verifying rebuild for this test entry
// ΔBTest converts xtest into ΔBTestEntry.
// xtest can be string|ΔBTestEntry.
func ΔBTest(xtest interface{}) ΔBTestEntry {
var test ΔBTestEntry
switch xtest := xtest.(type) {
case string:
test.tree = xtest
test.kadjOK = nil
test.flags = 0
case ΔBTestEntry:
test = xtest
default:
panicf("BUG: ΔBTest: bad type %T", xtest)
}
return test
}
// TestΔBTail verifies ΔBTail for explicitly provided tree topologies.
func TestΔBTail(t *testing.T) {
// K is shorthand for setKey
K := func(keyv ...Key) setKey {
ks := setKey{}
for _, k := range keyv { ks.Add(k) }
return ks
}
// oo is shorthand for KeyMax
const oo = KeyMax
// A is shorthand for KAdjMatrix
type A = KAdjMatrix
// Δ is shorthand for ΔBTestEntry
Δ := func(tree string, kadjOK A) (test ΔBTestEntry) {
test.tree = tree
test.kadjOK = kadjOK
return test
}
// test known cases going through tree1 -> tree2 -> ...
testv := []interface{} {
// start from non-empty tree to verify both ->empty and empty-> transitions
"T/B1:a,2:b",
// empty
"T/B:",
// +1
Δ("T/B1:a",
A{1: K(1,oo),
oo: K(1,oo)}),
// +2
Δ("T/B1:a,2:b",
A{1: K(1,2,oo),
2: K(1,2,oo),
oo: K(1,2,oo)}),
// -1
Δ("T/B2:b",
A{1: K(1,2,oo),
2: K(1,2,oo),
oo: K(1,2,oo)}),
// 2: b->c
Δ("T/B2:c",
A{2: K(2,oo),
oo: K(2,oo)}),
// +1 in new bucket (to the left)
Δ("T2/B1:a-B2:c",
A{1: K(1,2,oo),
2: K(1,2,oo),
oo: K(1,2,oo)}),
// +3 in new bucket (to the right)
Δ("T2,3/B1:a-B2:c-B3:c",
A{1: K(1),
2: K(2,3,oo),
3: K(2,3,oo),
oo: K(2,3,oo)}),
// bucket split; +3 in new bucket
"T/B1:a,2:b",
Δ("T2/B1:a-B2:b,3:c",
A{1: K(1,2,3,oo),
2: K(1,2,3,oo),
3: K(1,2,3,oo),
oo: K(1,2,3,oo)}),
// bucket split; +3 in new bucket; +4 +5 in another new bucket
// everything becomes tracked because original bucket had [-∞,∞) coverage
"T/B1:a,2:b",
Δ("T2,4/B1:a-B2:b,3:c-B4:d,5:e",
A{1: K(1,2,3,4,5,oo),
2: K(1,2,3,4,5,oo),
3: K(1,2,3,4,5,oo),
4: K(1,2,3,4,5,oo),
5: K(1,2,3,4,5,oo),
oo: K(1,2,3,4,5,oo)}),
// reflow of keys: even if tracked={1}, changes to all B nodes need to be rescanned:
// +B12 forces to look in -B23 which adds -3 into δ, which
// forces to look into +B34 and so on.
"T2,4,6/B1:a-B2:b,3:c-B4:d,5:e-B6:f,7:g",
Δ("T3,5,7/B1:g,2:f-B3:e,4:d-B5:c,6:b-B7:a",
A{1: K(1,2,3,4,5,6,7,oo),
2: K(1,2,3,4,5,6,7,oo),
3: K(1,2,3,4,5,6,7,oo),
4: K(1,2,3,4,5,6,7,oo),
5: K(1,2,3,4,5,6,7,oo),
6: K(1,2,3,4,5,6,7,oo),
7: K(1,2,3,4,5,6,7,oo),
oo: K(1,2,3,4,5,6,7,oo)}),
// reflow of keys for rebuild: even if tracked1={}, tracked2={1}, changes to
// all A/B/C nodes need to be rescanned. Contrary to the above case the reflow
// is not detectable at separate diff(A,B) and diff(B,C) runs.
"T3,5,7/B1:a,2:b-B3:c,4:d-B5:e,6:f-B7:g,8:h",
"T/B1:b",
"T2,4,6/B1:a-B2:b,3:c-B4:d,5:e-B6:f,7:g",
// similar situation where rebuild has to detect reflow in between non-neighbour trees
"T3,6/B1:a,2:b-B3:c,4:d-B6:f,7:g",
"T4,7/B1:b-B4:d,5:e-B7:g,8:h",
"T2,5,8/B1:a-B2:b,3:c-B5:e,6:f-B8:h,9:i",
// depth=2; bucket split; +3 in new bucket; left T remain
// _unchanged_ even though B under it is modified.
"T/T/B1:a,2:b",
Δ("T2/T-T/B1:a-B2:b,3:c",
A{1: K(1,2,3,oo),
2: K(1,2,3,oo),
3: K(1,2,3,oo),
oo: K(1,2,3,oo)}),
// depth=2; like prev. case, but additional right arm with +4 +5 is added.
"T/T/B1:a,2:b",
Δ("T2,4/T-T-T/B1:a-B2:b,3:c-B4:d,5:e",
A{1: K(1,2,3,4,5,oo),
2: K(1,2,3,4,5,oo),
3: K(1,2,3,4,5,oo),
4: K(1,2,3,4,5,oo),
5: K(1,2,3,4,5,oo),
oo: K(1,2,3,4,5,oo)}),
// depth=2; bucket split; +3 in new bucket; t0 and t1 split;
// +right arm (T7/B45-B89).
"T/T/B1:a,2:b",
Δ("T4/T2-T7/B1:a-B2:b,3:c-B4:d,5:e-B8:h,9:i",
A{1: K(1,2,3,4,5,8,9,oo),
2: K(1,2,3,4,5,8,9,oo),
3: K(1,2,3,4,5,8,9,oo),
4: K(1,2,3,4,5,8,9,oo),
5: K(1,2,3,4,5,8,9,oo),
8: K(1,2,3,4,5,8,9,oo),
9: K(1,2,3,4,5,8,9,oo),
oo: K(1,2,3,4,5,8,9,oo)}),
// 2 reflow to right B neighbour; 8 splits into new B; δ=ø
"T3/B1:a,2:b-B4:d,8:h",
"T2,5/B1:a-B2:b,4:d-B8:h",
// case where kadj does not grow too much as leafs coverage remains stable
"T4,8/B1:a,2:b-B5:d,6:e-B10:g,11:h",
Δ("T4,8/B2:b,3:c-B6:e,7:f-B11:h,12:i",
A{1: K(1,2,3),
2: K(1,2,3),
3: K(1,2,3),
5: K(5,6,7),
6: K(5,6,7),
7: K(5,6,7,),
10: K(10,11,12,oo),
11: K(10,11,12,oo),
12: K(10,11,12,oo),
oo: K(10,11,12,oo)}),
// tree deletion
// having ø in the middle of the test cases exercises all:
// * `ø -> Tree ...` (tree is created anew),
// * `... Tree -> ø` (tree is deleted), and
// * `Tree -> ø -> Tree` (tree is deleted and then recreated)
xbtreetest.DEL,
// tree rotation
"T3/B2:b-B3:c,4:d",
"T5/T3-T7/B2:a-B3:a,4:a-B6:a-B8:a",
// found by AllStructs ([1] is not changed, but because B1 is
// unlinked and 1 migrates to other bucket, changes in that
// other bucket must be included into δT)
"T1,2/B0:e-B1:d-B2:g,3:a",
"T1/B0:d-B1:d,2:d",
// ----//---- with depth=2
"T1,2/T-T-T/B0:a-B1:b-B2:c,3:d",
"T1/T-T/B0:e-B1:b,2:f",
// degenerate topology from ZODB tests
// https://github.com/zopefoundation/ZODB/commit/6cd24e99f89b
// https://github.com/zopefoundation/BTrees/blob/4.7.2-1-g078ba60/BTrees/tests/testBTrees.py#L20-L57
"T4/T2-T/T-T-T6,10/B1:a-B3:b-T-T-T/T-B7:c-B11:d/B5:e",
"T/B1:e,5:d,7:c,8:b,11:a", // -3 +8
// was leading treegen to generate corrupt trees
"T/T1/T-T/B0:g-B1:e,2:d,3:h",
"T1/T-T3/B0:g-T-T/B1:e,2:d-B3:h",
// was leading to wrongly computed trackSet2 due to top not
// being tracked to tree root.
"T/T1/B0:a-B1:b",
"T/T1/T-T/B0:c-B1:d",
// was leading to wrongly computed trackSet2: leaf bucket not
// reparented to root.
"T/T/B0:a",
"T/B0:a",
// δtkeycov grows due to change in parent tree only
"T3/B1:a-B8:c",
"T7/B1:a-B8:c",
// ----//----
"T3/B1:a,2:b-B8:c,9:d",
"T7/B1:a,2:b-B8:c,9:d",
// ----//---- depth=2
"T3/T-T/B1:a,2:b-B8:c,9:d",
"T7/T-T/B1:a,2:b-B8:c,9:d",
// ----//---- found by AllStructs
"T1,3/B0:d-B1:a-B3:d,4:g",
"T1,4/B0:e-B1:a-B4:c",
// ----//---- found by AllStructs
"T2,4/T-T-T/T1-T-B4:f/T-T-B3:f/B0:h-B1:f",
"T4/T-T/B3:f-T/B4:a",
// ---- found by AllStructs ----
// trackSet2 wrongly computed due to top not being tracked to tree root
"T2/T1-T/B0:g-B1:b-T/B2:b,3:a",
"T2/T1-T/T-T-B2:a/B0:c-B1:g",
// unchanged node is reparented
"T1/B0:c-B1:f",
"T1/T-T/B0:c-T/B1:h",
// SIGSEGV in ApplyΔ
"T1/T-T2/T-B1:c-B2:c/B0:g",
"T1/T-T/B0:g-T/B1:e",
// trackSet corruption: oid is pointed by some .parent but is not present
"T1/T-T/B0:g-T2/B1:h-B2:g",
"T/T1/T-T2/B0:e-B1:f-B2:g",
// ApplyΔ -> xunion: node is reachable from multiple parents
// ( because xdifference did not remove common non-leaf node
// under which there were also other changed, but not initially
// tracked, node )
"T4/T1-T/T-T2-B4:c/T-T-T/B0:f-B1:h-B2:g,3:b",
"T1/T-T/T-T2/T-T-T/B0:f-B1:h-B2:f",
// ----//----
"T3/T1-T/T-T2-T/B0:b-T-T-B3:h/B1:e-B2:a",
"T1/T-T4/T-T2-T/T-T-T-T/B0:b-B1:e-B2:a,3:c-B4:e",
// ----//----
"T/T1,3/T-T2-T4/B0:b-T-T-B3:g-B4:c/B1:b-B2:e",
"T1,4/T-T-T/T-T2-B4:f/T-T-T/B0:h-B1:b-B2:h,3:a",
"T2/B1:a-B7:g",
"T2,8/B1:a-B7:g-B9:i",
"T2/B1:a-B2:b", "T/B1:a,2:b",
"T2,3/B1:a-B2:b-B3:c", "T/B1:a,2:b",
"T2,3/B1:a-B2:c-B3:c", "T/B1:a,2:b",
"T2/B1:a-B2:c", "T2,3/B1:a-B2:c-B3:c",
"T2/B1:a-B3:c",
Δ("T2/T-T4/B1:b-B3:d-B99:h",
A{1: K(1),
3: K(3,99,oo),
99: K(3,99,oo),
oo: K(3,99,oo)}),
// Update was adding extra dup point to vδBroots
"T4/T1,3-T/T-T-T-T/B0:b-B1:c,2:j-T-B4:d/B3:h",
"T/T2,3/T-T-T/B1:d-B2:c-B3:i",
"T2/B1:g-B2:c,3:i",
}
// direct tree_i -> tree_{i+1} -> _{i+2} ... plus
// reverse ... tree_i <- _{i+1} <- _{i+2}
kadjOK := ΔBTest(testv[len(testv)-1]).kadjOK
for i := len(testv)-2; i >= 0; i-- {
test := ΔBTest(testv[i])
kadjOK, test.kadjOK = test.kadjOK, kadjOK
testv = append(testv, test)
}
testq := make(chan ΔBTestEntry)
go func() {
defer close(testq)
for _, test := range testv {
testq <- ΔBTest(test)
}
}()
testΔBTail(t, testq)
}
// TestΔBTailRandom verifies ΔBtail on random tree topologies generated by AllStructs.
func TestΔBTailRandom(t *testing.T) {
X := exc.Raiseif
// considerations:
// - maxdepth↑ better for testing (more tricky topologies)
// - maxsplit↑ not so better for testing (leave s=1, max s=2)
// - |kmin - kmax| affects N(variants) significantly
// -> keep key range small (dumb increase does not help testing)
// - N(keys) affects N(variants) significantly
// -> keep Nkeys reasonably small/medium (dumb increase does not help testing)
//
// - spawning python subprocess is very slow (takes 300-500ms for
// imports; https://github.com/pypa/setuptools/issues/510)
// -> we spawn `treegen allstructs` once and use request/response approach.
maxdepth := xbtreetest.N(2, 3, 4)
maxsplit := xbtreetest.N(1, 2, 2)
n := xbtreetest.N(10,10,100)
nkeys := xbtreetest.N(3, 5, 10)
// server to generate AllStructs(kv, ...)
sg, err := xbtreetest.StartAllStructsSrv(); X(err)
defer func() {
err := sg.Close(); X(err)
}()
// random-number generator
rng, seed := xbtreetest.NewRand()
t.Logf("# maxdepth=%d maxsplit=%d nkeys=%d n=%d seed=%d", maxdepth, maxsplit, nkeys, n, seed)
// generate (kv1, kv2, kv3) randomly
// keysv1, keysv2 and keysv3 are random shuffle of IntSets
var keysv1 [][]int
var keysv2 [][]int
var keysv3 [][]int
for keys := range IntSets(nkeys) {
keysv1 = append(keysv1, keys)
keysv2 = append(keysv2, keys)
keysv3 = append(keysv3, keys)
}
v := keysv1
rng.Shuffle(len(v), func(i,j int) { v[i], v[j] = v[j], v[i] })
v = keysv2
rng.Shuffle(len(v), func(i,j int) { v[i], v[j] = v[j], v[i] })
v = keysv3
rng.Shuffle(len(v), func(i,j int) { v[i], v[j] = v[j], v[i] })
// given random (kv1, kv2, kv3) generate corresponding set of random tree
// topology sets (T1, T2, T3). Then iterate through T1->T2->T3->T1...
// elements such that all right-directed triplets are visited and only once.
// Test Update and rebuild on the generated tree sequences.
vv := "abcdefghij"
randv := func() string {
i := rng.Intn(len(vv))
return vv[i:i+1]
}
// the number of pairs is 3·n^2
// the number of triplets is n^3
//
// limit n for emitted triplets, so that the amount of work for Update
// and rebuild tests is approximately of the same order.
nrebuild := int(math.Ceil(math.Pow(3*float64(n*n), 1./3)))
// in non-short mode rebuild tests are exercising more keys variants, plus every test case
// takes more time. Compensate for that as well.
if !testing.Short() {
nrebuild -= 4
}
testq := make(chan ΔBTestEntry)
go func() {
defer close(testq)
for i := range keysv1 {
keys1 := keysv1[i]
keys2 := keysv2[i]
keys3 := keysv3[i]
kv1 := map[Key]string{}
kv2 := map[Key]string{}
kv3 := map[Key]string{}
for _, k := range keys1 { kv1[Key(k)] = randv() }
for _, k := range keys2 { kv2[Key(k)] = randv() }
for _, k := range keys3 { kv3[Key(k)] = randv() }
treev1, err1 := sg.AllStructs(kv1, maxdepth, maxsplit, n, rng.Int63())
treev2, err2 := sg.AllStructs(kv2, maxdepth, maxsplit, n, rng.Int63())
treev3, err3 := sg.AllStructs(kv3, maxdepth, maxsplit, n, rng.Int63())
err := xerr.Merge(err1, err2, err3)
if err != nil {
t.Fatal(err)
}
emit := func(tree string, flags ΔBTestFlags) {
// skip emitting this entry if both Update and
// Rebuild are requested to be skipped.
if flags == (ΔBTest_SkipUpdate | ΔBTest_SkipRebuild) {
return
}
testq <- ΔBTestEntry{tree, nil, flags}
}
URSkipIf := func(ucond, rcond bool) ΔBTestFlags {
var flags ΔBTestFlags
if ucond {
flags |= ΔBTest_SkipUpdate
}
if rcond {
flags |= ΔBTest_SkipRebuild
}
return flags
}
for j := range treev1 {
for k := range treev2 {
for l := range treev3 {
// limit rebuild to subset of tree topologies,
// because #(triplets) grow as n^3. See nrebuild
// definition above for details.
norebuild := (j >= nrebuild ||
k >= nrebuild ||
l >= nrebuild)
// C_{l-1} -> Aj (pair first seen on k=0)
emit(treev1[j], URSkipIf(k != 0, norebuild))
// Aj -> Bk (pair first seen on l=0)
emit(treev2[k], URSkipIf(l != 0, norebuild))
// Bk -> Cl (pair first seen on j=0)
emit(treev3[l], URSkipIf(j != 0, norebuild))
}
}
}
}
}()
testΔBTail(t, testq)
}
// testΔBTail verifies ΔBTail on sequence of tree topologies coming from testq.
func testΔBTail(t_ *testing.T, testq chan ΔBTestEntry) {
t := xbtreetest.NewT(t_)
var t0 *xbtreetest.Commit
for test := range testq {
t1 := t.Head()
t2 := t.Commit(test.tree)
t.AtSymbReset(t2, 2)
subj := fmt.Sprintf("%s -> %s", t1.Tree, t2.Tree)
//t.Logf("\n\n\n**** %s ****\n\n", subj)
// KAdj
if kadjOK := test.kadjOK; kadjOK != nil {
t.Run(fmt.Sprintf("KAdj/%s→%s", t1.Tree, t2.Tree), func(t *testing.T) {
kadj := KAdj(t1, t2)
if !reflect.DeepEqual(kadj, kadjOK) {
t.Fatalf("BUG: computed kadj is wrong:\nkadjOK: %v\nkadj : %v\n\n", kadjOK, kadj)
}
})
}
// ΔBTail.Update
if test.flags & ΔBTest_SkipUpdate == 0 {
xverifyΔBTail_Update(t.T, subj, t.DB, t.Root(), t1,t2)
}
// ΔBTail.rebuild
if t0 != nil && (test.flags & ΔBTest_SkipRebuild == 0) {
xverifyΔBTail_rebuild(t.T, t.DB, t.Root(), t0,t1,t2)
}
t0, t1 = t1, t2
}
}
// xverifyΔBTail_Update verifies how ΔBTail handles ZODB update for a tree with changes in between t1->t2.
//
// Note: this test verifies only single treediff step of ΔBtail.Update.
// the cycling phase of update, that is responsible to recompute older
// entries when key coverage grows, is exercised by
// xverifyΔBTail_rebuild.
func xverifyΔBTail_Update(t *testing.T, subj string, db *zodb.DB, treeRoot zodb.Oid, t1, t2 *xbtreetest.Commit) {
// verify transition at1->at2 for all initial states of tracked {keys} from kv1 + kv2 + ∞
t.Run(fmt.Sprintf("Update/%s→%s", t1.Tree, t2.Tree), func(t *testing.T) {
allKeys := allTestKeys(t1, t2)
allKeyv := allKeys.SortedElements()
kadj12 := KAdj(t1, t2)
// verify at1->at2 for all combination of initial tracked keys.
for kidx := range IntSets(len(allKeyv)) {
keys := setKey{}
for _, idx := range kidx {
keys.Add(allKeyv[idx])
}
// this t.Run allocates and keeps too much memory in -verylong
// also it is not so useful as above "Update/t1->t2"
//t.Run(fmt.Sprintf(" track=%s", keys), func(t *testing.T) {
xverifyΔBTail_Update1(t, subj, db, treeRoot, t1,t2, keys, kadj12)
//})
}
})
}
// xverifyΔBTail_Update1 verifies how ΔBTail handles ZODB update at1->at2 from initial
// tracked state defined by initialTrackedKeys.
func xverifyΔBTail_Update1(t *testing.T, subj string, db *zodb.DB, treeRoot zodb.Oid, t1,t2 *xbtreetest.Commit, initialTrackedKeys setKey, kadj KAdjMatrix) {
X := exc.Raiseif
//t.Logf("\n>>> Track=%s\n", initialTrackedKeys)
δZ := t2.ΔZ
d12 := t2.Δxkv
var TrackedδZ setKey = nil
var kadjTrackedδZ setKey = nil
var δT, δTok map[Key]Δstring = nil, nil
δZset := setOid{}
for _, oid := range δZ.Changev {
δZset.Add(oid)
}
// badf queues error message to be reported on return.
var badv []string
badf := func(format string, argv ...interface{}) {
badv = append(badv, fmt.Sprintf(format, argv...))
}
defer func() {
if badv != nil || t.Failed() {
emsg := fmt.Sprintf("%s ; tracked=%v :\n\n", subj, initialTrackedKeys)
emsg += fmt.Sprintf("d12: %v\nδTok: %v\nδT: %v\n\n", d12, δTok, δT)
emsg += fmt.Sprintf("δZ: %v\n", δZset)
emsg += fmt.Sprintf("Tracked^δZ: %v\n", TrackedδZ)
emsg += fmt.Sprintf("kadj[Tracked^δZ]: %v\n", kadjTrackedδZ)
emsg += fmt.Sprintf("kadj: %v\n\n", kadj)
emsg += strings.Join(badv, "\n")
emsg += "\n"
t.Fatal(emsg)
}
}()
// δbtail @at1 with initial tracked set
δbtail := NewΔBtail(t1.At, db)
trackKeys(δbtail, t1, initialTrackedKeys)
// TrackedδZ = Tracked ^ δZ (i.e. a tracked node has changed, or its coverage was changed)
TrackedδZ = setKey{}
for k := range initialTrackedKeys {
leaf1 := t1.Xkv.Get(k)
oid1 := leaf1.Oid
if oid1 == zodb.InvalidOid { // embedded bucket
oid1 = leaf1.Parent.Oid
}
leaf2 := t2.Xkv.Get(k)
oid2 := leaf2.Oid
if oid2 == zodb.InvalidOid { // embedded bucket
oid2 = leaf2.Parent.Oid
}
if δZset.Has(oid1) || δZset.Has(oid2) || (leaf1.Keycov != leaf2.Keycov) {
TrackedδZ.Add(k)
}
}
kadjTrackedδZ = setKey{} // kadj[Tracked^δZ] (all keys adjacent to tracked^δZ)
for k := range TrackedδZ {
kadjTrackedδZ.Update(kadj[k])
}
// assert TrackedδZ ∈ kadj[TrackedδZ]
trackNotInKadj := TrackedδZ.Difference(kadjTrackedδZ)
if len(trackNotInKadj) > 0 {
badf("BUG: Tracked^δZ ∉ kadj[Tracked^δZ] ; extra=%v", trackNotInKadj)
return
}
// k ∈ d12
// k ∈ δT <=>
// k ∈ U kadj[·]
// ·∈tracking^δZ
δTok = map[Key]Δstring{} // d12[all keys that should be present in δT]
for k,δv := range d12 {
if kadjTrackedδZ.Has(k) {
δTok[k] = δv
}
}
ø := blib.PPTreeSubSet{}
:= &blib.RangedKeySet{}
// trackSet1 = xkv1[tracked1]
// trackSet2 = xkv2[tracked2] ( = xkv2[kadj[tracked1]]
trackSet1, tkeyCov1 := trackSetWithCov(t1.Xkv, initialTrackedKeys)
trackSet2, tkeyCov2 := trackSetWithCov(t2.Xkv, initialTrackedKeys.Union(kadjTrackedδZ))
// verify δbtail.trackSet against @at1
δbtail.assertTrack(t, "1", ø, trackSet1, tkeyCov1)
// δB <- δZ
//
// also call _Update1 directly to verify δtkeycov return from treediff
// the result of Update and _Update1 should be the same since δbtail is initially empty.
δbtail_ := δbtail.Clone()
δB1, err := δbtail_._Update1(δZ); X(err)
// assert tkeyCov1 ⊂ tkeyCov2
dkeycov12 := tkeyCov1.Difference(tkeyCov2)
if !dkeycov12.Empty() {
t.Errorf("BUG: tkeyCov1 ⊄ tkeyCov2:\n\ttkeyCov1: %s\n\ttkeyCov2: %s\n\ttkeyCov1 \\ tkeyCov2: %s", tkeyCov1, tkeyCov2, dkeycov12)
}
// assert δtkeycov == δ(tkeyCov1, tkeyCov2)
δtkeycovOK := tkeyCov2.Difference(tkeyCov1)
δtkeycov := &blib.RangedKeySet{}
if __, ok := δB1.ByRoot[treeRoot]; ok {
δtkeycov = __.δtkeycov1
}
if !δtkeycov.Equal(δtkeycovOK) {
badf("δtkeycov wrong:\nhave: %s\nwant: %s", δtkeycov, δtkeycovOK)
}
δB, err := δbtail.Update(δZ); X(err)
if δB.Rev != δZ.Tid {
badf("δB: rev != δZ.Tid ; rev=%s δZ.Tid=%s", δB.Rev, δZ.Tid)
return
}
// assert δBtail[root].vδT = δBtail_[root].vδT
var vδT, vδT_ []ΔTree
if δttail, ok := δbtail.byRoot[treeRoot]; ok {
vδT = δttail.vδT
}
if δttail_, ok := δbtail_.byRoot[treeRoot]; ok {
vδT_ = δttail_.vδT
}
if !reflect.DeepEqual(vδT, vδT_) {
badf("δBtail.vδT differs after Update and _Update1:\n_Update1: %v\n Update: %v", vδT_, vδT)
}
// verify δbtail.trackSet against @at2
δbtail.assertTrack(t, "2", trackSet2, ø, )
// assert δB.ByRoot == {treeRoot -> ...} if δTok != ø
// == ø if δTok == ø
rootsOK := setOid{}
if len(δTok) > 0 {
rootsOK.Add(treeRoot)
}
roots := setOid{}
for root := range δB.ByRoot {
roots.Add(root)
}
if !reflect.DeepEqual(roots, rootsOK) {
badf("δB: roots != rootsOK ; roots=%v rootsOK=%v", roots, rootsOK)
}
_, inδB := δB.ByRoot[treeRoot]
if !inδB {
return
}
// δT <- δB
δToid := δB.ByRoot[treeRoot] // {} k -> δoid
δT = xgetδKV(t1,t2, δToid) // {} k -> δ(ZBlk(oid).data)
// δT must be subset of d12.
// changed keys, that are
// - in tracked set -> must be present in δT
// - outside tracked set -> may be present in δT (kadj gives exact answer)
// δT is subset of d12
for _, k := range sortedKeys(δT) {
_, ind12 := d12[k]
if !ind12 {
badf("δT[%v] ∉ d12", k)
}
}
// k ∈ tracked set -> must be present in δT
// k ∉ tracked set -> may be present in δT (kadj gives exact answer)
for _, k := range sortedKeys(d12) {
_, inδT := δT[k]
_, inδTok := δTok[k]
if inδT && !inδTok {
badf("δT[%v] ∉ δTok", k)
}
if !inδT && inδTok {
badf("δT ∌ δTok[%v]", k)
}
if inδT {
if δT[k] != d12[k] {
badf("δT[%v] ≠ δTok[%v]", k, k)
}
}
}
}
// xverifyΔBTail_rebuild verifies ΔBtail.rebuild during t0->t1->t2 transition.
//
// t0->t1 exercises from-scratch rebuild,
// t1->t2 further exercises incremental rebuild.
//
// It also exercises rebuild phase of ΔBtail.Update.
func xverifyΔBTail_rebuild(t *testing.T, db *zodb.DB, treeRoot zodb.Oid, t0, t1, t2 *xbtreetest.Commit) {
t.Run(fmt.Sprintf("rebuild/%s→%s", t0.Tree, t1.Tree), func(t *testing.T) {
tAllKeys := allTestKeys(t0, t1, t2)
tAllKeyv := tAllKeys.SortedElements()
//fmt.Printf("@%s: %v\n", t0.AtSymb(), t0.Xkv.Flatten())
//fmt.Printf("@%s: %v\n", t1.AtSymb(), t1.Xkv.Flatten())
//fmt.Printf("@%s: %v\n", t2.AtSymb(), t2.Xkv.Flatten())
kadj10 := KAdj(t1,t0, allTestKeys(t0,t1,t2))
kadj21 := KAdj(t2,t1, allTestKeys(t0,t1,t2))
kadj12 := KAdj(t1,t2, allTestKeys(t0,t1,t2))
// kadj210 = kadj10·kadj21
kadj210 := kadj10.Mul(kadj21)
ø := blib.PPTreeSubSet{}
:= &blib.RangedKeySet{}
// verify t0 -> t1 Track(keys1) Rebuild -> t2 Track(keys2) Rebuild
// for all combinations of keys1 and keys2
for k1idx := range IntSets(len(tAllKeyv)) {
keys1 := setKey{}
for _, idx1 := range k1idx {
keys1.Add(tAllKeyv[idx1])
}
// δkv1_1 = t1.δxkv / kadj10(keys1)
keys1_0 := kadj10.Map(keys1)
δkv1_1 := map[Key]Δstring{}
for k := range keys1_0 {
δv, ok := t1.Δxkv[k]
if ok {
δkv1_1[k] = δv
}
}
Tkeys1, kTkeys1 := trackSetWithCov(t1.Xkv, keys1)
Tkeys1_0 := trackSet(t1.Xkv, keys1_0)
t.Run(fmt.Sprintf(" T%s;R", keys1), func(t *testing.T) {
δbtail := NewΔBtail(t0.At, db)
// assert trackSet=ø, trackNew=ø, vδB=[]
δbtail.assertTrack(t, "@at0", ø, ø, )
assertΔTtail(t, "@at0", δbtail, t0, treeRoot,
/*vδT=ø*/)
xverifyΔBTail_rebuild_U(t, δbtail, treeRoot, t0, t1,
/*trackSet=*/ø,
/*vδT=ø*/)
xverifyΔBTail_rebuild_TR(t, δbtail, t1, treeRoot,
// after Track(keys1)
keys1,
/*trackSet=*/ ø,
/*trackNew=*/ Tkeys1,
/*ktrackNew=*/ kTkeys1,
// after rebuild
/*trackSet=*/ Tkeys1_0,
/*vδT=*/ δkv1_1)
t.Run((" →" + t2.Tree), func(t *testing.T) {
// keys1R2 is full set of keys that should become tracked after
// Update() (which includes rebuild)
keys1R2 := kadj12.Map(keys1)
for {
keys1R2_ := kadj210.Map(keys1R2)
if keys1R2.Equal(keys1R2_) {
break
}
keys1R2 = keys1R2_
}
// δkvX_k1R2 = tX.δxkv / keys1R2
δkv1_k1R2 := map[Key]Δstring{}
δkv2_k1R2 := map[Key]Δstring{}
for k := range keys1R2 {
δv1, ok := t1.Δxkv[k]
if ok {
δkv1_k1R2[k] = δv1
}
δv2, ok := t2.Δxkv[k]
if ok {
δkv2_k1R2[k] = δv2
}
}
Tkeys1R2, kTkeys1R2 := trackSetWithCov(t2.Xkv, keys1R2)
xverifyΔBTail_rebuild_U(t, δbtail, treeRoot, t1, t2,
/*trackSet=*/ Tkeys1R2,
/*vδT=*/ δkv1_k1R2, δkv2_k1R2)
// tRestKeys2 = tAllKeys - keys1
// reduce that to = tAllKeys - keys1R2 in short/medium mode
// ( if key from keys2 already became tracked after Track(keys1) + Update,
// adding Track(that-key), is not adding much testing coverage to recompute paths )
var tRestKeys2 setKey
if !xbtreetest.VeryLong() {
tRestKeys2 = tAllKeys.Difference(keys1R2)
} else {
tRestKeys2 = tAllKeys.Difference(keys1)
}
tRestKeyv2 := tRestKeys2.SortedElements()
for k2idx := range IntSets(len(tRestKeyv2)) {
keys2 := setKey{}
for _, idx2 := range k2idx {
keys2.Add(tRestKeyv2[idx2])
}
// keys12R2 is full set of keys that should become tracked after
// Track(keys2) + rebuild
keys12R2 := keys1R2.Union(keys2)
for {
keys12R2_ := kadj210.Map(keys12R2)
if keys12R2.Equal(keys12R2_) {
break
}
keys12R2 = keys12R2_
}
Tkeys2, kTkeys2 := trackSetWithCov(t2.Xkv, keys2)
Tkeys12R2 := trackSet(t2.Xkv, keys12R2)
/*
fmt.Printf("\n\n\nKKK\nkeys1=%s keys2=%s\n", keys1, keys2)
fmt.Printf("keys1R2: %s\n", keys1R2)
fmt.Printf("keys12R2: %s\n", keys12R2)
fmt.Printf("t0.Xkv: %v\n", t0.Xkv)
fmt.Printf("t1.Xkv: %v\n", t1.Xkv)
fmt.Printf("t2.Xkv: %v\n", t2.Xkv)
fmt.Printf("kadj21: %v\n", kadj21)
fmt.Printf("kadj12: %v\n", kadj12)
fmt.Printf("Tkeys2 -> %s\n", Tkeys2)
fmt.Printf("Tkeys1R2 -> %s\n", Tkeys1R2)
fmt.Printf("Tkeys2 \\ Tkeys1R2 -> %s\n", Tkeys2.Difference(Tkeys1R2))
fmt.Printf("\n\n\n")
*/
// δkvX_k12R2 = tX.δxkv / keys12R2
δkv1_k12R2 := make(map[Key]Δstring, len(t1.Δxkv))
δkv2_k12R2 := make(map[Key]Δstring, len(t2.Δxkv))
for k := range keys12R2 {
δv1, ok := t1.Δxkv[k]
if ok {
δkv1_k12R2[k] = δv1
}
δv2, ok := t2.Δxkv[k]
if ok {
δkv2_k12R2[k] = δv2
}
}
// t.Run is expensive at this level of nest
//t.Run(" T"+keys2.String()+";R", func(t *testing.T) {
δbtail_ := δbtail.Clone()
xverifyΔBTail_rebuild_TR(t, δbtail_, t2, treeRoot,
// after Track(keys2)
keys2,
/*trackSet*/ Tkeys1R2,
/*trackNew*/ Tkeys2.Difference(
// trackNew should not cover ranges that are
// already in trackSet
Tkeys1R2),
/*ktrackNew*/ kTkeys2.Difference(
// see ^^^ about trackNew
kTkeys1R2),
// after rebuild
/* trackSet=*/ Tkeys12R2,
/*vδT=*/ δkv1_k12R2, δkv2_k12R2)
//})
}
})
})
}
})
}
// xverifyΔBTail_rebuild_U verifies ΔBtail state after Update(ti->tj).
func xverifyΔBTail_rebuild_U(t *testing.T, δbtail *ΔBtail, treeRoot zodb.Oid, ti, tj *xbtreetest.Commit, trackSet blib.PPTreeSubSet, vδTok ...map[Key]Δstring) {
t.Helper()
X := exc.Raiseif
ø := blib.PPTreeSubSet{}
:= &blib.RangedKeySet{}
subj := fmt.Sprintf("after Update(@%s→@%s)", ti.AtSymb(), tj.AtSymb())
// Update ati -> atj
δB, err := δbtail.Update(tj.ΔZ); X(err)
δbtail.assertTrack(t, subj, trackSet, ø, )
assertΔTtail(t, subj, δbtail, tj, treeRoot, vδTok...)
// assert δB = vδTok[-1]
var δT, δTok map[Key]Δstring
if l := len(vδTok); l > 0 {
δTok = vδTok[l-1]
}
if len(δTok) == 0 {
δTok = nil
}
δrootsOK := 1
if δTok == nil {
δrootsOK = 0
}
δroots := setOid{}
for root := range δbtail.byRoot {
δroots.Add(root)
}
δToid, ok := δB.ByRoot[treeRoot]
if ok {
δT = xgetδKV(ti, tj, δToid)
}
if δB.Rev != tj.At {
t.Errorf("%s: δB.Rev: have %s ; want %s", subj, δB.Rev, tj.At)
}
if len(δB.ByRoot) != δrootsOK {
t.Errorf("%s: len(δB.ByRoot) != %d ; δroots=%v", subj, δrootsOK, δroots)
}
if !δTEqual(δT, δTok) {
t.Errorf("%s: δB.ΔBByRoot[%s]:\nhave: %v\nwant: %v", subj, treeRoot, δT, δTok)
}
}
// xverifyΔBTail_rebuild_TR verifies ΔBtail state after Track(keys) + rebuild.
func xverifyΔBTail_rebuild_TR(t *testing.T, δbtail *ΔBtail, tj *xbtreetest.Commit, treeRoot zodb.Oid, keys setKey, trackSet, trackNew blib.PPTreeSubSet, ktrackNew *blib.RangedKeySet, trackSetAfterRebuild blib.PPTreeSubSet, vδTok ...map[Key]Δstring) {
t.Helper()
ø := blib.PPTreeSubSet{}
:= &blib.RangedKeySet{}
// Track(keys)
trackKeys(δbtail, tj, keys)
subj := fmt.Sprintf("@%s: after Track%v", tj.AtSymb(), keys)
δbtail.assertTrack(t, subj, trackSet, trackNew, ktrackNew)
δbtail._rebuildAll()
subj += " + rebuild"
δbtail.assertTrack(t, subj, trackSetAfterRebuild, ø, )
// verify δbtail.byRoot[treeRoot]
assertΔTtail(t, subj, δbtail, tj, treeRoot, vδTok...)
}
// ----------------------------------------
func TestΔBtailForget(t_ *testing.T) {
t := xbtreetest.NewT(t_)
X := exc.Raiseif
t0 := t.Commit("T/B:")
t1 := t.Commit("T/B1:a")
t2 := t.Commit("T2/B1:a-B2:b")
t3 := t.Commit("T/B2:b")
δbtail := NewΔBtail(t0.At, t.DB)
_, err := δbtail.Update(t1.ΔZ); X(err)
_, err = δbtail.Update(t2.ΔZ); X(err)
// start tracking. everything becomes tracked because t1's T/B1:a has [-∞,∞) coverage
// By starting tracking after t2 we verify vδBroots update in both Update and rebuild
_0 := setKey{}; _0.Add(0)
trackKeys(δbtail, t2, _0)
_, err = δbtail.Update(t3.ΔZ); X(err)
// forget calls ForgetPast(revCut) and returns state of vδT right after that.
forget := func(revCut zodb.Tid) []ΔTree {
δbtail.ForgetPast(revCut)
return δbtail.byRoot[t.Root()].vδT
}
vδT_init := δbtail.byRoot[t.Root()].vδT
assertΔTtail(t.T, "init", δbtail, t3, t.Root(), t1.Δxkv, t2.Δxkv, t3.Δxkv)
vδT_at0 := forget(t0.At)
assertΔTtail(t.T, "forget ≤ at0", δbtail, t3, t.Root(), t1.Δxkv, t2.Δxkv, t3.Δxkv)
vδT_at1 := forget(t1.At)
assertΔTtail(t.T, "forget ≤ at1", δbtail, t3, t.Root(), t2.Δxkv, t3.Δxkv)
_ = forget(t3.At)
assertΔTtail(t.T, "forget ≤ at3", δbtail, t3, t.Root(), )
// verify aliasing: init/at0 should be aliased, because there is no change @at0
// at1 must be unaliased from at0; at3 must be unualiased from at1.
vδT_at0[2].Rev = 0
if vδT_init[2].Rev != 0 {
t.Errorf("ForgetPast(at0) cloned vδT but should not")
}
if vδT_at1[1].Rev == 0 {
t.Errorf("ForgetPast(at1) remained vδT aliased but should not")
}
}
func TestΔBtailGetAt(t_ *testing.T) {
// GetAt is thin wrapper around data in ΔTtail.vδT and (TODO) lastRevOf index.
// Recomputing ΔTtail.vδT itself (and TODO lastRevOf) is exercised in depth
// by xverifyΔBTail_rebuild. Here we verify only properties of the wrapper.
t := xbtreetest.NewT(t_)
X := exc.Raiseif
const ø = "ø"
t.Commit("T/B:")
t1 := t.Commit("T/B2:b,3:c") ; at1 := t1.At // 2:b 3:c
t2 := t.Commit("T/B2:b,3:c,4:d") ; at2 := t2.At // 4:d
t3 := t.Commit("T/B2:b,3:e,4:d") ; at3 := t3.At // 3:e
t4 := t.Commit("T/B2:b,3:e,4:f") ; at4 := t4.At // 4:f
t5 := t.Commit("T/B2:b,3:g,4:f") ; at5 := t5.At // 3:g
δBtail := NewΔBtail(t1.At, t.DB)
_, err := δBtail.Update(t2.ΔZ); X(err)
_, err = δBtail.Update(t3.ΔZ); X(err)
_, err = δBtail.Update(t4.ΔZ); X(err)
_, err = δBtail.Update(t5.ΔZ); X(err)
// track everything
_1234 := setKey{}; _1234.Add(1); _1234.Add(2); _1234.Add(3); _1234.Add(4)
trackKeys(δBtail, t5, _1234)
// assertGetAt asserts that GetAt returns expected result.
assertGetAt := func(at zodb.Tid, key Key, valueOK string, revOK zodb.Tid, valueExactOK, revExactOK bool) {
t.Helper()
valueOid, rev, valueExact, revExact, err := δBtail.GetAt(t.Root(), key, at); X(err)
value := ø
if valueOid != VDEL {
value = t.XGetCommit(rev).XGetBlkData(valueOid)
}
if !(value == valueOK && rev == revOK && valueExact == valueExactOK && revExact == revExactOK) {
t.Errorf("GetAt(%v@%s):\nhave: %s @%s %t %t\nwant: %s @%s %t %t",
key, t.AtSymb(at),
value, t.AtSymb(rev), valueExact, revExact,
valueOK, t.AtSymb(revOK), valueExactOK, revExactOK,
)
}
}
// @at key value rev valueExact revExact
assertGetAt(at2, 1, ø, at1, false, false)
assertGetAt(at2, 2, ø, at1, false, false)
assertGetAt(at2, 3, "c", at1, true, false)
assertGetAt(at2, 4, "d", at2, true, true)
assertGetAt(at3, 1, ø, at1, false, false)
assertGetAt(at3, 2, ø, at1, false, false)
assertGetAt(at3, 3, "e", at3, true, true)
assertGetAt(at3, 4, "d", at2, true, true)
assertGetAt(at4, 1, ø, at1, false, false)
assertGetAt(at4, 2, ø, at1, false, false)
assertGetAt(at4, 3, "e", at3, true, true)
assertGetAt(at4, 4, "f", at4, true, true)
assertGetAt(at5, 1, ø, at1, false, false)
assertGetAt(at5, 2, ø, at1, false, false)
assertGetAt(at5, 3, "g", at5, true, true)
assertGetAt(at5, 4, "f", at4, true, true)
}
func TestΔBtailSliceByRootRev(t_ *testing.T) {
// SliceByRootRev is thin wrapper to return ΔTtail.vδT slice.
// Recomputing ΔTtail.vδT itself is exercised in depth by xverifyΔBTail_rebuild.
// Here we verify only properties of the wrapper.
t := xbtreetest.NewT(t_)
X := exc.Raiseif
// ΔT is similar to ΔTree but uses Δstring instead of ΔValue for KV
type ΔT struct {
Rev zodb.Tid
KV map[Key]Δstring
}
// δ is shorthand for ΔT.KV
type δ = map[Key]Δstring
t0 := t.Commit("T2/B1:a-B2:f")
t1 := t.Commit("T2/B1:b-B2:g")
t2 := t.Commit("T2/B1:c-B2:h")
const a, b, c = "a", "b", "c"
const f, g, h = "f", "g", "h"
δbtail := NewΔBtail(t0.At, t.DB)
_, err := δbtail.Update(t1.ΔZ); X(err)
_, err = δbtail.Update(t2.ΔZ); X(err)
// track 2 + rebuild.
_2 := setKey{}; _2.Add(2)
trackKeys(δbtail, t2, _2)
err = δbtail._rebuildAll(); X(err)
δttail := δbtail.byRoot[t.Root()]
// assertvδT asserts that vδT matches vδTok
assertvδT := func(subj string, vδT []ΔTree, vδTok ...ΔT) {
t.Helper()
// convert vδT from ΔTree to ΔT
var vδT_ []ΔT
for _, δT := range vδT {
tj := t.XGetCommit(δT.Rev)
δt := ΔT{δT.Rev, xgetδKV(tj.Prev, tj, δT.KV)}
vδT_ = append(vδT_, δt)
}
if reflect.DeepEqual(vδT_, vδTok) {
return
}
have := []string{}
for _, δT := range vδT_ {
have = append(have, fmt.Sprintf("@%s·%v", t.AtSymb(δT.Rev), δT.KV))
}
want := []string{}
for _, δT := range vδTok {
want = append(want, fmt.Sprintf("@%s·%v", t.AtSymb(δT.Rev), δT.KV))
}
t.Errorf("%s:\nhave: %s\nwant: %s", subj, have, want)
}
s00, err := δbtail.SliceByRootRev(t.Root(), t0.At, t0.At); X(err)
s01, err := δbtail.SliceByRootRev(t.Root(), t0.At, t1.At); X(err)
s02, err := δbtail.SliceByRootRev(t.Root(), t0.At, t2.At); X(err)
s12, err := δbtail.SliceByRootRev(t.Root(), t1.At, t2.At); X(err)
s22, err := δbtail.SliceByRootRev(t.Root(), t2.At, t2.At); X(err)
vδT := δttail.vδT
assertvδT("t2.vδT", vδT, ΔT{t1.At, δ{2:{f,g}}}, ΔT{t2.At, δ{2:{g,h}}})
assertvδT("t2.s00", s00)
assertvδT("t2.s01", s01, ΔT{t1.At, δ{2:{f,g}}})
assertvδT("t2.s02", s02, ΔT{t1.At, δ{2:{f,g}}}, ΔT{t2.At, δ{2:{g,h}}})
assertvδT("t2.s12", s12, ΔT{t2.At, δ{2:{g,h}}})
assertvδT("t2.s22", s22)
// sXX should be all aliased to vδT
gg, _ := t0.XGetBlkByName("g")
hh, _ := t0.XGetBlkByName("h")
vδT[0].Rev = t0.At; δkv0 := vδT[0].KV; vδT[0].KV = map[Key]ΔValue{11:{gg,gg}}
vδT[1].Rev = t0.At; δkv1 := vδT[1].KV; vδT[1].KV = map[Key]ΔValue{12:{hh,hh}}
assertvδT("t2.vδT*", vδT, ΔT{t0.At, δ{11:{g,g}}}, ΔT{t0.At, δ{12:{h,h}}})
assertvδT("t2.s00*", s00)
assertvδT("t2.s01*", s01, ΔT{t0.At, δ{11:{g,g}}})
assertvδT("t2.s02*", s02, ΔT{t0.At, δ{11:{g,g}}}, ΔT{t0.At, δ{12:{h,h}}})
assertvδT("t2.s12*", s12, ΔT{t0.At, δ{12:{h,h}}})
assertvδT("2.s22*", s22)
vδT[0].Rev = t1.At; vδT[0].KV = δkv0
vδT[1].Rev = t2.At; vδT[1].KV = δkv1
assertvδT("t2.vδT+", vδT, ΔT{t1.At, δ{2:{f,g}}}, ΔT{t2.At, δ{2:{g,h}}})
assertvδT("t2.s00+", s00)
assertvδT("t2.s01+", s01, ΔT{t1.At, δ{2:{f,g}}})
assertvδT("t2.s02+", s02, ΔT{t1.At, δ{2:{f,g}}}, ΔT{t2.At, δ{2:{g,h}}})
assertvδT("t2.s12+", s12, ΔT{t2.At, δ{2:{g,h}}})
assertvδT("t2.s22+", s22)
// after track 1 + rebuild old slices remain unchanged, but new queries return updated data
_1 := setKey{}; _1.Add(1)
trackKeys(δbtail, t2, _1)
err = δbtail._rebuildAll(); X(err)
s00_, err := δbtail.SliceByRootRev(t.Root(), t0.At, t0.At); X(err)
s01_, err := δbtail.SliceByRootRev(t.Root(), t0.At, t1.At); X(err)
s02_, err := δbtail.SliceByRootRev(t.Root(), t0.At, t2.At); X(err)
s12_, err := δbtail.SliceByRootRev(t.Root(), t1.At, t2.At); X(err)
s22_, err := δbtail.SliceByRootRev(t.Root(), t2.At, t2.At); X(err)
vδT = δttail.vδT
assertvδT("t12.vδT", vδT, ΔT{t1.At, δ{1:{a,b},2:{f,g}}}, ΔT{t2.At, δ{1:{b,c},2:{g,h}}})
assertvδT("t12.s00", s00)
assertvδT("t12.s00_", s00_)
assertvδT("t12.s01", s01, ΔT{t1.At, δ{ 2:{f,g}}})
assertvδT("t12.s01_", s01_, ΔT{t1.At, δ{1:{a,b},2:{f,g}}})
assertvδT("t12.s02", s02, ΔT{t1.At, δ{ 2:{f,g}}}, ΔT{t2.At, δ{ 2:{g,h}}})
assertvδT("t12.s02_", s02_, ΔT{t1.At, δ{1:{a,b},2:{f,g}}}, ΔT{t2.At, δ{1:{b,c},2:{g,h}}})
assertvδT("t12.s12", s12, ΔT{t2.At, δ{ 2:{g,h}}})
assertvδT("t12.s12_", s12_, ΔT{t2.At, δ{1:{b,c},2:{g,h}}})
assertvδT("t12.s22", s22)
assertvδT("t12.s22_", s22_)
// sXX_ should be all aliased to vδT, but not sXX
bb, _ := t0.XGetBlkByName("b")
cc, _ := t0.XGetBlkByName("c")
vδT[0].Rev = t0.At; δkv0 = vδT[0].KV; vδT[0].KV = map[Key]ΔValue{111:{bb,bb}}
vδT[1].Rev = t0.At; δkv1 = vδT[1].KV; vδT[1].KV = map[Key]ΔValue{112:{cc,cc}}
assertvδT("t12.vδT*", vδT, ΔT{t0.At, δ{111:{b,b}}}, ΔT{t0.At, δ{112:{c,c}}})
assertvδT("t12.s00*", s00)
assertvδT("t12.s00_*", s00_)
assertvδT("t12.s01*", s01, ΔT{t1.At, δ{ 2:{f,g}}})
assertvδT("t12.s01_*", s01_, ΔT{t0.At, δ{111:{b,b} }})
assertvδT("t12.s02*", s02, ΔT{t1.At, δ{ 2:{f,g}}}, ΔT{t2.At, δ{ 2:{g,h}}})
assertvδT("t12.s02_*", s02_, ΔT{t0.At, δ{111:{b,b} }}, ΔT{t0.At, δ{112:{c,c} }})
assertvδT("t12.s12*", s12, ΔT{t2.At, δ{ 2:{g,h}}})
assertvδT("t12.s12_*", s12_, ΔT{t0.At, δ{112:{c,c} }})
assertvδT("t12.s22*", s22)
assertvδT("t12.s22_*", s22_)
vδT[0].Rev = t1.At; vδT[0].KV = δkv0
vδT[1].Rev = t2.At; vδT[1].KV = δkv1
assertvδT("t12.vδT+", vδT, ΔT{t1.At, δ{1:{a,b},2:{f,g}}}, ΔT{t2.At, δ{1:{b,c},2:{g,h}}})
assertvδT("t12.s00+", s00)
assertvδT("t12.s00_+", s00_)
assertvδT("t12.s01+", s01, ΔT{t1.At, δ{ 2:{f,g}}})
assertvδT("t12.s01_+", s01_, ΔT{t1.At, δ{1:{a,b},2:{f,g}}})
assertvδT("t12.s02+", s02, ΔT{t1.At, δ{ 2:{f,g}}}, ΔT{t2.At, δ{ 2:{g,h}}})
assertvδT("t12.s02_+", s02_, ΔT{t1.At, δ{1:{a,b},2:{f,g}}}, ΔT{t2.At, δ{1:{b,c},2:{g,h}}})
assertvδT("t12.s12+", s12, ΔT{t2.At, δ{ 2:{g,h}}})
assertvδT("t12.s12_+", s12_, ΔT{t2.At, δ{1:{b,c},2:{g,h}}})
assertvδT("t12.s22+", s22)
assertvδT("t12.s22_+", s22_)
}
func TestΔBtailClone(t_ *testing.T) {
// ΔBtail.Clone had bug that aliased klon data to orig
t := xbtreetest.NewT(t_)
X := exc.Raiseif
t0 := t.Commit("T2/B1:a-B2:b")
t1 := t.Commit("T2/B1:c-B2:d")
δbtail := NewΔBtail(t0.At, t.DB)
_, err := δbtail.Update(t1.ΔZ); X(err)
_2 := setKey{}; _2.Add(2)
trackKeys(δbtail, t1, _2)
err = δbtail._rebuildAll(); X(err)
δkv1_1 := map[Key]Δstring{2:{"b","d"}}
assertΔTtail(t.T, "orig @at1", δbtail, t1, t.Root(), δkv1_1)
δbklon := δbtail.Clone()
assertΔTtail(t.T, "klon @at1", δbklon, t1, t.Root(), δkv1_1)
t2 := t.Commit("T/B1:b,2:a")
_, err = δbtail.Update(t2.ΔZ); X(err)
δkv1_2 := map[Key]Δstring{1:{"a","c"}, 2:{"b","d"}}
δkv2_2 := map[Key]Δstring{1:{"c","b"}, 2:{"d","a"}}
assertΔTtail(t.T, "orig @at2", δbtail, t2, t.Root(), δkv1_2, δkv2_2)
assertΔTtail(t.T, "klon @at1 after orig @at->@at2", δbklon, t1, t.Root(), δkv1_1)
}
// -------- vδTMerge --------
func TestVδTMerge(t *testing.T) {
vδTMerge1 := func(vδT []ΔTree, δT ΔTree) (_ []ΔTree, newRevEntry bool) {
vδT = vδTClone(vδT)
newRevEntry = vδTMerge1Inplace(&vδT, δT)
return vδT, newRevEntry
}
vδTMerge := func(vδT, vδTnew []ΔTree) (_ []ΔTree, δrevSet setTid) {
vδT = vδTClone(vδT)
δrevSet = vδTMergeInplace(&vδT, vδTnew)
return vδT, δrevSet
}
assertMerge1 := func(vδT []ΔTree, δT ΔTree, newRevEntryOK bool, mergeOK []ΔTree) {
t.Helper()
merge, newRevEntry := vδTMerge1(vδT, δT)
if !(reflect.DeepEqual(merge, mergeOK) && newRevEntry == newRevEntryOK) {
t.Errorf("merge1 %v + %v:\nhave: %v %t\nwant: %v %t",
vδT, δT, merge, newRevEntry, mergeOK, newRevEntryOK)
}
}
assertMerge := func(vδT, vδTnew []ΔTree, δrevSetOK setTid, mergeOK []ΔTree) {
t.Helper()
merge, δrevSet := vδTMerge(vδT, vδTnew)
if !(reflect.DeepEqual(merge, mergeOK) && δrevSet.Equal(δrevSetOK)) {
t.Errorf("merge %v + %v:\nhave: %v %v\nwant: %v %v",
vδT, vδTnew, merge, δrevSet, mergeOK, δrevSetOK)
}
}
// syntax sugar
type Δ = ΔTree
type δ = map[Key]ΔValue
v := func(vδT ...Δ) []Δ {
return vδT
}
r := func(tidv ...zodb.Tid) setTid {
s := setTid{}
for _, tid := range tidv {
s.Add(tid)
}
return s
}
δ1 := δ{1:{1,2}}
δ2 := δ{2:{2,3}}; δ22 := δ{22:{0,1}}; δ2_22 := δ{2:{2,3}, 22:{0,1}}
δ3 := δ{3:{3,4}}
δ4 := δ{4:{4,5}}; δ44 := δ{44:{0,1}}; δ4_44 := δ{4:{4,5}, 44:{0,1}}
δ5 := δ{5:{5,6}}
δ12 := δ{1:{1,2}, 2:{2,3}}
Δ101 := Δ{0x101, δ1}
Δ102 := Δ{0x102, δ2}
Δ103 := Δ{0x103, δ3}
Δ104 := Δ{0x104, δ4}
Δ105 := Δ{0x105, δ5}
// merge1
assertMerge1(nil, Δ{0x100, nil}, false,
nil)
assertMerge1(v(Δ{0x100, δ1}), Δ{0x100, δ2}, false,
v(Δ{0x100, δ12}))
assertMerge1(v(Δ101, Δ103), Δ102, true,
v(Δ101, Δ102, Δ103))
assertMerge1(v(Δ101, Δ102), Δ103, true,
v(Δ101, Δ102, Δ103))
assertMerge1(v(Δ102, Δ103), Δ101, true,
v(Δ101, Δ102, Δ103))
// merge
assertMerge(nil, nil, r(),
nil)
assertMerge(nil, v(Δ101, Δ103), r(0x101, 0x103),
v(Δ101, Δ103))
assertMerge(v(Δ101, Δ103), nil, r(),
v(Δ101, Δ103))
assertMerge(v(Δ102, Δ104), v(Δ101, Δ{0x102, δ22}, Δ103, Δ{0x104, δ44}, Δ105),
r(0x101, 0x103, 0x105),
v(Δ101, Δ{0x102, δ2_22}, Δ103, Δ{0x104, δ4_44}, Δ105))
}
// -------- KAdj --------
// Map returns kadj·keys.
func (kadj KAdjMatrix) Map(keys setKey) setKey {
res := make(setKey, len(keys))
for k := range keys {
to, ok := kadj[k]
if !ok {
panicf("kadj.Map: %d ∉ kadj\n\nkadj: %v", k, kadj)
}
res.Update(to)
}
return res
}
// Mul returns kadjA·kadjB.
//
// (kadjA·kadjB).Map(keys) = kadjA.Map(kadjB.Map(keys))
func (kadjA KAdjMatrix) Mul(kadjB KAdjMatrix) KAdjMatrix {
// ~ assert kadjA.keys == kadjB.keys
// check only len here; the rest will be asserted by Map
if len(kadjA) != len(kadjB) {
panicf("kadj.Mul: different keys:\n\nkadjA: %v\nkadjB: %v", kadjA, kadjB)
}
kadj := make(KAdjMatrix, len(kadjB))
for k, tob := range kadjB {
kadj[k] = kadjA.Map(tob)
}
return kadj
}
// KAdj computes adjacency matrix for t1 -> t2 transition.
//
// The set of keys for which kadj matrix is computed can be optionally provided.
// This set of keys defaults to allTestKeys(t1,t2).
//
// KAdj itself is verified by testΔBTail on entries with .kadjOK set.
func KAdj(t1, t2 *xbtreetest.Commit, keysv ...setKey) (kadj KAdjMatrix) {
// assert KAdj(A,B) == KAdj(B,A)
kadj12 := _KAdj(t1,t2, keysv...)
kadj21 := _KAdj(t2,t1, keysv...)
if !reflect.DeepEqual(kadj12, kadj21) {
panicf("KAdj not symmetric:\nt1: %s\nt2: %s\nkadj12: %v\nkadj21: %v",
t1.Tree, t2.Tree, kadj12, kadj21)
}
return kadj12
}
const debugKAdj = false
func debugfKAdj(format string, argv ...interface{}) {
if debugKAdj {
fmt.Printf(format, argv...)
}
}
func _KAdj(t1, t2 *xbtreetest.Commit, keysv ...setKey) (kadj KAdjMatrix) {
var keys setKey
switch len(keysv) {
case 0:
keys = allTestKeys(t1, t2)
case 1:
keys = keysv[0]
default:
panic("multiple key sets on the call")
}
debugfKAdj("\n\n_KAdj\n")
debugfKAdj("t1: %s\n", t1.Tree)
debugfKAdj("t2: %s\n", t2.Tree)
debugfKAdj("keys: %s\n", keys)
defer func() {
debugfKAdj("kadj -> %v\n", kadj)
}()
// kadj = {} k -> adjacent keys.
// if k is tracked and covered by changed leaf -> changes to adjacents must be in Update(t1->t2).
kadj = KAdjMatrix{}
for k := range keys {
adj1 := setKey{}
adj2 := setKey{}
q1 := &blib.RangedKeySet{}; q1.Add(k)
q2 := &blib.RangedKeySet{}; q2.Add(k)
done1 := &blib.RangedKeySet{}
done2 := &blib.RangedKeySet{}
debugfKAdj("\nk%s\n", kstr(k))
for !q1.Empty() || !q2.Empty() {
debugfKAdj("q1: %s\tdone1: %s\n", q1, done1)
debugfKAdj("q2: %s\tdone2: %s\n", q2, done2)
for _, r1 := range q1.AllRanges() {
lo1 := r1.Lo
for {
b1 := t1.Xkv.Get(lo1)
debugfKAdj(" b1: %s\n", b1)
for k_ := range keys {
if b1.Keycov.Has(k_) {
adj1.Add(k_)
debugfKAdj(" adj1 += %s\t-> %s\n", kstr(k_), adj1)
}
}
done1.AddRange(b1.Keycov)
// q2 |= (b1.keyrange \ done2)
δq2 := &blib.RangedKeySet{}
δq2.AddRange(b1.Keycov)
δq2.DifferenceInplace(done2)
q2.UnionInplace(δq2)
debugfKAdj("q2 += %s\t-> %s\n", δq2, q2)
// continue with next right bucket until r1 coverage is complete
if r1.Hi_ <= b1.Keycov.Hi_ {
break
}
lo1 = b1.Keycov.Hi_ + 1
}
}
q1.Clear()
for _, r2 := range q2.AllRanges() {
lo2 := r2.Lo
for {
b2 := t2.Xkv.Get(lo2)
debugfKAdj(" b2: %s\n", b2)
for k_ := range keys {
if b2.Keycov.Has(k_) {
adj2.Add(k_)
debugfKAdj(" adj2 += %s\t-> %s\n", kstr(k_), adj2)
}
}
done2.AddRange(b2.Keycov)
// q1 |= (b2.keyrange \ done1)
δq1 := &blib.RangedKeySet{}
δq1.AddRange(b2.Keycov)
δq1.DifferenceInplace(done1)
q1.UnionInplace(δq1)
debugfKAdj("q1 += %s\t-> %s\n", δq1, q1)
// continue with next right bucket until r2 coverage is complete
if r2.Hi_ <= b2.Keycov.Hi_ {
break
}
lo2 = b2.Keycov.Hi_ + 1
}
}
q2.Clear()
}
adj := setKey{}; adj.Update(adj1); adj.Update(adj2)
kadj[k] = adj
}
return kadj
}
// ----------------------------------------
// assertΔTtail verifies state of ΔTtail that corresponds to treeRoot in δbtail.
// it also verifies that δbtail.vδBroots matches ΔTtail data.
func assertΔTtail(t *testing.T, subj string, δbtail *ΔBtail, tj *xbtreetest.Commit, treeRoot zodb.Oid, vδTok ...map[Key]Δstring) {
t.Helper()
T := tj.T // TODO better require t to be xbtreetest.T instead
l := len(vδTok)
var vatOK []zodb.Tid
var vδTok_ []map[Key]Δstring
t0 := tj
for i := 0; i<l; i++ {
// empty vδTok entries means they should be absent in vδT
if δTok := vδTok[l-i-1]; len(δTok) != 0 {
vatOK = append([]zodb.Tid{t0.At}, vatOK...)
vδTok_ = append([]map[Key]Δstring{δTok}, vδTok_...)
}
t0 = t0.Prev
}
vδTok = vδTok_
δTtail, ok := δbtail.byRoot[treeRoot]
var vδToid []ΔTree
if ok {
vδToid = δTtail.vδT
}
var vat []zodb.Tid
var vδT []map[Key]Δstring
atPrev := t0.At
for _, δToid := range vδToid {
vat = append(vat, δToid.Rev)
δT := xgetδKV(T.XGetCommit(atPrev), T.XGetCommit(δToid.Rev), δToid.KV) // {} k -> δ(ZBlk(oid).data)
vδT = append(vδT, δT)
atPrev = δToid.Rev
}
var vatδB []zodb.Tid // δbtail.vδBroots/treeRoot
for _, δBroots := range δbtail.vδBroots {
if δBroots.Roots.Has(treeRoot) {
vatδB = append(vatδB, δBroots.Rev)
}
}
tok := tidvEqual(vat, vatOK) && vδTEqual(vδT, vδTok)
bok := tidvEqual(vatδB, vatOK)
if !(tok && bok) {
emsg := fmt.Sprintf("%s: vδT:\n", subj)
have := ""
for i := 0; i<len(vδT); i++ {
have += fmt.Sprintf("\n\t@%s: %v", T.AtSymb(vat[i]), vδT[i])
}
emsg += fmt.Sprintf("have: %s\n", have)
if !tok {
want := ""
for i := 0; i<len(vδTok); i++ {
want += fmt.Sprintf("\n\t@%s: %v", T.AtSymb(vatOK[i]), vδTok[i])
}
emsg += fmt.Sprintf("want: %s\n", want)
}
if !bok {
vδb_root := ""
for i := 0; i<len(vatδB); i++ {
vδb_root += fmt.Sprintf("\n\t@%s", T.AtSymb(vatδB[i]))
}
emsg += fmt.Sprintf("vδb/root: %s\n", vδb_root)
}
t.Error(emsg)
}
}
// assertTrack verifies state of .trackSet and ΔTtail.(k)trackNew.
// it assumes that only one tree root is being tracked.
func (δBtail *ΔBtail) assertTrack(t *testing.T, subj string, trackSetOK blib.PPTreeSubSet, trackNewOK blib.PPTreeSubSet, ktrackNewOK *blib.RangedKeySet) {
t.Helper()
if !δBtail.trackSet.Equal(trackSetOK) {
t.Errorf("%s: trackSet:\n\thave: %v\n\twant: %v", subj, δBtail.trackSet, trackSetOK)
}
roots := setOid{}
for root := range δBtail.byRoot {
roots.Add(root)
}
tEmpty := trackNewOK.Empty()
kEmpty := ktrackNewOK.Empty()
if tEmpty != kEmpty {
t.Errorf("BUG: %s: empty(trackNewOK) != empty(ktrackNewOK)", subj)
return
}
nrootsOK := 1
if trackSetOK.Empty() && tEmpty {
nrootsOK = 0
}
if len(roots) != nrootsOK {
t.Errorf("%s: len(byRoot) != %d ; roots=%v", subj, nrootsOK, roots)
return
}
if nrootsOK == 0 {
return
}
root := roots.Elements()[0]
δTtail := δBtail.byRoot[root]
trackNewRootsOK := setOid{}
if !trackNewOK.Empty() {
trackNewRootsOK.Add(root)
}
if !δBtail.trackNewRoots.Equal(trackNewRootsOK) {
t.Errorf("%s: trackNewRoots:\n\thave: %v\n\twant: %v", subj, δBtail.trackNewRoots, trackNewRootsOK)
}
if !δTtail.trackNew.Equal(trackNewOK) {
t.Errorf("%s: vδT.trackNew:\n\thave: %v\n\twant: %v", subj, δTtail.trackNew, trackNewOK)
}
if !δTtail.ktrackNew.Equal(ktrackNewOK) {
t.Errorf("%s: vδT.ktrackNew:\n\thave: %v\n\twant: %v", subj, δTtail.ktrackNew, ktrackNewOK)
}
}
// trackSet returns what should be ΔBtail.trackSet coverage for specified tracked key set.
func trackSet(rbs xbtreetest.RBucketSet, tracked setKey) blib.PPTreeSubSet {
// nil = don't compute keyCover
// (trackSet is called from inside hot inner loop of rebuild test)
return _trackSetWithCov(rbs, tracked, nil)
}
// trackSetWithCov returns what should be ΔBtail.trackSet and its key coverage for specified tracked key set.
func trackSetWithCov(rbs xbtreetest.RBucketSet, tracked setKey) (trackSet blib.PPTreeSubSet, keyCover *blib.RangedKeySet) {
keyCover = &blib.RangedKeySet{}
trackSet = _trackSetWithCov(rbs, tracked, keyCover)
return trackSet, keyCover
}
func _trackSetWithCov(rbs xbtreetest.RBucketSet, tracked setKey, outKeyCover *blib.RangedKeySet) (trackSet blib.PPTreeSubSet) {
trackSet = blib.PPTreeSubSet{}
for k := range tracked {
kb := rbs.Get(k)
if outKeyCover != nil {
outKeyCover.AddRange(kb.Keycov)
}
trackSet.AddPath(kb.Path())
}
return trackSet
}
// trackKeys issues δbtail.Track requests for tree[keys].
func trackKeys(δbtail *ΔBtail, t *xbtreetest.Commit, keys setKey) {
head := δbtail.Head()
if head != t.At {
panicf("BUG: δbtail.head: %s ; t.at: %s", head, t.At)
}
for k := range keys {
// NOTE: if tree is deleted - the following adds it to tracked
// set with every key being a hole. This aligns with the
// following situation
//
// T1 -> ø -> T2
//
// where after T1->ø, even though the tree becomes deleted, its root
// continues to be tracked and all keys migrate to holes in the
// tracking set. By aligning initial state to the same as after
// T1->ø, we test what will happen on ø->T2.
b := t.Xkv.Get(k)
δbtail.track(b.Path(), b.Keycov)
}
}
// xgetδKV translates {k -> δ<oid>} to {k -> δ(ZBlk(oid).data)} according to t1..t2 db snapshots.
func xgetδKV(t1, t2 *xbtreetest.Commit, δkvOid map[Key]ΔValue) map[Key]Δstring {
δkv := make(map[Key]Δstring, len(δkvOid))
for k, δvOid := range δkvOid {
δkv[k] = Δstring{
Old: t1.XGetBlkData(δvOid.Old),
New: t2.XGetBlkData(δvOid.New),
}
}
return δkv
}
// -------- misc --------
// IntSets generates all sets of integers in range [0,N)
func IntSets(N int) chan []int {
ch := make(chan []int)
go intSets(ch, 0, N)
return ch
}
// intSets generates all sets of integers in range [lo,hi)
func intSets(ch chan []int, lo, hi int) {
ch <- nil // ø
if lo < hi {
for i := lo; i < hi; i++ {
chTail := make(chan []int)
go intSets(chTail, i+1, hi)
for tail := range chTail {
ch <- append([]int{i}, tail...) // i + tail
}
}
}
close(ch)
}
func TestIntSets(t *testing.T) {
got := [][]int{}
for is := range IntSets(3) {
got = append(got, is)
}
I := func(v ...int) []int { return v }
want := [][]int{I(),
I(0), I(0,1), I(0,1,2), I(0,2),
I(1), I(1,2),
I(2),
}
if !reflect.DeepEqual(got, want) {
t.Fatalf("error:\ngot: %v\nwant: %v", got, want)
}
}
// allTestKeys returns all keys from vt + ∞.
func allTestKeys(vt ...*xbtreetest.Commit) setKey {
allKeys := setKey{}; allKeys.Add(KeyMax) // ∞ simulating ZBigFile.Size() query
for _, t := range vt {
for _, b := range t.Xkv {
for k := range b.KV {
allKeys.Add(k)
}
}
}
return allKeys
}
func sortedKeys(kv map[Key]Δstring) []Key {
keyv := []Key{}
for k := range kv {
keyv = append(keyv, k)
}
sort.Slice(keyv, func(i, j int) bool {
return keyv[i] < keyv[j]
})
return keyv
}
func tidvEqual(av, bv []zodb.Tid) bool {
if len(av) != len(bv) {
return false
}
for i, a := range av {
if bv[i] != a {
return false
}
}
return true
}
func vδTEqual(vδa, vδb []map[Key]Δstring) bool {
if len(vδa) != len(vδb) {
return false
}
for i, δa := range vδa {
if !δTEqual(δa, vδb[i]) {
return false
}
}
return true
}
func δTEqual(δa, δb map[Key]Δstring) bool {
if len(δa) != len(δb) {
return false
}
for k, δ := range δa {
δ_, ok := δb[k]
if !ok || δ != δ_ {
return false
}
}
return true
}
// Copyright (C) 2020-2021 Nexedi SA and Contributors.
// Kirill Smelkov <kirr@nexedi.com>
//
// This program is free software: you can Use, Study, Modify and Redistribute
// it under the terms of the GNU General Public License version 3, or (at your
// option) any later version, as published by the Free Software Foundation.
//
// You can also Link and Combine this program with other software covered by
// the terms of any of the Free Software licenses or any of the Open Source
// Initiative approved licenses and Convey the resulting work. Corresponding
// source of such a combination shall include the source code for all other
// software used.
//
// This program is distributed WITHOUT ANY WARRANTY; without even the implied
// warranty of MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.
//
// See COPYING file for full licensing terms.
// See https://www.nexedi.com/licensing for rationale and options.
package xbtree_test
import (
_ "lab.nexedi.com/nexedi/wendelin.core/wcfs/internal/xbtree/xbtreetest/init"
)
// Copyright (C) 2018-2021 Nexedi SA and Contributors.
// Kirill Smelkov <kirr@nexedi.com>
//
// This program is free software: you can Use, Study, Modify and Redistribute
// it under the terms of the GNU General Public License version 3, or (at your
// option) any later version, as published by the Free Software Foundation.
//
// You can also Link and Combine this program with other software covered by
// the terms of any of the Free Software licenses or any of the Open Source
// Initiative approved licenses and Convey the resulting work. Corresponding
// source of such a combination shall include the source code for all other
// software used.
//
// This program is distributed WITHOUT ANY WARRANTY; without even the implied
// warranty of MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.
//
// See COPYING file for full licensing terms.
// See https://www.nexedi.com/licensing for rationale and options.
// Package xtail provides utilities that are common to ΔBtail and ΔFtail.
package xtail
import (
"fmt"
"lab.nexedi.com/kirr/neo/go/zodb"
)
// AssertSlice asserts that δ.tail ≤ lo ≤ hi ≤ δ.head
func AssertSlice(δ interface { Head() zodb.Tid; Tail() zodb.Tid }, lo, hi zodb.Tid) {
tail := δ.Tail()
head := δ.Head()
if !(tail <= lo && lo <= hi && hi <= head) {
panic(fmt.Sprintf("invalid slice: (%s, %s]; (tail, head] = (%s, %s]", lo, hi, tail, head))
}
}
Markdown is supported
0%
or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment