updates

package
v1.2.0 Latest Latest
Warning

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

Go to latest
Published: May 17, 2024 License: Apache-2.0 Imports: 22 Imported by: 0

Documentation

Index

Constants

View Source
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

View Source
var (
	AppendNodeApproxSize int
	DeleteNodeApproxSize int

	DeleteChainApproxSize int
	MVCCHandleApproxSize  int
)

Functions

func CompareAppendNode added in v0.6.0

func CompareAppendNode(e, o *AppendNode) int

func MockTxnWithStartTS added in v0.8.0

func MockTxnWithStartTS(ts types.TS) *txnbase.Txn

func VisitDeltaloc added in v1.2.0

func VisitDeltaloc(bat, tnBatch *containers.Batch, object *catalog.ObjectEntry, blkID *objectio.Blockid, node *catalog.MVCCNode[*catalog.MetadataMVCCNode], commitTS, createTS types.TS)

Types

type AppendMVCCHandle added in v1.2.0

type AppendMVCCHandle struct {
	*sync.RWMutex
	// contains filtered or unexported fields
}

func NewAppendMVCCHandle added in v1.2.0

func NewAppendMVCCHandle(meta *catalog.ObjectEntry) *AppendMVCCHandle

func (*AppendMVCCHandle) AddAppendNodeLocked added in v1.2.0

func (n *AppendMVCCHandle) AddAppendNodeLocked(
	txn txnif.AsyncTxn,
	startRow uint32,
	maxRow uint32,
) (an *AppendNode, created bool)

AddAppendNodeLocked add a new appendnode to the list.

func (*AppendMVCCHandle) AllAppendsCommittedBefore added in v1.2.0

func (n *AppendMVCCHandle) AllAppendsCommittedBefore(ts types.TS) bool

AllAppendsCommittedBefore returns true if all appendnode is committed before ts.

func (*AppendMVCCHandle) CollectAppendLocked added in v1.2.0

func (n *AppendMVCCHandle) 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 (*AppendMVCCHandle) CollectUncommittedANodesPreparedBefore added in v1.2.0

func (n *AppendMVCCHandle) 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 (*AppendMVCCHandle) DeleteAppendNodeLocked added in v1.2.0

func (n *AppendMVCCHandle) DeleteAppendNodeLocked(node *AppendNode)

DeleteAppendNodeLocked deletes the appendnode from the append list. it is called when txn of the appendnode is aborted.

func (*AppendMVCCHandle) EstimateMemSizeLocked added in v1.2.0

func (n *AppendMVCCHandle) EstimateMemSizeLocked() int

func (*AppendMVCCHandle) GetAppendListener added in v1.2.0

func (n *AppendMVCCHandle) GetAppendListener() func(txnif.AppendNode) error

func (*AppendMVCCHandle) GetAppendNodeByRow added in v1.2.0

func (n *AppendMVCCHandle) GetAppendNodeByRow(row uint32) (an *AppendNode)

only for internal usage given a row, it returns the append node which contains the row

func (*AppendMVCCHandle) GetID added in v1.2.0

func (n *AppendMVCCHandle) GetID() *common.ID

func (*AppendMVCCHandle) GetLatestAppendPrepareTSLocked added in v1.2.0

func (n *AppendMVCCHandle) GetLatestAppendPrepareTSLocked() types.TS

func (*AppendMVCCHandle) GetTotalRow added in v1.2.0

func (n *AppendMVCCHandle) GetTotalRow() uint32

GetTotalRow is only for replay

func (*AppendMVCCHandle) GetVisibleRowLocked added in v1.2.0

