memiavl

package module
v0.0.0-...-a6f2824 Latest Latest
Warning

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

Go to latest
Published: May 13, 2023 License: Apache-2.0 Imports: 33 Imported by: 3

README

Alternative IAVL Implementation

Changelog

  • 11 Jan 2023: Initial version
  • 13 Jan 2023: Change changeset encoding from protobuf to plain one
  • 17 Jan 2023:
    • Add delete field to change set to support empty value
    • Add section about compression on snapshot format
  • 27 Jan 2023:
    • Update metadata file format
    • Encode key length with 4 bytes instead of 2.
  • 24 Feb 2023:
    • Reduce node size without hash from 32bytes to 16bytes, leverage properties of post-order traversal.
    • Merge key-values into single kvs file, build optional MPHF hash table to index it.

The Journey

It started for an use case of verifying the state change sets, we need to replay the change sets to rebuild IAVL tree and check the final IAVL root hash, compare the root hash with the on-chain hash to verify the integrity of the change sets.

The first implementation keeps the whole IAVL tree in memory, mutate nodes in-place, and don't update hashes for the intermediate versions, and one insight from the test run is it runs surprisingly fast. For the distribution store in our testnet, it can process from genesis to block 6698242 in 2 minutes, which is around 55818 blocks per second.

To support incremental replay, we further designed an IAVL snapshot format that's stored on disk, while supporting random access with mmap, which solves the memory usage issue, and reduce the time of replaying.

New Design

So the new idea is we can put the snapshot and change sets together, the change sets is the write-ahead-log for the IAVL tree.

It also integrates well with versiondb, because versiondb can also be derived from change sets to provide query service. IAVL tree is only used for consensus state machine and merkle proof generations.

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.

File Formats

NOTICE: the integers are always encoded with little endianness.

Change Set File
version: 8
size:    8         // size of whole payload
payload:
  delete: 1
  keyLen: varint-uint64
  key
  [                 // if delete is false
    valueLen: varint-uint64
    value
  ]

repeat with next version
  • Change set files can be splited with certain block ranges for incremental backup and restoration.

  • Historical files can be compressed with zlib, because it doesn't need to support random access.

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*
    
  • kvs.index, optional MPHF(Minimal-Perfect-Hash-Function) hash index build from kvs, support query key-values as a hash map.

Compression

The items in snapshot reference with each other by file offsets, we can apply some block compression techniques to compress keys and values files while maintain random accessbility by uncompressed file offset, for example zstd's experimental seekable format^1.

VersionDB

VersionDB is to support query and iterating historical versions of key-values pairs, currently implemented with rocksdb's experimental user-defined timestamp feature, support query and iterate key-value pairs by version, it's an alternative way to support grpc query service, and much more compact than IAVL trees, similar in size with the compressed change set files.

After versiondb is fully integrated, IAVL tree don't need to serve queries at all, it don't need to store the values at all, just store the value hashes would be enough.

Documentation

Index

Constants

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"
	FileNameKVIndex  = "kvs.index"
	FileNameMetadata = "metadata"
)
View Source
const MetadataFileName = "__metadata"
View Source
const (
	SnapshotPrefix = "snapshot-"
)

Variables

View Source
var (
	ErrInvalidLengthWal        = fmt.Errorf("proto: negative length found during unmarshaling")
	ErrIntOverflowWal          = fmt.Errorf("proto: integer overflow")
	ErrUnexpectedEndOfGroupWal = fmt.Errorf("proto: unexpected end of group")
)

Functions

func EncodeBytes

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 HashNode

func HashNode(node Node) []byte

HashNode computes the hash of the node.

func Import

func Import(
	dir string, height uint64, format uint32, protoReader protoio.Reader,
) (snapshottypes.SnapshotItem, error)

Import restore memiavl db from state-sync snapshot stream

func Mmap

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

func NewIterator

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

func VerifyHash

func VerifyHash(node Node) bool

VerifyHash compare node's cached hash with computed one

func WriteFileSync

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

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

Types

