Documentation ¶
Index ¶
- Constants
- Variables
- func AlterAutoIncrement(ctx context.Context, db *sql.DB, tableName string, incr uint64) error
- func AlterAutoRandom(ctx context.Context, db *sql.DB, tableName string, randomBase uint64, ...) error
- func DBFromConfig(ctx context.Context, dsn config.DBStore) (*sql.DB, error)
- func DoChecksum(ctx context.Context, table *checkpoints.TidbTableInfo) (*local.RemoteChecksum, error)
- func LoadSchemaInfo(ctx context.Context, schemas []*mydump.MDDatabaseMeta, ...) (map[string]*checkpoints.TidbDBInfo, error)
- func MaybeCleanupAllMetas(ctx context.Context, logger log.Logger, db *sql.DB, schemaName string, ...) error
- func NewCDCPITRCheckItem(cfg *config.Config, pdAddrsGetter func(context.Context) []string) precheck.Checker
- func NewCSVHeaderCheckItem(cfg *config.Config, preInfoGetter PreImportInfoGetter, ...) precheck.Checker
- func NewCheckpointCheckItem(cfg *config.Config, preInfoGetter PreImportInfoGetter, ...) precheck.Checker
- func NewChecksumManager(ctx context.Context, rc *Controller, store kv.Storage) (local.ChecksumManager, error)
- func NewClusterResourceCheckItem(preInfoGetter PreImportInfoGetter) precheck.Checker
- func NewClusterVersionCheckItem(preInfoGetter PreImportInfoGetter, dbMetas []*mydump.MDDatabaseMeta) precheck.Checker
- func NewEmptyRegionCheckItem(preInfoGetter PreImportInfoGetter, dbMetas []*mydump.MDDatabaseMeta) precheck.Checker
- func NewLargeFileCheckItem(cfg *config.Config, dbMetas []*mydump.MDDatabaseMeta) precheck.Checker
- func NewLocalDiskPlacementCheckItem(cfg *config.Config) precheck.Checker
- func NewLocalTempKVDirCheckItem(cfg *config.Config, preInfoGetter PreImportInfoGetter, ...) precheck.Checker
- func NewRegionDistributionCheckItem(preInfoGetter PreImportInfoGetter, dbMetas []*mydump.MDDatabaseMeta) precheck.Checker
- func NewSchemaCheckItem(cfg *config.Config, preInfoGetter PreImportInfoGetter, ...) precheck.Checker
- func NewStoragePermissionCheckItem(cfg *config.Config) precheck.Checker
- func NewTableEmptyCheckItem(cfg *config.Config, preInfoGetter PreImportInfoGetter, ...) precheck.Checker
- func ObtainImportantVariables(ctx context.Context, db *sql.DB, needTiDBVars bool) map[string]string
- func ObtainNewCollationEnabled(ctx context.Context, db *sql.DB) (bool, error)
- func RemoveTableMetaByTableName(ctx context.Context, db *sql.DB, metaTable, tableName string) error
- func WithPreInfoGetterDBMetas(ctx context.Context, dbMetas []*mydump.MDDatabaseMeta) context.Context
- func WithPrecheckKey(ctx context.Context, key precheckContextKey, val any) context.Context
- type CDCPITRCheckItem
- type Controller
- func (rc *Controller) Close()
- func (rc *Controller) ClusterIsAvailable(ctx context.Context) error
- func (rc *Controller) DataCheck(ctx context.Context) error
- func (rc *Controller) HasLargeCSV(ctx context.Context) error
- func (rc *Controller) Run(ctx context.Context) error
- func (rc *Controller) StoragePermission(ctx context.Context) error
- type ControllerParam
- type EstimateSourceDataSizeResult
- type LightningStatus
- type PreImportInfoGetter
- type PreImportInfoGetterImpl
- func (p *PreImportInfoGetterImpl) CheckVersionRequirements(ctx context.Context) error
- func (p *PreImportInfoGetterImpl) EstimateSourceDataSize(ctx context.Context, opts ...ropts.GetPreInfoOption) (*EstimateSourceDataSizeResult, error)
- func (p *PreImportInfoGetterImpl) FetchRemoteDBModels(ctx context.Context) ([]*model.DBInfo, error)
- func (p *PreImportInfoGetterImpl) FetchRemoteTableModels(ctx context.Context, schemaName string, tableNames []string) (map[string]*model.TableInfo, error)
- func (p *PreImportInfoGetterImpl) GetAllTableStructures(ctx context.Context, opts ...ropts.GetPreInfoOption) (map[string]*checkpoints.TidbDBInfo, error)
- func (p *PreImportInfoGetterImpl) GetEmptyRegionsInfo(ctx context.Context) (*pdhttp.RegionsInfo, error)
- func (p *PreImportInfoGetterImpl) GetMaxReplica(ctx context.Context) (uint64, error)
- func (p *PreImportInfoGetterImpl) GetStorageInfo(ctx context.Context) (*pdhttp.StoresInfo, error)
- func (p *PreImportInfoGetterImpl) GetTargetSysVariablesForImport(ctx context.Context, opts ...ropts.GetPreInfoOption) map[string]string
- func (p *PreImportInfoGetterImpl) Init()
- func (p *PreImportInfoGetterImpl) IsTableEmpty(ctx context.Context, schemaName string, tableName string) (*bool, error)
- func (p *PreImportInfoGetterImpl) ReadFirstNRowsByFileMeta(ctx context.Context, dataFileMeta mydump.SourceFileMeta, n int) ([]string, [][]types.Datum, error)
- func (p *PreImportInfoGetterImpl) ReadFirstNRowsByTableName(ctx context.Context, schemaName string, tableName string, n int) ([]string, [][]types.Datum, error)
- type PrecheckItemBuilder
- type SimpleTemplate
- type TableImporter
- type TargetInfoGetter
- type TargetInfoGetterImpl
- func (g *TargetInfoGetterImpl) CheckVersionRequirements(ctx context.Context) error
- func (g *TargetInfoGetterImpl) FetchRemoteDBModels(ctx context.Context) ([]*model.DBInfo, error)
- func (g *TargetInfoGetterImpl) FetchRemoteTableModels(ctx context.Context, schemaName string, tableNames []string) (map[string]*model.TableInfo, error)
- func (g *TargetInfoGetterImpl) GetEmptyRegionsInfo(ctx context.Context) (*pdhttp.RegionsInfo, error)
- func (g *TargetInfoGetterImpl) GetMaxReplica(ctx context.Context) (uint64, error)
- func (g *TargetInfoGetterImpl) GetStorageInfo(ctx context.Context) (*pdhttp.StoresInfo, error)
- func (g *TargetInfoGetterImpl) GetTargetSysVariablesForImport(ctx context.Context, _ ...ropts.GetPreInfoOption) map[string]string
- func (g *TargetInfoGetterImpl) IsTableEmpty(ctx context.Context, schemaName string, tableName string) (*bool, error)
- type Template
- type TiDBManager
Constants ¶
const ( FullLevelCompact = -1 Level1Compact = 1 )
compact levels
const ( TaskMetaTableName = "task_meta_v2" TableMetaTableName = "table_meta" // CreateTableMetadataTable stores the per-table sub jobs information used by TiDB Lightning CreateTableMetadataTable = `` /* 693-byte string literal not displayed */ // CreateTaskMetaTable stores the pre-lightning metadata used by TiDB Lightning CreateTaskMetaTable = `` /* 497-byte string literal not displayed */ )
task related table names and create table statements.
Variables ¶
var DeliverPauser = common.NewPauser()
DeliverPauser is a shared pauser to pause progress to (*chunkProcessor).encodeLoop
var ErrDuplicateKey = errors.Normalize("duplicate key detected on indexID %d of KeyID: %v", errors.RFCCodeText("Lightning:PreDedup:ErrDuplicateKey"))
ErrDuplicateKey is an error class for duplicate key error.
Functions ¶
func AlterAutoIncrement ¶
AlterAutoIncrement rebase the table auto increment id
NOTE: since tidb can make sure the auto id is always be rebase even if the `incr` value is smaller than the auto increment base in tidb side, we needn't fetch currently auto increment value here. See: https://github.com/pingcap/tidb/blob/64698ef9a3358bfd0fdc323996bb7928a56cadca/ddl/ddl_api.go#L2528-L2533
func AlterAutoRandom ¶
func AlterAutoRandom(ctx context.Context, db *sql.DB, tableName string, randomBase uint64, maxAutoRandom uint64) error
AlterAutoRandom rebase the table auto random id
func DBFromConfig ¶
DBFromConfig creates a new connection to the TiDB database.
func DoChecksum ¶
func DoChecksum(ctx context.Context, table *checkpoints.TidbTableInfo) (*local.RemoteChecksum, error)
DoChecksum do checksum for tables. table should be in <db>.<table>, format. e.g. foo.bar
func LoadSchemaInfo ¶
func LoadSchemaInfo( ctx context.Context, schemas []*mydump.MDDatabaseMeta, getTables func(context.Context, string) ([]*model.TableInfo, error), ) (map[string]*checkpoints.TidbDBInfo, error)
LoadSchemaInfo loads schema information from TiDB.
func MaybeCleanupAllMetas ¶
func MaybeCleanupAllMetas( ctx context.Context, logger log.Logger, db *sql.DB, schemaName string, tableMetaExist bool, ) error
MaybeCleanupAllMetas remove the meta schema if there is no unfinished tables
func NewCDCPITRCheckItem ¶
func NewCDCPITRCheckItem(cfg *config.Config, pdAddrsGetter func(context.Context) []string) precheck.Checker
NewCDCPITRCheckItem creates a checker to check downstream has enabled CDC or PiTR.
func NewCSVHeaderCheckItem ¶
func NewCSVHeaderCheckItem(cfg *config.Config, preInfoGetter PreImportInfoGetter, dbMetas []*mydump.MDDatabaseMeta) precheck.Checker
NewCSVHeaderCheckItem creates a new csvHeaderCheckItem.
func NewCheckpointCheckItem ¶
func NewCheckpointCheckItem(cfg *config.Config, preInfoGetter PreImportInfoGetter, dbMetas []*mydump.MDDatabaseMeta, checkpointsDB checkpoints.DB) precheck.Checker
NewCheckpointCheckItem creates a new checkpointCheckItem.
func NewChecksumManager ¶
func NewChecksumManager(ctx context.Context, rc *Controller, store kv.Storage) (local.ChecksumManager, error)
NewChecksumManager creates a new checksum manager.
func NewClusterResourceCheckItem ¶
func NewClusterResourceCheckItem(preInfoGetter PreImportInfoGetter) precheck.Checker
NewClusterResourceCheckItem creates a new clusterResourceCheckItem.
func NewClusterVersionCheckItem ¶
func NewClusterVersionCheckItem(preInfoGetter PreImportInfoGetter, dbMetas []*mydump.MDDatabaseMeta) precheck.Checker
NewClusterVersionCheckItem creates a new clusterVersionCheckItem.
func NewEmptyRegionCheckItem ¶
func NewEmptyRegionCheckItem(preInfoGetter PreImportInfoGetter, dbMetas []*mydump.MDDatabaseMeta) precheck.Checker
NewEmptyRegionCheckItem creates a new emptyRegionCheckItem.
func NewLargeFileCheckItem ¶
NewLargeFileCheckItem creates a new largeFileCheckItem.
func NewLocalDiskPlacementCheckItem ¶
NewLocalDiskPlacementCheckItem creates a new localDiskPlacementCheckItem.
func NewLocalTempKVDirCheckItem ¶
func NewLocalTempKVDirCheckItem(cfg *config.Config, preInfoGetter PreImportInfoGetter, dbMetas []*mydump.MDDatabaseMeta) precheck.Checker
NewLocalTempKVDirCheckItem creates a new localTempKVDirCheckItem.
func NewRegionDistributionCheckItem ¶
func NewRegionDistributionCheckItem(preInfoGetter PreImportInfoGetter, dbMetas []*mydump.MDDatabaseMeta) precheck.Checker
NewRegionDistributionCheckItem creates a new regionDistributionCheckItem.
func NewSchemaCheckItem ¶
func NewSchemaCheckItem(cfg *config.Config, preInfoGetter PreImportInfoGetter, dbMetas []*mydump.MDDatabaseMeta, cpdb checkpoints.DB) precheck.Checker
NewSchemaCheckItem creates a checker to check whether the schema is valid.
func NewStoragePermissionCheckItem ¶
NewStoragePermissionCheckItem creates a new storagePermissionCheckItem.
func NewTableEmptyCheckItem ¶
func NewTableEmptyCheckItem(cfg *config.Config, preInfoGetter PreImportInfoGetter, dbMetas []*mydump.MDDatabaseMeta, cpdb checkpoints.DB) precheck.Checker
NewTableEmptyCheckItem creates a new tableEmptyCheckItem
func ObtainImportantVariables ¶
ObtainImportantVariables obtains the important variables from TiDB.
func ObtainNewCollationEnabled ¶
ObtainNewCollationEnabled obtains the new collation enabled status from TiDB.
func RemoveTableMetaByTableName ¶
RemoveTableMetaByTableName remove table meta by table name
func WithPreInfoGetterDBMetas ¶
func WithPreInfoGetterDBMetas(ctx context.Context, dbMetas []*mydump.MDDatabaseMeta) context.Context
WithPreInfoGetterDBMetas returns a new context with the specified dbMetas.
Types ¶
type CDCPITRCheckItem ¶
type CDCPITRCheckItem struct { Instruction string // contains filtered or unexported fields }
CDCPITRCheckItem check downstream has enabled CDC or PiTR. It's exposed to let caller override the Instruction message.
func (*CDCPITRCheckItem) Check ¶
func (ci *CDCPITRCheckItem) Check(ctx context.Context) (*precheck.CheckResult, error)
Check implements Checker interface.
func (*CDCPITRCheckItem) GetCheckItemID ¶
func (*CDCPITRCheckItem) GetCheckItemID() precheck.CheckItemID
GetCheckItemID implements Checker interface.
type Controller ¶
type Controller struct {
// contains filtered or unexported fields
}
Controller controls the whole import process.
func NewImportController ¶
func NewImportController( ctx context.Context, cfg *config.Config, param *ControllerParam, ) (*Controller, error)
NewImportController creates a new Controller instance.
func NewImportControllerWithPauser ¶
func NewImportControllerWithPauser( ctx context.Context, cfg *config.Config, p *ControllerParam, ) (*Controller, error)
NewImportControllerWithPauser creates a new Controller instance with a pauser.
func (*Controller) ClusterIsAvailable ¶
func (rc *Controller) ClusterIsAvailable(ctx context.Context) error
ClusterIsAvailable check cluster is available to import data. this test can be skipped.
func (*Controller) DataCheck ¶
func (rc *Controller) DataCheck(ctx context.Context) error
DataCheck checks the data schema which needs #rc.restoreSchema finished.
func (*Controller) HasLargeCSV ¶
func (rc *Controller) HasLargeCSV(ctx context.Context) error
HasLargeCSV checks whether input csvs is fit for Lightning import. If strictFormat is false, and csv file is large. Lightning will have performance issue. this test cannot be skipped.
func (*Controller) Run ¶
func (rc *Controller) Run(ctx context.Context) error
Run starts the restore task.
func (*Controller) StoragePermission ¶
func (rc *Controller) StoragePermission(ctx context.Context) error
StoragePermission checks whether Lightning has enough permission to storage.
type ControllerParam ¶
type ControllerParam struct { // databases that dumper created DBMetas []*mydump.MDDatabaseMeta // a pointer to status to report it to caller Status *LightningStatus // storage interface to read the dump data DumpFileStorage storage.ExternalStorage // true if DumpFileStorage is created by lightning. In some cases where lightning is a library, the framework may pass an DumpFileStorage OwnExtStorage bool // used by lightning server mode to pause tasks Pauser *common.Pauser // DB is a connection pool to TiDB DB *sql.DB // storage interface to write file checkpoints CheckpointStorage storage.ExternalStorage // when CheckpointStorage is not nil, save file checkpoint to it with this name CheckpointName string // DupIndicator can expose the duplicate detection result to the caller DupIndicator *atomic.Bool // Keyspace name KeyspaceName string // ResourceGroup name for current TiDB user ResourceGroupName string // TaskType is the source component name use for background task control. TaskType string }
ControllerParam contains many parameters for creating a Controller.
type EstimateSourceDataSizeResult ¶
type EstimateSourceDataSizeResult struct { // SizeWithIndex is the tikv size with the index. SizeWithIndex int64 // SizeWithoutIndex is the tikv size without the index. SizeWithoutIndex int64 // HasUnsortedBigTables indicates whether the source data has unsorted big tables or not. HasUnsortedBigTables bool // TiFlashSize is the size of tiflash. TiFlashSize int64 }
EstimateSourceDataSizeResult is the object for estimated data size result.
type LightningStatus ¶
type LightningStatus struct { FinishedFileSize atomic.Int64 TotalFileSize atomic.Int64 // contains filtered or unexported fields }
LightningStatus provides the finished bytes and total bytes of the current task. It should keep the value after restart from checkpoint. When it is tidb backend, FinishedFileSize can be counted after chunk data is restored to tidb. When it is local backend it's counted after whole engine is imported. TotalFileSize may be an estimated value, so when the task is finished, it may not equal to FinishedFileSize.
type PreImportInfoGetter ¶
type PreImportInfoGetter interface { TargetInfoGetter // GetAllTableStructures gets all the table structures with the information from both the source and the target. GetAllTableStructures(ctx context.Context, opts ...ropts.GetPreInfoOption) (map[string]*checkpoints.TidbDBInfo, error) // ReadFirstNRowsByTableName reads the first N rows of data of an importing source table. ReadFirstNRowsByTableName(ctx context.Context, schemaName string, tableName string, n int) (cols []string, rows [][]types.Datum, err error) // ReadFirstNRowsByFileMeta reads the first N rows of an data file. ReadFirstNRowsByFileMeta(ctx context.Context, dataFileMeta mydump.SourceFileMeta, n int) (cols []string, rows [][]types.Datum, err error) // EstimateSourceDataSize estimates the datasize to generate during the import as well as some other sub-informaiton. // It will return: // * the estimated data size to generate during the import, // which might include some extra index data to generate besides the source file data // * the total data size of all the source files, // * whether there are some unsorted big tables EstimateSourceDataSize(ctx context.Context, opts ...ropts.GetPreInfoOption) (*EstimateSourceDataSizeResult, error) }
PreImportInfoGetter defines the operations to get information from sources and target. These information are used in the preparation of the import ( like precheck ).
type PreImportInfoGetterImpl ¶
type PreImportInfoGetterImpl struct {
// contains filtered or unexported fields
}
PreImportInfoGetterImpl implements the operations to get information used in importing preparation.
func NewPreImportInfoGetter ¶
func NewPreImportInfoGetter( cfg *config.Config, dbMetas []*mydump.MDDatabaseMeta, srcStorage storage.ExternalStorage, targetInfoGetter TargetInfoGetter, ioWorkers *worker.Pool, encBuilder encode.EncodingBuilder, opts ...ropts.GetPreInfoOption, ) (*PreImportInfoGetterImpl, error)
NewPreImportInfoGetter creates a PreImportInfoGetterImpl object.
func (*PreImportInfoGetterImpl) CheckVersionRequirements ¶
func (p *PreImportInfoGetterImpl) CheckVersionRequirements(ctx context.Context) error
CheckVersionRequirements performs the check whether the target satisfies the version requirements. It implements the PreImportInfoGetter interface. Mydump database metas are retrieved from the context.
func (*PreImportInfoGetterImpl) EstimateSourceDataSize ¶
func (p *PreImportInfoGetterImpl) EstimateSourceDataSize(ctx context.Context, opts ...ropts.GetPreInfoOption) (*EstimateSourceDataSizeResult, error)
EstimateSourceDataSize estimates the datasize to generate during the import as well as some other sub-informaiton. It implements the PreImportInfoGetter interface. It has a cache mechanism. The estimated size will only calculated once. The caching behavior can be changed by appending the `ForceReloadCache(true)` option.
func (*PreImportInfoGetterImpl) FetchRemoteDBModels ¶
FetchRemoteDBModels fetches the database structures from the remote target. It implements the PreImportInfoGetter interface.
func (*PreImportInfoGetterImpl) FetchRemoteTableModels ¶
func (p *PreImportInfoGetterImpl) FetchRemoteTableModels( ctx context.Context, schemaName string, tableNames []string, ) (map[string]*model.TableInfo, error)
FetchRemoteTableModels fetches the table structures from the remote target. It implements the PreImportInfoGetter interface.
func (*PreImportInfoGetterImpl) GetAllTableStructures ¶
func (p *PreImportInfoGetterImpl) GetAllTableStructures(ctx context.Context, opts ...ropts.GetPreInfoOption) (map[string]*checkpoints.TidbDBInfo, error)
GetAllTableStructures gets all the table structures with the information from both the source and the target. It implements the PreImportInfoGetter interface. It has a caching mechanism: the table structures will be obtained from the source only once.
func (*PreImportInfoGetterImpl) GetEmptyRegionsInfo ¶
func (p *PreImportInfoGetterImpl) GetEmptyRegionsInfo(ctx context.Context) (*pdhttp.RegionsInfo, error)
GetEmptyRegionsInfo gets the region information of all the empty regions on the target. It implements the PreImportInfoGetter interface.
func (*PreImportInfoGetterImpl) GetMaxReplica ¶
func (p *PreImportInfoGetterImpl) GetMaxReplica(ctx context.Context) (uint64, error)
GetMaxReplica implements the PreImportInfoGetter interface.
func (*PreImportInfoGetterImpl) GetStorageInfo ¶
func (p *PreImportInfoGetterImpl) GetStorageInfo(ctx context.Context) (*pdhttp.StoresInfo, error)
GetStorageInfo gets the storage information on the target. It implements the PreImportInfoGetter interface.
func (*PreImportInfoGetterImpl) GetTargetSysVariablesForImport ¶
func (p *PreImportInfoGetterImpl) GetTargetSysVariablesForImport(ctx context.Context, opts ...ropts.GetPreInfoOption) map[string]string
GetTargetSysVariablesForImport gets some important systam variables for importing on the target. It implements the PreImportInfoGetter interface. It has caching mechanism.
func (*PreImportInfoGetterImpl) Init ¶
func (p *PreImportInfoGetterImpl) Init()
Init initializes some internal data and states for PreImportInfoGetterImpl.
func (*PreImportInfoGetterImpl) IsTableEmpty ¶
func (p *PreImportInfoGetterImpl) IsTableEmpty(ctx context.Context, schemaName string, tableName string) (*bool, error)
IsTableEmpty checks whether the specified table on the target DB contains data or not. It implements the PreImportInfoGetter interface.
func (*PreImportInfoGetterImpl) ReadFirstNRowsByFileMeta ¶
func (p *PreImportInfoGetterImpl) ReadFirstNRowsByFileMeta(ctx context.Context, dataFileMeta mydump.SourceFileMeta, n int) ([]string, [][]types.Datum, error)
ReadFirstNRowsByFileMeta reads the first N rows of an data file. It implements the PreImportInfoGetter interface.
func (*PreImportInfoGetterImpl) ReadFirstNRowsByTableName ¶
func (p *PreImportInfoGetterImpl) ReadFirstNRowsByTableName(ctx context.Context, schemaName string, tableName string, n int) ([]string, [][]types.Datum, error)
ReadFirstNRowsByTableName reads the first N rows of data of an importing source table. It implements the PreImportInfoGetter interface.
type PrecheckItemBuilder ¶
type PrecheckItemBuilder struct {
// contains filtered or unexported fields
}
PrecheckItemBuilder is used to build precheck items
func NewPrecheckItemBuilder ¶
func NewPrecheckItemBuilder( cfg *config.Config, dbMetas []*mydump.MDDatabaseMeta, preInfoGetter PreImportInfoGetter, checkpointsDB checkpoints.DB, pdHTTPCli pdhttp.Client, ) *PrecheckItemBuilder
NewPrecheckItemBuilder creates a new PrecheckItemBuilder
func NewPrecheckItemBuilderFromConfig ¶
func NewPrecheckItemBuilderFromConfig( ctx context.Context, cfg *config.Config, pdHTTPCli pdhttp.Client, opts ...ropts.PrecheckItemBuilderOption, ) (*PrecheckItemBuilder, error)
NewPrecheckItemBuilderFromConfig creates a new PrecheckItemBuilder from config pdHTTPCli **must not** be nil for local backend
func (*PrecheckItemBuilder) BuildPrecheckItem ¶
func (b *PrecheckItemBuilder) BuildPrecheckItem(checkID precheck.CheckItemID) (precheck.Checker, error)
BuildPrecheckItem builds a Checker by the given checkID
func (*PrecheckItemBuilder) GetPreInfoGetter ¶
func (b *PrecheckItemBuilder) GetPreInfoGetter() PreImportInfoGetter
GetPreInfoGetter gets the pre restore info getter from the builder.
type SimpleTemplate ¶
type SimpleTemplate struct {
// contains filtered or unexported fields
}
SimpleTemplate is a simple template for lightning check.
func (*SimpleTemplate) Collect ¶
func (c *SimpleTemplate) Collect(t precheck.CheckType, passed bool, msg string)
Collect mainly collect performance related checks' results and critical level checks' results.
func (*SimpleTemplate) FailedCount ¶
func (c *SimpleTemplate) FailedCount(t precheck.CheckType) int
FailedCount represents (the warn check failed count, the critical check failed count)
func (*SimpleTemplate) FailedMsg ¶
func (c *SimpleTemplate) FailedMsg() string
FailedMsg returns the error msg for the failed check.
func (*SimpleTemplate) Output ¶
func (c *SimpleTemplate) Output() string
Output print all checks results.
func (*SimpleTemplate) Success ¶
func (c *SimpleTemplate) Success() bool
Success represents the whole check has passed or not.
type TableImporter ¶
type TableImporter struct {
// contains filtered or unexported fields
}
TableImporter is a helper struct to import a table.
func NewTableImporter ¶
func NewTableImporter( tableName string, tableMeta *mydump.MDTableMeta, dbInfo *checkpoints.TidbDBInfo, tableInfo *checkpoints.TidbTableInfo, cp *checkpoints.TableCheckpoint, ignoreColumns map[string]struct{}, kvStore tidbkv.Storage, etcdCli *clientv3.Client, logger log.Logger, ) (*TableImporter, error)
NewTableImporter creates a new TableImporter.
func (*TableImporter) AutoIDClient ¶
func (tr *TableImporter) AutoIDClient() *autoid.ClientDiscover
AutoIDClient implements the autoid.Requirement interface.
func (*TableImporter) Close ¶
func (tr *TableImporter) Close()
Close implements the Importer interface.
func (*TableImporter) RebaseChunkRowIDs ¶
func (*TableImporter) RebaseChunkRowIDs(cp *checkpoints.TableCheckpoint, rowIDBase int64)
RebaseChunkRowIDs rebase the row id of the chunks.
func (*TableImporter) Store ¶
func (tr *TableImporter) Store() tidbkv.Storage
Store implements the autoid.Requirement interface.
type TargetInfoGetter ¶
type TargetInfoGetter interface { // FetchRemoteDBModels fetches the database structures from the remote target. FetchRemoteDBModels(ctx context.Context) ([]*model.DBInfo, error) // FetchRemoteTableModels fetches the table structures from the remote target. FetchRemoteTableModels(ctx context.Context, schemaName string, tableNames []string) (map[string]*model.TableInfo, error) // CheckVersionRequirements performs the check whether the target satisfies the version requirements. CheckVersionRequirements(ctx context.Context) error // IsTableEmpty checks whether the specified table on the target DB contains data or not. IsTableEmpty(ctx context.Context, schemaName string, tableName string) (*bool, error) // GetTargetSysVariablesForImport gets some important systam variables for importing on the target. GetTargetSysVariablesForImport(ctx context.Context, opts ...ropts.GetPreInfoOption) map[string]string // GetMaxReplica gets the max-replica from replication config on the target. GetMaxReplica(ctx context.Context) (uint64, error) // GetStorageInfo gets the storage information on the target. GetStorageInfo(ctx context.Context) (*pdhttp.StoresInfo, error) // GetEmptyRegionsInfo gets the region information of all the empty regions on the target. GetEmptyRegionsInfo(ctx context.Context) (*pdhttp.RegionsInfo, error) }
TargetInfoGetter defines the operations to get information from target.
type TargetInfoGetterImpl ¶
type TargetInfoGetterImpl struct {
// contains filtered or unexported fields
}
TargetInfoGetterImpl implements the operations to get information from the target.
func NewTargetInfoGetterImpl ¶
func NewTargetInfoGetterImpl( cfg *config.Config, targetDB *sql.DB, pdHTTPCli pdhttp.Client, ) (*TargetInfoGetterImpl, error)
NewTargetInfoGetterImpl creates a TargetInfoGetterImpl object.
func (*TargetInfoGetterImpl) CheckVersionRequirements ¶
func (g *TargetInfoGetterImpl) CheckVersionRequirements(ctx context.Context) error
CheckVersionRequirements performs the check whether the target satisfies the version requirements. It implements the TargetInfoGetter interface. Mydump database metas are retrieved from the context.
func (*TargetInfoGetterImpl) FetchRemoteDBModels ¶
FetchRemoteDBModels implements TargetInfoGetter.
func (*TargetInfoGetterImpl) FetchRemoteTableModels ¶
func (g *TargetInfoGetterImpl) FetchRemoteTableModels( ctx context.Context, schemaName string, tableNames []string, ) (map[string]*model.TableInfo, error)
FetchRemoteTableModels fetches the table structures from the remote target. It implements the TargetInfoGetter interface.
func (*TargetInfoGetterImpl) GetEmptyRegionsInfo ¶
func (g *TargetInfoGetterImpl) GetEmptyRegionsInfo(ctx context.Context) (*pdhttp.RegionsInfo, error)
GetEmptyRegionsInfo gets the region information of all the empty regions on the target. It implements the TargetInfoGetter interface. It uses the PD interface through TLS to get the information.
func (*TargetInfoGetterImpl) GetMaxReplica ¶
func (g *TargetInfoGetterImpl) GetMaxReplica(ctx context.Context) (uint64, error)
GetMaxReplica implements the TargetInfoGetter interface.
func (*TargetInfoGetterImpl) GetStorageInfo ¶
func (g *TargetInfoGetterImpl) GetStorageInfo(ctx context.Context) (*pdhttp.StoresInfo, error)
GetStorageInfo gets the storage information on the target. It implements the TargetInfoGetter interface. It uses the PD interface through TLS to get the information.
func (*TargetInfoGetterImpl) GetTargetSysVariablesForImport ¶
func (g *TargetInfoGetterImpl) GetTargetSysVariablesForImport(ctx context.Context, _ ...ropts.GetPreInfoOption) map[string]string
GetTargetSysVariablesForImport gets some important system variables for importing on the target. It implements the TargetInfoGetter interface. It uses the SQL to fetch sys variables from the target.
func (*TargetInfoGetterImpl) IsTableEmpty ¶
func (g *TargetInfoGetterImpl) IsTableEmpty(ctx context.Context, schemaName string, tableName string) (*bool, error)
IsTableEmpty checks whether the specified table on the target DB contains data or not. It implements the TargetInfoGetter interface. It tries to select the row count from the target DB.
type Template ¶
type Template interface { // Collect mainly collect performance related checks' results and critical level checks' results. // If the performance is not as expect or one of critical check not passed. it will stop import task. Collect(t precheck.CheckType, passed bool, msg string) // Success represents the whole check has passed or not. Success() bool // FailedCount represents (the warn check failed count, the critical check failed count) FailedCount(t precheck.CheckType) int // Output print all checks results. Output() string // FailedMsg represents the error msg for the failed check. FailedMsg() string }
Template is the interface for lightning check.
func NewSimpleTemplate ¶
func NewSimpleTemplate() Template
NewSimpleTemplate returns a simple template.
type TiDBManager ¶
type TiDBManager struct {
// contains filtered or unexported fields
}
TiDBManager is a wrapper of *sql.DB which provides some helper methods for
func NewTiDBManager ¶
NewTiDBManager creates a new TiDB manager.
func NewTiDBManagerWithDB ¶
func NewTiDBManagerWithDB(db *sql.DB, sqlMode mysql.SQLMode) *TiDBManager
NewTiDBManagerWithDB creates a new TiDB manager with an existing database connection.
func (*TiDBManager) Close ¶
func (timgr *TiDBManager) Close()
Close closes the underlying database connection.