Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion br/pkg/lightning/backend/kv/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -428,7 +428,7 @@ func (se *Session) Value(key fmt.Stringer) any {
func (*Session) StmtAddDirtyTableOP(_ int, _ int64, _ kv.Handle) {}

// GetInfoSchema implements the sessionctx.Context interface.
func (*Session) GetInfoSchema() infoschema.InfoSchemaMetaVersion {
func (*Session) GetInfoSchema() infoschema.MetaOnlyInfoSchema {
return nil
}

Expand Down
9 changes: 5 additions & 4 deletions pkg/ddl/schematracker/info_store.go
Original file line number Diff line number Diff line change
Expand Up @@ -152,32 +152,33 @@ func (i *InfoStore) AllTableNamesOfSchema(schema model.CIStr) ([]string, error)

// InfoStoreAdaptor convert InfoStore to InfoSchema, it only implements a part of InfoSchema interface to be
// used by DDL interface.
// nolint:unused
type InfoStoreAdaptor struct {
infoschema.InfoSchema
inner *InfoStore
}

// SchemaByName implements the InfoSchema interface.
// nolint:unused
func (i InfoStoreAdaptor) SchemaByName(schema model.CIStr) (*model.DBInfo, bool) {
dbInfo := i.inner.SchemaByName(schema)
return dbInfo, dbInfo != nil
}

// TableExists implements the InfoSchema interface.
// nolint:unused
func (i InfoStoreAdaptor) TableExists(schema, table model.CIStr) bool {
tableInfo, _ := i.inner.TableByName(schema, table)
return tableInfo != nil
}

// TableByName implements the InfoSchema interface.
// nolint:unused
func (i InfoStoreAdaptor) TableByName(schema, table model.CIStr) (t table.Table, err error) {
tableInfo, err := i.inner.TableByName(schema, table)
if err != nil {
return nil, err
}
return tables.MockTableFromMeta(tableInfo), nil
}

// TableInfoByName implements the InfoSchema interface.
func (i InfoStoreAdaptor) TableInfoByName(schema, table model.CIStr) (*model.TableInfo, error) {
return i.inner.TableByName(schema, table)
}
2 changes: 1 addition & 1 deletion pkg/expression/builtin_info.go
Original file line number Diff line number Diff line change
Expand Up @@ -909,7 +909,7 @@ func (c *tidbDecodeKeyFunctionClass) getFunction(ctx BuildContext, args []Expres
}

// DecodeKeyFromString is used to decode key by expressions
var DecodeKeyFromString func(types.Context, infoschema.InfoSchemaMetaVersion, string) string
var DecodeKeyFromString func(types.Context, infoschema.MetaOnlyInfoSchema, string) string

type builtinTiDBDecodeKeySig struct {
baseBuiltinFunc
Expand Down
2 changes: 1 addition & 1 deletion pkg/expression/builtin_info_vec.go
Original file line number Diff line number Diff line change
Expand Up @@ -408,7 +408,7 @@ func (b *builtinTiDBDecodeKeySig) vecEvalString(ctx EvalContext, input *chunk.Ch

decode := DecodeKeyFromString
if decode == nil {
decode = func(_ types.Context, _ infoschema.InfoSchemaMetaVersion, s string) string {
decode = func(_ types.Context, _ infoschema.MetaOnlyInfoSchema, s string) string {
return s
}
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/expression/contextimpl/sessionctx.go
Original file line number Diff line number Diff line change
Expand Up @@ -215,7 +215,7 @@ func currentUserProp(sctx sessionctx.Context) exprctx.OptionalEvalPropProvider {
}

func infoSchemaProp(sctx sessionctx.Context) contextopt.InfoSchemaPropProvider {
return func(isDomain bool) infoschema.InfoSchemaMetaVersion {
return func(isDomain bool) infoschema.MetaOnlyInfoSchema {
if isDomain {
return sctx.GetDomainInfoSchema()
}
Expand Down
6 changes: 3 additions & 3 deletions pkg/expression/contextopt/infoschema.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ import (
)

// InfoSchemaPropProvider is the function to provide information schema.
type InfoSchemaPropProvider func(isDomain bool) infoschema.InfoSchemaMetaVersion
type InfoSchemaPropProvider func(isDomain bool) infoschema.MetaOnlyInfoSchema

// Desc returns the description for the property key.
func (InfoSchemaPropProvider) Desc() *context.OptionalEvalPropDesc {
Expand All @@ -36,7 +36,7 @@ func (InfoSchemaPropReader) RequiredOptionalEvalProps() context.OptionalEvalProp
}

// GetSessionInfoSchema returns session information schema.
func (InfoSchemaPropReader) GetSessionInfoSchema(ctx context.EvalContext) (infoschema.InfoSchemaMetaVersion, error) {
func (InfoSchemaPropReader) GetSessionInfoSchema(ctx context.EvalContext) (infoschema.MetaOnlyInfoSchema, error) {
p, err := getPropProvider[InfoSchemaPropProvider](ctx, context.OptPropInfoSchema)
if err != nil {
return nil, err
Expand All @@ -45,7 +45,7 @@ func (InfoSchemaPropReader) GetSessionInfoSchema(ctx context.EvalContext) (infos
}

// GetDomainInfoSchema return domain information schema.
func (InfoSchemaPropReader) GetDomainInfoSchema(ctx context.EvalContext) (infoschema.InfoSchemaMetaVersion, error) {
func (InfoSchemaPropReader) GetDomainInfoSchema(ctx context.EvalContext) (infoschema.MetaOnlyInfoSchema, error) {
p, err := getPropProvider[InfoSchemaPropProvider](ctx, context.OptPropInfoSchema)
if err != nil {
return nil, err
Expand Down
4 changes: 2 additions & 2 deletions pkg/expression/contextopt/optional_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -105,10 +105,10 @@ func TestOptionalEvalPropProviders(t *testing.T) {
}
case context.OptPropInfoSchema:
type mockIsType struct {
infoschema.InfoSchemaMetaVersion
infoschema.MetaOnlyInfoSchema
}
var is1, is2 mockIsType
p = InfoSchemaPropProvider(func(isDomain bool) infoschema.InfoSchemaMetaVersion {
p = InfoSchemaPropProvider(func(isDomain bool) infoschema.MetaOnlyInfoSchema {
if isDomain {
return &is1
}
Expand Down
1 change: 1 addition & 0 deletions pkg/infoschema/context/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -5,4 +5,5 @@ go_library(
srcs = ["infoschema.go"],
importpath = "github.com/pingcap/tidb/pkg/infoschema/context",
visibility = ["//visibility:public"],
deps = ["//pkg/parser/model"],
)
21 changes: 16 additions & 5 deletions pkg/infoschema/context/infoschema.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,10 +14,21 @@

package context

// InfoSchemaMetaVersion is a workaround. Due to circular dependency,
// can not return the complete interface. But SchemaMetaVersion is widely used for logging.
// So we give a convenience for that.
// FIXME: remove this interface
type InfoSchemaMetaVersion interface {
import "github.com/pingcap/tidb/pkg/parser/model"

// MetaOnlyInfoSchema is a workaround.
// Due to circular dependency cannot return the complete interface.
// But MetaOnlyInfoSchema is widely used for scenes that require meta only, so we give a convenience for that.
type MetaOnlyInfoSchema interface {
SchemaMetaVersion() int64
SchemaByName(schema model.CIStr) (*model.DBInfo, bool)
SchemaExists(schema model.CIStr) bool
TableInfoByName(schema, table model.CIStr) (*model.TableInfo, error)
TableInfoByID(id int64) (*model.TableInfo, bool)
FindTableInfoByPartitionID(partitionID int64) (*model.TableInfo, *model.DBInfo, *model.PartitionDefinition)
TableExists(schema, table model.CIStr) bool
SchemaByID(id int64) (*model.DBInfo, bool)
AllSchemas() []*model.DBInfo
AllSchemaNames() []model.CIStr
SchemaTableInfos(schema model.CIStr) []*model.TableInfo
}
68 changes: 66 additions & 2 deletions pkg/infoschema/infoschema.go
Original file line number Diff line number Diff line change
Expand Up @@ -177,6 +177,12 @@ func (is *infoSchema) TableByName(schema, table model.CIStr) (t table.Table, err
return nil, ErrTableNotExists.GenWithStackByArgs(schema, table)
}

// TableInfoByName implements InfoSchema.TableInfoByName
func (is *infoSchema) TableInfoByName(schema, table model.CIStr) (*model.TableInfo, error) {
tbl, err := is.TableByName(schema, table)
return getTableInfo(tbl), err
}

// TableIsView indicates whether the schema.table is a view.
func TableIsView(is InfoSchema, schema, table model.CIStr) bool {
tbl, err := is.TableByName(schema, table)
Expand Down Expand Up @@ -240,6 +246,25 @@ func (is *infoSchema) TableByID(id int64) (val table.Table, ok bool) {
return slice[idx], true
}

// TableInfoByID implements InfoSchema.TableInfoByID
func (is *infoSchema) TableInfoByID(id int64) (*model.TableInfo, bool) {
tbl, ok := is.TableByID(id)
return getTableInfo(tbl), ok
}

// FindTableInfoByPartitionID implements InfoSchema.FindTableInfoByPartitionID
func (is *infoSchema) FindTableInfoByPartitionID(
partitionID int64,
) (*model.TableInfo, *model.DBInfo, *model.PartitionDefinition) {
tbl, db, partDef := is.FindTableByPartitionID(partitionID)
return getTableInfo(tbl), db, partDef
}

// SchemaTableInfos implements InfoSchema.FindTableInfoByPartitionID
func (is *infoSchema) SchemaTableInfos(schema model.CIStr) []*model.TableInfo {
return getTableInfoList(is.SchemaTables(schema))
}

// allocByID returns the Allocators of a table.
func allocByID(is InfoSchema, id int64) (autoid.Allocators, bool) {
tbl, ok := is.TableByID(id)
Expand Down Expand Up @@ -351,10 +376,10 @@ func init() {
Tables: infoSchemaTables,
}
RegisterVirtualTable(infoSchemaDB, createInfoSchemaTable)
util.GetSequenceByName = func(is context.InfoSchemaMetaVersion, schema, sequence model.CIStr) (util.SequenceTable, error) {
util.GetSequenceByName = func(is context.MetaOnlyInfoSchema, schema, sequence model.CIStr) (util.SequenceTable, error) {
return GetSequenceByName(is.(InfoSchema), schema, sequence)
}
mock.MockInfoschema = func(tbList []*model.TableInfo) context.InfoSchemaMetaVersion {
mock.MockInfoschema = func(tbList []*model.TableInfo) context.MetaOnlyInfoSchema {
return MockInfoSchema(tbList)
}
}
Expand Down Expand Up @@ -666,6 +691,26 @@ func (ts *SessionExtendedInfoSchema) TableByName(schema, table model.CIStr) (tab
return ts.InfoSchema.TableByName(schema, table)
}

// TableInfoByName implements InfoSchema.TableInfoByName
func (ts *SessionExtendedInfoSchema) TableInfoByName(schema, table model.CIStr) (*model.TableInfo, error) {
tbl, err := ts.TableByName(schema, table)
return getTableInfo(tbl), err
}

// TableInfoByID implements InfoSchema.TableInfoByID
func (ts *SessionExtendedInfoSchema) TableInfoByID(id int64) (*model.TableInfo, bool) {
tbl, ok := ts.TableByID(id)
return getTableInfo(tbl), ok
}

// FindTableInfoByPartitionID implements InfoSchema.FindTableInfoByPartitionID
func (ts *SessionExtendedInfoSchema) FindTableInfoByPartitionID(
partitionID int64,
) (*model.TableInfo, *model.DBInfo, *model.PartitionDefinition) {
tbl, db, partDef := ts.FindTableByPartitionID(partitionID)
return getTableInfo(tbl), db, partDef
}

// TableByID implements InfoSchema.TableByID
func (ts *SessionExtendedInfoSchema) TableByID(id int64) (table.Table, bool) {
if ts.LocalTemporaryTables != nil {
Expand Down Expand Up @@ -739,3 +784,22 @@ func FindTableByTblOrPartID(is InfoSchema, id int64) (table.Table, *model.Partit
tbl, _, partDef := is.FindTableByPartitionID(id)
return tbl, partDef
}

func getTableInfo(tbl table.Table) *model.TableInfo {
if tbl == nil {
return nil
}
return tbl.Meta()
}

func getTableInfoList(tables []table.Table) []*model.TableInfo {
if tables == nil {
return nil
}

infoLost := make([]*model.TableInfo, 0, len(tables))
for _, tbl := range tables {
infoLost = append(infoLost, tbl.Meta())
}
return infoLost
}
60 changes: 60 additions & 0 deletions pkg/infoschema/infoschema_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -157,23 +157,46 @@ func TestBasic(t *testing.T) {
require.True(t, ok)
require.NotNil(t, tb)

gotTblInfo, ok := is.TableInfoByID(tbID)
require.True(t, ok)
require.Same(t, tb.Meta(), gotTblInfo)

tb, ok = is.TableByID(dbID)
require.False(t, ok)
require.Nil(t, tb)

gotTblInfo, ok = is.TableInfoByID(dbID)
require.False(t, ok)
require.Nil(t, gotTblInfo)

tb, err = is.TableByName(dbName, tbName)
require.NoError(t, err)
require.NotNil(t, tb)

gotTblInfo, err = is.TableInfoByName(dbName, tbName)
require.NoError(t, err)
require.Same(t, tb.Meta(), gotTblInfo)

_, err = is.TableByName(dbName, noexist)
require.Error(t, err)

gotTblInfo, err = is.TableInfoByName(dbName, noexist)
require.Error(t, err)
require.Nil(t, gotTblInfo)

tbs := is.SchemaTables(dbName)
require.Len(t, tbs, 1)

tblInfos := is.SchemaTableInfos(dbName)
require.Len(t, tblInfos, 1)
require.Same(t, tbs[0].Meta(), tblInfos[0])

tbs = is.SchemaTables(noexist)
require.Len(t, tbs, 0)

tblInfos = is.SchemaTableInfos(noexist)
require.Len(t, tblInfos, 0)

// Make sure partitions table exists
tb, err = is.TableByName(model.NewCIStr("information_schema"), model.NewCIStr("partitions"))
require.NoError(t, err)
Expand Down Expand Up @@ -747,29 +770,66 @@ func TestLocalTemporaryTables(t *testing.T) {
tbl, err := is.TableByName(dbTest.Name, normalTbTestA.Meta().Name)
require.NoError(t, err)
require.Equal(t, tmpTbTestA, tbl)
gotTblInfo, err := is.TableInfoByName(dbTest.Name, normalTbTestA.Meta().Name)
require.NoError(t, err)
require.Same(t, tmpTbTestA.Meta(), gotTblInfo)

tbl, err = is.TableByName(dbTest.Name, normalTbTestB.Meta().Name)
require.NoError(t, err)
require.Equal(t, normalTbTestB.Meta(), tbl.Meta())
gotTblInfo, err = is.TableInfoByName(dbTest.Name, normalTbTestB.Meta().Name)
require.NoError(t, err)
require.Same(t, tbl.Meta(), gotTblInfo)

tbl, err = is.TableByName(db1.Name, tb11.Meta().Name)
require.True(t, infoschema.ErrTableNotExists.Equal(err))
require.Nil(t, tbl)
gotTblInfo, err = is.TableInfoByName(dbTest.Name, tb11.Meta().Name)
require.True(t, infoschema.ErrTableNotExists.Equal(err))
require.Nil(t, gotTblInfo)

tbl, err = is.TableByName(db1.Name, tb12.Meta().Name)
require.NoError(t, err)
require.Equal(t, tb12, tbl)
gotTblInfo, err = is.TableInfoByName(db1.Name, tb12.Meta().Name)
require.NoError(t, err)
require.Same(t, tbl.Meta(), gotTblInfo)

// test TableByID
tbl, ok := is.TableByID(normalTbTestA.Meta().ID)
require.True(t, ok)
require.Equal(t, normalTbTestA.Meta(), tbl.Meta())
gotTblInfo, ok = is.TableInfoByID(normalTbTestA.Meta().ID)
require.True(t, ok)
require.Same(t, tbl.Meta(), gotTblInfo)

tbl, ok = is.TableByID(normalTbTestB.Meta().ID)
require.True(t, ok)
require.Equal(t, normalTbTestB.Meta(), tbl.Meta())
gotTblInfo, ok = is.TableInfoByID(normalTbTestB.Meta().ID)
require.True(t, ok)
require.Same(t, tbl.Meta(), gotTblInfo)

tbl, ok = is.TableByID(tmpTbTestA.Meta().ID)
require.True(t, ok)
require.Equal(t, tmpTbTestA, tbl)
gotTblInfo, ok = is.TableInfoByID(tmpTbTestA.Meta().ID)
require.True(t, ok)
require.Same(t, tbl.Meta(), gotTblInfo)

tbl, ok = is.TableByID(tb12.Meta().ID)
require.True(t, ok)
require.Equal(t, tb12, tbl)
gotTblInfo, ok = is.TableInfoByID(tb12.Meta().ID)
require.True(t, ok)
require.Same(t, tbl.Meta(), gotTblInfo)

tbl, ok = is.TableByID(1234567)
require.False(t, ok)
require.Nil(t, tbl)
gotTblInfo, ok = is.TableInfoByID(1234567)
require.False(t, ok)
require.Nil(t, gotTblInfo)

// test SchemaByTable
info, ok := is.SchemaByID(normalTbTestA.Meta().DBID)
Expand Down
25 changes: 25 additions & 0 deletions pkg/infoschema/infoschema_v2.go
Original file line number Diff line number Diff line change
Expand Up @@ -378,6 +378,31 @@ func (is *infoschemaV2) TableByName(schema, tbl model.CIStr) (t table.Table, err
return ret, nil
}

// TableInfoByName implements InfoSchema.TableInfoByName
func (is *infoschemaV2) TableInfoByName(schema, table model.CIStr) (*model.TableInfo, error) {
tbl, err := is.TableByName(schema, table)
return getTableInfo(tbl), err
}

// TableInfoByID implements InfoSchema.TableInfoByID
func (is *infoschemaV2) TableInfoByID(id int64) (*model.TableInfo, bool) {
tbl, ok := is.TableByID(id)
return getTableInfo(tbl), ok
}

// SchemaTableInfos implements InfoSchema.FindTableInfoByPartitionID
func (is *infoschemaV2) SchemaTableInfos(schema model.CIStr) []*model.TableInfo {
return getTableInfoList(is.SchemaTables(schema))
}

// FindTableInfoByPartitionID implements InfoSchema.FindTableInfoByPartitionID
func (is *infoschemaV2) FindTableInfoByPartitionID(
partitionID int64,
) (*model.TableInfo, *model.DBInfo, *model.PartitionDefinition) {
tbl, db, partDef := is.FindTableByPartitionID(partitionID)
return getTableInfo(tbl), db, partDef
}

func (is *infoschemaV2) SchemaByName(schema model.CIStr) (val *model.DBInfo, ok bool) {
if isSpecialDB(schema.L) {
return is.Data.specials[schema.L].dbInfo, true
Expand Down
Loading