iavl

package module
v1.1.2 Latest Latest
Warning

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

Go to latest
Published: Apr 8, 2024 License: Apache-2.0 Imports: 30 Imported by: 151

README

IAVL+ Tree

version license API Reference Lint Test Discord chat

Note: Requires Go 1.18+

A versioned, snapshottable (immutable) AVL+ tree for persistent data.

Benchmarks

The purpose of this data structure is to provide persistent storage for key-value pairs (say to store account balances) such that a deterministic merkle root hash can be computed. The tree is balanced using a variant of the AVL algorithm so all operations are O(log(n)).

Nodes of this tree are immutable and indexed by their hash. Thus any node serves as an immutable snapshot which lets us stage uncommitted transactions from the mempool cheaply, and we can instantly roll back to the last committed state to process transactions of a newly committed block (which may not be the same set of transactions as those from the mempool).

In an AVL tree, the heights of the two child subtrees of any node differ by at most one. Whenever this condition is violated upon an update, the tree is rebalanced by creating O(log(n)) new nodes that point to unmodified nodes of the old tree. In the original AVL algorithm, inner nodes can also hold key-value pairs. The AVL+ algorithm (note the plus) modifies the AVL algorithm to keep all values on leaf nodes, while only using branch-nodes to store keys. This simplifies the algorithm while keeping the merkle hash trail short.

In Ethereum, the analog is Patricia tries. There are tradeoffs. Keys do not need to be hashed prior to insertion in IAVL+ trees, so this provides faster iteration in the key space which may benefit some applications. The logic is simpler to implement, requiring only two types of nodes -- inner nodes and leaf nodes. On the other hand, while IAVL+ trees provide a deterministic merkle root hash, it depends on the order of transactions. In practice this shouldn't be a problem, since you can efficiently encode the tree structure when serializing the tree contents.

IAVL x Cosmos SDK

IAVL DB Interface Cosmos SDK
v0.19.x tm-db v0.45.x, v0.46.x
v0.20.x cometbft-db v0.47.x
v1.x.x cosmos-db -

NOTE: In the past, a v0.21.x release was published, but never used in production. It was retracted to avoid confusion.

Documentation

Overview

Package iavl implements a versioned, snapshottable (immutable) AVL+ tree for persisting key-value pairs.

The tree is not safe for concurrent use, and must be guarded by a Mutex or RWLock as appropriate - the exception is immutable trees returned by MutableTree.GetImmutable() which are safe for concurrent use as long as the version is not deleted via DeleteVersion().

Basic usage of MutableTree:

import "github.com/cosmos/iavl"
import "github.com/cosmos/cosmos-db"
...

tree := iavl.NewMutableTree(db.NewMemDB(), 128)

tree.IsEmpty() // true

tree.Set([]byte("alice"), []byte("abc"))
tree.SaveVersion(1)

tree.Set([]byte("alice"), []byte("xyz"))
tree.Set([]byte("bob"), []byte("xyz"))
tree.SaveVersion(2)

tree.LatestVersion() // 2

tree.GetVersioned([]byte("alice"), 1) // "abc"
tree.GetVersioned([]byte("alice"), 2) // "xyz"

Proof of existence:

root := tree.Hash()
val, proof, err := tree.GetVersionedWithProof([]byte("bob"), 2) // "xyz", RangeProof, nil
proof.Verify([]byte("bob"), val, root) // nil

Proof of absence:

_, proof, err = tree.GetVersionedWithProof([]byte("tom"), 2) // nil, RangeProof, nil
proof.Verify([]byte("tom"), nil, root) // nil

Now we delete an old version:

tree.DeleteVersion(1)
tree.VersionExists(1) // false
tree.Get([]byte("alice")) // "xyz"
tree.GetVersioned([]byte("alice"), 1) // nil

Can't create a proof of absence for a version we no longer have:

_, proof, err = tree.GetVersionedWithProof([]byte("tom"), 1) // nil, nil, error

Index

Examples

Constants

View Source
const (
	// ModeLegacyLeftNode is the mode for legacy left child in the node encoding/decoding.
	ModeLegacyLeftNode = 0x01
	// ModeLegacyRightNode is the mode for legacy right child in the node encoding/decoding.
	ModeLegacyRightNode = 0x02
)

Variables