type DB

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 > wal ```

func Load

func Load(dir string, opts Options) (*DB, error)

func (*DB) ApplyChangeSet

func (db *DB) ApplyChangeSet(changeSets []*NamedChangeSet, updateCommitInfo bool) ([]byte, int64, error)

ApplyChangeSet wraps MultiTree.ApplyChangeSet to add a lock.

func (*DB) ApplyUpgrades

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

ApplyUpgrades wraps MultiTree.ApplyUpgrades, it also append the upgrades in a temporary field, and include in the WAL entry in next Commit call.

func (*DB) Close

func (db *DB) Close() error

func (*DB) Commit

func (db *DB) Commit(changeSets []*NamedChangeSet) ([]byte, int64, error)

Commit wraps `MultiTree.ApplyChangeSet` to add some db level operations: - manage background snapshot rewriting - write WAL

func (*DB) Copy

func (db *DB) Copy() *DB

func (*DB) Hash

func (db *DB) Hash() []byte

Hash wraps MultiTree.Hash to add a lock.

func (*DB) LastCommitInfo

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

LastCommitInfo returns the last commit info.

func (*DB) Reload

func (db *DB) Reload() error

func (*DB) RewriteSnapshot

func (db *DB) RewriteSnapshot() error

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

func (*DB) RewriteSnapshotBackground

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) SetInitialVersion

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

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

func (*DB) Snapshot

func (db *DB) Snapshot(height uint64, protoWriter protoio.Writer) error

func (*DB) TreeByName

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

TreeByName wraps MultiTree.TreeByName to add a lock.

func (*DB) UpdateCommitInfo

func (db *DB) UpdateCommitInfo() []byte

UpdateCommitInfo wraps MultiTree.UpdateCommitInfo to add a lock.

func (*DB) Version

func (db *DB) Version() int64

Version wraps MultiTree.Version to add a lock.

func (*DB) WriteSnapshot

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

WriteSnapshot wraps MultiTree.WriteSnapshot to add a lock.

type Exporter

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

func (*Exporter) Close

func (e *Exporter) Close()

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

func (*Exporter) Next

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

type Iterator

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

func (*Iterator) Close

func (iter *Iterator) Close() error

Close implements dbm.Iterator

func (*Iterator) Domain

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

func (*Iterator) Error

func (iter *Iterator) Error() error

Error implements dbm.Iterator

func (*Iterator) Key

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

Key implements dbm.Iterator

func (*Iterator) Next

func (iter *Iterator) Next()

Next implements dbm.Iterator

func (*Iterator) Valid

func (iter *Iterator) Valid() bool

Valid implements dbm.Iterator.

func (*Iterator) Value

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

Value implements dbm.Iterator

type LeafLayout

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

func (LeafLayout) Hash

func (leaf LeafLayout) Hash() []byte

func (LeafLayout) KeyLength

func (leaf LeafLayout) KeyLength() uint32

func (LeafLayout) KeyOffset

func (leaf LeafLayout) KeyOffset() uint64

func (LeafLayout) Version

func (leaf LeafLayout) Version() uint32

type Leaves

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

Leaves is a continuously stored IAVL nodes

func NewLeaves

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

func (Leaves) Leaf

func (leaves Leaves) Leaf(i uint32) LeafLayout

type MemNode

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

func (*MemNode) Get

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

func (*MemNode) GetByIndex

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

func (*MemNode) Hash

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

func (node *MemNode) Height() uint8

func (*MemNode) IsLeaf

func (node *MemNode) IsLeaf() bool

func (*MemNode) Key

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

func (*MemNode) Left

func (node *MemNode) Left() Node

func (*MemNode) Mutate

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

func (node *MemNode) Right() Node

func (*MemNode) Size

func (node *MemNode) Size() int64

func (*MemNode) Value

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

func (*MemNode) Version

func (node *MemNode) Version() uint32

type MmapFile

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

MmapFile manage the resources of a mmap-ed file

func NewMmap

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

func (m *MmapFile) Close() error

Close closes the file and mmap handles

func (*MmapFile) Data

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

Data returns the mmap-ed buffer

type MultiTree

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

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

func NewEmptyMultiTree

func NewEmptyMultiTree(initialVersion uint32) *MultiTree

func (*MultiTree) ApplyChangeSet

func (t *MultiTree) ApplyChangeSet(changeSets []*NamedChangeSet, updateCommitInfo bool) ([]byte, int64, error)

ApplyChangeSet applies change sets for all trees. if `updateCommitInfo` is `false`, the `lastCommitInfo.StoreInfos` is dirty.

func (*MultiTree) ApplyUpgrades

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

ApplyUpgrades store name upgrades

func (*MultiTree) CatchupWAL

func (t *MultiTree) CatchupWAL(wal *wal.Log, endVersion int64) error

CatchupWAL replay the new entries in the WAL on the tree to catch-up to the target or latest version.

func (*MultiTree) Close

func (t *MultiTree) Close() error

func (*MultiTree) Copy

func (t *MultiTree) Copy() *MultiTree

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

func (*MultiTree) Hash

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

func (*MultiTree) LastCommitInfo

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

func (*MultiTree) SetInitialVersion

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

func (*MultiTree) SetZeroCopy

func (t *MultiTree) SetZeroCopy(zeroCopy bool)

func (*MultiTree) TreeByName

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

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

func (*MultiTree) UpdateCommitInfo

func (t *MultiTree) UpdateCommitInfo() []byte

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

func (*MultiTree) Version

func (t *MultiTree) Version() int64

func (*MultiTree) WriteSnapshot

func (t *MultiTree) WriteSnapshot(dir string) error

type MultiTreeMetadata

type MultiTreeMetadata struct {
	CommitInfo     *types.CommitInfo `protobuf:"bytes,1,opt,name=commit_info,json=commitInfo,proto3" json:"commit_info,omitempty"`
	InitialVersion int64             `protobuf:"varint,2,opt,name=initial_version,json=initialVersion,proto3" json:"initial_version,omitempty"`
}

MultiTreeMetadata stores the metadata for MultiTree

func (*MultiTreeMetadata) Descriptor

func (*MultiTreeMetadata) Descriptor() ([]byte, []int)

func (*MultiTreeMetadata) GetCommitInfo

func (m *MultiTreeMetadata) GetCommitInfo() *types.CommitInfo

func (*MultiTreeMetadata) GetInitialVersion

func (m *MultiTreeMetadata) GetInitialVersion() int64

func (*MultiTreeMetadata) Marshal

func (m *MultiTreeMetadata) Marshal() (dAtA []byte, err error)

func (*MultiTreeMetadata) MarshalTo

func (m *MultiTreeMetadata) MarshalTo(dAtA []byte) (int, error)

func (*MultiTreeMetadata) MarshalToSizedBuffer

func (m *MultiTreeMetadata) MarshalToSizedBuffer(dAtA []byte) (int, error)

func (*MultiTreeMetadata) ProtoMessage

func (*MultiTreeMetadata) ProtoMessage()

func (*MultiTreeMetadata) Reset

func (m *MultiTreeMetadata) Reset()

func (*MultiTreeMetadata) Size

func (m *MultiTreeMetadata) Size() (n int)

func (*MultiTreeMetadata) String

func (m *MultiTreeMetadata) String() string

func (*MultiTreeMetadata) Unmarshal

func (m *MultiTreeMetadata) Unmarshal(dAtA []byte) error

func (*MultiTreeMetadata) XXX_DiscardUnknown

func (m *MultiTreeMetadata) XXX_DiscardUnknown()

func (*MultiTreeMetadata) XXX_Marshal

func (m *MultiTreeMetadata) XXX_Marshal(b []byte, deterministic bool) ([]byte, error)

func (*MultiTreeMetadata) XXX_Merge

func (m *MultiTreeMetadata) XXX_Merge(src proto.Message)

func (*MultiTreeMetadata) XXX_Size

func (m *MultiTreeMetadata) XXX_Size() int

func (*MultiTreeMetadata) XXX_Unmarshal

func (m *MultiTreeMetadata) XXX_Unmarshal(b []byte) error

type NamedChangeSet

type NamedChangeSet struct {
	Changeset proto1.ChangeSet `protobuf:"bytes,1,opt,name=changeset,proto3" json:"changeset"`
	Name      string           `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"`
}

