Skip to content

Commit

Permalink
planner: move MemtablePredicateExtractor interface to a unified base …
Browse files Browse the repository at this point in the history
…place. (#53031)

ref #51664, ref #52714
  • Loading branch information
AilinKid committed May 6, 2024
1 parent 3ca57c1 commit 957e6ae
Show file tree
Hide file tree
Showing 7 changed files with 69 additions and 45 deletions.
2 changes: 1 addition & 1 deletion pkg/executor/infoschema_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -94,7 +94,7 @@ type memtableRetriever struct {
rowIdx int
retrieved bool
initialized bool
extractor plannercore.MemTablePredicateExtractor
extractor base.MemTablePredicateExtractor
is infoschema.InfoSchema
memTracker *memory.Tracker
}
Expand Down
22 changes: 22 additions & 0 deletions pkg/planner/core/base/misc_base.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,8 @@
package base

import (
"github.com/pingcap/tidb/pkg/expression"
"github.com/pingcap/tidb/pkg/types"
"github.com/pingcap/tidb/pkg/util/collate"
"github.com/pingcap/tipb/go-tipb"
)
Expand Down Expand Up @@ -43,3 +45,23 @@ type ShowPredicateExtractor interface {
Field() string
FieldPatternLike() collate.WildcardPattern
}

// MemTablePredicateExtractor is used to extract some predicates from `WHERE` clause
// and push the predicates down to the data retrieving on reading memory table stage.
//
// e.g:
// SELECT * FROM cluster_config WHERE type='tikv' AND instance='192.168.1.9:2379'
// We must request all components in the cluster via HTTP API for retrieving
// configurations and filter them by `type/instance` columns.
//
// The purpose of defining a `MemTablePredicateExtractor` is to optimize this
// 1. Define a `ClusterConfigTablePredicateExtractor`
// 2. Extract the `type/instance` columns on the logic optimizing stage and save them via fields.
// 3. Passing the extractor to the `ClusterReaderExecExec` executor
// 4. Executor sends requests to the target components instead of all of the components
type MemTablePredicateExtractor interface {
// Extract extracts predicates which can be pushed down and returns the remained predicates
Extract(PlanContext, *expression.Schema, []*types.FieldName, []expression.Expression) (remained []expression.Expression)
// ExplainInfo give the basic desc of this mem extractor, `p` indicates a PhysicalPlan here.
ExplainInfo(p PhysicalPlan) string
}
2 changes: 1 addition & 1 deletion pkg/planner/core/explain.go
Original file line number Diff line number Diff line change
Expand Up @@ -1124,7 +1124,7 @@ func (p *PhysicalMemTable) ExplainInfo() string {
// OperatorInfo implements dataAccesser interface.
func (p *PhysicalMemTable) OperatorInfo(_ bool) string {
if p.Extractor != nil {
return p.Extractor.explainInfo(p)
return p.Extractor.ExplainInfo(p)
}
return ""
}
2 changes: 1 addition & 1 deletion pkg/planner/core/logical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -1395,7 +1395,7 @@ type LogicalTableDual struct {
type LogicalMemTable struct {
logicalSchemaProducer

Extractor MemTablePredicateExtractor
Extractor base.MemTablePredicateExtractor
DBName model.CIStr
TableInfo *model.TableInfo
Columns []*model.ColumnInfo
Expand Down
72 changes: 37 additions & 35 deletions pkg/planner/core/memtable_predicate_extractor.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,25 +44,6 @@ import (
"go.uber.org/zap"
)

// MemTablePredicateExtractor is used to extract some predicates from `WHERE` clause
// and push the predicates down to the data retrieving on reading memory table stage.
//
// e.g:
// SELECT * FROM cluster_config WHERE type='tikv' AND instance='192.168.1.9:2379'
// We must request all components in the cluster via HTTP API for retrieving
// configurations and filter them by `type/instance` columns.
//
// The purpose of defining a `MemTablePredicateExtractor` is to optimize this
// 1. Define a `ClusterConfigTablePredicateExtractor`
// 2. Extract the `type/instance` columns on the logic optimizing stage and save them via fields.
// 3. Passing the extractor to the `ClusterReaderExecExec` executor
// 4. Executor sends requests to the target components instead of all of the components
type MemTablePredicateExtractor interface {
// Extracts predicates which can be pushed down and returns the remained predicates
Extract(base.PlanContext, *expression.Schema, []*types.FieldName, []expression.Expression) (remained []expression.Expression)
explainInfo(p *PhysicalMemTable) string
}

// extractHelper contains some common utililty functions for all extractor.
// define an individual struct instead of a bunch of un-exported functions
// to avoid polluting the global scope of current package.
Expand Down Expand Up @@ -732,7 +713,8 @@ func (e *ClusterTableExtractor) Extract(_ base.PlanContext,
return remained
}

func (e *ClusterTableExtractor) explainInfo(_ *PhysicalMemTable) string {
// ExplainInfo implements base.MemTablePredicateExtractor interface.
func (e *ClusterTableExtractor) ExplainInfo(_ base.PhysicalPlan) string {
if e.SkipRequest {
return "skip_request:true"
}
Expand Down Expand Up @@ -821,7 +803,9 @@ func (e *ClusterLogTableExtractor) Extract(ctx base.PlanContext,
return remained
}

func (e *ClusterLogTableExtractor) explainInfo(p *PhysicalMemTable) string {
// ExplainInfo implements base.MemTablePredicateExtractor interface.
func (e *ClusterLogTableExtractor) ExplainInfo(pp base.PhysicalPlan) string {
p := pp.(*PhysicalMemTable)
if e.SkipRequest {
return "skip_request: true"
}
Expand Down Expand Up @@ -955,7 +939,9 @@ func (e *HotRegionsHistoryTableExtractor) Extract(ctx base.PlanContext,
return remained
}

func (e *HotRegionsHistoryTableExtractor) explainInfo(p *PhysicalMemTable) string {
// ExplainInfo implements the base.MemTablePredicateExtractor interface.
func (e *HotRegionsHistoryTableExtractor) ExplainInfo(pp base.PhysicalPlan) string {
p := pp.(*PhysicalMemTable)
if e.SkipRequest {
return "skip_request: true"
}
Expand Down Expand Up @@ -1070,7 +1056,9 @@ func (e *MetricTableExtractor) getTimeRange(start, end int64) (time.Time, time.T
return startTime, endTime
}

func (e *MetricTableExtractor) explainInfo(p *PhysicalMemTable) string {
// ExplainInfo implements the base.MemTablePredicateExtractor interface.
func (e *MetricTableExtractor) ExplainInfo(pp base.PhysicalPlan) string {
p := pp.(*PhysicalMemTable)
if e.SkipRequest {
return "skip_request: true"
}
Expand Down Expand Up @@ -1130,7 +1118,8 @@ func (e *MetricSummaryTableExtractor) Extract(_ base.PlanContext,
return remained
}

func (*MetricSummaryTableExtractor) explainInfo(_ *PhysicalMemTable) string {
// ExplainInfo implements base.MemTablePredicateExtractor interface.
func (*MetricSummaryTableExtractor) ExplainInfo(_ base.PhysicalPlan) string {
return ""
}

Expand Down Expand Up @@ -1162,7 +1151,8 @@ func (e *InspectionResultTableExtractor) Extract(_ base.PlanContext,
return remained
}

func (e *InspectionResultTableExtractor) explainInfo(_ *PhysicalMemTable) string {
// ExplainInfo implements base.MemTablePredicateExtractor interface.
func (e *InspectionResultTableExtractor) ExplainInfo(_ base.PhysicalPlan) string {
if e.SkipInspection {
return "skip_inspection:true"
}
Expand Down Expand Up @@ -1203,7 +1193,8 @@ func (e *InspectionSummaryTableExtractor) Extract(_ base.PlanContext,
return remained
}

func (e *InspectionSummaryTableExtractor) explainInfo(_ *PhysicalMemTable) string {
// ExplainInfo implements base.MemTablePredicateExtractor interface.
func (e *InspectionSummaryTableExtractor) ExplainInfo(_ base.PhysicalPlan) string {
if e.SkipInspection {
return "skip_inspection: true"
}
Expand Down Expand Up @@ -1255,7 +1246,8 @@ func (e *InspectionRuleTableExtractor) Extract(_ base.PlanContext,
return remained
}

func (e *InspectionRuleTableExtractor) explainInfo(_ *PhysicalMemTable) string {
// ExplainInfo implements base.MemTablePredicateExtractor interface.
func (e *InspectionRuleTableExtractor) ExplainInfo(_ base.PhysicalPlan) string {
if e.SkipRequest {
return "skip_request: true"
}
Expand Down Expand Up @@ -1408,7 +1400,8 @@ func (e *TableStorageStatsExtractor) Extract(_ base.PlanContext,
return remained
}

func (e *TableStorageStatsExtractor) explainInfo(_ *PhysicalMemTable) string {
// ExplainInfo implements base.MemTablePredicateExtractor interface.
func (e *TableStorageStatsExtractor) ExplainInfo(_ base.PhysicalPlan) string {
if e.SkipRequest {
return "skip_request: true"
}
Expand All @@ -1426,7 +1419,9 @@ func (e *TableStorageStatsExtractor) explainInfo(_ *PhysicalMemTable) string {
return r.String()
}

func (e *SlowQueryExtractor) explainInfo(p *PhysicalMemTable) string {
// ExplainInfo implements the base.MemTablePredicateExtractor interface.
func (e *SlowQueryExtractor) ExplainInfo(pp base.PhysicalPlan) string {
p := pp.(*PhysicalMemTable)
if e.SkipRequest {
return "skip_request: true"
}
Expand Down Expand Up @@ -1481,7 +1476,8 @@ func (e *TiFlashSystemTableExtractor) Extract(_ base.PlanContext,
return remained
}

func (e *TiFlashSystemTableExtractor) explainInfo(_ *PhysicalMemTable) string {
// ExplainInfo implements base.MemTablePredicateExtractor interface.
func (e *TiFlashSystemTableExtractor) ExplainInfo(_ base.PhysicalPlan) string {
if e.SkipRequest {
return "skip_request:true"
}
Expand Down Expand Up @@ -1550,7 +1546,9 @@ func (e *StatementsSummaryExtractor) Extract(sctx base.PlanContext,
return remained
}

func (e *StatementsSummaryExtractor) explainInfo(p *PhysicalMemTable) string {
// ExplainInfo implements base.MemTablePredicateExtractor interface.
func (e *StatementsSummaryExtractor) ExplainInfo(pp base.PhysicalPlan) string {
p := pp.(*PhysicalMemTable)
if e.SkipRequest {
return "skip_request: true"
}
Expand Down Expand Up @@ -1641,7 +1639,8 @@ func (e *TikvRegionPeersExtractor) Extract(_ base.PlanContext,
return remained
}

func (e *TikvRegionPeersExtractor) explainInfo(_ *PhysicalMemTable) string {
// ExplainInfo implements base.MemTablePredicateExtractor interface.
func (e *TikvRegionPeersExtractor) ExplainInfo(_ base.PhysicalPlan) string {
if e.SkipRequest {
return "skip_request:true"
}
Expand Down Expand Up @@ -1706,7 +1705,8 @@ func (e *ColumnsTableExtractor) Extract(_ base.PlanContext,
return remained
}

func (e *ColumnsTableExtractor) explainInfo(_ *PhysicalMemTable) string {
// ExplainInfo implements base.MemTablePredicateExtractor interface.
func (e *ColumnsTableExtractor) ExplainInfo(_ base.PhysicalPlan) string {
if e.SkipRequest {
return "skip_request:true"
}
Expand Down Expand Up @@ -1767,7 +1767,8 @@ func (e *TiKVRegionStatusExtractor) Extract(_ base.PlanContext,
return remained
}

func (e *TiKVRegionStatusExtractor) explainInfo(_ *PhysicalMemTable) string {
// ExplainInfo implements base.MemTablePredicateExtractor interface.
func (e *TiKVRegionStatusExtractor) ExplainInfo(_ base.PhysicalPlan) string {
r := new(bytes.Buffer)
if len(e.tablesID) > 0 {
r.WriteString("table_id in {")
Expand Down Expand Up @@ -1827,7 +1828,8 @@ func (e *InfoSchemaTablesExtractor) Extract(_ base.PlanContext,
return remained
}

func (e *InfoSchemaTablesExtractor) explainInfo(_ *PhysicalMemTable) string {
// ExplainInfo implements base.MemTablePredicateExtractor interface.
func (e *InfoSchemaTablesExtractor) ExplainInfo(_ base.PhysicalPlan) string {
if e.SkipRequest {
return "skip_request:true"
}
Expand Down
12 changes: 6 additions & 6 deletions pkg/planner/core/memtable_predicate_extractor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1721,13 +1721,13 @@ func TestExtractorInPreparedStmt(t *testing.T) {
prepared string
userVars []any
params []any
checker func(extractor plannercore.MemTablePredicateExtractor)
checker func(extractor base.MemTablePredicateExtractor)
}{
{
prepared: "select * from information_schema.TIKV_REGION_STATUS where table_id = ?",
userVars: []any{1},
params: []any{1},
checker: func(extractor plannercore.MemTablePredicateExtractor) {
checker: func(extractor base.MemTablePredicateExtractor) {
rse := extractor.(*plannercore.TiKVRegionStatusExtractor)
tableids := rse.GetTablesID()
slices.Sort(tableids)
Expand All @@ -1738,7 +1738,7 @@ func TestExtractorInPreparedStmt(t *testing.T) {
prepared: "select * from information_schema.TIKV_REGION_STATUS where table_id = ? or table_id = ?",
userVars: []any{1, 2},
params: []any{1, 2},
checker: func(extractor plannercore.MemTablePredicateExtractor) {
checker: func(extractor base.MemTablePredicateExtractor) {
rse := extractor.(*plannercore.TiKVRegionStatusExtractor)
tableids := rse.GetTablesID()
slices.Sort(tableids)
Expand All @@ -1749,7 +1749,7 @@ func TestExtractorInPreparedStmt(t *testing.T) {
prepared: "select * from information_schema.TIKV_REGION_STATUS where table_id in (?,?)",
userVars: []any{1, 2},
params: []any{1, 2},
checker: func(extractor plannercore.MemTablePredicateExtractor) {
checker: func(extractor base.MemTablePredicateExtractor) {
rse := extractor.(*plannercore.TiKVRegionStatusExtractor)
tableids := rse.GetTablesID()
slices.Sort(tableids)
Expand All @@ -1760,7 +1760,7 @@ func TestExtractorInPreparedStmt(t *testing.T) {
prepared: "select * from information_schema.COLUMNS where table_name like ?",
userVars: []any{`"a%"`},
params: []any{"a%"},
checker: func(extractor plannercore.MemTablePredicateExtractor) {
checker: func(extractor base.MemTablePredicateExtractor) {
rse := extractor.(*plannercore.ColumnsTableExtractor)
require.EqualValues(t, []string{"a%"}, rse.TableNamePatterns)
},
Expand All @@ -1773,7 +1773,7 @@ func TestExtractorInPreparedStmt(t *testing.T) {
require.NoError(t, err)
return tt
}()},
checker: func(extractor plannercore.MemTablePredicateExtractor) {
checker: func(extractor base.MemTablePredicateExtractor) {
rse := extractor.(*plannercore.HotRegionsHistoryTableExtractor)
require.Equal(t, timestamp(t, "2019-10-10 10:10:10"), rse.StartTime)
},
Expand Down
2 changes: 1 addition & 1 deletion pkg/planner/core/physical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -838,7 +838,7 @@ type PhysicalMemTable struct {
DBName model.CIStr
Table *model.TableInfo
Columns []*model.ColumnInfo
Extractor MemTablePredicateExtractor
Extractor base.MemTablePredicateExtractor
QueryTimeRange QueryTimeRange
}

Expand Down

0 comments on commit 957e6ae

Please sign in to comment.