View Source
var (
	// ErrVersionDoesNotExist is returned if a requested version does not exist.
	ErrVersionDoesNotExist = errors.New("version does not exist")

	// ErrKeyDoesNotExist is returned if a key does not exist.
	ErrKeyDoesNotExist = errors.New("key does not exist")
)
View Source
var (
	ErrCloneLeafNode     = fmt.Errorf("attempt to copy a leaf node")
	ErrEmptyChild        = fmt.Errorf("found an empty child")
	ErrLeftNodeKeyEmpty  = fmt.Errorf("node.leftNodeKey was empty in writeBytes")
	ErrRightNodeKeyEmpty = fmt.Errorf("node.rightNodeKey was empty in writeBytes")
	ErrLeftHashIsNil     = fmt.Errorf("node.leftHash was nil in writeBytes")
	ErrRightHashIsNil    = fmt.Errorf("node.rightHash was nil in writeBytes")
)
View Source
var (
	// ErrInvalidProof is returned by Verify when a proof cannot be validated.
	ErrInvalidProof = fmt.Errorf("invalid proof")

	// ErrInvalidInputs is returned when the inputs passed to the function are invalid.
	ErrInvalidInputs = fmt.Errorf("invalid inputs")

	// ErrInvalidRoot is returned when the root passed in does not match the proof's.
	ErrInvalidRoot = fmt.Errorf("invalid root")
)
View Source
var (
	Version = ""
	Commit  = ""
	Branch  = ""
)

Version of iavl. Fill in fields with build flags

View Source
var ErrNoImport = errors.New("no import in progress")

ErrNoImport is returned when calling methods on a closed importer

View Source
var ErrNodeMissingNodeKey = fmt.Errorf("node does not have a nodeKey")
View Source
var ErrNotInitalizedTree = errors.New("iavl/export newExporter failed to create")

ErrNotInitalizedTree when chains introduce a store without initializing data

View Source
var ErrorExportDone = errors.New("export is complete")

ErrorExportDone is returned by Exporter.Next() when all items have been exported.

Functions

func GetRootKey added in v1.0.0

func GetRootKey(version int64) []byte

GetRootKey returns a byte slice of the root node key for the given version.

func NewIterator added in v0.19.0

func NewIterator(start, end []byte, ascending bool, tree *ImmutableTree) dbm.Iterator

Returns a new iterator over the immutable tree. If the tree is nil, the iterator will be invalid.

func PrintTree added in v0.8.0

func PrintTree(tree *ImmutableTree)

PrintTree prints the whole tree in an indented form.

func WriteDOTGraph

func WriteDOTGraph(w io.Writer, tree *ImmutableTree, paths []PathToLeaf)

func WriteDOTGraphToFile added in v1.0.0

func WriteDOTGraphToFile(filename string, tree *ImmutableTree)

WriteDOTGraphToFile writes the DOT graph to the given filename. Read like: $ dot /tmp/tree_one.dot -Tpng | display

func WriteDotGraphv2 added in v1.0.0

func WriteDotGraphv2(w io.Writer, tree *ImmutableTree)

WriteDotGraphv2 writes a DOT graph to the given writer. WriteDOTGraph failed to produce valid DOT graphs for large trees. This function is a rewrite of WriteDOTGraph that produces valid DOT graphs

Types

type BatchWithFlusher added in v1.0.0

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

BatchWithFlusher is a wrapper around batch that flushes batch's data to disk as soon as the configurable limit is reached.

func NewBatchWithFlusher added in v1.0.0

func NewBatchWithFlusher(db dbm.DB, flushThreshold int) *BatchWithFlusher

NewBatchWithFlusher returns new BatchWithFlusher wrapping the passed in batch

func (*BatchWithFlusher) Close added in v1.0.0

func (b *BatchWithFlusher) Close() error

func (*BatchWithFlusher) Delete added in v1.0.0

func (b *BatchWithFlusher) Delete(key []byte) error

Delete delete value at the given key to the db. If the deletion causes the underlying batch size to exceed batchSizeFlushThreshold, the batch is flushed to disk, cleared, and a new one is created with buffer pre-allocated to threshold. The deletion entry is then added to the batch.

func (*BatchWithFlusher) GetByteSize added in v1.0.0

func (b *BatchWithFlusher) GetByteSize() (int, error)

func (*BatchWithFlusher) Set added in v1.0.0

func (b *BatchWithFlusher) Set(key, value []byte) error

Set sets value at the given key to the db. If the set causes the underlying batch size to exceed flushThreshold, the batch is flushed to disk, cleared, and a new one is created with buffer pre-allocated to threshold. The addition entry is then added to the batch.

func (*BatchWithFlusher) Write added in v1.0.0

func (b *BatchWithFlusher) Write() error

func (*BatchWithFlusher) WriteSync added in v1.0.0

func (b *BatchWithFlusher) WriteSync() error

type ChangeSet added in v0.19.5

type ChangeSet = proto.ChangeSet

type CompressExporter added in v1.0.0

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

CompressExporter wraps the normal exporter to apply some compressions on `ExportNode`: - branch keys are skipped - leaf keys are encoded with delta compared with the previous leaf - branch node's version are encoded with delta compared with the max version in it's children

func (*CompressExporter) Next added in v1.0.0

func (e *CompressExporter) Next() (*ExportNode, error)

type CompressImporter added in v1.0.0

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

CompressImporter wraps the normal importer to do de-compressions before hand.

func (*CompressImporter) Add added in v1.0.0

