bkd

package
v0.0.0-...-53ff736 Latest Latest
Warning

This package is not in the latest version of its module.

Go to latest
Published: Mar 27, 2024 License: Apache-2.0 Imports: 23 Imported by: 1

README

BKD Tree

Block KD-tree, implementing the generic spatial data structure described in this paper.

Documentation

Index

Constants

View Source
const (
	// DEFAULT_MAX_POINTS_IN_LEAF_NODE Default maximum number of point in each leaf block
	DEFAULT_MAX_POINTS_IN_LEAF_NODE = 512

	// MAX_DIMS Maximum number of index dimensions (2 * max index dimensions)
	MAX_DIMS = 16

	// MAX_INDEX_DIMS Maximum number of index dimensions
	MAX_INDEX_DIMS = 8
)
View Source
const (
	HISTOGRAM_SIZE          = 256      // size of the histogram
	MAX_SIZE_OFFLINE_BUFFER = 1024 * 8 // size of the online buffer: 8 KB
	INTEGER_BYTES           = 4
)
View Source
const (
	CODEC_NAME                     = "BKD"
	VERSION_START                  = 4
	VERSION_LEAF_STORES_BOUNDS     = 5
	VERSION_SELECTIVE_INDEXING     = 6
	VERSION_LOW_CARDINALITY_LEAVES = 7
	VERSION_META_FILE              = 9
	VERSION_CURRENT                = VERSION_META_FILE

	// SPLITS_BEFORE_EXACT_BOUNDS Number of splits before we compute the exact bounding box of an inner node.
	SPLITS_BEFORE_EXACT_BOUNDS = 4

	// DEFAULT_MAX_MB_SORT_IN_HEAP Default maximum heap to use, before spilling to (slower) disk
	DEFAULT_MAX_MB_SORT_IN_HEAP = 16.0
)

Variables

This section is empty.

Functions

func Mismatch

func Mismatch(a, b []byte) int

Mismatch 返回a、b两个字节数组 * 第一个不相同字节的索引 * 完全一致则返回-1, * 返回较短的数组的长度

func ReadInts

func ReadInts(ctx context.Context, in store.DataInput, count int, docIDs []int) error

ReadInts Read count integers into docIDs.

func ReadIntsVisitor

func ReadIntsVisitor(ctx context.Context, in store.IndexInput, count int, visitor types.IntersectVisitor) error

ReadIntsVisitor Read count integers and feed the result directly to org.apache.lucene.index.PointValues.IntersectVisitor.visit(int).

func Subtract

func Subtract(bytesPerDim, dim int, a, b, result []byte) error

Subtract Result = a - b, where a >= b, else IllegalArgumentException is thrown.

func WriteDocIds

func WriteDocIds(ctx context.Context, docIds []int, out store.DataOutput) error

Types

type Config

type Config struct {
	// contains filtered or unexported fields
}

Config Basic parameters for indexing points on the BKD tree.

func NewConfig

func NewConfig(numDims, numIndexDims, bytesPerDim, maxPointsInLeafNode int) (*Config, error)

func (*Config) BytesPerDim

func (c *Config) BytesPerDim() int

func (*Config) BytesPerDoc

func (c *Config) BytesPerDoc() int

func (*Config) MaxPointsInLeafNode

func (c *Config) MaxPointsInLeafNode() int

func (*Config) NumDims

func (c *Config) NumDims() int

func (*Config) NumIndexDims

func (c *Config) NumIndexDims() int

func (*Config) PackedBytesLength

func (c *Config) PackedBytesLength() int

func (*Config) PackedIndexBytesLength

func (c *Config) PackedIndexBytesLength() int

type HeapPointReader

type HeapPointReader struct {
	// contains filtered or unexported fields
}

HeapPointReader Utility class to read buffered points from in-heap arrays. lucene.internal

func NewHeapPointReader

func NewHeapPointReader(config *Config, block []byte, start, end int) *HeapPointReader

func (*HeapPointReader) Close

func (h *HeapPointReader) Close() error

func (*HeapPointReader) Next

func (h *HeapPointReader) Next() (bool, error)

func (*HeapPointReader) PointValue

func (h *HeapPointReader) PointValue() PointValue

type HeapPointValue

type HeapPointValue struct {
	// contains filtered or unexported fields
}

HeapPointValue Reusable implementation for a point value on-heap

func NewHeapPointValue

func NewHeapPointValue(config *Config, value []byte) *HeapPointValue

func (*HeapPointValue) DocID

