memiavl

package
v0.0.36 Latest Latest
Warning

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

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

README

MemIAVL

Changelog

The Design

The idea of MemIAVL is to keep the whole chain state in memory as much as possible to speed up reads and writes.

  • MemIAVL uses a write-ahead-log(WAL) to persist the changeset from transaction commit to speed up writes.
  • Instead of updating and flushing nodes to disk, state changes at every height are actually only written to WAL file
  • MemIAVL snapshots are taken periodically and written to disk to materialize the tree at some given height H
  • Each snapshot is composed of 3 files per module, one for key/value pairs, one for leaf nodes and one for branch nodes
  • After snapshot is taken, the snapshot files are then loaded with mmap for faster reads and lazy loading via page cache. At the same time, older WAL files will be truncated till the snapshot height
  • Each MemIAVL tree is composed of 2 types of node: MemNode and Persistent Node
    • All nodes are persistent nodes to start with. Each persistent node maps to some data stored on file
    • During updates or insertion, persistent nodes will turn into MemNode
    • MemNodes are nodes stores only in memory for all future read and writes
  • If a node crash in the middle of commit, it will be able to load from the last snapshot and replay the WAL file to catch up to the last committed height
Advantages
  • Better write amplification, we only need to write the change sets in real time which is much more compact than IAVL nodes, IAVL snapshot can be created in much lower frequency.
  • Better read amplification, the IAVL snapshot is a plain file, the nodes are referenced with offset, the read amplification is simply 1.
  • Better space amplification, the archived change sets are much more compact than current IAVL tree, in our test case, the ratio could be as large as 1:100. We don't need to keep too old IAVL snapshots, because versiondb will handle the historical key-value queries, IAVL tree only takes care of merkle proof generations for blocks within an unbonding period. In very rare cases that do need IAVL tree of very old version, you can always replay the change sets from the genesis.
  • Facilitate async commit which improves commit latency by huge amount
Trade-offs
  • Performance can degrade when state size grows much larger than memory
  • MemIAVL makes historical proof much slower
  • Periodic snapshot creation is a very heavy operation and could become a bottleneck
IAVL Snapshot

IAVL snapshot is composed by four files:

  • metadata, 16bytes:

    magic: 4
    format: 4
    version: 4
    root node index: 4
    
  • nodes, array of fixed size(16+32bytes) nodes, the node format is like this:

    # branch
    height   : 1
    _padding : 3
    version  : 4
    size     : 4
    key node : 4
    hash     : [32]byte
    
    # leaf
    height      : 1
    _padding    : 3
    version     : 4
    key offset  : 8
    hash        : [32]byte
    

    The node has fixed length, can be indexed directly. The nodes references each other with the node index, nodes are written with post-order depth-first traversal, so the root node is always placed at the end.

    For branch node, the key node field reference the smallest leaf node in the right branch, the key slice is fetched from there indirectly, the leaf nodes stores the offset into the kvs file, where the key and value slices can be built.

    The branch node's left/child node indexes are inferenced from existing information and properties of post-order traversal:

    right child index = self index - 1
    left child index = key node - 1
    

    The version/size/node indexes are encoded with 4 bytes, should be enough in foreseeable future, but could be changed to more bytes in the future.

    The implementation will read the mmap-ed content in a zero-copy way, won't use extra node cache, it will only rely on the OS page cache.

  • kvs, sequence of leaf node key-value pairs, the keys are ordered and no duplication.

    keyLen: varint-uint64
    key
    valueLen: varint-uint64
    value
    *repeat*
    

Documentation

Index

Constants