NamedChangeSet combine a tree name with the changeset

func (*NamedChangeSet) Descriptor

func (*NamedChangeSet) Descriptor() ([]byte, []int)

func (*NamedChangeSet) GetChangeset

func (m *NamedChangeSet) GetChangeset() proto1.ChangeSet

func (*NamedChangeSet) GetName

func (m *NamedChangeSet) GetName() string

func (*NamedChangeSet) Marshal

func (m *NamedChangeSet) Marshal() (dAtA []byte, err error)

func (*NamedChangeSet) MarshalTo

func (m *NamedChangeSet) MarshalTo(dAtA []byte) (int, error)

func (*NamedChangeSet) MarshalToSizedBuffer

func (m *NamedChangeSet) MarshalToSizedBuffer(dAtA []byte) (int, error)

func (*NamedChangeSet) ProtoMessage

func (*NamedChangeSet) ProtoMessage()

func (*NamedChangeSet) Reset

func (m *NamedChangeSet) Reset()

func (*NamedChangeSet) Size

func (m *NamedChangeSet) Size() (n int)

func (*NamedChangeSet) String

func (m *NamedChangeSet) String() string

func (*NamedChangeSet) Unmarshal

func (m *NamedChangeSet) Unmarshal(dAtA []byte) error

func (*NamedChangeSet) XXX_DiscardUnknown