func (h *HeapPointValue) DocID() int

func (*HeapPointValue) PackedValue

func (h *HeapPointValue) PackedValue() []byte

func (*HeapPointValue) PackedValueDocIDBytes

func (h *HeapPointValue) PackedValueDocIDBytes() []byte

func (*HeapPointValue) SetOffset

func (h *HeapPointValue) SetOffset(offset int)

type HeapPointWriter

type HeapPointWriter struct {
	// contains filtered or unexported fields
}

HeapPointWriter Utility class to write new points into in-heap arrays. lucene.internal

func NewHeapPointWriter

func NewHeapPointWriter(config *Config, size int) *HeapPointWriter

func (*HeapPointWriter) Append

func (h *HeapPointWriter) Append(ctx context.Context, packedValue []byte, docID int) error

func (*HeapPointWriter) AppendPoint

func (h *HeapPointWriter) AppendPoint(pointValue PointValue) error

func (*HeapPointWriter) Close

func (h *HeapPointWriter) Close() error

func (*HeapPointWriter) ComputeCardinality

func (h *HeapPointWriter) ComputeCardinality(form, to int, commonPrefixLengths []int) int

func (*HeapPointWriter) Count

func (h *HeapPointWriter) Count() int

func (*HeapPointWriter) Destroy

func (h *HeapPointWriter) Destroy() error

func (*HeapPointWriter) GetPackedValueSlice

func (h *HeapPointWriter) GetPackedValueSlice(index int) PointValue

func (*HeapPointWriter) GetReader

func (h *HeapPointWriter) GetReader(start, length int) (PointReader, error)

func (*HeapPointWriter) Swap

func (h *HeapPointWriter) Swap(i, j int)

type IndexTree

type IndexTree struct {
	// contains filtered or unexported fields
}

IndexTree Used to walk the off-heap index. The format takes advantage of the limited access pattern to the BKD tree at search time, i.e. starting at the root node and recursing downwards one child at a time. 用于遍历堆外索引。该格式利用了在搜索时对BKD树的有限访问模式,即从根节点开始,一次向下递归一个子节点。 lucene.internal

func (*IndexTree) Clone

func (t *IndexTree) Clone() *IndexTree

func (*IndexTree) GetLeafBlockFP

func (t *IndexTree) GetLeafBlockFP() int64

GetLeafBlockFP Only valid after pushLeft or pushRight, not pop!

func (*IndexTree) GetNodeID

func (t *IndexTree) GetNodeID() int

func (*IndexTree) GetSplitDim

func (t *IndexTree) GetSplitDim() int

GetSplitDim Only valid after pushLeft or pushRight, not pop!

func (*IndexTree) GetSplitDimValue

func (t *IndexTree) GetSplitDimValue() []byte

GetSplitDimValue Only valid after pushLeft or pushRight, not pop!

func (*IndexTree) GetSplitPackedValue

func (t *IndexTree) GetSplitPackedValue() []byte

func (*IndexTree) IsLeafNode

func (t *IndexTree) IsLeafNode() bool

func (*IndexTree) NodeExists

func (t *IndexTree) NodeExists() bool

func (*IndexTree) Pop

func (t *IndexTree) Pop()

func (*IndexTree) PushLeft

func (t *IndexTree) PushLeft(ctx context.Context) error

func (*IndexTree) PushRight

func (t *IndexTree) PushRight(ctx context.Context) error

type IntersectState

type IntersectState struct {
	// contains filtered or unexported fields
}

IntersectState Used to track all state for a single call to intersect. 用于跟踪要相交的单个调用的所有状态。

func NewIntersectState

func NewIntersectState(in store.IndexInput, config *Config,
	visitor types.IntersectVisitor, indexVisitor *IndexTree) *IntersectState

type IntroSorter

type IntroSorter struct {
	// contains filtered or unexported fields
}

func NewIntroSorter

func NewIntroSorter(config *Config, sortedDim int, commonPrefixLengths []int, reader types.MutablePointValues, from int, to int, scratch1 *bytes.Buffer, scratch2 *bytes.Buffer) *IntroSorter

func (*IntroSorter) Len

func (r *IntroSorter) Len() int

func (*IntroSorter) Less

func (r *IntroSorter) Less(i, j int) bool

func (*IntroSorter) Swap

func (r *IntroSorter) Swap(i, j int)

type LeafNodes