func (i *CompressImporter) Add(node *ExportNode) error

type ExportNode added in v0.15.0

type ExportNode struct {
	Key     []byte
	Value   []byte
	Version int64
	Height  int8
}

ExportNode contains exported node data.

type Exporter added in v0.15.0

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

Exporter exports nodes from an ImmutableTree. It is created by ImmutableTree.Export().

Exported nodes can be imported into an empty tree with MutableTree.Import(). Nodes are exported depth-first post-order (LRN), this order must be preserved when importing in order to recreate the same tree structure.

func (*Exporter) Close added in v0.15.0

func (e *Exporter) Close()

Close closes the exporter. It is safe to call multiple times.

func (*Exporter) Next added in v0.15.0

func (e *Exporter) Next() (*ExportNode, error)

Next fetches the next exported node, or returns ExportDone when done.

type FastIterator added in v0.19.0

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

FastIterator is a dbm.Iterator for ImmutableTree it iterates over the latest state via fast nodes, taking advantage of keys being located in sequence in the underlying database.

func NewFastIterator added in v0.19.0

func NewFastIterator(start, end []byte, ascending bool, ndb *nodeDB) *FastIterator

func (*FastIterator) Close added in v0.19.0

func (iter *FastIterator) Close() error

Close implements dbm.Iterator

func (*FastIterator) Domain added in v0.19.0

func (iter *FastIterator) Domain() ([]byte, []byte)

Domain implements dbm.Iterator. Maps the underlying nodedb iterator domain, to the 'logical' keys involved.

func (*FastIterator) Error added in v0.19.0

func (iter *FastIterator) Error() error

Error implements dbm.Iterator

func (*FastIterator) Key added in v0.19.0

func (iter *FastIterator) Key() []byte

Key implements dbm.Iterator

func (*FastIterator) Next added in v0.19.0

func (iter *FastIterator) Next()

Next implements dbm.Iterator

func (*FastIterator) Valid added in v0.19.0

func (iter *FastIterator) Valid() bool

Valid implements dbm.Iterator.

func (*FastIterator) Value added in v0.19.0

func (iter *FastIterator) Value() []byte

Value implements dbm.Iterator

type ImmutableTree added in v0.10.0

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

ImmutableTree contains the immutable tree at a given version. It is typically created by calling MutableTree.GetImmutable(), in which case the returned tree is safe for concurrent access as long as the version is not deleted via DeleteVersion() or the tree's pruning settings.

Returned key/value byte slices must not be modified, since they may point to data located inside IAVL which would also be modified.

func NewImmutableTree added in v0.10.0

func NewImmutableTree(db dbm.DB, cacheSize int, skipFastStorageUpgrade bool, lg log.Logger, options ...Option) *ImmutableTree

NewImmutableTree creates both in-memory and persistent instances

func (*ImmutableTree) Export added in v0.15.0

func (t *ImmutableTree) Export() (*Exporter, error)

Export returns an iterator that exports tree nodes as ExportNodes. These nodes can be imported with MutableTree.Import() to recreate an identical tree.

func (*ImmutableTree) Get added in v0.10.0

func (t *ImmutableTree) Get(key []byte) ([]byte, error)

Get returns the value of the specified key if it exists, or nil. The returned value must not be modified, since it may point to data stored within IAVL. Get potentially employs a more performant strategy than GetWithIndex for retrieving the value. If tree.skipFastStorageUpgrade is true, this will work almost the same as GetWithIndex.

func (*ImmutableTree) GetByIndex added in v0.10.0

func (t *ImmutableTree) GetByIndex(index int64) (key []byte, value []byte, err error)

GetByIndex gets the key and value at the specified index.

func (*ImmutableTree) GetMembershipProof added in v0.15.0

func (t *ImmutableTree) GetMembershipProof(key []byte) (*ics23.CommitmentProof, error)

GetMembershipProof will produce a CommitmentProof that the given key (and queries value) exists in the iavl tree. If the key doesn't exist in the tree, this will return an error.

func (*ImmutableTree) GetNonMembershipProof added in v0.15.0

func (t *ImmutableTree) GetNonMembershipProof(key []byte) (*ics23.CommitmentProof, error)

GetNonMembershipProof will produce a CommitmentProof that the given key doesn't exist in the iavl tree. If the key exists in the tree, this will return an error.

func (*ImmutableTree) GetProof added in v0.20.0

func (t *ImmutableTree) GetProof(key []byte) (*ics23.CommitmentProof, error)

GetProof gets the proof for the given key.

func (*ImmutableTree) GetWithIndex added in v0.19.0

func (t *ImmutableTree) GetWithIndex(key []byte) (int64, []byte, error)

GetWithIndex returns the index and value of the specified key if it exists, or nil and the next index otherwise. The returned value must not be modified, since it may point to data stored within IAVL.

The index is the index in the list of leaf nodes sorted lexicographically by key. The leftmost leaf has index 0. It's neighbor has index 1 and so on.

