Skip to content

Commit

Permalink
infoschema: introduce MetaOnlyInfoSchema to provide meta only infor…
Browse files Browse the repository at this point in the history
…mation schema (#52070)

close #52072
  • Loading branch information
lcwangchao authored Mar 26, 2024
1 parent 311eef9 commit bd17acd
Show file tree
Hide file tree
Showing 22 changed files with 231 additions and 44 deletions.
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 @@ -223,7 +223,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 @@ -160,23 +160,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 @@ -748,29 +771,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 @@ -364,6 +364,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

0 comments on commit bd17acd

Please sign in to comment.