type LeafNodes interface {
	// NumLeaves number of leaf nodes
	// 叶子节点的数量
	NumLeaves() int

	// GetLeafLP pointer to the leaf node previously written.
	// Leaves are order from left to right,
	// so leaf at index 0 is the leftmost leaf and the the leaf at numleaves() -1 is the rightmost leaf
	// 指向先前写入的叶节点的指针。叶子是从左到右的顺序,所以索引0处的叶子是最左边的叶子,NumLeaves()-1 处的叶子则是最右边的叶子
	GetLeafLP(index int) int64

	// GetSplitValue split value between two leaves.
	// The split value at position n corresponds to the leaves at (n -1) and n.
	// 两片叶子之间的分割值。位置n处的分割值对应于(n-1)和n处的叶子。
	GetSplitValue(index int) []byte

	// GetSplitDimension split dimension between two leaves.
	// The split dimension at position n corresponds to the leaves at (n -1) and n.
	// 两片叶子之间的分割的维度。位置n处的分割尺寸对应于(n-1)和n处的叶子。
	GetSplitDimension(index int) int
}

LeafNodes flat representation of a kd-tree

type MergeQueue

type MergeQueue struct {
	structure.PriorityQueue[MergeReader]
	// contains filtered or unexported fields
}

type MergeReader

type MergeReader struct {
	// contains filtered or unexported fields
}

func (*MergeReader) Next

func (m *MergeReader) Next(ctx context.Context) (bool, error)

type MutablePointValuesSorter

type MutablePointValuesSorter struct {
	// contains filtered or unexported fields
}

func NewMutablePointValuesSorter

func NewMutablePointValuesSorter(config *Config, maxDoc int, reader types.MutablePointValues,
	from, to int) *MutablePointValuesSorter

func (*MutablePointValuesSorter) Len

func (m *MutablePointValuesSorter) Len() int

func (*MutablePointValuesSorter) Less

func (m *MutablePointValuesSorter) Less(i, j int) bool

func (*MutablePointValuesSorter) Swap

func (m *MutablePointValuesSorter) Swap(i, j int)

type OfflinePointReader

type OfflinePointReader struct {
	// contains filtered or unexported fields
}

func NewOfflinePointReader

func NewOfflinePointReader(config *Config, tempDir store.Directory,
	tempFileName string, start, length int, reusableBuffer []byte) (*OfflinePointReader, error)

func (*OfflinePointReader) Close

func (r *OfflinePointReader) Close() error

func (*OfflinePointReader) Next

func (r *OfflinePointReader) Next() (bool, error)

func (*OfflinePointReader) PointValue

func (r *OfflinePointReader) PointValue() PointValue

type OfflinePointValue

type OfflinePointValue struct {
	// contains filtered or unexported fields
}

OfflinePointValue Reusable implementation for a point value offline

func NewOfflinePointValue

func NewOfflinePointValue(config *Config, value []byte) *OfflinePointValue

func (*OfflinePointValue) DocID

func (v *OfflinePointValue) DocID() int

func (*OfflinePointValue) PackedValue

func (v *OfflinePointValue) PackedValue() []byte

func (*OfflinePointValue) PackedValueDocIDBytes

func (v *OfflinePointValue) PackedValueDocIDBytes() []byte

func (*OfflinePointValue) SetOffset

func (v *OfflinePointValue) SetOffset(offset int)

SetOffset Sets a new value by changing the offset.

type OfflinePointWriter

type OfflinePointWriter struct {
	// contains filtered or unexported fields
}

OfflinePointWriter Writes points to disk in a fixed-with format. lucene.internal

func NewOfflinePointWriter

func NewOfflinePointWriter(config *Config, tempDir store.Directory,
	tempFileNamePrefix, desc string, expectedCount int) *OfflinePointWriter

func (*OfflinePointWriter) Append

func (w *OfflinePointWriter) Append(ctx context.Context, packedValue []byte, docID int) error

func (*OfflinePointWriter) AppendPoint

func (w *OfflinePointWriter) AppendPoint(pointValue PointValue) error

func (*OfflinePointWriter) Close

func (w *OfflinePointWriter) Close() error

func (*OfflinePointWriter) Count

func (w *OfflinePointWriter) Count() int

func (*OfflinePointWriter) Destroy

func (w *OfflinePointWriter) Destroy() error

func (*OfflinePointWriter) GetIndexOutput

func (w *OfflinePointWriter) GetIndexOutput() store.IndexOutput

func (*OfflinePointWriter) GetReader

func (w *OfflinePointWriter) GetReader(startPoint, length int) (PointReader, error)

type OneDimensionBKDWriter