func (*ImmutableTree) Has added in v0.10.0

func (t *ImmutableTree) Has(key []byte) (bool, error)

Has returns whether or not a key exists.

func (*ImmutableTree) Hash added in v0.10.0

func (t *ImmutableTree) Hash() []byte

Hash returns the root hash.

func (*ImmutableTree) Height added in v0.10.0

func (t *ImmutableTree) Height() int8

Height returns the height of the tree.

func (*ImmutableTree) IsFastCacheEnabled added in v0.19.0

func (t *ImmutableTree) IsFastCacheEnabled() (bool, error)

IsFastCacheEnabled returns true if fast cache is enabled, false otherwise. For fast cache to be enabled, the following 2 conditions must be met: 1. The tree is of the latest version. 2. The underlying storage has been upgraded to fast cache

func (*ImmutableTree) Iterate added in v0.10.0

func (t *ImmutableTree) Iterate(fn func(key []byte, value []byte) bool) (bool, error)

Iterate iterates over all keys of the tree. The keys and values must not be modified, since they may point to data stored within IAVL. Returns true if stopped by callback, false otherwise

func (*ImmutableTree) IterateRange added in v0.10.0

func (t *ImmutableTree) IterateRange(start, end []byte, ascending bool, fn func(key []byte, value []byte) bool) (stopped bool)

IterateRange makes a callback for all nodes with key between start and end non-inclusive. If either are nil, then it is open on that side (nil, nil is the same as Iterate). The keys and values must not be modified, since they may point to data stored within IAVL.

func (*ImmutableTree) IterateRangeInclusive added in v0.10.0

func (t *ImmutableTree) IterateRangeInclusive(start, end []byte, ascending bool, fn func(key, value []byte, version int64) bool) (stopped bool)

IterateRangeInclusive makes a callback for all nodes with key between start and end inclusive. If either are nil, then it is open on that side (nil, nil is the same as Iterate). The keys and values must not be modified, since they may point to data stored within IAVL.

func (*ImmutableTree) Iterator added in v0.17.2

func (t *ImmutableTree) Iterator(start, end []byte, ascending bool) (dbm.Iterator, error)

Iterator returns an iterator over the immutable tree.

func (*ImmutableTree) RenderShape added in v0.15.0

func (t *ImmutableTree) RenderShape(indent string, encoder NodeEncoder) ([]string, error)

RenderShape provides a nested tree shape, ident is prepended in each level Returns an array of strings, one per line, to join with "\n" or display otherwise

func (*ImmutableTree) Size added in v0.10.0

func (t *ImmutableTree) Size() int64

Size returns the number of leaf nodes in the tree.

func (*ImmutableTree) String added in v0.10.0

func (t *ImmutableTree) String() string

String returns a string representation of Tree.

func (*ImmutableTree) TraverseStateChanges added in v0.19.5

func (t *ImmutableTree) TraverseStateChanges(startVersion, endVersion int64, fn func(version int64, changeSet *ChangeSet) error) error

TraverseStateChanges iterate the range of versions, compare each version to it's predecessor to extract the state changes of it. endVersion is exclusive.

func (*ImmutableTree) VerifyMembership added in v0.20.0

func (t *ImmutableTree) VerifyMembership(proof *ics23.CommitmentProof, key []byte) (bool, error)

VerifyMembership returns true iff proof is an ExistenceProof for the given key.

func (*ImmutableTree) VerifyNonMembership added in v0.20.0

func (t *ImmutableTree) VerifyNonMembership(proof *ics23.CommitmentProof, key []byte) (bool, error)

VerifyNonMembership returns true iff proof is a NonExistenceProof for the given key.

func (*ImmutableTree) VerifyProof added in v0.20.0

func (t *ImmutableTree) VerifyProof(proof *ics23.CommitmentProof, key []byte) (bool, error)

VerifyProof checks if the proof is correct for the given key.

func (*ImmutableTree) Version added in v0.10.0

func (t *ImmutableTree) Version() int64

Version returns the version of the tree.

type Importer added in v0.15.0

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

Importer imports data into an empty MutableTree. It is created by MutableTree.Import(). Users must call Close() when done.

ExportNodes must be imported in the order returned by Exporter, i.e. depth-first post-order (LRN).

Importer is not concurrency-safe, it is the caller's responsibility to ensure the tree is not modified while performing an import.

Example
tree := NewMutableTree(dbm.NewMemDB(), 0, false, log.NewNopLogger())

_, err := tree.Set([]byte("a"), []byte{1})
if err != nil {
	panic(err)
}

_, err = tree.Set([]byte("b"), []byte{2})
if err != nil {
	panic(err)
}
_, err = tree.Set([]byte("c"), []byte{3})
if err != nil {
	panic(err)
}
_, version, err := tree.SaveVersion()
if err != nil {
	panic(err)
}

