block_stm

package module
v0.0.0-...-9f11af1 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: 14 Imported by: 0

README

go-block-stm implements the block-stm algorithm, it follows the paper pseudocode pretty closely.

The main API is a simple function call ExecuteBlock:

type ExecuteFn func(TxnIndex, MultiStore)
func ExecuteBlock(
	ctx context.Context,           // context for cancellation
	blockSize int,                 // the number of the transactions to be executed
	stores []storetypes.StoreKey,  // the list of store keys to support
	storage MultiStore,            // the parent storage, after all transactions are executed, the whole change sets are written into parent storage at once
	executors int,                 // how many concurrent executors to spawn
	executeFn ExecuteFn,           // callback function to actually execute a transaction with a wrapped `MultiStore`.
) error

The main deviations from the paper are:

Optimisation

We applied the optimization described in section 4 of the paper:

Block-STM calls add_dependency from the VM itself, and can thus re-read and continue execution when false is returned.

When the VM execution reads an ESTIMATE mark, it'll hang on a CondVar, so it can resume execution after the dependency is resolved, much more efficient than abortion and rerun.

Support Deletion, Iteration, and MultiStore

These features are necessary for integration with cosmos-sdk.

The multi-version data structure is implemented with nested btree for easier iteration support, the WriteSet is also implemented with a btree, and it takes advantage of ordered property to optimize some logic.

The internal data structures are also adapted with multiple stores in mind.

Documentation

Index

Constants

View Source
const (
	// Since we do copy-on-write a lot, smaller degree means smaller allocations
	OuterBTreeDegree = 4
	InnerBTreeDegree = 4
)

Variables

View Source
var (
	StoreKeyAuth = storetypes.NewKVStoreKey("acc")
	StoreKeyBank = storetypes.NewKVStoreKey("bank")
)
View Source
var InvalidTxnVersion = TxnVersion{-1, 0}

Functions

func BytesBeyond

func BytesBeyond(a, b []byte, ascending bool) bool

BytesBeyond returns if a is beyond b in specified iteration order

func BytesIsZero

func BytesIsZero(v []byte) bool

func BytesLen

func BytesLen(v []byte) int

func DecrAtomic

func DecrAtomic(a *atomic.Uint64)

DecrAtomic decreases the atomic value by 1

func DiffOrderedList

func DiffOrderedList(old, new []Key, callback func(Key, bool) bool)

callback arguments: (value, is_new)

func ExecuteBlock

func ExecuteBlock(
	ctx context.Context,
	blockSize int,
	stores map[storetypes.StoreKey]int,
	storage MultiStore,
	executors int,
	txExecutor TxExecutor,
) error

func FetchIncr

func FetchIncr(a *atomic.Uint64) uint64

FetchIncr increaes the atomic value by 1 and returns the old value

func IncrAtomic

func IncrAtomic(a *atomic.Uint64)

IncrAtomic increases the atomic value by 1

func KeyItemLess

func KeyItemLess[T KeyItem](a, b T) bool

func NewCacheMergeIterator

func NewCacheMergeIterator[V any](
	parent, cache types.GIterator[V],
	ascending bool, onClose func(types.GIterator[V]),
	isZero func(V) bool,
) types.GIterator[V]

func NewNoopIterator

func NewNoopIterator[V any](start, end Key, ascending bool) storetypes.GIterator[V]

func ObjIsZero

func ObjIsZero(v any) bool

func ObjLen

func ObjLen(v any) int

func StoreMin

func StoreMin(a *atomic.Uint64, b uint64)

StoreMin implements a compare-and-swap operation that stores the minimum of the current value and the given value.

Types

type BTree

type BTree[T any] struct {
	atomic.Pointer[btree.BTreeG[T]]
}

BTree wraps an atomic pointer to an unsafe btree.BTreeG

func NewBTree

func NewBTree[T any](less func(a, b T) bool, degree int) *BTree[T]

NewBTree returns a new BTree.

func (*BTree[T]) Delete

func (bt *BTree[T]) Delete(item T) (prev T, ok bool)

func (*BTree[T]) Get

func (bt *BTree[T]) Get(item T) (result T, ok bool)

func (*BTree[T]) GetOrDefault

func (bt *BTree[T]) GetOrDefault(item T, fillDefaults func(*T)) T