type OneDimensionBKDWriter struct {
	// contains filtered or unexported fields
}

func (*OneDimensionBKDWriter) Add

func (r *OneDimensionBKDWriter) Add(ctx context.Context, packedValue []byte, docID int) error

func (*OneDimensionBKDWriter) Finish

func (r *OneDimensionBKDWriter) Finish() (Runnable, error)

type PathSlice

type PathSlice struct {
	// contains filtered or unexported fields
}

PathSlice Sliced reference to points in an PointWriter.

func NewPathSlice

func NewPathSlice(writer PointWriter, start, count int) *PathSlice

func (*PathSlice) Count

func (p *PathSlice) Count() int

func (*PathSlice) PointWriter

func (p *PathSlice) PointWriter() PointWriter

func (*PathSlice) Start

func (p *PathSlice) Start() int

type PointReader

type PointReader interface {
	io.Closer

	// Next Returns false once iteration is done, else true.
	Next() (bool, error)

	// PointValue Sets the packed value in the provided ByteRef
	PointValue() PointValue
}

PointReader One pass iterator through all points previously written with a PointWriter, abstracting away whether points are read from (offline) disk or simple arrays in heap. lucene.internal

type PointValue

type PointValue interface {

	// PackedValue Returns the packed values for the dimensions
	PackedValue() []byte

	// DocID Returns the docID
	DocID() int

	// PackedValueDocIDBytes Returns the byte representation of the packed value together with the docID
	PackedValueDocIDBytes() []byte
}

PointValue Represents a dimensional point value written in the BKD tree. lucene.internal

type PointWriter

type PointWriter interface {
	io.Closer

	// Append Add a new point from the packed value and docId
	Append(ctx context.Context, packedValue []byte, docID int) error

	// AppendPoint Add a new point from a PointValue
	AppendPoint(pointValue PointValue) error

	// GetReader Returns a PointReader iterator to step through all previously added points
	GetReader(startPoint, length int) (PointReader, error)

	// Count Return the number of points in this writer
	Count() int

	// Destroy Removes any temp files behind this writer
	Destroy() error
}

PointWriter Appends many points, and then at the end provides a PointReader to iterate those points. This abstracts away whether we write to disk, or use simple arrays in heap. lucene.internal

type RadixSelector

type RadixSelector struct {
	// contains filtered or unexported fields
}

RadixSelector Offline Radix selector for BKD tree. lucene.internal

func NewRadixSelector

func NewRadixSelector(config *Config, maxPointsSortInHeap int,
	tempDir store.Directory, tempFileNamePrefix string) *RadixSelector

NewRadixSelector Sole constructor.

func (*RadixSelector) HeapRadixSort

func (b *RadixSelector) HeapRadixSort(points *HeapPointWriter, from, to, dim, commonPrefixLength int)

HeapRadixSort Sort the heap writer by the specified dim. It is used to sort the leaves of the tree

func (*RadixSelector) Select

func (b *RadixSelector) Select(points *PathSlice, partitionSlices []*PathSlice,
	from, to, partitionPoint int, dim, dimCommonPrefix int) ([]byte, error)

Select It uses the provided points from the given from to the given to to populate the partitionSlices array holder (length > 1) with two path slices so the path slice at position 0 contains sortPartition - from points where the value of the dim is lower or equal to the to -from points on the slice at position 1. The dimCommonPrefix provides a hint for the length of the common prefix length for the dim where are partitioning the points. It return the value of the dim at the sortPartition point. If the provided points is wrapping an OfflinePointWriter, the writer is destroyed in the process to save disk space.

type Reader

type Reader struct {
	// contains filtered or unexported fields
}

Reader Handles intersection of an multi-dimensional shape in byte[] space with a block KD-tree previously written with BKDWriter. lucene.experimental

func NewReader

func NewReader(ctx context.Context, metaIn, indexIn, dataIn store.IndexInput) (*Reader, error)

NewReader Caller must pre-seek the provided IndexInput to the index location that BKDWriter.finish returned. BKD tree is always stored off-heap.

func (*Reader) EstimateDocCount

func (r *Reader) EstimateDocCount(visitor types.IntersectVisitor) (int, error)

func (*Reader) EstimatePointCount

func (r *Reader) EstimatePointCount(ctx context.Context, visitor types.IntersectVisitor) (int, error)

func (*Reader) GetBytesPerDimension

func (r *Reader) GetBytesPerDimension() (int, error)

func (*Reader) GetDocCount