itree, err := tree.GetImmutable(version)
if err != nil {
	panic(err)
}
exporter, err := itree.Export()
if err != nil {
	panic(err)
}
defer exporter.Close()
exported := []*ExportNode{}
for {
	var node *ExportNode
	node, err = exporter.Next()
	if err == ErrorExportDone {
		break
	} else if err != nil {
		panic(err)
	}
	exported = append(exported, node)
}

newTree := NewMutableTree(dbm.NewMemDB(), 0, false, log.NewNopLogger())
importer, err := newTree.Import(version)
if err != nil {
	panic(err)
}
defer importer.Close()
for _, node := range exported {
	err = importer.Add(node)
	if err != nil {
		panic(err)
	}
}
err = importer.Commit()
if err != nil {
	panic(err)
}
Output:

func (*Importer) Add added in v0.15.0

func (i *Importer) Add(exportNode *ExportNode) error

Add adds an ExportNode to the import. ExportNodes must be added in the order returned by Exporter, i.e. depth-first post-order (LRN). Nodes are periodically flushed to the database, but the imported version is not visible until Commit() is called.

func (*Importer) Close added in v0.15.0

func (i *Importer) Close()

Close frees all resources. It is safe to call multiple times. Uncommitted nodes may already have been flushed to the database, but will not be visible.

func (*Importer) Commit added in v0.15.0

func (i *Importer) Commit() error

Commit finalizes the import by flushing any outstanding nodes to the database, making the version visible, and updating the tree metadata. It can only be called once, and calls Close() internally.

type Iterator added in v0.17.2

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

Iterator is a dbm.Iterator for ImmutableTree

func (*Iterator) Close added in v0.17.2

func (iter *Iterator) Close() error

Close implements dbm.Iterator

func (*Iterator) Domain added in v0.17.2

func (iter *Iterator) Domain() ([]byte, []byte)

Domain implements dbm.Iterator.

func (*Iterator) Error added in v0.17.2

func (iter *Iterator) Error() error

Error implements dbm.Iterator

func (*Iterator) IsFast added in v0.19.0

func (iter *Iterator) IsFast() bool

IsFast returnts true if iterator uses fast strategy

func (*Iterator) Key added in v0.17.2

func (iter *Iterator) Key() []byte

Key implements dbm.Iterator

func (*Iterator) Next added in v0.17.2

func (iter *Iterator) Next()

Next implements dbm.Iterator

func (*Iterator) Valid added in v0.17.2

func (iter *Iterator) Valid() bool

Valid implements dbm.Iterator.

func (*Iterator) Value added in v0.17.2

func (iter *Iterator) Value() []byte

Value implements dbm.Iterator

type KVPair added in v0.19.5

type KVPair = proto.KVPair

type KVPairReceiver added in v0.19.5

type KVPairReceiver func(pair *KVPair) error

KVPairReceiver is callback parameter of method `extractStateChanges` to receive stream of `KVPair`s.

type MutableTree added in v0.10.0

type MutableTree struct {
	*ImmutableTree // The current, working tree.
	// contains filtered or unexported fields
}

MutableTree is a persistent tree which keeps track of versions. It is not safe for concurrent use, and should be guarded by a Mutex or RWLock as appropriate. An immutable tree at a given version can be returned via GetImmutable, which is safe for concurrent access.

Given and returned key/value byte slices must not be modified, since they may point to data located inside IAVL which would also be modified.

The inner ImmutableTree should not be used directly by callers.

func NewMutableTree added in v0.10.0

func NewMutableTree(db dbm.DB, cacheSize int, skipFastStorageUpgrade bool, lg log.Logger, options ...Option) *MutableTree

NewMutableTree returns a new tree with the specified optional options.

func (*MutableTree) AvailableVersions added in v0.15.0

func (tree *MutableTree) AvailableVersions() []int

AvailableVersions returns all available versions in ascending order

func (*MutableTree) Close added in v1.1.1

func (tree *MutableTree) Close() error

Close closes the tree.

func (*MutableTree) DeleteVersionsTo added in v1.0.0

func (tree *MutableTree) DeleteVersionsTo(toVersion int64) error

DeleteVersionsTo removes versions upto the given version from the MutableTree. It will not block the SaveVersion() call, instead it will be queued and executed deferred.

func (*MutableTree) Get added in v0.19.0

func (tree *MutableTree) Get(key []byte) ([]byte, error)

Get returns the value of the specified key if it exists, or nil otherwise. The returned value must not be modified, since it may point to data stored within IAVL.

func (*MutableTree) GetImmutable added in v0.10.0

func (tree *MutableTree) GetImmutable(version int64) (*ImmutableTree, error)

GetImmutable loads an ImmutableTree at a given version for querying. The returned tree is safe for concurrent access, provided the version is not deleted, e.g. via `DeleteVersion()`.

func (*MutableTree) GetVersioned added in v0.10.0