View Source
const (
	SnapshotPrefix = "snapshot-"
	SnapshotDirLen = len(SnapshotPrefix) + 20
)
View Source
const (
	OffsetHeight   = 0
	OffsetPreTrees = OffsetHeight + 1
	OffsetVersion  = OffsetHeight + 4
	OffsetSize     = OffsetVersion + 4
	OffsetKeyLeaf  = OffsetSize + 4

	OffsetHash          = OffsetKeyLeaf + 4
	SizeHash            = sha256.Size
	SizeNodeWithoutHash = OffsetHash
	SizeNode            = SizeNodeWithoutHash + SizeHash

	OffsetLeafVersion   = 0
	OffsetLeafKeyLen    = OffsetLeafVersion + 4
	OffsetLeafKeyOffset = OffsetLeafKeyLen + 4
	OffsetLeafHash      = OffsetLeafKeyOffset + 8
	SizeLeafWithoutHash = OffsetLeafHash
	SizeLeaf            = SizeLeafWithoutHash + SizeHash
)
View Source
const (
	// SnapshotFileMagic is little endian encoded b"IAVL"
	SnapshotFileMagic = 1280721225

	// the initial snapshot format
	SnapshotFormat = 0

	// magic: uint32, format: uint32, version: uint32
	SizeMetadata = 12

	FileNameNodes    = "nodes"
	FileNameLeaves   = "leaves"
	FileNameKVs      = "kvs"
	FileNameMetadata = "metadata"
)
View Source
const LockFileName = "LOCK"
View Source
const MetadataFileName = "__metadata"

Variables

This section is empty.

Functions

func EncodeBytes added in v0.0.23

func EncodeBytes(w io.Writer, bz []byte) error

EncodeBytes writes a varint length-prefixed byte slice to the writer, it's used for hash computation, must be compactible with the official IAVL implementation.

func GetLatestVersion added in v0.0.23

func GetLatestVersion(dir string) (int64, error)

GetLatestVersion finds the latest version number without loading the whole db, it's needed for upgrade module to check store upgrades, it returns 0 if db doesn't exist or is empty.

func HashNode added in v0.0.23

func HashNode(node Node) []byte

HashNode computes the hash of the node.

func Mmap added in v0.0.23

func Mmap(f *os.File) ([]byte, *[mmap.MaxMapSize]byte, error)

func VerifyHash added in v0.0.23

func VerifyHash(node Node) bool

VerifyHash compare node's cached hash with computed one

func WriteFileSync added in v0.0.23

func WriteFileSync(name string, data []byte) error

WriteFileSync calls `f.Sync` after before closing the file

Types

type DB added in v0.0.23

type DB struct {
	MultiTree
	// contains filtered or unexported fields
}

DB implements DB-like functionalities on top of MultiTree: - async snapshot rewriting - Write-ahead-log

The memiavl.db directory looks like this: ``` > current -> snapshot-N > snapshot-N > bank > kvs > nodes > metadata > acc > ... other stores > rlog ```

func OpenDB added in v0.0.23

func OpenDB(logger logger.Logger, targetVersion int64, opts Options) (*DB, error)

func (*DB) ApplyChangeSet added in v0.0.23

func (db *DB) ApplyChangeSet(name string, changeSet iavl.ChangeSet) error

ApplyChangeSet wraps MultiTree.ApplyChangeSet, it also appends the changesets in the pending log, which will be persisted to the rlog in next Commit call.

func (*DB) ApplyChangeSets added in v0.0.23

func (db *DB) ApplyChangeSets(changeSets []*proto.NamedChangeSet) error

ApplyChangeSets wraps MultiTree.ApplyChangeSets, it also appends the changesets in the pending log, which will be persisted to the rlog in next Commit call.

func (*DB) ApplyUpgrades added in v0.0.23

func (db *DB) ApplyUpgrades(upgrades []*proto.TreeNameUpgrade) error

ApplyUpgrades wraps MultiTree.ApplyUpgrades, it also appends the upgrades in a pending log, which will be persisted to the rlog in next Commit call.

func (*DB) Close added in v0.0.23

func (db *DB) Close() error

func (*DB) Commit added in v0.0.23

func (db *DB) Commit() (int64, error)

Commit wraps SaveVersion to bump the version and writes the pending changes into log files to persist on disk

func (*DB) CommittedVersion added in v0.0.23