func (r *Reader) GetDocCount() int

func (*Reader) GetIntersectState

func (r *Reader) GetIntersectState(ctx context.Context, visitor types.IntersectVisitor) (*IntersectState, error)

func (*Reader) GetMaxPackedValue

func (r *Reader) GetMaxPackedValue() ([]byte, error)

func (*Reader) GetMinPackedValue

func (r *Reader) GetMinPackedValue() ([]byte, error)

func (*Reader) GetNumDimensions

func (r *Reader) GetNumDimensions() (int, error)

func (*Reader) GetNumIndexDimensions

func (r *Reader) GetNumIndexDimensions() (int, error)

func (*Reader) Intersect

func (r *Reader) Intersect(ctx context.Context, visitor types.IntersectVisitor) error

func (*Reader) IsLeafNode

func (r *Reader) IsLeafNode(nodeID int) bool

func (*Reader) Size

func (r *Reader) Size() int

func (*Reader) VisitLeafBlockValues

func (r *Reader) VisitLeafBlockValues(ctx context.Context, index *IndexTree, state *IntersectState) error

VisitLeafBlockValues Visits all docIDs and packed values in a single leaf block

type Runnable

type Runnable func(ctx context.Context) error

type VerifyPointsVisitor

type VerifyPointsVisitor struct {
	// contains filtered or unexported fields
}

VerifyPointsVisitor Walks the entire N-dimensional points space, verifying that all points fall within the last cell's boundaries. lucene.internal

func NewVerifyPointsVisitor

func NewVerifyPointsVisitor(fieldName string, maxDoc int, values types.PointValues) (*VerifyPointsVisitor, error)

func (*VerifyPointsVisitor) Compare

func (v *VerifyPointsVisitor) Compare(minPackedValue, maxPackedValue []byte) types.Relation

func (*VerifyPointsVisitor) Grow

func (v *VerifyPointsVisitor) Grow(count int)

func (*VerifyPointsVisitor) Visit

func (v *VerifyPointsVisitor) Visit(ctx context.Context, docID int) error

func (*VerifyPointsVisitor) VisitLeaf

func (v *VerifyPointsVisitor) VisitLeaf(ctx context.Context, docID int, packedValue []byte) error

type Writer

type Writer struct {
	// contains filtered or unexported fields
}

Writer Recursively builds a block KD-tree to assign all incoming points in N-dim space to smaller and smaller N-dim rectangles (cells) until the number of points in a given rectangle is <= config.maxPointsInLeafNode. The tree is partially balanced, which means the leaf nodes will have the requested config.maxPointsInLeafNode except one that might have less. Leaf nodes may straddle the two bottom levels of the binary tree. Values that fall exactly on a cell boundary may be in either cell. The number of dimensions can be 1 to 8, but every byte[] value is fixed length. This consumes heap during writing: it allocates a Long[numLeaves], a byte[numLeaves*(1+config.bytesPerDim)] and then uses up to the specified maxMBSortInHeap heap space for writing. NOTE: This can write at most Integer.MAX_VALUE * config.maxPointsInLeafNode / config.bytesPerDim total points. lucene.experimental

func NewWriter

func NewWriter(maxDoc int, tempDir store.Directory, tempFileNamePrefix string,
	config *Config, maxMBSortInHeap float64, totalPointCount int) (*Writer, error)

func (*Writer) Add

func (w *Writer) Add(packedValue []byte, docID int) error

func (*Writer) Close

func (w *Writer) Close() error

func (*Writer) Finish

func (w *Writer) Finish(ctx context.Context, metaOut, indexOut, dataOut store.IndexOutput) (Runnable, error)

Finish Writes the BKD tree to the provided IndexOutputs and returns a Runnable that writes the index of the tree if at least one point has been added, or null otherwise.

func (*Writer) NewOneDimensionBKDWriter

func (w *Writer) NewOneDimensionBKDWriter(metaOut, indexOut, dataOut store.IndexOutput) (*OneDimensionBKDWriter, error)

func (*Writer) WriteField

func (w *Writer) WriteField(ctx context.Context, metaOut, indexOut, dataOut store.IndexOutput, fieldName string, reader types.MutablePointValues) (Runnable, error)

WriteField Write a field from a MutablePointValues. This way of writing points is faster than regular writes with add since there is opportunity for reordering points before writing them to disk. This method does not use transient disk in order to reorder points.

Jump to

Keyboard shortcuts

? : This menu
/ : Search site
f or F : Jump to
y or Y : Canonical URL