func (*BTree[T]) Iter

func (bt *BTree[T]) Iter() btree.IterG[T]

func (*BTree[T]) Max

func (bt *BTree[T]) Max() (T, bool)

func (*BTree[T]) ReverseSeek

func (bt *BTree[T]) ReverseSeek(pivot T) (result T, ok bool)

ReverseSeek returns the first item that is less than or equal to the pivot

func (*BTree[T]) Scan

func (bt *BTree[T]) Scan(iter func(item T) bool)

func (*BTree[T]) Set

func (bt *BTree[T]) Set(item T) (prev T, ok bool)

type BTreeIteratorG

type BTreeIteratorG[T KeyItem] struct {
	// contains filtered or unexported fields
}

BTreeIteratorG iterates over btree. Implements Iterator.

func NewBTreeIteratorG

func NewBTreeIteratorG[T KeyItem](
	startItem, endItem T,
	iter btree.IterG[T],
	ascending bool,
) *BTreeIteratorG[T]

func (*BTreeIteratorG[T]) Close

func (mi *BTreeIteratorG[T]) Close() error

func (*BTreeIteratorG[T]) Domain

func (mi *BTreeIteratorG[T]) Domain() (start []byte, end []byte)

func (*BTreeIteratorG[T]) Error

func (mi *BTreeIteratorG[T]) Error() error

func (*BTreeIteratorG[T]) Item

func (mi *BTreeIteratorG[T]) Item() T

func (*BTreeIteratorG[T]) Key

func (mi *BTreeIteratorG[T]) Key() []byte

func (*BTreeIteratorG[T]) Next

func (mi *BTreeIteratorG[T]) Next()

func (*BTreeIteratorG[T]) Valid

func (mi *BTreeIteratorG[T]) Valid() bool

type Condvar

type Condvar struct {
	sync.Mutex
	// contains filtered or unexported fields
}

func NewCondvar

func NewCondvar() *Condvar

func (*Condvar) Notify

func (cv *Condvar) Notify()

func (*Condvar) Wait

func (cv *Condvar) Wait()

type ErrReadError

type ErrReadError struct {
	BlockingTxn TxnIndex
}

func (ErrReadError) Error

func (e ErrReadError) Error() string

type Executor

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

Executor fields are not mutated during execution.

func NewExecutor

func NewExecutor(
	ctx context.Context,
	blockSize int,
	scheduler *Scheduler,
	storage MultiStore,
	txExecutor TxExecutor,
	mvMemory *MVMemory,
	i int,
) *Executor

func (*Executor) NeedsReexecution

func (e *Executor) NeedsReexecution(version TxnVersion) (TxnVersion, TaskKind)

func (*Executor) Run

func (e *Executor) Run()

Invariant `num_active_tasks`:

  • `NextTask` increases it if returns a valid task.
  • `TryExecute` and `NeedsReexecution` don't change it if it returns a new valid task to run, otherwise it decreases it.

func (*Executor) TryExecute

func (e *Executor) TryExecute(version TxnVersion) (TxnVersion, TaskKind)

type GKVPair

type GKVPair[V any] struct {
	Key   Key
	Value V
}

type GMVData

type GMVData[V any] struct {
	BTree[dataItem[V]]
	// contains filtered or unexported fields
}

func NewGMVData

func NewGMVData[V any](isZero func(V) bool, valueLen func(V) int) *GMVData[V]

func (*GMVData[V]) Delete

func (d *GMVData[V]) Delete(key Key, txn TxnIndex)

func (*GMVData[V]) Iterator

func (d *GMVData[V]) Iterator(
	opts IteratorOptions, txn TxnIndex,
	waitFn func(TxnIndex),
) *MVIterator[V]

func (*GMVData[V]) Read

func (d *GMVData[V]) Read(key Key, txn TxnIndex) (V, TxnVersion, bool)

Read returns the value and the version of the value that's less than the given txn. If the key is not found, returns `(nil, InvalidTxnVersion, false)`. If the key is found but value is an estimate, returns `(nil, BlockingTxn, true)`. If the key is found, returns `(value, version, false)`, `value` can be `nil` which means deleted.

func (*GMVData[V]) Snapshot

func (d *GMVData[V]) Snapshot() (snapshot []GKVPair[V])

