Skip to content

Commit

Permalink
*: update v2 infoschema reader support for tables Indexes, Views, Key…
Browse files Browse the repository at this point in the history
…ColumnUsage, TableConstraints (#55241)

ref #50959
  • Loading branch information
tiancaiamao authored Aug 13, 2024
1 parent fcc2f72 commit bc6a18d
Show file tree
Hide file tree
Showing 9 changed files with 530 additions and 235 deletions.
436 changes: 214 additions & 222 deletions pkg/executor/infoschema_reader.go

Large diffs are not rendered by default.

22 changes: 17 additions & 5 deletions pkg/planner/core/logical_plan_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)}
}
Expand Down
120 changes: 113 additions & 7 deletions pkg/planner/core/memtable_infoschema_extractor.go
Original file line number Diff line number Diff line change
Expand Up @@ -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{
Expand All @@ -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.
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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
Expand Down
1 change: 1 addition & 0 deletions pkg/planner/core/operator/logicalop/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
],
)
4 changes: 4 additions & 0 deletions pkg/planner/core/operator/logicalop/logical_mem_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}
Expand Down
40 changes: 39 additions & 1 deletion pkg/testkit/testkit.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,13 +23,15 @@ import (
"net"
"net/http"
"net/http/pprof"
"slices"
"strings"
"sync"
"testing"
"time"

"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"
Expand Down Expand Up @@ -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
Expand Down
Loading

0 comments on commit bc6a18d

Please sign in to comment.