BTree
is a scalable B+-Tree with copy-on-write
semantics mapping variable length unsigned byte[]
keys to variable
length byte[]
values (null values are allowed).See: Description
Interface | Description |
---|---|
AbstractBTree.IBTreeCounters |
Interface declaring namespaces for performance counters collected for a
B+Tree.
|
HTreeIndexMetadata.Options |
HTree specific options. |
IAbstractNode |
Interface for a node or a leaf of a B+-Tree.
|
IAutoboxBTree |
An interface defining non-batch methods for inserting, removing, lookup, and
containment tests where keys and values are implicitly converted to and from
byte[] s using the ITupleSerializer configured on the
IndexMetadata object for the IIndex . |
IBloomFilter |
Interface for bloom filter implementations using an unsigned byte[] key.
|
IBTreeStatistics |
Interface used to report out some statistics about a B+Tree.
|
IBTreeUtilizationReport |
B+Tree utilization report.
|
ICheckpoint |
Metadata for an index checkpoint record.
|
ICheckpointProtocol |
Interface in support of the
Checkpoint record protocol. |
ICounter |
An interface for a counter.
|
IDirty |
An interface that declares how we access the dirty state of an object.
|
IDirtyListener |
An interface that may be used to learn when a
BTree becomes
dirty. |
IEvictionListener |
Interface to handle evictions of nodes or leaves from the hard reference
queue.
|
IIdentityAccess |
An interface that declares how we access the persistent identity of an
object.
|
IIndex |
Interface for mutable B+-Tree mapping arbitrary non-null keys to arbitrary
values.
|
IIndexLocalCounter |
An interface for accessing an index local counter.
|
ILeafCursor<L extends Leaf> |
Leaf cursor interface.
|
ILinearList |
Interface for methods that return or accept an ordinal index into the entries
in the B+-Tree.
|
ILocalBTreeView |
Interface indicates that the index is local rather than remote.
|
IndexMetadata.Options |
Options and their defaults for the
com.bigdata.btree package and
the BTree and IndexSegment classes. |
INodeFactory |
Interface for creating nodes or leaves.
|
INodeIterator |
Interface for iterators that visit nodes and leaves rather than entries in
leaves.
|
IOverflowHandler |
An interface that allows you to inspect index entries during an
IndexSegmentBuilder operation. |
IRangeQuery |
Interface for range count and range query operations.
|
IRawRecordAccess |
Interface providing access to raw records.
|
IReadWriteLockManager |
Interface for managing read/write locks on persistence capable data
structures.
|
ISimpleBTree |
Interface for non-batch operations on a B+-Tree mapping non-null variable
length unsigned byte[] keys to arbitrary values.
|
ISimpleIndexAccess |
Generic data access methods defined for all persistence capable data
structures.
|
ISimpleSplitHandler |
Interface allows an application to constrain the choice of the separator
key when an index partition is split.
|
ISimpleTreeIndexAccess |
Extended interface for tree-structured indices.
|
ITuple<E> |
Interface exposes more direct access to keys and values visited by an
ITupleIterator . |
ITupleCursor<E> |
Interface for sequential and random-access cursor-based
ITuple
operations on an index or index partition. |
ITupleCursor2<E> |
Extended interface.
|
ITupleIterator<E> |
Interface visits
ITuple s populated with the data and metadata for
visited index entries. |
ITupleSerializer<K,V> |
An interface that provides for the (de)-serialization of the value of a tuple
stored in an index and, when possible, the key under which that value is
stored.
|
Leaf.ILeafListener |
An interface that may be used to register for and receive events when the
state of a
Leaf is changed. |
Class | Description |
---|---|
AbstractBTree |
Base class for mutable and immutable B+-Tree implementations.
|
AbstractBTreeCursorTestCase |
Abstract base class for some unit tests that can only be run against a
BTree . |
AbstractBTreeTestCase |
Abstract test case for
BTree tests. |
AbstractBTreeTupleCursor<I extends AbstractBTree,L extends Leaf,E> |
Class supporting random access to tuples and sequential tuple-based cursor
movement for an
AbstractBTree . |
AbstractBTreeTupleCursor.MutableBTreeTupleCursor<E> | |
AbstractBTreeTupleCursor.ReadOnlyBTreeTupleCursor<E> | |
AbstractChunkedTupleIterator<E> |
A chunked iterator that proceeds a
ResultSet at a time. |
AbstractIndexSegmentTestCase |
Adds some methods for testing an
IndexSegment for consistency. |
AbstractNode<T extends AbstractNode> |
Abstract node supporting incremental persistence and copy-on-write semantics.
|
AbstractTuple<E> |
Abstract base class with much of the functionality of
ITuple . |
AbstractTupleCursorTestCase |
Abstract base class for
ITupleCursor test suites. |
AsynchronousIndexWriteConfiguration |
Configuration for the asynchronous index write API.
|
BaseIndexStats |
Basic stats that are available for all index types and whose collection does
not require visitation of the index pages.
|
BigdataMap<K,V> | |
BigdataSet<E> |
A
SortedSet backed by a B+Tree. |
BloomFilter |
Encapsulates the actual implementation class and provides the protocol for
(de-)serialization.
|
BloomFilter.BloomFilterCounters |
Counters for bloom filter access and notification of false positives.
|
BloomFilterFactory |
An interface that is used to generate a bloom filter for an
AbstractBTree and which allows the caller to specify the expected
number of index entries, the desired error rate for the filter at that #of
index entries, and the maximum error rate before the bloom filter will be
disabled. |
BTree |
This class implements a variant of a B+Tree in which all values are stored in
leaves, but the leaves are not connected with prior-next links.
|
BTree.Counter |
Mutable counter.
|
BTree.NodeFactory |
Factory for mutable nodes and leaves used by the
NodeSerializer . |
BTree.PartitionedCounter |
Places the counter values into a namespace formed by the partition
identifier.
|
BTree.Stack |
A simple stack based on an array used to maintain hard references for the
parent
Node s in the BTree.LeafCursor . |
BTreeCounters |
A helper class that collects statistics on an
AbstractBTree . |
BTreePageStats | |
BTreeStatistics |
A snapshot of the B+Tree statistics.
|
BTreeUtilizationReport |
A btree utilization report.
|
Checkpoint |
A checkpoint record is written each time the btree is flushed to the
store.
|
ChunkedLocalRangeIterator<E> |
Chunked range iterator running against a local index or index view.
|
DefaultEvictionListener |
Hard reference cache eviction listener writes a dirty node or leaf onto the
persistence store.
|
DefaultTupleSerializer<K,V> |
Default implementation uses the
KeyBuilder to format the object as a
key and uses Java default serialization for the value. |
DelegateBTree |
An object that delegates the
IIndex and ILinearList
interfaces. |
DelegateIndex |
An object that delegates its
IIndex interface. |
DelegateTuple<E> |
An
ITuple wrapping a delegate that may be used to override some of
the methods on the delegate object. |
DumpIndex |
Utility class to dump an index in a variety of ways.
|
DumpIndexSegment |
Utility to examine the context of an
IndexSegmentStore . |
EntryScanIterator |
Iterator visits index entries (dereferencing visited tuples to the
application objects stored within those tuples).
|
Errors |
Error messages for the B+Tree package.
|
FixedLengthPrefixSplits |
Imposes constraint that the key before the separatorKey must differ in
the first N bytes from the key after the separator key.
|
HTreeIndexMetadata |
HTree specific implementation.
|
IndexMetadata |
The persistent and mostly immutable metadata for a
AbstractBTree . |
IndexSegment |
An index segment is read-only btree corresponding to some key range of a
potentially distributed index.
|
IndexSegment.ImmutableNodeFactory |
Factory for immutable nodes and leaves used by the
NodeSerializer . |
IndexSegment.ImmutableNodeFactory.ImmutableLeaf |
Immutable leaf throws
UnsupportedOperationException for the
public mutator API but does not try to override all low-level
mutation behaviors. |
IndexSegment.ImmutableNodeFactory.ImmutableNode |
Immutable node throws
UnsupportedOperationException for the
public mutator API but does not try to override all low-level
mutation behaviors. |
IndexSegment.IndexSegmentTupleCursor<E> |
Implementation for an immutable
IndexSegment . |
IndexSegmentAddressManager |
Address manager supporting offsets that are encoded for one of several
regions in an
IndexSegmentStore . |
IndexSegmentBuilder |
Builds an
IndexSegment given a source btree and a target branching
factor. |
IndexSegmentBuilder.AbstractSimpleNodeData |
Abstract base class for classes used to construct and serialize nodes and
leaves written onto the index segment.
|
IndexSegmentBuilder.NOPNodeFactory |
Factory does not support node or leaf creation.
|
IndexSegmentBuilder.SimpleLeafData |
A class that can be used to (de-)serialize the data for a leaf without
any of the logic for operations on the leaf.
|
IndexSegmentBuilder.SimpleNodeData |
A class that can be used to (de-)serialize the data for a node without
any of the logic for operations on the node.
|
IndexSegmentCheckpoint |
The checkpoint record for an
IndexSegment . |
IndexSegmentDumpUtil | |
IndexSegmentMultiBlockIterator<E> |
A fast iterator based on multi-block IO for the
IndexSegment . |
IndexSegmentPlan |
A plan for building a B+-Tree based on an input branching factor and #of
entries.
|
IndexSegmentStore |
A read-only store backed by a file containing a single
IndexSegment . |
Leaf |
A B+-Tree leaf.
|
LeafTupleIterator<E> |
Visits the values of a
Leaf in the external key ordering. |
MutableLeafData |
Implementation maintains Java objects corresponding to the persistent data
and defines methods for a variety of mutations on the
ILeafData
record which operate by direct manipulation of the Java objects. |
MutableNodeData |
Implementation maintains Java objects corresponding to the persistent data
and defines methods for a variety of mutations on the
INodeData
record which operate by direct manipulation of the Java objects. |
MyEvictionListener |
Extends
DefaultEvictionListener to explicitly control when an
eviction notice is expected and to verify that eviction notices are received
as expected. |
MyHardReferenceQueue<T> |
Wraps the basic implementation and exposes a protected method that we
need to write the tests in this suite.
|
Node |
A non-leaf node.
|
NodeSerializer |
An instance of this class is used to serialize and de-serialize the
INodeData s and ILeafData s of an AbstractBTree . |
NoEvictionListener |
Hard reference cache eviction listener for leaves always throws an
exception.
|
NOPBloomFilter |
A bloom filter that never reports
false (this means that you
must always check the index) and that does not permit anything to be added
and, in fact, has no state. |
NOPEvictionListener |
A listener that does nothing.
|
NOPTupleSerializer |
Default implementation uses the
KeyBuilder to format the object as a
key and requires that the values are byte[]s which it passes on without
change. |
PageStats |
Class reports various summary statistics for nodes and leaves.
|
PO |
A persistent object.
|
RangeCheckUtil |
Utility class to verify that a key lies within a key range.
|
ReadCommittedView |
A view of a named index that replaces its view for each high-level request if
there has been an intervening commit on the backing store.
|
ReadOnlyCounter |
A read-only view of an
ICounter . |
ReadOnlyEntryIterator<E> |
Iterator disallows
ReadOnlyEntryIterator.remove() . |
ReadOnlyIndex |
A fly-weight wrapper that does not permit write operations and reads through
onto an underlying
IIndex . |
ReadWriteLockManager |
Base class for managing read/write locks for unisolated
ICommitter s. |
ResultSet |
An object used to stream key scan results back to the client.
|
ScatterSplitConfiguration |
Configuration object for scatter split behavior for a scale-out index.
|
SimpleEntry |
Test helper provides an entry (aka value) for a
Leaf . |
StressTestBTreeRemove | |
TestAll |
Aggregates test suites into increasing dependency order.
|
TestAll_BTreeBasics |
Aggregates the unit tests for the core B+Tree operations, all of which are in
the same package as the
BTree . |
TestAll_IndexSegment |
Aggregates the unit tests for the
IndexSegment and its related
classes, all of which are in the same package as the BTree . |
TestAll_IndexSegmentBuilderWithSmallTrees |
Stress test suite for
TestIndexSegmentBuilderWithSmallTree . |
TestAll_Iterators |
Aggregates the unit tests for the
IndexSegment and its related
classes, all of which are in the same package as the BTree . |
TestBigdataMap |
Test suite for
BigdataMap . |
TestBigdataSet |
Test suite for
BigdataSet . |
TestBloomFilter |
Test suite for bloom filter functionality.
|
TestBTree |
Stress tests for basic tree operations (insert, lookup, and remove) without
causing node or leaf evictions (IO is disabled).
|
TestBTreeBranchingFactors |
Unit tests of non-default B+Tree branching factors.
|
TestBTreeLeafCursors |
Test suite for the
BTree 's ILeafCursor implementation. |
TestBTreeRecycle |
Test suite for recycling of B+Tree records.
|
TestBTreeWithBloomFilter |
Unit tests for a
BTree with its bloom filter enabled. |
TestBytesUtil |
Test suite for low-level operations on variable length byte[]s.
|
TestChunkedIterators |
Test suite for
AbstractChunkedTupleIterator and its concrete
ChunkedLocalRangeIterator implementation which depends on the
ResultSet . |
TestCommit |
Unit tests for commit functionality that do not trigger copy-on-write.
|
TestCompactingByteArrayBuffer |
Test harness used to develop a compacting buffer for maintaining branch nodes
and leaves in a B+Tree that minimizes copying on mutation of the node, helps
to minimize heap churn and GC latency related to long lived allocations, and
maintains the data in a serializable format.
|
TestConstrainKeys |
Test imposing constraint on a fromKey or toKey based on an index partition's
boundaries.
|
TestCopyOnWrite |
Test suite for copy-on-write semantics.
|
TestDeleteMarkers |
Test of basic btree operations when delete markers are maintained.
|
TestDirtyIterators |
Test suite for iterators that visit only dirty nodes or leaves.
|
TestDirtyListener |
Test suite for the
IDirtyListener protocol. |
TestFindChild |
Test code that chooses the child to search during recursive traversal of the
separator keys to find a leaf in which a key would be found.
|
TestGetBitsApplication |
Rather than run in a JUnit, the performance tests are best run
standalone.
|
TestGetBitsFromByteArray |
Unit tests for
BytesUtil.getBits(byte[], int, int) |
TestGetBitsFromInt32 |
Unit tests for {@link BytesUtil#getBits(int, int, int)
|
TestIncrementalWrite |
Test suite for the logic performing incremental writes of nodes and leaves
onto the store.
|
TestIndexCounter |
Test suite for the
IIndexLocalCounter.getCounter() interface. |
TestIndexPartitionFencePosts |
Tests for some fence posts for an index supporting delete markers and having
a
LocalPartitionMetadata that declares its legal key range. |
TestIndexSegmentAddressManager |
Tests logic to encode and decode the offsets within regions in an
IndexSegmentStore . |
TestIndexSegmentBuilder_EmptyIndex |
Test suite for building an
IndexSegment from an empty BTree . |
TestIndexSegmentBuilderWithBlobCapacity | |
TestIndexSegmentBuilderWithCompactingMerge |
A test of the
IndexSegmentBuilder in which there are some deleted
tuples in the source BTree and a compacting merge is performed. |
TestIndexSegmentBuilderWithIncrementalBuild |
A test of the
IndexSegmentBuilder in which there are some deleted
tuples in the source BTree which are to be copied to the destination
IndexSegment (this is really a test of the incremental build scenario
since deleted tuples are never copied for a compacting merge). |
TestIndexSegmentBuilderWithLargeTrees |
Test build trees on the journal, evicts them into an
IndexSegment ,
and then compares the trees for the same total ordering. |
TestIndexSegmentBuilderWithSmallTree |
Test suite based on a small btree with known keys and values.
|
TestIndexSegmentCheckpoint |
Test suite for
IndexSegmentCheckpoint . |
TestIndexSegmentCursors |
Test suite for
IndexSegment.IndexSegmentTupleCursor . |
TestIndexSegmentMultiBlockIterators |
Test suite for
IndexSegmentMultiBlockIterator . |
TestIndexSegmentPlan |
Test suite for efficient post-order rebuild of an index in an external index
segment.
|
TestIndexSegmentWithBloomFilter |
Test build trees on the journal, evicts them into an
IndexSegment ,
and then compares the performance and correctness of index point tests with
and without the use of the bloom filter. |
TestInsertLookupRemoveKeysInRootLeaf |
Test insert, lookup, and value scan for leaves.
|
TestIterators |
Test suite for iterators.
|
TestLeafSplitShortestSeparatorKey | |
TestLinearListMethods |
Test suite for the
ILinearList access methods. |
TestMutableBTreeCursors |
Test ability to traverse tuples using an
ITupleCursor while the SAME
THREAD is used to insert, update, or remove tuples from a mutable
BTree . |
TestNullValues |
Test of storing null values under a key with persistence.
|
TestPutIfAbsent |
Test of basic btree operations when delete markers are maintained.
|
TestRawRecords |
Unit tests for a B+Tree with raw record support enabled (this is where a
large
byte[] value is written directly onto the backing store
rather than being stored within the leaf). |
TestReadOnlyBTreeCursors |
Unit tests for
ITupleCursor for a read-only BTree . |
TestRemoveAll |
Test suite for
BTree.removeAll() . |
TestReopen |
Unit tests for the close/checkpoint/reopen protocol designed to manage the
resource burden of indices without invalidating the index objects (indices
opens can be reopened as long as their backing store remains available).
|
TestSplitJoinRootLeaf |
Test split and join of the root leaf (the tree never has more than two
levels).
|
TestSplitJoinThreeLevels |
Test suite using
AbstractBTree.insert(Object, Object) to split a tree to
height two (2) (three levels) and then using AbstractBTree.remove(Object) to
reduce the tree back to a single, empty root leaf. |
TestSplitRootLeaf |
Test split of the root leaf.
|
TestTouch |
Test suite for
AbstractBTree.touch(AbstractNode) . |
TestTransientBTree |
Unit tests for transient
BTree s (no backing store). |
TestTuple<E> |
Test helper for a tuple with static data.
|
TestUtilMethods |
Test suite for various utility methods, both static and instance, on
AbstractNode . |
Tuple<E> |
A key-value pair used to facilitate some iterator constructs.
|
UnisolatedReadWriteIndex |
A view onto an unisolated index partition which enforces the constraint that
either concurrent readers -or- a single writer may have access to the
unisolated index at any given time.
|
ViewStatistics |
Helper class collects some statistics about an
ILocalBTreeView . |
Enum | Description |
---|---|
IndexSegmentRegion |
Type-safe enumeration of the regions to which relative offsets may be
constructed for an
IndexSegmentStore . |
IndexTypeEnum |
Type safe enumeration of index types.
|
SeekEnum |
Typesafe enum used to indicate that an
ILeafCursor should
seek to the first or last leaf in the B+Tree. |
Exception | Description |
---|---|
KeyAfterPartitionException |
An exception thrown when a key lies after the half-open range of an index
partition.
|
KeyBeforePartitionException |
Exception thrown when a key is before the start of the half-open range of an
index partition.
|
KeyOutOfRangeException |
An exception thrown when the key is outside of the half-open range constraint
for a
ITupleCursor or an index partition. |
NotChildException |
An instance of this exception is thrown when a node or leaf is not a child
(or self) for some other node or leaf.
|
Error | Description |
---|---|
EvictionError |
Error marks an mutable index as in an inconsistent state arising from an
exception during eviction of a dirty node or leaf from a mutable index.
|
IndexInconsistentError |
Error marks an mutable index as in an inconsistent state.
|
The BTree
is a scalable B+-Tree with copy-on-write
semantics mapping variable length unsigned byte[]
keys to variable
length byte[]
values (null values are allowed). This class is
designed for read-write operations.
The B+-Tree uses a fixed branching factor (aka fan-out) but supports
variable length keys and values and does not directly constrain the
serialized size of a node or leaf. The branching factor is determined
when the B+Tree is created. Bulk index builds are supported from a
variety of sources, including a merge of mutable and immutable
B+-Trees. Bulk index builds result in immutable IndexSegment
s
which may have a branching distinct from that of their source
B+Tree(s).
Nodes (and leaves) are either dirty or immutable and persistent. If
the node is dirty, then it is always mutable. If a node is clean,
then it is always immutable and persistent. An attempt to write on a
clean node forces copy-on-write of the node and its ancestors up to
the root of the tree. In any case where the node has already been
copied and is dirty, the mutable node is always used. Therefore
mutations never overwrite the historical state of the B+-Tree and
always produce a new well-formed tree. The root of the new tree is
accessible from root block of the store after a commit (this is handled
by the AbstractJournal
.
The com.bigdata.btree.INodeData
and com.bigdata.btree.ILeafData
interfaces represent the persistent state of a node or leaf. The keys of a
node or leaf are represented by an IRaba
, which
is an abstraction for a logical byte[][]
. Likewise, the values
of a leaf are represented by an IRaba
. For keys,
the rabas support search. For values, they allow nulls. The node and leaf
data records maintain additional persistent state, for example, leaf data
records track tuple delete markers and tuple revision timestamps while node
data records track the persistent address of child nodes.
Coding of node and leaf data records is performed when they are evicted from a
write retention queue. The purpose of the write retention queue is
to maintain recently used nodes and leaves in their mutable form and to defer
eviction onto the disk until their data is stable (has not been changed recently).
When a dirty node or leaf is evicted from the write retention queue, it is first
coded (compressed) using a com.bigdata.btree.data.INodeDataCoder
or a
com.bigdata.btree.data.ILeafDataCoder
as appropriate. Those coders in
turn will apply the configured com.bigdata.btree.raba.IRabaCoder
to
code the keys and the values. Once the node or leaf has been coded, it is
represented as a slice on a byte[]. That slice can be wrapped by the same
com.bigdata.btree.raba.IRabaCoder
, returning an efficient view of the
compressed data record supporting search (for keys) and random access to the
keys and values.
Front-coding (prefix compression) generally works quite well for keys. A canonical huffman coder may be used for keys, but is significantly slower and is generally used to code values. Custom coders may be written for either the keys or values of a B+Tree leaf in order to take advantage of various properties for a specific application index. However, the coder for the B+Tree node keys MUST handle variable length keys since the keys stored in the node are separator keys, not full length application keys.
Coding, decoding, and promotion to a mutable data record are handled transparently
by the B+Tree. The NodeSerializer
provides a facility
for coding and decoding nodes and leaves. Coding uses a shared (per B+Tree
instance) extensible byte[] buffer to reduce heap churn. Decoding simply wraps
the coded record. Promotion to a mutable data record is handled by converting
to a MutableNodeData
or MutableLeafData
respectively.
When nodes or leaves are evicted from the write retention queue their coded
data record is written onto the backing IRawStore
.
The resulting int64 address is updated on the parent of the node or leaf. Weak
references are used between a child and its parent. The existence of the node
or leaf on the write retention queue prevents weak references from being cleared.
Once the node or leaf has been evicted from the write retention queue, it can
be reloaded from its persistent address if its weak reference is cleared.
When a node is evicted from the write retention queue, a depth-first traversal of its dirty children is performed and they are persisted as well. This ensures that we can recover the tree structure later. When a leaf is evicted, just that leaf is persisted.
A B+-Tree checkpoint record corresponds to a persistent state of the
B+Tree on the disk. The B+Tree may be reloaded from that checkpoint. After
a checkpoint operation, all nodes and leaves in the B+Tree will be clean. However,
a checkpoint IS NOT a commit. The commit protocol is handled by the
AbstractJournal
.
A BTree is designated as transient by specifying null
for the backing
IRawStore
. Nodes and leaves of a transient B+-Tree
are linked by hard references to ensure that they remain reachable. However,
mutable nodes and leaves are still converted to coded (compressed) nodes and
leaves when they are evicted from the write retention queue.
Copyright © 2006–2019 SYSTAP, LLC DBA Blazegraph. All rights reserved.