func (*GMVData[V]) SnapshotTo

func (d *GMVData[V]) SnapshotTo(cb func(Key, V) bool)

func (*GMVData[V]) SnapshotToStore

func (d *GMVData[V]) SnapshotToStore(store storetypes.Store)

func (*GMVData[V]) ValidateReadSet

func (d *GMVData[V]) ValidateReadSet(txn TxnIndex, rs *ReadSet) bool

ValidateReadSet validates the read descriptors, returns true if valid.

func (*GMVData[V]) Write

func (d *GMVData[V]) Write(key Key, value V, version TxnVersion)

func (*GMVData[V]) WriteEstimate

func (d *GMVData[V]) WriteEstimate(key Key, txn TxnIndex)

type GMVMemoryView

type GMVMemoryView[V any] struct {
	// contains filtered or unexported fields
}

GMVMemoryView[V] wraps `MVMemory` for execution of a single transaction.

func NewGMVMemoryView

func NewGMVMemoryView[V any](store int, storage storetypes.GKVStore[V], mvData *GMVData[V], scheduler *Scheduler, txn TxnIndex) *GMVMemoryView[V]

func (*GMVMemoryView[V]) ApplyWriteSet

func (s *GMVMemoryView[V]) ApplyWriteSet(version TxnVersion) Locations

func (*GMVMemoryView[V]) CacheWrap

func (s *GMVMemoryView[V]) CacheWrap() storetypes.CacheWrap

CacheWrap implements types.Store.

func (*GMVMemoryView[V]) Delete

func (s *GMVMemoryView[V]) Delete(key []byte)

func (*GMVMemoryView[V]) Get

func (s *GMVMemoryView[V]) Get(key []byte) V

func (*GMVMemoryView[V]) GetStoreType

func (s *GMVMemoryView[V]) GetStoreType() storetypes.StoreType

GetStoreType implements types.Store.

func (*GMVMemoryView[V]) Has

func (s *GMVMemoryView[V]) Has(key []byte) bool

func (*GMVMemoryView[V]) Iterator

func (s *GMVMemoryView[V]) Iterator(start, end []byte) storetypes.GIterator[V]

func (*GMVMemoryView[V]) ReadSet

func (s *GMVMemoryView[V]) ReadSet() *ReadSet

func (*GMVMemoryView[V]) ReverseIterator

func (s *GMVMemoryView[V]) ReverseIterator(start, end []byte) storetypes.GIterator[V]

func (*GMVMemoryView[V]) Set

func (s *GMVMemoryView[V]) Set(key []byte, value V)

type GMemDB

type GMemDB[V any] struct {
	btree.BTreeG[memdbItem[V]]
	// contains filtered or unexported fields
}

func NewGMemDB

func NewGMemDB[V any](
	isZero func(V) bool,
	valueLen func(V) int,
) *GMemDB[V]

func NewGMemDBNonConcurrent

func NewGMemDBNonConcurrent[V any](
	isZero func(V) bool,
	valueLen func(V) int,
) *GMemDB[V]

NewGMemDBNonConcurrent returns a new BTree which is not concurrency safe.

func (*GMemDB[V]) CacheWrap

func (db *GMemDB[V]) CacheWrap() storetypes.CacheWrap

CacheWrap implements types.KVStore.

func (*GMemDB[V]) Delete

func (db *GMemDB[V]) Delete(key []byte)

func (*GMemDB[V]) Get

func (db *GMemDB[V]) Get(key []byte) V

func (*GMemDB[V]) GetStoreType

func (db *GMemDB[V]) GetStoreType() storetypes.StoreType

func (*GMemDB[V]) Has

func (db *GMemDB[V]) Has(key []byte) bool

func (*GMemDB[V]) Iterator

func (db *GMemDB[V]) Iterator(start, end []byte) storetypes.GIterator[V]

func (*GMemDB[V]) OverlayGet

func (db *GMemDB[V]) OverlayGet(key Key) (V, bool)

When used as an overlay (e.g. WriteSet), it stores the `nil` value to represent deleted keys, so we return seperate bool value for found status.

func (*GMemDB[V]) OverlaySet

func (db *GMemDB[V]) OverlaySet(key Key, value V)

When used as an overlay (e.g. WriteSet), it stores the `nil` value to represent deleted keys,

