export

package
v0.0.0-...-503c688 Latest Latest
Warning

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

Go to latest
Published: Nov 8, 2023 License: Apache-2.0 Imports: 60 Imported by: 0

Documentation

Index

Constants

View Source
const (
	// UnspecifiedSize means the filesize/statement-size is unspecified
	UnspecifiedSize = 0
	// DefaultStatementSize is the default statement size
	DefaultStatementSize = 1000000
	// TiDBMemQuotaQueryName is the session variable TiDBMemQuotaQuery's name in TiDB
	TiDBMemQuotaQueryName = "tidb_mem_quota_query"
	// DefaultTableFilter is the default exclude table filter. It will exclude all system databases
	DefaultTableFilter = "!/^(mysql|sys|INFORMATION_SCHEMA|PERFORMANCE_SCHEMA|METRICS_SCHEMA|INSPECTION_SCHEMA)$/.*"

	// LooseCollationCompatible is used in DM, represents a collation setting for best compatibility.
	LooseCollationCompatible = "loose"
	// StrictCollationCompatible is used in DM, represents a collation setting for correctness.
	StrictCollationCompatible = "strict"
)
View Source
const (
	// ConsistencyTypeAuto will use flush for MySQL/MariaDB and snapshot for TiDB.
	ConsistencyTypeAuto = "auto"
	// ConsistencyTypeFlush will use FLUSH TABLES WITH READ LOCK to temporarily interrupt the DML and DDL operations of the replica database,
	// to ensure the global consistency of the backup connection.
	ConsistencyTypeFlush = "flush"
	// ConsistencyTypeLock will add read locks on all tables to be exported.
	ConsistencyTypeLock = "lock"
	// ConsistencyTypeSnapshot gets a consistent snapshot of the specified timestamp and exports it.
	ConsistencyTypeSnapshot = "snapshot"
	// ConsistencyTypeNone doesn't guarantee for consistency.
	ConsistencyTypeNone = "none"
)
View Source
const (
	// TableTypeBaseStr represents the basic table string
	TableTypeBaseStr = "BASE TABLE"
	// TableTypeViewStr represents the view table string
	TableTypeViewStr = "VIEW"
	// TableTypeSequenceStr represents the view table string
	TableTypeSequenceStr = "SEQUENCE"
)
View Source
const (
	// FileFormatSQLTextString indicates the string/suffix of sql type file
	FileFormatSQLTextString = "sql"
	// FileFormatCSVString indicates the string/suffix of csv type file
	FileFormatCSVString = "csv"
)
View Source
const (

	// DefaultAnonymousOutputFileTemplateText is the default anonymous output file templateText for dumpling's table data file name
	DefaultAnonymousOutputFileTemplateText = "result.{{.Index}}"
)
View Source
const (

	// ErrNoSuchTable is the error code no such table in MySQL/TiDB
	ErrNoSuchTable uint16 = 1146
)
View Source
const (

	// FlagHelp represents the help flag
	FlagHelp = "help"
)

Variables

View Source
var (

	// DefaultOutputFileTemplate is the default output file template for dumpling's table data file name
	DefaultOutputFileTemplate = template.Must(template.New("data").
		Option("missingkey=error").
		Funcs(template.FuncMap{
			"fn": func(input string) string {
				return filenameEscapeRegexp.ReplaceAllStringFunc(input, func(match string) string {
					return fmt.Sprintf("%%%02X%s", match[0], match[1:])
				})
			},
		}).
		Parse(defaultOutputFileTemplateBase))
)
View Source
var ServerInfoUnknown = version.ServerInfo{
	ServerType:    version.ServerTypeUnknown,
	ServerVersion: nil,
}

ServerInfoUnknown is the unknown database type to dumpling

Functions

func AddCounter

func AddCounter(counterVec *prometheus.CounterVec, v float64)

AddCounter adds a counter.

func AddGauge

func AddGauge(gaugeVec *prometheus.GaugeVec, v float64)

AddGauge adds a gauge

func CheckIfSeqExists

func CheckIfSeqExists(db *sql.Conn) (bool, error)

CheckIfSeqExists use sql to check whether sequence exists

func CheckTiDBEnableTableLock

func CheckTiDBEnableTableLock(db *sql.Conn) (bool, error)

CheckTiDBEnableTableLock use sql variable to check whether current TiDB has TiKV

func CheckTiDBWithTiKV

func CheckTiDBWithTiKV(db *sql.DB) (bool, error)

