ingest

package
v1.1.0-beta.0...-c0b86a3 Latest Latest
Warning

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

Go to latest
Published: Dec 18, 2024 License: Apache-2.0 Imports: 45 Imported by: 0

Documentation

Index

Constants

View Source
const (
	JobCheckpointVersionCurrent = JobCheckpointVersion1
	JobCheckpointVersion1       = 1
)

JobCheckpointVersionCurrent is the current version of the checkpoint.

View Source
const (
	LitErrAllocMemFail      string = "allocate memory failed"
	LitErrCreateDirFail     string = "create ingest sort path error"
	LitErrStatDirFail       string = "stat ingest sort path error"
	LitErrCreateBackendFail string = "build ingest backend failed"
	LitErrGetBackendFail    string = "cannot get ingest backend"
	LitErrCreateEngineFail  string = "build ingest engine failed"
	LitErrCreateContextFail string = "build ingest writer context failed"
	LitErrGetEngineFail     string = "can not get ingest engine info"
	LitErrGetStorageQuota   string = "get storage quota error"
	LitErrCloseEngineErr    string = "close engine error"
	LitErrCleanEngineErr    string = "clean engine error"
	LitErrFlushEngineErr    string = "flush engine data err"
	LitErrIngestDataErr     string = "ingest data into storage error"
	LitErrRemoteDupExistErr string = "remote duplicate index key exist"
	LitErrExceedConcurrency string = "the concurrency is greater than ingest limit"
	LitErrCloseWriterErr    string = "close writer error"
	LitErrReadSortPath      string = "cannot read sort path"
	LitErrCleanSortPath     string = "cannot cleanup sort path"
	LitErrResetEngineFail   string = "reset engine failed"
	LitWarnEnvInitFail      string = "initialize environment failed"
	LitWarnConfigError      string = "build config for backend failed"
	LitInfoEnvInitSucc      string = "init global ingest backend environment finished"
	LitInfoSortDir          string = "the ingest sorted directory"
	LitInfoCreateBackend    string = "create one backend for an DDL job"
	LitInfoCloseBackend     string = "close one backend for DDL job"
	LitInfoOpenEngine       string = "open an engine for index reorg task"
	LitInfoAddWriter        string = "reuse engine and add a writer for index reorg task"
	LitInfoCreateWrite      string = "create one local writer for index reorg task"
	LitInfoCloseEngine      string = "flush all writer and get closed engine"
	LitInfoRemoteDupCheck   string = "start remote duplicate checking"
	LitInfoStartImport      string = "start to import data"
	LitInfoChgMemSetting    string = "change memory setting for ingest"
	LitInfoInitMemSetting   string = "initial memory setting for ingest"
	LitInfoUnsafeImport     string = "do a partial import data into the storage"
)

Message const text

Variables

View Source
var (
	// LitBackCtxMgr is the entry for the lightning backfill process.
	LitBackCtxMgr BackendCtxMgr
	// LitMemRoot is used to track the memory usage of the lightning backfill process.
	LitMemRoot MemRoot
	// LitDiskRoot is used to track the disk usage of the lightning backfill process.
	LitDiskRoot DiskRoot
	// LitRLimit is the max open file number of the lightning backfill process.
	LitRLimit uint64
	// LitSortPath is the sort path for the lightning backfill process.
	LitSortPath string
	// LitInitialized is the flag indicates whether the lightning backfill process is initialized.
	LitInitialized bool
)
View Source
var (
	// StructSizeBackendCtx is the size of litBackendCtx.
	StructSizeBackendCtx int64
	// StructSizeEngineInfo is the size of engineInfo.
	StructSizeEngineInfo int64
	// StructSizeWriterCtx is the size of writerContext.
	StructSizeWriterCtx int64
)
View Source
var ForceSyncFlagForTest = false

ForceSyncFlagForTest is a flag to force sync only for test.

View Source
var GenLightningDataDirForTest = genLightningDataDir

GenLightningDataDirForTest is only used for test.

View Source
var ImporterRangeConcurrencyForTest *atomic.Int32

ImporterRangeConcurrencyForTest is only used for test.

View Source
var MockDMLExecutionStateBeforeImport func()

MockDMLExecutionStateBeforeImport is a failpoint to mock the DML execution state before import.

View Source
var MockExecAfterWriteRow func()

MockExecAfterWriteRow is only used for test.

View Source
var ResignOwnerForTest = atomic.NewBool(false)