func (tree *MutableTree) GetVersioned(key []byte, version int64) ([]byte, error)

GetVersioned gets the value at the specified key and version. The returned value must not be modified, since it may point to data stored within IAVL.

func (*MutableTree) GetVersionedProof added in v0.20.0

func (tree *MutableTree) GetVersionedProof(key []byte, version int64) (*ics23.CommitmentProof, error)

GetVersionedProof gets the proof for the given key at the specified version.

func (*MutableTree) Hash added in v0.10.0

func (tree *MutableTree) Hash() []byte

Hash returns the hash of the latest saved version of the tree, as returned by SaveVersion. If no versions have been saved, Hash returns nil.

func (*MutableTree) Import added in v0.15.0

func (tree *MutableTree) Import(version int64) (*Importer, error)

Import returns an importer for tree nodes previously exported by ImmutableTree.Export(), producing an identical IAVL tree. The caller must call Close() on the importer when done.

version should correspond to the version that was initially exported. It must be greater than or equal to the highest ExportNode version number given.

Import can only be called on an empty tree. It is the callers responsibility that no other modifications are made to the tree while importing.

func (*MutableTree) IsEmpty added in v0.10.0

func (tree *MutableTree) IsEmpty() bool

IsEmpty returns whether or not the tree has any keys. Only trees that are not empty can be saved.

func (*MutableTree) IsUpgradeable added in v0.19.0

func (tree *MutableTree) IsUpgradeable() (bool, error)

Returns true if the tree may be auto-upgraded, false otherwise An example of when an upgrade may be performed is when we are enaling fast storage for the first time or need to overwrite fast nodes due to mismatch with live state.

func (*MutableTree) Iterate added in v0.19.0

func (tree *MutableTree) Iterate(fn func(key []byte, value []byte) bool) (stopped bool, err error)

Iterate iterates over all keys of the tree. The keys and values must not be modified, since they may point to data stored within IAVL. Returns true if stopped by callnack, false otherwise

func (*MutableTree) Iterator added in v0.19.0

func (tree *MutableTree) Iterator(start, end []byte, ascending bool) (dbm.Iterator, error)

Iterator returns an iterator over the mutable tree. CONTRACT: no updates are made to the tree while an iterator is active.

func (*MutableTree) Load added in v0.10.0

func (tree *MutableTree) Load() (int64, error)

Load the latest versioned tree from disk.

func (*MutableTree) LoadVersion added in v0.10.0

func (tree *MutableTree) LoadVersion(targetVersion int64) (int64, error)

Returns the version number of the specific version found

func (*MutableTree) LoadVersionForOverwriting added in v0.11.1

func (tree *MutableTree) LoadVersionForOverwriting(targetVersion int64) error

loadVersionForOverwriting attempts to load a tree at a previously committed version, or the latest version below it. Any versions greater than targetVersion will be deleted.

func (*MutableTree) Remove added in v0.10.0

func (tree *MutableTree) Remove(key []byte) ([]byte, bool, error)

Remove removes a key from the working tree. The given key byte slice should not be modified after this call, since it may point to data stored inside IAVL.

func (*MutableTree) Rollback added in v0.10.0

func (tree *MutableTree) Rollback()

Rollback resets the working tree to the latest saved version, discarding any unsaved modifications.

func (*MutableTree) SaveChangeSet added in v0.20.1

func (tree *MutableTree) SaveChangeSet(cs *ChangeSet) (int64, error)

SaveChangeSet saves a ChangeSet to the tree. It is used to replay a ChangeSet as a new version.

func (*MutableTree) SaveVersion added in v0.10.0

func (tree *MutableTree) SaveVersion() ([]byte, int64, error)

SaveVersion saves a new tree version to disk, based on the current state of the tree. Returns the hash and new version number.

func (*MutableTree) Set added in v0.10.0

func (tree *MutableTree) Set(key, value []byte) (updated bool, err error)

Set sets a key in the working tree. Nil values are invalid. The given key/value byte slices must not be modified after this call, since they point to slices stored within IAVL. It returns true when an existing value was updated, while false means it was a new key.

func (*MutableTree) SetInitialVersion added in v0.15.0

func (tree *MutableTree) SetInitialVersion(version uint64)

SetInitialVersion sets the initial version of the tree, replacing Options.InitialVersion. It is only used during the initial SaveVersion() call for a tree with no other versions, and is otherwise ignored.

func (*MutableTree) String added in v0.10.0

func (tree *MutableTree) String() (string, error)

String returns a string representation of the tree.

func (*MutableTree) VersionExists added in v0.10.0

func (tree *MutableTree) VersionExists(version int64) bool

VersionExists returns whether or not a version exists.

func (*MutableTree) WorkingHash added in v0.10.0

func (tree *MutableTree) WorkingHash() []byte

WorkingHash returns the hash of the current working tree.

func (*MutableTree) WorkingVersion added in v1.0.0