func (m *NamedChangeSet) XXX_DiscardUnknown()

func (*NamedChangeSet) XXX_Marshal

func (m *NamedChangeSet) XXX_Marshal(b []byte, deterministic bool) ([]byte, error)

func (*NamedChangeSet) XXX_Merge

func (m *NamedChangeSet) XXX_Merge(src proto.Message)

func (*NamedChangeSet) XXX_Size

func (m *NamedChangeSet) XXX_Size() int

func (*NamedChangeSet) XXX_Unmarshal

func (m *NamedChangeSet) XXX_Unmarshal(b []byte) error

type Node

type Node interface {
	Height() uint8
	IsLeaf() bool
	Size() int64
	Version() uint32
	Key() []byte
	Value() []byte
	Left() Node
	Right() Node
	Hash() []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

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

see comment of `PersistedNode`

func (NodeLayout) Hash

func (node NodeLayout) Hash() []byte

func (NodeLayout) Height

func (node NodeLayout) Height() uint8

func (NodeLayout) KeyLeaf

func (node NodeLayout) KeyLeaf() uint32

func (NodeLayout) PreTrees

func (node NodeLayout) PreTrees() uint8

func (NodeLayout) Size

func (node NodeLayout) Size() uint32

func (NodeLayout) Version

func (node NodeLayout) Version() uint32

type Nodes

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

Nodes is a continuously stored IAVL nodes

func NewNodes

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

func (Nodes) Node

func (nodes Nodes) Node(i uint32) NodeLayout

type Options

type Options struct {
	CreateIfMissing bool
	InitialVersion  uint32
	// the initial stores when initialize the empty instance
	InitialStores      []string
	SnapshotKeepRecent uint32
	// load the target version instead of latest version
	TargetVersion uint32
	// Write WAL asynchronously, it's ok in blockchain case because we can always replay the raw blocks.
	AsyncWAL bool
	// ZeroCopy if true, the get and iterator methods could return a slice pointing to mmaped blob files.
	ZeroCopy bool
}

type PersistedNode

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

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

func (PersistedNode) GetByIndex

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

func (PersistedNode) Hash

func (node PersistedNode) Hash() []byte

func (PersistedNode) Height

func (node PersistedNode) Height() uint8

func (PersistedNode) IsLeaf

func (node PersistedNode) IsLeaf() bool

func (PersistedNode) Key

func (node PersistedNode) Key() []byte

func (PersistedNode) Left

func (node PersistedNode) Left() Node

Left result is not defined for leaf nodes.

func (PersistedNode) Mutate

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

func (PersistedNode) Right

func (node PersistedNode) Right() Node

Right result is not defined for leaf nodes.

func (PersistedNode) Size

func (node PersistedNode) Size() int64

func (PersistedNode) Value

func (node PersistedNode) Value() []byte

Value result is not defined for non-leaf node.

func (PersistedNode) Version

func (node PersistedNode) Version() uint32

type Snapshot

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

func NewEmptySnapshot(version uint32) *Snapshot

func OpenSnapshot

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

func (snapshot *Snapshot) Close() error

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

func (*Snapshot) Export

func (snapshot *Snapshot) Export() *Exporter

Export exports the nodes in DFS post-order, resemble the API of existing iavl library

func (*Snapshot) Get

func (snapshot *Snapshot) Get(key []byte) []byte

Get lookup the value for the key through the hash index

func (*Snapshot) IsEmpty

func (snapshot *Snapshot) IsEmpty() bool

IsEmpty returns if the snapshot is an empty tree.

func (*Snapshot) Key

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

Key returns a zero-copy slice of key by offset

func (*Snapshot) KeyValue

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

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

func (*Snapshot) Leaf

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

Leaf returns the leaf node by index

func (*Snapshot) LeafKey

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

func (*Snapshot) LeafKeyValue

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

func (*Snapshot) Node

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

Node returns the branch node by index

func (*Snapshot) RootHash

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

func (*Snapshot) RootNode

func (snapshot *Snapshot) RootNode() PersistedNode

RootNode returns the root node

func (*Snapshot) ScanNodes

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

func (snapshot *Snapshot) Version() uint32

Version returns the version of the snapshot

type Tree

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

func New() *Tree

New creates an empty tree at genesis version

func NewEmptyTree

func NewEmptyTree(version uint64) *Tree

NewEmptyTree creates an empty tree at an arbitrary version.

func NewFromSnapshot

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

NewFromSnapshot mmap the blob files and create the root node.

func NewWithInitialVersion

func NewWithInitialVersion(initialVersion uint32) *Tree

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

func (*Tree) ApplyChangeSet

func (t *Tree) ApplyChangeSet(changeSet iavl.ChangeSet, updateHash bool) ([]byte, int64, error)

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

func (*Tree) Close

func (t *Tree) Close() error

func (*Tree) Copy

func (t *Tree) Copy() *Tree

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

func (*Tree) Get

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

func (*Tree) GetByIndex

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

func (*Tree) GetMembershipProof

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

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) GetWithIndex

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