CheckTiDBWithTiKV use sql to check whether current TiDB has TiKV

func DatabaseTablesToMap

func DatabaseTablesToMap(d DatabaseTables) map[string]map[string]struct{}

DatabaseTablesToMap transfers DatabaseTables to Map

func DecGauge

func DecGauge(gaugeVec *prometheus.GaugeVec)

DecGauge decs a gauge

func FlushTableWithReadLock

func FlushTableWithReadLock(ctx context.Context, db *sql.Conn) error

FlushTableWithReadLock flush tables with read lock

func GetCharsetAndDefaultCollation

func GetCharsetAndDefaultCollation(ctx context.Context, db *sql.Conn) (map[string]string, error)

GetCharsetAndDefaultCollation gets charset and default collation map.

func GetColumnTypes

func GetColumnTypes(tctx *tcontext.Context, db *BaseConn, fields, database, table string) ([]*sql.ColumnType, error)

GetColumnTypes gets *sql.ColumnTypes from a specified table

func GetDBInfo

func GetDBInfo(db *sql.Conn, tables map[string]map[string]struct{}) ([]*model.DBInfo, error)

GetDBInfo get model.DBInfos from database sql interface. We need table_id to check whether a region belongs to this table

func GetPartitionNames

func GetPartitionNames(tctx *tcontext.Context, db *BaseConn, schema, table string) (partitions []string, err error)

GetPartitionNames get partition names from a specified table

func GetPartitionTableIDs

func GetPartitionTableIDs(db *sql.Conn, tables map[string]map[string]struct{}) (map[string]map[string]map[string]int64, error)

GetPartitionTableIDs get partition tableIDs through histograms. SHOW STATS_HISTOGRAMS has db_name,table_name,partition_name but doesn't have partition id mysql.stats_histograms has partition_id but doesn't have db_name,table_name,partition_name So we combine the results from these two sqls to get partition ids for each table If UPDATE_TIME,DISTINCT_COUNT are equal, we assume these two records can represent one line. If histograms are not accurate or (UPDATE_TIME,DISTINCT_COUNT) has duplicate data, it's still fine. Because the possibility is low and the effect is that we will select more than one regions in one time, this will not affect the correctness of the dumping data and will not affect the memory usage much. This method is tricky, but no better way is found. Because TiDB v3.0.0's information_schema.partition table doesn't have partition name or partition id info return (dbName -> tbName -> partitionName -> partitionID, error)

func GetPdAddrs

func GetPdAddrs(tctx *tcontext.Context, db *sql.DB) ([]string, error)

GetPdAddrs gets PD address from TiDB

func GetPrimaryKeyAndColumnTypes

func GetPrimaryKeyAndColumnTypes(tctx *tcontext.Context, conn *BaseConn, meta TableMeta) ([]string, []string, error)

GetPrimaryKeyAndColumnTypes gets all primary columns and their types in ordinal order

func GetPrimaryKeyColumns

func GetPrimaryKeyColumns(tctx *tcontext.Context, db *BaseConn, database, table string) ([]string, error)

GetPrimaryKeyColumns gets all primary columns in ordinal order

func GetRegionInfos

func GetRegionInfos(db *sql.Conn) (*helper.RegionsInfo, error)

GetRegionInfos get region info including regionID, start key, end key from database sql interface. start key, end key includes information to help split table

func GetSpecifiedColumnValueAndClose

func GetSpecifiedColumnValueAndClose(rows *sql.Rows, columnName string) ([]string, error)

GetSpecifiedColumnValueAndClose get columns' values whose name is equal to columnName and close the given rows

func GetSpecifiedColumnValuesAndClose

func GetSpecifiedColumnValuesAndClose(rows *sql.Rows, columnName ...string) ([][]string, error)

GetSpecifiedColumnValuesAndClose get columns' values whose name is equal to columnName

func GetSuitableRows

func GetSuitableRows(avgRowLength uint64) uint64

GetSuitableRows gets suitable rows for each table

func GetTiDBDDLIDs

func GetTiDBDDLIDs(tctx *tcontext.Context, db *sql.DB) ([]string, error)

GetTiDBDDLIDs gets DDL IDs from TiDB

func IncCounter

func IncCounter(counterVec *prometheus.CounterVec)

IncCounter incs a counter.

func IncGauge

func IncGauge(gaugeVec *prometheus.GaugeVec)

IncGauge incs a gauge

func ListAllPlacementPolicyNames