ResignOwnerForTest is only used for test.

Functions

func ConfigSortPath

func ConfigSortPath() string

ConfigSortPath returns the sort path for lightning.

func DecodeBackendTag

func DecodeBackendTag(name string) (int64, error)

DecodeBackendTag decodes the backend tag to job ID.

func EncodeBackendTag

func EncodeBackendTag(jobID int64) string

EncodeBackendTag encodes the job ID to backend tag. The backend tag is also used as the file name of the local index data files.

func InitGlobalLightningEnv

func InitGlobalLightningEnv()

InitGlobalLightningEnv initialize Lightning backfill environment.

func InstanceAddr

func InstanceAddr() string

InstanceAddr returns the string concat with instance address and temp-dir.

func NewDDLTLS

func NewDDLTLS() (*common.TLS, error)

NewDDLTLS creates a common.TLS from the tidb config for DDL.

func NewMemRootImpl

func NewMemRootImpl(maxQuota int64, bcCtxMgr *litBackendCtxMgr) *memRootImpl

NewMemRootImpl creates a new memRootImpl.

func RiskOfDiskFull

func RiskOfDiskFull(available, capacity uint64) bool

RiskOfDiskFull checks if the disk has less than 10% space.

func TryFlushAllIndexes

func TryFlushAllIndexes(flushCtrl FlushController, mode FlushMode, indexIDs []int64) (flushed, imported bool, failedIdxID int64, err error)

TryFlushAllIndexes tries to flush and import all indexes.

Types

type BackendCtx

type BackendCtx interface {
	Register(jobID, indexID int64, schemaName, tableName string) (Engine, error)
	Unregister(jobID, indexID int64)

	CollectRemoteDuplicateRows(indexID int64, tbl table.Table) error
	FinishImport(indexID int64, unique bool, tbl table.Table) error
	ResetWorkers(jobID int64)
	Flush(indexID int64, mode FlushMode) (flushed, imported bool, err error)
	Done() bool
	SetDone()

	AttachCheckpointManager(*CheckpointManager)
	GetCheckpointManager() *CheckpointManager

	GetLocalBackend() *local.Backend
}

BackendCtx is the backend context for one add index reorg task.

type BackendCtxMgr

type BackendCtxMgr interface {
	CheckAvailable() (bool, error)
	Register(ctx context.Context, unique bool, jobID int64, etcdClient *clientv3.Client, pdAddr string, resourceGroupName string) (BackendCtx, error)
	Unregister(jobID int64)
	Load(jobID int64) (BackendCtx, bool)
}

BackendCtxMgr is used to manage the backend context.

type CheckpointManager

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

CheckpointManager is a checkpoint manager implementation that used by non-distributed reorganization. It manages the data as two-level checkpoints: "flush"ed to local storage and "import"ed to TiKV. The checkpoint is saved in a table in the TiDB cluster.

func NewCheckpointManager

func NewCheckpointManager(
	ctx context.Context,
	flushCtrl FlushController,
	sessPool *sess.Pool,
	jobID int64,
	indexIDs []int64,
	localStoreDir string,
) (*CheckpointManager, error)

NewCheckpointManager creates a new checkpoint manager.

func (*CheckpointManager) Close

func (s *CheckpointManager) Close()

Close closes the checkpoint manager.

func (*CheckpointManager) Flush

func (s *CheckpointManager) Flush()

Flush flushed the data and updates checkpoint.

func (*CheckpointManager) IsKeyProcessed

func (s *CheckpointManager) IsKeyProcessed(end kv.Key) bool

IsKeyProcessed checks if the key is processed. The key may not be imported. This is called before the reader reads the data and decides whether to skip the current task.

func (*CheckpointManager) Register

func (s *CheckpointManager) Register(taskID int, end kv.Key)

Register registers a new task. taskID MUST be continuous ascending and start from 0.

TODO(lance6716): remove this constraint, use endKey as taskID and use ordered map type for checkpoints.

func (*CheckpointManager) Reset

func (s *CheckpointManager) Reset(newPhysicalID int64, start, end kv.Key)

Reset resets the checkpoint manager between two partitions.

func (*CheckpointManager) Status

func (s *CheckpointManager) Status() (keyCnt int, minKeyImported kv.Key)

Status returns the status of the checkpoint.

func (*CheckpointManager) UpdateTotalKeys

