memdb: refactor code of art to make it more understandable (#1508)

Signed-off-by: cfzjywxk <lsswxrxr@163.com>
This commit is contained in:
cfzjywxk 2024-12-16 18:46:15 +08:00 committed by GitHub
parent 0e4728cfaa
commit 43d2db704a
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
8 changed files with 135 additions and 127 deletions

View File

@ -294,13 +294,13 @@ func (hdr *MemdbVlogHdr) load(src []byte) {
}
// AppendValue appends a value and it's vlog header to the vlog.
func (l *MemdbVlog[G, M]) AppendValue(nodeAddr MemdbArenaAddr, oldValue MemdbArenaAddr, value []byte) MemdbArenaAddr {
size := memdbVlogHdrSize + len(value)
func (l *MemdbVlog[G, M]) AppendValue(nodeAddr MemdbArenaAddr, oldValueAddr MemdbArenaAddr, value []byte) MemdbArenaAddr {
prevBlocks := len(l.blocks)
size := memdbVlogHdrSize + len(value)
addr, mem := l.Alloc(size, false)
copy(mem, value)
hdr := MemdbVlogHdr{nodeAddr, oldValue, uint32(len(value))}
hdr := MemdbVlogHdr{nodeAddr, oldValueAddr, uint32(len(value))}
hdr.store(mem[len(value):])
addr.off += uint32(size)

View File

@ -75,11 +75,11 @@ func (t *ART) Get(key []byte) ([]byte, error) {
// 1. search the leaf node.
_, leaf := t.traverse(key, false)
if leaf == nil || leaf.vAddr.IsNull() {
if leaf == nil || leaf.vLogAddr.IsNull() {
return nil, tikverr.ErrNotExist
}
// 2. get the value from the vlog.
return t.allocator.vlogAllocator.GetValue(leaf.vAddr), nil
return t.allocator.vlogAllocator.GetValue(leaf.vLogAddr), nil
}
// GetFlags returns the latest flags associated with key.
@ -88,7 +88,7 @@ func (t *ART) GetFlags(key []byte) (kv.KeyFlags, error) {
if leaf == nil {
return 0, tikverr.ErrNotExist
}
if leaf.vAddr.IsNull() && leaf.isDeleted() {
if leaf.vLogAddr.IsNull() && leaf.isDeleted() {
return 0, tikverr.ErrNotExist
}
return leaf.GetKeyFlags(), nil
@ -184,10 +184,10 @@ func (t *ART) search(key artKey) (arena.MemdbArenaAddr, *artLeaf) {
if node.prefixLen > 0 {
prefixLen := node.match(key, depth)
if prefixLen < min(node.prefixLen, maxPrefixLen) {
if prefixLen < min(node.prefixLen, maxInNodePrefixLen) {
return arena.NullAddr, nil
}
// If node.prefixLen > maxPrefixLen, we optimistically match the prefix here.
// If node.prefixLen > maxInNodePrefixLen, we optimistically match the prefix here.
// False positive is possible, but it's fine since we will check the full artLeaf key at last.
depth += node.prefixLen
}
@ -209,51 +209,51 @@ func (t *ART) recursiveInsert(key artKey) (arena.MemdbArenaAddr, *artLeaf) {
t.root, _ = t.newNode4()
}
depth := uint32(0)
prevDepth := 0
prev := nullArtNode
current := t.root
var node *nodeBase
currentDepth := uint32(0)
prevNode := nullArtNode
currentNode := t.root
var nodeBasePtr *nodeBase
for {
if current.isLeaf() {
return t.expandLeaf(key, depth, prev, current)
if currentNode.isLeaf() {
return t.expandLeafIfNeeded(key, currentDepth, prevNode, currentNode)
}
// inline: performance critical path
// get the basic node information.
switch current.kind {
switch currentNode.kind {
case typeNode4:
node = &current.asNode4(&t.allocator).nodeBase
nodeBasePtr = &currentNode.asNode4(&t.allocator).nodeBase
case typeNode16:
node = &current.asNode16(&t.allocator).nodeBase
nodeBasePtr = &currentNode.asNode16(&t.allocator).nodeBase
case typeNode48:
node = &current.asNode48(&t.allocator).nodeBase
nodeBasePtr = &currentNode.asNode48(&t.allocator).nodeBase
case typeNode256:
node = &current.asNode256(&t.allocator).nodeBase
nodeBasePtr = &currentNode.asNode256(&t.allocator).nodeBase
default:
panic("invalid nodeBase kind")
}
if node.prefixLen > 0 {
mismatchIdx := node.matchDeep(&t.allocator, &current, key, depth)
if mismatchIdx < node.prefixLen {
if nodeBasePtr.prefixLen > 0 {
mismatchIdx := nodeBasePtr.matchDeep(&t.allocator, &currentNode, key, currentDepth)
if mismatchIdx < nodeBasePtr.prefixLen {
// if the prefix doesn't match, we split the node into different prefixes.
return t.expandNode(key, depth, mismatchIdx, prev, current, node)
return t.expandNode(key, currentDepth, mismatchIdx, prevNode, currentNode, nodeBasePtr)
}
depth += node.prefixLen
currentDepth += nodeBasePtr.prefixLen
}
// search next node
valid := key.valid(int(depth))
_, next := current.findChild(&t.allocator, key.charAt(int(depth)), !valid)
valid := key.valid(int(currentDepth))
_, next := currentNode.findChild(&t.allocator, key.charAt(int(currentDepth)), !valid)
if next == nullArtNode {
// insert as leaf if there is no child.
newAn, newLeaf := t.newLeaf(key)
if current.addChild(&t.allocator, key.charAt(int(depth)), !valid, newAn) {
if prev == nullArtNode {
t.root = current
if currentNode.addChild(&t.allocator, key.charAt(int(currentDepth)), !valid, newAn) {
if prevNode == nullArtNode {
t.root = currentNode
} else {
prev.replaceChild(&t.allocator, key.charAt(prevDepth), current)
prevNode.replaceChild(&t.allocator, key.charAt(prevDepth), currentNode)
}
}
return newAn.addr, newLeaf
@ -262,17 +262,18 @@ func (t *ART) recursiveInsert(key artKey) (arena.MemdbArenaAddr, *artLeaf) {
// key is drained, return the leaf.
return next.addr, next.asLeaf(&t.allocator)
}
prev = current
current = next
prevDepth = int(depth)
depth++
prevNode = currentNode
currentNode = next
prevDepth = int(currentDepth)
currentDepth++
continue
}
}
// expandLeaf expands the existing artLeaf to a node4 if the keys are different.
// it returns the addr and leaf of the given key.
func (t *ART) expandLeaf(key artKey, depth uint32, prev, current artNode) (arena.MemdbArenaAddr, *artLeaf) {
// expandLeafIfNeeded expands the existing artLeaf to a node4 if the keys are different.
// Otherwise, the existing leaf node is returned and would be reused.
// It returns the addr and leaf of the given key.
func (t *ART) expandLeafIfNeeded(key artKey, depth uint32, prevNode, currentNode artNode) (arena.MemdbArenaAddr, *artLeaf) {
// Expand the artLeaf to a node4.
//
// ┌────────────┐
@ -286,30 +287,32 @@ func (t *ART) expandLeaf(key artKey, depth uint32, prev, current artNode) (arena
// │ old │ │ new │
// │ leaf1 │ │ leaf2 │
// └─────────┘ └─────────┘
leaf1 := current.asLeaf(&t.allocator)
if depth == 0 {
panic("Invalid expandLeafIfNeeded input with zero depth")
}
leaf1 := currentNode.asLeaf(&t.allocator)
if leaf1.match(depth-1, key) {
// same key, return the artLeaf and overwrite the value.
return current.addr, leaf1
return currentNode.addr, leaf1
}
prevDepth := int(depth - 1)
leaf2Addr, leaf2 := t.newLeaf(key)
l1Key, l2Key := artKey(leaf1.GetKey()), artKey(leaf2.GetKey())
lcp := longestCommonPrefix(l1Key, l2Key, depth)
newLeafNode, leaf2 := t.newLeaf(key)
// calculate the common prefix length of new node.
l1Key, l2Key := artKey(leaf1.GetKey()), artKey(leaf2.GetKey())
lcp := longestCommonPrefix(l1Key, l2Key, depth)
newAn, newN4 := t.newNode4()
newN4.setPrefix(key[depth:], lcp)
depth += lcp
newAn.addChild(&t.allocator, l1Key.charAt(int(depth)), !l1Key.valid(int(depth)), current)
newAn.addChild(&t.allocator, l2Key.charAt(int(depth)), !l2Key.valid(int(depth)), leaf2Addr)
newAn.addChild(&t.allocator, l1Key.charAt(int(depth)), !l1Key.valid(int(depth)), currentNode)
newAn.addChild(&t.allocator, l2Key.charAt(int(depth)), !l2Key.valid(int(depth)), newLeafNode)
// swap the old leaf with the new node4.
prev.replaceChild(&t.allocator, key.charAt(prevDepth), newAn)
return leaf2Addr.addr, leaf2
prevNode.replaceChild(&t.allocator, key.charAt(prevDepth), newAn)
return newLeafNode.addr, leaf2
}
func (t *ART) expandNode(key artKey, depth, mismatchIdx uint32, prev, current artNode, currNode *nodeBase) (arena.MemdbArenaAddr, *artLeaf) {
func (t *ART) expandNode(key artKey, depth, mismatchIdx uint32, prevNode, currentNode artNode, nodeBasePtr *nodeBase) (arena.MemdbArenaAddr, *artLeaf) {
// prefix mismatch, create a new parent node which has a shorter prefix.
// example of insert "acc" into node with "abc prefix:
// ┌────────────┐
@ -330,24 +333,24 @@ func (t *ART) expandNode(key artKey, depth, mismatchIdx uint32, prev, current ar
// update prefix for old node and move it as a child of the new node.
var prefix artKey
if currNode.prefixLen <= maxPrefixLen {
// node.prefixLen <= maxPrefixLen means all the prefix is in the prefix array.
if nodeBasePtr.prefixLen <= maxInNodePrefixLen {
// The node.prefixLen <= maxInNodePrefixLen means all the prefix is in the prefix array.
// The char at mismatchIdx will be stored in the index of new node.
prefix = currNode.prefix[:]
prefix = nodeBasePtr.prefix[:]
} else {
// Unless, we need to find the prefix in the leaf.
// Any leaves in the node should have the same prefix, we use minimum node here.
leafArtNode := minimum(&t.allocator, current)
prefix = leafArtNode.asLeaf(&t.allocator).GetKey()[depth : depth+currNode.prefixLen]
// Otherwise, we need to find the prefix in the leaf.
// Any leaves in the node should have the same prefix, we use minimumLeafNode node here.
leafArtNode := minimumLeafNode(&t.allocator, currentNode)
prefix = leafArtNode.asLeaf(&t.allocator).GetKey()[depth : depth+nodeBasePtr.prefixLen]
}
nodeChar := prefix[mismatchIdx]
currNode.setPrefix(prefix[mismatchIdx+1:], currNode.prefixLen-mismatchIdx-1)
newAn.addChild(&t.allocator, nodeChar, false, current)
nodeBasePtr.setPrefix(prefix[mismatchIdx+1:], nodeBasePtr.prefixLen-mismatchIdx-1)
newAn.addChild(&t.allocator, nodeChar, false, currentNode)
// insert the artLeaf into new node
// Insert the new artLeaf node.
newLeafAddr, newLeaf := t.newLeaf(key)
newAn.addChild(&t.allocator, key.charAt(int(depth+mismatchIdx)), !key.valid(int(depth+mismatchIdx)), newLeafAddr)
prev.replaceChild(&t.allocator, key.charAt(prevDepth), newAn)
prevNode.replaceChild(&t.allocator, key.charAt(prevDepth), newAn)
return newLeafAddr.addr, newLeaf
}
@ -363,9 +366,9 @@ func (t *ART) newLeaf(key artKey) (artNode, *artLeaf) {
func (t *ART) setValue(addr arena.MemdbArenaAddr, l *artLeaf, value []byte, ops []kv.FlagsOp) {
flags := l.GetKeyFlags()
if flags == 0 && l.vAddr.IsNull() || l.isDeleted() {
if flags == 0 && l.vLogAddr.IsNull() || l.isDeleted() {
t.len++
t.size += int(l.klen)
t.size += int(l.keyLen)
}
if value != nil {
flags = kv.ApplyFlagsOps(flags, append([]kv.FlagsOp{kv.DelNeedConstraintCheckInPrewrite}, ops...)...)
@ -381,13 +384,13 @@ func (t *ART) setValue(addr arena.MemdbArenaAddr, l *artLeaf, value []byte, ops
// value == nil means it updates flags only.
return
}
oldSize, swapper := t.trySwapValue(l.vAddr, value)
oldSize, swapper := t.trySwapValue(l.vLogAddr, value)
if swapper {
return
}
t.size += len(value) - oldSize
vAddr := t.allocator.vlogAllocator.AppendValue(addr, l.vAddr, value)
l.vAddr = vAddr
vAddr := t.allocator.vlogAllocator.AppendValue(addr, l.vLogAddr, value)
l.vLogAddr = vAddr
}
// trySwapValue checks if the value can be updated in place.
@ -438,7 +441,7 @@ func (t *ART) RevertVAddr(hdr *arena.MemdbVlogHdr) {
if lf == nil {
panic("revert an invalid node")
}
lf.vAddr = hdr.OldValue
lf.vLogAddr = hdr.OldValue
t.size -= int(hdr.ValueLen)
if hdr.OldValue.IsNull() {
keptFlags := lf.GetKeyFlags()
@ -446,7 +449,7 @@ func (t *ART) RevertVAddr(hdr *arena.MemdbVlogHdr) {
if keptFlags == 0 {
lf.markDelete()
t.len--
t.size -= int(lf.klen)
t.size -= int(lf.keyLen)
} else {
lf.setKeyFlags(keptFlags)
}
@ -457,7 +460,7 @@ func (t *ART) RevertVAddr(hdr *arena.MemdbVlogHdr) {
func (t *ART) InspectNode(addr arena.MemdbArenaAddr) (*artLeaf, arena.MemdbArenaAddr) {
lf := t.allocator.getLeaf(addr)
return lf, lf.vAddr
return lf, lf.vLogAddr
}
// IsStaging returns whether the MemBuffer is in staging status.
@ -562,11 +565,11 @@ func (t *ART) SelectValueHistory(key []byte, predicate func(value []byte) bool)
if x == nil {
return nil, tikverr.ErrNotExist
}
if x.vAddr.IsNull() {
if x.vLogAddr.IsNull() {
// A flags only key, act as value not exists
return nil, tikverr.ErrNotExist
}
result := t.allocator.vlogAllocator.SelectValueHistory(x.vAddr, func(addr arena.MemdbArenaAddr) bool {
result := t.allocator.vlogAllocator.SelectValueHistory(x.vLogAddr, func(addr arena.MemdbArenaAddr) bool {
return predicate(t.allocator.vlogAllocator.GetValue(addr))
})
if result.IsNull() {
@ -600,10 +603,10 @@ func (t *ART) GetValueByHandle(handle arena.MemKeyHandle) ([]byte, bool) {
return nil, false
}
lf := t.allocator.getLeaf(handle.ToAddr())
if lf.vAddr.IsNull() {
if lf.vLogAddr.IsNull() {
return nil, false
}
return t.allocator.vlogAllocator.GetValue(lf.vAddr), true
return t.allocator.vlogAllocator.GetValue(lf.vLogAddr), true
}
// GetEntrySizeLimit gets the size limit for each entry and total buffer.

View File

@ -167,9 +167,9 @@ func (f *artAllocator) allocLeaf(key []byte) (arena.MemdbArenaAddr, *artLeaf) {
size := leafSize + len(key)
addr, data := f.nodeAllocator.Alloc(size, true)
lf := (*artLeaf)(unsafe.Pointer(&data[0]))
lf.klen = uint16(len(key))
lf.keyLen = uint16(len(key))
lf.flags = 0
lf.vAddr = arena.NullAddr
lf.vLogAddr = arena.NullAddr
copy(data[leafSize:], key)
return addr, lf
}

View File

@ -82,10 +82,10 @@ func (it *Iterator) Valid() bool { return it.valid }
func (it *Iterator) Key() []byte { return it.currLeaf.GetKey() }
func (it *Iterator) Flags() kv.KeyFlags { return it.currLeaf.GetKeyFlags() }
func (it *Iterator) Value() []byte {
if it.currLeaf.vAddr.IsNull() {
if it.currLeaf.vLogAddr.IsNull() {
return nil
}
return it.tree.allocator.vlogAllocator.GetValue(it.currLeaf.vAddr)
return it.tree.allocator.vlogAllocator.GetValue(it.currLeaf.vLogAddr)
}
// HasValue returns false if it is flags only.
@ -94,7 +94,7 @@ func (it *Iterator) HasValue() bool {
}
func (it *Iterator) isFlagsOnly() bool {
return it.currLeaf != nil && it.currLeaf.vAddr.IsNull()
return it.currLeaf != nil && it.currLeaf.vLogAddr.IsNull()
}
func (it *Iterator) Next() error {
@ -119,7 +119,7 @@ func (it *Iterator) Next() error {
return nil
}
it.setCurrLeaf(nextLeaf.addr)
if it.currLeaf.vAddr.IsNull() {
if it.currLeaf.vLogAddr.IsNull() {
// if it.includeFlags is true, the iterator should return even the value is null.
if it.includeFlags && !it.currLeaf.isDeleted() {
return nil
@ -251,10 +251,10 @@ func (it *baseIter) seek(root artNode, key artKey) {
// If the seek key is smaller than the prefix, all the children are located on the right side of the seek key.
// Otherwise, the children are located on the left side of the seek key.
var prefix []byte
if mismatchIdx < maxPrefixLen {
if mismatchIdx < maxInNodePrefixLen {
prefix = node.prefix[:]
} else {
leafNode := minimum(it.allocator, curr)
leafNode := minimumLeafNode(it.allocator, curr)
prefix = leafNode.asLeaf(it.allocator).getKeyDepth(depth)
}
if mismatchIdx+depth == uint32(len(key)) || key[depth+mismatchIdx] < prefix[mismatchIdx] {

View File

@ -16,6 +16,7 @@ package art
import (
"bytes"
"fmt"
"math"
"math/bits"
"runtime"
@ -39,11 +40,12 @@ const (
)
const (
maxPrefixLen = 20
node4cap = 4
node16cap = 16
node48cap = 48
node256cap = 256
// maxInNodePrefixLen is the maximum length of the prefix stored within an art node.
maxInNodePrefixLen = 20
node4cap = 4
node16cap = 16
node48cap = 48
node256cap = 256
// inplaceIndex is a special index to indicate the index of an in-place leaf in a node,
// the in-place leaf has the same key with its parent node and doesn't occupy the quota of the node.
// the other valid index of a node is [0, nodeNum), all the other leaves in the node have larger key than the in-place leaf.
@ -71,7 +73,7 @@ type artNode struct {
type nodeBase struct {
nodeNum uint8
prefixLen uint32
prefix [maxPrefixLen]byte
prefix [maxInNodePrefixLen]byte
inplaceLeaf artNode
}
@ -124,9 +126,9 @@ type node256 struct {
const MaxKeyLen = math.MaxUint16
type artLeaf struct {
vAddr arena.MemdbArenaAddr
klen uint16
flags uint16
vLogAddr arena.MemdbArenaAddr
keyLen uint16
flags uint16
}
func (an *artNode) isLeaf() bool {
@ -260,13 +262,13 @@ func (k artKey) valid(pos int) bool {
// GetKey gets the full key of the leaf
func (l *artLeaf) GetKey() []byte {
base := unsafe.Add(unsafe.Pointer(l), leafSize)
return unsafe.Slice((*byte)(base), int(l.klen))
return unsafe.Slice((*byte)(base), int(l.keyLen))
}
// getKeyDepth gets the partial key start from depth of the artLeaf
func (l *artLeaf) getKeyDepth(depth uint32) []byte {
base := unsafe.Add(unsafe.Pointer(l), leafSize+int(depth))
return unsafe.Slice((*byte)(base), int(l.klen)-int(depth))
return unsafe.Slice((*byte)(base), int(l.keyLen)-int(depth))
}
func (l *artLeaf) match(depth uint32, key artKey) bool {
@ -304,30 +306,33 @@ func (l *artLeaf) isDeleted() bool {
// node methods
func (n *nodeBase) setPrefix(key artKey, prefixLen uint32) {
n.prefixLen = prefixLen
copy(n.prefix[:], key[:min(prefixLen, maxPrefixLen)])
copy(n.prefix[:], key[:min(prefixLen, maxInNodePrefixLen)])
}
// match returns the mismatch index of the key and the node's prefix within maxPrefixLen.
// Node if the nodeBase.prefixLen > maxPrefixLen and the returned mismatch index equals to maxPrefixLen,
// key[maxPrefixLen:] will not be checked by this function.
// match returns the mismatch index of the key and the node's prefix within maxInNodePrefixLen.
// Node if the nodeBase.prefixLen > maxInNodePrefixLen and the returned mismatch index equals to maxInNodePrefixLen,
// key[maxInNodePrefixLen:] will not be checked by this function.
func (n *nodeBase) match(key artKey, depth uint32) uint32 /* mismatch index */ {
return longestCommonPrefix(key[depth:], n.prefix[:min(n.prefixLen, maxPrefixLen)], 0)
return longestCommonPrefix(key[depth:], n.prefix[:min(n.prefixLen, maxInNodePrefixLen)], 0)
}
// matchDeep returns the mismatch index of the key and the node's prefix.
// If the key is fully match, the mismatch index is equal to the nodeBase.prefixLen.
// The nodeBase only stores prefix within maxPrefixLen, if it's not enough for matching,
// The nodeBase only stores prefix within maxInNodePrefixLen, if it's not enough for matching,
// the matchDeep func looks up and match by the leaf, this function always returns the actual mismatch index.
func (n *nodeBase) matchDeep(a *artAllocator, an *artNode, key artKey, depth uint32) uint32 /* mismatch index */ {
// match in-node prefix
// Match in-node prefix, if the whole prefix is longer than maxInNodePrefixLen, we need to match deeper.
mismatchIdx := n.match(key, depth)
if mismatchIdx < maxPrefixLen || n.prefixLen <= maxPrefixLen {
if mismatchIdx < maxInNodePrefixLen || n.prefixLen <= maxInNodePrefixLen {
return mismatchIdx
}
// if the prefixLen is longer maxPrefixLen and mismatchIdx == maxPrefixLen, we need to match deeper with any leaf.
leafArtNode := minimum(a, *an)
if mismatchIdx > maxInNodePrefixLen {
panic(fmt.Sprintf("matchDeep invalid state, the mismatchIdx=%v and maxInNodePrefixLen=%v", mismatchIdx, maxInNodePrefixLen))
}
// If the whole prefix is longer maxInNodePrefixLen and mismatchIdx == maxInNodePrefixLen, we need to match deeper with any leaf.
leafArtNode := minimumLeafNode(a, *an)
lKey := leafArtNode.asLeaf(a).GetKey()
return longestCommonPrefix(lKey, key, depth+maxPrefixLen) + maxPrefixLen
return longestCommonPrefix(lKey, key, depth+maxInNodePrefixLen) + maxInNodePrefixLen
}
func (an *artNode) asNode4(a *artAllocator) *node4 {
@ -401,8 +406,8 @@ func longestCommonPrefixByChunk(l1Key, l2Key artKey, depth uint32) uint32 {
return idx - depth
}
// Find the minimum artLeaf under an artNode
func minimum(a *artAllocator, an artNode) artNode {
// Find the minimumLeafNode artLeaf under an artNode
func minimumLeafNode(a *artAllocator, an artNode) artNode {
for {
switch an.kind {
case typeLeaf:

View File

@ -42,7 +42,7 @@ func TestAllocNode(t *testing.T) {
require.NotNil(t, n4)
checkNodeInitialization(t, n4)
n4.nodeNum = uint8(i % 4)
n4.prefixLen = uint32(i % maxPrefixLen)
n4.prefixLen = uint32(i % maxInNodePrefixLen)
n4s = append(n4s, addr)
}
@ -54,7 +54,7 @@ func TestAllocNode(t *testing.T) {
require.NotNil(t, n16)
checkNodeInitialization(t, n16)
n16.nodeNum = uint8(i % 16)
n16.prefixLen = uint32(i % maxPrefixLen)
n16.prefixLen = uint32(i % maxInNodePrefixLen)
n16s = append(n16s, addr)
}
@ -66,7 +66,7 @@ func TestAllocNode(t *testing.T) {
require.NotNil(t, n48)
checkNodeInitialization(t, n48)
n48.nodeNum = uint8(i % 48)
n48.prefixLen = uint32(i % maxPrefixLen)
n48.prefixLen = uint32(i % maxInNodePrefixLen)
n48s = append(n48s, addr)
}
@ -78,7 +78,7 @@ func TestAllocNode(t *testing.T) {
require.NotNil(t, n256)
checkNodeInitialization(t, n256)
n256.nodeNum = uint8(i % 256)
n256.prefixLen = uint32(i % maxPrefixLen)
n256.prefixLen = uint32(i % maxInNodePrefixLen)
n256s = append(n256s, addr)
}
@ -97,22 +97,22 @@ func TestAllocNode(t *testing.T) {
for i, addr := range n4s {
n4 := allocator.getNode4(addr)
require.Equal(t, uint8(i%4), n4.nodeNum, i)
require.Equal(t, uint32(i%maxPrefixLen), n4.prefixLen, i)
require.Equal(t, uint32(i%maxInNodePrefixLen), n4.prefixLen, i)
}
for i, addr := range n16s {
n16 := allocator.getNode16(addr)
require.Equal(t, uint8(i%16), n16.nodeNum)
require.Equal(t, uint32(i%maxPrefixLen), n16.prefixLen, i)
require.Equal(t, uint32(i%maxInNodePrefixLen), n16.prefixLen, i)
}
for i, addr := range n48s {
n48 := allocator.getNode48(addr)
require.Equal(t, uint8(i%48), n48.nodeNum)
require.Equal(t, uint32(i%maxPrefixLen), n48.prefixLen, i)
require.Equal(t, uint32(i%maxInNodePrefixLen), n48.prefixLen, i)
}
for i, addr := range n256s {
n256 := allocator.getNode256(addr)
require.Equal(t, uint8(i%256), n256.nodeNum)
require.Equal(t, uint32(i%maxPrefixLen), n256.prefixLen, i)
require.Equal(t, uint32(i%maxInNodePrefixLen), n256.prefixLen, i)
}
for i, addr := range leafs {
key := []byte(strconv.Itoa(i))
@ -142,15 +142,15 @@ func TestNodePrefix(t *testing.T) {
require.Equal(t, uint32(5), idx)
// deep match
leafKey := append(make([]byte, maxPrefixLen), []byte{1, 2, 3, 4, 5}...)
leafKey := append(make([]byte, maxInNodePrefixLen), []byte{1, 2, 3, 4, 5}...)
leafAddr, _ := allocator.allocLeaf(leafKey)
an.addChild(&allocator, 2, false, artNode{kind: typeLeaf, addr: leafAddr})
// the real prefix is [0, ..., 0, 1], but the node.prefix only store [0, ..., 0]
n.setPrefix(leafKey, maxPrefixLen+1)
matchKey := append(make([]byte, maxPrefixLen), []byte{1, 22, 33, 44, 55}...)
mismatchKey := append(make([]byte, maxPrefixLen), []byte{11, 22, 33, 44, 55}...)
require.Equal(t, uint32(maxPrefixLen+1), n.matchDeep(&allocator, an, matchKey, 0))
require.Equal(t, uint32(maxPrefixLen), n.matchDeep(&allocator, an, mismatchKey, 0))
n.setPrefix(leafKey, maxInNodePrefixLen+1)
matchKey := append(make([]byte, maxInNodePrefixLen), []byte{1, 22, 33, 44, 55}...)
mismatchKey := append(make([]byte, maxInNodePrefixLen), []byte{11, 22, 33, 44, 55}...)
require.Equal(t, uint32(maxInNodePrefixLen+1), n.matchDeep(&allocator, an, matchKey, 0))
require.Equal(t, uint32(maxInNodePrefixLen), n.matchDeep(&allocator, an, mismatchKey, 0))
// deep match with depth
leafKey = append(make([]byte, 10), leafKey...)
@ -158,9 +158,9 @@ func TestNodePrefix(t *testing.T) {
mismatchKey = append(make([]byte, 10), mismatchKey...)
leafAddr, _ = allocator.allocLeaf(leafKey)
an.replaceChild(&allocator, 2, artNode{kind: typeLeaf, addr: leafAddr})
n.setPrefix(leafKey[10:], maxPrefixLen+1)
require.Equal(t, uint32(maxPrefixLen+1), n.matchDeep(&allocator, an, matchKey, 10))
require.Equal(t, uint32(maxPrefixLen), n.matchDeep(&allocator, an, mismatchKey, 10))
n.setPrefix(leafKey[10:], maxInNodePrefixLen+1)
require.Equal(t, uint32(maxInNodePrefixLen+1), n.matchDeep(&allocator, an, matchKey, 10))
require.Equal(t, uint32(maxInNodePrefixLen), n.matchDeep(&allocator, an, mismatchKey, 10))
}
addr, n4 := allocator.allocNode4()
@ -397,7 +397,7 @@ func TestMinimumNode(t *testing.T) {
lfAddr, _ := allocator.allocLeaf([]byte{char})
lfNode := artNode{kind: typeLeaf, addr: lfAddr}
node.addChild(&allocator, char, false, lfNode)
minNode := minimum(&allocator, node)
minNode := minimumLeafNode(&allocator, node)
require.Equal(t, typeLeaf, minNode.kind)
require.Equal(t, lfAddr, minNode.addr)
}
@ -405,7 +405,7 @@ func TestMinimumNode(t *testing.T) {
lfAddr, _ := allocator.allocLeaf([]byte{0})
lfNode := artNode{kind: typeLeaf, addr: lfAddr}
node.addChild(&allocator, 0, true, lfNode)
minNode := minimum(&allocator, node)
minNode := minimumLeafNode(&allocator, node)
require.Equal(t, typeLeaf, minNode.kind)
require.Equal(t, lfAddr, minNode.addr)
}

View File

@ -80,11 +80,11 @@ func (snap *SnapGetter) Get(ctx context.Context, key []byte) ([]byte, error) {
if addr.IsNull() {
return nil, tikverr.ErrNotExist
}
if lf.vAddr.IsNull() {
if lf.vLogAddr.IsNull() {
// A flags only key, act as value not exists
return nil, tikverr.ErrNotExist
}
v, ok := snap.tree.allocator.vlogAllocator.GetSnapshotValue(lf.vAddr, &snap.cp)
v, ok := snap.tree.allocator.vlogAllocator.GetSnapshotValue(lf.vLogAddr, &snap.cp)
if !ok {
return nil, tikverr.ErrNotExist
}
@ -125,7 +125,7 @@ func (i *SnapIter) setValue() bool {
if !i.Valid() {
return false
}
if v, ok := i.tree.allocator.vlogAllocator.GetSnapshotValue(i.currLeaf.vAddr, &i.cp); ok {
if v, ok := i.tree.allocator.vlogAllocator.GetSnapshotValue(i.currLeaf.vLogAddr, &i.cp); ok {
i.value = v
return true
}

View File

@ -197,7 +197,7 @@ func TestSearchOptimisticMismatch(t *testing.T) {
prefix := make([]byte, 22)
tree.Set(append(prefix, []byte{1}...), prefix)
tree.Set(append(prefix, []byte{2}...), prefix)
// the search key is matched within maxPrefixLen, but the full key is not matched.
// the search key is matched within maxInNodePrefixLen, but the full key is not matched.
_, err := tree.Get(append(make([]byte, 21), []byte{1, 1}...))
require.NotNil(t, err)
}
@ -205,7 +205,7 @@ func TestSearchOptimisticMismatch(t *testing.T) {
func TestExpansion(t *testing.T) {
// expand leaf
tree := New()
prefix := make([]byte, maxPrefixLen)
prefix := make([]byte, maxInNodePrefixLen)
tree.Set(append(prefix, []byte{1, 1, 1, 1}...), []byte{1})
an := tree.root.asNode4(&tree.allocator).children[0]
require.Equal(t, an.kind, typeLeaf)