func ListAllPlacementPolicyNames(tctx *tcontext.Context, db *BaseConn) ([]string, error)

ListAllPlacementPolicyNames returns all placement policy names.

func LockTables

func LockTables(ctx context.Context, db *sql.Conn, database, table string) error

LockTables locks table with read lock

func ObserveHistogram

func ObserveHistogram(histogramVec *prometheus.HistogramVec, v float64)

ObserveHistogram observes a histogram

func ParseCompressType

func ParseCompressType(compressType string) (storage.CompressType, error)

ParseCompressType parses compressType string to storage.CompressType

func ParseFileSize

func ParseFileSize(fileSizeStr string) (uint64, error)

ParseFileSize parses file size from tables-list and filter arguments

func ParseOutputFileTemplate

func ParseOutputFileTemplate(text string) (*template.Template, error)

ParseOutputFileTemplate parses template from the specified text

func ParseTableFilter

func ParseTableFilter(tablesList, filters []string) (filter.Filter, error)

ParseTableFilter parses table filter from tables-list and filter arguments

func ReadCounter

func ReadCounter(counterVec *prometheus.CounterVec) float64

ReadCounter reports the current value of the counter.

func ReadGauge

func ReadGauge(gaugeVec *prometheus.GaugeVec) float64

ReadGauge reports the current value of the gauge.

func RestoreCharset

func RestoreCharset(w io.StringWriter)

RestoreCharset builds the restore charset SQLs

func SelectTiDBRowID

func SelectTiDBRowID(tctx *tcontext.Context, db *BaseConn, database, table string) (bool, error)

SelectTiDBRowID checks whether this table has _tidb_rowid column

func SelectVersion

func SelectVersion(db *sql.DB) (string, error)

SelectVersion gets the version information from the database server

func SetCharset

func SetCharset(w *strings.Builder, characterSet, collationConnection string)

SetCharset builds the set charset SQLs

func ShowCreateDatabase

func ShowCreateDatabase(tctx *tcontext.Context, db *BaseConn, database string) (string, error)

ShowCreateDatabase constructs the create database SQL for a specified database returns (createDatabaseSQL, error)

func ShowCreatePlacementPolicy

func ShowCreatePlacementPolicy(tctx *tcontext.Context, db *BaseConn, policy string) (string, error)

ShowCreatePlacementPolicy constructs the create policy SQL for a specified table returns (createPolicySQL, error)

func ShowCreateSequence

func ShowCreateSequence(tctx *tcontext.Context, db *BaseConn, database, sequence string, conf *Config) (string, error)

ShowCreateSequence constructs the create sequence SQL for a specified sequence returns (createSequenceSQL, error)

func ShowCreateTable

func ShowCreateTable(tctx *tcontext.Context, db *BaseConn, database, table string) (string, error)

ShowCreateTable constructs the create table SQL for a specified table returns (createTableSQL, error)

func ShowCreateView

func ShowCreateView(tctx *tcontext.Context, db *BaseConn, database, view string) (createFakeTableSQL string, createRealViewSQL string, err error)

ShowCreateView constructs the create view SQL for a specified view returns (createFakeTableSQL, createViewSQL, error)

func ShowDatabases

func ShowDatabases(db *sql.Conn) ([]string, error)

ShowDatabases shows the databases of a database server.

func ShowMasterStatus

func ShowMasterStatus(db *sql.Conn) ([]string, error)

ShowMasterStatus get SHOW MASTER STATUS result from database

func ShowTables

func ShowTables(db *sql.Conn) ([]string, error)

ShowTables shows the tables of a database, the caller should use the correct database.

func SubGauge

func SubGauge(gaugeVec *prometheus.GaugeVec, v float64)

SubGauge subs a gauge

func UnlockTables

func UnlockTables(ctx context.Context, db *sql.Conn) error

UnlockTables unlocks all tables' lock

func WriteInsert

func WriteInsert(
	pCtx *tcontext.Context,
	cfg *Config,
	meta TableMeta,
	tblIR TableDataIR,
	w storage.ExternalFileWriter,
	metrics *metrics,
) (n uint64, err error)

WriteInsert writes TableDataIR to a storage.ExternalFileWriter in sql type

func WriteInsertInCsv

func WriteInsertInCsv(
	pCtx *tcontext.Context,
	cfg *Config,
	meta TableMeta,
	tblIR TableDataIR,
	w storage.ExternalFileWriter,
	metrics *metrics,
) (n uint64, err error)