func (db *DB) CommittedVersion() (int64, error)

CommittedVersion returns the latest version written in rlog file, or snapshot version if rlog is empty.

func (*DB) Copy added in v0.0.23

func (db *DB) Copy() *DB

func (*DB) LastCommitInfo added in v0.0.23

func (db *DB) LastCommitInfo() *proto.CommitInfo

LastCommitInfo returns the last commit info.

func (*DB) ReadOnly added in v0.0.23

func (db *DB) ReadOnly() bool

ReadOnly returns whether the DB is opened in read-only mode.

func (*DB) Reload added in v0.0.23

func (db *DB) Reload() error

func (*DB) RewriteSnapshot added in v0.0.23

func (db *DB) RewriteSnapshot(ctx context.Context) error

RewriteSnapshot writes the current version of memiavl into a snapshot, and update the `current` symlink.

func (*DB) RewriteSnapshotBackground added in v0.0.23

func (db *DB) RewriteSnapshotBackground() error

RewriteSnapshotBackground rewrite snapshot in a background goroutine, `Commit` will check the complete status, and switch to the new snapshot.

func (*DB) SaveVersion added in v0.0.23

func (db *DB) SaveVersion(updateCommitInfo bool) (int64, error)

func (*DB) SetInitialVersion added in v0.0.23

func (db *DB) SetInitialVersion(initialVersion int64) error

SetInitialVersion wraps `MultiTree.SetInitialVersion`. it will do a snapshot rewrite, because we can't use rlog to record this change, we need it to convert versions to rlog index in the first place.

func (*DB) TreeByName added in v0.0.23

func (db *DB) TreeByName(name string) *Tree

TreeByName wraps MultiTree.TreeByName to add a lock.

func (*DB) UpdateCommitInfo added in v0.0.23

func (db *DB) UpdateCommitInfo()

UpdateCommitInfo wraps MultiTree.UpdateCommitInfo to add a lock.

func (*DB) Version added in v0.0.23

func (db *DB) Version() int64

Version wraps MultiTree.Version to add a lock.

func (*DB) WorkingCommitInfo added in v0.0.23

func (db *DB) WorkingCommitInfo() *proto.CommitInfo

func (*DB) WriteSnapshot added in v0.0.23

func (db *DB) WriteSnapshot(dir string) error

WriteSnapshot wraps MultiTree.WriteSnapshot to add a lock.

type Exporter added in v0.0.23

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

func (*Exporter) Close added in v0.0.23

func (e *Exporter) Close()

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

func (*Exporter) Next added in v0.0.23

func (e *Exporter) Next() (*types.SnapshotNode, error)

type FileLock added in v0.0.23

type FileLock interface {
	Unlock() error
	Destroy() error
}

func LockFile added in v0.0.23

func LockFile(fname string) (FileLock, error)

type Iterator added in v0.0.23

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

func NewIterator added in v0.0.23

func NewIterator(start, end []byte, ascending bool, root Node, zeroCopy bool) *Iterator

func (*Iterator) Close added in v0.0.23

func (iter *Iterator) Close() error

Close implements dbm.Iterator

func (*Iterator) Domain added in v0.0.23

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

func (*Iterator) Error added in v0.0.23

func (iter *Iterator) Error() error

Error implements dbm.Iterator

func (*Iterator) Key added in v0.0.23

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

Key implements dbm.Iterator

func (*Iterator) Next added in v0.0.23

func (iter *Iterator) Next()

Next implements dbm.Iterator

func (*Iterator) Valid added in v0.0.23

func (iter *Iterator) Valid() bool

Valid implements dbm.Iterator.

func (*Iterator) Value added in v0.0.23

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

Value implements dbm.Iterator

type LeafLayout added in v0.0.23

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

func (LeafLayout) Hash added in v0.0.23

func (leaf LeafLayout) Hash() []byte

func (LeafLayout) KeyLength added in v0.0.23

func (leaf LeafLayout) KeyLength() uint32

