diff --git a/pkg/planner/core/generator/hash64_equals/hash64_equals_generator.go b/pkg/planner/core/generator/hash64_equals/hash64_equals_generator.go index 3237a717a34c7..8c8e7bf9d1b64 100644 --- a/pkg/planner/core/generator/hash64_equals/hash64_equals_generator.go +++ b/pkg/planner/core/generator/hash64_equals/hash64_equals_generator.go @@ -38,7 +38,9 @@ import ( func GenHash64Equals4LogicalOps() ([]byte, error) { var structures = []any{logicalop.LogicalJoin{}, logicalop.LogicalAggregation{}, logicalop.LogicalApply{}, logicalop.LogicalExpand{}, logicalop.LogicalLimit{}, logicalop.LogicalMaxOneRow{}, logicalop.DataSource{}, - logicalop.LogicalMemTable{}, logicalop.LogicalUnionAll{}, logicalop.LogicalPartitionUnionAll{}} + logicalop.LogicalMemTable{}, logicalop.LogicalUnionAll{}, logicalop.LogicalPartitionUnionAll{}, logicalop.LogicalProjection{}, + logicalop.LogicalSelection{}, logicalop.LogicalShow{}, logicalop.LogicalShowDDLJobs{}, logicalop.LogicalSort{}, + } c := new(cc) c.write(codeGenHash64EqualsPrefix) for _, s := range structures { @@ -132,7 +134,16 @@ func logicalOpName2PlanCodecString(name string) string { return "plancodec.TypeUnion" case "LogicalPartitionUnionAll": return "plancodec.TypePartitionUnion" - + case "LogicalProjection": + return "plancodec.TypeProj" + case "LogicalSelection": + return "plancodec.TypeSel" + case "LogicalShow": + return "plancodec.TypeShow" + case "LogicalShowDDLJobs": + return "plancodec.TypeShowDDLJobs" + case "LogicalSort": + return "plancodec.TypeSort" default: return "" } diff --git a/pkg/planner/core/operator/logicalop/hash64_equals_generated.go b/pkg/planner/core/operator/logicalop/hash64_equals_generated.go index a1e3345bc141c..6664067b8ea44 100644 --- a/pkg/planner/core/operator/logicalop/hash64_equals_generated.go +++ b/pkg/planner/core/operator/logicalop/hash64_equals_generated.go @@ -638,3 +638,174 @@ func (op *LogicalPartitionUnionAll) Equals(other any) bool { } return true } + +// Hash64 implements the Hash64Equals interface. +func (op *LogicalProjection) Hash64(h base.Hasher) { + h.HashString(plancodec.TypeProj) + op.LogicalSchemaProducer.Hash64(h) + if op.Exprs == nil { + h.HashByte(base.NilFlag) + } else { + h.HashByte(base.NotNilFlag) + h.HashInt(len(op.Exprs)) + for _, one := range op.Exprs { + one.Hash64(h) + } + } + h.HashBool(op.CalculateNoDelay) + h.HashBool(op.Proj4Expand) +} + +// Equals implements the Hash64Equals interface, only receive *LogicalProjection pointer. +func (op *LogicalProjection) Equals(other any) bool { + op2, ok := other.(*LogicalProjection) + if !ok { + return false + } + if op == nil { + return op2 == nil + } + if op2 == nil { + return false + } + if !op.LogicalSchemaProducer.Equals(&op2.LogicalSchemaProducer) { + return false + } + if (op.Exprs == nil && op2.Exprs != nil) || (op.Exprs != nil && op2.Exprs == nil) || len(op.Exprs) != len(op2.Exprs) { + return false + } + for i, one := range op.Exprs { + if !one.Equals(op2.Exprs[i]) { + return false + } + } + if op.CalculateNoDelay != op2.CalculateNoDelay { + return false + } + if op.Proj4Expand != op2.Proj4Expand { + return false + } + return true +} + +// Hash64 implements the Hash64Equals interface. +func (op *LogicalSelection) Hash64(h base.Hasher) { + h.HashString(plancodec.TypeSel) + if op.Conditions == nil { + h.HashByte(base.NilFlag) + } else { + h.HashByte(base.NotNilFlag) + h.HashInt(len(op.Conditions)) + for _, one := range op.Conditions { + one.Hash64(h) + } + } +} + +// Equals implements the Hash64Equals interface, only receive *LogicalSelection pointer. +func (op *LogicalSelection) Equals(other any) bool { + op2, ok := other.(*LogicalSelection) + if !ok { + return false + } + if op == nil { + return op2 == nil + } + if op2 == nil { + return false + } + if (op.Conditions == nil && op2.Conditions != nil) || (op.Conditions != nil && op2.Conditions == nil) || len(op.Conditions) != len(op2.Conditions) { + return false + } + for i, one := range op.Conditions { + if !one.Equals(op2.Conditions[i]) { + return false + } + } + return true +} + +// Hash64 implements the Hash64Equals interface. +func (op *LogicalShow) Hash64(h base.Hasher) { + h.HashString(plancodec.TypeShow) + op.LogicalSchemaProducer.Hash64(h) +} + +// Equals implements the Hash64Equals interface, only receive *LogicalShow pointer. +func (op *LogicalShow) Equals(other any) bool { + op2, ok := other.(*LogicalShow) + if !ok { + return false + } + if op == nil { + return op2 == nil + } + if op2 == nil { + return false + } + if !op.LogicalSchemaProducer.Equals(&op2.LogicalSchemaProducer) { + return false + } + return true +} + +// Hash64 implements the Hash64Equals interface. +func (op *LogicalShowDDLJobs) Hash64(h base.Hasher) { + h.HashString(plancodec.TypeShowDDLJobs) + op.LogicalSchemaProducer.Hash64(h) +} + +// Equals implements the Hash64Equals interface, only receive *LogicalShowDDLJobs pointer. +func (op *LogicalShowDDLJobs) Equals(other any) bool { + op2, ok := other.(*LogicalShowDDLJobs) + if !ok { + return false + } + if op == nil { + return op2 == nil + } + if op2 == nil { + return false + } + if !op.LogicalSchemaProducer.Equals(&op2.LogicalSchemaProducer) { + return false + } + return true +} + +// Hash64 implements the Hash64Equals interface. +func (op *LogicalSort) Hash64(h base.Hasher) { + h.HashString(plancodec.TypeSort) + if op.ByItems == nil { + h.HashByte(base.NilFlag) + } else { + h.HashByte(base.NotNilFlag) + h.HashInt(len(op.ByItems)) + for _, one := range op.ByItems { + one.Hash64(h) + } + } +} + +// Equals implements the Hash64Equals interface, only receive *LogicalSort pointer. +func (op *LogicalSort) Equals(other any) bool { + op2, ok := other.(*LogicalSort) + if !ok { + return false + } + if op == nil { + return op2 == nil + } + if op2 == nil { + return false + } + if (op.ByItems == nil && op2.ByItems != nil) || (op.ByItems != nil && op2.ByItems == nil) || len(op.ByItems) != len(op2.ByItems) { + return false + } + for i, one := range op.ByItems { + if !one.Equals(op2.ByItems[i]) { + return false + } + } + return true +} diff --git a/pkg/planner/core/operator/logicalop/logical_projection.go b/pkg/planner/core/operator/logicalop/logical_projection.go index 15bc95a0f9448..d4b4b6fa5d816 100644 --- a/pkg/planner/core/operator/logicalop/logical_projection.go +++ b/pkg/planner/core/operator/logicalop/logical_projection.go @@ -20,7 +20,6 @@ import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/cardinality" - base2 "github.com/pingcap/tidb/pkg/planner/cascades/base" "github.com/pingcap/tidb/pkg/planner/core/base" ruleutil "github.com/pingcap/tidb/pkg/planner/core/rule/util" fd "github.com/pingcap/tidb/pkg/planner/funcdep" @@ -35,19 +34,19 @@ import ( // LogicalProjection represents a select fields plan. type LogicalProjection struct { - LogicalSchemaProducer + LogicalSchemaProducer `hash64-equals:"true"` - Exprs []expression.Expression + Exprs []expression.Expression `hash64-equals:"true"` // CalculateNoDelay indicates this Projection is the root Plan and should be // calculated without delay and will not return any result to client. // Currently it is "true" only when the current sql query is a "DO" statement. // See "https://dev.mysql.com/doc/refman/5.7/en/do.html" for more detail. - CalculateNoDelay bool + CalculateNoDelay bool `hash64-equals:"true"` // Proj4Expand is used for expand to project same column reference, while these // col may be filled with null so we couldn't just eliminate this projection itself. - Proj4Expand bool + Proj4Expand bool `hash64-equals:"true"` } // Init initializes LogicalProjection. @@ -56,43 +55,6 @@ func (p LogicalProjection) Init(ctx base.PlanContext, qbOffset int) *LogicalProj return &p } -// *************************** start implementation of HashEquals interface **************************** - -// Hash64 implements the base.Hash64.<0th> interface. -func (p *LogicalProjection) Hash64(h base2.Hasher) { - h.HashInt(len(p.Exprs)) - for _, one := range p.Exprs { - one.Hash64(h) - } - h.HashBool(p.CalculateNoDelay) - h.HashBool(p.Proj4Expand) -} - -// Equals implements the base.HashEquals.<1st> interface. -func (p *LogicalProjection) Equals(other any) bool { - if other == nil { - return false - } - var p2 *LogicalProjection - switch x := other.(type) { - case *LogicalProjection: - p2 = x - case LogicalProjection: - p2 = &x - default: - return false - } - if len(p.Exprs) != len(p2.Exprs) { - return false - } - for i, one := range p.Exprs { - if !one.Equals(p2.Exprs[i]) { - return false - } - } - return p.CalculateNoDelay == p2.CalculateNoDelay && p.Proj4Expand == p2.Proj4Expand -} - // *************************** start implementation of Plan interface ********************************** // ExplainInfo implements Plan interface. diff --git a/pkg/planner/core/operator/logicalop/logical_selection.go b/pkg/planner/core/operator/logicalop/logical_selection.go index 304ea704508f7..426ff36590933 100644 --- a/pkg/planner/core/operator/logicalop/logical_selection.go +++ b/pkg/planner/core/operator/logicalop/logical_selection.go @@ -43,7 +43,7 @@ type LogicalSelection struct { // Originally the WHERE or ON condition is parsed into a single expression, // but after we converted to CNF(Conjunctive normal form), it can be // split into a list of AND conditions. - Conditions []expression.Expression + Conditions []expression.Expression `hash64-equals:"true"` } // Init initializes LogicalSelection. diff --git a/pkg/planner/core/operator/logicalop/logical_show.go b/pkg/planner/core/operator/logicalop/logical_show.go index bf400ee91ab03..2f6eac4d36e50 100644 --- a/pkg/planner/core/operator/logicalop/logical_show.go +++ b/pkg/planner/core/operator/logicalop/logical_show.go @@ -32,7 +32,7 @@ import ( // LogicalShow represents a show plan. type LogicalShow struct { - LogicalSchemaProducer + LogicalSchemaProducer `hash64-equals:"true"` ShowContents Extractor base.ShowPredicateExtractor diff --git a/pkg/planner/core/operator/logicalop/logical_show_ddl_jobs.go b/pkg/planner/core/operator/logicalop/logical_show_ddl_jobs.go index c4eea62fa5ceb..5ca302da4ea55 100644 --- a/pkg/planner/core/operator/logicalop/logical_show_ddl_jobs.go +++ b/pkg/planner/core/operator/logicalop/logical_show_ddl_jobs.go @@ -25,7 +25,7 @@ import ( // LogicalShowDDLJobs is for showing DDL job list. type LogicalShowDDLJobs struct { - LogicalSchemaProducer + LogicalSchemaProducer `hash64-equals:"true"` JobNumber int64 } diff --git a/pkg/planner/core/operator/logicalop/logical_sort.go b/pkg/planner/core/operator/logicalop/logical_sort.go index c3b4cd3ed3683..4028a2128cc57 100644 --- a/pkg/planner/core/operator/logicalop/logical_sort.go +++ b/pkg/planner/core/operator/logicalop/logical_sort.go @@ -33,7 +33,7 @@ import ( type LogicalSort struct { BaseLogicalPlan - ByItems []*util.ByItems + ByItems []*util.ByItems `hash64-equals:"true"` } // Init initializes LogicalSort.