Skip to content

Commit

Permalink
*: use model.CIStr in infoschema v2 to handle case insensitive (pingc…
Browse files Browse the repository at this point in the history
  • Loading branch information
tiancaiamao authored Aug 19, 2024
1 parent db54a3b commit f751f73
Show file tree
Hide file tree
Showing 17 changed files with 57 additions and 57 deletions.
2 changes: 1 addition & 1 deletion pkg/ddl/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -417,7 +417,7 @@ func (e *executor) getPendingTiFlashTableCount(originVersion int64, pendingCount
cnt := uint32(0)
dbs := is.ListTablesWithSpecialAttribute(infoschema.TiFlashAttribute)
for _, db := range dbs {
if util.IsMemOrSysDB(db.DBName) {
if util.IsMemOrSysDB(db.DBName.L) {
continue
}
for _, tbl := range db.TableInfos {
Expand Down
2 changes: 1 addition & 1 deletion pkg/executor/coprocessor.go
Original file line number Diff line number Diff line change
Expand Up @@ -256,7 +256,7 @@ func (h *CoprocessorDAGHandler) buildStreamResponse(chunk *tipb.Chunk) *coproces

func (*CoprocessorDAGHandler) buildErrorResponse(err error) *coprocessor.Response {
return &coprocessor.Response{
OtherError: err.Error(),
OtherError: errors.ErrorStack(err),
}
}

Expand Down
4 changes: 2 additions & 2 deletions pkg/executor/infoschema_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -2580,7 +2580,7 @@ func (e *memtableRetriever) dataForTableTiFlashReplica(_ context.Context, sctx s
rs := e.is.ListTablesWithSpecialAttribute(infoschema.TiFlashAttribute)
for _, schema := range rs {
for _, tbl := range schema.TableInfos {
if checker != nil && !checker.RequestVerification(sctx.GetSessionVars().ActiveRoles, schema.DBName, tbl.Name.L, "", mysql.AllPrivMask) {
if checker != nil && !checker.RequestVerification(sctx.GetSessionVars().ActiveRoles, schema.DBName.L, tbl.Name.L, "", mysql.AllPrivMask) {
continue
}
var progress float64
Expand All @@ -2603,7 +2603,7 @@ func (e *memtableRetriever) dataForTableTiFlashReplica(_ context.Context, sctx s
progressString := types.TruncateFloatToString(progress, 2)
progress, _ = strconv.ParseFloat(progressString, 64)
record := types.MakeDatums(
schema.DBName, // TABLE_SCHEMA
schema.DBName.O, // TABLE_SCHEMA
tbl.Name.O, // TABLE_NAME
tbl.ID, // TABLE_ID
int64(tbl.TiFlashReplica.Count), // REPLICA_COUNT
Expand Down
8 changes: 4 additions & 4 deletions pkg/infoschema/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -954,9 +954,9 @@ func (b *Builder) createSchemaTablesForDB(di *model.DBInfo, tableFromMeta tableF
if b.enableV2 {
for name, id := range di.TableName2ID {
item := tableItem{
dbName: di.Name.L,
dbName: di.Name,
dbID: di.ID,
tableName: name,
tableName: model.NewCIStr(name),
tableID: id,
schemaVersion: schemaVersion,
}
Expand Down Expand Up @@ -984,9 +984,9 @@ func (b *Builder) addDB(schemaVersion int64, di *model.DBInfo, schTbls *schemaTa
func (b *Builder) addTable(schemaVersion int64, di *model.DBInfo, tblInfo *model.TableInfo, tbl table.Table) {
if b.enableV2 {
b.infoData.add(tableItem{
dbName: di.Name.L,
dbName: di.Name,
dbID: di.ID,
tableName: tblInfo.Name.L,
tableName: tblInfo.Name,
tableID: tblInfo.ID,
schemaVersion: schemaVersion,
}, tbl)
Expand Down
2 changes: 1 addition & 1 deletion pkg/infoschema/builder_misc.go
Original file line number Diff line number Diff line change
Expand Up @@ -118,7 +118,7 @@ func (b *Builder) initMisc(dbInfos []*model.DBInfo, policies []*model.PolicyInfo
rs := b.ListTablesWithSpecialAttribute(ForeignKeysAttribute)
for _, db := range rs {
for _, tbl := range db.TableInfos {
info.addReferredForeignKeys(model.NewCIStr(db.DBName), tbl)
info.addReferredForeignKeys(db.DBName, tbl)
}
}
return
Expand Down
4 changes: 2 additions & 2 deletions pkg/infoschema/infoschema.go
Original file line number Diff line number Diff line change
Expand Up @@ -351,14 +351,14 @@ func (is *infoSchema) SchemaSimpleTableInfos(ctx stdctx.Context, schema model.CI
}

type tableInfoResult struct {
DBName string
DBName model.CIStr
TableInfos []*model.TableInfo
}

func (is *infoSchema) ListTablesWithSpecialAttribute(filter specialAttributeFilter) []tableInfoResult {
ret := make([]tableInfoResult, 0, 10)
for _, dbName := range is.AllSchemaNames() {
res := tableInfoResult{DBName: dbName.O}
res := tableInfoResult{DBName: dbName}
tblInfos, err := is.SchemaTableInfos(stdctx.Background(), dbName)
terror.Log(err)
for _, tblInfo := range tblInfos {
Expand Down
52 changes: 26 additions & 26 deletions pkg/infoschema/infoschema_v2.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,9 +42,9 @@ import (

// tableItem is the btree item sorted by name or by id.
type tableItem struct {
dbName string
dbName model.CIStr
dbID int64
tableName string
tableName model.CIStr
tableID int64
schemaVersion int64
tomb bool
Expand All @@ -59,7 +59,7 @@ type schemaItem struct {
type schemaIDName struct {
schemaVersion int64
id int64
name string
name model.CIStr
tomb bool
}

Expand Down Expand Up @@ -123,7 +123,7 @@ type Data struct {
}

type tableInfoItem struct {
dbName string
dbName model.CIStr
tableID int64
schemaVersion int64
tableInfo *model.TableInfo
Expand Down Expand Up @@ -228,7 +228,7 @@ func (isd *Data) addSpecialDB(di *model.DBInfo, tables *schemaTables) {

func (isd *Data) addDB(schemaVersion int64, dbInfo *model.DBInfo) {
dbInfo.Deprecated.Tables = nil
isd.schemaID2Name.Set(schemaIDName{schemaVersion: schemaVersion, id: dbInfo.ID, name: dbInfo.Name.O})
isd.schemaID2Name.Set(schemaIDName{schemaVersion: schemaVersion, id: dbInfo.ID, name: dbInfo.Name})
isd.schemaMap.Set(schemaItem{schemaVersion: schemaVersion, dbInfo: dbInfo})
}

Expand All @@ -248,7 +248,7 @@ func (isd *Data) remove(item tableItem) {
func (isd *Data) deleteDB(dbInfo *model.DBInfo, schemaVersion int64) {
item := schemaItem{schemaVersion: schemaVersion, dbInfo: dbInfo, tomb: true}
isd.schemaMap.Set(item)
isd.schemaID2Name.Set(schemaIDName{schemaVersion: schemaVersion, id: dbInfo.ID, name: dbInfo.Name.O, tomb: true})
isd.schemaID2Name.Set(schemaIDName{schemaVersion: schemaVersion, id: dbInfo.ID, name: dbInfo.Name, tomb: true})
}

// resetBeforeFullLoad is called before a full recreate operation within builder.InitWithDBInfos().
Expand Down Expand Up @@ -463,28 +463,28 @@ func compareByID(a, b tableItem) bool {
}

func compareByName(a, b tableItem) bool {
if a.dbName < b.dbName {
if a.dbName.L < b.dbName.L {
return true
}
if a.dbName > b.dbName {
if a.dbName.L > b.dbName.L {
return false
}

if a.tableName < b.tableName {
if a.tableName.L < b.tableName.L {
return true
}
if a.tableName > b.tableName {
if a.tableName.L > b.tableName.L {
return false
}

return a.schemaVersion < b.schemaVersion
}

func compareTableInfoItem(a, b tableInfoItem) bool {
if a.dbName < b.dbName {
if a.dbName.L < b.dbName.L {
return true
}
if a.dbName > b.dbName {
if a.dbName.L > b.dbName.L {
return false
}

Expand Down Expand Up @@ -611,9 +611,9 @@ func (is *infoschemaV2) tableByID(ctx context.Context, id int64, noRefill bool)
}

if isTableVirtual(id) {
if raw, exist := is.Data.specials.Load(itm.dbName); exist {
if raw, exist := is.Data.specials.Load(itm.dbName.L); exist {
schTbls := raw.(*schemaTables)
val, ok = schTbls.tables[itm.tableName]
val, ok = schTbls.tables[itm.tableName.L]
return
}
return nil, false
Expand Down Expand Up @@ -648,7 +648,7 @@ func IsSpecialDB(dbName string) bool {
}

// EvictTable is exported for testing only.
func (is *infoschemaV2) EvictTable(schema, tbl string) {
func (is *infoschemaV2) EvictTable(schema, tbl model.CIStr) {
eq := func(a, b *tableItem) bool { return a.dbName == b.dbName && a.tableName == b.tableName }
itm, ok := search(is.byName, is.infoSchema.schemaMetaVersion, tableItem{dbName: schema, tableName: tbl, schemaVersion: math.MaxInt64}, eq)
if !ok {
Expand All @@ -666,7 +666,7 @@ type tableByNameHelper struct {
}

func (h *tableByNameHelper) onItem(item tableItem) bool {
if item.dbName != h.end.dbName || item.tableName != h.end.tableName {
if item.dbName.L != h.end.dbName.L || item.tableName.L != h.end.tableName.L {
h.found = false
return false
}
Expand Down Expand Up @@ -694,7 +694,7 @@ func (is *infoschemaV2) TableByName(ctx context.Context, schema, tbl model.CIStr
start := time.Now()

var h tableByNameHelper
h.end = tableItem{dbName: schema.L, tableName: tbl.L, schemaVersion: math.MaxInt64}
h.end = tableItem{dbName: schema, tableName: tbl, schemaVersion: math.MaxInt64}
h.schemaVersion = is.infoSchema.schemaMetaVersion
is.byName.Descend(h.end, h.onItem)

Expand Down Expand Up @@ -800,8 +800,8 @@ func (is *infoschemaV2) SchemaSimpleTableInfos(ctx context.Context, schema model
// Ascend is much more difficult than Descend.
// So the data is taken out first and then dedup in Descend order.
var tableItems []tableItem
is.byName.Ascend(tableItem{dbName: schema.L}, func(item tableItem) bool {
if item.dbName != schema.L {
is.byName.Ascend(tableItem{dbName: schema}, func(item tableItem) bool {
if item.dbName.L != schema.L {
return false
}
if is.infoSchema.schemaMetaVersion >= item.schemaVersion {
Expand All @@ -821,7 +821,7 @@ func (is *infoschemaV2) SchemaSimpleTableInfos(ctx context.Context, schema model
if !item.tomb {
tblInfos = append(tblInfos, &model.TableNameInfo{
ID: item.tableID,
Name: model.NewCIStr(item.tableName),
Name: item.tableName,
})
}
}
Expand Down Expand Up @@ -986,7 +986,7 @@ func (is *infoschemaV2) SchemaByID(id int64) (*model.DBInfo, bool) {
return st.dbInfo, true
}
var ok bool
var name string
var name model.CIStr
is.Data.schemaID2Name.Descend(schemaIDName{
id: id,
schemaVersion: math.MaxInt64,
Expand All @@ -1007,7 +1007,7 @@ func (is *infoschemaV2) SchemaByID(id int64) (*model.DBInfo, bool) {
if !ok {
return nil, false
}
return is.SchemaByName(model.NewCIStr(name))
return is.SchemaByName(name)
}

func (is *infoschemaV2) loadTableInfo(ctx context.Context, tblID, dbID int64, ts uint64, schemaVersion int64) (table.Table, error) {
Expand Down Expand Up @@ -1272,9 +1272,9 @@ func (b *Builder) applyDropTableV2(diff *model.SchemaDiff, dbInfo *model.DBInfo,
}

b.infoData.remove(tableItem{
dbName: dbInfo.Name.L,
dbName: dbInfo.Name,
dbID: dbInfo.ID,
tableName: tblInfo.Name.L,
tableName: tblInfo.Name,
tableID: tblInfo.ID,
schemaVersion: diff.Version,
})
Expand Down Expand Up @@ -1439,10 +1439,10 @@ func (is *infoschemaV2) ListTablesWithSpecialAttribute(filter specialAttributeFi
}

if currDB == "" {
currDB = item.dbName
currDB = item.dbName.L
res = tableInfoResult{DBName: item.dbName}
res.TableInfos = append(res.TableInfos, item.tableInfo)
} else if currDB == item.dbName {
} else if currDB == item.dbName.L {
res.TableInfos = append(res.TableInfos, item.tableInfo)
} else {
ret = append(ret, res)
Expand Down
2 changes: 1 addition & 1 deletion pkg/infoschema/infoschema_v2_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@ func TestV2Basic(t *testing.T) {
internal.AddDB(t, r.Store(), dbInfo)
tblInfo := internal.MockTableInfo(t, r.Store(), tableName.O)
tblInfo.DBID = dbInfo.ID
is.Data.add(tableItem{schemaName.L, dbInfo.ID, tableName.L, tblInfo.ID, 2, false}, internal.MockTable(t, r.Store(), tblInfo))
is.Data.add(tableItem{schemaName, dbInfo.ID, tableName, tblInfo.ID, 2, false}, internal.MockTable(t, r.Store(), tblInfo))
internal.AddTable(t, r.Store(), dbInfo, tblInfo)
is.base().schemaMetaVersion = 1
require.Equal(t, 1, len(is.AllSchemas()))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -403,7 +403,7 @@ func TestStmtSummaryEvictedCountTable(t *testing.T) {

err := tk.QueryToErr("select * from information_schema.CLUSTER_STATEMENTS_SUMMARY_EVICTED")
// This error is come from cop(TiDB) fetch from rpc server.
require.EqualError(t, err, "other error: [planner:1227]Access denied; you need (at least one of) the PROCESS privilege(s) for this operation")
require.ErrorContains(t, err, "other error: [planner:1227]Access denied; you need (at least one of) the PROCESS privilege(s) for this operation")

require.NoError(t, tk.Session().Auth(&auth.UserIdentity{
Username: "testuser2",
Expand Down
8 changes: 4 additions & 4 deletions pkg/infoschema/test/infoschemav2test/v2_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -232,7 +232,7 @@ func checkResult(t *testing.T, tk *testkit.TestKit, result ...string) {
var rows []string
for _, v := range ch {
for _, tblInfo := range v.TableInfos {
rows = append(rows, v.DBName+" "+tblInfo.Name.L)
rows = append(rows, v.DBName.L+" "+tblInfo.Name.L)
}
}
slices.SortFunc(rows, strings.Compare)
Expand Down Expand Up @@ -278,7 +278,7 @@ func TestUnrelatedDDLTriggerReload(t *testing.T) {
is := dom.InfoSchema()
ok, v2 := infoschema.IsV2(is)
require.True(t, ok)
v2.EvictTable("test", "t1")
v2.EvictTable(model.NewCIStr("test"), model.NewCIStr("t1"))

tk.MustExec("create table t2 (id int)")

Expand Down Expand Up @@ -309,7 +309,7 @@ func TestTrace(t *testing.T) {
require.True(t, ok)

// Evict the table cache and check the trace information can catch this calling.
raw.EvictTable("test", "t_trace")
raw.EvictTable(model.NewCIStr("test"), model.NewCIStr("t_trace"))
tk.MustQuery("trace select * from information_schema.tables where table_schema='test' and table_name='t_trace'").CheckContain("infoschema.loadTableInfo")
}

Expand All @@ -326,7 +326,7 @@ func TestCachedTable(t *testing.T) {
require.True(t, ok)

// Cover a case that after cached table evict and load, table.Table goes wrong.
raw.EvictTable("test", "t_cache")
raw.EvictTable(model.NewCIStr("test"), model.NewCIStr("t_cache"))
tk.MustExec("insert into t_cache values (2)") // no panic here
tk.MustQuery("select * from t_cache").Check(testkit.Rows("1", "2"))
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/meta/meta.go
Original file line number Diff line number Diff line change
Expand Up @@ -1042,7 +1042,7 @@ func IsTableInfoMustLoad(json []byte) bool {
}

// NameExtractRegexp is exported for testing.
const NameExtractRegexp = `"L":"([^"\\]*(?:\\.[^"\\]*)*)"}`
const NameExtractRegexp = `"O":"([^"\\]*(?:\\.[^"\\]*)*)",`

// Unescape is exported for testing.
func Unescape(s string) string {
Expand Down Expand Up @@ -1179,7 +1179,7 @@ func FastUnmarshalTableNameInfo(data []byte) (*model.TableNameInfo, error) {
if !ok {
return nil, errors.New("name field not found in JSON")
}
// 6 tokens; {, O, ..., L, ..., }
// 6 tokens; {, O, ..., L, ..., }, the data looks like this {123,"O","t","L","t",125}
if len(nameTokens) != 6 {
return nil, errors.Errorf("unexpected name field in JSON, %v", nameTokens)
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/store/copr/coprocessor.go
Original file line number Diff line number Diff line change
Expand Up @@ -1463,7 +1463,7 @@ func (worker *copIteratorWorker) handleCopResponse(bo *Backoffer, rpcCtx *tikv.R
zap.ByteString("firstRangeStartKey", firstRangeStartKey),
zap.ByteString("lastRangeEndKey", lastRangeEndKey),
zap.String("storeAddr", task.storeAddr),
zap.Error(err))
zap.String("error", otherErr))
if strings.Contains(err.Error(), "write conflict") {
return nil, kv.ErrWriteConflict.FastGen("%s", otherErr)
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/ttl/cache/infoschema.go
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,7 @@ func (isc *InfoSchemaCache) Update(se session.Session) error {
if tblInfo.TTLInfo == nil || !tblInfo.TTLInfo.Enable || tblInfo.State != model.StatePublic {
continue
}
dbName := model.NewCIStr(v.DBName)
dbName := v.DBName
logger := logutil.BgLogger().
With(zap.String("schema", dbName.L),
zap.Int64("tableID", tblInfo.ID), zap.String("tableName", tblInfo.Name.L))
Expand Down
2 changes: 1 addition & 1 deletion pkg/ttl/ttlworker/job_manager.go
Original file line number Diff line number Diff line change
Expand Up @@ -1082,7 +1082,7 @@ GROUP BY
if err != nil {
logutil.Logger(ctx).Error("failed to get table's job interval",
zap.Error(err),
zap.String("db", v.DBName),
zap.String("db", v.DBName.O),
zap.String("table", tblInfo.Name.String()),
)
interval = time.Hour
Expand Down
2 changes: 1 addition & 1 deletion pkg/ttl/ttlworker/timer_sync.go
Original file line number Diff line number Diff line change
Expand Up @@ -188,7 +188,7 @@ func (g *TTLTimersSyncer) SyncTimers(ctx context.Context, is infoschema.InfoSche
ch := is.ListTablesWithSpecialAttribute(infoschema.TTLAttribute)
for _, v := range ch {
for _, tblInfo := range v.TableInfos {
for _, key := range g.syncTimersForTable(ctx, se, model.NewCIStr(v.DBName), tblInfo) {
for _, key := range g.syncTimersForTable(ctx, se, v.DBName, tblInfo) {
currentTimerKeys[key] = struct{}{}
}
}
Expand Down
Loading

0 comments on commit f751f73

Please sign in to comment.