func (LeafLayout) KeyOffset added in v0.0.23

func (leaf LeafLayout) KeyOffset() uint64

func (LeafLayout) Version added in v0.0.23

func (leaf LeafLayout) Version() uint32

type Leaves added in v0.0.23

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

Leaves is a continuously stored IAVL nodes

func NewLeaves added in v0.0.23

func NewLeaves(data []byte) (Leaves, error)

func (Leaves) Leaf added in v0.0.23

func (leaves Leaves) Leaf(i uint32) LeafLayout

type MemNode added in v0.0.23

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

func (*MemNode) Get added in v0.0.23

func (node *MemNode) Get(key []byte) ([]byte, uint32)

func (*MemNode) GetByIndex added in v0.0.23

func (node *MemNode) GetByIndex(index uint32) ([]byte, []byte)

func (*MemNode) Hash added in v0.0.23

func (node *MemNode) Hash() []byte

Computes the hash of the node without computing its descendants. Must be called on nodes which have descendant node hashes already computed.

func (*MemNode) Height added in v0.0.23

func (node *MemNode) Height() uint8

func (*MemNode) IsLeaf added in v0.0.23

func (node *MemNode) IsLeaf() bool

func (*MemNode) Key added in v0.0.23

func (node *MemNode) Key() []byte

func (*MemNode) Left added in v0.0.23

func (node *MemNode) Left() Node

func (*MemNode) Mutate added in v0.0.23

func (node *MemNode) Mutate(version, cowVersion uint32) *MemNode

Mutate clones the node if it's version is smaller than or equal to cowVersion, otherwise modify in-place

func (*MemNode) Right added in v0.0.23

func (node *MemNode) Right() Node

func (*MemNode) SafeHash added in v0.0.23

func (node *MemNode) SafeHash() []byte

func (*MemNode) Size added in v0.0.23

func (node *MemNode) Size() int64

func (*MemNode) Value added in v0.0.23

func (node *MemNode) Value() []byte

func (*MemNode) Version added in v0.0.23

func (node *MemNode) Version() uint32

type MmapFile added in v0.0.23

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

MmapFile manage the resources of a mmap-ed file

func NewMmap added in v0.0.23

func NewMmap(path string) (*MmapFile, error)

Open openes the file and create the mmap. the mmap is created with flags: PROT_READ, MAP_SHARED, MADV_RANDOM.

func (*MmapFile) Close added in v0.0.23

func (m *MmapFile) Close() error

Close closes the file and mmap handles

func (*MmapFile) Data added in v0.0.23

func (m *MmapFile) Data() []byte

Data returns the mmap-ed buffer

type MultiTree added in v0.0.23

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

MultiTree manages multiple memiavl tree together, all the trees share the same latest version, the snapshots are always created at the same version.

The snapshot structure is like this: ``` > snapshot-V > metadata > bank > kvs > nodes > metadata > acc > other stores... ```

func LoadMultiTree added in v0.0.23

func LoadMultiTree(dir string, zeroCopy bool, cacheSize int) (*MultiTree, error)

func NewEmptyMultiTree added in v0.0.23

func NewEmptyMultiTree(initialVersion uint32, cacheSize int) *MultiTree

func (*MultiTree) ApplyChangeSet added in v0.0.23

func (t *MultiTree) ApplyChangeSet(name string, changeSet iavl.ChangeSet) error

ApplyChangeSet applies change set for a single tree.

func (*MultiTree) ApplyChangeSets added in v0.0.23

func (t *MultiTree) ApplyChangeSets(changeSets []*proto.NamedChangeSet) error

ApplyChangeSets applies change sets for multiple trees.

func (*MultiTree) ApplyUpgrades added in v0.0.23

func (t *MultiTree) ApplyUpgrades(upgrades []*proto.TreeNameUpgrade) error

ApplyUpgrades store name upgrades

func (*MultiTree) Catchup added in v0.0.23

func (t *MultiTree) Catchup(stream types.Stream[proto.ChangelogEntry], endVersion int64) error