func (*GMemDB[V]) ReverseIterator

func (db *GMemDB[V]) ReverseIterator(start, end []byte) storetypes.GIterator[V]

func (*GMemDB[V]) Scan

func (db *GMemDB[V]) Scan(cb func(key Key, value V) bool)

func (*GMemDB[V]) Set

func (db *GMemDB[V]) Set(key []byte, value V)

type Incarnation

type Incarnation uint

type IteratorDescriptor

type IteratorDescriptor struct {
	IteratorOptions
	// Stop is not `nil` if the iteration is not exhausted and stops at a key before reaching the end of the range,
	// the effective range is `[start, stop]`.
	// when replaying, it should also stops at the stop key.
	Stop Key
	// Reads is the list of keys that is observed by the iterator.
	Reads []ReadDescriptor
}

type IteratorOptions

type IteratorOptions struct {
	// [Start, End) is the range of the iterator
	Start     Key
	End       Key
	Ascending bool
}

type KVPair

type KVPair = GKVPair[[]byte]

type Key

type Key []byte

type KeyItem

type KeyItem interface {
	GetKey() []byte
}

type Locations

type Locations []Key

keys are sorted

type MVData

type MVData = GMVData[[]byte]

func NewMVData

func NewMVData() *MVData

type MVIterator

type MVIterator[V any] struct {
	BTreeIteratorG[dataItem[V]]
	// contains filtered or unexported fields
}

MVIterator is an iterator for a multi-versioned store.

func NewMVIterator

func NewMVIterator[V any](
	opts IteratorOptions, txn TxnIndex, iter btree.IterG[dataItem[V]],
	waitFn func(TxnIndex),
) *MVIterator[V]

func (*MVIterator[V]) Executing

func (it *MVIterator[V]) Executing() bool

Executing returns if the iterator is running in execution mode.

func (*MVIterator[V]) Next

func (it *MVIterator[V]) Next()

func (*MVIterator[V]) ReadEstimateValue

func (it *MVIterator[V]) ReadEstimateValue() bool

func (*MVIterator[V]) Reads

func (it *MVIterator[V]) Reads() []ReadDescriptor

func (*MVIterator[V]) Value

func (it *MVIterator[V]) Value() V

func (*MVIterator[V]) Version

func (it *MVIterator[V]) Version() TxnVersion

type MVMemory

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

MVMemory implements `Algorithm 2 The MVMemory module`

func NewMVMemory

func NewMVMemory(block_size int, stores map[storetypes.StoreKey]int) *MVMemory

func (*MVMemory) ConvertWritesToEstimates

func (mv *MVMemory) ConvertWritesToEstimates(txn TxnIndex)

func (*MVMemory) GetMVStore

func (mv *MVMemory) GetMVStore(i int) MVStore

func (*MVMemory) Record

func (mv *MVMemory) Record(version TxnVersion, view *MultiMVMemoryView) bool

func (*MVMemory) ValidateReadSet

func (mv *MVMemory) ValidateReadSet(txn TxnIndex) bool

func (*MVMemory) View

func (mv *MVMemory) View(txn TxnIndex, storage MultiStore, scheduler *Scheduler) *MultiMVMemoryView

View creates a view for a particular transaction.

func (*MVMemory) WriteSnapshot

func (mv *MVMemory) WriteSnapshot(storage MultiStore)

type MVStore

type MVStore interface {
	Delete(Key, TxnIndex)
	WriteEstimate(Key, TxnIndex)
	ValidateReadSet(TxnIndex, *ReadSet) bool
	SnapshotToStore(storetypes.Store)
}

MVStore is a value type agnostic interface for `MVData`, to keep `MVMemory` value type agnostic.

func NewMVStore

func NewMVStore(key storetypes.StoreKey) MVStore

type MVView

type MVView interface {
	storetypes.Store

	ApplyWriteSet(TxnVersion) Locations
	ReadSet() *ReadSet
}

MVView is a value type agnostic interface for `MVMemoryView`, to keep `MultiMVMemoryView` value type agnostic.

func NewMVView

func NewMVView(store int, storage storetypes.Store, mvData MVStore, scheduler *Scheduler, txn TxnIndex) MVView

type MemDB

type MemDB = GMemDB[[]byte]

func NewMemDB