func (n *AppendMVCCHandle) GetVisibleRowLocked(
	ctx context.Context,
	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 (*AppendMVCCHandle) OnReplayAppendNode added in v1.2.0

func (n *AppendMVCCHandle) OnReplayAppendNode(an *AppendNode)

func (*AppendMVCCHandle) PrepareCompact added in v1.2.0

func (n *AppendMVCCHandle) PrepareCompact() bool

func (*AppendMVCCHandle) PrepareCompactLocked added in v1.2.0

func (n *AppendMVCCHandle) PrepareCompactLocked() bool

Reschedule until all appendnode is committed. Pending appendnode is not visible for compaction txn.

func (*AppendMVCCHandle) ReleaseAppends added in v1.2.0

func (n *AppendMVCCHandle) 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 (*AppendMVCCHandle) SetAppendListener added in v1.2.0

func (n *AppendMVCCHandle) SetAppendListener(l func(txnif.AppendNode) error)

func (*AppendMVCCHandle) StringLocked added in v1.2.0

func (n *AppendMVCCHandle) StringLocked() string

type AppendNode

type AppendNode struct {
	*txnbase.TxnMVCCNode
	// contains filtered or unexported fields
}

func MockAppendNode

func MockAppendNode(ts types.TS, startRow, maxRow uint32, mvcc *AppendMVCCHandle) *AppendNode

func NewAppendNode

func NewAppendNode(
	txn txnif.AsyncTxn,
	startRow, maxRow uint32,
	mvcc *AppendMVCCHandle) *AppendNode

func NewCommittedAppendNode

func NewCommittedAppendNode(
	ts types.TS,
	startRow, maxRow uint32,
	mvcc *AppendMVCCHandle) *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) ReadFrom

func (node *AppendNode) ReadFrom(r io.Reader) (n int64, 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)

func (*AppendNode) WriteTo

func (node *AppendNode) WriteTo(w io.Writer) (n int64, err error)

type DeleteChain

type DeleteChain struct {
	*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) CollectDeletesInRangeWithLock added in v1.2.0

func (chain *DeleteChain) CollectDeletesInRangeWithLock(
	startTs, endTs types.TS,
	rwlocker *sync.RWMutex,
) (mask *nulls.Bitmap, err error)

CollectDeletesInRange collects [startTs, endTs)

func (*DeleteChain) CollectDeletesLocked

func (chain *DeleteChain) CollectDeletesLocked(
	txn txnif.TxnReader,
	rwlocker *sync.RWMutex) (merged *nulls.Bitmap, err error)

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

func (chain *DeleteChain) IsDeleted(row uint32, txn txnif.TxnReader, rwlocker *sync.RWMutex) (deleted bool, err error)

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) ResetPersistedMask added in v1.2.0

func (chain *DeleteChain) ResetPersistedMask()

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) GetBlockID added in v1.2.0

func (node *DeleteNode) GetBlockID() *objectio.Blockid

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.ObjectEntry

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

func (node *DeleteNode) ReadFrom(r io.Reader) (n int64, err error)

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)

func (*DeleteNode) WriteTo

func (node *DeleteNode) WriteTo(w io.Writer) (n int64, err error)

type DeltalocChain added in v1.2.0

type DeltalocChain struct {
	*catalog.BaseEntryImpl[*catalog.MetadataMVCCNode]
	// contains filtered or unexported fields
}

func NewDeltalocChain added in v1.2.0

func NewDeltalocChain(mvcc *MVCCHandle) *DeltalocChain

func (*DeltalocChain) GetBlockID added in v1.2.0

func (d *DeltalocChain) GetBlockID() *objectio.Blockid

func (*DeltalocChain) GetMeta added in v1.2.0

func (d *DeltalocChain) GetMeta() *catalog.ObjectEntry

func (*DeltalocChain) Is1PC added in v1.2.0

func (d *DeltalocChain) Is1PC() bool

func (*DeltalocChain) MakeCommand added in v1.2.0

func (d *DeltalocChain) MakeCommand(id uint32) (cmd txnif.TxnCmd, err error)

func (*DeltalocChain) PrepareCommit added in v1.2.0

func (d *DeltalocChain) PrepareCommit() (err error)

func (*DeltalocChain) PrepareRollback added in v1.2.0

func (d *DeltalocChain) PrepareRollback() error