Catchup replay the new entries in the Rlog file on the tree to catch up to the target or latest version.

func (*MultiTree) Close added in v0.0.23

func (t *MultiTree) Close() error

func (*MultiTree) Copy added in v0.0.23

func (t *MultiTree) Copy(cacheSize int) *MultiTree

Copy returns a snapshot of the tree which won't be corrupted by further modifications on the main tree.

func (*MultiTree) LastCommitInfo added in v0.0.23

func (t *MultiTree) LastCommitInfo() *proto.CommitInfo

func (*MultiTree) ReplaceWith added in v0.0.29

func (t *MultiTree) ReplaceWith(other *MultiTree) error

func (*MultiTree) SaveVersion added in v0.0.23

func (t *MultiTree) SaveVersion(updateCommitInfo bool) (int64, error)

SaveVersion bumps the versions of all the stores and optionally returns the new app hash

func (*MultiTree) SetInitialVersion added in v0.0.23

func (t *MultiTree) SetInitialVersion(initialVersion int64) error

func (*MultiTree) SetZeroCopy added in v0.0.23

func (t *MultiTree) SetZeroCopy(zeroCopy bool)

func (*MultiTree) SnapshotVersion added in v0.0.23

func (t *MultiTree) SnapshotVersion() int64

func (*MultiTree) TreeByName added in v0.0.23

func (t *MultiTree) TreeByName(name string) *Tree

TreeByName returns the tree by name, returns nil if not found

func (*MultiTree) Trees added in v0.0.23

func (t *MultiTree) Trees() []NamedTree

Trees returns all the trees together with the name, ordered by name.

func (*MultiTree) UpdateCommitInfo added in v0.0.23

func (t *MultiTree) UpdateCommitInfo()

UpdateCommitInfo update lastCommitInfo based on current status of trees. it's needed if `updateCommitInfo` is set to `false` in `ApplyChangeSet`.

func (*MultiTree) Version added in v0.0.23

func (t *MultiTree) Version() int64

func (*MultiTree) WorkingCommitInfo added in v0.0.23

func (t *MultiTree) WorkingCommitInfo() *proto.CommitInfo

WorkingCommitInfo returns the commit info for the working tree

func (*MultiTree) WriteSnapshot added in v0.0.23

func (t *MultiTree) WriteSnapshot(ctx context.Context, dir string, wp *pond.WorkerPool) error

type MultiTreeExporter added in v0.0.23

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

func NewMultiTreeExporter added in v0.0.23

func NewMultiTreeExporter(dir string, version uint32, supportExportNonSnapshotVersion bool) (exporter *MultiTreeExporter, err error)

func (*MultiTreeExporter) Close added in v0.0.23

func (mte *MultiTreeExporter) Close() error

func (*MultiTreeExporter) Next added in v0.0.23

func (mte *MultiTreeExporter) Next() (interface{}, error)

type MultiTreeImporter added in v0.0.23

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

func NewMultiTreeImporter added in v0.0.23

func NewMultiTreeImporter(dir string, height uint64) (*MultiTreeImporter, error)

func (*MultiTreeImporter) Add added in v0.0.23

func (mti *MultiTreeImporter) Add(item interface{}) error

func (*MultiTreeImporter) AddNode added in v0.0.23

func (mti *MultiTreeImporter) AddNode(node *types.SnapshotNode)

func (*MultiTreeImporter) AddTree added in v0.0.23

func (mti *MultiTreeImporter) AddTree(name string) error

func (*MultiTreeImporter) Close added in v0.0.23

func (mti *MultiTreeImporter) Close() error

type NamedTree added in v0.0.23

type NamedTree struct {
	*Tree
	Name string
}

type Node added in v0.0.23