func NewMemDB() *MemDB

type MemDBIterator

type MemDBIterator[V any] struct {
	BTreeIteratorG[memdbItem[V]]
}

func NewMemDBIterator

func NewMemDBIterator[V any](start, end Key, iter btree.IterG[memdbItem[V]], ascending bool) *MemDBIterator[V]

func (*MemDBIterator[V]) Value

func (it *MemDBIterator[V]) Value() V

type MockBlock

type MockBlock struct {
	Txs     []Tx
	Results []error
}

func NewMockBlock

func NewMockBlock(txs []Tx) *MockBlock

func (*MockBlock) ExecuteTx

func (b *MockBlock) ExecuteTx(txn TxnIndex, store MultiStore)

func (*MockBlock) Size

func (b *MockBlock) Size() int

type MultiLocations

type MultiLocations map[int]Locations

type MultiMVMemoryView

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

MultiMVMemoryView don't need to be thread-safe, there's a dedicated instance for each tx execution.

func NewMultiMVMemoryView

func NewMultiMVMemoryView(
	stores map[storetypes.StoreKey]int,
	storage MultiStore,
	mvMemory *MVMemory,
	schedule *Scheduler,
	txn TxnIndex,
) *MultiMVMemoryView

func (*MultiMVMemoryView) ApplyWriteSet

func (mv *MultiMVMemoryView) ApplyWriteSet(version TxnVersion) MultiLocations

func (*MultiMVMemoryView) GetKVStore

func (*MultiMVMemoryView) GetObjKVStore

func (mv *MultiMVMemoryView) GetObjKVStore(name storetypes.StoreKey) storetypes.ObjKVStore

func (*MultiMVMemoryView) GetStore

func (*MultiMVMemoryView) ReadSet

func (mv *MultiMVMemoryView) ReadSet() *MultiReadSet

type MultiMemDB

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

func NewMultiMemDB

func NewMultiMemDB(stores map[storetypes.StoreKey]int) *MultiMemDB

func (*MultiMemDB) GetKVStore

func (mmdb *MultiMemDB) GetKVStore(store storetypes.StoreKey) storetypes.KVStore

func (*MultiMemDB) GetObjKVStore

func (mmdb *MultiMemDB) GetObjKVStore(store storetypes.StoreKey) storetypes.ObjKVStore

func (*MultiMemDB) GetStore

func (mmdb *MultiMemDB) GetStore(store storetypes.StoreKey) storetypes.Store

type MultiReadSet

type MultiReadSet = map[int]*ReadSet

type MultiStore

type MultiStore interface {
	GetStore(storetypes.StoreKey) storetypes.Store
	GetKVStore(storetypes.StoreKey) storetypes.KVStore
	GetObjKVStore(storetypes.StoreKey) storetypes.ObjKVStore
}

type ObjMemDB

type ObjMemDB = GMemDB[any]

func NewObjMemDB

func NewObjMemDB() *ObjMemDB

type ReadDescriptor

type ReadDescriptor struct {
	Key Key
	// invalid Version means the key is read from storage
	Version TxnVersion
}

type ReadSet

type ReadSet struct {
	Reads     []ReadDescriptor
	Iterators []IteratorDescriptor
}

type Scheduler

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

Scheduler implements the scheduler for the block-stm ref: `Algorithm 4 The Scheduler module, variables, utility APIs and next task logic`

func NewScheduler

func NewScheduler(block_size int) *Scheduler

func (*Scheduler) CheckDone

func (s *Scheduler) CheckDone()

func (*Scheduler) DecreaseExecutionIdx

func (s *Scheduler) DecreaseExecutionIdx(target TxnIndex)

func (*Scheduler) DecreaseValidationIdx

func (s *Scheduler) DecreaseValidationIdx(target TxnIndex)

func (*Scheduler) Done

func (s *Scheduler) Done() bool

func (*Scheduler) FinishExecution

func (s *Scheduler) FinishExecution(version TxnVersion, wroteNewPath bool) (TxnVersion, TaskKind)

Invariant `num_active_tasks`: decreased if an invalid task is returned.

func (*Scheduler) FinishValidation

func (s *Scheduler) FinishValidation(txn TxnIndex, aborted bool) (TxnVersion, TaskKind)

Invariant `num_active_tasks`: decreased if an invalid task is returned.

