1 Star 0 Fork 0

jiangplus / tidb-parser

加入 Gitee
与超过 1200万 开发者一起发现、参与优秀开源项目,私有仓库也完全免费 :)
免费加入
克隆/下载
model.go 26.45 KB
一键复制 编辑 原始数据 按行查看 历史
123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889
// Copyright 2015 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.
package model
import (
"encoding/json"
"strconv"
"strings"
"time"
"github.com/pingcap/errors"
"github.com/pingcap/parser/auth"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/parser/types"
"github.com/pingcap/tipb/go-tipb"
)
// SchemaState is the state for schema elements.
type SchemaState byte
const (
// StateNone means this schema element is absent and can't be used.
StateNone SchemaState = iota
// StateDeleteOnly means we can only delete items for this schema element.
StateDeleteOnly
// StateWriteOnly means we can use any write operation on this schema element,
// but outer can't read the changed data.
StateWriteOnly
// StateWriteReorganization means we are re-organizing whole data after write only state.
StateWriteReorganization
// StateDeleteReorganization means we are re-organizing whole data after delete only state.
StateDeleteReorganization
// StatePublic means this schema element is ok for all write and read operations.
StatePublic
)
// String implements fmt.Stringer interface.
func (s SchemaState) String() string {
switch s {
case StateDeleteOnly:
return "delete only"
case StateWriteOnly:
return "write only"
case StateWriteReorganization:
return "write reorganization"
case StateDeleteReorganization:
return "delete reorganization"
case StatePublic:
return "public"
default:
return "none"
}
}
const (
// ColumnInfoVersion0 means the column info version is 0.
ColumnInfoVersion0 = uint64(0)
// ColumnInfoVersion1 means the column info version is 1.
ColumnInfoVersion1 = uint64(1)
// ColumnInfoVersion2 means the column info version is 2.
// This is for v2.1.7 to Compatible with older versions charset problem.
// Old version such as v2.0.8 treat utf8 as utf8mb4, because there is no UTF8 check in v2.0.8.
// After version V2.1.2 (PR#8738) , TiDB add UTF8 check, then the user upgrade from v2.0.8 insert some UTF8MB4 characters will got error.
// This is not compatibility for user. Then we try to fix this in PR #9820, and increase the version number.
ColumnInfoVersion2 = uint64(2)
// CurrLatestColumnInfoVersion means the latest column info in the current TiDB.
CurrLatestColumnInfoVersion = ColumnInfoVersion2
)
// ColumnInfo provides meta data describing of a table column.
type ColumnInfo struct {
ID int64 `json:"id"`
Name CIStr `json:"name"`
Offset int `json:"offset"`
OriginDefaultValue interface{} `json:"origin_default"`
DefaultValue interface{} `json:"default"`
DefaultValueBit []byte `json:"default_bit"`
GeneratedExprString string `json:"generated_expr_string"`
GeneratedStored bool `json:"generated_stored"`
Dependences map[string]struct{} `json:"dependences"`
types.FieldType `json:"type"`
State SchemaState `json:"state"`
Comment string `json:"comment"`
// Version means the version of the column info.
// Version = 0: For OriginDefaultValue and DefaultValue of timestamp column will stores the default time in system time zone.
// That is a bug if multiple TiDB servers in different system time zone.
// Version = 1: For OriginDefaultValue and DefaultValue of timestamp column will stores the default time in UTC time zone.
// This will fix bug in version 0. For compatibility with version 0, we add version field in column info struct.
Version uint64 `json:"version"`
}
// Clone clones ColumnInfo.
func (c *ColumnInfo) Clone() *ColumnInfo {
nc := *c
return &nc
}
// IsGenerated returns true if the column is generated column.
func (c *ColumnInfo) IsGenerated() bool {
return len(c.GeneratedExprString) != 0
}
// SetDefaultValue sets the default value.
func (c *ColumnInfo) SetDefaultValue(value interface{}) error {
c.DefaultValue = value
if c.Tp == mysql.TypeBit {
// For mysql.TypeBit type, the default value storage format must be a string.
// Other value such as int must convert to string format first.
// The mysql.TypeBit type supports the null default value.
if value == nil {
return nil
}
if v, ok := value.(string); ok {
c.DefaultValueBit = []byte(v)
return nil
}
return types.ErrInvalidDefault.GenWithStackByArgs(c.Name)
}
return nil
}
// GetDefaultValue gets the default value of the column.
// Default value use to stored in DefaultValue field, but now,
// bit type default value will store in DefaultValueBit for fix bit default value decode/encode bug.
func (c *ColumnInfo) GetDefaultValue() interface{} {
if c.Tp == mysql.TypeBit && c.DefaultValueBit != nil {
return string(c.DefaultValueBit)
}
return c.DefaultValue
}
// GetTypeDesc gets the description for column type.
func (c *ColumnInfo) GetTypeDesc() string {
desc := c.FieldType.CompactStr()
if mysql.HasUnsignedFlag(c.Flag) && c.Tp != mysql.TypeBit && c.Tp != mysql.TypeYear {
desc += " unsigned"
}
if mysql.HasZerofillFlag(c.Flag) && c.Tp != mysql.TypeYear {
desc += " zerofill"
}
return desc
}
// FindColumnInfo finds ColumnInfo in cols by name.
func FindColumnInfo(cols []*ColumnInfo, name string) *ColumnInfo {
name = strings.ToLower(name)
for _, col := range cols {
if col.Name.L == name {
return col
}
}
return nil
}
// ExtraHandleID is the column ID of column which we need to append to schema to occupy the handle's position
// for use of execution phase.
const ExtraHandleID = -1
const (
// TableInfoVersion0 means the table info version is 0.
// Upgrade from v2.1.1 or v2.1.2 to v2.1.3 and later, and then execute a "change/modify column" statement
// that does not specify a charset value for column. Then the following error may be reported:
// ERROR 1105 (HY000): unsupported modify charset from utf8mb4 to utf8.
// To eliminate this error, we will not modify the charset of this column
// when executing a change/modify column statement that does not specify a charset value for column.
// This behavior is not compatible with MySQL.
TableInfoVersion0 = uint16(0)
// TableInfoVersion1 means the table info version is 1.
// When we execute a change/modify column statement that does not specify a charset value for column,
// we set the charset of this column to the charset of table. This behavior is compatible with MySQL.
TableInfoVersion1 = uint16(1)
// TableInfoVersion2 means the table info version is 2.
// This is for v2.1.7 to Compatible with older versions charset problem.
// Old version such as v2.0.8 treat utf8 as utf8mb4, because there is no UTF8 check in v2.0.8.
// After version V2.1.2 (PR#8738) , TiDB add UTF8 check, then the user upgrade from v2.0.8 insert some UTF8MB4 characters will got error.
// This is not compatibility for user. Then we try to fix this in PR #9820, and increase the version number.
TableInfoVersion2 = uint16(2)
// TableInfoVersion3 means the table info version is 3.
// This version aims to deal with upper-cased charset name in TableInfo stored by versions prior to TiDB v2.1.9:
// TiDB always suppose all charsets / collations as lower-cased and try to convert them if they're not.
// However, the convert is missed in some scenarios before v2.1.9, so for all those tables prior to TableInfoVersion3, their
// charsets / collations will be converted to lower-case while loading from the storage.
TableInfoVersion3 = uint16(3)
// CurrLatestTableInfoVersion means the latest table info in the current TiDB.
CurrLatestTableInfoVersion = TableInfoVersion3
)
// ExtraHandleName is the name of ExtraHandle Column.
var ExtraHandleName = NewCIStr("_tidb_rowid")
// TableInfo provides meta data describing a DB table.
type TableInfo struct {
ID int64 `json:"id"`
Name CIStr `json:"name"`
Charset string `json:"charset"`
Collate string `json:"collate"`
// Columns are listed in the order in which they appear in the schema.
Columns []*ColumnInfo `json:"cols"`
Indices []*IndexInfo `json:"index_info"`
ForeignKeys []*FKInfo `json:"fk_info"`
State SchemaState `json:"state"`
PKIsHandle bool `json:"pk_is_handle"`
Comment string `json:"comment"`
AutoIncID int64 `json:"auto_inc_id"`
AutoIdCache int64 `json:"auto_id_cache"`
AutoRandID int64 `json:"auto_rand_id"`
MaxColumnID int64 `json:"max_col_id"`
MaxIndexID int64 `json:"max_idx_id"`
// UpdateTS is used to record the timestamp of updating the table's schema information.
// These changing schema operations don't include 'truncate table' and 'rename table'.
UpdateTS uint64 `json:"update_timestamp"`
// OldSchemaID :
// Because auto increment ID has schemaID as prefix,
// We need to save original schemaID to keep autoID unchanged
// while renaming a table from one database to another.
// TODO: Remove it.
// Now it only uses for compatibility with the old version that already uses this field.
OldSchemaID int64 `json:"old_schema_id,omitempty"`
// ShardRowIDBits specify if the implicit row ID is sharded.
ShardRowIDBits uint64
// MaxShardRowIDBits uses to record the max ShardRowIDBits be used so far.
MaxShardRowIDBits uint64 `json:"max_shard_row_id_bits"`
// AutoRandomBits is used to set the bit number to shard automatically when PKIsHandle.
AutoRandomBits uint64 `json:"auto_random_bits"`
// PreSplitRegions specify the pre-split region when create table.
// The pre-split region num is 2^(PreSplitRegions-1).
// And the PreSplitRegions should less than or equal to ShardRowIDBits.
PreSplitRegions uint64 `json:"pre_split_regions"`
Partition *PartitionInfo `json:"partition"`
Compression string `json:"compression"`
View *ViewInfo `json:"view"`
// Lock represent the table lock info.
Lock *TableLockInfo `json:"Lock"`
// Version means the version of the table info.
Version uint16 `json:"version"`
// TiFlashReplica means the TiFlash replica info.
TiFlashReplica *TiFlashReplicaInfo `json:"tiflash_replica"`
}
// TableLockInfo provides meta data describing a table lock.
type TableLockInfo struct {
Tp TableLockType
// Use array because there may be multiple sessions holding the same read lock.
Sessions []SessionInfo
State TableLockState
// TS is used to record the timestamp this table lock been locked.
TS uint64
}
// SessionInfo contain the session ID and the server ID.
type SessionInfo struct {
ServerID string
SessionID uint64
}
func (s SessionInfo) String() string {
return "server: " + s.ServerID + "_session: " + strconv.FormatUint(s.SessionID, 10)
}
// TableLockTpInfo is composed by schema ID, table ID and table lock type.
type TableLockTpInfo struct {
SchemaID int64
TableID int64
Tp TableLockType
}
// TableLockState is the state for table lock.
type TableLockState byte
const (
// TableLockStateNone means this table lock is absent.
TableLockStateNone TableLockState = iota
// TableLockStatePreLock means this table lock is pre-lock state. Other session doesn't hold this lock should't do corresponding operation according to the lock type.
TableLockStatePreLock
// TableLockStatePublic means this table lock is public state.
TableLockStatePublic
)
// String implements fmt.Stringer interface.
func (t TableLockState) String() string {
switch t {
case TableLockStatePreLock:
return "pre-lock"
case TableLockStatePublic:
return "public"
default:
return "none"
}
}
// TableLockType is the type of the table lock.
type TableLockType byte
const (
TableLockNone TableLockType = iota
// TableLockRead means the session with this lock can read the table (but not write it).
// Multiple sessions can acquire a READ lock for the table at the same time.
// Other sessions can read the table without explicitly acquiring a READ lock.
TableLockRead
// TableLockReadLocal is not supported.
TableLockReadLocal
// TableLockWrite means only the session with this lock has write/read permission.
// Only the session that holds the lock can access the table. No other session can access it until the lock is released.
TableLockWrite
// TableLockWriteLocal means the session with this lock has write/read permission, and the other session still has read permission.
TableLockWriteLocal
)
func (t TableLockType) String() string {
switch t {
case TableLockNone:
return "NONE"
case TableLockRead:
return "READ"
case TableLockReadLocal:
return "READ LOCAL"
case TableLockWriteLocal:
return "WRITE LOCAL"
case TableLockWrite:
return "WRITE"
}
return ""
}
// TiFlashReplicaInfo means the flash replica info.
type TiFlashReplicaInfo struct {
Count uint64
LocationLabels []string
Available bool
AvailablePartitionIDs []int64
}
// IsPartitionAvailable checks whether the partition table replica was available.
func (tr *TiFlashReplicaInfo) IsPartitionAvailable(pid int64) bool {
for _, id := range tr.AvailablePartitionIDs {
if id == pid {
return true
}
}
return false
}
// GetPartitionInfo returns the partition information.
func (t *TableInfo) GetPartitionInfo() *PartitionInfo {
if t.Partition != nil && t.Partition.Enable {
return t.Partition
}
return nil
}
// GetUpdateTime gets the table's updating time.
func (t *TableInfo) GetUpdateTime() time.Time {
return TSConvert2Time(t.UpdateTS)
}
// GetDBID returns the schema ID that is used to create an allocator.
// TODO: Remove it after removing OldSchemaID.
func (t *TableInfo) GetDBID(dbID int64) int64 {
if t.OldSchemaID != 0 {
return t.OldSchemaID
}
return dbID
}
// Clone clones TableInfo.
func (t *TableInfo) Clone() *TableInfo {
nt := *t
nt.Columns = make([]*ColumnInfo, len(t.Columns))
nt.Indices = make([]*IndexInfo, len(t.Indices))
nt.ForeignKeys = make([]*FKInfo, len(t.ForeignKeys))
for i := range t.Columns {
nt.Columns[i] = t.Columns[i].Clone()
}
for i := range t.Indices {
nt.Indices[i] = t.Indices[i].Clone()
}
for i := range t.ForeignKeys {
nt.ForeignKeys[i] = t.ForeignKeys[i].Clone()
}
return &nt
}
// GetPkName will return the pk name if pk exists.
func (t *TableInfo) GetPkName() CIStr {
for _, colInfo := range t.Columns {
if mysql.HasPriKeyFlag(colInfo.Flag) {
return colInfo.Name
}
}
return CIStr{}
}
// GetPkColInfo gets the ColumnInfo of pk if exists.
// Make sure PkIsHandle checked before call this method.
func (t *TableInfo) GetPkColInfo() *ColumnInfo {
for _, colInfo := range t.Columns {
if mysql.HasPriKeyFlag(colInfo.Flag) {
return colInfo
}
}
return nil
}
func (t *TableInfo) GetAutoIncrementColInfo() *ColumnInfo {
for _, colInfo := range t.Columns {
if mysql.HasAutoIncrementFlag(colInfo.Flag) {
return colInfo
}
}
return nil
}
func (t *TableInfo) IsAutoIncColUnsigned() bool {
col := t.GetAutoIncrementColInfo()
if col == nil {
return false
}
return mysql.HasUnsignedFlag(col.Flag)
}
// ContainsAutoRandomBits indicates whether a table contains auto_random column.
func (t *TableInfo) ContainsAutoRandomBits() bool {
return t.AutoRandomBits != 0
}
// IsAutoRandomBitColUnsigned indicates whether the auto_random column is unsigned. Make sure the table contains auto_random before calling this method.
func (t *TableInfo) IsAutoRandomBitColUnsigned() bool {
if !t.PKIsHandle || t.AutoRandomBits == 0 {
return false
}
return mysql.HasUnsignedFlag(t.GetPkColInfo().Flag)
}
// Cols returns the columns of the table in public state.
func (t *TableInfo) Cols() []*ColumnInfo {
publicColumns := make([]*ColumnInfo, len(t.Columns))
maxOffset := -1
for _, col := range t.Columns {
if col.State != StatePublic {
continue
}
publicColumns[col.Offset] = col
if maxOffset < col.Offset {
maxOffset = col.Offset
}
}
return publicColumns[0 : maxOffset+1]
}
// FindIndexByName finds index by name.
func (t *TableInfo) FindIndexByName(idxName string) *IndexInfo {
for _, idx := range t.Indices {
if idx.Name.L == idxName {
return idx
}
}
return nil
}
// IsLocked checks whether the table was locked.
func (t *TableInfo) IsLocked() bool {
return t.Lock != nil && len(t.Lock.Sessions) > 0
}
// NewExtraHandleColInfo mocks a column info for extra handle column.
func NewExtraHandleColInfo() *ColumnInfo {
colInfo := &ColumnInfo{
ID: ExtraHandleID,
Name: ExtraHandleName,
}
colInfo.Flag = mysql.PriKeyFlag
colInfo.Tp = mysql.TypeLonglong
colInfo.Flen, colInfo.Decimal = mysql.GetDefaultFieldLengthAndDecimal(mysql.TypeLonglong)
return colInfo
}
// ColumnIsInIndex checks whether c is included in any indices of t.
func (t *TableInfo) ColumnIsInIndex(c *ColumnInfo) bool {
for _, index := range t.Indices {
for _, column := range index.Columns {
if column.Name.L == c.Name.L {
return true
}
}
}
return false
}
// IsView checks if tableinfo is a view
func (t *TableInfo) IsView() bool {
return t.View != nil
}
// ViewAlgorithm is VIEW's SQL AlGORITHM characteristic.
// See https://dev.mysql.com/doc/refman/5.7/en/view-algorithms.html
type ViewAlgorithm int
const (
AlgorithmUndefined ViewAlgorithm = iota
AlgorithmMerge
AlgorithmTemptable
)
func (v *ViewAlgorithm) String() string {
switch *v {
case AlgorithmMerge:
return "MERGE"
case AlgorithmTemptable:
return "TEMPTABLE"
case AlgorithmUndefined:
return "UNDEFINED"
default:
return "UNDEFINED"
}
}
// ViewSecurity is VIEW's SQL SECURITY characteristic.
// See https://dev.mysql.com/doc/refman/5.7/en/create-view.html
type ViewSecurity int
const (
SecurityDefiner ViewSecurity = iota
SecurityInvoker
)
func (v *ViewSecurity) String() string {
switch *v {
case SecurityInvoker:
return "INVOKER"
case SecurityDefiner:
return "DEFINER"
default:
return "DEFINER"
}
}
// ViewCheckOption is VIEW's WITH CHECK OPTION clause part.
// See https://dev.mysql.com/doc/refman/5.7/en/view-check-option.html
type ViewCheckOption int
const (
CheckOptionLocal ViewCheckOption = iota
CheckOptionCascaded
)
func (v *ViewCheckOption) String() string {
switch *v {
case CheckOptionLocal:
return "LOCAL"
case CheckOptionCascaded:
return "CASCADED"
default:
return "CASCADED"
}
}
// ViewInfo provides meta data describing a DB view.
type ViewInfo struct {
Algorithm ViewAlgorithm `json:"view_algorithm"`
Definer *auth.UserIdentity `json:"view_definer"`
Security ViewSecurity `json:"view_security"`
SelectStmt string `json:"view_select"`
CheckOption ViewCheckOption `json:"view_checkoption"`
Cols []CIStr `json:"view_cols"`
}
// PartitionType is the type for PartitionInfo
type PartitionType int
// Partition types.
const (
PartitionTypeRange PartitionType = 1
PartitionTypeHash = 2
PartitionTypeList = 3
PartitionTypeKey = 4
PartitionTypeSystemTime = 5
)
func (p PartitionType) String() string {
switch p {
case PartitionTypeRange:
return "RANGE"
case PartitionTypeHash:
return "HASH"
case PartitionTypeList:
return "LIST"
case PartitionTypeKey:
return "KEY"
case PartitionTypeSystemTime:
return "SYSTEM_TIME"
default:
return ""
}
}
// PartitionInfo provides table partition info.
type PartitionInfo struct {
Type PartitionType `json:"type"`
Expr string `json:"expr"`
Columns []CIStr `json:"columns"`
// User may already creates table with partition but table partition is not
// yet supported back then. When Enable is true, write/read need use tid
// rather than pid.
Enable bool `json:"enable"`
Definitions []PartitionDefinition `json:"definitions"`
Num uint64 `json:"num"`
}
// GetNameByID gets the partition name by ID.
func (pi *PartitionInfo) GetNameByID(id int64) string {
for _, def := range pi.Definitions {
if id == def.ID {
return def.Name.L
}
}
return ""
}
// PartitionDefinition defines a single partition.
type PartitionDefinition struct {
ID int64 `json:"id"`
Name CIStr `json:"name"`
LessThan []string `json:"less_than"`
Comment string `json:"comment,omitempty"`
}
// IndexColumn provides index column info.
type IndexColumn struct {
Name CIStr `json:"name"` // Index name
Offset int `json:"offset"` // Index offset
// Length of prefix when using column prefix
// for indexing;
// UnspecifedLength if not using prefix indexing
Length int `json:"length"`
}
// Clone clones IndexColumn.
func (i *IndexColumn) Clone() *IndexColumn {
ni := *i
return &ni
}
// IndexType is the type of index
type IndexType int
// String implements Stringer interface.
func (t IndexType) String() string {
switch t {
case IndexTypeBtree:
return "BTREE"
case IndexTypeHash:
return "HASH"
default:
return ""
}
}
// IndexTypes
const (
IndexTypeInvalid IndexType = iota
IndexTypeBtree
IndexTypeHash
)
// IndexInfo provides meta data describing a DB index.
// It corresponds to the statement `CREATE INDEX Name ON Table (Column);`
// See https://dev.mysql.com/doc/refman/5.7/en/create-index.html
type IndexInfo struct {
ID int64 `json:"id"`
Name CIStr `json:"idx_name"` // Index name.
Table CIStr `json:"tbl_name"` // Table name.
Columns []*IndexColumn `json:"idx_cols"` // Index columns.
Unique bool `json:"is_unique"` // Whether the index is unique.
Primary bool `json:"is_primary"` // Whether the index is primary key.
State SchemaState `json:"state"`
Comment string `json:"comment"` // Comment
Tp IndexType `json:"index_type"` // Index type: Btree or Hash
}
// Clone clones IndexInfo.
func (index *IndexInfo) Clone() *IndexInfo {
ni := *index
ni.Columns = make([]*IndexColumn, len(index.Columns))
for i := range index.Columns {
ni.Columns[i] = index.Columns[i].Clone()
}
return &ni
}
// HasPrefixIndex returns whether any columns of this index uses prefix length.
func (index *IndexInfo) HasPrefixIndex() bool {
for _, ic := range index.Columns {
if ic.Length != types.UnspecifiedLength {
return true
}
}
return false
}
// FKInfo provides meta data describing a foreign key constraint.
type FKInfo struct {
ID int64 `json:"id"`
Name CIStr `json:"fk_name"`
RefTable CIStr `json:"ref_table"`
RefCols []CIStr `json:"ref_cols"`
Cols []CIStr `json:"cols"`
OnDelete int `json:"on_delete"`
OnUpdate int `json:"on_update"`
State SchemaState `json:"state"`
}
// Clone clones FKInfo.
func (fk *FKInfo) Clone() *FKInfo {
nfk := *fk
nfk.RefCols = make([]CIStr, len(fk.RefCols))
nfk.Cols = make([]CIStr, len(fk.Cols))
copy(nfk.RefCols, fk.RefCols)
copy(nfk.Cols, fk.Cols)
return &nfk
}
// DBInfo provides meta data describing a DB.
type DBInfo struct {
ID int64 `json:"id"` // Database ID
Name CIStr `json:"db_name"` // DB name.
Charset string `json:"charset"`
Collate string `json:"collate"`
Tables []*TableInfo `json:"-"` // Tables in the DB.
State SchemaState `json:"state"`
}
// Clone clones DBInfo.
func (db *DBInfo) Clone() *DBInfo {
newInfo := *db
newInfo.Tables = make([]*TableInfo, len(db.Tables))
for i := range db.Tables {
newInfo.Tables[i] = db.Tables[i].Clone()
}
return &newInfo
}
// Copy shallow copies DBInfo.
func (db *DBInfo) Copy() *DBInfo {
newInfo := *db
newInfo.Tables = make([]*TableInfo, len(db.Tables))
copy(newInfo.Tables, db.Tables)
return &newInfo
}
// CIStr is case insensitive string.
type CIStr struct {
O string `json:"O"` // Original string.
L string `json:"L"` // Lower case string.
}
// String implements fmt.Stringer interface.
func (cis CIStr) String() string {
return cis.O
}
// NewCIStr creates a new CIStr.
func NewCIStr(s string) (cs CIStr) {
cs.O = s
cs.L = strings.ToLower(s)
return
}
// UnmarshalJSON implements the user defined unmarshal method.
// CIStr can be unmarshaled from a single string, so PartitionDefinition.Name
// in this change https://github.com/pingcap/tidb/pull/6460/files would be
// compatible during TiDB upgrading.
func (cis *CIStr) UnmarshalJSON(b []byte) error {
type T CIStr
if err := json.Unmarshal(b, (*T)(cis)); err == nil {
return nil
}
// Unmarshal CIStr from a single string.
err := json.Unmarshal(b, &cis.O)
if err != nil {
return errors.Trace(err)
}
cis.L = strings.ToLower(cis.O)
return nil
}
// ColumnsToProto converts a slice of model.ColumnInfo to a slice of tipb.ColumnInfo.
func ColumnsToProto(columns []*ColumnInfo, pkIsHandle bool) []*tipb.ColumnInfo {
cols := make([]*tipb.ColumnInfo, 0, len(columns))
for _, c := range columns {
col := ColumnToProto(c)
// TODO: Here `PkHandle`'s meaning is changed, we will change it to `IsHandle` when tikv's old select logic
// is abandoned.
if (pkIsHandle && mysql.HasPriKeyFlag(c.Flag)) || c.ID == ExtraHandleID {
col.PkHandle = true
} else {
col.PkHandle = false
}
cols = append(cols, col)
}
return cols
}
// IndexToProto converts a model.IndexInfo to a tipb.IndexInfo.
func IndexToProto(t *TableInfo, idx *IndexInfo) *tipb.IndexInfo {
pi := &tipb.IndexInfo{
TableId: t.ID,
IndexId: idx.ID,
Unique: idx.Unique,
}
cols := make([]*tipb.ColumnInfo, 0, len(idx.Columns)+1)
for _, c := range idx.Columns {
cols = append(cols, ColumnToProto(t.Columns[c.Offset]))
}
if t.PKIsHandle {
// Coprocessor needs to know PKHandle column info, so we need to append it.
for _, col := range t.Columns {
if mysql.HasPriKeyFlag(col.Flag) {
colPB := ColumnToProto(col)
colPB.PkHandle = true
cols = append(cols, colPB)
break
}
}
}
pi.Columns = cols
return pi
}
// ColumnToProto converts model.ColumnInfo to tipb.ColumnInfo.
func ColumnToProto(c *ColumnInfo) *tipb.ColumnInfo {
pc := &tipb.ColumnInfo{
ColumnId: c.ID,
Collation: collationToProto(c.FieldType.Collate),
ColumnLen: int32(c.FieldType.Flen),
Decimal: int32(c.FieldType.Decimal),
Flag: int32(c.Flag),
Elems: c.Elems,
}
pc.Tp = int32(c.FieldType.Tp)
return pc
}
// TODO: update it when more collate is supported.
func collationToProto(c string) int32 {
v := mysql.CollationNames[c]
if v == mysql.BinaryDefaultCollationID {
return int32(mysql.BinaryDefaultCollationID)
}
// We only support binary and utf8_bin collation.
// Setting other collations to utf8_bin for old data compatibility.
// For the data created when we didn't enforce utf8_bin collation in create table.
return int32(mysql.DefaultCollationID)
}
// TableColumnID is composed by table ID and column ID.
type TableColumnID struct {
TableID int64
ColumnID int64
}
1
https://gitee.com/jiangplus/tidb-parser.git
git@gitee.com:jiangplus/tidb-parser.git
jiangplus
tidb-parser
tidb-parser
v3.1.2

搜索帮助

53164aa7 5694891 3bd8fe86 5694891