func (*Tree) Has

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

func (*Tree) IsEmpty

func (t *Tree) IsEmpty() bool

func (*Tree) Iterator

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

func (*Tree) RootHash

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

RootHash updates the hashes and return the current root hash

func (*Tree) SetInitialVersion

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

func (*Tree) SetZeroCopy

func (t *Tree) SetZeroCopy(zeroCopy bool)

func (*Tree) VerifyMembership

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

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

func (t *Tree) Version() int64

Version returns the current tree version

func (*Tree) WriteSnapshot

func (t *Tree) WriteSnapshot(snapshotDir string, writeHashIndex bool) error

WriteSnapshot save the IAVL tree to a new snapshot directory.

type TreeImporter

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

TreeImporter import a single memiavl tree from state-sync snapshot

func NewTreeImporter

func NewTreeImporter(dir string, version int64) *TreeImporter

func (*TreeImporter) Add

func (ai *TreeImporter) Add(node *iavl.ExportNode)

func (*TreeImporter) Close

func (ai *TreeImporter) Close() error

type TreeNameUpgrade

type TreeNameUpgrade struct {
	Name       string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
	RenameFrom string `protobuf:"bytes,2,opt,name=rename_from,json=renameFrom,proto3" json:"rename_from,omitempty"`
	Delete     bool   `protobuf:"varint,3,opt,name=delete,proto3" json:"delete,omitempty"`
}

TreeNameUpgrade defines upgrade of tree names: - New tree: { name: "tree" } - Delete tree: { name: "tree", delete: true } - Rename tree: { name: "new-tree", rename_from: "old-tree" }

func (*TreeNameUpgrade) Descriptor

func (*TreeNameUpgrade) Descriptor() ([]byte, []int)

func (*TreeNameUpgrade) GetDelete

