Documentation ¶
Index ¶
- Constants
- Variables
- func ActiveWithNoTxnFilter(be *BaseEntryImpl[*MetadataMVCCNode]) bool
- func AppendableBlkFilter(be *BlockEntry) bool
- func CmdName(t uint16) string
- func CompareBaseNode[T BaseNode[T]](e, o *MVCCNode[T]) int
- func CompareUint64(left, right uint64) int
- func EncodeAccessInfo(ai *accessInfo) []byte
- func EncodeBlockNode(node *BlockNode) []byte
- func EncodeEntryMVCCNode(node *EntryMVCCNode) []byte
- func GetAttrIdx(attrs []string, name string) int
- func IsFakePkName(name string) bool
- func MockBatch(schema *Schema, rows int) *containers.Batch
- func MockTxnFactory(catalog *Catalog) txnbase.TxnFactory
- func MockTxnStoreFactory(catalog *Catalog) txnbase.TxnStoreFactory
- func NewEmptyMVCCNodeFactory[T BaseNode[T]](factory func() T) func() *MVCCNode[T]
- func NonAppendableBlkFilter(be *BlockEntry) bool
- type BaseEntry
- type BaseEntryImpl
- func (be *BaseEntryImpl[T]) CloneCommittedInRange(start, end types.TS) BaseEntry
- func (be *BaseEntryImpl[T]) ConflictCheck(txn txnif.TxnReader) (err error)
- func (be *BaseEntryImpl[T]) CreateWithTS(ts types.TS, baseNode T)
- func (be *BaseEntryImpl[T]) CreateWithTxn(txn txnif.AsyncTxn, baseNode T)
- func (be *BaseEntryImpl[T]) DeleteAfter(ts types.TS) bool
- func (be *BaseEntryImpl[T]) DeleteBefore(ts types.TS) bool
- func (be *BaseEntryImpl[T]) DeleteLocked(txn txnif.TxnReader) (isNewNode bool, err error)
- func (be *BaseEntryImpl[T]) DropEntryLocked(txn txnif.TxnReader) (isNewNode bool, err error)
- func (be *BaseEntryImpl[T]) GetCreatedAtLocked() types.TS
- func (be *BaseEntryImpl[T]) GetDeleteAt() types.TS
- func (be *BaseEntryImpl[T]) GetVisibility(txn txnif.TxnReader) (visible, dropped bool)
- func (be *BaseEntryImpl[T]) GetVisibilityLocked(txn txnif.TxnReader) (visible, dropped bool)
- func (be *BaseEntryImpl[T]) HasDropCommitted() bool
- func (be *BaseEntryImpl[T]) HasDropCommittedLocked() bool
- func (be *BaseEntryImpl[T]) IsVisible(txn txnif.TxnReader, mu *sync.RWMutex) (ok bool, err error)
- func (be *BaseEntryImpl[T]) NeedWaitCommitting(startTS types.TS) (bool, txnif.TxnReader)
- func (be *BaseEntryImpl[T]) PPString(level common.PPLevel, depth int, prefix string) string
- func (be *BaseEntryImpl[T]) PrepareAdd(txn txnif.TxnReader) (err error)
- func (be *BaseEntryImpl[T]) String() string
- func (be *BaseEntryImpl[T]) StringLocked() string
- func (be *BaseEntryImpl[T]) TryGetTerminatedTS(waitIfcommitting bool) (terminated bool, TS types.TS)
- type BaseNode
- type BlockDataFactory
- type BlockEntry
- func NewBlockEntry(segment *SegmentEntry, id *objectio.Blockid, txn txnif.AsyncTxn, ...) *BlockEntry
- func NewBlockEntryWithMeta(segment *SegmentEntry, id *objectio.Blockid, txn txnif.AsyncTxn, ...) *BlockEntry
- func NewReplayBlockEntry() *BlockEntry
- func NewStandaloneBlock(segment *SegmentEntry, id *objectio.Blockid, ts types.TS) *BlockEntry
- func NewStandaloneBlockWithLoc(segment *SegmentEntry, id *objectio.Blockid, ts types.TS, ...) *BlockEntry
- func NewSysBlockEntry(segment *SegmentEntry, id types.Blockid) *BlockEntry
- func (entry *BlockEntry) AsCommonID() *common.ID
- func (entry *BlockEntry) BuildDeleteObjectName() objectio.ObjectName
- func (entry *BlockEntry) CreateWithLoc(ts types.TS, metaLoc objectio.Location, deltaLoc objectio.Location)
- func (entry *BlockEntry) CreateWithTxnAndMeta(txn txnif.AsyncTxn, metaLoc objectio.Location, deltaLoc objectio.Location)
- func (entry *BlockEntry) FastGetMetaLoc() objectio.Location
- func (entry *BlockEntry) GetBlockData() data.Block
- func (entry *BlockEntry) GetCatalog() *Catalog
- func (entry *BlockEntry) GetDeltaLoc() objectio.Location
- func (entry *BlockEntry) GetDeltaLocAndCommitTS() (objectio.Location, types.TS)
- func (entry *BlockEntry) GetDeltaPersistedTS() types.TS
- func (entry *BlockEntry) GetDeltaPersistedTSByTxn(txn txnif.TxnReader) types.TS
- func (entry *BlockEntry) GetMetaLoc() objectio.Location
- func (entry *BlockEntry) GetPKZoneMap(ctx context.Context, fs fileservice.FileService) (zm *index.ZM, err error)
- func (entry *BlockEntry) GetSchema() *Schema
- func (entry *BlockEntry) GetSegment() *SegmentEntry
- func (entry *BlockEntry) GetTerminationTS() (ts types.TS, terminated bool)
- func (entry *BlockEntry) GetVisibleDeltaLoc(txn txnif.TxnReader) objectio.Location
- func (entry *BlockEntry) GetVisibleMetaLoc(txn txnif.TxnReader) objectio.Location
- func (entry *BlockEntry) HasPersistedData() bool
- func (entry *BlockEntry) HasPersistedDeltaData() bool
- func (entry *BlockEntry) InitData(factory DataFactory)
- func (entry *BlockEntry) Is1PC() bool
- func (entry *BlockEntry) IsActive() bool
- func (entry *BlockEntry) IsAppendable() bool
- func (entry *BlockEntry) Less(b *BlockEntry) int
- func (entry *BlockEntry) MakeCommand(id uint32) (cmd txnif.TxnCmd, err error)
- func (entry *BlockEntry) MakeKey() []byte
- func (entry *BlockEntry) PPString(level common.PPLevel, depth int, prefix string) string
- func (entry *BlockEntry) PrepareCompact() bool
- func (entry *BlockEntry) PrepareRollback() (err error)
- func (entry *BlockEntry) Repr() string
- func (entry *BlockEntry) Set1PC()
- func (entry *BlockEntry) String() string
- func (entry *BlockEntry) StringLocked() string
- func (entry *BlockEntry) StringWithLevel(level common.PPLevel) string
- func (entry *BlockEntry) StringWithLevelLocked(level common.PPLevel) string
- func (entry *BlockEntry) UpdateDeltaLoc(txn txnif.TxnReader, deltaloc objectio.Location) (isNewNode bool, err error)
- func (entry *BlockEntry) UpdateMetaLoc(txn txnif.TxnReader, metaLoc objectio.Location) (isNewNode bool, err error)
- type BlockNode
- type Catalog
- func (catalog *Catalog) AddEntryLocked(database *DBEntry, txn txnif.TxnReader, skipDedup bool) error
- func (catalog *Catalog) Close() error
- func (catalog *Catalog) CoarseDBCnt() int
- func (catalog *Catalog) CreateDBEntry(name, createSql, datTyp string, txn txnif.AsyncTxn) (*DBEntry, error)
- func (catalog *Catalog) CreateDBEntryByTS(name string, ts types.TS) (*DBEntry, error)
- func (catalog *Catalog) CreateDBEntryWithID(name, createSql, datTyp string, id uint64, txn txnif.AsyncTxn) (*DBEntry, error)
- func (catalog *Catalog) DropDBEntry(name string, txn txnif.AsyncTxn) (newEntry bool, deleted *DBEntry, err error)
- func (catalog *Catalog) DropDBEntryByID(id uint64, txn txnif.AsyncTxn) (newEntry bool, deleted *DBEntry, err error)
- func (catalog *Catalog) GCByTS(ctx context.Context, ts types.TS)
- func (catalog *Catalog) GetDBEntryByName(tenantID uint32, name string, txn txnif.TxnReader) (db *DBEntry, err error)
- func (catalog *Catalog) GetDatabaseByID(id uint64) (db *DBEntry, err error)
- func (catalog *Catalog) GetItemNodeByIDLocked(id uint64) *common.GenericDLNode[*DBEntry]
- func (catalog *Catalog) InitSystemDB()
- func (catalog *Catalog) MakeDBIt(reverse bool) *common.GenericSortedDListIt[*DBEntry]
- func (catalog *Catalog) OnReplayBlockBatch(ins, insTxn, del, delTxn *containers.Batch, dataFactory DataFactory)
- func (catalog *Catalog) OnReplayDatabaseBatch(ins, insTxn, del, delTxn *containers.Batch)
- func (catalog *Catalog) OnReplaySegmentBatch(ins, insTxn, del, delTxn *containers.Batch, dataFactory DataFactory)
- func (catalog *Catalog) OnReplayTableBatch(ins, insTxn, insCol, del, delTxn *containers.Batch, dataFactory DataFactory)
- func (catalog *Catalog) PPString(level common.PPLevel, depth int, prefix string) string
- func (catalog *Catalog) RecurLoop(processor Processor) (err error)
- func (catalog *Catalog) RemoveEntry(database *DBEntry) error
- func (catalog *Catalog) ReplayCmd(txncmd txnif.TxnCmd, dataFactory DataFactory, observer wal.ReplayObserver)
- func (catalog *Catalog) ReplayTableRows()
- func (catalog *Catalog) SimplePPString(level common.PPLevel) string
- func (catalog *Catalog) TxnGetDBEntryByID(id uint64, txn txnif.AsyncTxn) (*DBEntry, error)
- func (catalog *Catalog) TxnGetDBEntryByName(name string, txn txnif.AsyncTxn) (*DBEntry, error)
- type ColDef
- func (def *ColDef) GetName() string
- func (def *ColDef) GetType() types.Type
- func (def *ColDef) IsAutoIncrement() bool
- func (def *ColDef) IsClusterBy() bool
- func (def *ColDef) IsHidden() bool
- func (def *ColDef) IsPhyAddr() bool
- func (def *ColDef) IsPrimary() bool
- func (def *ColDef) IsRealPrimary() bool
- func (def *ColDef) IsSortKey() bool
- func (def *ColDef) Nullable() bool
- type ComposedFilter
- func (filter *ComposedFilter) AddBlockFilter(f func(*BlockEntry) bool)
- func (filter *ComposedFilter) AddCommitFilter(f func(*BaseEntryImpl[*MetadataMVCCNode]) bool)
- func (filter *ComposedFilter) FilteBlock(be *BlockEntry) bool
- func (filter *ComposedFilter) FilteCommit(be *BaseEntryImpl[*MetadataMVCCNode]) bool
- type DBEntry
- func NewDBEntry(catalog *Catalog, name, createSql, datTyp string, txn txnif.AsyncTxn) *DBEntry
- func NewDBEntryByTS(catalog *Catalog, name string, ts types.TS) *DBEntry
- func NewDBEntryWithID(catalog *Catalog, name string, createSql, datTyp string, id uint64, ...) *DBEntry
- func NewReplayDBEntry() *DBEntry
- func NewSystemDBEntry(catalog *Catalog) *DBEntry
- func (e *DBEntry) AddEntryLocked(table *TableEntry, txn txnif.TxnReader, skipDedup bool) (err error)
- func (e *DBEntry) AsCommonID() *common.ID
- func (e *DBEntry) CoarseTableCnt() int
- func (e *DBEntry) CreateTableEntry(schema *Schema, txn txnif.AsyncTxn, dataFactory TableDataFactory) (created *TableEntry, err error)
- func (e *DBEntry) CreateTableEntryWithTableId(schema *Schema, txn txnif.AsyncTxn, dataFactory TableDataFactory, ...) (created *TableEntry, err error)
- func (e *DBEntry) DropTableEntry(name string, txn txnif.AsyncTxn) (newEntry bool, deleted *TableEntry, err error)
- func (e *DBEntry) DropTableEntryByID(id uint64, txn txnif.AsyncTxn) (newEntry bool, deleted *TableEntry, err error)
- func (e *DBEntry) GetBlockEntryByID(id *common.ID) (blk *BlockEntry, err error)
- func (e *DBEntry) GetCatalog() *Catalog
- func (e *DBEntry) GetCreateAt() types.Timestamp
- func (e *DBEntry) GetCreateSql() string
- func (e *DBEntry) GetDatType() string
- func (e *DBEntry) GetFullName() string
- func (e *DBEntry) GetID() uint64
- func (e *DBEntry) GetItemNodeByIDLocked(id uint64) *common.GenericDLNode[*TableEntry]
- func (e *DBEntry) GetName() string
- func (e *DBEntry) GetRoleID() uint32
- func (e *DBEntry) GetTableEntryByID(id uint64) (table *TableEntry, err error)
- func (e *DBEntry) GetTableEntryByName(tenantID uint32, name string, txn txnif.TxnReader) (entry *TableEntry, err error)
- func (e *DBEntry) GetTenantID() uint32
- func (e *DBEntry) GetUserID() uint32
- func (e *DBEntry) Is1PC() bool
- func (e *DBEntry) IsActive() bool
- func (e *DBEntry) IsSubscription() bool
- func (e *DBEntry) IsSystemDB() bool
- func (e *DBEntry) Less(b *DBEntry) int
- func (e *DBEntry) MakeCommand(id uint32) (txnif.TxnCmd, error)
- func (e *DBEntry) MakeTableIt(reverse bool) *common.GenericSortedDListIt[*TableEntry]
- func (e *DBEntry) PPString(level common.PPLevel, depth int, prefix string) string
- func (e *DBEntry) PrepareRollback() (err error)
- func (e *DBEntry) PrettyNameIndex() string
- func (e *DBEntry) RecurLoop(processor Processor) (err error)
- func (e *DBEntry) RemoveEntry(table *TableEntry) (err error)
- func (e *DBEntry) RenameTableInTxn(old, new string, tid uint64, tenantID uint32, txn txnif.TxnReader, first bool) error
- func (e *DBEntry) RollbackRenameTable(fullname string, tid uint64)
- func (e *DBEntry) Set1PC()
- func (e *DBEntry) String() string
- func (e *DBEntry) StringLocked() string
- func (e *DBEntry) StringWithLevel(level common.PPLevel) string
- func (e *DBEntry) StringWithlevelLocked(level common.PPLevel) string
- func (e *DBEntry) TxnGetTableEntryByID(id uint64, txn txnif.AsyncTxn) (entry *TableEntry, err error)
- func (e *DBEntry) TxnGetTableEntryByName(name string, txn txnif.AsyncTxn) (entry *TableEntry, err error)
- type DBNode
- type DataFactory
- type EmptyMVCCNode
- func (e *EmptyMVCCNode) CloneAll() *EmptyMVCCNode
- func (e *EmptyMVCCNode) CloneData() *EmptyMVCCNode
- func (e *EmptyMVCCNode) ReadFromWithVersion(r io.Reader, ver uint16) (n int64, err error)
- func (e *EmptyMVCCNode) String() string
- func (e *EmptyMVCCNode) Update(vun *EmptyMVCCNode)
- func (e *EmptyMVCCNode) WriteTo(w io.Writer) (n int64, err error)
- type EntryCommand
- func (cmd *EntryCommand[T, N]) ApplyCommit()
- func (cmd *EntryCommand[T, N]) ApplyRollback()
- func (cmd *EntryCommand[T, N]) Desc() string
- func (cmd *EntryCommand[T, N]) GetCurrVersion() uint16
- func (cmd *EntryCommand[T, N]) GetID() *common.ID
- func (cmd *EntryCommand[T, N]) GetTs() types.TS
- func (cmd *EntryCommand[T, N]) GetType() uint16
- func (cmd *EntryCommand[T, N]) IDString() string
- func (cmd *EntryCommand[T, N]) MarshalBinary() (buf []byte, err error)
- func (cmd *EntryCommand[T, N]) ReadFrom(r io.Reader) (n int64, err error)
- func (cmd *EntryCommand[T, N]) SetReplayTxn(txn txnif.AsyncTxn)
- func (cmd *EntryCommand[T, N]) String() string
- func (cmd *EntryCommand[T, N]) UnmarshalBinary(buf []byte) (err error)
- func (cmd *EntryCommand[T, N]) VerboseString() string
- func (cmd *EntryCommand[T, N]) WriteTo(w io.Writer) (n int64, err error)
- type EntryMVCCNode
- func (un *EntryMVCCNode) ApplyCommit(ts types.TS) (err error)
- func (un *EntryMVCCNode) Clone() *EntryMVCCNode
- func (un *EntryMVCCNode) CloneData() *EntryMVCCNode
- func (un *EntryMVCCNode) Delete()
- func (un *EntryMVCCNode) GetCreatedAt() types.TS
- func (un *EntryMVCCNode) GetDeletedAt() types.TS
- func (un *EntryMVCCNode) HasDropCommitted() bool
- func (un *EntryMVCCNode) HasDropIntent() bool
- func (un *EntryMVCCNode) IsCreating() bool
- func (un *EntryMVCCNode) PrepareCommit() (err error)
- func (un *EntryMVCCNode) ReadFrom(r io.Reader) (n int64, err error)
- func (un *EntryMVCCNode) String() string
- func (un *EntryMVCCNode) WriteTo(w io.Writer) (n int64, err error)
- type EntryState
- type IDAlloctor
- func (alloc *IDAlloctor) CurrBlock() uint64
- func (alloc *IDAlloctor) CurrDB() uint64
- func (alloc *IDAlloctor) CurrSegment() uint64
- func (alloc *IDAlloctor) CurrTable() uint64
- func (alloc *IDAlloctor) IDStates() string
- func (alloc *IDAlloctor) Init(prevDb, prevTbl, prevSeg, prevBlk uint64)
- func (alloc *IDAlloctor) NextBlock() uint64
- func (alloc *IDAlloctor) NextDB() uint64
- func (alloc *IDAlloctor) NextSegment() uint64
- func (alloc *IDAlloctor) NextTable() uint64
- func (alloc *IDAlloctor) OnReplayBlockID(id uint64)
- func (alloc *IDAlloctor) OnReplayDBID(id uint64)
- func (alloc *IDAlloctor) OnReplaySegmentID(id uint64)
- func (alloc *IDAlloctor) OnReplayTableID(id uint64)
- type LoopProcessor
- func (p *LoopProcessor) OnBlock(block *BlockEntry) error
- func (p *LoopProcessor) OnDatabase(database *DBEntry) error
- func (p *LoopProcessor) OnPostDatabase(database *DBEntry) error
- func (p *LoopProcessor) OnPostSegment(segment *SegmentEntry) error
- func (p *LoopProcessor) OnPostTable(table *TableEntry) error
- func (p *LoopProcessor) OnSegment(segment *SegmentEntry) error
- func (p *LoopProcessor) OnTable(table *TableEntry) error
- type MVCCNode
- func (e *MVCCNode[T]) ApplyCommit() (err error)
- func (e *MVCCNode[T]) ApplyRollback() (err error)
- func (e *MVCCNode[T]) CloneAll() *MVCCNode[T]
- func (e *MVCCNode[T]) CloneData() *MVCCNode[T]
- func (e *MVCCNode[T]) IsNil() bool
- func (e *MVCCNode[T]) PrepareCommit() (err error)
- func (e *MVCCNode[T]) PrepareRollback() (err error)
- func (e *MVCCNode[T]) ReadFromWithVersion(r io.Reader, ver uint16) (n int64, err error)
- func (e *MVCCNode[T]) String() string
- func (e *MVCCNode[T]) Update(un *MVCCNode[T])
- func (e *MVCCNode[T]) WriteTo(w io.Writer) (n int64, err error)
- type MetadataMVCCNode
- func (e *MetadataMVCCNode) CloneAll() *MetadataMVCCNode
- func (e *MetadataMVCCNode) CloneData() *MetadataMVCCNode
- func (e *MetadataMVCCNode) ReadFromWithVersion(r io.Reader, ver uint16) (n int64, err error)
- func (e *MetadataMVCCNode) String() string
- func (e *MetadataMVCCNode) Update(un *MetadataMVCCNode)
- func (e *MetadataMVCCNode) WriteTo(w io.Writer) (n int64, err error)
- type Node
- type Processor
- type Schema
- func (s *Schema) AllNames() []string
- func (s *Schema) AllSeqnums() []uint16
- func (s *Schema) AllTypes() []types.Type
- func (s *Schema) AppendCol(name string, typ types.Type) error
- func (s *Schema) AppendColDef(def *ColDef) (err error)
- func (s *Schema) AppendColWithAttribute(attr engine.Attribute) error
- func (s *Schema) AppendFakePKCol() error
- func (s *Schema) AppendPKCol(name string, typ types.Type, idx int) error
- func (s *Schema) AppendSortColWithAttribute(attr engine.Attribute, sorIdx int, isPrimary bool) error
- func (s *Schema) AppendSortKey(name string, typ types.Type, idx int, isPrimary bool) error
- func (s *Schema) ApplyAlterTable(req *apipb.AlterTableReq) error
- func (s *Schema) Attrs() []string
- func (s *Schema) Clone() *Schema
- func (s *Schema) EstimateRowSize() (size int)
- func (s *Schema) Finalize(withoutPhyAddr bool) (err error)
- func (s *Schema) GetColIdx(attr string) int
- func (s *Schema) GetPrimaryKey() *ColDef
- func (s *Schema) GetSeqnum(attr string) uint16
- func (s *Schema) GetSingleSortKey() *ColDef
- func (s *Schema) GetSingleSortKeyIdx() int
- func (s *Schema) GetSingleSortKeyType() types.Type
- func (s *Schema) HasPK() bool
- func (s *Schema) HasSortKey() bool
- func (s *Schema) IsSameColumns(other *Schema) bool
- func (s *Schema) Marshal() (buf []byte, err error)
- func (s *Schema) MustGetExtraBytes() []byte
- func (s *Schema) MustRestoreExtra(data []byte)
- func (s *Schema) ReadFromBatch(bat *containers.Batch, offset int, targetTid uint64) (next int)
- func (s *Schema) ReadFromWithVersion(r io.Reader, ver uint16) (n int64, err error)
- func (s *Schema) String() string
- func (s *Schema) Types() []types.Type
- type SegStat
- func (s *SegStat) GetCompSize() int
- func (s *SegStat) GetLoaded() bool
- func (s *SegStat) GetOriginSize() int
- func (s *SegStat) GetRemainingRows() int
- func (s *SegStat) GetRows() int
- func (s *SegStat) GetSortKeyZonemap() index.ZM
- func (s *SegStat) SetOriginSize(size int)
- func (s *SegStat) SetRemainingRows(rows int)
- func (s *SegStat) SetRows(rows int)
- func (s *SegStat) String(composeSortKey bool) string
- type SegmentDataFactory
- type SegmentEntry
- func (entry *SegmentEntry) AddEntryLocked(block *BlockEntry)
- func (entry *SegmentEntry) AsCommonID() *common.ID
- func (entry *SegmentEntry) BlockCnt() int
- func (entry *SegmentEntry) CollectBlockEntries(commitFilter func(be *BaseEntryImpl[*MetadataMVCCNode]) bool, ...) []*BlockEntry
- func (entry *SegmentEntry) CreateBlock(txn txnif.AsyncTxn, state EntryState, dataFactory BlockDataFactory, ...) (created *BlockEntry, err error)
- func (entry *SegmentEntry) DropBlockEntry(id *objectio.Blockid, txn txnif.AsyncTxn) (deleted *BlockEntry, err error)
- func (entry *SegmentEntry) GetAppendableBlock() (blk *BlockEntry)
- func (entry *SegmentEntry) GetAppendableBlockCnt() int
- func (entry *SegmentEntry) GetBlockEntryByID(id *objectio.Blockid) (blk *BlockEntry, err error)
- func (entry *SegmentEntry) GetBlockEntryByIDLocked(id *objectio.Blockid) (blk *BlockEntry, err error)
- func (entry *SegmentEntry) GetCatalog() *Catalog
- func (entry *SegmentEntry) GetFirstBlkEntry() *BlockEntry
- func (entry *SegmentEntry) GetNextObjectIndex() uint16
- func (entry *SegmentEntry) GetNonAppendableBlockCnt() int
- func (entry *SegmentEntry) GetSegmentData() data.Segment
- func (entry *SegmentEntry) GetTable() *TableEntry
- func (entry *SegmentEntry) GetTerminationTS() (ts types.TS, terminated bool)
- func (entry *SegmentEntry) InitData(factory DataFactory)
- func (entry *SegmentEntry) Is1PC() bool
- func (entry *SegmentEntry) IsActive() bool
- func (entry *SegmentEntry) IsAppendable() bool
- func (entry *SegmentEntry) IsSorted() bool
- func (entry *SegmentEntry) IsSortedLocked() bool
- func (entry *SegmentEntry) LastAppendableBlock() (blk *BlockEntry)
- func (entry *SegmentEntry) Less(b *SegmentEntry) int
- func (entry *SegmentEntry) LoadObjectInfo() error
- func (entry *SegmentEntry) MakeBlockIt(reverse bool) *common.GenericSortedDListIt[*BlockEntry]
- func (entry *SegmentEntry) MakeCommand(id uint32) (cmd txnif.TxnCmd, err error)
- func (entry *SegmentEntry) PPString(level common.PPLevel, depth int, prefix string) string
- func (entry *SegmentEntry) PrepareRollback() (err error)
- func (entry *SegmentEntry) RemoveEntry(block *BlockEntry) (err error)
- func (entry *SegmentEntry) ReplayAddEntryLocked(block *BlockEntry)
- func (entry *SegmentEntry) Repr() string
- func (entry *SegmentEntry) Set1PC()
- func (entry *SegmentEntry) SetSorted()
- func (entry *SegmentEntry) String() string
- func (entry *SegmentEntry) StringLocked() string
- func (entry *SegmentEntry) StringWithLevel(level common.PPLevel) string
- func (entry *SegmentEntry) StringWithLevelLocked(level common.PPLevel) string
- func (entry *SegmentEntry) TreeMaxDropCommitEntry() BaseEntry
- type SegmentNode
- type SortKey
- type TableDataFactory
- type TableEntry
- func MockStaloneTableEntry(id uint64, schema *Schema) *TableEntry
- func NewReplayTableEntry() *TableEntry
- func NewSystemTableEntry(db *DBEntry, id uint64, schema *Schema) *TableEntry
- func NewTableEntry(db *DBEntry, schema *Schema, txnCtx txnif.AsyncTxn, ...) *TableEntry
- func NewTableEntryWithTableId(db *DBEntry, schema *Schema, txnCtx txnif.AsyncTxn, ...) *TableEntry
- func (entry *TableEntry) AddEntryLocked(segment *SegmentEntry)
- func (entry *TableEntry) AddRows(delta uint64) uint64
- func (entry *TableEntry) AlterTable(ctx context.Context, txn txnif.TxnReader, req *apipb.AlterTableReq) (isNewNode bool, newSchema *Schema, err error)
- func (entry *TableEntry) ApplyCommit() (err error)
- func (entry *TableEntry) AsCommonID() *common.ID
- func (entry *TableEntry) CreateSegment(txn txnif.AsyncTxn, state EntryState, dataFactory SegmentDataFactory, ...) (created *SegmentEntry, err error)
- func (entry *TableEntry) CreateWithTxnAndSchema(txn txnif.AsyncTxn, schema *Schema)
- func (entry *TableEntry) DropSegmentEntry(id *types.Segmentid, txn txnif.AsyncTxn) (deleted *SegmentEntry, err error)
- func (entry *TableEntry) FreezeAppend()
- func (entry *TableEntry) GetCatalog() *Catalog
- func (entry *TableEntry) GetColDefs() []*ColDef
- func (entry *TableEntry) GetDB() *DBEntry
- func (entry *TableEntry) GetFullName() string
- func (entry *TableEntry) GetID() uint64
- func (entry *TableEntry) GetLastestSchema() *Schema
- func (entry *TableEntry) GetRows() uint64
- func (entry *TableEntry) GetSegmentByID(id *types.Segmentid) (seg *SegmentEntry, err error)
- func (entry *TableEntry) GetTableData() data.Table
- func (entry *TableEntry) GetTerminationTS() (ts types.TS, terminated bool)
- func (entry *TableEntry) GetVersionSchema(ver uint32) *Schema
- func (entry *TableEntry) GetVisibilityAndName(txn txnif.TxnReader) (visible, dropped bool, name string)
- func (entry *TableEntry) GetVisibleSchema(txn txnif.TxnReader) *Schema
- func (entry *TableEntry) Is1PC() bool
- func (entry *TableEntry) IsActive() bool
- func (entry *TableEntry) IsVirtual() bool
- func (entry *TableEntry) LastAppendableSegmemt() (seg *SegmentEntry)
- func (entry *TableEntry) LastNonAppendableSegmemt() (seg *SegmentEntry)
- func (entry *TableEntry) Less(b *TableEntry) int
- func (entry *TableEntry) MakeCommand(id uint32) (cmd txnif.TxnCmd, err error)
- func (entry *TableEntry) MakeSegmentIt(reverse bool) *common.GenericSortedDListIt[*SegmentEntry]
- func (entry *TableEntry) ObjectStats(level common.PPLevel) (stat TableStat, w bytes.Buffer)
- func (entry *TableEntry) ObjectStatsString(level common.PPLevel) string
- func (entry *TableEntry) PPString(level common.PPLevel, depth int, prefix string) string
- func (entry *TableEntry) PrepareRollback() (err error)
- func (entry *TableEntry) RecurLoop(processor Processor) (err error)
- func (entry *TableEntry) RemoveEntry(segment *SegmentEntry) (err error)
- func (entry *TableEntry) RemoveRows(delta uint64) uint64
- func (entry *TableEntry) Set1PC()
- func (entry *TableEntry) String() string
- func (entry *TableEntry) StringLocked() string
- func (entry *TableEntry) StringLockedWithLevel(level common.PPLevel) string
- func (entry *TableEntry) StringWithLevel(level common.PPLevel) string
- type TableMVCCNode
- func (e *TableMVCCNode) CloneAll() *TableMVCCNode
- func (e *TableMVCCNode) CloneData() *TableMVCCNode
- func (e *TableMVCCNode) ReadFromWithVersion(r io.Reader, ver uint16) (n int64, err error)
- func (e *TableMVCCNode) String() string
- func (e *TableMVCCNode) Update(un *TableMVCCNode)
- func (e *TableMVCCNode) WriteTo(w io.Writer) (n int64, err error)
- type TableNode
- type TableStat
Constants ¶
const ( SnapshotAttr_TID = "table_id" SnapshotAttr_DBID = "db_id" SegmentAttr_ID = "id" SegmentAttr_CreateAt = "create_at" SegmentAttr_SegNode = "seg_node" SnapshotAttr_BlockMaxRow = "block_max_row" SnapshotAttr_SegmentMaxBlock = "segment_max_block" SnapshotAttr_SchemaExtra = "schema_extra" AccountIDDbNameTblName = "account_id_db_name_tbl_name" AccountIDDbName = "account_id_db_name" )
+--------+---------+----------+----------+------------+ | ID | Name | CreateAt | DeleteAt | CommitInfo | +--------+---------+----------+----------+------------+ |(uint64)|(varchar)| (uint64) | (uint64) | (varchar) | +--------+---------+----------+----------+------------+
const ( IOET_WALTxnCommand_Database uint16 = 3009 IOET_WALTxnCommand_Table uint16 = 3010 IOET_WALTxnCommand_Segment uint16 = 3011 IOET_WALTxnCommand_Block uint16 = 3012 IOET_WALTxnCommand_Database_V1 uint16 = 1 IOET_WALTxnCommand_Table_V1 uint16 = 1 IOET_WALTxnCommand_Table_V2 uint16 = 2 IOET_WALTxnCommand_Table_V3 uint16 = 3 IOET_WALTxnCommand_Segment_V1 uint16 = 1 IOET_WALTxnCommand_Block_V1 uint16 = 1 IOET_WALTxnCommand_Database_CurrVer = IOET_WALTxnCommand_Database_V1 IOET_WALTxnCommand_Table_CurrVer = IOET_WALTxnCommand_Table_V3 IOET_WALTxnCommand_Segment_CurrVer = IOET_WALTxnCommand_Segment_V1 IOET_WALTxnCommand_Block_CurrVer = IOET_WALTxnCommand_Block_V1 )
const ( PhyAddrColumnName = catalog.Row_ID PhyAddrColumnComment = "Physical address" SortKeyNamePrefx = "_SORT_" AttrRowID = PhyAddrColumnName AttrCommitTs = "commit_time" AttrAborted = "aborted" TenantSysID = uint32(0) )
const ( ModelSchemaName = "_ModelSchema" ModelAttrET = "ET" ModelAttrID = "ID" ModelAttrName = "NAME" ModelAttrTS = "TS" ModelAttrOpT = "OPT" ModelAttrLogIdx = "LOGIDX" ModelAttrInfo = "INFO" ModelAttrParentID = "PARENTID" )
const (
AccessInfoSize int64 = int64(unsafe.Sizeof(accessInfo{}))
)
const (
EntryMVCCNodeSize int = int(unsafe.Sizeof(EntryMVCCNode{}))
)
Variables ¶
var SystemBlock_Columns_ID types.Blockid
var SystemBlock_DB_ID types.Blockid
var SystemBlock_Table_ID types.Blockid
var SystemSegment_Columns_ID types.Uuid
var SystemSegment_DB_ID types.Uuid
var SystemSegment_Table_ID types.Uuid
Functions ¶
func ActiveWithNoTxnFilter ¶
func ActiveWithNoTxnFilter(be *BaseEntryImpl[*MetadataMVCCNode]) bool
func AppendableBlkFilter ¶
func AppendableBlkFilter(be *BlockEntry) bool
func CompareBaseNode ¶ added in v0.8.0
func CompareUint64 ¶
func EncodeAccessInfo ¶ added in v0.8.0
func EncodeAccessInfo(ai *accessInfo) []byte
func EncodeBlockNode ¶ added in v0.8.0
func EncodeEntryMVCCNode ¶ added in v0.8.0
func EncodeEntryMVCCNode(node *EntryMVCCNode) []byte
func GetAttrIdx ¶
func IsFakePkName ¶ added in v0.8.0
func MockTxnFactory ¶
func MockTxnFactory(catalog *Catalog) txnbase.TxnFactory
func MockTxnStoreFactory ¶
func MockTxnStoreFactory(catalog *Catalog) txnbase.TxnStoreFactory
func NewEmptyMVCCNodeFactory ¶ added in v0.8.0
func NonAppendableBlkFilter ¶
func NonAppendableBlkFilter(be *BlockEntry) bool
Types ¶
type BaseEntryImpl ¶ added in v0.8.0
type BaseEntryImpl[T BaseNode[T]] struct { //chain of MetadataMVCCNode *txnbase.MVCCChain[*MVCCNode[T]] }
func NewBaseEntry ¶ added in v0.8.0
func NewBaseEntry[T BaseNode[T]](factory func() T) *BaseEntryImpl[T]
func NewReplayBaseEntry ¶
func NewReplayBaseEntry[T BaseNode[T]](factory func() T) *BaseEntryImpl[T]
func (*BaseEntryImpl[T]) CloneCommittedInRange ¶ added in v0.8.0
func (be *BaseEntryImpl[T]) CloneCommittedInRange(start, end types.TS) BaseEntry
func (*BaseEntryImpl[T]) ConflictCheck ¶ added in v0.8.0
func (be *BaseEntryImpl[T]) ConflictCheck(txn txnif.TxnReader) (err error)
func (*BaseEntryImpl[T]) CreateWithTS ¶ added in v0.8.0
func (be *BaseEntryImpl[T]) CreateWithTS(ts types.TS, baseNode T)
func (*BaseEntryImpl[T]) CreateWithTxn ¶ added in v0.8.0
func (be *BaseEntryImpl[T]) CreateWithTxn(txn txnif.AsyncTxn, baseNode T)
func (*BaseEntryImpl[T]) DeleteAfter ¶ added in v0.8.0
func (be *BaseEntryImpl[T]) DeleteAfter(ts types.TS) bool
func (*BaseEntryImpl[T]) DeleteBefore ¶ added in v0.8.0
func (be *BaseEntryImpl[T]) DeleteBefore(ts types.TS) bool
func (*BaseEntryImpl[T]) DeleteLocked ¶ added in v0.8.0
func (be *BaseEntryImpl[T]) DeleteLocked(txn txnif.TxnReader) (isNewNode bool, err error)
func (*BaseEntryImpl[T]) DropEntryLocked ¶ added in v0.8.0
func (be *BaseEntryImpl[T]) DropEntryLocked(txn txnif.TxnReader) (isNewNode bool, err error)
func (*BaseEntryImpl[T]) GetCreatedAtLocked ¶ added in v1.0.0
func (be *BaseEntryImpl[T]) GetCreatedAtLocked() types.TS
func (*BaseEntryImpl[T]) GetDeleteAt ¶ added in v0.8.0
func (be *BaseEntryImpl[T]) GetDeleteAt() types.TS
func (*BaseEntryImpl[T]) GetVisibility ¶ added in v0.8.0
func (be *BaseEntryImpl[T]) GetVisibility(txn txnif.TxnReader) (visible, dropped bool)
func (*BaseEntryImpl[T]) GetVisibilityLocked ¶ added in v0.8.0
func (be *BaseEntryImpl[T]) GetVisibilityLocked(txn txnif.TxnReader) (visible, dropped bool)
func (*BaseEntryImpl[T]) HasDropCommitted ¶ added in v0.8.0
func (be *BaseEntryImpl[T]) HasDropCommitted() bool
func (*BaseEntryImpl[T]) HasDropCommittedLocked ¶ added in v0.8.0
func (be *BaseEntryImpl[T]) HasDropCommittedLocked() bool
func (*BaseEntryImpl[T]) NeedWaitCommitting ¶ added in v0.8.0
func (*BaseEntryImpl[T]) PrepareAdd ¶ added in v0.8.0
func (be *BaseEntryImpl[T]) PrepareAdd(txn txnif.TxnReader) (err error)
func (*BaseEntryImpl[T]) String ¶ added in v0.8.0
func (be *BaseEntryImpl[T]) String() string
func (*BaseEntryImpl[T]) StringLocked ¶ added in v0.8.0
func (be *BaseEntryImpl[T]) StringLocked() string
func (*BaseEntryImpl[T]) TryGetTerminatedTS ¶ added in v0.8.0
func (be *BaseEntryImpl[T]) TryGetTerminatedTS(waitIfcommitting bool) (terminated bool, TS types.TS)
type BlockDataFactory ¶
type BlockDataFactory = func(meta *BlockEntry) data.Block
type BlockEntry ¶
type BlockEntry struct { *BaseEntryImpl[*MetadataMVCCNode] *BlockNode ID types.Blockid // contains filtered or unexported fields }
func NewBlockEntry ¶
func NewBlockEntry( segment *SegmentEntry, id *objectio.Blockid, txn txnif.AsyncTxn, state EntryState, dataFactory BlockDataFactory, ) *BlockEntry
func NewBlockEntryWithMeta ¶ added in v0.7.0
func NewBlockEntryWithMeta( segment *SegmentEntry, id *objectio.Blockid, txn txnif.AsyncTxn, state EntryState, dataFactory BlockDataFactory, metaLoc objectio.Location, deltaLoc objectio.Location) *BlockEntry
func NewReplayBlockEntry ¶
func NewReplayBlockEntry() *BlockEntry
func NewStandaloneBlock ¶
func NewStandaloneBlock(segment *SegmentEntry, id *objectio.Blockid, ts types.TS) *BlockEntry
func NewStandaloneBlockWithLoc ¶ added in v0.7.0
func NewStandaloneBlockWithLoc( segment *SegmentEntry, id *objectio.Blockid, ts types.TS, metaLoc objectio.Location, delLoc objectio.Location) *BlockEntry
func NewSysBlockEntry ¶
func NewSysBlockEntry(segment *SegmentEntry, id types.Blockid) *BlockEntry
func (*BlockEntry) AsCommonID ¶
func (entry *BlockEntry) AsCommonID() *common.ID
func (*BlockEntry) BuildDeleteObjectName ¶ added in v0.8.0
func (entry *BlockEntry) BuildDeleteObjectName() objectio.ObjectName
func (*BlockEntry) CreateWithLoc ¶ added in v0.8.0
func (*BlockEntry) CreateWithTxnAndMeta ¶ added in v0.8.0
func (*BlockEntry) FastGetMetaLoc ¶ added in v0.8.0
func (entry *BlockEntry) FastGetMetaLoc() objectio.Location
func (*BlockEntry) GetBlockData ¶
func (entry *BlockEntry) GetBlockData() data.Block
func (*BlockEntry) GetCatalog ¶
func (entry *BlockEntry) GetCatalog() *Catalog
func (*BlockEntry) GetDeltaLoc ¶ added in v0.8.0
func (entry *BlockEntry) GetDeltaLoc() objectio.Location
func (*BlockEntry) GetDeltaLocAndCommitTS ¶ added in v1.0.0
func (entry *BlockEntry) GetDeltaLocAndCommitTS() (objectio.Location, types.TS)
func (*BlockEntry) GetDeltaPersistedTS ¶ added in v0.8.0
func (entry *BlockEntry) GetDeltaPersistedTS() types.TS
func (*BlockEntry) GetDeltaPersistedTSByTxn ¶ added in v0.8.0
func (entry *BlockEntry) GetDeltaPersistedTSByTxn(txn txnif.TxnReader) types.TS
func (*BlockEntry) GetMetaLoc ¶ added in v0.8.0
func (entry *BlockEntry) GetMetaLoc() objectio.Location
func (*BlockEntry) GetPKZoneMap ¶ added in v0.8.0
func (entry *BlockEntry) GetPKZoneMap( ctx context.Context, fs fileservice.FileService, ) (zm *index.ZM, err error)
func (*BlockEntry) GetSchema ¶
func (entry *BlockEntry) GetSchema() *Schema
func (*BlockEntry) GetSegment ¶
func (entry *BlockEntry) GetSegment() *SegmentEntry
func (*BlockEntry) GetTerminationTS ¶
func (entry *BlockEntry) GetTerminationTS() (ts types.TS, terminated bool)
GetTerminationTS is coarse API: no consistency check
func (*BlockEntry) GetVisibleDeltaLoc ¶ added in v0.8.0
func (entry *BlockEntry) GetVisibleDeltaLoc(txn txnif.TxnReader) objectio.Location
func (*BlockEntry) GetVisibleMetaLoc ¶ added in v0.8.0
func (entry *BlockEntry) GetVisibleMetaLoc(txn txnif.TxnReader) objectio.Location
func (*BlockEntry) HasPersistedData ¶ added in v0.8.0
func (entry *BlockEntry) HasPersistedData() bool
func (*BlockEntry) HasPersistedDeltaData ¶ added in v0.8.0
func (entry *BlockEntry) HasPersistedDeltaData() bool
func (*BlockEntry) InitData ¶
func (entry *BlockEntry) InitData(factory DataFactory)
func (*BlockEntry) Is1PC ¶ added in v0.6.0
func (entry *BlockEntry) Is1PC() bool
func (*BlockEntry) IsActive ¶
func (entry *BlockEntry) IsActive() bool
IsActive is coarse API: no consistency check
func (*BlockEntry) IsAppendable ¶
func (entry *BlockEntry) IsAppendable() bool
func (*BlockEntry) Less ¶ added in v0.8.0
func (entry *BlockEntry) Less(b *BlockEntry) int
func (*BlockEntry) MakeCommand ¶
func (entry *BlockEntry) MakeCommand(id uint32) (cmd txnif.TxnCmd, err error)
func (*BlockEntry) MakeKey ¶
func (entry *BlockEntry) MakeKey() []byte
func (*BlockEntry) PrepareCompact ¶ added in v0.6.0
func (entry *BlockEntry) PrepareCompact() bool
PrepareCompact is performance insensitive a block can be compacted: 1. no uncommited node 2. at least one committed node 3. not compacted
func (*BlockEntry) PrepareRollback ¶
func (entry *BlockEntry) PrepareRollback() (err error)
func (*BlockEntry) Repr ¶
func (entry *BlockEntry) Repr() string
func (*BlockEntry) Set1PC ¶ added in v0.6.0
func (entry *BlockEntry) Set1PC()
func (*BlockEntry) String ¶
func (entry *BlockEntry) String() string
func (*BlockEntry) StringLocked ¶
func (entry *BlockEntry) StringLocked() string
func (*BlockEntry) StringWithLevel ¶ added in v0.6.0
func (entry *BlockEntry) StringWithLevel(level common.PPLevel) string
func (*BlockEntry) StringWithLevelLocked ¶ added in v0.6.0
func (entry *BlockEntry) StringWithLevelLocked(level common.PPLevel) string
func (*BlockEntry) UpdateDeltaLoc ¶ added in v0.8.0
func (*BlockEntry) UpdateMetaLoc ¶ added in v0.8.0
type BlockNode ¶ added in v0.8.0
type BlockNode struct {
// contains filtered or unexported fields
}
type Catalog ¶
type Catalog struct { *IDAlloctor *sync.RWMutex // contains filtered or unexported fields }
func MockCatalog ¶
func MockCatalog() *Catalog
func NewEmptyCatalog ¶ added in v0.6.0
func NewEmptyCatalog() *Catalog
func OpenCatalog ¶
func (*Catalog) AddEntryLocked ¶
func (*Catalog) CoarseDBCnt ¶
func (*Catalog) CreateDBEntry ¶
func (*Catalog) CreateDBEntryByTS ¶ added in v0.6.0
func (*Catalog) CreateDBEntryWithID ¶ added in v0.6.0
func (*Catalog) DropDBEntry ¶
func (*Catalog) DropDBEntryByID ¶ added in v0.6.0
func (*Catalog) GetDBEntryByName ¶ added in v0.6.0
func (*Catalog) GetDatabaseByID ¶
func (*Catalog) GetItemNodeByIDLocked ¶ added in v0.6.0
func (catalog *Catalog) GetItemNodeByIDLocked(id uint64) *common.GenericDLNode[*DBEntry]
func (*Catalog) InitSystemDB ¶
func (catalog *Catalog) InitSystemDB()
func (*Catalog) MakeDBIt ¶
func (catalog *Catalog) MakeDBIt(reverse bool) *common.GenericSortedDListIt[*DBEntry]
func (*Catalog) OnReplayBlockBatch ¶ added in v0.6.0
func (catalog *Catalog) OnReplayBlockBatch(ins, insTxn, del, delTxn *containers.Batch, dataFactory DataFactory)
func (*Catalog) OnReplayDatabaseBatch ¶ added in v0.6.0
func (catalog *Catalog) OnReplayDatabaseBatch(ins, insTxn, del, delTxn *containers.Batch)
func (*Catalog) OnReplaySegmentBatch ¶ added in v0.6.0
func (catalog *Catalog) OnReplaySegmentBatch(ins, insTxn, del, delTxn *containers.Batch, dataFactory DataFactory)
func (*Catalog) OnReplayTableBatch ¶ added in v0.6.0
func (catalog *Catalog) OnReplayTableBatch(ins, insTxn, insCol, del, delTxn *containers.Batch, dataFactory DataFactory)
func (*Catalog) RemoveEntry ¶
func (*Catalog) ReplayCmd ¶
func (catalog *Catalog) ReplayCmd( txncmd txnif.TxnCmd, dataFactory DataFactory, observer wal.ReplayObserver)
func (*Catalog) ReplayTableRows ¶
func (catalog *Catalog) ReplayTableRows()
func (*Catalog) SimplePPString ¶
func (*Catalog) TxnGetDBEntryByID ¶ added in v0.6.0
type ColDef ¶
type ColDef struct { Name string Idx int // indicates its position in all coldefs SeqNum uint16 // Type types.Type Hidden bool // Hidden Column is generated by compute layer, keep hidden from user PhyAddr bool // PhyAddr Column is generated by tae as rowid NullAbility bool AutoIncrement bool Primary bool SortIdx int8 // indicates its position in all sort keys SortKey bool Comment string ClusterBy bool FakePK bool // TODO: use column.flag instead of column.fakepk Default []byte OnUpdate []byte EnumValues string }
func ColDefFromAttribute ¶ added in v0.7.0
make a basic coldef without sortKey info
func (*ColDef) IsAutoIncrement ¶ added in v0.6.0
func (*ColDef) IsClusterBy ¶ added in v0.7.0
func (*ColDef) IsRealPrimary ¶ added in v0.8.0
type ComposedFilter ¶
type ComposedFilter struct { CommitFilters []func(*BaseEntryImpl[*MetadataMVCCNode]) bool BlockFilters []func(*BlockEntry) bool }
func NewComposedFilter ¶
func NewComposedFilter() *ComposedFilter
func (*ComposedFilter) AddBlockFilter ¶
func (filter *ComposedFilter) AddBlockFilter(f func(*BlockEntry) bool)
func (*ComposedFilter) AddCommitFilter ¶
func (filter *ComposedFilter) AddCommitFilter(f func(*BaseEntryImpl[*MetadataMVCCNode]) bool)
func (*ComposedFilter) FilteBlock ¶
func (filter *ComposedFilter) FilteBlock(be *BlockEntry) bool
func (*ComposedFilter) FilteCommit ¶
func (filter *ComposedFilter) FilteCommit(be *BaseEntryImpl[*MetadataMVCCNode]) bool
type DBEntry ¶
type DBEntry struct { ID uint64 *BaseEntryImpl[*EmptyMVCCNode] *DBNode // contains filtered or unexported fields }
func NewDBEntry ¶
func NewDBEntryByTS ¶ added in v0.6.0
func NewDBEntryWithID ¶ added in v0.6.0
func NewReplayDBEntry ¶
func NewReplayDBEntry() *DBEntry
func NewSystemDBEntry ¶
func (*DBEntry) AddEntryLocked ¶
func (e *DBEntry) AddEntryLocked(table *TableEntry, txn txnif.TxnReader, skipDedup bool) (err error)
Catalog entry is created in following steps: 1. Locate the record. Creating always gets the latest DBEntry. 2.1 If there doesn't exist a DBEntry, add new entry and return. 2.2 If there exists a DBEntry: 2.2.1 Check conflication.
- Wait for the related txn if need.
- w-w conflict when: there's an active txn; or he CommitTS of the latest related txn is larger than StartTS of write txn
2.2.2 Check duplicate/not found. If the entry hasn't been dropped, return ErrDuplicate.
func (*DBEntry) AsCommonID ¶ added in v0.8.0
func (*DBEntry) CoarseTableCnt ¶
func (*DBEntry) CreateTableEntry ¶
func (e *DBEntry) CreateTableEntry(schema *Schema, txn txnif.AsyncTxn, dataFactory TableDataFactory) (created *TableEntry, err error)
func (*DBEntry) CreateTableEntryWithTableId ¶ added in v0.6.0
func (e *DBEntry) CreateTableEntryWithTableId(schema *Schema, txn txnif.AsyncTxn, dataFactory TableDataFactory, tableId uint64) (created *TableEntry, err error)
func (*DBEntry) DropTableEntry ¶
func (e *DBEntry) DropTableEntry(name string, txn txnif.AsyncTxn) (newEntry bool, deleted *TableEntry, err error)
Catalog entry is dropped in following steps: 1. Locate the record by timestamp 2. Check conflication. 2.1 Wait for the related txn if need. 2.2 w-w conflict when 1. there's an active txn; or
- the CommitTS of the latest related txn is larger than StartTS of write txn
3. Check duplicate/not found. If the entry has already been dropped, return ErrNotFound.
func (*DBEntry) DropTableEntryByID ¶ added in v0.6.0
func (*DBEntry) GetBlockEntryByID ¶
func (e *DBEntry) GetBlockEntryByID(id *common.ID) (blk *BlockEntry, err error)
func (*DBEntry) GetCatalog ¶
func (*DBEntry) GetCreateAt ¶ added in v0.6.0
func (*DBEntry) GetCreateSql ¶ added in v0.6.0
func (*DBEntry) GetDatType ¶ added in v0.8.0
func (*DBEntry) GetFullName ¶ added in v0.6.0
func (*DBEntry) GetItemNodeByIDLocked ¶ added in v0.6.0
func (e *DBEntry) GetItemNodeByIDLocked(id uint64) *common.GenericDLNode[*TableEntry]
func (*DBEntry) GetTableEntryByID ¶
func (e *DBEntry) GetTableEntryByID(id uint64) (table *TableEntry, err error)
func (*DBEntry) GetTableEntryByName ¶ added in v0.6.0
func (*DBEntry) GetTenantID ¶ added in v0.6.0
func (*DBEntry) IsSubscription ¶ added in v0.8.0
func (*DBEntry) IsSystemDB ¶
func (*DBEntry) MakeTableIt ¶
func (e *DBEntry) MakeTableIt(reverse bool) *common.GenericSortedDListIt[*TableEntry]
func (*DBEntry) PrepareRollback ¶
func (*DBEntry) PrettyNameIndex ¶ added in v0.8.0
For test only
func (*DBEntry) RemoveEntry ¶
func (e *DBEntry) RemoveEntry(table *TableEntry) (err error)
func (*DBEntry) RenameTableInTxn ¶ added in v0.8.0
func (*DBEntry) RollbackRenameTable ¶ added in v0.8.0
func (*DBEntry) StringLocked ¶
func (*DBEntry) StringWithLevel ¶ added in v0.6.0
func (*DBEntry) StringWithlevelLocked ¶ added in v0.6.0
func (*DBEntry) TxnGetTableEntryByID ¶ added in v0.6.0
func (*DBEntry) TxnGetTableEntryByName ¶ added in v0.6.0
type DBNode ¶ added in v0.8.0
type DBNode struct {
// contains filtered or unexported fields
}
type DataFactory ¶
type DataFactory interface { MakeTableFactory() TableDataFactory MakeSegmentFactory() SegmentDataFactory MakeBlockFactory() BlockDataFactory }
type EmptyMVCCNode ¶ added in v0.8.0
type EmptyMVCCNode struct{}
func NewEmptyEmptyMVCCNode ¶ added in v0.8.0
func NewEmptyEmptyMVCCNode() *EmptyMVCCNode
func (*EmptyMVCCNode) CloneAll ¶ added in v0.8.0
func (e *EmptyMVCCNode) CloneAll() *EmptyMVCCNode
func (*EmptyMVCCNode) CloneData ¶ added in v0.8.0
func (e *EmptyMVCCNode) CloneData() *EmptyMVCCNode
func (*EmptyMVCCNode) ReadFromWithVersion ¶ added in v1.0.0
func (*EmptyMVCCNode) String ¶ added in v0.8.0
func (e *EmptyMVCCNode) String() string
func (*EmptyMVCCNode) Update ¶ added in v0.8.0
func (e *EmptyMVCCNode) Update(vun *EmptyMVCCNode)
for create drop in one txn
type EntryCommand ¶
type EntryCommand[T BaseNode[T], N Node] struct { *txnbase.BaseCustomizedCmd ID *common.ID // contains filtered or unexported fields }
func (*EntryCommand[T, N]) ApplyCommit ¶ added in v0.6.0
func (cmd *EntryCommand[T, N]) ApplyCommit()
func (*EntryCommand[T, N]) ApplyRollback ¶ added in v0.6.0
func (cmd *EntryCommand[T, N]) ApplyRollback()
func (*EntryCommand[T, N]) Desc ¶
func (cmd *EntryCommand[T, N]) Desc() string
func (*EntryCommand[T, N]) GetCurrVersion ¶ added in v0.8.0
func (cmd *EntryCommand[T, N]) GetCurrVersion() uint16
func (*EntryCommand[T, N]) GetID ¶
func (cmd *EntryCommand[T, N]) GetID() *common.ID
func (*EntryCommand[T, N]) GetTs ¶
func (cmd *EntryCommand[T, N]) GetTs() types.TS
func (*EntryCommand[T, N]) GetType ¶
func (cmd *EntryCommand[T, N]) GetType() uint16
func (*EntryCommand[T, N]) IDString ¶
func (cmd *EntryCommand[T, N]) IDString() string
func (*EntryCommand[T, N]) MarshalBinary ¶ added in v0.8.0
func (cmd *EntryCommand[T, N]) MarshalBinary() (buf []byte, err error)
func (*EntryCommand[T, N]) ReadFrom ¶
func (cmd *EntryCommand[T, N]) ReadFrom(r io.Reader) (n int64, err error)
func (*EntryCommand[T, N]) SetReplayTxn ¶ added in v0.6.0
func (cmd *EntryCommand[T, N]) SetReplayTxn(txn txnif.AsyncTxn)
func (*EntryCommand[T, N]) String ¶
func (cmd *EntryCommand[T, N]) String() string
func (*EntryCommand[T, N]) UnmarshalBinary ¶ added in v0.8.0
func (cmd *EntryCommand[T, N]) UnmarshalBinary(buf []byte) (err error)
func (*EntryCommand[T, N]) VerboseString ¶
func (cmd *EntryCommand[T, N]) VerboseString() string
type EntryMVCCNode ¶ added in v0.6.0
func DecodeEntryMVCCNode ¶ added in v0.8.0
func DecodeEntryMVCCNode(v []byte) *EntryMVCCNode
func (*EntryMVCCNode) ApplyCommit ¶ added in v0.6.0
func (un *EntryMVCCNode) ApplyCommit(ts types.TS) (err error)
func (*EntryMVCCNode) Clone ¶ added in v0.6.0
func (un *EntryMVCCNode) Clone() *EntryMVCCNode
func (*EntryMVCCNode) CloneData ¶ added in v0.6.0
func (un *EntryMVCCNode) CloneData() *EntryMVCCNode
func (*EntryMVCCNode) Delete ¶ added in v0.6.0
func (un *EntryMVCCNode) Delete()
func (*EntryMVCCNode) GetCreatedAt ¶ added in v0.6.0
func (un *EntryMVCCNode) GetCreatedAt() types.TS
func (*EntryMVCCNode) GetDeletedAt ¶ added in v0.6.0
func (un *EntryMVCCNode) GetDeletedAt() types.TS
func (*EntryMVCCNode) HasDropCommitted ¶ added in v0.6.0
func (un *EntryMVCCNode) HasDropCommitted() bool
Dropped committed
func (*EntryMVCCNode) HasDropIntent ¶ added in v0.6.0
func (un *EntryMVCCNode) HasDropIntent() bool
Dropped committed or uncommitted
func (*EntryMVCCNode) IsCreating ¶ added in v0.6.0
func (un *EntryMVCCNode) IsCreating() bool
func (*EntryMVCCNode) PrepareCommit ¶ added in v0.6.0
func (un *EntryMVCCNode) PrepareCommit() (err error)
func (*EntryMVCCNode) ReadFrom ¶ added in v0.6.0
func (un *EntryMVCCNode) ReadFrom(r io.Reader) (n int64, err error)
func (*EntryMVCCNode) String ¶ added in v0.6.0
func (un *EntryMVCCNode) String() string
type EntryState ¶
type EntryState int8
const ( ES_Appendable EntryState = iota ES_NotAppendable ES_Frozen )
func (EntryState) Repr ¶
func (es EntryState) Repr() string
type IDAlloctor ¶
type IDAlloctor struct {
// contains filtered or unexported fields
}
func NewIDAllocator ¶
func NewIDAllocator() *IDAlloctor
func (*IDAlloctor) CurrBlock ¶
func (alloc *IDAlloctor) CurrBlock() uint64
func (*IDAlloctor) CurrDB ¶
func (alloc *IDAlloctor) CurrDB() uint64
func (*IDAlloctor) CurrSegment ¶
func (alloc *IDAlloctor) CurrSegment() uint64
func (*IDAlloctor) CurrTable ¶
func (alloc *IDAlloctor) CurrTable() uint64
func (*IDAlloctor) IDStates ¶
func (alloc *IDAlloctor) IDStates() string
func (*IDAlloctor) Init ¶
func (alloc *IDAlloctor) Init(prevDb, prevTbl, prevSeg, prevBlk uint64)
func (*IDAlloctor) NextBlock ¶
func (alloc *IDAlloctor) NextBlock() uint64
func (*IDAlloctor) NextDB ¶
func (alloc *IDAlloctor) NextDB() uint64
func (*IDAlloctor) NextSegment ¶
func (alloc *IDAlloctor) NextSegment() uint64
func (*IDAlloctor) NextTable ¶
func (alloc *IDAlloctor) NextTable() uint64
func (*IDAlloctor) OnReplayBlockID ¶
func (alloc *IDAlloctor) OnReplayBlockID(id uint64)
func (*IDAlloctor) OnReplayDBID ¶
func (alloc *IDAlloctor) OnReplayDBID(id uint64)
func (*IDAlloctor) OnReplaySegmentID ¶
func (alloc *IDAlloctor) OnReplaySegmentID(id uint64)
func (*IDAlloctor) OnReplayTableID ¶
func (alloc *IDAlloctor) OnReplayTableID(id uint64)
type LoopProcessor ¶
type LoopProcessor struct { DatabaseFn func(*DBEntry) error TableFn func(*TableEntry) error SegmentFn func(*SegmentEntry) error BlockFn func(*BlockEntry) error PostDatabaseFn func(*DBEntry) error PostTableFn func(*TableEntry) error PostSegmentFn func(*SegmentEntry) error }
func (*LoopProcessor) OnBlock ¶
func (p *LoopProcessor) OnBlock(block *BlockEntry) error
func (*LoopProcessor) OnDatabase ¶
func (p *LoopProcessor) OnDatabase(database *DBEntry) error
func (*LoopProcessor) OnPostDatabase ¶ added in v1.0.0
func (p *LoopProcessor) OnPostDatabase(database *DBEntry) error
func (*LoopProcessor) OnPostSegment ¶
func (p *LoopProcessor) OnPostSegment(segment *SegmentEntry) error
func (*LoopProcessor) OnPostTable ¶ added in v1.0.0
func (p *LoopProcessor) OnPostTable(table *TableEntry) error
func (*LoopProcessor) OnSegment ¶
func (p *LoopProcessor) OnSegment(segment *SegmentEntry) error
func (*LoopProcessor) OnTable ¶
func (p *LoopProcessor) OnTable(table *TableEntry) error
type MVCCNode ¶ added in v0.8.0
type MVCCNode[T BaseNode[T]] struct { *EntryMVCCNode *txnbase.TxnMVCCNode BaseNode T }
func (*MVCCNode[T]) ApplyCommit ¶ added in v0.8.0
func (*MVCCNode[T]) ApplyRollback ¶ added in v0.8.0
func (*MVCCNode[T]) PrepareCommit ¶ added in v0.8.0
func (*MVCCNode[T]) PrepareRollback ¶ added in v0.8.0
func (*MVCCNode[T]) ReadFromWithVersion ¶ added in v1.0.0
type MetadataMVCCNode ¶ added in v0.6.0
func NewEmptyMetadataMVCCNode ¶ added in v0.6.0
func NewEmptyMetadataMVCCNode() *MetadataMVCCNode
func (*MetadataMVCCNode) CloneAll ¶ added in v0.6.0
func (e *MetadataMVCCNode) CloneAll() *MetadataMVCCNode
func (*MetadataMVCCNode) CloneData ¶ added in v0.6.0
func (e *MetadataMVCCNode) CloneData() *MetadataMVCCNode
func (*MetadataMVCCNode) ReadFromWithVersion ¶ added in v1.0.0
func (*MetadataMVCCNode) String ¶ added in v0.6.0
func (e *MetadataMVCCNode) String() string
func (*MetadataMVCCNode) Update ¶ added in v0.6.0
func (e *MetadataMVCCNode) Update(un *MetadataMVCCNode)
for create drop in one txn
type Processor ¶
type Processor interface { OnDatabase(database *DBEntry) error OnPostDatabase(database *DBEntry) error OnTable(table *TableEntry) error OnPostTable(table *TableEntry) error OnPostSegment(segment *SegmentEntry) error OnSegment(segment *SegmentEntry) error OnBlock(block *BlockEntry) error }
XXX this API is broken. In case of inplementing a cursor like interface we cannot use error. moerr is a very heavy mechanism.
Return a int code.
type Schema ¶
type Schema struct { Version uint32 CatalogVersion uint32 AcInfo accessInfo Name string ColDefs []*ColDef Comment string Partitioned int8 // 1: the table has partitions ; 0: no partition Partition string // the info about partitions when the table has partitions Relkind string Createsql string View string Constraint []byte // do not send to cn BlockMaxRows uint32 SegmentMaxBlocks uint16 Extra *apipb.SchemaExtra // do not write down, reconstruct them when reading NameMap map[string]int // name -> logical idx SeqnumMap map[uint16]int // seqnum -> logical idx SortKey *SortKey PhyAddrKey *ColDef }
var SystemColumnSchema *Schema
var SystemColumnSchema_V1 *Schema
var SystemDBSchema *Schema
var SystemTableSchema *Schema
var SystemTableSchema_V1 *Schema
func MockSchema ¶
func MockSchemaAll ¶
MockSchemaAll if char/varchar is needed, colCnt = 14, otherwise colCnt = 12 pkIdx == -1 means no pk defined
func NewEmptySchema ¶
func (*Schema) AllSeqnums ¶ added in v0.8.0
func (*Schema) AppendColDef ¶
func (*Schema) AppendColWithAttribute ¶ added in v0.6.0
func (*Schema) AppendFakePKCol ¶ added in v0.8.0
func (*Schema) AppendPKCol ¶
func (*Schema) AppendSortColWithAttribute ¶ added in v0.7.0
func (s *Schema) AppendSortColWithAttribute(attr engine.Attribute, sorIdx int, isPrimary bool) error
non-cn doesn't set IsPrimary in attr, so isPrimary is used explicitly here
func (*Schema) AppendSortKey ¶
func (*Schema) ApplyAlterTable ¶ added in v0.8.0
func (s *Schema) ApplyAlterTable(req *apipb.AlterTableReq) error
ApplyAlterTable modify the schema in place. Unless you know what you are doing, it is recommended to close schema first and then apply alter table.
func (*Schema) EstimateRowSize ¶ added in v1.0.0
func (*Schema) Finalize ¶
Finalize runs various checks and create shortcuts to phyaddr and sortkey Note: NameMap is already set before calling Finalize
func (*Schema) GetColIdx ¶
GetColIdx returns column index for the given column name if found, otherwise returns -1.
func (*Schema) GetPrimaryKey ¶ added in v0.8.0
GetPrimaryKey gets the primary key, including fake primary key.
func (*Schema) GetSingleSortKey ¶
GetSingleSortKey should be call only if IsSinglePK is checked
func (*Schema) GetSingleSortKeyIdx ¶
func (*Schema) GetSingleSortKeyType ¶ added in v0.7.0
func (*Schema) HasSortKey ¶
func (*Schema) IsSameColumns ¶ added in v0.8.0
func (*Schema) MustGetExtraBytes ¶ added in v0.8.0
func (*Schema) MustRestoreExtra ¶ added in v0.8.0
func (*Schema) ReadFromBatch ¶ added in v0.6.0
func (*Schema) ReadFromWithVersion ¶ added in v1.0.0
type SegStat ¶ added in v1.0.0
type SegStat struct { // min max etc. later sync.RWMutex // contains filtered or unexported fields }
func (*SegStat) GetCompSize ¶ added in v1.0.1
func (*SegStat) GetOriginSize ¶ added in v1.0.1
func (*SegStat) GetRemainingRows ¶ added in v1.0.1
func (*SegStat) GetSortKeyZonemap ¶ added in v1.0.1
func (*SegStat) SetOriginSize ¶ added in v1.0.1
func (*SegStat) SetRemainingRows ¶ added in v1.0.1
type SegmentDataFactory ¶
type SegmentDataFactory = func(meta *SegmentEntry) data.Segment
type SegmentEntry ¶
type SegmentEntry struct { ID objectio.Segmentid Stat SegStat *BaseEntryImpl[*MetadataMVCCNode] *SegmentNode // contains filtered or unexported fields }
func NewReplaySegmentEntry ¶
func NewReplaySegmentEntry() *SegmentEntry
func NewSegmentEntry ¶
func NewSegmentEntry(table *TableEntry, id *objectio.Segmentid, txn txnif.AsyncTxn, state EntryState, dataFactory SegmentDataFactory) *SegmentEntry
func NewStandaloneSegment ¶
func NewStandaloneSegment(table *TableEntry, ts types.TS) *SegmentEntry
func NewSysSegmentEntry ¶
func NewSysSegmentEntry(table *TableEntry, id types.Uuid) *SegmentEntry
func (*SegmentEntry) AddEntryLocked ¶
func (entry *SegmentEntry) AddEntryLocked(block *BlockEntry)
func (*SegmentEntry) AsCommonID ¶
func (entry *SegmentEntry) AsCommonID() *common.ID
func (*SegmentEntry) BlockCnt ¶
func (entry *SegmentEntry) BlockCnt() int
func (*SegmentEntry) CollectBlockEntries ¶
func (entry *SegmentEntry) CollectBlockEntries(commitFilter func(be *BaseEntryImpl[*MetadataMVCCNode]) bool, blockFilter func(be *BlockEntry) bool) []*BlockEntry
func (*SegmentEntry) CreateBlock ¶
func (entry *SegmentEntry) CreateBlock( txn txnif.AsyncTxn, state EntryState, dataFactory BlockDataFactory, opts *objectio.CreateBlockOpt) (created *BlockEntry, err error)
func (*SegmentEntry) DropBlockEntry ¶
func (entry *SegmentEntry) DropBlockEntry(id *objectio.Blockid, txn txnif.AsyncTxn) (deleted *BlockEntry, err error)
func (*SegmentEntry) GetAppendableBlock ¶ added in v0.6.0
func (entry *SegmentEntry) GetAppendableBlock() (blk *BlockEntry)
func (*SegmentEntry) GetAppendableBlockCnt ¶
func (entry *SegmentEntry) GetAppendableBlockCnt() int
func (*SegmentEntry) GetBlockEntryByID ¶
func (entry *SegmentEntry) GetBlockEntryByID(id *objectio.Blockid) (blk *BlockEntry, err error)
func (*SegmentEntry) GetBlockEntryByIDLocked ¶
func (entry *SegmentEntry) GetBlockEntryByIDLocked(id *objectio.Blockid) (blk *BlockEntry, err error)
XXX API like this, why do we need the error? Isn't blk is nil enough?
func (*SegmentEntry) GetCatalog ¶
func (entry *SegmentEntry) GetCatalog() *Catalog
func (*SegmentEntry) GetFirstBlkEntry ¶ added in v1.0.1
func (entry *SegmentEntry) GetFirstBlkEntry() *BlockEntry
func (*SegmentEntry) GetNextObjectIndex ¶ added in v0.8.0
func (entry *SegmentEntry) GetNextObjectIndex() uint16
func (*SegmentEntry) GetNonAppendableBlockCnt ¶ added in v0.7.0
func (entry *SegmentEntry) GetNonAppendableBlockCnt() int
GetNonAppendableBlockCnt Non-appendable segment only can contain non-appendable blocks; Appendable segment can contain both of appendable blocks and non-appendable blocks
func (*SegmentEntry) GetSegmentData ¶
func (entry *SegmentEntry) GetSegmentData() data.Segment
func (*SegmentEntry) GetTable ¶
func (entry *SegmentEntry) GetTable() *TableEntry
func (*SegmentEntry) GetTerminationTS ¶ added in v0.6.0
func (entry *SegmentEntry) GetTerminationTS() (ts types.TS, terminated bool)
GetTerminationTS is coarse API: no consistency check
func (*SegmentEntry) InitData ¶
func (entry *SegmentEntry) InitData(factory DataFactory)
func (*SegmentEntry) Is1PC ¶ added in v0.6.0
func (entry *SegmentEntry) Is1PC() bool
func (*SegmentEntry) IsActive ¶
func (entry *SegmentEntry) IsActive() bool
IsActive is coarse API: no consistency check
func (*SegmentEntry) IsAppendable ¶
func (entry *SegmentEntry) IsAppendable() bool
func (*SegmentEntry) IsSorted ¶ added in v0.7.0
func (entry *SegmentEntry) IsSorted() bool
func (*SegmentEntry) IsSortedLocked ¶ added in v1.0.0
func (entry *SegmentEntry) IsSortedLocked() bool
func (*SegmentEntry) LastAppendableBlock ¶
func (entry *SegmentEntry) LastAppendableBlock() (blk *BlockEntry)
func (*SegmentEntry) Less ¶ added in v0.8.0
func (entry *SegmentEntry) Less(b *SegmentEntry) int
func (*SegmentEntry) LoadObjectInfo ¶ added in v1.0.0
func (entry *SegmentEntry) LoadObjectInfo() error
LoadObjectInfo is called only in merge scanner goroutine, no need to hold lock
func (*SegmentEntry) MakeBlockIt ¶
func (entry *SegmentEntry) MakeBlockIt(reverse bool) *common.GenericSortedDListIt[*BlockEntry]
func (*SegmentEntry) MakeCommand ¶
func (entry *SegmentEntry) MakeCommand(id uint32) (cmd txnif.TxnCmd, err error)
func (*SegmentEntry) PrepareRollback ¶
func (entry *SegmentEntry) PrepareRollback() (err error)
func (*SegmentEntry) RemoveEntry ¶
func (entry *SegmentEntry) RemoveEntry(block *BlockEntry) (err error)
func (*SegmentEntry) ReplayAddEntryLocked ¶ added in v0.8.0
func (entry *SegmentEntry) ReplayAddEntryLocked(block *BlockEntry)
func (*SegmentEntry) Repr ¶
func (entry *SegmentEntry) Repr() string
func (*SegmentEntry) Set1PC ¶ added in v0.6.0
func (entry *SegmentEntry) Set1PC()
func (*SegmentEntry) SetSorted ¶ added in v0.7.0
func (entry *SegmentEntry) SetSorted()
func (*SegmentEntry) String ¶
func (entry *SegmentEntry) String() string
func (*SegmentEntry) StringLocked ¶
func (entry *SegmentEntry) StringLocked() string
func (*SegmentEntry) StringWithLevel ¶ added in v0.6.0
func (entry *SegmentEntry) StringWithLevel(level common.PPLevel) string
func (*SegmentEntry) StringWithLevelLocked ¶ added in v0.6.0
func (entry *SegmentEntry) StringWithLevelLocked(level common.PPLevel) string
func (*SegmentEntry) TreeMaxDropCommitEntry ¶
func (entry *SegmentEntry) TreeMaxDropCommitEntry() BaseEntry
type SegmentNode ¶ added in v0.8.0
type SegmentNode struct { IsLocal bool // this segment is hold by a localsegment SortHint uint64 // sort segment by create time, make iteration on segment determined // contains filtered or unexported fields }
func (*SegmentNode) ReadFrom ¶ added in v0.8.0
func (node *SegmentNode) ReadFrom(r io.Reader) (n int64, err error)
not marshal nextObjectIdx
func (*SegmentNode) String ¶ added in v0.8.0
func (node *SegmentNode) String() string
type SortKey ¶
type SortKey struct { Defs []*ColDef // contains filtered or unexported fields }
func NewSortKey ¶
func NewSortKey() *SortKey
func (*SortKey) GetSingleIdx ¶
type TableDataFactory ¶
type TableDataFactory = func(meta *TableEntry) data.Table
type TableEntry ¶
type TableEntry struct { *BaseEntryImpl[*TableMVCCNode] *TableNode Stats common.TableCompactStat ID uint64 // used for the next flush table tail. DeletedDirties []*BlockEntry // contains filtered or unexported fields }
func MockStaloneTableEntry ¶
func MockStaloneTableEntry(id uint64, schema *Schema) *TableEntry
func NewReplayTableEntry ¶
func NewReplayTableEntry() *TableEntry
func NewSystemTableEntry ¶
func NewSystemTableEntry(db *DBEntry, id uint64, schema *Schema) *TableEntry
func NewTableEntry ¶
func NewTableEntry(db *DBEntry, schema *Schema, txnCtx txnif.AsyncTxn, dataFactory TableDataFactory) *TableEntry
func NewTableEntryWithTableId ¶ added in v0.6.0
func NewTableEntryWithTableId(db *DBEntry, schema *Schema, txnCtx txnif.AsyncTxn, dataFactory TableDataFactory, tableId uint64) *TableEntry
func (*TableEntry) AddEntryLocked ¶
func (entry *TableEntry) AddEntryLocked(segment *SegmentEntry)
func (*TableEntry) AddRows ¶
func (entry *TableEntry) AddRows(delta uint64) uint64
func (*TableEntry) AlterTable ¶ added in v0.8.0
func (entry *TableEntry) AlterTable(ctx context.Context, txn txnif.TxnReader, req *apipb.AlterTableReq) (isNewNode bool, newSchema *Schema, err error)
func (*TableEntry) ApplyCommit ¶ added in v0.8.0
func (entry *TableEntry) ApplyCommit() (err error)
s: start p: prepare c: commit
old schema <- | -> new schema | s------------------p-----c AlterColumn Txn
Append Txn:
s------------p----c Yes s-------------p--------c Yes s-----------------------p---------c Yes s----------------------p No, schema at s is not same with schema at p
func (*TableEntry) AsCommonID ¶
func (entry *TableEntry) AsCommonID() *common.ID
func (*TableEntry) CreateSegment ¶
func (entry *TableEntry) CreateSegment( txn txnif.AsyncTxn, state EntryState, dataFactory SegmentDataFactory, opts *objectio.CreateSegOpt) (created *SegmentEntry, err error)
func (*TableEntry) CreateWithTxnAndSchema ¶ added in v0.8.0
func (entry *TableEntry) CreateWithTxnAndSchema(txn txnif.AsyncTxn, schema *Schema)
func (*TableEntry) DropSegmentEntry ¶
func (entry *TableEntry) DropSegmentEntry(id *types.Segmentid, txn txnif.AsyncTxn) (deleted *SegmentEntry, err error)
func (*TableEntry) FreezeAppend ¶ added in v0.8.0
func (entry *TableEntry) FreezeAppend()
func (*TableEntry) GetCatalog ¶
func (entry *TableEntry) GetCatalog() *Catalog
func (*TableEntry) GetColDefs ¶ added in v0.6.0
func (entry *TableEntry) GetColDefs() []*ColDef
func (*TableEntry) GetDB ¶
func (entry *TableEntry) GetDB() *DBEntry
func (*TableEntry) GetFullName ¶ added in v0.6.0
func (entry *TableEntry) GetFullName() string
func (*TableEntry) GetID ¶ added in v0.8.0
func (entry *TableEntry) GetID() uint64
func (*TableEntry) GetLastestSchema ¶ added in v0.8.0
func (entry *TableEntry) GetLastestSchema() *Schema
GetLastestSchema returns the latest committed schema
func (*TableEntry) GetRows ¶
func (entry *TableEntry) GetRows() uint64
func (*TableEntry) GetSegmentByID ¶
func (entry *TableEntry) GetSegmentByID(id *types.Segmentid) (seg *SegmentEntry, err error)
func (*TableEntry) GetTableData ¶
func (entry *TableEntry) GetTableData() data.Table
func (*TableEntry) GetTerminationTS ¶ added in v0.6.0
func (entry *TableEntry) GetTerminationTS() (ts types.TS, terminated bool)
GetTerminationTS is coarse API: no consistency check
func (*TableEntry) GetVersionSchema ¶ added in v0.8.0
func (entry *TableEntry) GetVersionSchema(ver uint32) *Schema
func (*TableEntry) GetVisibilityAndName ¶ added in v0.8.0
func (entry *TableEntry) GetVisibilityAndName(txn txnif.TxnReader) (visible, dropped bool, name string)
func (*TableEntry) GetVisibleSchema ¶ added in v0.8.0
func (entry *TableEntry) GetVisibleSchema(txn txnif.TxnReader) *Schema
GetVisibleSchema returns committed schema visible at the given txn
func (*TableEntry) Is1PC ¶ added in v0.6.0
func (entry *TableEntry) Is1PC() bool
func (*TableEntry) IsActive ¶
func (entry *TableEntry) IsActive() bool
IsActive is coarse API: no consistency check
func (*TableEntry) IsVirtual ¶
func (entry *TableEntry) IsVirtual() bool
func (*TableEntry) LastAppendableSegmemt ¶
func (entry *TableEntry) LastAppendableSegmemt() (seg *SegmentEntry)
func (*TableEntry) LastNonAppendableSegmemt ¶ added in v0.7.0
func (entry *TableEntry) LastNonAppendableSegmemt() (seg *SegmentEntry)
func (*TableEntry) Less ¶ added in v0.8.0
func (entry *TableEntry) Less(b *TableEntry) int
func (*TableEntry) MakeCommand ¶
func (entry *TableEntry) MakeCommand(id uint32) (cmd txnif.TxnCmd, err error)
func (*TableEntry) MakeSegmentIt ¶
func (entry *TableEntry) MakeSegmentIt(reverse bool) *common.GenericSortedDListIt[*SegmentEntry]
func (*TableEntry) ObjectStats ¶ added in v1.0.2
func (*TableEntry) ObjectStatsString ¶ added in v1.0.0
func (entry *TableEntry) ObjectStatsString(level common.PPLevel) string
func (*TableEntry) PrepareRollback ¶
func (entry *TableEntry) PrepareRollback() (err error)
func (*TableEntry) RecurLoop ¶
func (entry *TableEntry) RecurLoop(processor Processor) (err error)
func (*TableEntry) RemoveEntry ¶
func (entry *TableEntry) RemoveEntry(segment *SegmentEntry) (err error)
func (*TableEntry) RemoveRows ¶
func (entry *TableEntry) RemoveRows(delta uint64) uint64
func (*TableEntry) Set1PC ¶ added in v0.6.0
func (entry *TableEntry) Set1PC()
func (*TableEntry) String ¶
func (entry *TableEntry) String() string
func (*TableEntry) StringLocked ¶
func (entry *TableEntry) StringLocked() string
func (*TableEntry) StringLockedWithLevel ¶ added in v0.6.0
func (entry *TableEntry) StringLockedWithLevel(level common.PPLevel) string
func (*TableEntry) StringWithLevel ¶ added in v0.6.0
func (entry *TableEntry) StringWithLevel(level common.PPLevel) string
type TableMVCCNode ¶ added in v0.6.0
type TableMVCCNode struct { // history schema Schema *Schema }
func NewEmptyTableMVCCNode ¶ added in v0.6.0
func NewEmptyTableMVCCNode() *TableMVCCNode
func (*TableMVCCNode) CloneAll ¶ added in v0.6.0
func (e *TableMVCCNode) CloneAll() *TableMVCCNode
func (*TableMVCCNode) CloneData ¶ added in v0.6.0
func (e *TableMVCCNode) CloneData() *TableMVCCNode
func (*TableMVCCNode) ReadFromWithVersion ¶ added in v1.0.0
func (*TableMVCCNode) String ¶ added in v0.6.0
func (e *TableMVCCNode) String() string
func (*TableMVCCNode) Update ¶ added in v0.6.0
func (e *TableMVCCNode) Update(un *TableMVCCNode)
for create drop in one txn