WriteInsertInCsv writes TableDataIR to a storage.ExternalFileWriter in csv type

func WriteMeta

func WriteMeta(tctx *tcontext.Context, meta MetaIR, w storage.ExternalFileWriter) error

WriteMeta writes MetaIR to a storage.ExternalFileWriter

Types

type BaseConn

type BaseConn struct {
	DBConn *sql.Conn
	// contains filtered or unexported fields
}

BaseConn wraps connection instance.

func (*BaseConn) ExecSQL

func (conn *BaseConn) ExecSQL(tctx *tcontext.Context, canRetryFunc func(sql.Result, error) error, query string, args ...interface{}) error

ExecSQL defines exec statement, and connect to real DB.

func (*BaseConn) QuerySQL

func (conn *BaseConn) QuerySQL(tctx *tcontext.Context, handleOneRow func(*sql.Rows) error, reset func(), query string, args ...interface{}) error

QuerySQL defines query statement, and connect to real DB.

func (*BaseConn) QuerySQLWithColumns

func (conn *BaseConn) QuerySQLWithColumns(tctx *tcontext.Context, columns []string, query string, args ...interface{}) ([][]string, error)

QuerySQLWithColumns defines query statement, and connect to real DB and get results for special column names

type Config

type Config struct {
	storage.BackendOptions

	AllowCleartextPasswords  bool
	SortByPk                 bool
	NoViews                  bool
	NoSequences              bool
	NoHeader                 bool
	NoSchemas                bool
	NoData                   bool
	CompleteInsert           bool
	TransactionalConsistency bool
	EscapeBackslash          bool
	DumpEmptyDatabase        bool
	PosAfterConnect          bool
	CompressType             storage.CompressType

	Host     string
	Port     int
	Threads  int
	User     string
	Password string `json:"-"`
	Security struct {
		TLS          *tls.Config `json:"-"`
		CAPath       string
		CertPath     string
		KeyPath      string
		SSLCABytes   []byte `json:"-"`
		SSLCertBytes []byte `json:"-"`
		SSLKeyBytes  []byte `json:"-"`
	}

	LogLevel      string
	LogFile       string
	LogFormat     string
	OutputDirPath string
	StatusAddr    string
	Snapshot      string
	Consistency   string
	CsvNullValue  string
	SQL           string
	CsvSeparator  string
	CsvDelimiter  string
	Databases     []string

	TableFilter         filter.Filter `json:"-"`
	Where               string
	FileType            string
	ServerInfo          version.ServerInfo
	Logger              *zap.Logger        `json:"-"`
	OutputFileTemplate  *template.Template `json:"-"`
	Rows                uint64
	ReadTimeout         time.Duration
	TiDBMemQuotaQuery   uint64
	FileSize            uint64
	StatementSize       uint64
	SessionParams       map[string]interface{}
	Tables              DatabaseTables
	CollationCompatible string

	Labels       prometheus.Labels       `json:"-"`
	PromFactory  promutil.Factory        `json:"-"`
	PromRegistry promutil.Registry       `json:"-"`
	ExtStorage   storage.ExternalStorage `json:"-"`

	IOTotalBytes *atomic.Uint64
	Net          string
	// contains filtered or unexported fields
}

Config is the dump config for dumpling

func DefaultConfig

func DefaultConfig() *Config

DefaultConfig returns the default export Config for dumpling

func (*Config) DefineFlags

func (*Config) DefineFlags(flags *pflag.FlagSet)

DefineFlags defines flags of dumpling's configuration

func (*Config) GetDriverConfig

func (conf *Config) GetDriverConfig(db string) *mysql.Config

GetDriverConfig returns the MySQL driver config from Config.

func (*Config) ParseFromFlags

func (conf *Config) ParseFromFlags(flags *pflag.FlagSet) error

ParseFromFlags parses dumpling's export.Config from flags nolint: gocyclo

func (*Config) String

func (conf *Config) String() string

String returns dumpling's config in json format

type ConsistencyController

type ConsistencyController interface {
	Setup(*tcontext.Context) error
	TearDown(context.Context) error
	PingContext(context.Context) error
}

ConsistencyController is the interface that controls the consistency of exporting progress

func NewConsistencyController

func NewConsistencyController(ctx context.Context, conf *Config, session *sql.DB) (ConsistencyController, error)

NewConsistencyController returns a new consistency controller

type ConsistencyFlushTableWithReadLock

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

ConsistencyFlushTableWithReadLock uses FlushTableWithReadLock before the dump