func (s *CheckpointManager) UpdateTotalKeys(taskID int, delta int, last bool)

UpdateTotalKeys updates the total keys of the task. This is called by the reader after reading the data to update the number of rows contained in the current chunk.

func (*CheckpointManager) UpdateWrittenKeys

func (s *CheckpointManager) UpdateWrittenKeys(taskID int, delta int) error

UpdateWrittenKeys updates the written keys of the task. This is called by the writer after writing the local engine to update the current number of rows written.

type Config

type Config struct {
	Lightning    *lightning.Config
	KeyspaceName string
	IsRaftKV2    bool
}

Config is the configuration for the lightning local backend used in DDL.

type DiskRoot

type DiskRoot interface {
	UpdateUsage()
	ShouldImport() bool
	UsageInfo() string
	PreCheckUsage() error
	StartupCheck() error
}

DiskRoot is used to track the disk usage for the lightning backfill process.

func NewDiskRootImpl

func NewDiskRootImpl(path string, bcCtx *litBackendCtxMgr) DiskRoot

NewDiskRootImpl creates a new DiskRoot.

type Engine

type Engine interface {
	Flush() error
	ImportAndClean() error
	Clean()
	CreateWriter(id int) (Writer, error)
}

Engine is the interface for the engine that can be used to write key-value pairs.

type FlushController

type FlushController interface {
	Flush(indexID int64, mode FlushMode) (flushed, imported bool, err error)
}

FlushController is an interface to control the flush of the checkpoint.

type FlushMode

type FlushMode byte

FlushMode is used to control how to flush.

const (
	// FlushModeAuto means caller does not enforce any flush, the implementation can
	// decide it.
	FlushModeAuto FlushMode = iota
	// FlushModeForceFlushNoImport means flush all data to local storage, but don't
	// import the data to TiKV.
	FlushModeForceFlushNoImport
	// FlushModeForceFlushAndImport means flush and import all data to TiKV.
	FlushModeForceFlushAndImport
)

type JobReorgMeta

type JobReorgMeta struct {
	Checkpoint *ReorgCheckpoint `json:"reorg_checkpoint"`
}

JobReorgMeta is the metadata for a reorg job.

type MemRoot

type MemRoot interface {
	Consume(size int64)
	Release(size int64)
	CheckConsume(size int64) bool
	ConsumeWithTag(tag string, size int64)
	ReleaseWithTag(tag string)

	SetMaxMemoryQuota(quota int64)
	MaxMemoryQuota() int64
	CurrentUsage() int64
	CurrentUsageWithTag(tag string) int64
	RefreshConsumption()
}

MemRoot is used to track the memory usage for the lightning backfill process.

type MockBackendCtx

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

MockBackendCtx is a mock backend context.

func (*MockBackendCtx) AttachCheckpointManager

func (m *MockBackendCtx) AttachCheckpointManager(mgr *CheckpointManager)

AttachCheckpointManager attaches a checkpoint manager to the backend context.

func (*MockBackendCtx) CollectRemoteDuplicateRows

func (*MockBackendCtx) CollectRemoteDuplicateRows(indexID int64, _ table.Table) error

CollectRemoteDuplicateRows implements BackendCtx.CollectRemoteDuplicateRows interface.

func (*MockBackendCtx) Done

func (*MockBackendCtx) Done() bool

Done implements BackendCtx.Done interface.

func (*MockBackendCtx) FinishImport

func (*MockBackendCtx) FinishImport(indexID int64, _ bool, _ table.Table) error

FinishImport implements BackendCtx.FinishImport interface.

func (*MockBackendCtx) Flush

func (*MockBackendCtx) Flush(_ int64, _ FlushMode) (flushed bool, imported bool, err error)

Flush implements BackendCtx.Flush interface.

func (*MockBackendCtx) GetCheckpointManager

func (m *MockBackendCtx) GetCheckpointManager() *CheckpointManager

GetCheckpointManager returns the checkpoint manager attached to the backend context.

func (*MockBackendCtx) GetLocalBackend

func (m *MockBackendCtx) GetLocalBackend() *local.Backend

GetLocalBackend returns the local backend.

func (*MockBackendCtx) Register

func (m *MockBackendCtx) Register(jobID, indexID int64, _, _ string) (Engine, error)

Register implements BackendCtx.Register interface.

func (*MockBackendCtx) ResetWorkers

func (*MockBackendCtx) ResetWorkers(_ int64)