type Node interface {
	Height() uint8
	IsLeaf() bool
	Size() int64
	Version() uint32
	Key() []byte
	Value() []byte
	Left() Node
	Right() Node
	Hash() []byte

	// SafeHash returns byte slice that's safe to retain
	SafeHash() []byte

	// PersistedNode clone a new node, MemNode modify in place
	Mutate(version, cowVersion uint32) *MemNode

	// Get query the value for a key, it's put into interface because a specialized implementation is more efficient.
	Get(key []byte) ([]byte, uint32)
	GetByIndex(uint32) ([]byte, []byte)
}

Node interface encapsulate the interface of both PersistedNode and MemNode.

type NodeLayout added in v0.0.23

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

see comment of `PersistedNode`

func (NodeLayout) Hash added in v0.0.23

func (node NodeLayout) Hash() []byte

func (NodeLayout) Height added in v0.0.23

func (node NodeLayout) Height() uint8

func (NodeLayout) KeyLeaf added in v0.0.23

func (node NodeLayout) KeyLeaf() uint32

func (NodeLayout) PreTrees added in v0.0.23

func (node NodeLayout) PreTrees() uint8

func (NodeLayout) Size added in v0.0.23

func (node NodeLayout) Size() uint32

func (NodeLayout) Version added in v0.0.23

func (node NodeLayout) Version() uint32

type Nodes added in v0.0.23

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

Nodes is a continuously stored IAVL nodes

func NewNodes added in v0.0.23

func NewNodes(data []byte) (Nodes, error)

func (Nodes) Node added in v0.0.23

func (nodes Nodes) Node(i uint32) NodeLayout

type Options added in v0.0.23

type Options struct {
	Dir             string
	CreateIfMissing bool
	InitialVersion  uint32
	ReadOnly        bool
	// the initial stores when initialize the empty instance
	InitialStores []string
	// keep how many snapshots
	SnapshotKeepRecent uint32
	// how often to take a snapshot
	SnapshotInterval uint32
	// Buffer size for the asynchronous commit queue, -1 means synchronous commit,
	// default to 0.
	AsyncCommitBuffer int
	// ZeroCopy if true, the get and iterator methods could return a slice pointing to mmaped blob files.
	ZeroCopy bool
	// CacheSize defines the cache's max entry size for each memiavl store.
	CacheSize int
	// LoadForOverwriting if true rollbacks the state, specifically the OpenDB method will
	// truncate the versions after the `TargetVersion`, the `TargetVersion` becomes the latest version.
	// it do nothing if the target version is `0`.
	LoadForOverwriting bool

	// Limit the number of concurrent snapshot writers
	SnapshotWriterLimit int
}

func (*Options) FillDefaults added in v0.0.23

func (opts *Options) FillDefaults()

func (Options) Validate added in v0.0.23

func (opts Options) Validate() error

type PersistedNode added in v0.0.23

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

PersistedNode is backed by serialized byte array, usually mmap-ed from disk file. Encoding format (all integers are encoded in little endian):

Branch node: - height : 1 - preTrees : 1 - _padding : 2 - version : 4 - size : 4 - key node : 4 // node index of the smallest leaf in right branch - hash : 32 Leaf node: - version : 4 - key len : 4 - key offset : 8 - hash : 32

func (PersistedNode) Get added in v0.0.23

func (node PersistedNode) Get(key []byte) ([]byte, uint32)

func (PersistedNode) GetByIndex added in v0.0.23

func (node PersistedNode) GetByIndex(leafIndex uint32) ([]byte, []byte)

func (PersistedNode) Hash added in v0.0.23

func (node PersistedNode) Hash() []byte

func (PersistedNode) Height added in v0.0.23

func (node PersistedNode) Height() uint8

func (PersistedNode) IsLeaf added in v0.0.23

func (node PersistedNode) IsLeaf() bool

func (PersistedNode) Key added in v0.0.23

func (node PersistedNode) Key() []byte

func (PersistedNode) Left added in v0.0.23

func (node PersistedNode) Left() Node

Left result is not defined for leaf nodes.

func (PersistedNode) Mutate added in v0.0.23

func (node PersistedNode) Mutate(version, _ uint32) *MemNode