func (*ConsistencyFlushTableWithReadLock) PingContext

PingContext implements ConsistencyController.PingContext

func (*ConsistencyFlushTableWithReadLock) Setup

Setup implements ConsistencyController.Setup

func (*ConsistencyFlushTableWithReadLock) TearDown

TearDown implements ConsistencyController.TearDown

type ConsistencyLockDumpingTables

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

ConsistencyLockDumpingTables execute lock tables read on all tables before dump

func (*ConsistencyLockDumpingTables) PingContext

func (c *ConsistencyLockDumpingTables) PingContext(ctx context.Context) error

PingContext implements ConsistencyController.PingContext

func (*ConsistencyLockDumpingTables) Setup

Setup implements ConsistencyController.Setup

func (*ConsistencyLockDumpingTables) TearDown

TearDown implements ConsistencyController.TearDown

type ConsistencyNone

type ConsistencyNone struct{}

ConsistencyNone dumps without adding locks, which cannot guarantee consistency

func (*ConsistencyNone) PingContext

func (*ConsistencyNone) PingContext(_ context.Context) error

PingContext implements ConsistencyController.PingContext

func (*ConsistencyNone) Setup

Setup implements ConsistencyController.Setup

func (*ConsistencyNone) TearDown

func (*ConsistencyNone) TearDown(_ context.Context) error

TearDown implements ConsistencyController.TearDown

type DatabaseTables

type DatabaseTables map[databaseName][]*TableInfo

DatabaseTables is the type that represents tables in a database

func GetConfTables

func GetConfTables(tablesList []string) (DatabaseTables, error)

GetConfTables parses tables from tables-list and filter arguments

func ListAllDatabasesTables

func ListAllDatabasesTables(tctx *tcontext.Context, db *sql.Conn, databaseNames []string,
	listType listTableType, tableTypes ...TableType) (DatabaseTables, error)

ListAllDatabasesTables lists all the databases and tables from the database listTableByInfoSchema list tables by table information_schema in MySQL listTableByShowTableStatus has better performance than listTableByInfoSchema listTableByShowFullTables is used in mysql8 version [8.0.3,8.0.23), more details can be found in the comments of func matchMysqlBugversion

func NewDatabaseTables

func NewDatabaseTables() DatabaseTables

NewDatabaseTables returns a new DatabaseTables

func (DatabaseTables) AppendTable

func (d DatabaseTables) AppendTable(dbName string, table *TableInfo) DatabaseTables

AppendTable appends a TableInfo to DatabaseTables

func (DatabaseTables) AppendTables

func (d DatabaseTables) AppendTables(dbName string, tableNames []string, avgRowLengths []uint64) DatabaseTables

AppendTables appends several basic tables to DatabaseTables

func (DatabaseTables) AppendViews

func (d DatabaseTables) AppendViews(dbName string, viewNames ...string) DatabaseTables

AppendViews appends several views to DatabaseTables

func (DatabaseTables) Literal

func (d DatabaseTables) Literal() string

Literal returns a user-friendly output for DatabaseTables

func (DatabaseTables) Merge

func (d DatabaseTables) Merge(other DatabaseTables)

Merge merges another DatabaseTables

type DumpStatus

type DumpStatus struct {
	CompletedTables   float64
	FinishedBytes     float64
	FinishedRows      float64
	EstimateTotalRows float64
	TotalTables       int64
	CurrentSpeedBPS   float64
	Progress          string
}

DumpStatus is the status of dumping.

type Dumper

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

Dumper is the dump progress structure

func NewDumper

func NewDumper(ctx context.Context, conf *Config) (*Dumper, error)

NewDumper returns a new Dumper

func (*Dumper) Close

func (d *Dumper) Close() error

Close closes a Dumper and stop dumping immediately

func (*Dumper) Dump

func (d *Dumper) Dump() (dumpErr error)

Dump dumps table from database nolint: gocyclo

func (*Dumper) GetStatus

func (d *Dumper) GetStatus() *DumpStatus

GetStatus returns the status of dumping by reading metrics.

func (*Dumper) L

func (d *Dumper) L() log.Logger

L returns real logger

type FileFormat

type FileFormat int32

FileFormat is the format that output to file. Currently we support SQL text and CSV file format.

const (
	// FileFormatUnknown indicates the given file type is unknown
	FileFormatUnknown FileFormat = iota
	// FileFormatSQLText indicates the given file type is sql type
	FileFormatSQLText
	// FileFormatCSV indicates the given file type is csv type
	FileFormatCSV
)