ResetWorkers implements BackendCtx.ResetWorkers interface.

func (*MockBackendCtx) SetDone

func (*MockBackendCtx) SetDone()

SetDone implements BackendCtx.SetDone interface.

func (*MockBackendCtx) Unregister

func (*MockBackendCtx) Unregister(jobID, indexID int64)

Unregister implements BackendCtx.Unregister interface.

type MockBackendCtxMgr

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

MockBackendCtxMgr is a mock backend context manager.

func NewMockBackendCtxMgr

func NewMockBackendCtxMgr(sessCtxProvider func() sessionctx.Context) *MockBackendCtxMgr

NewMockBackendCtxMgr creates a new mock backend context manager.

func (*MockBackendCtxMgr) CheckAvailable

func (m *MockBackendCtxMgr) CheckAvailable() (bool, error)

CheckAvailable implements BackendCtxMgr.Available interface.

func (*MockBackendCtxMgr) Load

func (m *MockBackendCtxMgr) Load(jobID int64) (BackendCtx, bool)

Load implements BackendCtxMgr.Load interface.

func (*MockBackendCtxMgr) Register

func (m *MockBackendCtxMgr) Register(_ context.Context, _ bool, jobID int64, _ *clientv3.Client, _ string, _ string) (BackendCtx, error)

Register implements BackendCtxMgr.Register interface.

func (*MockBackendCtxMgr) ResetSessCtx

func (m *MockBackendCtxMgr) ResetSessCtx()

ResetSessCtx is only used for mocking test.

func (*MockBackendCtxMgr) Unregister

func (m *MockBackendCtxMgr) Unregister(jobID int64)

Unregister implements BackendCtxMgr.Unregister interface.

type MockEngineInfo

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

MockEngineInfo is a mock engine info.

func NewMockEngineInfo

func NewMockEngineInfo(sessCtx sessionctx.Context) *MockEngineInfo

NewMockEngineInfo creates a new mock engine info.

func (*MockEngineInfo) Clean

func (*MockEngineInfo) Clean()

Clean implements Engine.Clean interface.

func (*MockEngineInfo) CreateWriter

func (m *MockEngineInfo) CreateWriter(id int) (Writer, error)

CreateWriter implements Engine.CreateWriter interface.

func (*MockEngineInfo) Flush

func (*MockEngineInfo) Flush() error

Flush implements Engine.Flush interface.

func (*MockEngineInfo) ImportAndClean

func (*MockEngineInfo) ImportAndClean() error

ImportAndClean implements Engine.ImportAndClean interface.

func (*MockEngineInfo) SetHook

func (m *MockEngineInfo) SetHook(onWrite func(key, val []byte))

SetHook set the write hook.

type MockWriteHook

type MockWriteHook func(key, val []byte)

MockWriteHook the hook for write in mock engine.

type MockWriter

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

MockWriter is a mock writer.

func (*MockWriter) Close

func (*MockWriter) Close(_ context.Context) error

Close implements Writer.Close interface.

func (*MockWriter) LockForWrite

func (*MockWriter) LockForWrite() func()

LockForWrite implements Writer.LockForWrite interface.

func (*MockWriter) WriteRow

func (m *MockWriter) WriteRow(_ context.Context, key, idxVal []byte, _ kv.Handle) error

WriteRow implements Writer.WriteRow interface.

type ReorgCheckpoint

type ReorgCheckpoint struct {
	LocalSyncKey   kv.Key `json:"local_sync_key"`
	LocalKeyCount  int    `json:"local_key_count"`
	GlobalSyncKey  kv.Key `json:"global_sync_key"`
	GlobalKeyCount int    `json:"global_key_count"`
	InstanceAddr   string `json:"instance_addr"`

	PhysicalID int64  `json:"physical_id"`
	StartKey   kv.Key `json:"start_key"`
	EndKey     kv.Key `json:"end_key"`

	Version int64 `json:"version"`
}

ReorgCheckpoint is the checkpoint for a reorg job.

type Writer

type Writer interface {
	// WriteRow writes one row into downstream.
	// To enable uniqueness check, the handle should be non-empty.
	WriteRow(ctx context.Context, idxKey, idxVal []byte, handle tidbkv.Handle) error
	LockForWrite() (unlock func())
	Close(ctx context.Context) error
}

Writer is the interface for the writer that can be used to write key-value pairs.

Directories

Path Synopsis

Jump to

Keyboard shortcuts

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