func (*DeltalocChain) Set1PC added in v1.2.0

func (d *DeltalocChain) Set1PC()

type MVCCHandle

type MVCCHandle struct {
	*ObjectMVCCHandle
	// contains filtered or unexported fields
}

func NewMVCCHandle

func NewMVCCHandle(meta *ObjectMVCCHandle, blkID uint16) *MVCCHandle

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) CollectDeleteInRangeAfterDeltalocation added in v1.2.0

func (n *MVCCHandle) CollectDeleteInRangeAfterDeltalocation(
	ctx context.Context,
	start, end types.TS,
	withAborted bool,
	mp *mpool.MPool,
) (bat *containers.Batch, err error)

CollectDeleteInRangeAfterDeltalocation collects deletes after a certain delta location and committed in [start,end] When subscribe a table, it collects delta location, then it collects deletes. To avoid collecting duplicate deletes, it collects after start ts of the delta location. If the delta location is from CN, deletes is committed after startTS. CollectDeleteInRange still collect duplicate deletes.

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, persistedTS types.TS,
)

it collects all deletes in the range [start, end)

func (*MVCCHandle) CreateDeleteNode

func (n *MVCCHandle) CreateDeleteNode(txn txnif.AsyncTxn, deleteType handle.DeleteType) txnif.DeleteNode

func (*MVCCHandle) DecChangeIntentionCnt added in v0.8.0

func (n *MVCCHandle) DecChangeIntentionCnt()

func (*MVCCHandle) EstimateMemSizeLocked added in v1.0.0

func (n *MVCCHandle) EstimateMemSizeLocked() (dsize int)

func (*MVCCHandle) ExistDeleteInRangeLocked added in v1.2.0

func (n *MVCCHandle) ExistDeleteInRangeLocked(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) 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) GetDeltaLocAndCommitTS added in v1.2.0

func (n *MVCCHandle) GetDeltaLocAndCommitTS() (objectio.Location, types.TS, types.TS)

func (*MVCCHandle) GetDeltaLocAndCommitTSByTxn added in v1.2.0

func (n *MVCCHandle) GetDeltaLocAndCommitTSByTxn(txn txnif.TxnReader) (objectio.Location, types.TS)

func (*MVCCHandle) GetDeltaLocAndCommitTSLocked added in v1.2.0

func (n *MVCCHandle) GetDeltaLocAndCommitTSLocked() (objectio.Location, types.TS, types.TS)

func (*MVCCHandle) GetEntry added in v0.6.0

func (n *MVCCHandle) GetEntry() *catalog.ObjectEntry

func (*MVCCHandle) GetID

func (n *MVCCHandle) GetID() *common.ID

func (*MVCCHandle) InMemoryCollectDeleteInRange added in v1.2.0

func (n *MVCCHandle) InMemoryCollectDeleteInRange(
	ctx context.Context,
	start, end types.TS,
	withAborted bool,
	mp *mpool.MPool,
) (bat *containers.Batch, minTS, persisitedTS types.TS, err error)

func (*MVCCHandle) IncChangeIntentionCnt added in v0.8.0

func (n *MVCCHandle) IncChangeIntentionCnt()

func (*MVCCHandle) IsDeletedLocked

func (n *MVCCHandle) IsDeletedLocked(
	row uint32, txn txnif.TxnReader,
) (bool, error)

func (*MVCCHandle) OnReplayDeleteNode

func (n *MVCCHandle) OnReplayDeleteNode(deleteNode txnif.DeleteNode)

func (*MVCCHandle) ReplayDeltaLoc added in v1.2.0

func (n *MVCCHandle) ReplayDeltaLoc(mvcc *catalog.MVCCNode[*catalog.MetadataMVCCNode])

func (*MVCCHandle) StringLocked

func (n *MVCCHandle) StringLocked(level common.PPLevel, depth int, prefix string) string

func (*MVCCHandle) TryDeleteByDeltalocLocked added in v1.2.0