func (FileFormat) Extension

func (f FileFormat) Extension() string

Extension returns the extension for specific format.

text -> "sql"
csv  -> "csv"

func (FileFormat) String

func (f FileFormat) String() string

String implement Stringer.String method.

func (FileFormat) WriteInsert

func (f FileFormat) WriteInsert(
	pCtx *tcontext.Context,
	cfg *Config,
	meta TableMeta,
	tblIR TableDataIR,
	w storage.ExternalFileWriter,
	metrics *metrics,
) (uint64, error)

WriteInsert writes TableDataIR to a storage.ExternalFileWriter in sql/csv type

type InterceptFileWriter

type InterceptFileWriter struct {
	storage.ExternalFileWriter
	sync.Once
	SomethingIsWritten bool
	// contains filtered or unexported fields
}

InterceptFileWriter is an interceptor of os.File, tracking whether a StringWriter has written something.

func (*InterceptFileWriter) Close

func (w *InterceptFileWriter) Close(ctx context.Context) error

Close closes the InterceptFileWriter

func (*InterceptFileWriter) Write

func (w *InterceptFileWriter) Write(ctx context.Context, p []byte) (int, error)

Write implements storage.ExternalFileWriter.Write. It check whether writer has written something and init a file at first time

type LazyStringWriter

type LazyStringWriter struct {
	sync.Once
	io.StringWriter
	// contains filtered or unexported fields
}

LazyStringWriter is an interceptor of io.StringWriter, will lazily create file the first time StringWriter need to write something.

func (*LazyStringWriter) WriteString

func (l *LazyStringWriter) WriteString(str string) (int, error)

WriteString implements io.StringWriter. It check whether writer has written something and init a file at first time

type MetaIR

type MetaIR interface {
	SpecialComments() StringIter
	TargetName() string
	MetaSQL() string
}

MetaIR is the interface that wraps database/table/view's metadata

type RowReceiver

type RowReceiver interface {
	BindAddress([]interface{})
}

RowReceiver is an interface which represents sql types that support bind address for *sql.Rows

type RowReceiverArr

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

RowReceiverArr is the combined RowReceiver array

func MakeRowReceiver

func MakeRowReceiver(colTypes []string) RowReceiverArr

MakeRowReceiver constructs RowReceiverArr from column types

func (RowReceiverArr) BindAddress

func (r RowReceiverArr) BindAddress(args []interface{})

BindAddress implements RowReceiver.BindAddress

func (RowReceiverArr) WriteToBuffer

func (r RowReceiverArr) WriteToBuffer(bf *bytes.Buffer, escapeBackslash bool)

WriteToBuffer implements Stringer.WriteToBuffer

func (RowReceiverArr) WriteToBufferInCsv

func (r RowReceiverArr) WriteToBufferInCsv(bf *bytes.Buffer, escapeBackslash bool, opt *csvOption)

WriteToBufferInCsv implements Stringer.WriteToBufferInCsv

type RowReceiverStringer

type RowReceiverStringer interface {
	RowReceiver
	Stringer
}

RowReceiverStringer is a combined interface of RowReceiver and Stringer

func SQLTypeBytesMaker

func SQLTypeBytesMaker() RowReceiverStringer

SQLTypeBytesMaker returns a SQLTypeBytes

func SQLTypeNumberMaker

func SQLTypeNumberMaker() RowReceiverStringer

SQLTypeNumberMaker returns a SQLTypeNumber

func SQLTypeStringMaker

func SQLTypeStringMaker() RowReceiverStringer

SQLTypeStringMaker returns a SQLTypeString

type SQLRowIter

type SQLRowIter interface {
	Decode(RowReceiver) error
	Next()
	Error() error
	HasNext() bool
	// release SQLRowIter
	Close() error
}

SQLRowIter is the iterator on a collection of sql.Row.

type SQLTypeBytes

type SQLTypeBytes struct {
	sql.RawBytes
}

SQLTypeBytes implements RowReceiverStringer which represents bytes type columns in database

func (*SQLTypeBytes) BindAddress

func (s *SQLTypeBytes) BindAddress(arg []interface{})

BindAddress implements RowReceiver.BindAddress

func (*SQLTypeBytes) WriteToBuffer

func (s *SQLTypeBytes) WriteToBuffer(bf *bytes.Buffer, _ bool)

WriteToBuffer implements Stringer.WriteToBuffer

func (*SQLTypeBytes) WriteToBufferInCsv