func (PersistedNode) Right added in v0.0.23

func (node PersistedNode) Right() Node

Right result is not defined for leaf nodes.

func (PersistedNode) SafeHash added in v0.0.23

func (node PersistedNode) SafeHash() []byte

func (PersistedNode) Size added in v0.0.23

func (node PersistedNode) Size() int64

func (PersistedNode) Value added in v0.0.23

func (node PersistedNode) Value() []byte

Value returns nil for non-leaf node.

func (PersistedNode) Version added in v0.0.23

func (node PersistedNode) Version() uint32

type Snapshot added in v0.0.23

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

Snapshot manage the lifecycle of mmap-ed files for the snapshot, it must out live the objects that derived from it.

func NewEmptySnapshot added in v0.0.23

func NewEmptySnapshot(version uint32) *Snapshot

func OpenSnapshot added in v0.0.23

func OpenSnapshot(snapshotDir string) (*Snapshot, error)

OpenSnapshot parse the version number and the root node index from metadata file, and mmap the other files.

func (*Snapshot) Close added in v0.0.23

func (snapshot *Snapshot) Close() error

Close closes the file and mmap handles, clears the buffers.

func (*Snapshot) Export added in v0.0.23

func (snapshot *Snapshot) Export() *Exporter

Export exports the nodes from snapshot file sequentially, more efficient than a post-order traversal.

func (*Snapshot) IsEmpty added in v0.0.23

func (snapshot *Snapshot) IsEmpty() bool

IsEmpty returns if the snapshot is an empty tree.

func (*Snapshot) Key added in v0.0.23

func (snapshot *Snapshot) Key(offset uint64) []byte

Key returns a zero-copy slice of key by offset

func (*Snapshot) KeyValue added in v0.0.23

func (snapshot *Snapshot) KeyValue(offset uint64) ([]byte, []byte)

KeyValue returns a zero-copy slice of key/value pair by offset

func (*Snapshot) Leaf added in v0.0.23

func (snapshot *Snapshot) Leaf(index uint32) PersistedNode

Leaf returns the leaf node by index

func (*Snapshot) LeafKey added in v0.0.23

func (snapshot *Snapshot) LeafKey(index uint32) []byte

func (*Snapshot) LeafKeyValue added in v0.0.23

func (snapshot *Snapshot) LeafKeyValue(index uint32) ([]byte, []byte)

func (*Snapshot) Node added in v0.0.23

func (snapshot *Snapshot) Node(index uint32) PersistedNode

Node returns the branch node by index

func (*Snapshot) RootHash added in v0.0.23

func (snapshot *Snapshot) RootHash() []byte

func (*Snapshot) RootNode added in v0.0.23

func (snapshot *Snapshot) RootNode() PersistedNode

RootNode returns the root node

func (*Snapshot) ScanNodes added in v0.0.23

func (snapshot *Snapshot) ScanNodes(callback func(node PersistedNode) error) error

ScanNodes iterate over the nodes in the snapshot order (depth-first post-order, leaf nodes before branch nodes)

func (*Snapshot) Version added in v0.0.23

func (snapshot *Snapshot) Version() uint32

Version returns the version of the snapshot

type Tree added in v0.0.23

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

verify change sets by replay them to rebuild iavl tree and verify the root hashes

func New added in v0.0.23

func New(_ int) *Tree

New creates an empty tree at genesis version

func NewEmptyTree added in v0.0.23

func NewEmptyTree(version uint64, initialVersion uint32) *Tree

NewEmptyTree creates an empty tree at an arbitrary version.

func NewFromSnapshot added in v0.0.23

func NewFromSnapshot(snapshot *Snapshot, zeroCopy bool, _ int) *Tree

NewFromSnapshot mmap the blob files and create the root node.

func NewWithInitialVersion added in v0.0.23

func NewWithInitialVersion(initialVersion uint32) *Tree

NewWithInitialVersion creates an empty tree with initial-version, it happens when a new store created at the middle of the chain.