func (tree *MutableTree) WorkingVersion() int64

type Node

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

Node represents a node in a Tree.

func MakeLegacyNode added in v1.0.0

func MakeLegacyNode(hash, buf []byte) (*Node, error)

MakeLegacyNode constructs a legacy *Node from an encoded byte slice.

func MakeNode

func MakeNode(nk, buf []byte) (*Node, error)

MakeNode constructs an *Node from an encoded byte slice.

func NewNode

func NewNode(key []byte, value []byte) *Node

NewNode returns a new node from a key, value and version.

func (*Node) GetKey added in v0.19.1

func (node *Node) GetKey() []byte

GetKey returns the key of the node.

func (*Node) PathToLeaf added in v0.8.0

func (node *Node) PathToLeaf(t *ImmutableTree, key []byte, version int64) (PathToLeaf, *Node, error)

If the key does not exist, returns the path to the next leaf left of key (w/ path), except when key is less than the least item, in which case it returns a path to the least item.

func (*Node) String

func (node *Node) String() string

String returns a string representation of the node.

type NodeEncoder added in v0.15.0

type NodeEncoder func(id []byte, depth int, isLeaf bool) string

NodeEncoder will take an id (hash, or key for leaf nodes), the depth of the node, and whether or not this is a leaf node. It returns the string we wish to print, for iaviwer

type NodeExporter added in v1.0.0

type NodeExporter interface {
	Next() (*ExportNode, error)
}

func NewCompressExporter added in v1.0.0

func NewCompressExporter(exporter NodeExporter) NodeExporter

type NodeImporter added in v1.0.0

type NodeImporter interface {
	Add(*ExportNode) error
}

func NewCompressImporter added in v1.0.0

func NewCompressImporter(importer NodeImporter) NodeImporter

type NodeIterator added in v0.19.5

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

NodeIterator is an iterator for nodeDB to traverse a tree in depth-first, preorder manner.

func NewNodeIterator added in v0.19.5

func NewNodeIterator(rootKey []byte, ndb *nodeDB) (*NodeIterator, error)

NewNodeIterator returns a new NodeIterator to traverse the tree of the root node.

func (*NodeIterator) Error added in v0.19.5

func (iter *NodeIterator) Error() error

Error returns an error if any errors.

func (*NodeIterator) GetNode added in v0.19.5

func (iter *NodeIterator) GetNode() *Node

GetNode returns the current visiting node.

func (*NodeIterator) Next added in v0.19.5

func (iter *NodeIterator) Next(isSkipped bool)

Next moves forward the traversal. if isSkipped is true, the subtree under the current node is skipped.

func (*NodeIterator) Valid added in v0.19.5

func (iter *NodeIterator) Valid() bool

Valid checks if the validator is valid.

type NodeKey added in v1.0.0

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

NodeKey represents a key of node in the DB.

func GetNodeKey added in v1.0.0

func GetNodeKey(key []byte) *NodeKey

GetNodeKey returns a NodeKey from a byte slice.

func (*NodeKey) GetKey added in v1.0.0

func (nk *NodeKey) GetKey() []byte

GetKey returns a byte slice of the NodeKey.

func (*NodeKey) String added in v1.0.0

func (nk *NodeKey) String() string

String returns a string representation of the node key.

type Option added in v1.0.0

type Option func(*Options)

func FlushThresholdOption added in v1.0.0

func FlushThresholdOption(ft int) Option

FlushThresholdOption sets the FlushThreshold for the batcher.

func InitialVersionOption added in v1.0.0

func InitialVersionOption(iv uint64) Option

InitialVersionOption sets the initial version for the tree.

func StatOption added in v1.0.0

func StatOption(stats *Statistics) Option

StatOption sets the Statistics for the tree.

func SyncOption added in v1.0.0

func SyncOption(sync bool) Option

SyncOption sets the Sync option.

type Options added in v0.15.0

type Options struct {
	// Sync synchronously flushes all writes to storage, using e.g. the fsync syscall.
	// Disabling this significantly improves performance, but can lose data on e.g. power loss.
	Sync bool

	// InitialVersion specifies the initial version number. If any versions already exist below
	// this, an error is returned when loading the tree. Only used for the initial SaveVersion()
	// call.
	InitialVersion uint64

	// When Stat is not nil, statistical logic needs to be executed
	Stat *Statistics

	// Ethereum has found that commit of 100KB is optimal, ref ethereum/go-ethereum#15115
	FlushThreshold int
}

Options define tree options.

func DefaultOptions added in v0.15.0

func DefaultOptions() Options

DefaultOptions returns the default options for IAVL.

type PathToLeaf added in v0.8.0

type PathToLeaf []ProofInnerNode

PathToLeaf represents an inner path to a leaf node. Note that the nodes are ordered such that the last one is closest to the root of the tree.

func (PathToLeaf) Index added in v0.9.1

