Documentation
¶
Index ¶
- Constants
- func Mismatch(a, b []byte) int
- func ReadInts(ctx context.Context, in store.DataInput, count int, docIDs []int) error
- func ReadIntsVisitor(ctx context.Context, in store.IndexInput, count int, ...) error
- func Subtract(bytesPerDim, dim int, a, b, result []byte) error
- func WriteDocIds(ctx context.Context, docIds []int, out store.DataOutput) error
- type Config
- type HeapPointReader
- type HeapPointValue
- type HeapPointWriter
- func (h *HeapPointWriter) Append(ctx context.Context, packedValue []byte, docID int) error
- func (h *HeapPointWriter) AppendPoint(pointValue PointValue) error
- func (h *HeapPointWriter) Close() error
- func (h *HeapPointWriter) ComputeCardinality(form, to int, commonPrefixLengths []int) int
- func (h *HeapPointWriter) Count() int
- func (h *HeapPointWriter) Destroy() error
- func (h *HeapPointWriter) GetPackedValueSlice(index int) PointValue
- func (h *HeapPointWriter) GetReader(start, length int) (PointReader, error)
- func (h *HeapPointWriter) Swap(i, j int)
- type IndexTree
- func (t *IndexTree) Clone() *IndexTree
- func (t *IndexTree) GetLeafBlockFP() int64
- func (t *IndexTree) GetNodeID() int
- func (t *IndexTree) GetSplitDim() int
- func (t *IndexTree) GetSplitDimValue() []byte
- func (t *IndexTree) GetSplitPackedValue() []byte
- func (t *IndexTree) IsLeafNode() bool
- func (t *IndexTree) NodeExists() bool
- func (t *IndexTree) Pop()
- func (t *IndexTree) PushLeft(ctx context.Context) error
- func (t *IndexTree) PushRight(ctx context.Context) error
- type IntersectState
- type IntroSorter
- type LeafNodes
- type MergeQueue
- type MergeReader
- type MutablePointValuesSorter
- type OfflinePointReader
- type OfflinePointValue
- type OfflinePointWriter
- func (w *OfflinePointWriter) Append(ctx context.Context, packedValue []byte, docID int) error
- func (w *OfflinePointWriter) AppendPoint(pointValue PointValue) error
- func (w *OfflinePointWriter) Close() error
- func (w *OfflinePointWriter) Count() int
- func (w *OfflinePointWriter) Destroy() error
- func (w *OfflinePointWriter) GetIndexOutput() store.IndexOutput
- func (w *OfflinePointWriter) GetReader(startPoint, length int) (PointReader, error)
- type OneDimensionBKDWriter
- type PathSlice
- type PointReader
- type PointValue
- type PointWriter
- type RadixSelector
- type Reader
- func (r *Reader) EstimateDocCount(ctx context.Context, visitor types.IntersectVisitor) (int, error)
- func (r *Reader) EstimatePointCount(ctx context.Context, visitor types.IntersectVisitor) (int, error)
- func (r *Reader) GetBytesPerDimension() (int, error)
- func (r *Reader) GetDocCount() int
- func (r *Reader) GetIntersectState(ctx context.Context, visitor types.IntersectVisitor) (*IntersectState, error)
- func (r *Reader) GetMaxPackedValue() ([]byte, error)
- func (r *Reader) GetMinPackedValue() ([]byte, error)
- func (r *Reader) GetNumDimensions() (int, error)
- func (r *Reader) GetNumIndexDimensions() (int, error)
- func (r *Reader) Intersect(ctx context.Context, visitor types.IntersectVisitor) error
- func (r *Reader) IsLeafNode(nodeID int) bool
- func (r *Reader) Size() int
- func (r *Reader) VisitLeafBlockValues(ctx context.Context, index *IndexTree, state *IntersectState) error
- type Runnable
- type VerifyPointsVisitor
- func (v *VerifyPointsVisitor) Compare(minPackedValue, maxPackedValue []byte) types.Relation
- func (v *VerifyPointsVisitor) Grow(count int)
- func (v *VerifyPointsVisitor) Visit(ctx context.Context, docID int) error
- func (v *VerifyPointsVisitor) VisitLeaf(ctx context.Context, docID int, packedValue []byte) error
- type Writer
- func (w *Writer) Add(ctx context.Context, packedValue []byte, docID int) error
- func (w *Writer) Close() error
- func (w *Writer) Finish(ctx context.Context, metaOut, indexOut, dataOut store.IndexOutput) (Runnable, error)
- func (w *Writer) NewOneDimensionBKDWriter(metaOut, indexOut, dataOut store.IndexOutput) (*OneDimensionBKDWriter, error)
- func (w *Writer) WriteField(ctx context.Context, metaOut, indexOut, dataOut store.IndexOutput, ...) (Runnable, error)
Constants ¶
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 )
const ( HISTOGRAM_SIZE = 256 // size of the histogram MAX_SIZE_OFFLINE_BUFFER = 1024 * 8 // size of the online buffer: 8 KB INTEGER_BYTES = 4 )
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 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 WriteDocIds ¶
Types ¶
type Config ¶
type Config struct {
// contains filtered or unexported fields
}
Config Basic parameters for indexing points on the BKD tree.
func (*Config) BytesPerDim ¶
func (*Config) BytesPerDoc ¶
func (*Config) MaxPointsInLeafNode ¶
func (*Config) NumIndexDims ¶
func (*Config) PackedBytesLength ¶
func (*Config) PackedIndexBytesLength ¶
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) 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) GetLeafBlockFP ¶
GetLeafBlockFP Only valid after pushLeft or pushRight, not pop!
func (*IndexTree) GetSplitDim ¶
GetSplitDim Only valid after pushLeft or pushRight, not pop!
func (*IndexTree) GetSplitDimValue ¶
GetSplitDimValue Only valid after pushLeft or pushRight, not pop!
func (*IndexTree) GetSplitPackedValue ¶
func (*IndexTree) IsLeafNode ¶
func (*IndexTree) NodeExists ¶
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 (*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
}
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 (*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 (*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
}
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) PointWriter ¶
func (p *PathSlice) PointWriter() PointWriter
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 ¶
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 (*Reader) EstimatePointCount ¶
func (*Reader) GetBytesPerDimension ¶
func (*Reader) GetDocCount ¶
func (*Reader) GetIntersectState ¶
func (r *Reader) GetIntersectState(ctx context.Context, visitor types.IntersectVisitor) (*IntersectState, error)
func (*Reader) GetMaxPackedValue ¶
func (*Reader) GetMinPackedValue ¶
func (*Reader) GetNumDimensions ¶
func (*Reader) GetNumIndexDimensions ¶
func (*Reader) IsLeafNode ¶
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 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)
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 (*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.
Source Files
¶
- config.go
- docidswriter.go
- heappointreader.go
- heappointwriter.go
- indextree.go
- intersect_state.go
- mergereader.go
- mutablepointsreaderutils.go
- offlinepointreader.go
- offlinepointwriter.go
- onedimensionwriter.go
- point.go
- radixselector.go
- reader.go
- readerdocidsetiterator.go
- utils.go
- verifypointsvisitor.go
- writer.go
- writerbuild.go
- writerpersistence.go
- writerutils.go