diff --git a/pkg/executor/infoschema_reader.go b/pkg/executor/infoschema_reader.go index 341d30fddc386..2ff025aa49d74 100644 --- a/pkg/executor/infoschema_reader.go +++ b/pkg/executor/infoschema_reader.go @@ -144,11 +144,9 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex case infoschema.TableAnalyzeStatus: err = e.setDataForAnalyzeStatus(ctx, sctx) case infoschema.TableTiDBIndexes: - dbs := getAllSchemas() - err = e.setDataFromIndexes(ctx, sctx, dbs) + err = e.setDataFromIndexes(ctx, sctx) case infoschema.TableViews: - dbs := getAllSchemas() - err = e.setDataFromViews(ctx, sctx, dbs) + err = e.setDataFromViews(ctx, sctx) case infoschema.TableEngines: e.setDataFromEngines() case infoschema.TableCharacterSets: @@ -156,8 +154,7 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex case infoschema.TableCollations: e.setDataFromCollations() case infoschema.TableKeyColumn: - dbs := getAllSchemas() - err = e.setDataFromKeyColumnUsage(ctx, sctx, dbs) + err = e.setDataFromKeyColumnUsage(ctx, sctx) case infoschema.TableMetricTables: e.setDataForMetricTables() case infoschema.TableProfiling: @@ -175,8 +172,7 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex case infoschema.TableTiDBHotRegions: err = e.setDataForTiDBHotRegions(ctx, sctx) case infoschema.TableConstraints: - dbs := getAllSchemas() - err = e.setDataFromTableConstraints(ctx, sctx, dbs) + err = e.setDataFromTableConstraints(ctx, sctx) case infoschema.TableSessionVar: e.rows, err = infoschema.GetDataFromSessionVariables(ctx, sctx) case infoschema.TableTiDBServersInfo: @@ -1320,155 +1316,160 @@ func (e *memtableRetriever) setDataFromPartitions(ctx context.Context, sctx sess return nil } -func (e *memtableRetriever) setDataFromIndexes(ctx context.Context, sctx sessionctx.Context, schemas []model.CIStr) error { - checker := privilege.GetPrivilegeManager(sctx) - extractor, ok := e.extractor.(*plannercore.InfoSchemaBaseExtractor) - if ok && extractor.SkipRequest { +func (e *memtableRetriever) setDataFromIndexes(ctx context.Context, sctx sessionctx.Context) error { + ex, ok := e.extractor.(*plannercore.InfoSchemaIndexesExtractor) + if !ok { + return errors.Errorf("wrong extractor type: %T, expected InfoSchemaIndexesExtractor", e.extractor) + } + if ex.SkipRequest { return nil } + + schemas, tables, err := ex.ListSchemasAndTables(ctx, e.is) + if err != nil { + return errors.Trace(err) + } + var rows [][]types.Datum - for _, schema := range schemas { - if ok && extractor.Filter("table_schema", schema.L) { - continue - } - tables, err := e.is.SchemaTableInfos(ctx, schema) + for i, table := range tables { + rows, err = e.setDataFromIndex(sctx, schemas[i], table, rows) if err != nil { return errors.Trace(err) } - for _, tb := range tables { - if ok && extractor.Filter("table_name", tb.Name.L) { - continue - } - if checker != nil && !checker.RequestVerification(sctx.GetSessionVars().ActiveRoles, schema.L, tb.Name.L, "", mysql.AllPrivMask) { - continue - } - - if tb.PKIsHandle { - var pkCol *model.ColumnInfo - for _, col := range tb.Cols() { - if mysql.HasPriKeyFlag(col.GetFlag()) { - pkCol = col - break - } - } - record := types.MakeDatums( - schema.O, // TABLE_SCHEMA - tb.Name.O, // TABLE_NAME - 0, // NON_UNIQUE - "PRIMARY", // KEY_NAME - 1, // SEQ_IN_INDEX - pkCol.Name.O, // COLUMN_NAME - nil, // SUB_PART - "", // INDEX_COMMENT - nil, // Expression - 0, // INDEX_ID - "YES", // IS_VISIBLE - "YES", // CLUSTERED - 0, // IS_GLOBAL - ) - rows = append(rows, record) - } - for _, idxInfo := range tb.Indices { - if idxInfo.State != model.StatePublic { - continue - } - isClustered := "NO" - if tb.IsCommonHandle && idxInfo.Primary { - isClustered = "YES" - } - for i, col := range idxInfo.Columns { - nonUniq := 1 - if idxInfo.Unique { - nonUniq = 0 - } - var subPart any - if col.Length != types.UnspecifiedLength { - subPart = col.Length - } - colName := col.Name.O - var expression any - expression = nil - tblCol := tb.Columns[col.Offset] - if tblCol.Hidden { - colName = "NULL" - expression = tblCol.GeneratedExprString - } - visible := "YES" - if idxInfo.Invisible { - visible = "NO" - } - record := types.MakeDatums( - schema.O, // TABLE_SCHEMA - tb.Name.O, // TABLE_NAME - nonUniq, // NON_UNIQUE - idxInfo.Name.O, // KEY_NAME - i+1, // SEQ_IN_INDEX - colName, // COLUMN_NAME - subPart, // SUB_PART - idxInfo.Comment, // INDEX_COMMENT - expression, // Expression - idxInfo.ID, // INDEX_ID - visible, // IS_VISIBLE - isClustered, // CLUSTERED - idxInfo.Global, // IS_GLOBAL - ) - rows = append(rows, record) - } - } - } } e.rows = rows return nil } -func (e *memtableRetriever) setDataFromViews(ctx context.Context, sctx sessionctx.Context, schemas []model.CIStr) error { +func (*memtableRetriever) setDataFromIndex( + sctx sessionctx.Context, + schema model.CIStr, + tb *model.TableInfo, + rows [][]types.Datum) ([][]types.Datum, error) { checker := privilege.GetPrivilegeManager(sctx) - extractor, ok := e.extractor.(*plannercore.InfoSchemaBaseExtractor) - if ok && extractor.SkipRequest { - return nil + if checker != nil && !checker.RequestVerification(sctx.GetSessionVars().ActiveRoles, schema.L, tb.Name.L, "", mysql.AllPrivMask) { + return rows, nil } - var rows [][]types.Datum - for _, schema := range schemas { - if ok && extractor.Filter("table_schema", schema.L) { + + if tb.PKIsHandle { + var pkCol *model.ColumnInfo + for _, col := range tb.Cols() { + if mysql.HasPriKeyFlag(col.GetFlag()) { + pkCol = col + break + } + } + record := types.MakeDatums( + schema.O, // TABLE_SCHEMA + tb.Name.O, // TABLE_NAME + 0, // NON_UNIQUE + "PRIMARY", // KEY_NAME + 1, // SEQ_IN_INDEX + pkCol.Name.O, // COLUMN_NAME + nil, // SUB_PART + "", // INDEX_COMMENT + nil, // Expression + 0, // INDEX_ID + "YES", // IS_VISIBLE + "YES", // CLUSTERED + 0, // IS_GLOBAL + ) + rows = append(rows, record) + } + for _, idxInfo := range tb.Indices { + if idxInfo.State != model.StatePublic { continue } - tables, err := e.is.SchemaTableInfos(ctx, schema) - if err != nil { - return errors.Trace(err) + isClustered := "NO" + if tb.IsCommonHandle && idxInfo.Primary { + isClustered = "YES" } - for _, table := range tables { - if ok && extractor.Filter("table_name", table.Name.L) { - continue - } - if !table.IsView() { - continue + for i, col := range idxInfo.Columns { + nonUniq := 1 + if idxInfo.Unique { + nonUniq = 0 } - collation := table.Collate - charset := table.Charset - if collation == "" { - collation = mysql.DefaultCollationName + var subPart any + if col.Length != types.UnspecifiedLength { + subPart = col.Length } - if charset == "" { - charset = mysql.DefaultCharset + colName := col.Name.O + var expression any + expression = nil + tblCol := tb.Columns[col.Offset] + if tblCol.Hidden { + colName = "NULL" + expression = tblCol.GeneratedExprString } - if checker != nil && !checker.RequestVerification(sctx.GetSessionVars().ActiveRoles, schema.L, table.Name.L, "", mysql.AllPrivMask) { - continue + visible := "YES" + if idxInfo.Invisible { + visible = "NO" } record := types.MakeDatums( - infoschema.CatalogVal, // TABLE_CATALOG - schema.O, // TABLE_SCHEMA - table.Name.O, // TABLE_NAME - table.View.SelectStmt, // VIEW_DEFINITION - table.View.CheckOption.String(), // CHECK_OPTION - "NO", // IS_UPDATABLE - table.View.Definer.String(), // DEFINER - table.View.Security.String(), // SECURITY_TYPE - charset, // CHARACTER_SET_CLIENT - collation, // COLLATION_CONNECTION + schema.O, // TABLE_SCHEMA + tb.Name.O, // TABLE_NAME + nonUniq, // NON_UNIQUE + idxInfo.Name.O, // KEY_NAME + i+1, // SEQ_IN_INDEX + colName, // COLUMN_NAME + subPart, // SUB_PART + idxInfo.Comment, // INDEX_COMMENT + expression, // Expression + idxInfo.ID, // INDEX_ID + visible, // IS_VISIBLE + isClustered, // CLUSTERED + idxInfo.Global, // IS_GLOBAL ) rows = append(rows, record) } } + return rows, nil +} + +func (e *memtableRetriever) setDataFromViews(ctx context.Context, sctx sessionctx.Context) error { + checker := privilege.GetPrivilegeManager(sctx) + ex, ok := e.extractor.(*plannercore.InfoSchemaViewsExtractor) + if !ok { + return errors.Errorf("wrong extractor type: %T, expected InfoSchemaIndexesExtractor", e.extractor) + } + if ex.SkipRequest { + return nil + } + schemas, tables, err := ex.ListSchemasAndTables(ctx, e.is) + if err != nil { + return errors.Trace(err) + } + rows := make([][]types.Datum, 0, len(tables)) + for i, table := range tables { + schema := schemas[i] + if !table.IsView() { + continue + } + collation := table.Collate + charset := table.Charset + if collation == "" { + collation = mysql.DefaultCollationName + } + if charset == "" { + charset = mysql.DefaultCharset + } + if checker != nil && !checker.RequestVerification(sctx.GetSessionVars().ActiveRoles, schema.L, table.Name.L, "", mysql.AllPrivMask) { + continue + } + record := types.MakeDatums( + infoschema.CatalogVal, // TABLE_CATALOG + schema.O, // TABLE_SCHEMA + table.Name.O, // TABLE_NAME + table.View.SelectStmt, // VIEW_DEFINITION + table.View.CheckOption.String(), // CHECK_OPTION + "NO", // IS_UPDATABLE + table.View.Definer.String(), // DEFINER + table.View.Security.String(), // SECURITY_TYPE + charset, // CHARACTER_SET_CLIENT + collation, // COLLATION_CONNECTION + ) + rows = append(rows, record) + } e.rows = rows return nil } @@ -1722,35 +1723,30 @@ func (e *memtableRetriever) dataForTiDBClusterInfo(ctx sessionctx.Context) error return nil } -func (e *memtableRetriever) setDataFromKeyColumnUsage(ctx context.Context, sctx sessionctx.Context, schemas []model.CIStr) error { +func (e *memtableRetriever) setDataFromKeyColumnUsage(ctx context.Context, sctx sessionctx.Context) error { checker := privilege.GetPrivilegeManager(sctx) - rows := make([][]types.Datum, 0, len(schemas)) // The capacity is not accurate, but it is not a big problem. - extractor, ok := e.extractor.(*plannercore.InfoSchemaBaseExtractor) - if ok && extractor.SkipRequest { + ex, ok := e.extractor.(*plannercore.InfoSchemaKeyColumnUsageExtractor) + if !ok { + return errors.Errorf("wrong extractor type: %T, expected InfoSchemaIndexesExtractor", e.extractor) + } + if ex.SkipRequest { return nil } - for _, schema := range schemas { - // `constraint_schema` and `table_schema` are always the same in MySQL. - if ok && extractor.Filter("constraint_schema", schema.L) { + schemas, tables, err := ex.ListSchemasAndTables(ctx, e.is) + if err != nil { + return errors.Trace(err) + } + rows := make([][]types.Datum, 0, len(tables)) + for i, table := range tables { + schema := schemas[i] + if checker != nil && !checker.RequestVerification(sctx.GetSessionVars().ActiveRoles, schema.L, table.Name.L, "", mysql.AllPrivMask) { continue } - if ok && extractor.Filter("table_schema", schema.L) { + if ex.Filter("constraint_schema", schema.O) { continue } - tables, err := e.is.SchemaTableInfos(ctx, schema) - if err != nil { - return errors.Trace(err) - } - for _, table := range tables { - if ok && extractor.Filter("table_name", table.Name.L) { - continue - } - if checker != nil && !checker.RequestVerification(sctx.GetSessionVars().ActiveRoles, schema.L, table.Name.L, "", mysql.AllPrivMask) { - continue - } - rs := keyColumnUsageInTable(schema, table, extractor) - rows = append(rows, rs...) - } + rs := keyColumnUsageInTable(schema, table, ex) + rows = append(rows, rs...) } e.rows = rows return nil @@ -1818,7 +1814,7 @@ func (e *memtableRetriever) setDataForMetricTables() { e.rows = rows } -func keyColumnUsageInTable(schema model.CIStr, table *model.TableInfo, extractor *plannercore.InfoSchemaBaseExtractor) [][]types.Datum { +func keyColumnUsageInTable(schema model.CIStr, table *model.TableInfo, extractor *plannercore.InfoSchemaKeyColumnUsageExtractor) [][]types.Datum { var rows [][]types.Datum if table.PKIsHandle { if extractor == nil || !extractor.Filter("constraint_name", lowerPrimaryKeyName) { @@ -2131,90 +2127,86 @@ func (e *memtableRetriever) setDataForHotRegionByMetrics(metrics []helper.HotTab } // setDataFromTableConstraints constructs data for table information_schema.constraints.See https://dev.mysql.com/doc/refman/5.7/en/table-constraints-table.html -func (e *memtableRetriever) setDataFromTableConstraints(ctx context.Context, sctx sessionctx.Context, schemas []model.CIStr) error { +func (e *memtableRetriever) setDataFromTableConstraints(ctx context.Context, sctx sessionctx.Context) error { checker := privilege.GetPrivilegeManager(sctx) - extractor, ok := e.extractor.(*plannercore.InfoSchemaBaseExtractor) - if ok && extractor.SkipRequest { + ex, ok := e.extractor.(*plannercore.InfoSchemaTableConstraintsExtractor) + if !ok { + return errors.Errorf("wrong extractor type: %T, expected InfoSchemaIndexesExtractor", e.extractor) + } + if ex.SkipRequest { return nil } - var rows [][]types.Datum - for _, schema := range schemas { - if ok && extractor.Filter("constraint_schema", schema.L) { + schemas, tables, err := ex.ListSchemasAndTables(ctx, e.is) + if err != nil { + return errors.Trace(err) + } + rows := make([][]types.Datum, 0, len(tables)) + for i, tbl := range tables { + schema := schemas[i] + if ex.Filter("constraint_schema", schema.L) { continue } - if ok && extractor.Filter("table_schema", schema.L) { + if checker != nil && !checker.RequestVerification(sctx.GetSessionVars().ActiveRoles, schema.L, tbl.Name.L, "", mysql.AllPrivMask) { continue } - tables, err := e.is.SchemaTableInfos(ctx, schema) - if err != nil { - return errors.Trace(err) - } - for _, tbl := range tables { - if ok && extractor.Filter("table_name", tbl.Name.L) { - continue - } - if checker != nil && !checker.RequestVerification(sctx.GetSessionVars().ActiveRoles, schema.L, tbl.Name.L, "", mysql.AllPrivMask) { - continue - } - if tbl.PKIsHandle { - if !ok || !extractor.Filter("constraint_name", lowerPrimaryKeyName) { - record := types.MakeDatums( - infoschema.CatalogVal, // CONSTRAINT_CATALOG - schema.O, // CONSTRAINT_SCHEMA - mysql.PrimaryKeyName, // CONSTRAINT_NAME - schema.O, // TABLE_SCHEMA - tbl.Name.O, // TABLE_NAME - infoschema.PrimaryKeyType, // CONSTRAINT_TYPE - ) - rows = append(rows, record) - } - } - - for _, idx := range tbl.Indices { - var cname, ctype string - var filterName string - if idx.Primary { - cname = mysql.PrimaryKeyName - filterName = lowerPrimaryKeyName - ctype = infoschema.PrimaryKeyType - } else if idx.Unique { - cname = idx.Name.O - filterName = idx.Name.L - ctype = infoschema.UniqueKeyType - } else { - // The index has no constriant. - continue - } - if ok && extractor.Filter("constraint_name", filterName) { - continue - } - record := types.MakeDatums( - infoschema.CatalogVal, // CONSTRAINT_CATALOG - schema.O, // CONSTRAINT_SCHEMA - cname, // CONSTRAINT_NAME - schema.O, // TABLE_SCHEMA - tbl.Name.O, // TABLE_NAME - ctype, // CONSTRAINT_TYPE - ) - rows = append(rows, record) - } - // TiDB includes foreign key information for compatibility but foreign keys are not yet enforced. - for _, fk := range tbl.ForeignKeys { - if ok && extractor.Filter("constraint_name", fk.Name.L) { - continue - } + if tbl.PKIsHandle { + if !ex.Filter("constraint_name", lowerPrimaryKeyName) { record := types.MakeDatums( infoschema.CatalogVal, // CONSTRAINT_CATALOG schema.O, // CONSTRAINT_SCHEMA - fk.Name.O, // CONSTRAINT_NAME + mysql.PrimaryKeyName, // CONSTRAINT_NAME schema.O, // TABLE_SCHEMA tbl.Name.O, // TABLE_NAME - infoschema.ForeignKeyType, // CONSTRAINT_TYPE + infoschema.PrimaryKeyType, // CONSTRAINT_TYPE ) rows = append(rows, record) } } + + for _, idx := range tbl.Indices { + var cname, ctype string + var filterName string + if idx.Primary { + cname = mysql.PrimaryKeyName + filterName = lowerPrimaryKeyName + ctype = infoschema.PrimaryKeyType + } else if idx.Unique { + cname = idx.Name.O + filterName = idx.Name.L + ctype = infoschema.UniqueKeyType + } else { + // The index has no constriant. + continue + } + if ex.Filter("constraint_name", filterName) { + continue + } + record := types.MakeDatums( + infoschema.CatalogVal, // CONSTRAINT_CATALOG + schema.O, // CONSTRAINT_SCHEMA + cname, // CONSTRAINT_NAME + schema.O, // TABLE_SCHEMA + tbl.Name.O, // TABLE_NAME + ctype, // CONSTRAINT_TYPE + ) + rows = append(rows, record) + } + // TiDB includes foreign key information for compatibility but foreign keys are not yet enforced. + for _, fk := range tbl.ForeignKeys { + if ex.Filter("constraint_name", fk.Name.L) { + continue + } + record := types.MakeDatums( + infoschema.CatalogVal, // CONSTRAINT_CATALOG + schema.O, // CONSTRAINT_SCHEMA + fk.Name.O, // CONSTRAINT_NAME + schema.O, // TABLE_SCHEMA + tbl.Name.O, // TABLE_NAME + infoschema.ForeignKeyType, // CONSTRAINT_TYPE + ) + rows = append(rows, record) + } } e.rows = rows return nil diff --git a/pkg/planner/core/logical_plan_builder.go b/pkg/planner/core/logical_plan_builder.go index 0b7c94f26b083..cf1a04448419b 100644 --- a/pkg/planner/core/logical_plan_builder.go +++ b/pkg/planner/core/logical_plan_builder.go @@ -4832,17 +4832,29 @@ func (b *PlanBuilder) buildMemTable(_ context.Context, dbName model.CIStr, table ex.initExtractableColNames(upTbl) p.Extractor = ex case infoschema.TableReferConst, - infoschema.TableKeyColumn, infoschema.TableSequences, infoschema.TableCheckConstraints, infoschema.TableTiDBCheckConstraints, - infoschema.TableTiDBIndexUsage, - infoschema.TableTiDBIndexes, - infoschema.TableViews, - infoschema.TableConstraints: + infoschema.TableTiDBIndexUsage: ex := &InfoSchemaBaseExtractor{} ex.initExtractableColNames(upTbl) p.Extractor = ex + case infoschema.TableTiDBIndexes: + ex := &InfoSchemaIndexesExtractor{} + ex.initExtractableColNames(upTbl) + p.Extractor = ex + case infoschema.TableViews: + ex := &InfoSchemaViewsExtractor{} + ex.initExtractableColNames(upTbl) + p.Extractor = ex + case infoschema.TableKeyColumn: + ex := &InfoSchemaKeyColumnUsageExtractor{} + ex.initExtractableColNames(upTbl) + p.Extractor = ex + case infoschema.TableConstraints: + ex := &InfoSchemaTableConstraintsExtractor{} + ex.initExtractableColNames(upTbl) + p.Extractor = ex case infoschema.TableTiKVRegionStatus: p.Extractor = &TiKVRegionStatusExtractor{tablesID: make([]int64, 0)} } diff --git a/pkg/planner/core/memtable_infoschema_extractor.go b/pkg/planner/core/memtable_infoschema_extractor.go index 024520f497ef7..0d7dde7a3459c 100644 --- a/pkg/planner/core/memtable_infoschema_extractor.go +++ b/pkg/planner/core/memtable_infoschema_extractor.go @@ -35,13 +35,15 @@ import ( ) const ( - _tableSchema = "table_schema" - _tableName = "table_name" - _tidbTableID = "tidb_table_id" - _partitionName = "partition_name" - _tidbPartitionID = "tidb_partition_id" - _indexName = "index_name" - _schemaName = "schema_name" + _tableSchema = "table_schema" + _tableName = "table_name" + _tidbTableID = "tidb_table_id" + _partitionName = "partition_name" + _tidbPartitionID = "tidb_partition_id" + _indexName = "index_name" + _schemaName = "schema_name" + _constraintSchema = "constraint_schema" + _constraintName = "constraint_name" ) var extractableColumns = map[string][]string{ @@ -67,6 +69,34 @@ var extractableColumns = map[string][]string{ infoschema.TableSchemata: { _schemaName, }, + // See infoschema.tableTiDBIndexesCols for full columns. + // Used by InfoSchemaIndexesExtractor and setDataFromIndexes. + infoschema.TableTiDBIndexes: { + _tableSchema, + _tableName, + }, + // See infoschema.tableViewsCols for full columns. + // Used by InfoSchemaViewsExtractor and setDataFromViews. + infoschema.TableViews: { + _tableSchema, + _tableName, + }, + // See infoschema.keyColumnUsageCols for full columns. + // Used by InfoSchemaViewsExtractor and setDataFromKeyColumn + infoschema.TableKeyColumn: { + _tableSchema, + _constraintSchema, + _tableName, + _constraintName, + }, + // See infoschema.tableConstraintsCols for full columns. + // Used by InfoSchemaTableConstraintsExtractor and setDataFromTableConstraints. + infoschema.TableConstraints: { + _tableSchema, + _constraintSchema, + _tableName, + _constraintName, + }, } // InfoSchemaBaseExtractor is used to extract infoSchema tables related predicates. @@ -184,6 +214,25 @@ func (e *InfoSchemaBaseExtractor) Filter(colName string, val string) bool { return false } +// InfoSchemaIndexesExtractor is the predicate extractor for information_schema.tidb_indexes. +type InfoSchemaIndexesExtractor struct { + InfoSchemaBaseExtractor +} + +// ListSchemasAndTables lists related tables and their corresponding schemas from predicate. +// If there is no error, returning schema slice and table slice are guaranteed to have the same length. +func (e *InfoSchemaIndexesExtractor) ListSchemasAndTables( + ctx context.Context, + is infoschema.InfoSchema, +) ([]model.CIStr, []*model.TableInfo, error) { + schemas := e.listSchemas(is, _tableSchema) + tableNames := e.getSchemaObjectNames(_tableName) + if len(tableNames) > 0 { + return findTableAndSchemaByName(ctx, is, schemas, tableNames) + } + return listTablesForEachSchema(ctx, is, schemas) +} + // InfoSchemaTablesExtractor is the predicate extractor for information_schema.tables. type InfoSchemaTablesExtractor struct { InfoSchemaBaseExtractor @@ -211,6 +260,63 @@ func (e *InfoSchemaTablesExtractor) ListSchemasAndTables( return listTablesForEachSchema(ctx, is, schemas) } +// InfoSchemaViewsExtractor is the predicate extractor for information_schema.views. +type InfoSchemaViewsExtractor struct { + InfoSchemaBaseExtractor +} + +// ListSchemasAndTables lists related tables and their corresponding schemas from predicate. +// If there is no error, returning schema slice and table slice are guaranteed to have the same length. +func (e *InfoSchemaViewsExtractor) ListSchemasAndTables( + ctx context.Context, + is infoschema.InfoSchema, +) ([]model.CIStr, []*model.TableInfo, error) { + schemas := e.listSchemas(is, _tableSchema) + tableNames := e.getSchemaObjectNames(_tableName) + if len(tableNames) > 0 { + return findTableAndSchemaByName(ctx, is, schemas, tableNames) + } + return listTablesForEachSchema(ctx, is, schemas) +} + +// InfoSchemaKeyColumnUsageExtractor is the predicate extractor for information_schema.key_column_usage. +type InfoSchemaKeyColumnUsageExtractor struct { + InfoSchemaBaseExtractor +} + +// ListSchemasAndTables lists related tables and their corresponding schemas from predicate. +// If there is no error, returning schema slice and table slice are guaranteed to have the same length. +func (e *InfoSchemaKeyColumnUsageExtractor) ListSchemasAndTables( + ctx context.Context, + is infoschema.InfoSchema, +) ([]model.CIStr, []*model.TableInfo, error) { + schemas := e.listSchemas(is, _tableSchema) + tableNames := e.getSchemaObjectNames(_tableName) + if len(tableNames) > 0 { + return findTableAndSchemaByName(ctx, is, schemas, tableNames) + } + return listTablesForEachSchema(ctx, is, schemas) +} + +// InfoSchemaTableConstraintsExtractor is the predicate extractor for information_schema.constraints. +type InfoSchemaTableConstraintsExtractor struct { + InfoSchemaBaseExtractor +} + +// ListSchemasAndTables lists related tables and their corresponding schemas from predicate. +// If there is no error, returning schema slice and table slice are guaranteed to have the same length. +func (e *InfoSchemaTableConstraintsExtractor) ListSchemasAndTables( + ctx context.Context, + is infoschema.InfoSchema, +) ([]model.CIStr, []*model.TableInfo, error) { + schemas := e.listSchemas(is, _tableSchema) + tableNames := e.getSchemaObjectNames(_tableName) + if len(tableNames) > 0 { + return findTableAndSchemaByName(ctx, is, schemas, tableNames) + } + return listTablesForEachSchema(ctx, is, schemas) +} + // InfoSchemaPartitionsExtractor is the predicate extractor for information_schema.partitions. type InfoSchemaPartitionsExtractor struct { InfoSchemaBaseExtractor diff --git a/pkg/planner/core/operator/logicalop/BUILD.bazel b/pkg/planner/core/operator/logicalop/BUILD.bazel index fa57f0fe3c68d..10b454e4d2a4c 100644 --- a/pkg/planner/core/operator/logicalop/BUILD.bazel +++ b/pkg/planner/core/operator/logicalop/BUILD.bazel @@ -56,6 +56,7 @@ go_library( "//pkg/util/size", "//pkg/util/tracing", "@com_github_pingcap_errors//:errors", + "@com_github_pingcap_failpoint//:failpoint", "@com_github_pingcap_tipb//go-tipb", ], ) diff --git a/pkg/planner/core/operator/logicalop/logical_mem_table.go b/pkg/planner/core/operator/logicalop/logical_mem_table.go index 568627ae2d764..3874e19030a33 100644 --- a/pkg/planner/core/operator/logicalop/logical_mem_table.go +++ b/pkg/planner/core/operator/logicalop/logical_mem_table.go @@ -15,6 +15,7 @@ package logicalop import ( + "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/parser/model" @@ -65,6 +66,9 @@ func (p LogicalMemTable) Init(ctx base.PlanContext, offset int) *LogicalMemTable // PredicatePushDown implements base.LogicalPlan.<1st> interface. func (p *LogicalMemTable) PredicatePushDown(predicates []expression.Expression, _ *optimizetrace.LogicalOptimizeOp) ([]expression.Expression, base.LogicalPlan) { if p.Extractor != nil { + failpoint.Inject("skipExtractor", func(_ failpoint.Value) { + failpoint.Return(predicates, p.Self()) + }) predicates = p.Extractor.Extract(p.SCtx(), p.Schema(), p.OutputNames(), predicates) } return predicates, p.Self() diff --git a/pkg/planner/core/operator/logicalop/logicalop_test/logical_mem_table_predicate_extractor_test.go b/pkg/planner/core/operator/logicalop/logicalop_test/logical_mem_table_predicate_extractor_test.go index 39219d5a45622..1fc32f2d0ece9 100644 --- a/pkg/planner/core/operator/logicalop/logicalop_test/logical_mem_table_predicate_extractor_test.go +++ b/pkg/planner/core/operator/logicalop/logicalop_test/logical_mem_table_predicate_extractor_test.go @@ -2050,6 +2050,14 @@ func TestInfoSchemaTableExtract(t *testing.T) { base = &ex.InfoSchemaBaseExtractor case *plannercore.InfoSchemaSchemataExtractor: base = &ex.InfoSchemaBaseExtractor + case *plannercore.InfoSchemaIndexesExtractor: + base = &ex.InfoSchemaBaseExtractor + case *plannercore.InfoSchemaViewsExtractor: + base = &ex.InfoSchemaBaseExtractor + case *plannercore.InfoSchemaKeyColumnUsageExtractor: + base = &ex.InfoSchemaBaseExtractor + case *plannercore.InfoSchemaTableConstraintsExtractor: + base = &ex.InfoSchemaBaseExtractor default: require.Failf(t, "unexpected extractor type", "%T", ex) } diff --git a/pkg/testkit/testkit.go b/pkg/testkit/testkit.go index fbfd9d8855d6a..037ce3e6d50ae 100644 --- a/pkg/testkit/testkit.go +++ b/pkg/testkit/testkit.go @@ -23,6 +23,7 @@ import ( "net" "net/http" "net/http/pprof" + "slices" "strings" "sync" "testing" @@ -30,6 +31,7 @@ import ( "github.com/gorilla/mux" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/ast" @@ -171,7 +173,43 @@ func (tk *TestKit) MustQuery(sql string, args ...any) *Result { tk.alloc.Reset() } }() - return tk.MustQueryWithContext(context.Background(), sql, args...) + rs1 := tk.MustQueryWithContext(context.Background(), sql, args...) + if !strings.Contains(sql, "information_schema") || + strings.Contains(sql, "trace") || + strings.Contains(sql, "statements_summary") || + strings.Contains(sql, "slow_query") || + strings.Contains(sql, "cluster_config") || + strings.Contains(sql, "CLUSTER_") || + strings.Contains(sql, "STATEMENTS_SUMMARY_EVICTED") || + strings.Contains(sql, "TIDB_TRX") { + return rs1 + } + err := failpoint.Enable("github.com/pingcap/tidb/pkg/planner/core/skipExtractor", "return(true)") + if err != nil { + panic(err) + } + rs2 := tk.MustQueryWithContext(context.Background(), sql, args...) + err = failpoint.Disable("github.com/pingcap/tidb/pkg/planner/core/skipExtractor") + if err != nil { + panic(err) + } + rs1Row := make([][]string, 0, len(rs1.rows)) + for _, row := range rs1.rows { + rs1SubRow := make([]string, 0, len(row)) + for _, col := range row { + rs1SubRow = append(rs1SubRow, strings.Clone(col)) + } + rs1Row = append(rs1Row, rs1SubRow) + } + slices.SortFunc(rs1.rows, func(a, b []string) int { + return slices.Compare(a, b) + }) + slices.SortFunc(rs2.rows, func(a, b []string) int { + return slices.Compare(a, b) + }) + rs2.Check(rs1.Rows()) + rs1.rows = rs1Row + return rs1 } // EventuallyMustQueryAndCheck query the statements and assert that diff --git a/tests/integrationtest/r/executor/infoschema_reader.result b/tests/integrationtest/r/executor/infoschema_reader.result index 2650467c5ff74..7a5be54f476a8 100644 --- a/tests/integrationtest/r/executor/infoschema_reader.result +++ b/tests/integrationtest/r/executor/infoschema_reader.result @@ -331,3 +331,94 @@ sleep(1) select table_rows, avg_row_length, data_length, index_length from information_schema.tables where table_name='t' AND TABLE_SCHEMA='executor__infoschema_reader'; table_rows avg_row_length data_length index_length 3 18 54 6 +drop table if exists test.t; +select * from information_schema.tidb_indexes where table_name = 't'; +TABLE_SCHEMA TABLE_NAME NON_UNIQUE KEY_NAME SEQ_IN_INDEX COLUMN_NAME SUB_PART INDEX_COMMENT Expression INDEX_ID IS_VISIBLE CLUSTERED IS_GLOBAL +executor__infoschema_reader t 0 PRIMARY 1 a NULL NULL 0 YES YES 0 +executor__infoschema_reader t 1 idx 1 c NULL NULL 1 YES NO 0 +select * from information_schema.tidb_indexes where table_schema = 'executor__infoschema_reader'; +TABLE_SCHEMA TABLE_NAME NON_UNIQUE KEY_NAME SEQ_IN_INDEX COLUMN_NAME SUB_PART INDEX_COMMENT Expression INDEX_ID IS_VISIBLE CLUSTERED IS_GLOBAL +executor__infoschema_reader t 0 PRIMARY 1 a NULL NULL 0 YES YES 0 +executor__infoschema_reader t 1 idx 1 c NULL NULL 1 YES NO 0 +executor__infoschema_reader t_common 0 PRIMARY 1 a NULL NULL 1 YES YES 0 +executor__infoschema_reader t_implicit 0 PRIMARY 1 a NULL NULL 1 YES NO 0 +executor__infoschema_reader t_int 0 PRIMARY 1 a NULL NULL 0 YES YES 0 +select * from information_schema.tidb_indexes where table_schema = 'executor__infoschema_reader' and table_name = 't'; +TABLE_SCHEMA TABLE_NAME NON_UNIQUE KEY_NAME SEQ_IN_INDEX COLUMN_NAME SUB_PART INDEX_COMMENT Expression INDEX_ID IS_VISIBLE CLUSTERED IS_GLOBAL +executor__infoschema_reader t 0 PRIMARY 1 a NULL NULL 0 YES YES 0 +executor__infoschema_reader t 1 idx 1 c NULL NULL 1 YES NO 0 +select * from information_schema.tidb_indexes where table_schema = 'executor__infoschema_reader' or table_name = 't'; +TABLE_SCHEMA TABLE_NAME NON_UNIQUE KEY_NAME SEQ_IN_INDEX COLUMN_NAME SUB_PART INDEX_COMMENT Expression INDEX_ID IS_VISIBLE CLUSTERED IS_GLOBAL +executor__infoschema_reader t 0 PRIMARY 1 a NULL NULL 0 YES YES 0 +executor__infoschema_reader t 1 idx 1 c NULL NULL 1 YES NO 0 +executor__infoschema_reader t_common 0 PRIMARY 1 a NULL NULL 1 YES YES 0 +executor__infoschema_reader t_implicit 0 PRIMARY 1 a NULL NULL 1 YES NO 0 +executor__infoschema_reader t_int 0 PRIMARY 1 a NULL NULL 0 YES YES 0 +select * from information_schema.tidb_indexes where table_schema = 'executor__infoschema_reader' and column_name = 'c'; +TABLE_SCHEMA TABLE_NAME NON_UNIQUE KEY_NAME SEQ_IN_INDEX COLUMN_NAME SUB_PART INDEX_COMMENT Expression INDEX_ID IS_VISIBLE CLUSTERED IS_GLOBAL +executor__infoschema_reader t 1 idx 1 c NULL NULL 1 YES NO 0 +select * from information_schema.tidb_indexes where table_schema = 'executor__infoschema_reader' and table_name = 'non_exist'; +TABLE_SCHEMA TABLE_NAME NON_UNIQUE KEY_NAME SEQ_IN_INDEX COLUMN_NAME SUB_PART INDEX_COMMENT Expression INDEX_ID IS_VISIBLE CLUSTERED IS_GLOBAL +select * from information_schema.views where table_name = 'v1'; +TABLE_CATALOG TABLE_SCHEMA TABLE_NAME VIEW_DEFINITION CHECK_OPTION IS_UPDATABLE DEFINER SECURITY_TYPE CHARACTER_SET_CLIENT COLLATION_CONNECTION +def executor__infoschema_reader v1 SELECT 1 AS `1` CASCADED NO root@localhost DEFINER utf8mb4 utf8mb4_general_ci +select * from information_schema.views where table_name = 'non_exist'; +TABLE_CATALOG TABLE_SCHEMA TABLE_NAME VIEW_DEFINITION CHECK_OPTION IS_UPDATABLE DEFINER SECURITY_TYPE CHARACTER_SET_CLIENT COLLATION_CONNECTION +select * from information_schema.views where table_schema = 'executor__infoschema_reader'; +TABLE_CATALOG TABLE_SCHEMA TABLE_NAME VIEW_DEFINITION CHECK_OPTION IS_UPDATABLE DEFINER SECURITY_TYPE CHARACTER_SET_CLIENT COLLATION_CONNECTION +def executor__infoschema_reader v1 SELECT 1 AS `1` CASCADED NO root@localhost DEFINER utf8mb4 utf8mb4_general_ci +def executor__infoschema_reader v_test SELECT NULL AS `type` FROM `executor__infoschema_reader`.`t` AS `f` CASCADED NO root@1.1.1.1 DEFINER utf8mb4 utf8mb4_general_ci +select * from information_schema.views where table_schema = 'non_exist'; +TABLE_CATALOG TABLE_SCHEMA TABLE_NAME VIEW_DEFINITION CHECK_OPTION IS_UPDATABLE DEFINER SECURITY_TYPE CHARACTER_SET_CLIENT COLLATION_CONNECTION +select * from information_schema.views where table_schema = 'executor__infoschema_reader' and table_name = 'v1'; +TABLE_CATALOG TABLE_SCHEMA TABLE_NAME VIEW_DEFINITION CHECK_OPTION IS_UPDATABLE DEFINER SECURITY_TYPE CHARACTER_SET_CLIENT COLLATION_CONNECTION +def executor__infoschema_reader v1 SELECT 1 AS `1` CASCADED NO root@localhost DEFINER utf8mb4 utf8mb4_general_ci +select * from information_schema.views where table_schema = 'executor__infoschema_reader' or table_name = 'v1'; +TABLE_CATALOG TABLE_SCHEMA TABLE_NAME VIEW_DEFINITION CHECK_OPTION IS_UPDATABLE DEFINER SECURITY_TYPE CHARACTER_SET_CLIENT COLLATION_CONNECTION +def executor__infoschema_reader v1 SELECT 1 AS `1` CASCADED NO root@localhost DEFINER utf8mb4 utf8mb4_general_ci +def executor__infoschema_reader v_test SELECT NULL AS `type` FROM `executor__infoschema_reader`.`t` AS `f` CASCADED NO root@1.1.1.1 DEFINER utf8mb4 utf8mb4_general_ci +select * from information_schema.key_column_usage where table_name = 't'; +CONSTRAINT_CATALOG CONSTRAINT_SCHEMA CONSTRAINT_NAME TABLE_CATALOG TABLE_SCHEMA TABLE_NAME COLUMN_NAME ORDINAL_POSITION POSITION_IN_UNIQUE_CONSTRAINT REFERENCED_TABLE_SCHEMA REFERENCED_TABLE_NAME REFERENCED_COLUMN_NAME +def executor__infoschema_reader PRIMARY def executor__infoschema_reader t a 1 1 NULL NULL NULL +select * from information_schema.key_column_usage where table_schema = 'executor__infoschema_reader'; +CONSTRAINT_CATALOG CONSTRAINT_SCHEMA CONSTRAINT_NAME TABLE_CATALOG TABLE_SCHEMA TABLE_NAME COLUMN_NAME ORDINAL_POSITION POSITION_IN_UNIQUE_CONSTRAINT REFERENCED_TABLE_SCHEMA REFERENCED_TABLE_NAME REFERENCED_COLUMN_NAME +def executor__infoschema_reader PRIMARY def executor__infoschema_reader t a 1 1 NULL NULL NULL +def executor__infoschema_reader PRIMARY def executor__infoschema_reader t_common a 1 NULL NULL NULL NULL +def executor__infoschema_reader PRIMARY def executor__infoschema_reader t_implicit a 1 NULL NULL NULL NULL +def executor__infoschema_reader PRIMARY def executor__infoschema_reader t_int a 1 1 NULL NULL NULL +select * from information_schema.key_column_usage where table_schema = 'executor__infoschema_reader' and table_name = 't'; +CONSTRAINT_CATALOG CONSTRAINT_SCHEMA CONSTRAINT_NAME TABLE_CATALOG TABLE_SCHEMA TABLE_NAME COLUMN_NAME ORDINAL_POSITION POSITION_IN_UNIQUE_CONSTRAINT REFERENCED_TABLE_SCHEMA REFERENCED_TABLE_NAME REFERENCED_COLUMN_NAME +def executor__infoschema_reader PRIMARY def executor__infoschema_reader t a 1 1 NULL NULL NULL +select * from information_schema.key_column_usage where table_schema = 'executor__infoschema_reader' or table_name = 't'; +CONSTRAINT_CATALOG CONSTRAINT_SCHEMA CONSTRAINT_NAME TABLE_CATALOG TABLE_SCHEMA TABLE_NAME COLUMN_NAME ORDINAL_POSITION POSITION_IN_UNIQUE_CONSTRAINT REFERENCED_TABLE_SCHEMA REFERENCED_TABLE_NAME REFERENCED_COLUMN_NAME +def executor__infoschema_reader PRIMARY def executor__infoschema_reader t a 1 1 NULL NULL NULL +def executor__infoschema_reader PRIMARY def executor__infoschema_reader t_common a 1 NULL NULL NULL NULL +def executor__infoschema_reader PRIMARY def executor__infoschema_reader t_implicit a 1 NULL NULL NULL NULL +def executor__infoschema_reader PRIMARY def executor__infoschema_reader t_int a 1 1 NULL NULL NULL +select * from information_schema.key_column_usage where table_schema = 'executor__infoschema_reader' and column_name = 'c'; +CONSTRAINT_CATALOG CONSTRAINT_SCHEMA CONSTRAINT_NAME TABLE_CATALOG TABLE_SCHEMA TABLE_NAME COLUMN_NAME ORDINAL_POSITION POSITION_IN_UNIQUE_CONSTRAINT REFERENCED_TABLE_SCHEMA REFERENCED_TABLE_NAME REFERENCED_COLUMN_NAME +select * from information_schema.key_column_usage where table_schema = 'executor__infoschema_reader' and column_name = 'non_exist'; +CONSTRAINT_CATALOG CONSTRAINT_SCHEMA CONSTRAINT_NAME TABLE_CATALOG TABLE_SCHEMA TABLE_NAME COLUMN_NAME ORDINAL_POSITION POSITION_IN_UNIQUE_CONSTRAINT REFERENCED_TABLE_SCHEMA REFERENCED_TABLE_NAME REFERENCED_COLUMN_NAME +CREATE TABLE tc(a INT CHECK(a > 10) NOT ENFORCED, b INT, c INT, CONSTRAINT c1 CHECK (b > c)); +select * from information_schema.table_constraints where table_name = 'tc'; +CONSTRAINT_CATALOG CONSTRAINT_SCHEMA CONSTRAINT_NAME TABLE_SCHEMA TABLE_NAME CONSTRAINT_TYPE +select * from information_schema.table_constraints where table_schema = 'executor__infoschema_reader'; +CONSTRAINT_CATALOG CONSTRAINT_SCHEMA CONSTRAINT_NAME TABLE_SCHEMA TABLE_NAME CONSTRAINT_TYPE +def executor__infoschema_reader PRIMARY executor__infoschema_reader t PRIMARY KEY +def executor__infoschema_reader PRIMARY executor__infoschema_reader t_common PRIMARY KEY +def executor__infoschema_reader PRIMARY executor__infoschema_reader t_implicit PRIMARY KEY +def executor__infoschema_reader PRIMARY executor__infoschema_reader t_int PRIMARY KEY +select * from information_schema.table_constraints where table_schema = 'executor__infoschema_reader' and table_name = 'tc'; +CONSTRAINT_CATALOG CONSTRAINT_SCHEMA CONSTRAINT_NAME TABLE_SCHEMA TABLE_NAME CONSTRAINT_TYPE +select * from information_schema.table_constraints where table_schema = 'executor__infoschema_reader' or table_name = 'tc'; +CONSTRAINT_CATALOG CONSTRAINT_SCHEMA CONSTRAINT_NAME TABLE_SCHEMA TABLE_NAME CONSTRAINT_TYPE +def executor__infoschema_reader PRIMARY executor__infoschema_reader t PRIMARY KEY +def executor__infoschema_reader PRIMARY executor__infoschema_reader t_common PRIMARY KEY +def executor__infoschema_reader PRIMARY executor__infoschema_reader t_implicit PRIMARY KEY +def executor__infoschema_reader PRIMARY executor__infoschema_reader t_int PRIMARY KEY +select * from information_schema.table_constraints where table_schema = 'executor__infoschema_reader' and table_name = 'non_exist'; +CONSTRAINT_CATALOG CONSTRAINT_SCHEMA CONSTRAINT_NAME TABLE_SCHEMA TABLE_NAME CONSTRAINT_TYPE +select * from information_schema.table_constraints where table_schema = 'executor__infoschema_reader' and CONSTRAINT_NAME = 'c1'; +CONSTRAINT_CATALOG CONSTRAINT_SCHEMA CONSTRAINT_NAME TABLE_SCHEMA TABLE_NAME CONSTRAINT_TYPE +def executor__infoschema_reader PRIMARY executor__infoschema_reader t PRIMARY KEY +def executor__infoschema_reader PRIMARY executor__infoschema_reader t_int PRIMARY KEY diff --git a/tests/integrationtest/t/executor/infoschema_reader.test b/tests/integrationtest/t/executor/infoschema_reader.test index 8d934c2610c48..4e8498e7f0ca0 100644 --- a/tests/integrationtest/t/executor/infoschema_reader.test +++ b/tests/integrationtest/t/executor/infoschema_reader.test @@ -264,3 +264,46 @@ insert into t(a, b, c) values(1, 2, 'c'), (7, 3, 'd'), (12, 4, 'e'); analyze table t; select sleep(1); select table_rows, avg_row_length, data_length, index_length from information_schema.tables where table_name='t' AND TABLE_SCHEMA='executor__infoschema_reader'; + + +# Cover reading from tables: Indexes, Views, KeyColumnUsage, TableConstraints +drop table if exists test.t; +select * from information_schema.tidb_indexes where table_name = 't'; +--sorted_result +select * from information_schema.tidb_indexes where table_schema = 'executor__infoschema_reader'; +--sorted_result +select * from information_schema.tidb_indexes where table_schema = 'executor__infoschema_reader' and table_name = 't'; +--sorted_result +select * from information_schema.tidb_indexes where table_schema = 'executor__infoschema_reader' or table_name = 't'; +select * from information_schema.tidb_indexes where table_schema = 'executor__infoschema_reader' and column_name = 'c'; +select * from information_schema.tidb_indexes where table_schema = 'executor__infoschema_reader' and table_name = 'non_exist'; + +select * from information_schema.views where table_name = 'v1'; +select * from information_schema.views where table_name = 'non_exist'; +--sorted_result +select * from information_schema.views where table_schema = 'executor__infoschema_reader'; +select * from information_schema.views where table_schema = 'non_exist'; +select * from information_schema.views where table_schema = 'executor__infoschema_reader' and table_name = 'v1'; +--sorted_result +select * from information_schema.views where table_schema = 'executor__infoschema_reader' or table_name = 'v1'; + +select * from information_schema.key_column_usage where table_name = 't'; +--sorted_result +select * from information_schema.key_column_usage where table_schema = 'executor__infoschema_reader'; +select * from information_schema.key_column_usage where table_schema = 'executor__infoschema_reader' and table_name = 't'; +--sorted_result +select * from information_schema.key_column_usage where table_schema = 'executor__infoschema_reader' or table_name = 't'; +select * from information_schema.key_column_usage where table_schema = 'executor__infoschema_reader' and column_name = 'c'; +select * from information_schema.key_column_usage where table_schema = 'executor__infoschema_reader' and column_name = 'non_exist'; + +CREATE TABLE tc(a INT CHECK(a > 10) NOT ENFORCED, b INT, c INT, CONSTRAINT c1 CHECK (b > c)); +--sorted_result +select * from information_schema.table_constraints where table_name = 'tc'; +--sorted_result +select * from information_schema.table_constraints where table_schema = 'executor__infoschema_reader'; +select * from information_schema.table_constraints where table_schema = 'executor__infoschema_reader' and table_name = 'tc'; +--sorted_result +select * from information_schema.table_constraints where table_schema = 'executor__infoschema_reader' or table_name = 'tc'; +select * from information_schema.table_constraints where table_schema = 'executor__infoschema_reader' and table_name = 'non_exist'; +--sorted_result +select * from information_schema.table_constraints where table_schema = 'executor__infoschema_reader' and CONSTRAINT_NAME = 'c1';