func (pl PathToLeaf) Index() (idx int64)

returns -1 if invalid.

func (PathToLeaf) String added in v0.8.0

func (pl PathToLeaf) String() string

type ProofInnerNode added in v0.15.0

type ProofInnerNode struct {
	Height  int8   `json:"height"`
	Size    int64  `json:"size"`
	Version int64  `json:"version"`
	Left    []byte `json:"left"`
	Right   []byte `json:"right"`
}

func (ProofInnerNode) Hash added in v0.15.0

func (pin ProofInnerNode) Hash(childHash []byte) ([]byte, error)

func (ProofInnerNode) String added in v0.15.0

func (pin ProofInnerNode) String() string

type ProofLeafNode added in v0.15.0

type ProofLeafNode struct {
	Key       hexbytes.HexBytes `json:"key"`
	ValueHash hexbytes.HexBytes `json:"value"`
	Version   int64             `json:"version"`
}

func (ProofLeafNode) Hash added in v0.15.0

func (pln ProofLeafNode) Hash() ([]byte, error)

func (ProofLeafNode) String added in v0.15.0

func (pln ProofLeafNode) String() string

type Statistics added in v0.19.0

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

Statisc about db runtime state

func (*Statistics) GetCacheHitCnt added in v0.19.0

func (stat *Statistics) GetCacheHitCnt() uint64

func (*Statistics) GetCacheMissCnt added in v0.19.0

func (stat *Statistics) GetCacheMissCnt() uint64

func (*Statistics) GetFastCacheHitCnt added in v0.19.0

func (stat *Statistics) GetFastCacheHitCnt() uint64

func (*Statistics) GetFastCacheMissCnt added in v0.19.0

func (stat *Statistics) GetFastCacheMissCnt() uint64

func (*Statistics) IncCacheHitCnt added in v0.19.0

func (stat *Statistics) IncCacheHitCnt()

func (*Statistics) IncCacheMissCnt added in v0.19.0

func (stat *Statistics) IncCacheMissCnt()

func (*Statistics) IncFastCacheHitCnt added in v0.19.0

func (stat *Statistics) IncFastCacheHitCnt()

func (*Statistics) IncFastCacheMissCnt added in v0.19.0

func (stat *Statistics) IncFastCacheMissCnt()

func (*Statistics) Reset added in v0.19.0

func (stat *Statistics) Reset()

type UnsavedFastIterator added in v0.19.0

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

UnsavedFastIterator is a dbm.Iterator for ImmutableTree it iterates over the latest state via fast nodes, taking advantage of keys being located in sequence in the underlying database.

func NewUnsavedFastIterator added in v0.19.0

func NewUnsavedFastIterator(start, end []byte, ascending bool, ndb *nodeDB, unsavedFastNodeAdditions, unsavedFastNodeRemovals *sync.Map) *UnsavedFastIterator

func (*UnsavedFastIterator) Close added in v0.19.0

func (iter *UnsavedFastIterator) Close() error

Close implements dbm.Iterator

func (*UnsavedFastIterator) Domain added in v0.19.0

func (iter *UnsavedFastIterator) Domain() ([]byte, []byte)

Domain implements dbm.Iterator. Maps the underlying nodedb iterator domain, to the 'logical' keys involved.

func (*UnsavedFastIterator) Error added in v0.19.0

func (iter *UnsavedFastIterator) Error() error

Error implements dbm.Iterator

func (*UnsavedFastIterator) Key added in v0.19.0

func (iter *UnsavedFastIterator) Key() []byte

Key implements dbm.Iterator

func (*UnsavedFastIterator) Next added in v0.19.0

func (iter *UnsavedFastIterator) Next()

Next implements dbm.Iterator Its effectively running the constant space overhead algorithm for streaming through sorted lists: the sorted lists being underlying fast nodes & unsavedFastNodeChanges

func (*UnsavedFastIterator) Valid added in v0.19.0

func (iter *UnsavedFastIterator) Valid() bool

Valid implements dbm.Iterator.

func (*UnsavedFastIterator) Value added in v0.19.0

func (iter *UnsavedFastIterator) Value() []byte

Value implements dbm.Iterator

type VersionInfo added in v0.15.0

type VersionInfo struct {
	IAVL      string `json:"iavl"`
	GitCommit string `json:"commit"`
	Branch    string `json:"branch"`
	GoVersion string `json:"go"`
}

VersionInfo contains useful versioning information in struct

func GetVersionInfo added in v0.15.0

func GetVersionInfo() VersionInfo

Returns VersionInfo with global vars filled in

func (VersionInfo) String added in v0.15.0

func (v VersionInfo) String() string

Directories

Path Synopsis
benchmarks
cmd
legacydump Module
internal
color
TODO: Can we delete this package.
TODO: Can we delete this package.
Package mock is a generated GoMock package.
Package mock is a generated GoMock package.

Jump to

Keyboard shortcuts

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