func (s *SQLTypeBytes) WriteToBufferInCsv(bf *bytes.Buffer, escapeBackslash bool, opt *csvOption)

WriteToBufferInCsv implements Stringer.WriteToBufferInCsv

type SQLTypeNumber

type SQLTypeNumber struct {
	SQLTypeString
}

SQLTypeNumber implements RowReceiverStringer which represents numeric type columns in database

func (SQLTypeNumber) WriteToBuffer

func (s SQLTypeNumber) WriteToBuffer(bf *bytes.Buffer, _ bool)

WriteToBuffer implements Stringer.WriteToBuffer

func (SQLTypeNumber) WriteToBufferInCsv

func (s SQLTypeNumber) WriteToBufferInCsv(bf *bytes.Buffer, _ bool, opt *csvOption)

WriteToBufferInCsv implements Stringer.WriteToBufferInCsv

type SQLTypeString

type SQLTypeString struct {
	sql.RawBytes
}

SQLTypeString implements RowReceiverStringer which represents string type columns in database

func (*SQLTypeString) BindAddress

func (s *SQLTypeString) BindAddress(arg []interface{})

BindAddress implements RowReceiver.BindAddress

func (*SQLTypeString) WriteToBuffer

func (s *SQLTypeString) WriteToBuffer(bf *bytes.Buffer, escapeBackslash bool)

WriteToBuffer implements Stringer.WriteToBuffer

func (*SQLTypeString) WriteToBufferInCsv

func (s *SQLTypeString) WriteToBufferInCsv(bf *bytes.Buffer, escapeBackslash bool, opt *csvOption)

WriteToBufferInCsv implements Stringer.WriteToBufferInCsv

type SpeedRecorder

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

SpeedRecorder record the finished bytes and calculate its speed.

func NewSpeedRecorder

func NewSpeedRecorder() *SpeedRecorder

NewSpeedRecorder new a SpeedRecorder.

func (*SpeedRecorder) GetSpeed

func (s *SpeedRecorder) GetSpeed(finished float64) float64

GetSpeed calculate status speed.

type StringIter

type StringIter interface {
	Next() string
	HasNext() bool
}

StringIter is the iterator on a collection of strings.

type Stringer

type Stringer interface {
	WriteToBuffer(*bytes.Buffer, bool)
	WriteToBufferInCsv(*bytes.Buffer, bool, *csvOption)
}

Stringer is an interface which represents sql types that support writing to buffer in sql/csv type

type TableDataIR

type TableDataIR interface {
	Start(*tcontext.Context, *sql.Conn) error
	Rows() SQLRowIter
	Close() error
	RawRows() *sql.Rows
}

TableDataIR is table data intermediate representation. A table may be split into multiple TableDataIRs.

func SelectAllFromTable

func SelectAllFromTable(conf *Config, meta TableMeta, partition, orderByClause string) TableDataIR

SelectAllFromTable dumps data serialized from a specified table

type TableInfo

type TableInfo struct {
	Name         string
	AvgRowLength uint64
	Type         TableType
}

TableInfo is the table info for a table in database

func (*TableInfo) Equals

func (t *TableInfo) Equals(other *TableInfo) bool

Equals returns true the table info is the same with another one

type TableMeta

type TableMeta interface {
	DatabaseName() string
	TableName() string
	ColumnCount() uint
	ColumnTypes() []string
	ColumnNames() []string
	SelectedField() string
	SelectedLen() int
	SpecialComments() StringIter
	ShowCreateTable() string
	ShowCreateView() string
	AvgRowLength() uint64
	HasImplicitRowID() bool
}

TableMeta contains the meta information of a table.

type TableType

type TableType int8

TableType represents the type of table

const (
	// TableTypeBase represents the basic table
	TableTypeBase TableType = iota
	// TableTypeView represents the view table
	TableTypeView
	// TableTypeSequence represents the view table
	// TODO: need to be supported
	TableTypeSequence
)

func ParseTableType

func ParseTableType(s string) (TableType, error)

ParseTableType parses table type string to TableType

func (TableType) String

func (t TableType) String() string

type Task

type Task interface {
	// Brief is the brief for a dumping task
	Brief() string
}

Task is a file dump task for dumpling, it could either be dumping database/table/view/policy metadata, table data

type TaskDatabaseMeta

type TaskDatabaseMeta struct {
	Task
	DatabaseName      string
	CreateDatabaseSQL string
}

TaskDatabaseMeta is a dumping database metadata task

