Documentation ¶
Index ¶
- Constants
- Variables
- func CompareAppendNode(e, o *AppendNode) int
- func MockTxnWithStartTS(ts types.TS) *txnbase.Txn
- type AppendNode
- func MockAppendNode(ts types.TS, startRow, maxRow uint32, mvcc *MVCCHandle) *AppendNode
- func NewAppendNode(txn txnif.AsyncTxn, startRow, maxRow uint32, mvcc *MVCCHandle) *AppendNode
- func NewCommittedAppendNode(ts types.TS, startRow, maxRow uint32, mvcc *MVCCHandle) *AppendNode
- func NewEmptyAppendNode() *AppendNode
- func (node *AppendNode) ApplyCommit() error
- func (node *AppendNode) ApplyRollback() (err error)
- func (node *AppendNode) CloneAll() *AppendNode
- func (node *AppendNode) CloneData() *AppendNode
- func (node *AppendNode) GeneralDesc() string
- func (node *AppendNode) GeneralString() string
- func (node *AppendNode) GeneralVerboseString() string
- func (node *AppendNode) GetCommitTS() types.TS
- func (node *AppendNode) GetID() *common.ID
- func (node *AppendNode) GetMaxRow() uint32
- func (node *AppendNode) GetStartRow() uint32
- func (node *AppendNode) Is1PC() bool
- func (node *AppendNode) IsNil() bool
- func (node *AppendNode) MakeCommand(id uint32) (cmd txnif.TxnCmd, err error)
- func (node *AppendNode) PrepareCommit() error
- func (node *AppendNode) PrepareRollback() (err error)
- func (node *AppendNode) ReadFrom(r io.Reader) (n int64, err error)
- func (node *AppendNode) Set1PC()
- func (node *AppendNode) SetMaxRow(row uint32)
- func (node *AppendNode) String() string
- func (node *AppendNode) Update(*AppendNode)
- func (node *AppendNode) WriteTo(w io.Writer) (n int64, err error)
- type DeleteChain
- func (chain *DeleteChain) AddDeleteCnt(cnt uint32)
- func (chain *DeleteChain) AddMergeNode() txnif.DeleteNode
- func (chain *DeleteChain) AddNodeLocked(txn txnif.AsyncTxn, deleteType handle.DeleteType) txnif.DeleteNode
- func (chain *DeleteChain) AddPersistedNodeLocked(txn txnif.AsyncTxn, deltaloc objectio.Location) txnif.DeleteNode
- func (chain *DeleteChain) CollectDeletesInRange(startTs, endTs types.TS, rwlocker *sync.RWMutex) (mask *nulls.Bitmap, err error)
- func (chain *DeleteChain) CollectDeletesLocked(txn txnif.TxnReader, rwlocker *sync.RWMutex) (merged *nulls.Bitmap, err error)
- func (chain *DeleteChain) DeleteInDeleteView(deleteNode *DeleteNode)
- func (chain *DeleteChain) DepthLocked() int
- func (chain *DeleteChain) EstimateMemSizeLocked() int
- func (chain *DeleteChain) GetController() *MVCCHandle
- func (chain *DeleteChain) GetDeleteCnt() uint32
- func (chain *DeleteChain) GetDeleteNodeByRow(row uint32) (n *DeleteNode)
- func (chain *DeleteChain) HasDeleteIntentsPreparedInLocked(from, to types.TS) (found, isPersisted bool)
- func (chain *DeleteChain) InsertInDeleteView(row uint32, deleteNode *DeleteNode)
- func (chain *DeleteChain) IsDeleted(row uint32, txn txnif.TxnReader, rwlocker *sync.RWMutex) (deleted bool, err error)
- func (chain *DeleteChain) IsEmpty() bool
- func (chain *DeleteChain) OnReplayNode(deleteNode *DeleteNode)
- func (chain *DeleteChain) PrepareRangeDelete(start, end uint32, ts types.TS) (err error)
- func (chain *DeleteChain) RemoveNodeLocked(node txnif.DeleteNode)
- func (chain *DeleteChain) StringLocked() string
- func (chain *DeleteChain) UpdateLocked(node *DeleteNode)
- type DeleteNode
- func NewDeleteNode(txn txnif.AsyncTxn, dt handle.DeleteType, version uint16) *DeleteNode
- func NewEmptyDeleteNode() *DeleteNode
- func NewEmptyPersistedDeleteNode() *DeleteNode
- func NewMergedNode(commitTs types.TS) *DeleteNode
- func NewPersistedDeleteNode(txn txnif.AsyncTxn, deltaloc objectio.Location) *DeleteNode
- func (node *DeleteNode) ApplyCommit() (err error)
- func (node *DeleteNode) ApplyRollback() (err error)
- func (node *DeleteNode) AttachTo(chain *DeleteChain)
- func (node *DeleteNode) CloneAll() *DeleteNode
- func (node *DeleteNode) CloneData() *DeleteNode
- func (node *DeleteNode) Close()
- func (node *DeleteNode) DeletedPK() (pkVec map[uint32]containers.Vector)
- func (node *DeleteNode) DeletedRows() (rows []uint32)
- func (node *DeleteNode) GeneralDesc() string
- func (node *DeleteNode) GeneralString() string
- func (node *DeleteNode) GeneralVerboseString() string
- func (node *DeleteNode) GetCardinalityLocked() uint32
- func (node *DeleteNode) GetChain() txnif.DeleteChain
- func (node *DeleteNode) GetCommitTSLocked() types.TS
- func (node *DeleteNode) GetDeleteMaskLocked() *roaring.Bitmap
- func (node *DeleteNode) GetID() *common.ID
- func (node *DeleteNode) GetMeta() *catalog.BlockEntry
- func (node *DeleteNode) GetPrefix() []byte
- func (node *DeleteNode) GetPrepareTS() types.TS
- func (node *DeleteNode) GetRowMaskRefLocked() *roaring.Bitmap
- func (node *DeleteNode) GetStartTS() types.TS
- func (node *DeleteNode) HasOverlapLocked(start, end uint32) bool
- func (node *DeleteNode) Is1PC() bool
- func (node *DeleteNode) IsDeletedLocked(row uint32) bool
- func (node *DeleteNode) IsMerged() bool
- func (node *DeleteNode) IsNil() bool
- func (node *DeleteNode) IsPersistedDeletedNode() bool
- func (node *DeleteNode) Less(b *DeleteNode) int
- func (node *DeleteNode) MakeCommand(id uint32) (cmd txnif.TxnCmd, err error)
- func (node *DeleteNode) MergeLocked(o *DeleteNode)
- func (node *DeleteNode) OnApply() (err error)
- func (node *DeleteNode) PrepareCommit() (err error)
- func (node *DeleteNode) PrepareRollback() (err error)
- func (node *DeleteNode) RangeDeleteLocked(start, end uint32, pk containers.Vector, mp *mpool.MPool)
- func (node *DeleteNode) ReadFrom(r io.Reader) (n int64, err error)
- func (node *DeleteNode) Set1PC()
- func (node *DeleteNode) SetDeletes(mask *roaring.Bitmap)
- func (node *DeleteNode) StringLocked() string
- func (node *DeleteNode) Update(*DeleteNode)
- func (node *DeleteNode) WriteTo(w io.Writer) (n int64, err error)
- type MVCCHandle
- func (n *MVCCHandle) AddAppendNodeLocked(txn txnif.AsyncTxn, startRow uint32, maxRow uint32) (an *AppendNode, created bool)
- func (n *MVCCHandle) AllAppendsCommittedBefore(ts types.TS) bool
- func (n *MVCCHandle) CheckNotDeleted(start, end uint32, ts types.TS) error
- func (n *MVCCHandle) CollectAppendLocked(start, end types.TS, mp *mpool.MPool) (minRow, maxRow uint32, commitTSVec, abortVec containers.Vector, ...)
- func (n *MVCCHandle) CollectDeleteLocked(start, end types.TS, pkType types.Type, mp *mpool.MPool) (rowIDVec, commitTSVec, pkVec, abortVec containers.Vector, aborts *nulls.Bitmap, ...)
- func (n *MVCCHandle) CollectUncommittedANodesPreparedBefore(ts types.TS, foreachFn func(*AppendNode)) (anyWaitable bool)
- func (n *MVCCHandle) CreateDeleteNode(txn txnif.AsyncTxn, deleteType handle.DeleteType) txnif.DeleteNode
- func (n *MVCCHandle) CreatePersistedDeleteNode(txn txnif.AsyncTxn, deltaloc objectio.Location) txnif.DeleteNode
- func (n *MVCCHandle) DecChangeIntentionCnt()
- func (n *MVCCHandle) DeleteAppendNodeLocked(node *AppendNode)
- func (n *MVCCHandle) EstimateMemSizeLocked() (asize int, dsize int)
- func (n *MVCCHandle) ExistDeleteInRange(start, end types.TS) (exist bool)
- func (n *MVCCHandle) GetAppendListener() func(txnif.AppendNode) error
- func (n *MVCCHandle) GetAppendNodeByRow(row uint32) (an *AppendNode)
- func (n *MVCCHandle) GetChangeIntentionCnt() uint32
- func (n *MVCCHandle) GetDeleteChain() *DeleteChain
- func (n *MVCCHandle) GetDeleteCnt() uint32
- func (n *MVCCHandle) GetDeleteNodeByRow(row uint32) (an *DeleteNode)
- func (n *MVCCHandle) GetDeletesListener() func(uint64, types.TS) error
- func (n *MVCCHandle) GetEntry() *catalog.BlockEntry
- func (n *MVCCHandle) GetID() *common.ID
- func (n *MVCCHandle) GetTotalRow() uint32
- func (n *MVCCHandle) GetVisibleRowLocked(txn txnif.TxnReader) (maxrow uint32, visible bool, holes *nulls.Bitmap, err error)
- func (n *MVCCHandle) IncChangeIntentionCnt()
- func (n *MVCCHandle) IsDeletedLocked(row uint32, txn txnif.TxnReader, rwlocker *sync.RWMutex) (bool, error)
- func (n *MVCCHandle) OnReplayAppendNode(an *AppendNode)
- func (n *MVCCHandle) OnReplayDeleteNode(deleteNode txnif.DeleteNode)
- func (n *MVCCHandle) PrepareCompact() bool
- func (n *MVCCHandle) ReleaseAppends()
- func (n *MVCCHandle) SetAppendListener(l func(txnif.AppendNode) error)
- func (n *MVCCHandle) SetDeletesListener(l func(uint64, types.TS) error)
- func (n *MVCCHandle) StringLocked() string
- func (n *MVCCHandle) UpgradeDeleteChainByTS(flushed types.TS)
- type NodeType
- type UpdateCmd
- func (c *UpdateCmd) ApplyCommit()
- func (c *UpdateCmd) ApplyRollback()
- func (c *UpdateCmd) Desc() string
- func (c *UpdateCmd) GetAppendNode() *AppendNode
- func (c *UpdateCmd) GetCurrentVersion() uint16
- func (c *UpdateCmd) GetDeleteNode() *DeleteNode
- func (c *UpdateCmd) GetDest() *common.ID
- func (c *UpdateCmd) GetType() uint16
- func (c *UpdateCmd) MarshalBinary() (buf []byte, err error)
- func (c *UpdateCmd) ReadFrom(r io.Reader) (n int64, err error)
- func (c *UpdateCmd) SetReplayTxn(txn txnif.AsyncTxn)
- func (c *UpdateCmd) String() string
- func (c *UpdateCmd) UnmarshalBinary(buf []byte) error
- func (c *UpdateCmd) VerboseString() string
- func (c *UpdateCmd) WriteTo(w io.Writer) (n int64, err error)
Constants ¶
const ( IOET_WALTxnCommand_AppendNode uint16 = 3004 IOET_WALTxnCommand_DeleteNode uint16 = 3005 IOET_WALTxnCommand_PersistedDeleteNode uint16 = 3013 IOET_WALTxnCommand_AppendNode_V1 uint16 = 1 IOET_WALTxnCommand_DeleteNode_V1 uint16 = 1 IOET_WALTxnCommand_DeleteNode_V2 uint16 = 2 IOET_WALTxnCommand_PersistedDeleteNode_V1 uint16 = 1 IOET_WALTxnCommand_AppendNode_CurrVer = IOET_WALTxnCommand_AppendNode_V1 IOET_WALTxnCommand_DeleteNode_CurrVer = IOET_WALTxnCommand_DeleteNode_V2 IOET_WALTxnCommand_PersistedDeleteNode_CurrVer = IOET_WALTxnCommand_PersistedDeleteNode_V1 )
Variables ¶
var ( AppendNodeApproxSize int DeleteNodeApproxSize int DeleteChainApproxSize int MVCCHandleApproxSize int )
Functions ¶
func CompareAppendNode ¶ added in v0.6.0
func CompareAppendNode(e, o *AppendNode) int
Types ¶
type AppendNode ¶
type AppendNode struct { *txnbase.TxnMVCCNode // contains filtered or unexported fields }
func MockAppendNode ¶
func MockAppendNode(ts types.TS, startRow, maxRow uint32, mvcc *MVCCHandle) *AppendNode
func NewAppendNode ¶
func NewAppendNode( txn txnif.AsyncTxn, startRow, maxRow uint32, mvcc *MVCCHandle) *AppendNode
func NewCommittedAppendNode ¶
func NewCommittedAppendNode( ts types.TS, startRow, maxRow uint32, mvcc *MVCCHandle) *AppendNode
func NewEmptyAppendNode ¶ added in v0.6.0
func NewEmptyAppendNode() *AppendNode
func (*AppendNode) ApplyCommit ¶
func (node *AppendNode) ApplyCommit() error
func (*AppendNode) ApplyRollback ¶
func (node *AppendNode) ApplyRollback() (err error)
func (*AppendNode) CloneAll ¶ added in v0.6.0
func (node *AppendNode) CloneAll() *AppendNode
func (*AppendNode) CloneData ¶ added in v0.6.0
func (node *AppendNode) CloneData() *AppendNode
func (*AppendNode) GeneralDesc ¶
func (node *AppendNode) GeneralDesc() string
func (*AppendNode) GeneralString ¶
func (node *AppendNode) GeneralString() string
func (*AppendNode) GeneralVerboseString ¶
func (node *AppendNode) GeneralVerboseString() string
func (*AppendNode) GetCommitTS ¶
func (node *AppendNode) GetCommitTS() types.TS
func (*AppendNode) GetID ¶
func (node *AppendNode) GetID() *common.ID
func (*AppendNode) GetMaxRow ¶
func (node *AppendNode) GetMaxRow() uint32
func (*AppendNode) GetStartRow ¶ added in v0.6.0
func (node *AppendNode) GetStartRow() uint32
func (*AppendNode) Is1PC ¶ added in v0.6.0
func (node *AppendNode) Is1PC() bool
func (*AppendNode) IsNil ¶ added in v0.8.0
func (node *AppendNode) IsNil() bool
func (*AppendNode) MakeCommand ¶
func (node *AppendNode) MakeCommand(id uint32) (cmd txnif.TxnCmd, err error)
func (*AppendNode) PrepareCommit ¶
func (node *AppendNode) PrepareCommit() error
func (*AppendNode) PrepareRollback ¶
func (node *AppendNode) PrepareRollback() (err error)
func (*AppendNode) Set1PC ¶ added in v0.6.0
func (node *AppendNode) Set1PC()
func (*AppendNode) SetMaxRow ¶
func (node *AppendNode) SetMaxRow(row uint32)
func (*AppendNode) String ¶ added in v0.6.0
func (node *AppendNode) String() string
func (*AppendNode) Update ¶ added in v0.6.0
func (node *AppendNode) Update(*AppendNode)
type DeleteChain ¶
type DeleteChain struct { *sync.RWMutex *txnbase.MVCCChain[*DeleteNode] // contains filtered or unexported fields }
func NewDeleteChain ¶
func NewDeleteChain(rwlocker *sync.RWMutex, mvcc *MVCCHandle) *DeleteChain
func (*DeleteChain) AddDeleteCnt ¶
func (chain *DeleteChain) AddDeleteCnt(cnt uint32)
func (*DeleteChain) AddMergeNode ¶
func (chain *DeleteChain) AddMergeNode() txnif.DeleteNode
func (*DeleteChain) AddNodeLocked ¶
func (chain *DeleteChain) AddNodeLocked(txn txnif.AsyncTxn, deleteType handle.DeleteType) txnif.DeleteNode
func (*DeleteChain) AddPersistedNodeLocked ¶ added in v1.0.0
func (chain *DeleteChain) AddPersistedNodeLocked(txn txnif.AsyncTxn, deltaloc objectio.Location) txnif.DeleteNode
func (*DeleteChain) CollectDeletesInRange ¶
func (chain *DeleteChain) CollectDeletesInRange( startTs, endTs types.TS, rwlocker *sync.RWMutex) (mask *nulls.Bitmap, err error)
CollectDeletesInRange collects [startTs, endTs)
func (*DeleteChain) CollectDeletesLocked ¶
func (*DeleteChain) DeleteInDeleteView ¶ added in v0.6.0
func (chain *DeleteChain) DeleteInDeleteView(deleteNode *DeleteNode)
func (*DeleteChain) DepthLocked ¶
func (chain *DeleteChain) DepthLocked() int
func (*DeleteChain) EstimateMemSizeLocked ¶ added in v1.0.0
func (chain *DeleteChain) EstimateMemSizeLocked() int
func (*DeleteChain) GetController ¶
func (chain *DeleteChain) GetController() *MVCCHandle
func (*DeleteChain) GetDeleteCnt ¶
func (chain *DeleteChain) GetDeleteCnt() uint32
func (*DeleteChain) GetDeleteNodeByRow ¶ added in v0.6.0
func (chain *DeleteChain) GetDeleteNodeByRow(row uint32) (n *DeleteNode)
func (*DeleteChain) HasDeleteIntentsPreparedInLocked ¶ added in v0.6.0
func (chain *DeleteChain) HasDeleteIntentsPreparedInLocked(from, to types.TS) (found, isPersisted bool)
any uncommited node, return true any committed node with prepare ts within [from, to], return true
func (*DeleteChain) InsertInDeleteView ¶ added in v0.6.0
func (chain *DeleteChain) InsertInDeleteView(row uint32, deleteNode *DeleteNode)
func (*DeleteChain) IsEmpty ¶ added in v1.0.0
func (chain *DeleteChain) IsEmpty() bool
IsEmtpy indicates whether memory deletes are empty
func (*DeleteChain) OnReplayNode ¶
func (chain *DeleteChain) OnReplayNode(deleteNode *DeleteNode)
func (*DeleteChain) PrepareRangeDelete ¶
func (chain *DeleteChain) PrepareRangeDelete(start, end uint32, ts types.TS) (err error)
func (*DeleteChain) RemoveNodeLocked ¶
func (chain *DeleteChain) RemoveNodeLocked(node txnif.DeleteNode)
func (*DeleteChain) StringLocked ¶
func (chain *DeleteChain) StringLocked() string
func (*DeleteChain) UpdateLocked ¶
func (chain *DeleteChain) UpdateLocked(node *DeleteNode)
type DeleteNode ¶
type DeleteNode struct { *common.GenericDLNode[*DeleteNode] *txnbase.TxnMVCCNode // contains filtered or unexported fields }
func NewDeleteNode ¶
func NewDeleteNode(txn txnif.AsyncTxn, dt handle.DeleteType, version uint16) *DeleteNode
func NewEmptyDeleteNode ¶ added in v0.6.0
func NewEmptyDeleteNode() *DeleteNode
func NewEmptyPersistedDeleteNode ¶ added in v1.0.0
func NewEmptyPersistedDeleteNode() *DeleteNode
func NewMergedNode ¶
func NewMergedNode(commitTs types.TS) *DeleteNode
func NewPersistedDeleteNode ¶ added in v1.0.0
func NewPersistedDeleteNode(txn txnif.AsyncTxn, deltaloc objectio.Location) *DeleteNode
func (*DeleteNode) ApplyCommit ¶
func (node *DeleteNode) ApplyCommit() (err error)
func (*DeleteNode) ApplyRollback ¶
func (node *DeleteNode) ApplyRollback() (err error)
func (*DeleteNode) AttachTo ¶
func (node *DeleteNode) AttachTo(chain *DeleteChain)
func (*DeleteNode) CloneAll ¶ added in v0.6.0
func (node *DeleteNode) CloneAll() *DeleteNode
func (*DeleteNode) CloneData ¶ added in v0.6.0
func (node *DeleteNode) CloneData() *DeleteNode
func (*DeleteNode) Close ¶ added in v1.1.0
func (node *DeleteNode) Close()
func (*DeleteNode) DeletedPK ¶ added in v1.0.0
func (node *DeleteNode) DeletedPK() (pkVec map[uint32]containers.Vector)
func (*DeleteNode) DeletedRows ¶ added in v0.6.0
func (node *DeleteNode) DeletedRows() (rows []uint32)
func (*DeleteNode) GeneralDesc ¶
func (node *DeleteNode) GeneralDesc() string
func (*DeleteNode) GeneralString ¶
func (node *DeleteNode) GeneralString() string
func (*DeleteNode) GeneralVerboseString ¶
func (node *DeleteNode) GeneralVerboseString() string
func (*DeleteNode) GetCardinalityLocked ¶
func (node *DeleteNode) GetCardinalityLocked() uint32
func (*DeleteNode) GetChain ¶
func (node *DeleteNode) GetChain() txnif.DeleteChain
func (*DeleteNode) GetCommitTSLocked ¶
func (node *DeleteNode) GetCommitTSLocked() types.TS
func (*DeleteNode) GetDeleteMaskLocked ¶
func (node *DeleteNode) GetDeleteMaskLocked() *roaring.Bitmap
func (*DeleteNode) GetID ¶
func (node *DeleteNode) GetID() *common.ID
func (*DeleteNode) GetMeta ¶ added in v0.8.0
func (node *DeleteNode) GetMeta() *catalog.BlockEntry
func (*DeleteNode) GetPrefix ¶ added in v0.8.0
func (node *DeleteNode) GetPrefix() []byte
func (*DeleteNode) GetPrepareTS ¶ added in v0.6.0
func (node *DeleteNode) GetPrepareTS() types.TS
func (*DeleteNode) GetRowMaskRefLocked ¶
func (node *DeleteNode) GetRowMaskRefLocked() *roaring.Bitmap
func (*DeleteNode) GetStartTS ¶
func (node *DeleteNode) GetStartTS() types.TS
func (*DeleteNode) HasOverlapLocked ¶
func (node *DeleteNode) HasOverlapLocked(start, end uint32) bool
func (*DeleteNode) Is1PC ¶ added in v0.6.0
func (node *DeleteNode) Is1PC() bool
func (*DeleteNode) IsDeletedLocked ¶
func (node *DeleteNode) IsDeletedLocked(row uint32) bool
func (*DeleteNode) IsMerged ¶
func (node *DeleteNode) IsMerged() bool
func (*DeleteNode) IsNil ¶ added in v0.8.0
func (node *DeleteNode) IsNil() bool
func (*DeleteNode) IsPersistedDeletedNode ¶ added in v1.0.0
func (node *DeleteNode) IsPersistedDeletedNode() bool
func (*DeleteNode) Less ¶ added in v0.8.0
func (node *DeleteNode) Less(b *DeleteNode) int
func (*DeleteNode) MakeCommand ¶
func (node *DeleteNode) MakeCommand(id uint32) (cmd txnif.TxnCmd, err error)
func (*DeleteNode) MergeLocked ¶
func (node *DeleteNode) MergeLocked(o *DeleteNode)
func (*DeleteNode) OnApply ¶
func (node *DeleteNode) OnApply() (err error)
func (*DeleteNode) PrepareCommit ¶
func (node *DeleteNode) PrepareCommit() (err error)
func (*DeleteNode) PrepareRollback ¶
func (node *DeleteNode) PrepareRollback() (err error)
func (*DeleteNode) RangeDeleteLocked ¶
func (node *DeleteNode) RangeDeleteLocked( start, end uint32, pk containers.Vector, mp *mpool.MPool, )
func (*DeleteNode) Set1PC ¶ added in v0.6.0
func (node *DeleteNode) Set1PC()
func (*DeleteNode) SetDeletes ¶
func (node *DeleteNode) SetDeletes(mask *roaring.Bitmap)
func (*DeleteNode) StringLocked ¶
func (node *DeleteNode) StringLocked() string
func (*DeleteNode) Update ¶ added in v0.6.0
func (node *DeleteNode) Update(*DeleteNode)
type MVCCHandle ¶
func NewMVCCHandle ¶
func NewMVCCHandle(meta *catalog.BlockEntry) *MVCCHandle
func (*MVCCHandle) AddAppendNodeLocked ¶
func (n *MVCCHandle) AddAppendNodeLocked( txn txnif.AsyncTxn, startRow uint32, maxRow uint32, ) (an *AppendNode, created bool)
AddAppendNodeLocked add a new appendnode to the list.
func (*MVCCHandle) AllAppendsCommittedBefore ¶ added in v0.8.0
func (n *MVCCHandle) AllAppendsCommittedBefore(ts types.TS) bool
AllAppendsCommittedBefore returns true if all appendnode is committed before ts.
func (*MVCCHandle) CheckNotDeleted ¶
func (n *MVCCHandle) CheckNotDeleted(start, end uint32, ts types.TS) error
it checks whether there is any delete in the range [start, end) ts is not used for now
func (*MVCCHandle) CollectAppendLocked ¶ added in v0.7.0
func (n *MVCCHandle) CollectAppendLocked( start, end types.TS, mp *mpool.MPool, ) ( minRow, maxRow uint32, commitTSVec, abortVec containers.Vector, aborts *nulls.Bitmap, )
it collects all append nodes in the range [start, end] minRow: is the min row maxRow: is the max row commitTSVec: is the commit ts vector abortVec: is the abort vector aborts: is the aborted bitmap
func (*MVCCHandle) CollectDeleteLocked ¶ added in v0.8.0
func (n *MVCCHandle) CollectDeleteLocked( start, end types.TS, pkType types.Type, mp *mpool.MPool, ) (rowIDVec, commitTSVec, pkVec, abortVec containers.Vector, aborts *nulls.Bitmap, deletes []uint32, minTS types.TS, )
it collects all deletes in the range [start, end)
func (*MVCCHandle) CollectUncommittedANodesPreparedBefore ¶ added in v0.6.0
func (n *MVCCHandle) CollectUncommittedANodesPreparedBefore( ts types.TS, foreachFn func(*AppendNode), ) (anyWaitable bool)
it collects all append nodes that are prepared before the given ts foreachFn is called for each append node that is prepared before the given ts
func (*MVCCHandle) CreateDeleteNode ¶
func (n *MVCCHandle) CreateDeleteNode(txn txnif.AsyncTxn, deleteType handle.DeleteType) txnif.DeleteNode
func (*MVCCHandle) CreatePersistedDeleteNode ¶ added in v1.0.0
func (n *MVCCHandle) CreatePersistedDeleteNode(txn txnif.AsyncTxn, deltaloc objectio.Location) txnif.DeleteNode
func (*MVCCHandle) DecChangeIntentionCnt ¶ added in v0.8.0
func (n *MVCCHandle) DecChangeIntentionCnt()
func (*MVCCHandle) DeleteAppendNodeLocked ¶ added in v0.6.0
func (n *MVCCHandle) DeleteAppendNodeLocked(node *AppendNode)
DeleteAppendNodeLocked deletes the appendnode from the append list. it is called when txn of the appendnode is aborted.
func (*MVCCHandle) EstimateMemSizeLocked ¶ added in v1.0.0
func (n *MVCCHandle) EstimateMemSizeLocked() (asize int, dsize int)
func (*MVCCHandle) ExistDeleteInRange ¶ added in v0.6.0
func (n *MVCCHandle) ExistDeleteInRange(start, end types.TS) (exist bool)
ExistDeleteInRange check if there is any delete in the range [start, end] it loops the delete chain and check if there is any delete node in the range
func (*MVCCHandle) GetAppendListener ¶ added in v0.6.0
func (n *MVCCHandle) GetAppendListener() func(txnif.AppendNode) error
func (*MVCCHandle) GetAppendNodeByRow ¶ added in v0.6.0
func (n *MVCCHandle) GetAppendNodeByRow(row uint32) (an *AppendNode)
only for internal usage given a row, it returns the append node which contains the row
func (*MVCCHandle) GetChangeIntentionCnt ¶ added in v0.8.0
func (n *MVCCHandle) GetChangeIntentionCnt() uint32
GetChangeIntentionCnt returns the number of operation of delete, which is updated before commiting. Note: Now it is ** only ** used in checkpointe runner to check whether this block has any chance to be flushed
func (*MVCCHandle) GetDeleteChain ¶
func (n *MVCCHandle) GetDeleteChain() *DeleteChain
func (*MVCCHandle) GetDeleteCnt ¶
func (n *MVCCHandle) GetDeleteCnt() uint32
GetDeleteCnt returns committed deleted rows
func (*MVCCHandle) GetDeleteNodeByRow ¶ added in v0.6.0
func (n *MVCCHandle) GetDeleteNodeByRow(row uint32) (an *DeleteNode)
func (*MVCCHandle) GetDeletesListener ¶
func (n *MVCCHandle) GetDeletesListener() func(uint64, types.TS) error
func (*MVCCHandle) GetEntry ¶ added in v0.6.0
func (n *MVCCHandle) GetEntry() *catalog.BlockEntry
func (*MVCCHandle) GetID ¶
func (n *MVCCHandle) GetID() *common.ID
func (*MVCCHandle) GetTotalRow ¶
func (n *MVCCHandle) GetTotalRow() uint32
GetTotalRow is only for replay
func (*MVCCHandle) GetVisibleRowLocked ¶ added in v0.6.0
func (n *MVCCHandle) GetVisibleRowLocked( txn txnif.TxnReader, ) (maxrow uint32, visible bool, holes *nulls.Bitmap, err error)
it is used to get the visible max row for a txn maxrow: is the max row that the txn can see visible: is true if the txn can see any row holes: is the bitmap of the holes that the txn cannot see holes exists only if any append node was rollbacked
func (*MVCCHandle) IncChangeIntentionCnt ¶ added in v0.8.0
func (n *MVCCHandle) IncChangeIntentionCnt()
func (*MVCCHandle) IsDeletedLocked ¶
func (*MVCCHandle) OnReplayAppendNode ¶
func (n *MVCCHandle) OnReplayAppendNode(an *AppendNode)
func (*MVCCHandle) OnReplayDeleteNode ¶
func (n *MVCCHandle) OnReplayDeleteNode(deleteNode txnif.DeleteNode)
func (*MVCCHandle) PrepareCompact ¶ added in v0.8.0
func (n *MVCCHandle) PrepareCompact() bool
Reschedule until all appendnode is committed. Pending appendnode is not visible for compaction txn.
func (*MVCCHandle) ReleaseAppends ¶ added in v0.8.0
func (n *MVCCHandle) ReleaseAppends()
NOTE: after this call all appends related APIs should not be called ReleaseAppends release all append nodes. it is only called when the appendable block is persisted and the memory node is released
func (*MVCCHandle) SetAppendListener ¶ added in v0.6.0
func (n *MVCCHandle) SetAppendListener(l func(txnif.AppendNode) error)
func (*MVCCHandle) SetDeletesListener ¶
func (n *MVCCHandle) SetDeletesListener(l func(uint64, types.TS) error)
func (*MVCCHandle) StringLocked ¶
func (n *MVCCHandle) StringLocked() string
func (*MVCCHandle) UpgradeDeleteChainByTS ¶ added in v0.8.0
func (n *MVCCHandle) UpgradeDeleteChainByTS(flushed types.TS)
type UpdateCmd ¶
type UpdateCmd struct { *txnbase.BaseCustomizedCmd // contains filtered or unexported fields }
func NewAppendCmd ¶
func NewAppendCmd(id uint32, app *AppendNode) *UpdateCmd
func NewDeleteCmd ¶
func NewDeleteCmd(id uint32, del *DeleteNode) *UpdateCmd
func NewEmptyCmd ¶
func NewPersistedDeleteCmd ¶ added in v1.0.0
func NewPersistedDeleteCmd(id uint32, del *DeleteNode) *UpdateCmd
func (*UpdateCmd) ApplyCommit ¶ added in v0.6.0
func (c *UpdateCmd) ApplyCommit()
func (*UpdateCmd) ApplyRollback ¶ added in v0.6.0
func (c *UpdateCmd) ApplyRollback()
func (*UpdateCmd) GetAppendNode ¶
func (c *UpdateCmd) GetAppendNode() *AppendNode
func (*UpdateCmd) GetCurrentVersion ¶ added in v0.8.0
func (*UpdateCmd) GetDeleteNode ¶
func (c *UpdateCmd) GetDeleteNode() *DeleteNode