func (n *MVCCHandle) TryDeleteByDeltalocLocked(txn txnif.AsyncTxn, deltaLoc objectio.Location, needCheckWhenCommit bool) (entry txnif.TxnEntry, ok bool, err error)

func (*MVCCHandle) UpdateDeltaLocLocked added in v1.2.0

func (n *MVCCHandle) UpdateDeltaLocLocked(txn txnif.TxnReader, deltaloc objectio.Location, needCheckWhenCommit bool) (isNewNode bool, entry txnif.TxnEntry, err error)

type NodeType

type NodeType int8
const (
	NT_Normal NodeType = iota
	NT_Merge
	NT_Persisted
)

type ObjectMVCCHandle added in v1.2.0

type ObjectMVCCHandle struct {
	*sync.RWMutex
	// contains filtered or unexported fields
}

func NewObjectMVCCHandle added in v1.2.0

func NewObjectMVCCHandle(meta *catalog.ObjectEntry) *ObjectMVCCHandle

func (*ObjectMVCCHandle) EstimateMemSizeLocked added in v1.2.0

func (n *ObjectMVCCHandle) EstimateMemSizeLocked() (dsize int)

func (*ObjectMVCCHandle) GetChangeIntentionCntLocked added in v1.2.0

func (n *ObjectMVCCHandle) GetChangeIntentionCntLocked() uint32

func (*ObjectMVCCHandle) GetDeleteCnt added in v1.2.0

func (n *ObjectMVCCHandle) GetDeleteCnt() uint32

func (*ObjectMVCCHandle) GetDeletesListener added in v1.2.0

func (n *ObjectMVCCHandle) GetDeletesListener() func(uint64, types.TS) error

func (*ObjectMVCCHandle) GetDeltaLocAndCommitTS added in v1.2.0

func (n *ObjectMVCCHandle) GetDeltaLocAndCommitTS(blkID uint16) (loc objectio.Location, start, end types.TS)

func (*ObjectMVCCHandle) GetDeltaLocAndCommitTSByTxn added in v1.2.0

func (n *ObjectMVCCHandle) GetDeltaLocAndCommitTSByTxn(blkID uint16, txn txnif.TxnReader) (objectio.Location, types.TS)

func (*ObjectMVCCHandle) GetDeltaPersistedTS added in v1.2.0

func (n *ObjectMVCCHandle) GetDeltaPersistedTS() types.TS

func (*ObjectMVCCHandle) GetLatestDeltaloc added in v1.2.0

func (n *ObjectMVCCHandle) GetLatestDeltaloc(blkOffset uint16) objectio.Location

func (*ObjectMVCCHandle) GetLatestMVCCNode added in v1.2.0

func (n *ObjectMVCCHandle) GetLatestMVCCNode(blkOffset uint16) *catalog.MVCCNode[*catalog.MetadataMVCCNode]

func (*ObjectMVCCHandle) GetObject added in v1.2.0

func (n *ObjectMVCCHandle) GetObject() any

func (*ObjectMVCCHandle) GetOrCreateDeleteChainLocked added in v1.2.0

func (n *ObjectMVCCHandle) GetOrCreateDeleteChainLocked(blkID uint16) *MVCCHandle

func (*ObjectMVCCHandle) HasDeleteIntentsPreparedIn added in v1.2.0

func (n *ObjectMVCCHandle) HasDeleteIntentsPreparedIn(from, to types.TS) (found, isPersist bool)

func (*ObjectMVCCHandle) HasInMemoryDeleteIntentsPreparedInByBlock added in v1.2.0

func (n *ObjectMVCCHandle) HasInMemoryDeleteIntentsPreparedInByBlock(blkID uint16, from, to types.TS) (found, isPersist bool)

func (*ObjectMVCCHandle) InMemoryDeletesExisted added in v1.2.0

func (n *ObjectMVCCHandle) InMemoryDeletesExisted() bool