func (*Tree) ApplyChangeSet added in v0.0.23

func (t *Tree) ApplyChangeSet(changeSet iavl.ChangeSet)

ApplyChangeSet apply the change set of a whole version, and update hashes.

func (*Tree) Close added in v0.0.23

func (t *Tree) Close() error

func (*Tree) Copy added in v0.0.23

func (t *Tree) Copy(_ int) *Tree

Copy returns a snapshot of the tree which won't be modified by further modifications on the main tree, the returned new tree can be accessed concurrently with the main tree.

func (*Tree) Export added in v0.0.23

func (t *Tree) Export() *Exporter

Export returns a snapshot of the tree which won't be corrupted by further modifications on the main tree.

func (*Tree) Get added in v0.0.23

func (t *Tree) Get(key []byte) []byte

func (*Tree) GetByIndex added in v0.0.23

func (t *Tree) GetByIndex(index int64) ([]byte, []byte)

func (*Tree) GetMembershipProof added in v0.0.23

func (t *Tree) 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 (*Tree) GetNonMembershipProof added in v0.0.23

func (t *Tree) 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 (*Tree) GetProof added in v0.0.23

func (t *Tree) GetProof(key []byte) *ics23.CommitmentProof

GetProof takes a key for creating existence or absence proof and returns the appropriate merkle.Proof. Since this must be called after querying for the value, this function should never error Thus, it will panic on error rather than returning it

func (*Tree) GetWithIndex added in v0.0.23

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

func (*Tree) Has added in v0.0.23

func (t *Tree) Has(key []byte) bool

func (*Tree) IsEmpty added in v0.0.23

func (t *Tree) IsEmpty() bool

func (*Tree) Iterator added in v0.0.23

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

func (*Tree) Remove added in v0.0.23

func (t *Tree) Remove(key []byte)

func (*Tree) ReplaceWith added in v0.0.29

func (t *Tree) ReplaceWith(other *Tree) error

ReplaceWith is used during reload to replace the current tree with the newly loaded snapshot

func (*Tree) RootHash added in v0.0.23

func (t *Tree) RootHash() []byte

RootHash updates the hashes and return the current root hash, it clones the persisted node's bytes, so the returned bytes is safe to retain.

func (*Tree) SaveVersion added in v0.0.23

func (t *Tree) SaveVersion(updateHash bool) ([]byte, int64, error)

SaveVersion increases the version number and optionally updates the hashes

func (*Tree) ScanPostOrder added in v0.0.23

func (t *Tree) ScanPostOrder(callback func(node Node) bool)

ScanPostOrder scans the tree in post-order, and call the callback function on each node. If the callback function returns false, the scan will be stopped.

func (*Tree) Set added in v0.0.23

func (t *Tree) Set(key, value []byte)

func (*Tree) SetInitialVersion added in v0.0.23

func (t *Tree) SetInitialVersion(initialVersion int64) error

func (*Tree) SetZeroCopy added in v0.0.23

func (t *Tree) SetZeroCopy(zeroCopy bool)

func (*Tree) VerifyMembership added in v0.0.23

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

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

func (*Tree) VerifyNonMembership added in v0.0.23

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

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

func (*Tree) Version added in v0.0.23

func (t *Tree) Version() int64

Version returns the current tree version

func (*Tree) WriteSnapshot added in v0.0.23

func (t *Tree) WriteSnapshot(ctx context.Context, snapshotDir string) error

WriteSnapshot save the IAVL tree to a new snapshot directory.

type TreeImporter added in v0.0.23

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

TreeImporter import a single memiavl tree from state-sync snapshot

func NewTreeImporter added in v0.0.23

func NewTreeImporter(dir string, version int64) *TreeImporter

func (*TreeImporter) Add added in v0.0.23

func (ai *TreeImporter) Add(node *types.SnapshotNode)

func (*TreeImporter) Close added in v0.0.23

func (ai *TreeImporter) Close() error

Jump to

Keyboard shortcuts

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