func NewTaskDatabaseMeta

func NewTaskDatabaseMeta(dbName, createSQL string) *TaskDatabaseMeta

NewTaskDatabaseMeta returns a new dumping database metadata task

func (*TaskDatabaseMeta) Brief

func (t *TaskDatabaseMeta) Brief() string

Brief implements task.Brief

type TaskPolicyMeta

type TaskPolicyMeta struct {
	Task
	PolicyName      string
	CreatePolicySQL string
}

TaskPolicyMeta is a dumping view metadata task

func NewTaskPolicyMeta

func NewTaskPolicyMeta(policyName, createPolicySQL string) *TaskPolicyMeta

NewTaskPolicyMeta returns a new dumping placement policy metadata task

func (*TaskPolicyMeta) Brief

func (t *TaskPolicyMeta) Brief() string

Brief implements task.Brief

type TaskSequenceMeta

type TaskSequenceMeta struct {
	Task
	DatabaseName      string
	SequenceName      string
	CreateSequenceSQL string
}

TaskSequenceMeta is a dumping sequence metadata task

func NewTaskSequenceMeta

func NewTaskSequenceMeta(dbName, tblName, createSequenceSQL string) *TaskSequenceMeta

NewTaskSequenceMeta returns a new dumping sequence metadata task

func (*TaskSequenceMeta) Brief

func (t *TaskSequenceMeta) Brief() string

Brief implements task.Brief

type TaskTableData

type TaskTableData struct {
	Task
	Meta        TableMeta
	Data        TableDataIR
	ChunkIndex  int
	TotalChunks int
}

TaskTableData is a dumping table data task

func NewTaskTableData

func NewTaskTableData(meta TableMeta, data TableDataIR, currentChunk, totalChunks int) *TaskTableData

NewTaskTableData returns a new dumping table data task

func (*TaskTableData) Brief

func (t *TaskTableData) Brief() string

Brief implements task.Brief

type TaskTableMeta

type TaskTableMeta struct {
	Task
	DatabaseName   string
	TableName      string
	CreateTableSQL string
}

TaskTableMeta is a dumping table metadata task

func NewTaskTableMeta

func NewTaskTableMeta(dbName, tblName, createSQL string) *TaskTableMeta

NewTaskTableMeta returns a new dumping table metadata task

func (*TaskTableMeta) Brief

func (t *TaskTableMeta) Brief() string

Brief implements task.Brief

type TaskViewMeta

type TaskViewMeta struct {
	Task
	DatabaseName   string
	ViewName       string
	CreateTableSQL string
	CreateViewSQL  string
}

TaskViewMeta is a dumping view metadata task

func NewTaskViewMeta

func NewTaskViewMeta(dbName, tblName, createTableSQL, createViewSQL string) *TaskViewMeta

NewTaskViewMeta returns a new dumping view metadata task

func (*TaskViewMeta) Brief

func (t *TaskViewMeta) Brief() string

Brief implements task.Brief

type Writer

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

Writer is the abstraction that keep pulling data from database and write to files. Every writer owns a snapshot connection, and will try to get a task from task stream chan and work on it.

func NewWriter

func NewWriter(
	tctx *tcontext.Context,
	id int64,
	config *Config,
	conn *sql.Conn,
	externalStore storage.ExternalStorage,
	metrics *metrics,
) *Writer

NewWriter returns a new Writer with given configurations

func (*Writer) WriteDatabaseMeta

func (w *Writer) WriteDatabaseMeta(db, createSQL string) error

WriteDatabaseMeta writes database meta to a file

func (*Writer) WritePolicyMeta

func (w *Writer) WritePolicyMeta(policy, createSQL string) error

WritePolicyMeta writes database meta to a file

func (*Writer) WriteSequenceMeta

func (w *Writer) WriteSequenceMeta(db, sequence, createSQL string) error

WriteSequenceMeta writes sequence meta to a file

func (*Writer) WriteTableData

func (w *Writer) WriteTableData(meta TableMeta, ir TableDataIR, currentChunk int) error

WriteTableData writes table data to a file with retry

func (*Writer) WriteTableMeta

func (w *Writer) WriteTableMeta(db, table, createSQL string) error

WriteTableMeta writes table meta to a file

func (*Writer) WriteViewMeta

func (w *Writer) WriteViewMeta(db, view, createTableSQL, createViewSQL string) error

WriteViewMeta writes view meta to a file

Jump to

Keyboard shortcuts

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