func (*ObjectMVCCHandle) IsDeletedLocked added in v1.2.0

func (n *ObjectMVCCHandle) IsDeletedLocked(
	row uint32, txn txnif.TxnReader, blkID uint16,
) (bool, error)

func (*ObjectMVCCHandle) OnApplyDelete added in v1.2.0

func (n *ObjectMVCCHandle) OnApplyDelete(
	deleted uint64,
	ts types.TS) (err error)

func (*ObjectMVCCHandle) ReplayDeltaLoc added in v1.2.0

func (n *ObjectMVCCHandle) ReplayDeltaLoc(vMVCCNode any, blkID uint16)

func (*ObjectMVCCHandle) SetDeletesListener added in v1.2.0

func (n *ObjectMVCCHandle) SetDeletesListener(l func(uint64, types.TS) error)

func (*ObjectMVCCHandle) StringBlkLocked added in v1.2.0

func (n *ObjectMVCCHandle) StringBlkLocked(level common.PPLevel, depth int, prefix string, blkid int) string

func (*ObjectMVCCHandle) StringLocked added in v1.2.0

func (n *ObjectMVCCHandle) StringLocked(level common.PPLevel, depth int, prefix string) string

func (*ObjectMVCCHandle) TryGetDeleteChain added in v1.2.0

func (n *ObjectMVCCHandle) TryGetDeleteChain(blkID uint16) *MVCCHandle

func (*ObjectMVCCHandle) UpgradeAllDeleteChain added in v1.2.0

func (n *ObjectMVCCHandle) UpgradeAllDeleteChain()

func (*ObjectMVCCHandle) UpgradeDeleteChain added in v1.2.0

func (n *ObjectMVCCHandle) UpgradeDeleteChain(blkID uint16)

func (*ObjectMVCCHandle) UpgradeDeleteChainByTSLocked added in v1.2.0

func (n *ObjectMVCCHandle) UpgradeDeleteChainByTSLocked(ts types.TS)

for test

func (*ObjectMVCCHandle) VisitDeletes added in v1.2.0

func (n *ObjectMVCCHandle) VisitDeletes(
	ctx context.Context,
	start, end types.TS,
	deltalocBat *containers.Batch,
	tnInsertBat *containers.Batch,
	skipInMemory bool) (delBatch *containers.Batch, deltalocStart, deltalocEnd int, err error)

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 NewEmptyCmd(cmdType uint16, version uint16) *UpdateCmd

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

func (c *UpdateCmd) Desc() string

func (*UpdateCmd) GetAppendNode

func (c *UpdateCmd) GetAppendNode() *AppendNode

func (*UpdateCmd) GetCurrentVersion added in v0.8.0

func (c *UpdateCmd) GetCurrentVersion() uint16

func (*UpdateCmd) GetDeleteNode

func (c *UpdateCmd) GetDeleteNode() *DeleteNode

func (*UpdateCmd) GetDest

func (c *UpdateCmd) GetDest() *common.ID

func (*UpdateCmd) GetType

func (c *UpdateCmd) GetType() uint16

func (*UpdateCmd) MarshalBinary added in v0.8.0

func (c *UpdateCmd) MarshalBinary() (buf []byte, err error)

func (*UpdateCmd) ReadFrom

func (c *UpdateCmd) ReadFrom(r io.Reader) (n int64, err error)

func (*UpdateCmd) SetReplayTxn added in v0.6.0

func (c *UpdateCmd) SetReplayTxn(txn txnif.AsyncTxn)

func (*UpdateCmd) String

func (c *UpdateCmd) String() string

func (*UpdateCmd) UnmarshalBinary added in v0.8.0

func (c *UpdateCmd) UnmarshalBinary(buf []byte) error

func (*UpdateCmd) VerboseString

func (c *UpdateCmd) VerboseString() string

func (*UpdateCmd) WriteTo

func (c *UpdateCmd) WriteTo(w io.Writer) (n int64, err error)

Jump to

Keyboard shortcuts

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