func (m *TreeNameUpgrade) GetDelete() bool

func (*TreeNameUpgrade) GetName

func (m *TreeNameUpgrade) GetName() string

func (*TreeNameUpgrade) GetRenameFrom

func (m *TreeNameUpgrade) GetRenameFrom() string

func (*TreeNameUpgrade) Marshal

func (m *TreeNameUpgrade) Marshal() (dAtA []byte, err error)

func (*TreeNameUpgrade) MarshalTo

func (m *TreeNameUpgrade) MarshalTo(dAtA []byte) (int, error)

func (*TreeNameUpgrade) MarshalToSizedBuffer

func (m *TreeNameUpgrade) MarshalToSizedBuffer(dAtA []byte) (int, error)

func (*TreeNameUpgrade) ProtoMessage

func (*TreeNameUpgrade) ProtoMessage()

func (*TreeNameUpgrade) Reset

func (m *TreeNameUpgrade) Reset()

func (*TreeNameUpgrade) Size

func (m *TreeNameUpgrade) Size() (n int)

func (*TreeNameUpgrade) String

func (m *TreeNameUpgrade) String() string

func (*TreeNameUpgrade) Unmarshal

func (m *TreeNameUpgrade) Unmarshal(dAtA []byte) error

func (*TreeNameUpgrade) XXX_DiscardUnknown

func (m *TreeNameUpgrade) XXX_DiscardUnknown()

func (*TreeNameUpgrade) XXX_Marshal

func (m *TreeNameUpgrade) XXX_Marshal(b []byte, deterministic bool) ([]byte, error)

func (*TreeNameUpgrade) XXX_Merge

func (m *TreeNameUpgrade) XXX_Merge(src proto.Message)

func (*TreeNameUpgrade) XXX_Size

func (m *TreeNameUpgrade) XXX_Size() int

func (*TreeNameUpgrade) XXX_Unmarshal

func (m *TreeNameUpgrade) XXX_Unmarshal(b []byte) error

type WALEntry

type WALEntry struct {
	Changesets []*NamedChangeSet  `protobuf:"bytes,1,rep,name=changesets,proto3" json:"changesets,omitempty"`
	Upgrades   []*TreeNameUpgrade `protobuf:"bytes,2,rep,name=upgrades,proto3" json:"upgrades,omitempty"`
}

WALEntry is a single Write-Ahead-Log entry

func (*WALEntry) Descriptor

func (*WALEntry) Descriptor() ([]byte, []int)

func (*WALEntry) GetChangesets

func (m *WALEntry) GetChangesets() []*NamedChangeSet

func (*WALEntry) GetUpgrades

func (m *WALEntry) GetUpgrades() []*TreeNameUpgrade

func (*WALEntry) Marshal

func (m *WALEntry) Marshal() (dAtA []byte, err error)

func (*WALEntry) MarshalTo

func (m *WALEntry) MarshalTo(dAtA []byte) (int, error)

func (*WALEntry) MarshalToSizedBuffer

func (m *WALEntry) MarshalToSizedBuffer(dAtA []byte) (int, error)

func (*WALEntry) ProtoMessage

func (*WALEntry) ProtoMessage()

func (*WALEntry) Reset

func (m *WALEntry) Reset()

func (*WALEntry) Size

func (m *WALEntry) Size() (n int)

func (*WALEntry) String

func (m *WALEntry) String() string

func (*WALEntry) Unmarshal

func (m *WALEntry) Unmarshal(dAtA []byte) error

func (*WALEntry) XXX_DiscardUnknown

func (m *WALEntry) XXX_DiscardUnknown()

func (*WALEntry) XXX_Marshal

func (m *WALEntry) XXX_Marshal(b []byte, deterministic bool) ([]byte, error)

func (*WALEntry) XXX_Merge

func (m *WALEntry) XXX_Merge(src proto.Message)

func (*WALEntry) XXX_Size

func (m *WALEntry) XXX_Size() int

func (*WALEntry) XXX_Unmarshal

func (m *WALEntry) XXX_Unmarshal(b []byte) error

Jump to

Keyboard shortcuts

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