func (*Scheduler) NextTask

func (s *Scheduler) NextTask() (TxnVersion, TaskKind)

NextTask returns the transaction index and task kind for the next task to execute or validate, returns invalid version if no task is available.

Invariant `num_active_tasks`: increased if a valid task is returned.

func (*Scheduler) NextVersionToExecute

func (s *Scheduler) NextVersionToExecute() TxnVersion

NextVersionToExecute get the next transaction index to execute, returns invalid version if no task is available

Invariant `num_active_tasks`: increased if a valid task is returned.

func (*Scheduler) NextVersionToValidate

func (s *Scheduler) NextVersionToValidate() TxnVersion

NextVersionToValidate get the next transaction index to validate, returns invalid version if no task is available.

Invariant `num_active_tasks`: increased if a valid task is returned.

func (*Scheduler) ResumeDependencies

func (s *Scheduler) ResumeDependencies(txns []TxnIndex)

func (*Scheduler) Stats

func (s *Scheduler) Stats() string

func (*Scheduler) TryIncarnate

func (s *Scheduler) TryIncarnate(idx TxnIndex) TxnVersion

TryIncarnate tries to incarnate a transaction index to execute. Returns the transaction version if successful, otherwise returns invalid version.

Invariant `num_active_tasks`: decreased if an invalid task is returned.

func (*Scheduler) TryValidationAbort

func (s *Scheduler) TryValidationAbort(version TxnVersion) bool

func (*Scheduler) WaitForDependency

func (s *Scheduler) WaitForDependency(txn TxnIndex, blocking_txn TxnIndex) *Condvar

type Status

type Status uint
const (
	StatusReadyToExecute Status = iota
	StatusExecuting
	StatusExecuted
	StatusAborting
	StatusSuspended
)

type StatusEntry

type StatusEntry struct {
	sync.Mutex
	// contains filtered or unexported fields
}

StatusEntry is a state machine for the status of a transaction, all the transitions are atomic protected by a mutex.

```mermaid stateDiagram-v2

[*] --> ReadyToExecute
ReadyToExecute --> Executing: TrySetExecuting()
Executing --> Executed: SetExecuted()
Executing --> Suspended: Suspend(cond)\nset cond
Executed --> Aborting: TryValidationAbort(incarnation)
Aborting --> ReadyToExecute: SetReadyStatus()\nincarnation++
Suspended --> Executing: Resume()

```

func (*StatusEntry) IsExecuted

func (s *StatusEntry) IsExecuted() (ok bool, incarnation Incarnation)

func (*StatusEntry) Resume

func (s *StatusEntry) Resume()

func (*StatusEntry) SetExecuted

func (s *StatusEntry) SetExecuted()

func (*StatusEntry) SetReadyStatus

func (s *StatusEntry) SetReadyStatus()

func (*StatusEntry) Suspend

func (s *StatusEntry) Suspend(cond *Condvar)

func (*StatusEntry) TrySetExecuting

func (s *StatusEntry) TrySetExecuting() (Incarnation, bool)

func (*StatusEntry) TryValidationAbort

func (s *StatusEntry) TryValidationAbort(incarnation Incarnation) bool

type TaskKind

type TaskKind int
const (
	TaskKindExecution TaskKind = iota
	TaskKindValidation
)

type Tx

type Tx func(MultiStore) error

func BankTransferTx

func BankTransferTx(i int, sender, receiver string, amount uint64) Tx

func IterateTx

func IterateTx(i int, sender, receiver string, amount uint64) Tx

func NoopTx

func NoopTx(i int, sender string) Tx

NoopTx verifies a signature and increases the nonce of the sender

type TxDependency

type TxDependency struct {
	sync.Mutex
	// contains filtered or unexported fields
}

func (*TxDependency) Swap

func (t *TxDependency) Swap(new []TxnIndex) []TxnIndex

type TxExecutor

type TxExecutor func(TxnIndex, MultiStore)

TxExecutor executes transactions on top of a multi-version memory view.

type TxnIndex

type TxnIndex int

type TxnVersion

type TxnVersion struct {
	Index       TxnIndex
	Incarnation Incarnation
}

func (TxnVersion) Valid

func (v TxnVersion) Valid() bool

Jump to

Keyboard shortcuts

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