diff --git a/docs/design/2022-01-04-integer-shard-index.md b/docs/design/2022-01-04-integer-shard-index.md index 5ee835d8e8c93..f56280f42d6e5 100644 --- a/docs/design/2022-01-04-integer-shard-index.md +++ b/docs/design/2022-01-04-integer-shard-index.md @@ -115,7 +115,7 @@ The entry point to add the `tidb_shard` expression is the function as bellow. We func (ds *DataSource) PredicatePushDown(predicates []expression.Expression, opt *logicalOptimizeOp) ([]expression.Expression, LogicalPlan) { predicates = expression.PropagateConstant(ds.ctx, predicates) - predicates = DeleteTrueExprs(ds, predicates) + predicates = constraint.DeleteTrueExprs(ds, predicates) // Add tidb_shard() prefix to the condtion for shard index in some scenarios // TODO: remove it to the place building logical plan predicates = ds.AddPrefix4ShardIndexes(ds.ctx, predicates) diff --git a/pkg/executor/benchmark_test.go b/pkg/executor/benchmark_test.go index 2f91a32e8abc4..dfe65f0954bb3 100644 --- a/pkg/executor/benchmark_test.go +++ b/pkg/executor/benchmark_test.go @@ -586,7 +586,7 @@ type hashJoinTestCase struct { concurrency int ctx sessionctx.Context keyIdx []int - joinType core.JoinType + joinType logicalop.JoinType disk bool useOuterToBuild bool rawData string @@ -607,7 +607,7 @@ func (tc hashJoinTestCase) String() string { tc.rows, tc.cols, tc.concurrency, tc.keyIdx, tc.disk) } -func defaultHashJoinTestCase(cols []*types.FieldType, joinType core.JoinType, useOuterToBuild bool) *hashJoinTestCase { +func defaultHashJoinTestCase(cols []*types.FieldType, joinType logicalop.JoinType, useOuterToBuild bool) *hashJoinTestCase { ctx := mock.NewContext() ctx.GetSessionVars().InitChunkSize = variable.DefInitChunkSize ctx.GetSessionVars().MaxChunkSize = variable.DefMaxChunkSize @@ -621,10 +621,10 @@ func defaultHashJoinTestCase(cols []*types.FieldType, joinType core.JoinType, us return tc } -func prepareResolveIndices(joinSchema, lSchema, rSchema *expression.Schema, joinType core.JoinType) *expression.Schema { +func prepareResolveIndices(joinSchema, lSchema, rSchema *expression.Schema, joinType logicalop.JoinType) *expression.Schema { colsNeedResolving := joinSchema.Len() // The last output column of this two join is the generated column to indicate whether the row is matched or not. - if joinType == core.LeftOuterSemiJoin || joinType == core.AntiLeftOuterSemiJoin { + if joinType == logicalop.LeftOuterSemiJoin || joinType == logicalop.AntiLeftOuterSemiJoin { colsNeedResolving-- } mergedSchema := expression.MergeSchema(lSchema, rSchema) @@ -687,7 +687,7 @@ func prepare4HashJoinV2(testCase *hashJoinTestCase, innerExec, outerExec exec.Ex // todo: need systematic way to protect. // physical join should resolveIndices to get right schema column index. // otherwise, markChildrenUsedColsForTest will fail below. - joinSchema = prepareResolveIndices(joinSchema, innerExec.Schema(), outerExec.Schema(), core.InnerJoin) + joinSchema = prepareResolveIndices(joinSchema, innerExec.Schema(), outerExec.Schema(), logicalop.InnerJoin) joinKeysColIdx := make([]int, 0, len(testCase.keyIdx)) joinKeysColIdx = append(joinKeysColIdx, testCase.keyIdx...) @@ -776,7 +776,7 @@ func prepare4HashJoin(testCase *hashJoinTestCase, innerExec, outerExec exec.Exec // todo: need systematic way to protect. // physical join should resolveIndices to get right schema column index. // otherwise, markChildrenUsedColsForTest will fail below. - joinSchema = prepareResolveIndices(joinSchema, innerExec.Schema(), outerExec.Schema(), core.InnerJoin) + joinSchema = prepareResolveIndices(joinSchema, innerExec.Schema(), outerExec.Schema(), logicalop.InnerJoin) joinKeysColIdx := make([]int, 0, len(testCase.keyIdx)) joinKeysColIdx = append(joinKeysColIdx, testCase.keyIdx...) diff --git a/pkg/executor/builder.go b/pkg/executor/builder.go index 1b652ff62f477..387817efda142 100644 --- a/pkg/executor/builder.go +++ b/pkg/executor/builder.go @@ -1428,7 +1428,7 @@ func (b *executorBuilder) buildMergeJoin(v *plannercore.PhysicalMergeJoin) exec. defaultValues := v.DefaultValues if defaultValues == nil { - if v.JoinType == plannercore.RightOuterJoin { + if v.JoinType == logicalop.RightOuterJoin { defaultValues = make([]types.Datum, leftExec.Schema().Len()) } else { defaultValues = make([]types.Datum, rightExec.Schema().Len()) @@ -1436,7 +1436,7 @@ func (b *executorBuilder) buildMergeJoin(v *plannercore.PhysicalMergeJoin) exec. } colsFromChildren := v.Schema().Columns - if v.JoinType == plannercore.LeftOuterSemiJoin || v.JoinType == plannercore.AntiLeftOuterSemiJoin { + if v.JoinType == logicalop.LeftOuterSemiJoin || v.JoinType == logicalop.AntiLeftOuterSemiJoin { colsFromChildren = colsFromChildren[:len(colsFromChildren)-1] } @@ -1447,7 +1447,7 @@ func (b *executorBuilder) buildMergeJoin(v *plannercore.PhysicalMergeJoin) exec. Joiner: join.NewJoiner( b.ctx, v.JoinType, - v.JoinType == plannercore.RightOuterJoin, + v.JoinType == logicalop.RightOuterJoin, defaultValues, v.OtherConditions, exec.RetTypes(leftExec), @@ -1470,7 +1470,7 @@ func (b *executorBuilder) buildMergeJoin(v *plannercore.PhysicalMergeJoin) exec. Filters: v.RightConditions, } - if v.JoinType == plannercore.RightOuterJoin { + if v.JoinType == logicalop.RightOuterJoin { e.InnerTable = leftTable e.OuterTable = rightTable } else { @@ -1603,7 +1603,7 @@ func (b *executorBuilder) buildHashJoinV2(v *plannercore.PhysicalHashJoin) exec. } colsFromChildren := v.Schema().Columns - if v.JoinType == plannercore.LeftOuterSemiJoin || v.JoinType == plannercore.AntiLeftOuterSemiJoin { + if v.JoinType == logicalop.LeftOuterSemiJoin || v.JoinType == logicalop.AntiLeftOuterSemiJoin { // the matched column is added inside join colsFromChildren = colsFromChildren[:len(colsFromChildren)-1] } @@ -1772,7 +1772,7 @@ func (b *executorBuilder) buildHashJoin(v *plannercore.PhysicalHashJoin) exec.Ex } isNAJoin := len(v.LeftNAJoinKeys) > 0 colsFromChildren := v.Schema().Columns - if v.JoinType == plannercore.LeftOuterSemiJoin || v.JoinType == plannercore.AntiLeftOuterSemiJoin { + if v.JoinType == logicalop.LeftOuterSemiJoin || v.JoinType == logicalop.AntiLeftOuterSemiJoin { colsFromChildren = colsFromChildren[:len(colsFromChildren)-1] } childrenUsedSchema := markChildrenUsedCols(colsFromChildren, v.Children()[0].Schema(), v.Children()[1].Schema()) @@ -2464,7 +2464,7 @@ func (b *executorBuilder) buildApply(v *plannercore.PhysicalApply) exec.Executor OuterExec: outerExec, OuterFilter: outerFilter, InnerFilter: innerFilter, - Outer: v.JoinType != plannercore.InnerJoin, + Outer: v.JoinType != logicalop.InnerJoin, Joiner: tupleJoiner, OuterSchema: v.OuterSchema, Sctx: b.ctx, @@ -2505,7 +2505,7 @@ func (b *executorBuilder) buildApply(v *plannercore.PhysicalApply) exec.Executor outerExec: outerExec, outerFilter: outerFilter, innerFilter: innerFilters, - outer: v.JoinType != plannercore.InnerJoin, + outer: v.JoinType != logicalop.InnerJoin, joiners: joiners, corCols: corCols, concurrency: v.Concurrency, @@ -3294,7 +3294,7 @@ func (b *executorBuilder) buildIndexLookUpJoin(v *plannercore.PhysicalIndexJoin) Finished: &atomic.Value{}, } colsFromChildren := v.Schema().Columns - if v.JoinType == plannercore.LeftOuterSemiJoin || v.JoinType == plannercore.AntiLeftOuterSemiJoin { + if v.JoinType == logicalop.LeftOuterSemiJoin || v.JoinType == logicalop.AntiLeftOuterSemiJoin { colsFromChildren = colsFromChildren[:len(colsFromChildren)-1] } childrenUsedSchema := markChildrenUsedCols(colsFromChildren, v.Children()[0].Schema(), v.Children()[1].Schema()) @@ -3420,7 +3420,7 @@ func (b *executorBuilder) buildIndexLookUpMergeJoin(v *plannercore.PhysicalIndex LastColHelper: v.CompareFilters, } colsFromChildren := v.Schema().Columns - if v.JoinType == plannercore.LeftOuterSemiJoin || v.JoinType == plannercore.AntiLeftOuterSemiJoin { + if v.JoinType == logicalop.LeftOuterSemiJoin || v.JoinType == logicalop.AntiLeftOuterSemiJoin { colsFromChildren = colsFromChildren[:len(colsFromChildren)-1] } childrenUsedSchema := markChildrenUsedCols(colsFromChildren, v.Children()[0].Schema(), v.Children()[1].Schema()) diff --git a/pkg/executor/executor_required_rows_test.go b/pkg/executor/executor_required_rows_test.go index 734f67859636e..a81dda7e1512a 100644 --- a/pkg/executor/executor_required_rows_test.go +++ b/pkg/executor/executor_required_rows_test.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" plannercore "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" @@ -697,8 +698,8 @@ func TestMergeJoinRequiredRows(t *testing.T) { panic("not support") } } - joinTypes := []plannercore.JoinType{plannercore.RightOuterJoin, plannercore.LeftOuterJoin, - plannercore.LeftOuterSemiJoin, plannercore.AntiLeftOuterSemiJoin} + joinTypes := []logicalop.JoinType{logicalop.RightOuterJoin, logicalop.LeftOuterJoin, + logicalop.LeftOuterSemiJoin, logicalop.AntiLeftOuterSemiJoin} for _, joinType := range joinTypes { ctx := defaultCtx() required := make([]int, 100) @@ -720,8 +721,8 @@ func TestMergeJoinRequiredRows(t *testing.T) { } } -func buildMergeJoinExec(ctx sessionctx.Context, joinType plannercore.JoinType, innerSrc, outerSrc exec.Executor) exec.Executor { - if joinType == plannercore.RightOuterJoin { +func buildMergeJoinExec(ctx sessionctx.Context, joinType logicalop.JoinType, innerSrc, outerSrc exec.Executor) exec.Executor { + if joinType == logicalop.RightOuterJoin { innerSrc, outerSrc = outerSrc, innerSrc } diff --git a/pkg/executor/join/BUILD.bazel b/pkg/executor/join/BUILD.bazel index 8c0179a7a6e59..e5280312ad640 100644 --- a/pkg/executor/join/BUILD.bazel +++ b/pkg/executor/join/BUILD.bazel @@ -31,6 +31,7 @@ go_library( "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/planner/core", + "//pkg/planner/core/operator/logicalop", "//pkg/sessionctx", "//pkg/sessionctx/stmtctx", "//pkg/sessionctx/variable", @@ -87,7 +88,7 @@ go_test( "//pkg/expression", "//pkg/parser/ast", "//pkg/parser/mysql", - "//pkg/planner/core", + "//pkg/planner/core/operator/logicalop", "//pkg/session", "//pkg/sessionctx", "//pkg/sessionctx/variable", diff --git a/pkg/executor/join/base_join_probe.go b/pkg/executor/join/base_join_probe.go index 64a815bafdc51..6ef5704733956 100644 --- a/pkg/executor/join/base_join_probe.go +++ b/pkg/executor/join/base_join_probe.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/parser/mysql" - "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" "github.com/pingcap/tidb/pkg/util/codec" @@ -498,7 +498,7 @@ func isKeyMatched(keyMode keyMode, serializedKey []byte, rowStart unsafe.Pointer } // NewJoinProbe create a join probe used for hash join v2 -func NewJoinProbe(ctx *HashJoinCtxV2, workID uint, joinType core.JoinType, keyIndex []int, joinedColumnTypes, probeKeyTypes []*types.FieldType, rightAsBuildSide bool) ProbeV2 { +func NewJoinProbe(ctx *HashJoinCtxV2, workID uint, joinType logicalop.JoinType, keyIndex []int, joinedColumnTypes, probeKeyTypes []*types.FieldType, rightAsBuildSide bool) ProbeV2 { base := baseJoinProbe{ ctx: ctx, workID: workID, @@ -540,11 +540,11 @@ func NewJoinProbe(ctx *HashJoinCtxV2, workID uint, joinType core.JoinType, keyIn base.rowIndexInfos = make([]*matchedRowInfo, 0, chunk.InitialCapacity) } switch joinType { - case core.InnerJoin: + case logicalop.InnerJoin: return &innerJoinProbe{base} - case core.LeftOuterJoin: + case logicalop.LeftOuterJoin: return newOuterJoinProbe(base, !rightAsBuildSide, rightAsBuildSide) - case core.RightOuterJoin: + case logicalop.RightOuterJoin: return newOuterJoinProbe(base, rightAsBuildSide, rightAsBuildSide) default: panic("unsupported join type") diff --git a/pkg/executor/join/hash_join_base.go b/pkg/executor/join/hash_join_base.go index 7947b96653e6c..573f7614b99b1 100644 --- a/pkg/executor/join/hash_join_base.go +++ b/pkg/executor/join/hash_join_base.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/executor/internal/exec" - plannercore "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/chunk" @@ -55,7 +55,7 @@ type hashJoinCtxBase struct { finished atomic.Bool IsNullEQ []bool buildFinished chan error - JoinType plannercore.JoinType + JoinType logicalop.JoinType IsNullAware bool memTracker *memory.Tracker // track memory usage. diskTracker *disk.Tracker // track disk usage. diff --git a/pkg/executor/join/hash_join_v1.go b/pkg/executor/join/hash_join_v1.go index 649eac1eb466b..981c652e5a5a3 100644 --- a/pkg/executor/join/hash_join_v1.go +++ b/pkg/executor/join/hash_join_v1.go @@ -30,7 +30,7 @@ import ( "github.com/pingcap/tidb/pkg/executor/unionexec" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/parser/terror" - plannercore "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/types" @@ -216,7 +216,7 @@ func (e *HashJoinV1Exec) fetchAndProbeHashTable(ctx context.Context) { defer trace.StartRegion(ctx, "HashJoinProbeSideFetcher").End() e.ProbeSideTupleFetcher.fetchProbeSideChunks(ctx, e.MaxChunkSize(), func() bool { return e.ProbeSideTupleFetcher.RowContainer.Len() == uint64(0) - }, e.ProbeSideTupleFetcher.JoinType == plannercore.InnerJoin || e.ProbeSideTupleFetcher.JoinType == plannercore.SemiJoin, + }, e.ProbeSideTupleFetcher.JoinType == logicalop.InnerJoin || e.ProbeSideTupleFetcher.JoinType == logicalop.SemiJoin, false, e.ProbeSideTupleFetcher.IsOuterJoin, &e.ProbeSideTupleFetcher.hashJoinCtxBase) }, e.ProbeSideTupleFetcher.handleProbeSideFetcherPanic) @@ -737,8 +737,8 @@ func (w *ProbeWorkerV1) joinNAASJMatchProbeSideRow2Chunk(probeKey uint64, probeK // For NA-AntiLeftOuterSemiJoin, we couldn't match null-bucket first, because once y set has a same key x and null // key, we should return the result as left side row appended with a scalar value 0 which is from same key matching failure. func (w *ProbeWorkerV1) joinNAAJMatchProbeSideRow2Chunk(probeKey uint64, probeKeyNullBits *bitmap.ConcurrentBitmap, probeSideRow chunk.Row, hCtx *HashContext, joinResult *hashjoinWorkerResult) (bool, int64, *hashjoinWorkerResult) { - naAntiSemiJoin := w.HashJoinCtx.JoinType == plannercore.AntiSemiJoin && w.HashJoinCtx.IsNullAware - naAntiLeftOuterSemiJoin := w.HashJoinCtx.JoinType == plannercore.AntiLeftOuterSemiJoin && w.HashJoinCtx.IsNullAware + naAntiSemiJoin := w.HashJoinCtx.JoinType == logicalop.AntiSemiJoin && w.HashJoinCtx.IsNullAware + naAntiLeftOuterSemiJoin := w.HashJoinCtx.JoinType == logicalop.AntiLeftOuterSemiJoin && w.HashJoinCtx.IsNullAware if naAntiSemiJoin { return w.joinNAASJMatchProbeSideRow2Chunk(probeKey, probeKeyNullBits, probeSideRow, hCtx, joinResult) } diff --git a/pkg/executor/join/hash_join_v2.go b/pkg/executor/join/hash_join_v2.go index 9fbb587af4a3e..e510e82c528f4 100644 --- a/pkg/executor/join/hash_join_v2.go +++ b/pkg/executor/join/hash_join_v2.go @@ -28,7 +28,7 @@ import ( "github.com/pingcap/tidb/pkg/executor/internal/exec" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/parser/mysql" - plannercore "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/channel" @@ -335,10 +335,10 @@ func (e *HashJoinV2Exec) Open(ctx context.Context) error { } func (fetcher *ProbeSideTupleFetcherV2) shouldLimitProbeFetchSize() bool { - if fetcher.JoinType == plannercore.LeftOuterJoin && fetcher.RightAsBuildSide { + if fetcher.JoinType == logicalop.LeftOuterJoin && fetcher.RightAsBuildSide { return true } - if fetcher.JoinType == plannercore.RightOuterJoin && !fetcher.RightAsBuildSide { + if fetcher.JoinType == logicalop.RightOuterJoin && !fetcher.RightAsBuildSide { return true } return false @@ -374,13 +374,13 @@ func (w *BuildWorkerV2) splitPartitionAndAppendToRowTable(typeCtx types.Context, func (e *HashJoinV2Exec) canSkipProbeIfHashTableIsEmpty() bool { switch e.JoinType { - case plannercore.InnerJoin: + case logicalop.InnerJoin: return true - case plannercore.LeftOuterJoin: + case logicalop.LeftOuterJoin: return !e.RightAsBuildSide - case plannercore.RightOuterJoin: + case logicalop.RightOuterJoin: return e.RightAsBuildSide - case plannercore.SemiJoin: + case logicalop.SemiJoin: return e.RightAsBuildSide default: return false diff --git a/pkg/executor/join/inner_join_probe_test.go b/pkg/executor/join/inner_join_probe_test.go index ddb43a1662453..dea29dce5c248 100644 --- a/pkg/executor/join/inner_join_probe_test.go +++ b/pkg/executor/join/inner_join_probe_test.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" - plannercore "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" @@ -199,7 +199,7 @@ func checkChunksEqual(t *testing.T, expectedChunks []*chunk.Chunk, resultChunks func testJoinProbe(t *testing.T, withSel bool, leftKeyIndex []int, rightKeyIndex []int, leftKeyTypes []*types.FieldType, rightKeyTypes []*types.FieldType, leftTypes []*types.FieldType, rightTypes []*types.FieldType, rightAsBuildSide bool, leftUsed []int, rightUsed []int, leftUsedByOtherCondition []int, rightUsedByOtherCondition []int, leftFilter expression.CNFExprs, rightFilter expression.CNFExprs, - otherCondition expression.CNFExprs, partitionNumber int, joinType plannercore.JoinType, inputRowNumber int) { + otherCondition expression.CNFExprs, partitionNumber int, joinType logicalop.JoinType, inputRowNumber int) { buildKeyIndex, probeKeyIndex := leftKeyIndex, rightKeyIndex buildKeyTypes, probeKeyTypes := leftKeyTypes, rightKeyTypes buildTypes, probeTypes := leftTypes, rightTypes @@ -214,29 +214,29 @@ func testJoinProbe(t *testing.T, withSel bool, leftKeyIndex []int, rightKeyIndex buildUsed = rightUsed buildUsedByOtherCondition = rightUsedByOtherCondition buildFilter, probeFilter = rightFilter, leftFilter - if joinType == plannercore.RightOuterJoin { + if joinType == logicalop.RightOuterJoin { needUsedFlag = true } } else { switch joinType { - case plannercore.LeftOuterJoin, plannercore.SemiJoin, plannercore.AntiSemiJoin: + case logicalop.LeftOuterJoin, logicalop.SemiJoin, logicalop.AntiSemiJoin: needUsedFlag = true - case plannercore.LeftOuterSemiJoin, plannercore.AntiLeftOuterSemiJoin: + case logicalop.LeftOuterSemiJoin, logicalop.AntiLeftOuterSemiJoin: require.NoError(t, errors.New("left semi/anti join does not support use left as build side")) } } switch joinType { - case plannercore.InnerJoin: + case logicalop.InnerJoin: require.Equal(t, 0, len(leftFilter), "inner join does not support left filter") require.Equal(t, 0, len(rightFilter), "inner join does not support right filter") - case plannercore.LeftOuterJoin: + case logicalop.LeftOuterJoin: require.Equal(t, 0, len(rightFilter), "left outer join does not support right filter") - case plannercore.RightOuterJoin: + case logicalop.RightOuterJoin: require.Equal(t, 0, len(leftFilter), "right outer join does not support left filter") - case plannercore.SemiJoin, plannercore.AntiSemiJoin: + case logicalop.SemiJoin, logicalop.AntiSemiJoin: require.Equal(t, 0, len(leftFilter), "semi/anti join does not support left filter") require.Equal(t, 0, len(rightFilter), "semi/anti join does not support right filter") - case plannercore.LeftOuterSemiJoin, plannercore.AntiLeftOuterSemiJoin: + case logicalop.LeftOuterSemiJoin, logicalop.AntiLeftOuterSemiJoin: require.Equal(t, 0, len(rightFilter), "left outer semi/anti join does not support right filter") } joinedTypes := make([]*types.FieldType, 0, len(leftTypes)+len(rightTypes)) @@ -245,13 +245,13 @@ func testJoinProbe(t *testing.T, withSel bool, leftKeyIndex []int, rightKeyIndex resultTypes := make([]*types.FieldType, 0, len(leftUsed)+len(rightUsed)) for _, colIndex := range leftUsed { resultTypes = append(resultTypes, leftTypes[colIndex].Clone()) - if joinType == plannercore.RightOuterJoin { + if joinType == logicalop.RightOuterJoin { resultTypes[len(resultTypes)-1].DelFlag(mysql.NotNullFlag) } } for _, colIndex := range rightUsed { resultTypes = append(resultTypes, rightTypes[colIndex].Clone()) - if joinType == plannercore.LeftOuterJoin { + if joinType == logicalop.LeftOuterJoin { resultTypes[len(resultTypes)-1].DelFlag(mysql.NotNullFlag) } } @@ -390,15 +390,15 @@ func testJoinProbe(t *testing.T, withSel bool, leftKeyIndex []int, rightKeyIndex checkVirtualRows(t, resultChunks) switch joinType { - case plannercore.InnerJoin: + case logicalop.InnerJoin: expectedChunks := genInnerJoinResult(t, hashJoinCtx.SessCtx, leftChunks, rightChunks, leftKeyIndex, rightKeyIndex, leftTypes, rightTypes, leftKeyTypes, rightKeyTypes, leftUsed, rightUsed, otherCondition, resultTypes) checkChunksEqual(t, expectedChunks, resultChunks, resultTypes) - case plannercore.LeftOuterJoin: + case logicalop.LeftOuterJoin: expectedChunks := genLeftOuterJoinResult(t, hashJoinCtx.SessCtx, leftFilter, leftChunks, rightChunks, leftKeyIndex, rightKeyIndex, leftTypes, rightTypes, leftKeyTypes, rightKeyTypes, leftUsed, rightUsed, otherCondition, resultTypes) checkChunksEqual(t, expectedChunks, resultChunks, resultTypes) - case plannercore.RightOuterJoin: + case logicalop.RightOuterJoin: expectedChunks := genRightOuterJoinResult(t, hashJoinCtx.SessCtx, rightFilter, leftChunks, rightChunks, leftKeyIndex, rightKeyIndex, leftTypes, rightTypes, leftKeyTypes, rightKeyTypes, leftUsed, rightUsed, otherCondition, resultTypes) checkChunksEqual(t, expectedChunks, resultChunks, resultTypes) @@ -461,9 +461,9 @@ func TestInnerJoinProbeBasic(t *testing.T) { // inner join does not have left/right Filter for _, rightAsBuild := range rightAsBuildSide { testJoinProbe(t, false, tc.leftKeyIndex, tc.rightKeyIndex, tc.leftKeyTypes, tc.rightKeyTypes, tc.leftTypes, tc.rightTypes, rightAsBuild, tc.leftUsed, - tc.rightUsed, tc.leftUsedByOtherCondition, tc.rightUsedByOtherCondition, nil, nil, tc.otherCondition, partitionNumber, plannercore.InnerJoin, 200) + tc.rightUsed, tc.leftUsedByOtherCondition, tc.rightUsedByOtherCondition, nil, nil, tc.otherCondition, partitionNumber, logicalop.InnerJoin, 200) testJoinProbe(t, false, tc.leftKeyIndex, tc.rightKeyIndex, toNullableTypes(tc.leftKeyTypes), toNullableTypes(tc.rightKeyTypes), - toNullableTypes(tc.leftTypes), toNullableTypes(tc.rightTypes), rightAsBuild, tc.leftUsed, tc.rightUsed, tc.leftUsedByOtherCondition, tc.rightUsedByOtherCondition, nil, nil, tc.otherCondition, partitionNumber, plannercore.InnerJoin, 200) + toNullableTypes(tc.leftTypes), toNullableTypes(tc.rightTypes), rightAsBuild, tc.leftUsed, tc.rightUsed, tc.leftUsedByOtherCondition, tc.rightUsedByOtherCondition, nil, nil, tc.otherCondition, partitionNumber, logicalop.InnerJoin, 200) } } } @@ -522,8 +522,8 @@ func TestInnerJoinProbeAllJoinKeys(t *testing.T) { for _, rightAsBuild := range rightAsBuildSide { lKeyTypes := []*types.FieldType{lTypes[i]} rKeyTypes := []*types.FieldType{rTypes[i]} - testJoinProbe(t, false, []int{i}, []int{i}, lKeyTypes, rKeyTypes, lTypes, rTypes, rightAsBuild, lUsed, rUsed, nil, nil, nil, nil, nil, partitionNumber, plannercore.InnerJoin, 100) - testJoinProbe(t, false, []int{i}, []int{i}, toNullableTypes(lKeyTypes), toNullableTypes(rKeyTypes), nullableLTypes, nullableRTypes, rightAsBuild, lUsed, rUsed, nil, nil, nil, nil, nil, partitionNumber, plannercore.InnerJoin, 100) + testJoinProbe(t, false, []int{i}, []int{i}, lKeyTypes, rKeyTypes, lTypes, rTypes, rightAsBuild, lUsed, rUsed, nil, nil, nil, nil, nil, partitionNumber, logicalop.InnerJoin, 100) + testJoinProbe(t, false, []int{i}, []int{i}, toNullableTypes(lKeyTypes), toNullableTypes(rKeyTypes), nullableLTypes, nullableRTypes, rightAsBuild, lUsed, rUsed, nil, nil, nil, nil, nil, partitionNumber, logicalop.InnerJoin, 100) } } // composed key @@ -531,29 +531,29 @@ func TestInnerJoinProbeAllJoinKeys(t *testing.T) { for _, rightAsBuild := range rightAsBuildSide { lKeyTypes := []*types.FieldType{intTp, uintTp} rKeyTypes := []*types.FieldType{intTp, uintTp} - testJoinProbe(t, false, []int{1, 2}, []int{1, 2}, lKeyTypes, rKeyTypes, lTypes, rTypes, rightAsBuild, lUsed, rUsed, nil, nil, nil, nil, nil, partitionNumber, plannercore.InnerJoin, 100) - testJoinProbe(t, false, []int{1, 2}, []int{1, 2}, toNullableTypes(lKeyTypes), toNullableTypes(rKeyTypes), nullableLTypes, nullableRTypes, rightAsBuild, lUsed, rUsed, nil, nil, nil, nil, nil, partitionNumber, plannercore.InnerJoin, 100) + testJoinProbe(t, false, []int{1, 2}, []int{1, 2}, lKeyTypes, rKeyTypes, lTypes, rTypes, rightAsBuild, lUsed, rUsed, nil, nil, nil, nil, nil, partitionNumber, logicalop.InnerJoin, 100) + testJoinProbe(t, false, []int{1, 2}, []int{1, 2}, toNullableTypes(lKeyTypes), toNullableTypes(rKeyTypes), nullableLTypes, nullableRTypes, rightAsBuild, lUsed, rUsed, nil, nil, nil, nil, nil, partitionNumber, logicalop.InnerJoin, 100) } // variable size, inlined for _, rightAsBuild := range rightAsBuildSide { lKeyTypes := []*types.FieldType{intTp, binaryStringTp} rKeyTypes := []*types.FieldType{intTp, binaryStringTp} - testJoinProbe(t, false, []int{1, 17}, []int{1, 17}, lKeyTypes, rKeyTypes, lTypes, rTypes, rightAsBuild, lUsed, rUsed, nil, nil, nil, nil, nil, partitionNumber, plannercore.InnerJoin, 100) - testJoinProbe(t, false, []int{1, 17}, []int{1, 17}, toNullableTypes(lKeyTypes), toNullableTypes(rKeyTypes), nullableLTypes, nullableRTypes, rightAsBuild, lUsed, rUsed, nil, nil, nil, nil, nil, partitionNumber, plannercore.InnerJoin, 100) + testJoinProbe(t, false, []int{1, 17}, []int{1, 17}, lKeyTypes, rKeyTypes, lTypes, rTypes, rightAsBuild, lUsed, rUsed, nil, nil, nil, nil, nil, partitionNumber, logicalop.InnerJoin, 100) + testJoinProbe(t, false, []int{1, 17}, []int{1, 17}, toNullableTypes(lKeyTypes), toNullableTypes(rKeyTypes), nullableLTypes, nullableRTypes, rightAsBuild, lUsed, rUsed, nil, nil, nil, nil, nil, partitionNumber, logicalop.InnerJoin, 100) } // fixed size, not inlined for _, rightAsBuild := range rightAsBuildSide { lKeyTypes := []*types.FieldType{intTp, datetimeTp} rKeyTypes := []*types.FieldType{intTp, datetimeTp} - testJoinProbe(t, false, []int{1, 13}, []int{1, 13}, lKeyTypes, rKeyTypes, lTypes, rTypes, rightAsBuild, lUsed, rUsed, nil, nil, nil, nil, nil, partitionNumber, plannercore.InnerJoin, 100) - testJoinProbe(t, false, []int{1, 13}, []int{1, 13}, toNullableTypes(lKeyTypes), toNullableTypes(rKeyTypes), nullableLTypes, nullableRTypes, rightAsBuild, lUsed, rUsed, nil, nil, nil, nil, nil, partitionNumber, plannercore.InnerJoin, 100) + testJoinProbe(t, false, []int{1, 13}, []int{1, 13}, lKeyTypes, rKeyTypes, lTypes, rTypes, rightAsBuild, lUsed, rUsed, nil, nil, nil, nil, nil, partitionNumber, logicalop.InnerJoin, 100) + testJoinProbe(t, false, []int{1, 13}, []int{1, 13}, toNullableTypes(lKeyTypes), toNullableTypes(rKeyTypes), nullableLTypes, nullableRTypes, rightAsBuild, lUsed, rUsed, nil, nil, nil, nil, nil, partitionNumber, logicalop.InnerJoin, 100) } // variable size, not inlined for _, rightAsBuild := range rightAsBuildSide { lKeyTypes := []*types.FieldType{intTp, decimalTp} rKeyTypes := []*types.FieldType{intTp, decimalTp} - testJoinProbe(t, false, []int{1, 14}, []int{1, 14}, lKeyTypes, rKeyTypes, lTypes, rTypes, rightAsBuild, lUsed, rUsed, nil, nil, nil, nil, nil, partitionNumber, plannercore.InnerJoin, 100) - testJoinProbe(t, false, []int{1, 14}, []int{1, 14}, toNullableTypes(lKeyTypes), toNullableTypes(rKeyTypes), nullableLTypes, nullableRTypes, rightAsBuild, lUsed, rUsed, nil, nil, nil, nil, nil, partitionNumber, plannercore.InnerJoin, 100) + testJoinProbe(t, false, []int{1, 14}, []int{1, 14}, lKeyTypes, rKeyTypes, lTypes, rTypes, rightAsBuild, lUsed, rUsed, nil, nil, nil, nil, nil, partitionNumber, logicalop.InnerJoin, 100) + testJoinProbe(t, false, []int{1, 14}, []int{1, 14}, toNullableTypes(lKeyTypes), toNullableTypes(rKeyTypes), nullableLTypes, nullableRTypes, rightAsBuild, lUsed, rUsed, nil, nil, nil, nil, nil, partitionNumber, logicalop.InnerJoin, 100) } } @@ -581,9 +581,9 @@ func TestInnerJoinProbeOtherCondition(t *testing.T) { partitionNumber := 4 for _, rightAsBuild := range rightAsBuildSide { - testJoinProbe(t, false, []int{0}, []int{0}, []*types.FieldType{intTp}, []*types.FieldType{intTp}, lTypes, rTypes, rightAsBuild, []int{1, 2, 4}, []int{0}, []int{1}, []int{3}, nil, nil, otherCondition, partitionNumber, plannercore.InnerJoin, 200) - testJoinProbe(t, false, []int{0}, []int{0}, []*types.FieldType{intTp}, []*types.FieldType{intTp}, lTypes, rTypes, rightAsBuild, []int{}, []int{}, []int{1}, []int{3}, nil, nil, otherCondition, partitionNumber, plannercore.InnerJoin, 200) - testJoinProbe(t, false, []int{0}, []int{0}, []*types.FieldType{nullableIntTp}, []*types.FieldType{nullableIntTp}, toNullableTypes(lTypes), toNullableTypes(rTypes), rightAsBuild, []int{1, 2, 4}, []int{0}, []int{1}, []int{3}, nil, nil, otherCondition, partitionNumber, plannercore.InnerJoin, 200) + testJoinProbe(t, false, []int{0}, []int{0}, []*types.FieldType{intTp}, []*types.FieldType{intTp}, lTypes, rTypes, rightAsBuild, []int{1, 2, 4}, []int{0}, []int{1}, []int{3}, nil, nil, otherCondition, partitionNumber, logicalop.InnerJoin, 200) + testJoinProbe(t, false, []int{0}, []int{0}, []*types.FieldType{intTp}, []*types.FieldType{intTp}, lTypes, rTypes, rightAsBuild, []int{}, []int{}, []int{1}, []int{3}, nil, nil, otherCondition, partitionNumber, logicalop.InnerJoin, 200) + testJoinProbe(t, false, []int{0}, []int{0}, []*types.FieldType{nullableIntTp}, []*types.FieldType{nullableIntTp}, toNullableTypes(lTypes), toNullableTypes(rTypes), rightAsBuild, []int{1, 2, 4}, []int{0}, []int{1}, []int{3}, nil, nil, otherCondition, partitionNumber, logicalop.InnerJoin, 200) } } @@ -616,9 +616,9 @@ func TestInnerJoinProbeWithSel(t *testing.T) { for _, rightAsBuild := range rightAsBuildSide { for _, oc := range otherConditions { - testJoinProbe(t, true, []int{0}, []int{0}, []*types.FieldType{intTp}, []*types.FieldType{intTp}, lTypes, rTypes, rightAsBuild, []int{1, 2, 4}, []int{0}, []int{1}, []int{3}, nil, nil, oc, partitionNumber, plannercore.InnerJoin, 500) - testJoinProbe(t, true, []int{0}, []int{0}, []*types.FieldType{intTp}, []*types.FieldType{intTp}, lTypes, rTypes, rightAsBuild, []int{}, []int{}, []int{1}, []int{3}, nil, nil, oc, partitionNumber, plannercore.InnerJoin, 500) - testJoinProbe(t, true, []int{0}, []int{0}, []*types.FieldType{nullableIntTp}, []*types.FieldType{nullableIntTp}, toNullableTypes(lTypes), toNullableTypes(rTypes), rightAsBuild, []int{1, 2, 4}, []int{0}, []int{1}, []int{3}, nil, nil, oc, partitionNumber, plannercore.InnerJoin, 500) + testJoinProbe(t, true, []int{0}, []int{0}, []*types.FieldType{intTp}, []*types.FieldType{intTp}, lTypes, rTypes, rightAsBuild, []int{1, 2, 4}, []int{0}, []int{1}, []int{3}, nil, nil, oc, partitionNumber, logicalop.InnerJoin, 500) + testJoinProbe(t, true, []int{0}, []int{0}, []*types.FieldType{intTp}, []*types.FieldType{intTp}, lTypes, rTypes, rightAsBuild, []int{}, []int{}, []int{1}, []int{3}, nil, nil, oc, partitionNumber, logicalop.InnerJoin, 500) + testJoinProbe(t, true, []int{0}, []int{0}, []*types.FieldType{nullableIntTp}, []*types.FieldType{nullableIntTp}, toNullableTypes(lTypes), toNullableTypes(rTypes), rightAsBuild, []int{1, 2, 4}, []int{0}, []int{1}, []int{3}, nil, nil, oc, partitionNumber, logicalop.InnerJoin, 500) } } } diff --git a/pkg/executor/join/joiner.go b/pkg/executor/join/joiner.go index 22a76eea1486f..bae727a840d50 100644 --- a/pkg/executor/join/joiner.go +++ b/pkg/executor/join/joiner.go @@ -16,7 +16,7 @@ package join import ( "github.com/pingcap/tidb/pkg/expression" - plannercore "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" @@ -116,27 +116,27 @@ type Joiner interface { } // JoinerType returns the join type of a Joiner. -func JoinerType(j Joiner) plannercore.JoinType { +func JoinerType(j Joiner) logicalop.JoinType { switch j.(type) { case *semiJoiner: - return plannercore.SemiJoin + return logicalop.SemiJoin case *antiSemiJoiner: - return plannercore.AntiSemiJoin + return logicalop.AntiSemiJoin case *leftOuterSemiJoiner: - return plannercore.LeftOuterSemiJoin + return logicalop.LeftOuterSemiJoin case *antiLeftOuterSemiJoiner: - return plannercore.AntiLeftOuterSemiJoin + return logicalop.AntiLeftOuterSemiJoin case *leftOuterJoiner: - return plannercore.LeftOuterJoin + return logicalop.LeftOuterJoin case *rightOuterJoiner: - return plannercore.RightOuterJoin + return logicalop.RightOuterJoin default: - return plannercore.InnerJoin + return logicalop.InnerJoin } } // NewJoiner create a joiner -func NewJoiner(ctx sessionctx.Context, joinType plannercore.JoinType, +func NewJoiner(ctx sessionctx.Context, joinType logicalop.JoinType, outerIsRight bool, defaultInner []types.Datum, filter []expression.Expression, lhsColTypes, rhsColTypes []*types.FieldType, childrenUsed [][]int, isNA bool) Joiner { base := baseJoiner{ @@ -159,7 +159,7 @@ func NewJoiner(ctx sessionctx.Context, joinType plannercore.JoinType, zap.Ints("lUsed", base.lUsed), zap.Ints("rUsed", base.rUsed), zap.Int("lCount", len(lhsColTypes)), zap.Int("rCount", len(rhsColTypes))) } - if joinType == plannercore.LeftOuterJoin || joinType == plannercore.RightOuterJoin { + if joinType == logicalop.LeftOuterJoin || joinType == logicalop.RightOuterJoin { innerColTypes := lhsColTypes if !outerIsRight { innerColTypes = rhsColTypes @@ -173,34 +173,34 @@ func NewJoiner(ctx sessionctx.Context, joinType plannercore.JoinType, shallowRowType = append(shallowRowType, lhsColTypes...) shallowRowType = append(shallowRowType, rhsColTypes...) switch joinType { - case plannercore.SemiJoin: + case logicalop.SemiJoin: base.shallowRow = chunk.MutRowFromTypes(shallowRowType) return &semiJoiner{base} - case plannercore.AntiSemiJoin: + case logicalop.AntiSemiJoin: base.shallowRow = chunk.MutRowFromTypes(shallowRowType) if isNA { return &nullAwareAntiSemiJoiner{baseJoiner: base} } return &antiSemiJoiner{base} - case plannercore.LeftOuterSemiJoin: + case logicalop.LeftOuterSemiJoin: base.shallowRow = chunk.MutRowFromTypes(shallowRowType) return &leftOuterSemiJoiner{base} - case plannercore.AntiLeftOuterSemiJoin: + case logicalop.AntiLeftOuterSemiJoin: base.shallowRow = chunk.MutRowFromTypes(shallowRowType) if isNA { return &nullAwareAntiLeftOuterSemiJoiner{baseJoiner: base} } return &antiLeftOuterSemiJoiner{base} - case plannercore.LeftOuterJoin, plannercore.RightOuterJoin, plannercore.InnerJoin: + case logicalop.LeftOuterJoin, logicalop.RightOuterJoin, logicalop.InnerJoin: if len(base.conditions) > 0 { base.chk = chunk.NewChunkWithCapacity(shallowRowType, ctx.GetSessionVars().MaxChunkSize) } switch joinType { - case plannercore.LeftOuterJoin: + case logicalop.LeftOuterJoin: return &leftOuterJoiner{base} - case plannercore.RightOuterJoin: + case logicalop.RightOuterJoin: return &rightOuterJoiner{base} - case plannercore.InnerJoin: + case logicalop.InnerJoin: return &innerJoiner{base} } } diff --git a/pkg/executor/join/joiner_test.go b/pkg/executor/join/joiner_test.go index 626b129f22d55..796f1b17ec398 100644 --- a/pkg/executor/join/joiner_test.go +++ b/pkg/executor/join/joiner_test.go @@ -20,7 +20,7 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/parser/mysql" - "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/types" @@ -43,7 +43,7 @@ func defaultCtx() sessionctx.Context { } func TestRequiredRows(t *testing.T) { - joinTypes := []core.JoinType{core.InnerJoin, core.LeftOuterJoin, core.RightOuterJoin} + joinTypes := []logicalop.JoinType{logicalop.InnerJoin, logicalop.LeftOuterJoin, logicalop.RightOuterJoin} lTypes := [][]byte{ {mysql.TypeLong}, {mysql.TypeFloat}, diff --git a/pkg/executor/join/left_outer_join_probe_test.go b/pkg/executor/join/left_outer_join_probe_test.go index ace560d9bcee2..7b3a71393a84b 100644 --- a/pkg/executor/join/left_outer_join_probe_test.go +++ b/pkg/executor/join/left_outer_join_probe_test.go @@ -20,7 +20,7 @@ import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" - plannercore "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" @@ -152,10 +152,10 @@ func TestLeftOuterJoinProbeBasic(t *testing.T) { leftFilter = nil } testJoinProbe(t, false, tc.leftKeyIndex, tc.rightKeyIndex, tc.leftKeyTypes, tc.rightKeyTypes, tc.leftTypes, tc.rightTypes, value, tc.leftUsed, - tc.rightUsed, tc.leftUsedByOtherCondition, tc.rightUsedByOtherCondition, leftFilter, nil, tc.otherCondition, partitionNumber, plannercore.LeftOuterJoin, 200) + tc.rightUsed, tc.leftUsedByOtherCondition, tc.rightUsedByOtherCondition, leftFilter, nil, tc.otherCondition, partitionNumber, logicalop.LeftOuterJoin, 200) testJoinProbe(t, false, tc.leftKeyIndex, tc.rightKeyIndex, toNullableTypes(tc.leftKeyTypes), toNullableTypes(tc.rightKeyTypes), toNullableTypes(tc.leftTypes), toNullableTypes(tc.rightTypes), value, tc.leftUsed, tc.rightUsed, tc.leftUsedByOtherCondition, tc.rightUsedByOtherCondition, - leftFilter, nil, tc.otherCondition, partitionNumber, plannercore.LeftOuterJoin, 200) + leftFilter, nil, tc.otherCondition, partitionNumber, logicalop.LeftOuterJoin, 200) } } } @@ -205,7 +205,7 @@ func TestLeftOuterJoinProbeAllJoinKeys(t *testing.T) { rTypes := lTypes lUsed := []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17} rUsed := lUsed - joinType := plannercore.LeftOuterJoin + joinType := logicalop.LeftOuterJoin partitionNumber := 4 rightAsBuildSide := []bool{true, false} @@ -270,7 +270,7 @@ func TestLeftOuterJoinProbeOtherCondition(t *testing.T) { require.NoError(t, err, "error when create other condition") otherCondition := make(expression.CNFExprs, 0) otherCondition = append(otherCondition, sf) - joinType := plannercore.LeftOuterJoin + joinType := logicalop.LeftOuterJoin simpleFilter := createSimpleFilter(t) hasFilter := []bool{false, true} rightAsBuildSide := []bool{false, true} @@ -311,7 +311,7 @@ func TestLeftOuterJoinProbeWithSel(t *testing.T) { require.NoError(t, err, "error when create other condition") otherCondition := make(expression.CNFExprs, 0) otherCondition = append(otherCondition, sf) - joinType := plannercore.LeftOuterJoin + joinType := logicalop.LeftOuterJoin rightAsBuildSide := []bool{false, true} simpleFilter := createSimpleFilter(t) hasFilter := []bool{false, true} diff --git a/pkg/executor/join/right_outer_join_probe_test.go b/pkg/executor/join/right_outer_join_probe_test.go index b9ad5d4ada1d4..3961b27d6bd7f 100644 --- a/pkg/executor/join/right_outer_join_probe_test.go +++ b/pkg/executor/join/right_outer_join_probe_test.go @@ -20,7 +20,7 @@ import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" - plannercore "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" @@ -152,10 +152,10 @@ func TestRightOuterJoinProbeBasic(t *testing.T) { rightFilter = nil } testJoinProbe(t, false, tc.leftKeyIndex, tc.rightKeyIndex, tc.leftKeyTypes, tc.rightKeyTypes, tc.leftTypes, tc.rightTypes, value, tc.leftUsed, - tc.rightUsed, tc.leftUsedByOtherCondition, tc.rightUsedByOtherCondition, nil, rightFilter, tc.otherCondition, partitionNumber, plannercore.RightOuterJoin, 200) + tc.rightUsed, tc.leftUsedByOtherCondition, tc.rightUsedByOtherCondition, nil, rightFilter, tc.otherCondition, partitionNumber, logicalop.RightOuterJoin, 200) testJoinProbe(t, false, tc.leftKeyIndex, tc.rightKeyIndex, toNullableTypes(tc.leftKeyTypes), toNullableTypes(tc.rightKeyTypes), toNullableTypes(tc.leftTypes), toNullableTypes(tc.rightTypes), value, tc.leftUsed, tc.rightUsed, tc.leftUsedByOtherCondition, tc.rightUsedByOtherCondition, - nil, rightFilter, tc.otherCondition, partitionNumber, plannercore.RightOuterJoin, 200) + nil, rightFilter, tc.otherCondition, partitionNumber, logicalop.RightOuterJoin, 200) } } } @@ -205,7 +205,7 @@ func TestRightOuterJoinProbeAllJoinKeys(t *testing.T) { rTypes := lTypes lUsed := []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17} rUsed := lUsed - joinType := plannercore.RightOuterJoin + joinType := logicalop.RightOuterJoin rightAsBuildSide := []bool{true, false} @@ -269,7 +269,7 @@ func TestRightOuterJoinProbeOtherCondition(t *testing.T) { require.NoError(t, err, "error when create other condition") otherCondition := make(expression.CNFExprs, 0) otherCondition = append(otherCondition, sf) - joinType := plannercore.RightOuterJoin + joinType := logicalop.RightOuterJoin simpleFilter := createSimpleFilter(t) hasFilter := []bool{false, true} rightAsBuildSide := []bool{false, true} @@ -309,7 +309,7 @@ func TestRightOuterJoinProbeWithSel(t *testing.T) { require.NoError(t, err, "error when create other condition") otherCondition := make(expression.CNFExprs, 0) otherCondition = append(otherCondition, sf) - joinType := plannercore.RightOuterJoin + joinType := logicalop.RightOuterJoin rightAsBuildSide := []bool{false, true} simpleFilter := createSimpleFilter(t) hasFilter := []bool{false, true} diff --git a/pkg/executor/pkg_test.go b/pkg/executor/pkg_test.go index 6ae1f0c3f29f5..264bee6fc3acb 100644 --- a/pkg/executor/pkg_test.go +++ b/pkg/executor/pkg_test.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" - plannercore "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" "github.com/pingcap/tidb/pkg/util/mock" @@ -63,7 +63,7 @@ func TestNestedLoopApply(t *testing.T) { outerFilter := expression.NewFunctionInternal(sctx, ast.LT, types.NewFieldType(mysql.TypeTiny), col0, con) innerFilter := outerFilter.Clone() otherFilter := expression.NewFunctionInternal(sctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), col0, col1) - joiner := join.NewJoiner(sctx, plannercore.InnerJoin, false, + joiner := join.NewJoiner(sctx, logicalop.InnerJoin, false, make([]types.Datum, innerExec.Schema().Len()), []expression.Expression{otherFilter}, exec.RetTypes(outerExec), exec.RetTypes(innerExec), nil, false) joinSchema := expression.NewSchema(col0, col1) diff --git a/pkg/expression/constant_propagation.go b/pkg/expression/constant_propagation.go index 41ad3637308ed..2db05ec13ad15 100644 --- a/pkg/expression/constant_propagation.go +++ b/pkg/expression/constant_propagation.go @@ -68,6 +68,29 @@ func (s *basePropConstSolver) tryToUpdateEQList(col *Column, con *Constant) (boo return true, false } +// ValidCompareConstantPredicate checks if the predicate is an expression like [column '>'|'>='|'<'|'<='|'=' constant]. +// return param1: return true, if the predicate is a compare constant predicate. +// return param2: return the column side of predicate. +func ValidCompareConstantPredicate(ctx EvalContext, candidatePredicate Expression) bool { + scalarFunction, ok := candidatePredicate.(*ScalarFunction) + if !ok { + return false + } + if scalarFunction.FuncName.L != ast.GT && scalarFunction.FuncName.L != ast.GE && + scalarFunction.FuncName.L != ast.LT && scalarFunction.FuncName.L != ast.LE && + scalarFunction.FuncName.L != ast.EQ { + return false + } + column, _ := ValidCompareConstantPredicateHelper(ctx, scalarFunction, true) + if column == nil { + column, _ = ValidCompareConstantPredicateHelper(ctx, scalarFunction, false) + } + if column == nil { + return false + } + return true +} + // ValidCompareConstantPredicateHelper checks if the predicate is a compare constant predicate, like "Column xxx Constant" func ValidCompareConstantPredicateHelper(ctx EvalContext, eq *ScalarFunction, colIsLeft bool) (*Column, *Constant) { var col *Column diff --git a/pkg/expression/integration_test/BUILD.bazel b/pkg/expression/integration_test/BUILD.bazel index 8d15c35a41453..ff4331b513ebc 100644 --- a/pkg/expression/integration_test/BUILD.bazel +++ b/pkg/expression/integration_test/BUILD.bazel @@ -21,6 +21,7 @@ go_test( "//pkg/parser/terror", "//pkg/planner/core", "//pkg/planner/core/base", + "//pkg/planner/core/operator/logicalop", "//pkg/session", "//pkg/sessionctx/variable", "//pkg/store/mockstore", diff --git a/pkg/expression/integration_test/integration_test.go b/pkg/expression/integration_test/integration_test.go index 8d289da9ca611..778ad3c2f139d 100644 --- a/pkg/expression/integration_test/integration_test.go +++ b/pkg/expression/integration_test/integration_test.go @@ -42,6 +42,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/terror" plannercore "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/planner/core/base" + "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/session" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/store/mockstore" @@ -435,7 +436,7 @@ func TestFilterExtractFromDNF(t *testing.T) { require.NoError(t, err, "error %v, for resolve name, expr %s", err, tt.exprStr) p, err := plannercore.BuildLogicalPlanForTest(ctx, sctx, stmts[0], ret.InfoSchema) require.NoError(t, err, "error %v, for build plan, expr %s", err, tt.exprStr) - selection := p.(base.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) + selection := p.(base.LogicalPlan).Children()[0].(*logicalop.LogicalSelection) conds := make([]expression.Expression, len(selection.Conditions)) for i, cond := range selection.Conditions { conds[i] = expression.PushDownNot(sctx.GetExprCtx(), cond) diff --git a/pkg/planner/cardinality/BUILD.bazel b/pkg/planner/cardinality/BUILD.bazel index 4d480ba2e5674..51b059bfec680 100644 --- a/pkg/planner/cardinality/BUILD.bazel +++ b/pkg/planner/cardinality/BUILD.bazel @@ -71,6 +71,7 @@ go_test( "//pkg/parser/mysql", "//pkg/planner/core", "//pkg/planner/core/base", + "//pkg/planner/core/operator/logicalop", "//pkg/session", "//pkg/sessionctx", "//pkg/sessionctx/stmtctx", diff --git a/pkg/planner/cardinality/selectivity_test.go b/pkg/planner/cardinality/selectivity_test.go index 591b9da71813d..af92484ba2bd1 100644 --- a/pkg/planner/cardinality/selectivity_test.go +++ b/pkg/planner/cardinality/selectivity_test.go @@ -33,6 +33,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/cardinality" plannercore "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/planner/core/base" + "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/session" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" @@ -104,7 +105,7 @@ func BenchmarkSelectivity(b *testing.B) { b.Run("Selectivity", func(b *testing.B) { b.ResetTimer() for i := 0; i < b.N; i++ { - _, _, err := cardinality.Selectivity(sctx.GetPlanCtx(), &statsTbl.HistColl, p.(base.LogicalPlan).Children()[0].(*plannercore.LogicalSelection).Conditions, nil) + _, _, err := cardinality.Selectivity(sctx.GetPlanCtx(), &statsTbl.HistColl, p.(base.LogicalPlan).Children()[0].(*logicalop.LogicalSelection).Conditions, nil) require.NoError(b, err) } b.ReportAllocs() @@ -450,7 +451,7 @@ func TestSelectivity(t *testing.T) { p, err := plannercore.BuildLogicalPlanForTest(ctx, sctx, stmts[0], ret.InfoSchema) require.NoErrorf(t, err, "for building plan, expr %s", err, tt.exprs) - sel := p.(base.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) + sel := p.(base.LogicalPlan).Children()[0].(*logicalop.LogicalSelection) ds := sel.Children()[0].(*plannercore.DataSource) histColl := statsTbl.GenerateHistCollFromColumnInfo(ds.TableInfo, ds.Schema().Columns) @@ -508,7 +509,7 @@ func TestDNFCondSelectivity(t *testing.T) { p, err := plannercore.BuildLogicalPlanForTest(ctx, sctx, stmts[0], ret.InfoSchema) require.NoErrorf(t, err, "error %v, for building plan, sql %s", err, tt) - sel := p.(base.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) + sel := p.(base.LogicalPlan).Children()[0].(*logicalop.LogicalSelection) ds := sel.Children()[0].(*plannercore.DataSource) histColl := statsTbl.GenerateHistCollFromColumnInfo(ds.TableInfo, ds.Schema().Columns) diff --git a/pkg/planner/cardinality/trace_test.go b/pkg/planner/cardinality/trace_test.go index c44fb88abfe3c..cc194f9cffe56 100644 --- a/pkg/planner/cardinality/trace_test.go +++ b/pkg/planner/cardinality/trace_test.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/cardinality" plannercore "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/planner/core/base" + "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/testdata" @@ -209,7 +210,7 @@ func TestTraceDebugSelectivity(t *testing.T) { p, err := plannercore.BuildLogicalPlanForTest(context.Background(), sctx, stmt, ret.InfoSchema) require.NoError(t, err) - sel := p.(base.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) + sel := p.(base.LogicalPlan).Children()[0].(*logicalop.LogicalSelection) ds := sel.Children()[0].(*plannercore.DataSource) dsSchemaCols = append(dsSchemaCols, ds.Schema().Columns) diff --git a/pkg/planner/cascades/implementation_rules.go b/pkg/planner/cascades/implementation_rules.go index 4abda4a43e856..48ba22d54b355 100644 --- a/pkg/planner/cascades/implementation_rules.go +++ b/pkg/planner/cascades/implementation_rules.go @@ -269,7 +269,7 @@ func (*ImplSelection) Match(_ *memo.GroupExpr, _ *property.PhysicalProperty) (ma // OnImplement implements ImplementationRule OnImplement interface. func (*ImplSelection) OnImplement(expr *memo.GroupExpr, reqProp *property.PhysicalProperty) ([]memo.Implementation, error) { - logicalSel := expr.ExprNode.(*plannercore.LogicalSelection) + logicalSel := expr.ExprNode.(*logicalop.LogicalSelection) physicalSel := plannercore.PhysicalSelection{ Conditions: logicalSel.Conditions, }.Init(logicalSel.SCtx(), expr.Group.Prop.Stats.ScaleByExpectCnt(reqProp.ExpectedCnt), logicalSel.QueryBlockOffset(), reqProp.CloneEssentialFields()) @@ -429,7 +429,7 @@ func (*ImplTopNAsLimit) OnImplement(expr *memo.GroupExpr, _ *property.PhysicalPr } func getImplForHashJoin(expr *memo.GroupExpr, prop *property.PhysicalProperty, innerIdx int, useOuterToBuild bool) memo.Implementation { - join := expr.ExprNode.(*plannercore.LogicalJoin) + join := expr.ExprNode.(*logicalop.LogicalJoin) chReqProps := make([]*property.PhysicalProperty, 2) chReqProps[0] = &property.PhysicalProperty{ExpectedCnt: math.MaxFloat64} chReqProps[1] = &property.PhysicalProperty{ExpectedCnt: math.MaxFloat64} @@ -449,8 +449,8 @@ type ImplHashJoinBuildLeft struct { // Match implements ImplementationRule Match interface. func (*ImplHashJoinBuildLeft) Match(expr *memo.GroupExpr, prop *property.PhysicalProperty) (matched bool) { - switch expr.ExprNode.(*plannercore.LogicalJoin).JoinType { - case plannercore.InnerJoin, plannercore.LeftOuterJoin, plannercore.RightOuterJoin: + switch expr.ExprNode.(*logicalop.LogicalJoin).JoinType { + case logicalop.InnerJoin, logicalop.LeftOuterJoin, logicalop.RightOuterJoin: return prop.IsSortItemEmpty() default: return false @@ -459,13 +459,13 @@ func (*ImplHashJoinBuildLeft) Match(expr *memo.GroupExpr, prop *property.Physica // OnImplement implements ImplementationRule OnImplement interface. func (*ImplHashJoinBuildLeft) OnImplement(expr *memo.GroupExpr, reqProp *property.PhysicalProperty) ([]memo.Implementation, error) { - join := expr.ExprNode.(*plannercore.LogicalJoin) + join := expr.ExprNode.(*logicalop.LogicalJoin) switch join.JoinType { - case plannercore.InnerJoin: + case logicalop.InnerJoin: return []memo.Implementation{getImplForHashJoin(expr, reqProp, 0, false)}, nil - case plannercore.LeftOuterJoin: + case logicalop.LeftOuterJoin: return []memo.Implementation{getImplForHashJoin(expr, reqProp, 1, true)}, nil - case plannercore.RightOuterJoin: + case logicalop.RightOuterJoin: return []memo.Implementation{getImplForHashJoin(expr, reqProp, 0, false)}, nil default: return nil, nil @@ -483,16 +483,16 @@ func (*ImplHashJoinBuildRight) Match(_ *memo.GroupExpr, prop *property.PhysicalP // OnImplement implements ImplementationRule OnImplement interface. func (*ImplHashJoinBuildRight) OnImplement(expr *memo.GroupExpr, reqProp *property.PhysicalProperty) ([]memo.Implementation, error) { - join := expr.ExprNode.(*plannercore.LogicalJoin) + join := expr.ExprNode.(*logicalop.LogicalJoin) switch join.JoinType { - case plannercore.SemiJoin, plannercore.AntiSemiJoin, - plannercore.LeftOuterSemiJoin, plannercore.AntiLeftOuterSemiJoin: + case logicalop.SemiJoin, logicalop.AntiSemiJoin, + logicalop.LeftOuterSemiJoin, logicalop.AntiLeftOuterSemiJoin: return []memo.Implementation{getImplForHashJoin(expr, reqProp, 1, false)}, nil - case plannercore.InnerJoin: + case logicalop.InnerJoin: return []memo.Implementation{getImplForHashJoin(expr, reqProp, 1, false)}, nil - case plannercore.LeftOuterJoin: + case logicalop.LeftOuterJoin: return []memo.Implementation{getImplForHashJoin(expr, reqProp, 1, false)}, nil - case plannercore.RightOuterJoin: + case logicalop.RightOuterJoin: return []memo.Implementation{getImplForHashJoin(expr, reqProp, 0, true)}, nil } return nil, nil @@ -509,7 +509,7 @@ func (*ImplMergeJoin) Match(_ *memo.GroupExpr, _ *property.PhysicalProperty) (ma // OnImplement implements ImplementationRule OnImplement interface. func (*ImplMergeJoin) OnImplement(expr *memo.GroupExpr, reqProp *property.PhysicalProperty) ([]memo.Implementation, error) { - join := expr.ExprNode.(*plannercore.LogicalJoin) + join := expr.ExprNode.(*logicalop.LogicalJoin) physicalMergeJoins := plannercore.GetMergeJoin(join, reqProp, expr.Schema(), expr.Group.Prop.Stats, expr.Children[0].Prop.Stats, expr.Children[1].Prop.Stats) mergeJoinImpls := make([]memo.Implementation, 0, len(physicalMergeJoins)) for _, physicalPlan := range physicalMergeJoins { diff --git a/pkg/planner/cascades/transformation_rules.go b/pkg/planner/cascades/transformation_rules.go index 911c8f2dade4d..8072771a1ad86 100644 --- a/pkg/planner/cascades/transformation_rules.go +++ b/pkg/planner/cascades/transformation_rules.go @@ -197,7 +197,7 @@ func NewRulePushSelDownTableScan() Transformation { // Filters of the old `sel` operator are removed if they are used to calculate // the key ranges of the `ts` operator. func (*PushSelDownTableScan) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - sel := old.GetExpr().ExprNode.(*plannercore.LogicalSelection) + sel := old.GetExpr().ExprNode.(*logicalop.LogicalSelection) ts := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalTableScan) if ts.HandleCols == nil { return nil, false, false, nil @@ -219,7 +219,7 @@ func (*PushSelDownTableScan) OnTransform(old *memo.ExprIter) (newExprs []*memo.G } schema := old.GetExpr().Group.Prop.Schema tblScanGroup := memo.NewGroupWithSchema(tblScanExpr, schema) - newSel := plannercore.LogicalSelection{Conditions: remained}.Init(sel.SCtx(), sel.QueryBlockOffset()) + newSel := logicalop.LogicalSelection{Conditions: remained}.Init(sel.SCtx(), sel.QueryBlockOffset()) selExpr := memo.NewGroupExpr(newSel) selExpr.Children = append(selExpr.Children, tblScanGroup) // `sel -> ts` is transformed to `newSel ->newTS`. @@ -250,7 +250,7 @@ func NewRulePushSelDownIndexScan() Transformation { // `Selection -> IndexScan(with a new access range)` // or just keep the two GroupExprs unchanged. func (*PushSelDownIndexScan) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - sel := old.GetExpr().ExprNode.(*plannercore.LogicalSelection) + sel := old.GetExpr().ExprNode.(*logicalop.LogicalSelection) is := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalIndexScan) if len(is.IdxCols) == 0 { return nil, false, false, nil @@ -302,7 +302,7 @@ func (*PushSelDownIndexScan) OnTransform(old *memo.ExprIter) (newExprs []*memo.G return []*memo.GroupExpr{isExpr}, true, false, nil } isGroup := memo.NewGroupWithSchema(isExpr, old.Children[0].GetExpr().Group.Prop.Schema) - newSel := plannercore.LogicalSelection{Conditions: res.RemainedConds}.Init(sel.SCtx(), sel.QueryBlockOffset()) + newSel := logicalop.LogicalSelection{Conditions: res.RemainedConds}.Init(sel.SCtx(), sel.QueryBlockOffset()) selExpr := memo.NewGroupExpr(newSel) selExpr.SetChildren(isGroup) return []*memo.GroupExpr{selExpr}, true, false, nil @@ -331,16 +331,16 @@ func NewRulePushSelDownTiKVSingleGather() Transformation { // 1. `newTg -> pushedSel -> any` // 2. `remainedSel -> newTg -> pushedSel -> any` func (*PushSelDownTiKVSingleGather) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - sel := old.GetExpr().ExprNode.(*plannercore.LogicalSelection) + sel := old.GetExpr().ExprNode.(*logicalop.LogicalSelection) sg := old.Children[0].GetExpr().ExprNode.(*plannercore.TiKVSingleGather) childGroup := old.Children[0].Children[0].Group var pushed, remained []expression.Expression sctx := sg.SCtx() - pushed, remained = expression.PushDownExprs(plannercore.GetPushDownCtx(sctx), sel.Conditions, kv.TiKV) + pushed, remained = expression.PushDownExprs(util.GetPushDownCtx(sctx), sel.Conditions, kv.TiKV) if len(pushed) == 0 { return nil, false, false, nil } - pushedSel := plannercore.LogicalSelection{Conditions: pushed}.Init(sctx, sel.QueryBlockOffset()) + pushedSel := logicalop.LogicalSelection{Conditions: pushed}.Init(sctx, sel.QueryBlockOffset()) pushedSelExpr := memo.NewGroupExpr(pushedSel) pushedSelExpr.Children = append(pushedSelExpr.Children, childGroup) pushedSelGroup := memo.NewGroupWithSchema(pushedSelExpr, childGroup.Prop.Schema).SetEngineType(childGroup.EngineType) @@ -356,7 +356,7 @@ func (*PushSelDownTiKVSingleGather) OnTransform(old *memo.ExprIter) (newExprs [] return []*memo.GroupExpr{tblGatherExpr}, true, false, nil } tblGatherGroup := memo.NewGroupWithSchema(tblGatherExpr, pushedSelGroup.Prop.Schema) - remainedSel := plannercore.LogicalSelection{Conditions: remained}.Init(sel.SCtx(), sel.QueryBlockOffset()) + remainedSel := logicalop.LogicalSelection{Conditions: remained}.Init(sel.SCtx(), sel.QueryBlockOffset()) remainedSelExpr := memo.NewGroupExpr(remainedSel) remainedSelExpr.Children = append(remainedSelExpr.Children, tblGatherGroup) // `oldSel -> oldTg -> any` is transformed to `remainedSel -> newTg -> pushedSel -> any`. @@ -507,7 +507,7 @@ func NewRulePushSelDownSort() Transformation { // OnTransform implements Transformation interface. // It will transform `sel->sort->x` to `sort->sel->x`. func (*PushSelDownSort) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - sel := old.GetExpr().ExprNode.(*plannercore.LogicalSelection) + sel := old.GetExpr().ExprNode.(*logicalop.LogicalSelection) sort := old.Children[0].GetExpr().ExprNode.(*logicalop.LogicalSort) childGroup := old.Children[0].GetExpr().Children[0] @@ -543,7 +543,7 @@ func NewRulePushSelDownProjection() Transformation { // 2. `selection -> projection -> selection -> x` or // 3. just keep unchanged. func (*PushSelDownProjection) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - sel := old.GetExpr().ExprNode.(*plannercore.LogicalSelection) + sel := old.GetExpr().ExprNode.(*logicalop.LogicalSelection) proj := old.Children[0].GetExpr().ExprNode.(*logicalop.LogicalProjection) projSchema := old.Children[0].Prop.Schema childGroup := old.Children[0].GetExpr().Children[0] @@ -566,7 +566,7 @@ func (*PushSelDownProjection) OnTransform(old *memo.ExprIter) (newExprs []*memo. if len(canBePushed) == 0 { return nil, false, false, nil } - newBottomSel := plannercore.LogicalSelection{Conditions: canBePushed}.Init(sel.SCtx(), sel.QueryBlockOffset()) + newBottomSel := logicalop.LogicalSelection{Conditions: canBePushed}.Init(sel.SCtx(), sel.QueryBlockOffset()) newBottomSelExpr := memo.NewGroupExpr(newBottomSel) newBottomSelExpr.SetChildren(childGroup) newBottomSelGroup := memo.NewGroupWithSchema(newBottomSelExpr, childGroup.Prop.Schema) @@ -576,7 +576,7 @@ func (*PushSelDownProjection) OnTransform(old *memo.ExprIter) (newExprs []*memo. return []*memo.GroupExpr{newProjExpr}, true, false, nil } newProjGroup := memo.NewGroupWithSchema(newProjExpr, projSchema) - newTopSel := plannercore.LogicalSelection{Conditions: canNotBePushed}.Init(sel.SCtx(), sel.QueryBlockOffset()) + newTopSel := logicalop.LogicalSelection{Conditions: canNotBePushed}.Init(sel.SCtx(), sel.QueryBlockOffset()) newTopSelExpr := memo.NewGroupExpr(newTopSel) newTopSelExpr.SetChildren(newProjGroup) return []*memo.GroupExpr{newTopSelExpr}, true, false, nil @@ -603,7 +603,7 @@ func NewRulePushSelDownAggregation() Transformation { // It will transform `sel->agg->x` to `agg->sel->x` or `sel->agg->sel->x` // or just keep the selection unchanged. func (*PushSelDownAggregation) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - sel := old.GetExpr().ExprNode.(*plannercore.LogicalSelection) + sel := old.GetExpr().ExprNode.(*logicalop.LogicalSelection) agg := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalAggregation) aggSchema := old.Children[0].Prop.Schema var pushedExprs []expression.Expression @@ -641,7 +641,7 @@ func (*PushSelDownAggregation) OnTransform(old *memo.ExprIter) (newExprs []*memo } sctx := sel.SCtx() childGroup := old.Children[0].GetExpr().Children[0] - pushedSel := plannercore.LogicalSelection{Conditions: pushedExprs}.Init(sctx, sel.QueryBlockOffset()) + pushedSel := logicalop.LogicalSelection{Conditions: pushedExprs}.Init(sctx, sel.QueryBlockOffset()) pushedGroupExpr := memo.NewGroupExpr(pushedSel) pushedGroupExpr.SetChildren(childGroup) pushedGroup := memo.NewGroupWithSchema(pushedGroupExpr, childGroup.Prop.Schema) @@ -654,7 +654,7 @@ func (*PushSelDownAggregation) OnTransform(old *memo.ExprIter) (newExprs []*memo } aggGroup := memo.NewGroupWithSchema(aggGroupExpr, aggSchema) - remainedSel := plannercore.LogicalSelection{Conditions: remainedExprs}.Init(sctx, sel.QueryBlockOffset()) + remainedSel := logicalop.LogicalSelection{Conditions: remainedExprs}.Init(sctx, sel.QueryBlockOffset()) remainedGroupExpr := memo.NewGroupExpr(remainedSel) remainedGroupExpr.SetChildren(aggGroup) return []*memo.GroupExpr{remainedGroupExpr}, true, false, nil @@ -683,7 +683,7 @@ func NewRulePushSelDownWindow() Transformation { // 2. `sel -> window -> sel -> x` or // 3. just keep unchanged. func (*PushSelDownWindow) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - sel := old.GetExpr().ExprNode.(*plannercore.LogicalSelection) + sel := old.GetExpr().ExprNode.(*logicalop.LogicalSelection) window := old.Children[0].GetExpr().ExprNode.(*logicalop.LogicalWindow) windowSchema := old.Children[0].Prop.Schema childGroup := old.Children[0].GetExpr().Children[0] @@ -706,7 +706,7 @@ func (*PushSelDownWindow) OnTransform(old *memo.ExprIter) (newExprs []*memo.Grou } // construct return GroupExpr - newBottomSel := plannercore.LogicalSelection{Conditions: canBePushed}.Init(sel.SCtx(), sel.QueryBlockOffset()) + newBottomSel := logicalop.LogicalSelection{Conditions: canBePushed}.Init(sel.SCtx(), sel.QueryBlockOffset()) newBottomSelExpr := memo.NewGroupExpr(newBottomSel) newBottomSelExpr.SetChildren(childGroup) newBottomSelGroup := memo.NewGroupWithSchema(newBottomSelExpr, childGroup.Prop.Schema) @@ -717,7 +717,7 @@ func (*PushSelDownWindow) OnTransform(old *memo.ExprIter) (newExprs []*memo.Grou } newWindowGroup := memo.NewGroupWithSchema(newWindowExpr, windowSchema) - newTopSel := plannercore.LogicalSelection{Conditions: canNotBePushed}.Init(sel.SCtx(), sel.QueryBlockOffset()) + newTopSel := logicalop.LogicalSelection{Conditions: canNotBePushed}.Init(sel.SCtx(), sel.QueryBlockOffset()) newTopSelExpr := memo.NewGroupExpr(newTopSel) newTopSelExpr.SetChildren(newWindowGroup) return []*memo.GroupExpr{newTopSelExpr}, true, false, nil @@ -855,7 +855,7 @@ type pushDownJoin struct { func (*pushDownJoin) predicatePushDown( sctx context.PlanContext, predicates []expression.Expression, - join *plannercore.LogicalJoin, + join *logicalop.LogicalJoin, leftSchema *expression.Schema, rightSchema *expression.Schema, ) ( @@ -867,7 +867,7 @@ func (*pushDownJoin) predicatePushDown( var equalCond []*expression.ScalarFunction var leftPushCond, rightPushCond, otherCond []expression.Expression switch join.JoinType { - case plannercore.SemiJoin, plannercore.InnerJoin: + case logicalop.SemiJoin, logicalop.InnerJoin: tempCond := make([]expression.Expression, 0, len(join.LeftConditions)+len(join.RightConditions)+len(join.EqualConditions)+len(join.OtherConditions)+len(predicates)) tempCond = append(tempCond, join.LeftConditions...) @@ -878,7 +878,7 @@ func (*pushDownJoin) predicatePushDown( tempCond = expression.ExtractFiltersFromDNFs(sctx.GetExprCtx(), tempCond) tempCond = expression.PropagateConstant(sctx.GetExprCtx(), tempCond) // Return table dual when filter is constant false or null. - dual := plannercore.Conds2TableDual(join, tempCond) + dual := logicalop.Conds2TableDual(join, tempCond) if dual != nil { return leftCond, rightCond, remainCond, dual } @@ -889,8 +889,8 @@ func (*pushDownJoin) predicatePushDown( join.OtherConditions = otherCond leftCond = leftPushCond rightCond = rightPushCond - case plannercore.LeftOuterJoin, plannercore.LeftOuterSemiJoin, plannercore.AntiLeftOuterSemiJoin, - plannercore.RightOuterJoin: + case logicalop.LeftOuterJoin, logicalop.LeftOuterSemiJoin, logicalop.AntiLeftOuterSemiJoin, + logicalop.RightOuterJoin: lenJoinConds := len(join.EqualConditions) + len(join.LeftConditions) + len(join.RightConditions) + len(join.OtherConditions) joinConds := make([]expression.Expression, 0, lenJoinConds) for _, equalCond := range join.EqualConditions { @@ -905,8 +905,8 @@ func (*pushDownJoin) predicatePushDown( join.OtherConditions = nil remainCond = make([]expression.Expression, len(predicates)) copy(remainCond, predicates) - nullSensitive := join.JoinType == plannercore.AntiLeftOuterSemiJoin || join.JoinType == plannercore.LeftOuterSemiJoin - if join.JoinType == plannercore.RightOuterJoin { + nullSensitive := join.JoinType == logicalop.AntiLeftOuterSemiJoin || join.JoinType == logicalop.LeftOuterSemiJoin + if join.JoinType == logicalop.RightOuterJoin { joinConds, remainCond = expression.PropConstOverOuterJoin(join.SCtx().GetExprCtx(), joinConds, remainCond, rightSchema, leftSchema, nullSensitive) } else { joinConds, remainCond = expression.PropConstOverOuterJoin(join.SCtx().GetExprCtx(), joinConds, remainCond, leftSchema, rightSchema, nullSensitive) @@ -914,17 +914,17 @@ func (*pushDownJoin) predicatePushDown( eq, left, right, other := join.ExtractOnCondition(joinConds, leftSchema, rightSchema, false, false) join.AppendJoinConds(eq, left, right, other) // Return table dual when filter is constant false or null. - dual := plannercore.Conds2TableDual(join, remainCond) + dual := logicalop.Conds2TableDual(join, remainCond) if dual != nil { return leftCond, rightCond, remainCond, dual } - if join.JoinType == plannercore.RightOuterJoin { + if join.JoinType == logicalop.RightOuterJoin { remainCond = expression.ExtractFiltersFromDNFs(join.SCtx().GetExprCtx(), remainCond) // Only derive right where condition, because left where condition cannot be pushed down equalCond, leftPushCond, rightPushCond, otherCond = join.ExtractOnCondition(remainCond, leftSchema, rightSchema, false, true) rightCond = rightPushCond // Handle join conditions, only derive left join condition, because right join condition cannot be pushed down - derivedLeftJoinCond, _ := plannercore.DeriveOtherConditions(join, leftSchema, rightSchema, true, false) + derivedLeftJoinCond, _ := logicalop.DeriveOtherConditions(join, leftSchema, rightSchema, true, false) leftCond = append(join.LeftConditions, derivedLeftJoinCond...) join.LeftConditions = nil remainCond = append(expression.ScalarFuncs2Exprs(equalCond), otherCond...) @@ -935,7 +935,7 @@ func (*pushDownJoin) predicatePushDown( equalCond, leftPushCond, rightPushCond, otherCond = join.ExtractOnCondition(remainCond, leftSchema, rightSchema, true, false) leftCond = leftPushCond // Handle join conditions, only derive right join condition, because left join condition cannot be pushed down - _, derivedRightJoinCond := plannercore.DeriveOtherConditions(join, leftSchema, rightSchema, false, true) + _, derivedRightJoinCond := logicalop.DeriveOtherConditions(join, leftSchema, rightSchema, false, true) rightCond = append(join.RightConditions, derivedRightJoinCond...) join.RightConditions = nil remainCond = append(expression.ScalarFuncs2Exprs(equalCond), otherCond...) @@ -982,7 +982,7 @@ func buildChildSelectionGroup( if len(conditions) == 0 { return childGroup } - newSel := plannercore.LogicalSelection{Conditions: conditions}.Init(sctx, qbOffset) + newSel := logicalop.LogicalSelection{Conditions: conditions}.Init(sctx, qbOffset) groupExpr := memo.NewGroupExpr(newSel) groupExpr.SetChildren(childGroup) newChild := memo.NewGroupWithSchema(groupExpr, childGroup.Prop.Schema) @@ -992,9 +992,9 @@ func buildChildSelectionGroup( // OnTransform implements Transformation interface. // This rule tries to pushes the Selection through Join. Besides, this rule fulfills the `XXXConditions` field of Join. func (r *PushSelDownJoin) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - sel := old.GetExpr().ExprNode.(*plannercore.LogicalSelection) + sel := old.GetExpr().ExprNode.(*logicalop.LogicalSelection) joinExpr := old.Children[0].GetExpr() - join := joinExpr.ExprNode.(*plannercore.LogicalJoin) + join := joinExpr.ExprNode.(*logicalop.LogicalJoin) newJoin := join.Shallow() sctx := sel.SCtx() leftGroup := old.Children[0].GetExpr().Children[0] @@ -1010,7 +1010,7 @@ func (r *PushSelDownJoin) OnTransform(old *memo.ExprIter) (newExprs []*memo.Grou newJoinExpr := memo.NewGroupExpr(newJoin) newJoinExpr.SetChildren(leftGroup, rightGroup) if len(remainCond) > 0 { - newSel := plannercore.LogicalSelection{Conditions: remainCond}.Init(sctx, sel.QueryBlockOffset()) + newSel := logicalop.LogicalSelection{Conditions: remainCond}.Init(sctx, sel.QueryBlockOffset()) newSel.Conditions = remainCond newSelExpr := memo.NewGroupExpr(newSel) newSelExpr.SetChildren(memo.NewGroupWithSchema(newJoinExpr, old.Children[0].Prop.Schema)) @@ -1039,7 +1039,7 @@ func (r *TransformJoinCondToSel) Match(expr *memo.ExprIter) bool { if expr.GetExpr().HasAppliedRule(r) { return false } - join := expr.GetExpr().ExprNode.(*plannercore.LogicalJoin) + join := expr.GetExpr().ExprNode.(*logicalop.LogicalJoin) return len(join.EqualConditions) > 0 || len(join.LeftConditions) > 0 || len(join.RightConditions) > 0 || len(join.OtherConditions) > 0 } @@ -1047,7 +1047,7 @@ func (r *TransformJoinCondToSel) Match(expr *memo.ExprIter) bool { // OnTransform implements Transformation interface. // This rule tries to transform Join conditions to Selection. Besides, this rule fulfills the `XXXConditions` field of Join. func (r *TransformJoinCondToSel) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - join := old.GetExpr().ExprNode.(*plannercore.LogicalJoin) + join := old.GetExpr().ExprNode.(*logicalop.LogicalJoin) newJoin := join.Shallow() sctx := join.SCtx() leftGroup := old.GetExpr().Children[0] @@ -1085,7 +1085,7 @@ func NewRulePushSelDownUnionAll() Transformation { // OnTransform implements Transformation interface. // It will transform `Selection->UnionAll->x` to `UnionAll->Selection->x`. func (*PushSelDownUnionAll) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - sel := old.GetExpr().ExprNode.(*plannercore.LogicalSelection) + sel := old.GetExpr().ExprNode.(*logicalop.LogicalSelection) unionAll := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalUnionAll) childGroups := old.Children[0].GetExpr().Children @@ -1214,9 +1214,9 @@ func (r *PushTopNDownOuterJoin) Match(expr *memo.ExprIter) bool { if expr.GetExpr().HasAppliedRule(r) { return false } - join := expr.Children[0].GetExpr().ExprNode.(*plannercore.LogicalJoin) + join := expr.Children[0].GetExpr().ExprNode.(*logicalop.LogicalJoin) switch join.JoinType { - case plannercore.LeftOuterJoin, plannercore.LeftOuterSemiJoin, plannercore.AntiLeftOuterSemiJoin, plannercore.RightOuterJoin: + case logicalop.LeftOuterJoin, logicalop.LeftOuterSemiJoin, logicalop.AntiLeftOuterSemiJoin, logicalop.RightOuterJoin: return true default: return false @@ -1252,15 +1252,15 @@ func pushTopNDownOuterJoinToChild(topN *logicalop.LogicalTopN, outerGroup *memo. func (r *PushTopNDownOuterJoin) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { topN := old.GetExpr().ExprNode.(*logicalop.LogicalTopN) joinExpr := old.Children[0].GetExpr() - join := joinExpr.ExprNode.(*plannercore.LogicalJoin) + join := joinExpr.ExprNode.(*logicalop.LogicalJoin) joinSchema := old.Children[0].Group.Prop.Schema leftGroup := joinExpr.Children[0] rightGroup := joinExpr.Children[1] switch join.JoinType { - case plannercore.LeftOuterJoin, plannercore.LeftOuterSemiJoin, plannercore.AntiLeftOuterSemiJoin: + case logicalop.LeftOuterJoin, logicalop.LeftOuterSemiJoin, logicalop.AntiLeftOuterSemiJoin: leftGroup = pushTopNDownOuterJoinToChild(topN, leftGroup) - case plannercore.RightOuterJoin: + case logicalop.RightOuterJoin: rightGroup = pushTopNDownOuterJoinToChild(topN, rightGroup) default: return nil, false, false, nil @@ -1614,7 +1614,7 @@ func (r *EliminateSingleMaxMin) OnTransform(old *memo.ExprIter) (newExprs []*mem if len(expression.ExtractColumns(f.Args[0])) > 0 { // If it can be NULL, we need to filter NULL out first. if !mysql.HasNotNullFlag(f.Args[0].GetType(ectx).GetFlag()) { - sel := plannercore.LogicalSelection{}.Init(ctx, agg.QueryBlockOffset()) + sel := logicalop.LogicalSelection{}.Init(ctx, agg.QueryBlockOffset()) isNullFunc := expression.NewFunctionInternal(ctx.GetExprCtx(), ast.IsNull, types.NewFieldType(mysql.TypeTiny), f.Args[0]) notNullFunc := expression.NewFunctionInternal(ctx.GetExprCtx(), ast.UnaryNot, types.NewFieldType(mysql.TypeTiny), isNullFunc) sel.Conditions = []expression.Expression{notNullFunc} @@ -1677,14 +1677,14 @@ func NewRuleMergeAdjacentSelection() Transformation { // OnTransform implements Transformation interface. // This rule tries to merge adjacent selection, with no simplification. func (*MergeAdjacentSelection) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - sel := old.GetExpr().ExprNode.(*plannercore.LogicalSelection) - child := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalSelection) + sel := old.GetExpr().ExprNode.(*logicalop.LogicalSelection) + child := old.Children[0].GetExpr().ExprNode.(*logicalop.LogicalSelection) childGroups := old.Children[0].GetExpr().Children conditions := make([]expression.Expression, 0, len(sel.Conditions)+len(child.Conditions)) conditions = append(conditions, sel.Conditions...) conditions = append(conditions, child.Conditions...) - newSel := plannercore.LogicalSelection{Conditions: conditions}.Init(sel.SCtx(), sel.QueryBlockOffset()) + newSel := logicalop.LogicalSelection{Conditions: conditions}.Init(sel.SCtx(), sel.QueryBlockOffset()) newSelExpr := memo.NewGroupExpr(newSel) newSelExpr.SetChildren(childGroups...) return []*memo.GroupExpr{newSelExpr}, true, false, nil @@ -1786,7 +1786,7 @@ func (r *PushLimitDownOuterJoin) Match(expr *memo.ExprIter) bool { if expr.GetExpr().HasAppliedRule(r) { return false } - join := expr.Children[0].GetExpr().ExprNode.(*plannercore.LogicalJoin) + join := expr.Children[0].GetExpr().ExprNode.(*logicalop.LogicalJoin) return join.JoinType.IsOuterJoin() } @@ -1794,15 +1794,15 @@ func (r *PushLimitDownOuterJoin) Match(expr *memo.ExprIter) bool { // This rule tries to pushes the Limit through outer Join. func (r *PushLimitDownOuterJoin) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { limit := old.GetExpr().ExprNode.(*logicalop.LogicalLimit) - join := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalJoin) + join := old.Children[0].GetExpr().ExprNode.(*logicalop.LogicalJoin) joinSchema := old.Children[0].Group.Prop.Schema leftGroup := old.Children[0].GetExpr().Children[0] rightGroup := old.Children[0].GetExpr().Children[1] switch join.JoinType { - case plannercore.LeftOuterJoin, plannercore.LeftOuterSemiJoin, plannercore.AntiLeftOuterSemiJoin: + case logicalop.LeftOuterJoin, logicalop.LeftOuterSemiJoin, logicalop.AntiLeftOuterSemiJoin: leftGroup = r.pushLimitDownOuterJoinToChild(limit, leftGroup) - case plannercore.RightOuterJoin: + case logicalop.RightOuterJoin: rightGroup = r.pushLimitDownOuterJoinToChild(limit, rightGroup) default: return nil, false, false, nil @@ -1877,12 +1877,12 @@ type outerJoinEliminator struct { } func (*outerJoinEliminator) prepareForEliminateOuterJoin(joinExpr *memo.GroupExpr) (ok bool, innerChildIdx int, outerGroup *memo.Group, innerGroup *memo.Group, outerUniqueIDs set.Int64Set) { - join := joinExpr.ExprNode.(*plannercore.LogicalJoin) + join := joinExpr.ExprNode.(*logicalop.LogicalJoin) switch join.JoinType { - case plannercore.LeftOuterJoin: + case logicalop.LeftOuterJoin: innerChildIdx = 1 - case plannercore.RightOuterJoin: + case logicalop.RightOuterJoin: innerChildIdx = 0 default: ok = false @@ -1939,8 +1939,8 @@ func NewRuleEliminateOuterJoinBelowAggregation() Transformation { // Match implements Transformation interface. func (*EliminateOuterJoinBelowAggregation) Match(expr *memo.ExprIter) bool { - joinType := expr.Children[0].GetExpr().ExprNode.(*plannercore.LogicalJoin).JoinType - return joinType == plannercore.LeftOuterJoin || joinType == plannercore.RightOuterJoin + joinType := expr.Children[0].GetExpr().ExprNode.(*logicalop.LogicalJoin).JoinType + return joinType == logicalop.LeftOuterJoin || joinType == logicalop.RightOuterJoin } // OnTransform implements Transformation interface. @@ -1948,7 +1948,7 @@ func (*EliminateOuterJoinBelowAggregation) Match(expr *memo.ExprIter) bool { func (r *EliminateOuterJoinBelowAggregation) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { agg := old.GetExpr().ExprNode.(*plannercore.LogicalAggregation) joinExpr := old.Children[0].GetExpr() - join := joinExpr.ExprNode.(*plannercore.LogicalJoin) + join := joinExpr.ExprNode.(*logicalop.LogicalJoin) ok, innerChildIdx, outerGroup, innerGroup, outerUniqueIDs := r.prepareForEliminateOuterJoin(joinExpr) if !ok { @@ -2001,8 +2001,8 @@ func NewRuleEliminateOuterJoinBelowProjection() Transformation { // Match implements Transformation interface. func (*EliminateOuterJoinBelowProjection) Match(expr *memo.ExprIter) bool { - joinType := expr.Children[0].GetExpr().ExprNode.(*plannercore.LogicalJoin).JoinType - return joinType == plannercore.LeftOuterJoin || joinType == plannercore.RightOuterJoin + joinType := expr.Children[0].GetExpr().ExprNode.(*logicalop.LogicalJoin).JoinType + return joinType == logicalop.LeftOuterJoin || joinType == logicalop.RightOuterJoin } // OnTransform implements Transformation interface. @@ -2010,7 +2010,7 @@ func (*EliminateOuterJoinBelowProjection) Match(expr *memo.ExprIter) bool { func (r *EliminateOuterJoinBelowProjection) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { proj := old.GetExpr().ExprNode.(*logicalop.LogicalProjection) joinExpr := old.Children[0].GetExpr() - join := joinExpr.ExprNode.(*plannercore.LogicalJoin) + join := joinExpr.ExprNode.(*logicalop.LogicalJoin) ok, innerChildIdx, outerGroup, innerGroup, outerUniqueIDs := r.prepareForEliminateOuterJoin(joinExpr) if !ok { @@ -2068,7 +2068,7 @@ func (r *TransformAggregateCaseToSelection) OnTransform(old *memo.ExprIter) (new return nil, false, false, nil } - newSel := plannercore.LogicalSelection{Conditions: newConditions}.Init(agg.SCtx(), agg.QueryBlockOffset()) + newSel := logicalop.LogicalSelection{Conditions: newConditions}.Init(agg.SCtx(), agg.QueryBlockOffset()) newSelExpr := memo.NewGroupExpr(newSel) newSelExpr.SetChildren(old.GetExpr().Children...) newSelGroup := memo.NewGroupWithSchema(newSelExpr, old.GetExpr().Children[0].Prop.Schema) @@ -2505,7 +2505,7 @@ func (*PullSelectionUpApply) OnTransform(old *memo.ExprIter) (newExprs []*memo.G apply := old.GetExpr().ExprNode.(*plannercore.LogicalApply) outerChildGroup := old.Children[0].Group innerChildGroup := old.Children[1].Group - sel := old.Children[1].GetExpr().ExprNode.(*plannercore.LogicalSelection) + sel := old.Children[1].GetExpr().ExprNode.(*logicalop.LogicalSelection) newConds := make([]expression.Expression, 0, len(sel.Conditions)) for _, cond := range sel.Conditions { newConds = append(newConds, cond.Clone().Decorrelate(outerChildGroup.Prop.Schema)) diff --git a/pkg/planner/core/BUILD.bazel b/pkg/planner/core/BUILD.bazel index 193263dbebb9e..86bffb8678eb9 100644 --- a/pkg/planner/core/BUILD.bazel +++ b/pkg/planner/core/BUILD.bazel @@ -29,11 +29,9 @@ go_library( "logical_expand.go", "logical_index_scan.go", "logical_initialize.go", - "logical_join.go", "logical_partition_union_all.go", "logical_plan_builder.go", "logical_plans.go", - "logical_selection.go", "logical_table_scan.go", "logical_tikv_single_gather.go", "logical_union_all.go", @@ -134,6 +132,7 @@ go_library( "//pkg/planner/cardinality", "//pkg/planner/context", "//pkg/planner/core/base", + "//pkg/planner/core/constraint", "//pkg/planner/core/cost", "//pkg/planner/core/metrics", "//pkg/planner/core/operator/baseimpl", diff --git a/pkg/planner/core/casetest/BUILD.bazel b/pkg/planner/core/casetest/BUILD.bazel index e005981526031..839db4514a465 100644 --- a/pkg/planner/core/casetest/BUILD.bazel +++ b/pkg/planner/core/casetest/BUILD.bazel @@ -20,6 +20,7 @@ go_test( "//pkg/parser/model", "//pkg/planner/core", "//pkg/planner/core/base", + "//pkg/planner/core/operator/logicalop", "//pkg/planner/property", "//pkg/planner/util/coretestsdk", "//pkg/testkit", diff --git a/pkg/planner/core/casetest/stats_test.go b/pkg/planner/core/casetest/stats_test.go index ea57b23854697..1125c15317c96 100644 --- a/pkg/planner/core/casetest/stats_test.go +++ b/pkg/planner/core/casetest/stats_test.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/planner/core/base" + "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/testdata" @@ -68,7 +69,7 @@ func TestGroupNDVs(t *testing.T) { _, err = core.RecursiveDeriveStats4Test(lp) require.NoError(t, err, comment) var agg *core.LogicalAggregation - var join *core.LogicalJoin + var join *logicalop.LogicalJoin stack := make([]base.LogicalPlan, 0, 2) traversed := false for !traversed { @@ -76,7 +77,7 @@ func TestGroupNDVs(t *testing.T) { case *core.LogicalAggregation: agg = v lp = lp.Children()[0] - case *core.LogicalJoin: + case *logicalop.LogicalJoin: join = v lp = v.Children()[0] stack = append(stack, v.Children()[1]) diff --git a/pkg/planner/core/collect_column_stats_usage.go b/pkg/planner/core/collect_column_stats_usage.go index 54c427c82289f..3a6ffdef2d8ae 100644 --- a/pkg/planner/core/collect_column_stats_usage.go +++ b/pkg/planner/core/collect_column_stats_usage.go @@ -143,7 +143,7 @@ func (c *columnStatsUsageCollector) collectPredicateColumnsForDataSource(ds *Dat c.addPredicateColumnsFromExpressions(ds.PushedDownConds) } -func (c *columnStatsUsageCollector) collectPredicateColumnsForJoin(p *LogicalJoin) { +func (c *columnStatsUsageCollector) collectPredicateColumnsForJoin(p *logicalop.LogicalJoin) { // The only schema change is merging two schemas so there is no new column. // Assume statistics of all the columns in EqualConditions/LeftConditions/RightConditions/OtherConditions are needed. exprs := make([]expression.Expression, 0, len(p.EqualConditions)+len(p.LeftConditions)+len(p.RightConditions)+len(p.OtherConditions)) @@ -246,7 +246,7 @@ func (c *columnStatsUsageCollector) collectFromPlan(lp base.LogicalPlan) { for i, expr := range x.Exprs { c.updateColMapFromExpressions(schema.Columns[i], []expression.Expression{expr}) } - case *LogicalSelection: + case *logicalop.LogicalSelection: // Though the conditions in LogicalSelection are complex conditions which cannot be pushed down to DataSource, we still // regard statistics of the columns in the conditions as needed. c.addPredicateColumnsFromExpressions(x.Conditions) @@ -269,7 +269,7 @@ func (c *columnStatsUsageCollector) collectFromPlan(lp base.LogicalPlan) { for i, col := range windowColumns { c.updateColMapFromExpressions(col, x.WindowFuncDescs[i].Args) } - case *LogicalJoin: + case *logicalop.LogicalJoin: c.collectPredicateColumnsForJoin(x) case *LogicalApply: c.collectPredicateColumnsForJoin(&x.LogicalJoin) diff --git a/pkg/planner/core/constraint/BUILD.bazel b/pkg/planner/core/constraint/BUILD.bazel new file mode 100644 index 0000000000000..6fd507bd5f9bc --- /dev/null +++ b/pkg/planner/core/constraint/BUILD.bazel @@ -0,0 +1,12 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "constraint", + srcs = ["exprs.go"], + importpath = "github.com/pingcap/tidb/pkg/planner/core/constraint", + visibility = ["//visibility:public"], + deps = [ + "//pkg/expression", + "//pkg/planner/core/base", + ], +) diff --git a/pkg/planner/core/constraint/exprs.go b/pkg/planner/core/constraint/exprs.go new file mode 100644 index 0000000000000..07d5ea9c6aeec --- /dev/null +++ b/pkg/planner/core/constraint/exprs.go @@ -0,0 +1,42 @@ +// Copyright 2024 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package constraint + +import ( + "github.com/pingcap/tidb/pkg/expression" + "github.com/pingcap/tidb/pkg/planner/core/base" +) + +// DeleteTrueExprs deletes the surely true expressions +func DeleteTrueExprs(p base.LogicalPlan, conds []expression.Expression) []expression.Expression { + newConds := make([]expression.Expression, 0, len(conds)) + for _, cond := range conds { + con, ok := cond.(*expression.Constant) + if !ok { + newConds = append(newConds, cond) + continue + } + if expression.MaybeOverOptimized4PlanCache(p.SCtx().GetExprCtx(), []expression.Expression{con}) { + newConds = append(newConds, cond) + continue + } + sc := p.SCtx().GetSessionVars().StmtCtx + if isTrue, err := con.Value.ToBool(sc.TypeCtx()); err == nil && isTrue == 1 { + continue + } + newConds = append(newConds, cond) + } + return newConds +} diff --git a/pkg/planner/core/core_init.go b/pkg/planner/core/core_init.go index e44106903368c..23d47d8bbb61d 100644 --- a/pkg/planner/core/core_init.go +++ b/pkg/planner/core/core_init.go @@ -44,9 +44,11 @@ func init() { utilfuncp.ExhaustPhysicalPlans4LogicalSort = exhaustPhysicalPlans4LogicalSort utilfuncp.ExhaustPhysicalPlans4LogicalTopN = exhaustPhysicalPlans4LogicalTopN utilfuncp.ExhaustPhysicalPlans4LogicalLock = exhaustPhysicalPlans4LogicalLock + utilfuncp.ExhaustPhysicalPlans4LogicalJoin = exhaustPhysicalPlans4LogicalJoin utilfuncp.ExhaustPhysicalPlans4LogicalLimit = exhaustPhysicalPlans4LogicalLimit utilfuncp.ExhaustPhysicalPlans4LogicalWindow = exhaustPhysicalPlans4LogicalWindow utilfuncp.ExhaustPhysicalPlans4LogicalSequence = exhaustPhysicalPlans4LogicalSequence + utilfuncp.ExhaustPhysicalPlans4LogicalSelection = exhaustPhysicalPlans4LogicalSelection utilfuncp.ExhaustPhysicalPlans4LogicalMaxOneRow = exhaustPhysicalPlans4LogicalMaxOneRow utilfuncp.ExhaustPhysicalPlans4LogicalUnionScan = exhaustPhysicalPlans4LogicalUnionScan utilfuncp.ExhaustPhysicalPlans4LogicalProjection = exhaustPhysicalPlans4LogicalProjection @@ -55,7 +57,9 @@ func init() { utilfuncp.GetEstimatedProbeCntFromProbeParents = getEstimatedProbeCntFromProbeParents utilfuncp.AppendCandidate4PhysicalOptimizeOp = appendCandidate4PhysicalOptimizeOp + utilfuncp.PushDownTopNForBaseLogicalPlan = pushDownTopNForBaseLogicalPlan utilfuncp.AttachPlan2Task = attachPlan2Task + utilfuncp.WindowIsTopN = windowIsTopN // For mv index init. cardinality.GetTblInfoForUsedStatsByPhysicalID = getTblInfoForUsedStatsByPhysicalID diff --git a/pkg/planner/core/exhaust_physical_plans.go b/pkg/planner/core/exhaust_physical_plans.go index 08a8fbb7c055e..4befccae0431c 100644 --- a/pkg/planner/core/exhaust_physical_plans.go +++ b/pkg/planner/core/exhaust_physical_plans.go @@ -78,7 +78,7 @@ func findMaxPrefixLen(candidates [][]*expression.Column, keys []*expression.Colu return maxLen } -func moveEqualToOtherConditions(p *LogicalJoin, offsets []int) []expression.Expression { +func moveEqualToOtherConditions(p *logicalop.LogicalJoin, offsets []int) []expression.Expression { // Construct used equal condition set based on the equal condition offsets. usedEqConds := set.NewIntSet() for _, eqCondIdx := range offsets { @@ -114,10 +114,10 @@ func (p *PhysicalMergeJoin) tryToGetChildReqProp(prop *property.PhysicalProperty if !prop.IsPrefix(lProp) && !prop.IsPrefix(rProp) { return nil, false } - if prop.IsPrefix(rProp) && p.JoinType == LeftOuterJoin { + if prop.IsPrefix(rProp) && p.JoinType == logicalop.LeftOuterJoin { return nil, false } - if prop.IsPrefix(lProp) && p.JoinType == RightOuterJoin { + if prop.IsPrefix(lProp) && p.JoinType == logicalop.RightOuterJoin { return nil, false } } @@ -141,7 +141,7 @@ func checkJoinKeyCollation(leftKeys, rightKeys []*expression.Column) bool { } // GetMergeJoin convert the logical join to physical merge join based on the physical property. -func GetMergeJoin(p *LogicalJoin, prop *property.PhysicalProperty, schema *expression.Schema, statsInfo *property.StatsInfo, leftStatsInfo *property.StatsInfo, rightStatsInfo *property.StatsInfo) []base.PhysicalPlan { +func GetMergeJoin(p *logicalop.LogicalJoin, prop *property.PhysicalProperty, schema *expression.Schema, statsInfo *property.StatsInfo, leftStatsInfo *property.StatsInfo, rightStatsInfo *property.StatsInfo) []base.PhysicalPlan { joins := make([]base.PhysicalPlan, 0, len(p.LeftProperties)+1) // The LeftProperties caches all the possible properties that are provided by its children. leftJoinKeys, rightJoinKeys, isNullEQ, hasNullEQ := p.GetJoinKeys() @@ -277,7 +277,7 @@ func getNewNullEQByOffsets(oldNullEQ []bool, offsets []int) []bool { return newNullEQ } -func getEnforcedMergeJoin(p *LogicalJoin, prop *property.PhysicalProperty, schema *expression.Schema, statsInfo *property.StatsInfo) []base.PhysicalPlan { +func getEnforcedMergeJoin(p *logicalop.LogicalJoin, prop *property.PhysicalProperty, schema *expression.Schema, statsInfo *property.StatsInfo) []base.PhysicalPlan { // Check whether SMJ can satisfy the required property leftJoinKeys, rightJoinKeys, isNullEQ, hasNullEQ := p.GetJoinKeys() // TODO: support null equal join keys for merge join @@ -321,10 +321,10 @@ func getEnforcedMergeJoin(p *LogicalJoin, prop *property.PhysicalProperty, schem return nil } // If the output wants the order of the inner side. We should reject it since we might add null-extend rows of that side. - if p.JoinType == LeftOuterJoin && hasRightColInProp { + if p.JoinType == logicalop.LeftOuterJoin && hasRightColInProp { return nil } - if p.JoinType == RightOuterJoin && hasLeftColInProp { + if p.JoinType == logicalop.RightOuterJoin && hasLeftColInProp { return nil } } @@ -368,11 +368,11 @@ func (p *PhysicalMergeJoin) initCompareFuncs() { } } -func shouldSkipHashJoin(p *LogicalJoin) bool { +func shouldSkipHashJoin(p *logicalop.LogicalJoin) bool { return (p.PreferJoinType&h.PreferNoHashJoin) > 0 || (p.SCtx().GetSessionVars().DisableHashJoin) } -func getHashJoins(p *LogicalJoin, prop *property.PhysicalProperty) (joins []base.PhysicalPlan, forced bool) { +func getHashJoins(p *logicalop.LogicalJoin, prop *property.PhysicalProperty) (joins []base.PhysicalPlan, forced bool) { if !prop.IsSortItemEmpty() { // hash join doesn't promise any orders return } @@ -387,7 +387,7 @@ func getHashJoins(p *LogicalJoin, prop *property.PhysicalProperty) (joins []base joins = make([]base.PhysicalPlan, 0, 2) switch p.JoinType { - case SemiJoin, AntiSemiJoin, LeftOuterSemiJoin, AntiLeftOuterSemiJoin: + case logicalop.SemiJoin, logicalop.AntiSemiJoin, logicalop.LeftOuterSemiJoin, logicalop.AntiLeftOuterSemiJoin: joins = append(joins, getHashJoin(p, prop, 1, false)) if forceLeftToBuild || forceRightToBuild { // Do not support specifying the build and probe side for semi join. @@ -395,21 +395,21 @@ func getHashJoins(p *LogicalJoin, prop *property.PhysicalProperty) (joins []base forceLeftToBuild = false forceRightToBuild = false } - case LeftOuterJoin: + case logicalop.LeftOuterJoin: if !forceLeftToBuild { joins = append(joins, getHashJoin(p, prop, 1, false)) } if !forceRightToBuild { joins = append(joins, getHashJoin(p, prop, 1, true)) } - case RightOuterJoin: + case logicalop.RightOuterJoin: if !forceLeftToBuild { joins = append(joins, getHashJoin(p, prop, 0, true)) } if !forceRightToBuild { joins = append(joins, getHashJoin(p, prop, 0, false)) } - case InnerJoin: + case logicalop.InnerJoin: if forceLeftToBuild { joins = append(joins, getHashJoin(p, prop, 0, false)) } else if forceRightToBuild { @@ -432,7 +432,7 @@ func getHashJoins(p *LogicalJoin, prop *property.PhysicalProperty) (joins []base return } -func getHashJoin(p *LogicalJoin, prop *property.PhysicalProperty, innerIdx int, useOuterToBuild bool) *PhysicalHashJoin { +func getHashJoin(p *logicalop.LogicalJoin, prop *property.PhysicalProperty, innerIdx int, useOuterToBuild bool) *PhysicalHashJoin { chReqProps := make([]*property.PhysicalProperty, 2) chReqProps[innerIdx] = &property.PhysicalProperty{ExpectedCnt: math.MaxFloat64, CTEProducerStatus: prop.CTEProducerStatus} chReqProps[1-innerIdx] = &property.PhysicalProperty{ExpectedCnt: math.MaxFloat64, CTEProducerStatus: prop.CTEProducerStatus} @@ -448,7 +448,7 @@ func getHashJoin(p *LogicalJoin, prop *property.PhysicalProperty, innerIdx int, // When inner plan is TableReader, the parameter `ranges` will be nil. Because pk only have one column. So all of its range // is generated during execution time. func constructIndexJoin( - p *LogicalJoin, + p *logicalop.LogicalJoin, prop *property.PhysicalProperty, outerIdx int, innerTask base.Task, @@ -563,7 +563,7 @@ func constructIndexJoin( } func constructIndexMergeJoin( - p *LogicalJoin, + p *logicalop.LogicalJoin, prop *property.PhysicalProperty, outerIdx int, innerTask base.Task, @@ -671,7 +671,7 @@ func constructIndexMergeJoin( } func constructIndexHashJoin( - p *LogicalJoin, + p *logicalop.LogicalJoin, prop *property.PhysicalProperty, outerIdx int, innerTask base.Task, @@ -699,7 +699,7 @@ func constructIndexHashJoin( // First of all, we'll check whether the inner child is DataSource. // Then, we will extract the join keys of p's equal conditions. Then check whether all of them are just the primary key // or match some part of on index. If so we will choose the best one and construct a index join. -func getIndexJoinByOuterIdx(p *LogicalJoin, prop *property.PhysicalProperty, outerIdx int) (joins []base.PhysicalPlan) { +func getIndexJoinByOuterIdx(p *logicalop.LogicalJoin, prop *property.PhysicalProperty, outerIdx int) (joins []base.PhysicalPlan) { outerChild, innerChild := p.Children()[outerIdx], p.Children()[1-outerIdx] all, _ := prop.AllSameOrder() // If the order by columns are not all from outer child, index join cannot promise the order. @@ -751,7 +751,7 @@ type indexJoinInnerChildWrapper struct { zippedChildren []base.LogicalPlan } -func extractIndexJoinInnerChildPattern(p *LogicalJoin, innerChild base.LogicalPlan) *indexJoinInnerChildWrapper { +func extractIndexJoinInnerChildPattern(p *logicalop.LogicalJoin, innerChild base.LogicalPlan) *indexJoinInnerChildWrapper { wrapper := &indexJoinInnerChildWrapper{} nextChild := func(pp base.LogicalPlan) base.LogicalPlan { if len(pp.Children()) != 1 { @@ -765,7 +765,7 @@ childLoop: case *DataSource: wrapper.ds = child break childLoop - case *logicalop.LogicalProjection, *LogicalSelection, *LogicalAggregation: + case *logicalop.LogicalProjection, *logicalop.LogicalSelection, *LogicalAggregation: if !p.SCtx().GetSessionVars().EnableINLJoinInnerMultiPattern { return nil } @@ -789,7 +789,7 @@ childLoop: // fetched from the inner side for every tuple from the outer side. This will be // promised to be no worse than building IndexScan as the inner child. func buildIndexJoinInner2TableScan( - p *LogicalJoin, + p *logicalop.LogicalJoin, prop *property.PhysicalProperty, wrapper *indexJoinInnerChildWrapper, innerJoinKeys, outerJoinKeys []*expression.Column, outerIdx int, avgInnerRowCnt float64) (joins []base.PhysicalPlan) { @@ -889,7 +889,7 @@ func buildIndexJoinInner2TableScan( } func buildIndexJoinInner2IndexScan( - p *LogicalJoin, + p *logicalop.LogicalJoin, prop *property.PhysicalProperty, wrapper *indexJoinInnerChildWrapper, innerJoinKeys, outerJoinKeys []*expression.Column, outerIdx int, avgInnerRowCnt float64) (joins []base.PhysicalPlan) { ds := wrapper.ds @@ -950,7 +950,7 @@ func buildIndexJoinInner2IndexScan( // constructInnerTableScanTask is specially used to construct the inner plan for PhysicalIndexJoin. func constructInnerTableScanTask( - p *LogicalJoin, + p *logicalop.LogicalJoin, prop *property.PhysicalProperty, wrapper *indexJoinInnerChildWrapper, ranges ranger.Ranges, @@ -1034,7 +1034,7 @@ func constructInnerByZippedChildren(prop *property.PhysicalProperty, zippedChild child = constructInnerUnionScan(prop, x, child) case *logicalop.LogicalProjection: child = constructInnerProj(prop, x, child) - case *LogicalSelection: + case *logicalop.LogicalSelection: child = constructInnerSel(prop, x, child) case *LogicalAggregation: child = constructInnerAgg(prop, x, child) @@ -1053,7 +1053,7 @@ func constructInnerAgg(prop *property.PhysicalProperty, logicalAgg *LogicalAggre return physicalHashAgg } -func constructInnerSel(prop *property.PhysicalProperty, sel *LogicalSelection, child base.PhysicalPlan) base.PhysicalPlan { +func constructInnerSel(prop *property.PhysicalProperty, sel *logicalop.LogicalSelection, child base.PhysicalPlan) base.PhysicalPlan { if sel == nil { return child } @@ -1143,7 +1143,7 @@ func getColsNDVLowerBoundFromHistColl(colUIDs []int64, histColl *statistics.Hist // constructInnerIndexScanTask is specially used to construct the inner plan for PhysicalIndexJoin. func constructInnerIndexScanTask( - p *LogicalJoin, + p *logicalop.LogicalJoin, prop *property.PhysicalProperty, wrapper *indexJoinInnerChildWrapper, path *util.AccessPath, @@ -1337,7 +1337,7 @@ func constructInnerIndexScanTask( // There are two kinds of agg: stream agg and hash agg. Stream agg depends on some conditions, such as the group by cols // // Step2: build other inner plan node to task -func constructIndexJoinInnerSideTask(p *LogicalJoin, prop *property.PhysicalProperty, dsCopTask *CopTask, ds *DataSource, path *util.AccessPath, wrapper *indexJoinInnerChildWrapper) base.Task { +func constructIndexJoinInnerSideTask(p *logicalop.LogicalJoin, prop *property.PhysicalProperty, dsCopTask *CopTask, ds *DataSource, path *util.AccessPath, wrapper *indexJoinInnerChildWrapper) base.Task { var la *LogicalAggregation var canPushAggToCop bool if len(wrapper.zippedChildren) > 0 { @@ -1483,16 +1483,16 @@ func getIndexJoinSideAndMethod(join base.PhysicalPlan) (innerSide, joinMethod in } // tryToGetIndexJoin returns all available index join plans, and the second returned value indicates whether this plan is enforced by hints. -func tryToGetIndexJoin(p *LogicalJoin, prop *property.PhysicalProperty) (indexJoins []base.PhysicalPlan, canForced bool) { +func tryToGetIndexJoin(p *logicalop.LogicalJoin, prop *property.PhysicalProperty) (indexJoins []base.PhysicalPlan, canForced bool) { // supportLeftOuter and supportRightOuter indicates whether this type of join // supports the left side or right side to be the outer side. var supportLeftOuter, supportRightOuter bool switch p.JoinType { - case SemiJoin, AntiSemiJoin, LeftOuterSemiJoin, AntiLeftOuterSemiJoin, LeftOuterJoin: + case logicalop.SemiJoin, logicalop.AntiSemiJoin, logicalop.LeftOuterSemiJoin, logicalop.AntiLeftOuterSemiJoin, logicalop.LeftOuterJoin: supportLeftOuter = true - case RightOuterJoin: + case logicalop.RightOuterJoin: supportRightOuter = true - case InnerJoin: + case logicalop.InnerJoin: supportLeftOuter, supportRightOuter = true, true } candidates := make([]base.PhysicalPlan, 0, 2) @@ -1525,7 +1525,7 @@ func tryToGetIndexJoin(p *LogicalJoin, prop *property.PhysicalProperty) (indexJo return filterIndexJoinBySessionVars(p.SCtx(), candidates), false } -func handleFilterIndexJoinHints(p *LogicalJoin, candidates []base.PhysicalPlan) []base.PhysicalPlan { +func handleFilterIndexJoinHints(p *logicalop.LogicalJoin, candidates []base.PhysicalPlan) []base.PhysicalPlan { if !p.PreferAny(h.PreferNoIndexJoin, h.PreferNoIndexHashJoin, h.PreferNoIndexMergeJoin) { return candidates // no filter index join hints } @@ -1546,7 +1546,7 @@ func handleFilterIndexJoinHints(p *LogicalJoin, candidates []base.PhysicalPlan) } // handleForceIndexJoinHints handles the force index join hints and returns all plans that can satisfy the hints. -func handleForceIndexJoinHints(p *LogicalJoin, prop *property.PhysicalProperty, candidates []base.PhysicalPlan) (indexJoins []base.PhysicalPlan, canForced bool) { +func handleForceIndexJoinHints(p *logicalop.LogicalJoin, prop *property.PhysicalProperty, candidates []base.PhysicalPlan) (indexJoins []base.PhysicalPlan, canForced bool) { if !p.PreferAny(h.PreferRightAsINLJInner, h.PreferRightAsINLHJInner, h.PreferRightAsINLMJInner, h.PreferLeftAsINLJInner, h.PreferLeftAsINLHJInner, h.PreferLeftAsINLMJInner) { return candidates, false // no force index join hints @@ -1665,7 +1665,7 @@ func calcHashExchangeSizeByChild(p1 base.Plan, p2 base.Plan, mppStoreCnt int) (r // It will cost more time to construct `Build` hash table and search `Probe` while using broadcast join. // Set a scale factor (`mppStoreCnt^*`) when estimating broadcast join in `isJoinFitMPPBCJ` and `isJoinChildFitMPPBCJ` (based on TPCH benchmark, it has been verified in Q9). -func isJoinFitMPPBCJ(p *LogicalJoin, mppStoreCnt int) bool { +func isJoinFitMPPBCJ(p *logicalop.LogicalJoin, mppStoreCnt int) bool { rowBC, szBC, hasSizeBC := calcBroadcastExchangeSizeByChild(p.Children()[0], p.Children()[1], mppStoreCnt) rowHash, szHash, hasSizeHash := calcHashExchangeSizeByChild(p.Children()[0], p.Children()[1], mppStoreCnt) if hasSizeBC && hasSizeHash { @@ -1674,7 +1674,7 @@ func isJoinFitMPPBCJ(p *LogicalJoin, mppStoreCnt int) bool { return rowBC*float64(mppStoreCnt) <= rowHash } -func isJoinChildFitMPPBCJ(p *LogicalJoin, childIndexToBC int, mppStoreCnt int) bool { +func isJoinChildFitMPPBCJ(p *logicalop.LogicalJoin, childIndexToBC int, mppStoreCnt int) bool { rowBC, szBC, hasSizeBC := calcBroadcastExchangeSize(p.Children()[childIndexToBC], mppStoreCnt) rowHash, szHash, hasSizeHash := calcHashExchangeSizeByChild(p.Children()[0], p.Children()[1], mppStoreCnt) @@ -1685,13 +1685,13 @@ func isJoinChildFitMPPBCJ(p *LogicalJoin, childIndexToBC int, mppStoreCnt int) b } // If we can use mpp broadcast join, that's our first choice. -func preferMppBCJ(p *LogicalJoin) bool { +func preferMppBCJ(p *logicalop.LogicalJoin) bool { if len(p.EqualConditions) == 0 && p.SCtx().GetSessionVars().AllowCartesianBCJ == 2 { return true } - onlyCheckChild1 := p.JoinType == LeftOuterJoin || p.JoinType == SemiJoin || p.JoinType == AntiSemiJoin - onlyCheckChild0 := p.JoinType == RightOuterJoin + onlyCheckChild1 := p.JoinType == logicalop.LeftOuterJoin || p.JoinType == logicalop.SemiJoin || p.JoinType == logicalop.AntiSemiJoin + onlyCheckChild0 := p.JoinType == logicalop.RightOuterJoin if p.SCtx().GetSessionVars().PreferBCJByExchangeDataSize { mppStoreCnt, err := p.SCtx().GetMPPClient().GetMPPStoreCount() @@ -1723,7 +1723,7 @@ func preferMppBCJ(p *LogicalJoin) bool { return checkChildFitBC(p.Children()[0]) || checkChildFitBC(p.Children()[1]) } -func canExprsInJoinPushdown(p *LogicalJoin, storeType kv.StoreType) bool { +func canExprsInJoinPushdown(p *logicalop.LogicalJoin, storeType kv.StoreType) bool { equalExprs := make([]expression.Expression, 0, len(p.EqualConditions)) for _, eqCondition := range p.EqualConditions { if eqCondition.FuncName.L == ast.NullEQ { @@ -1731,7 +1731,7 @@ func canExprsInJoinPushdown(p *LogicalJoin, storeType kv.StoreType) bool { } equalExprs = append(equalExprs, eqCondition) } - pushDownCtx := GetPushDownCtx(p.SCtx()) + pushDownCtx := util.GetPushDownCtx(p.SCtx()) if !expression.CanExprsPushDown(pushDownCtx, equalExprs, storeType) { return false } @@ -1747,7 +1747,7 @@ func canExprsInJoinPushdown(p *LogicalJoin, storeType kv.StoreType) bool { return true } -func tryToGetMppHashJoin(p *LogicalJoin, prop *property.PhysicalProperty, useBCJ bool) []base.PhysicalPlan { +func tryToGetMppHashJoin(p *logicalop.LogicalJoin, prop *property.PhysicalProperty, useBCJ bool) []base.PhysicalPlan { if !prop.IsSortItemEmpty() { return nil } @@ -1760,7 +1760,7 @@ func tryToGetMppHashJoin(p *LogicalJoin, prop *property.PhysicalProperty, useBCJ return nil } - if p.JoinType != InnerJoin && p.JoinType != LeftOuterJoin && p.JoinType != RightOuterJoin && p.JoinType != SemiJoin && p.JoinType != AntiSemiJoin && p.JoinType != LeftOuterSemiJoin && p.JoinType != AntiLeftOuterSemiJoin { + if p.JoinType != logicalop.InnerJoin && p.JoinType != logicalop.LeftOuterJoin && p.JoinType != logicalop.RightOuterJoin && p.JoinType != logicalop.SemiJoin && p.JoinType != logicalop.AntiSemiJoin && p.JoinType != logicalop.LeftOuterSemiJoin && p.JoinType != logicalop.AntiLeftOuterSemiJoin { p.SCtx().GetSessionVars().RaiseWarningWhenMPPEnforced("MPP mode may be blocked because join type `" + p.JoinType.String() + "` is not supported now.") return nil } @@ -1775,11 +1775,11 @@ func tryToGetMppHashJoin(p *LogicalJoin, prop *property.PhysicalProperty, useBCJ return nil } } - if len(p.LeftConditions) != 0 && p.JoinType != LeftOuterJoin { + if len(p.LeftConditions) != 0 && p.JoinType != logicalop.LeftOuterJoin { p.SCtx().GetSessionVars().RaiseWarningWhenMPPEnforced("MPP mode may be blocked because there is a join that is not `left join` but has left conditions, which is not supported by mpp now, see github.com/pingcap/tidb/issues/26090 for more information.") return nil } - if len(p.RightConditions) != 0 && p.JoinType != RightOuterJoin { + if len(p.RightConditions) != 0 && p.JoinType != logicalop.RightOuterJoin { p.SCtx().GetSessionVars().RaiseWarningWhenMPPEnforced("MPP mode may be blocked because there is a join that is not `right join` but has right conditions, which is not supported by mpp now.") return nil } @@ -1815,12 +1815,12 @@ func tryToGetMppHashJoin(p *LogicalJoin, prop *property.PhysicalProperty, useBCJ } preferredBuildIndex := 0 fixedBuildSide := false // Used to indicate whether the build side for the MPP join is fixed or not. - if p.JoinType == InnerJoin { + if p.JoinType == logicalop.InnerJoin { if p.Children()[0].StatsInfo().Count() > p.Children()[1].StatsInfo().Count() { preferredBuildIndex = 1 } } else if p.JoinType.IsSemiJoin() { - if !useBCJ && !p.IsNAAJ() && len(p.EqualConditions) > 0 && (p.JoinType == SemiJoin || p.JoinType == AntiSemiJoin) { + if !useBCJ && !p.IsNAAJ() && len(p.EqualConditions) > 0 && (p.JoinType == logicalop.SemiJoin || p.JoinType == logicalop.AntiSemiJoin) { // TiFlash only supports Non-null_aware non-cross semi/anti_semi join to use both sides as build side preferredBuildIndex = 1 // MPPOuterJoinFixedBuildSide default value is false @@ -1833,7 +1833,7 @@ func tryToGetMppHashJoin(p *LogicalJoin, prop *property.PhysicalProperty, useBCJ fixedBuildSide = true } } - if p.JoinType == LeftOuterJoin || p.JoinType == RightOuterJoin { + if p.JoinType == logicalop.LeftOuterJoin || p.JoinType == logicalop.RightOuterJoin { // TiFlash does not require that the build side must be the inner table for outer join. // so we can choose the build side based on the row count, except that: // 1. it is a broadcast join(for broadcast join, it makes sense to use the broadcast side as the build side) @@ -1844,7 +1844,7 @@ func tryToGetMppHashJoin(p *LogicalJoin, prop *property.PhysicalProperty, useBCJ // The hint has higher priority than variable. fixedBuildSide = true } - if p.JoinType == LeftOuterJoin { + if p.JoinType == logicalop.LeftOuterJoin { preferredBuildIndex = 1 } } else if p.Children()[0].StatsInfo().Count() > p.Children()[1].StatsInfo().Count() { @@ -1900,11 +1900,11 @@ func tryToGetMppHashJoin(p *LogicalJoin, prop *property.PhysicalProperty, useBCJ lPartitionKeys, rPartitionKeys := p.GetPotentialPartitionKeys() if prop.MPPPartitionTp == property.HashType { var matches []int - if p.JoinType == InnerJoin { + if p.JoinType == logicalop.InnerJoin { if matches = prop.IsSubsetOf(lPartitionKeys); len(matches) == 0 { matches = prop.IsSubsetOf(rPartitionKeys) } - } else if p.JoinType == RightOuterJoin { + } else if p.JoinType == logicalop.RightOuterJoin { // for right out join, only the right partition keys can possibly matches the prop, because // the left partition keys will generate NULL values randomly // todo maybe we can add a null-sensitive flag in the MPPPartitionColumn to indicate whether the partition column is @@ -1947,6 +1947,104 @@ func choosePartitionKeys(keys []*property.MPPPartitionColumn, matches []int) []* return newKeys } +// it can generates hash join, index join and sort merge join. +// Firstly we check the hint, if hint is figured by user, we force to choose the corresponding physical plan. +// If the hint is not matched, it will get other candidates. +// If the hint is not figured, we will pick all candidates. +func exhaustPhysicalPlans4LogicalJoin(lp base.LogicalPlan, prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) { + p := lp.(*logicalop.LogicalJoin) + failpoint.Inject("MockOnlyEnableIndexHashJoin", func(val failpoint.Value) { + if val.(bool) && !p.SCtx().GetSessionVars().InRestrictedSQL { + indexJoins, _ := tryToGetIndexJoin(p, prop) + failpoint.Return(indexJoins, true, nil) + } + }) + + if !isJoinHintSupportedInMPPMode(p.PreferJoinType) { + if hasMPPJoinHints(p.PreferJoinType) { + // If there are MPP hints but has some conflicts join method hints, all the join hints are invalid. + p.SCtx().GetSessionVars().StmtCtx.SetHintWarning("The MPP join hints are in conflict, and you can only specify join method hints that are currently supported by MPP mode now") + p.PreferJoinType = 0 + } else { + // If there are no MPP hints but has some conflicts join method hints, the MPP mode will be blocked. + p.SCtx().GetSessionVars().RaiseWarningWhenMPPEnforced("MPP mode may be blocked because you have used hint to specify a join algorithm which is not supported by mpp now.") + if prop.IsFlashProp() { + return nil, false, nil + } + } + } + if prop.MPPPartitionTp == property.BroadcastType { + return nil, false, nil + } + joins := make([]base.PhysicalPlan, 0, 8) + canPushToTiFlash := p.CanPushToCop(kv.TiFlash) + if p.SCtx().GetSessionVars().IsMPPAllowed() && canPushToTiFlash { + if (p.PreferJoinType & h.PreferShuffleJoin) > 0 { + if shuffleJoins := tryToGetMppHashJoin(p, prop, false); len(shuffleJoins) > 0 { + return shuffleJoins, true, nil + } + } + if (p.PreferJoinType & h.PreferBCJoin) > 0 { + if bcastJoins := tryToGetMppHashJoin(p, prop, true); len(bcastJoins) > 0 { + return bcastJoins, true, nil + } + } + if preferMppBCJ(p) { + mppJoins := tryToGetMppHashJoin(p, prop, true) + joins = append(joins, mppJoins...) + } else { + mppJoins := tryToGetMppHashJoin(p, prop, false) + joins = append(joins, mppJoins...) + } + } else { + hasMppHints := false + var errMsg string + if (p.PreferJoinType & h.PreferShuffleJoin) > 0 { + errMsg = "The join can not push down to the MPP side, the shuffle_join() hint is invalid" + hasMppHints = true + } + if (p.PreferJoinType & h.PreferBCJoin) > 0 { + errMsg = "The join can not push down to the MPP side, the broadcast_join() hint is invalid" + hasMppHints = true + } + if hasMppHints { + p.SCtx().GetSessionVars().StmtCtx.SetHintWarning(errMsg) + } + } + if prop.IsFlashProp() { + return joins, true, nil + } + + if !p.IsNAAJ() { + // naaj refuse merge join and index join. + mergeJoins := GetMergeJoin(p, prop, p.Schema(), p.StatsInfo(), p.Children()[0].StatsInfo(), p.Children()[1].StatsInfo()) + if (p.PreferJoinType&h.PreferMergeJoin) > 0 && len(mergeJoins) > 0 { + return mergeJoins, true, nil + } + joins = append(joins, mergeJoins...) + + indexJoins, forced := tryToGetIndexJoin(p, prop) + if forced { + return indexJoins, true, nil + } + joins = append(joins, indexJoins...) + } + + hashJoins, forced := getHashJoins(p, prop) + if forced && len(hashJoins) > 0 { + return hashJoins, true, nil + } + joins = append(joins, hashJoins...) + + if p.PreferJoinType > 0 { + // If we reach here, it means we have a hint that doesn't work. + // It might be affected by the required property, so we enforce + // this property and try the hint again. + return joins, false, nil + } + return joins, true, nil +} + func exhaustPhysicalPlans4LogicalExpand(p *LogicalExpand, prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) { // under the mpp task type, if the sort item is not empty, refuse it, cause expanded data doesn't support any sort items. if !prop.IsSortItemEmpty() { @@ -2012,7 +2110,7 @@ func exhaustPhysicalPlans4LogicalProjection(lp base.LogicalPlan, prop *property. newProps := []*property.PhysicalProperty{newProp} // generate a mpp task candidate if mpp mode is allowed ctx := p.SCtx() - pushDownCtx := GetPushDownCtx(ctx) + pushDownCtx := util.GetPushDownCtx(ctx) if newProp.TaskTp != property.MppTaskType && ctx.GetSessionVars().IsMPPAllowed() && p.CanPushToCop(kv.TiFlash) && expression.CanExprsPushDown(pushDownCtx, p.Exprs, kv.TiFlash) { mppProp := newProp.CloneEssentialFields() @@ -2201,7 +2299,7 @@ func tryToGetMppWindows(lw *logicalop.LogicalWindow, prop *property.PhysicalProp allSupported := true sctx := lw.SCtx() for _, windowFunc := range lw.WindowFuncDescs { - if !windowFunc.CanPushDownToTiFlash(GetPushDownCtx(sctx)) { + if !windowFunc.CanPushDownToTiFlash(util.GetPushDownCtx(sctx)) { lw.SCtx().GetSessionVars().RaiseWarningWhenMPPEnforced( "MPP mode may be blocked because window function `" + windowFunc.Name + "` or its arguments are not supported now.") allSupported = false @@ -2370,7 +2468,7 @@ func canPushToCopImpl(lp base.LogicalPlan, storeTp kv.StoreType, considerDual bo ret = ret && canPushToCopImpl(&c.BaseLogicalPlan, storeTp, considerDual) case *logicalop.LogicalTableDual: return storeTp == kv.TiFlash && considerDual - case *LogicalAggregation, *LogicalSelection, *LogicalJoin, *logicalop.LogicalWindow: + case *LogicalAggregation, *logicalop.LogicalSelection, *logicalop.LogicalJoin, *logicalop.LogicalWindow: if storeTp != kv.TiFlash { return false } @@ -2768,14 +2866,15 @@ func getHashAggs(lp base.LogicalPlan, prop *property.PhysicalProperty) []base.Ph return hashAggs } -func exhaustPhysicalPlans4LogicalSelection(p *LogicalSelection, prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) { +func exhaustPhysicalPlans4LogicalSelection(lp base.LogicalPlan, prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) { + p := lp.(*logicalop.LogicalSelection) newProps := make([]*property.PhysicalProperty, 0, 2) childProp := prop.CloneEssentialFields() newProps = append(newProps, childProp) if prop.TaskTp != property.MppTaskType && p.SCtx().GetSessionVars().IsMPPAllowed() && - p.canPushDown(kv.TiFlash) { + p.CanPushDown(kv.TiFlash) { childPropMpp := prop.CloneEssentialFields() childPropMpp.TaskTp = property.MppTaskType newProps = append(newProps, childPropMpp) diff --git a/pkg/planner/core/exhaust_physical_plans_test.go b/pkg/planner/core/exhaust_physical_plans_test.go index ef89bc1d1a712..982806a3b3740 100644 --- a/pkg/planner/core/exhaust_physical_plans_test.go +++ b/pkg/planner/core/exhaust_physical_plans_test.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/core/base" + "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/statistics" @@ -53,7 +54,7 @@ type indexJoinContext struct { dataSourceNode *DataSource dsNames types.NameSlice path *util.AccessPath - joinNode *LogicalJoin + joinNode *logicalop.LogicalJoin joinColNames types.NameSlice } @@ -64,7 +65,7 @@ func prepareForAnalyzeLookUpFilters() *indexJoinContext { do.StatsHandle().Close() }() ctx.GetSessionVars().PlanID.Store(-1) - joinNode := LogicalJoin{}.Init(ctx.GetPlanCtx(), 0) + joinNode := logicalop.LogicalJoin{}.Init(ctx.GetPlanCtx(), 0) dataSourceNode := DataSource{}.Init(ctx.GetPlanCtx(), 0) dsSchema := expression.NewSchema() var dsNames types.NameSlice diff --git a/pkg/planner/core/expression_rewriter.go b/pkg/planner/core/expression_rewriter.go index 2be09441220b4..9c7c7e5f82203 100644 --- a/pkg/planner/core/expression_rewriter.go +++ b/pkg/planner/core/expression_rewriter.go @@ -912,7 +912,7 @@ func (er *expressionRewriter) buildQuantifierPlan(planCtx *exprRewriterPlanCtx, } // If we treat the result as a scalar value, we will add a projection with a extra column to output true, false or null. outerSchemaLen := planCtx.plan.Schema().Len() - planCtx.plan = planCtx.builder.buildApplyWithJoinType(planCtx.plan, plan4Agg, InnerJoin, markNoDecorrelate) + planCtx.plan = planCtx.builder.buildApplyWithJoinType(planCtx.plan, plan4Agg, logicalop.InnerJoin, markNoDecorrelate) joinSchema := planCtx.plan.Schema() proj := logicalop.LogicalProjection{ Exprs: expression.Column2Exprs(joinSchema.Clone().Columns[:outerSchemaLen]), @@ -1237,7 +1237,7 @@ func (er *expressionRewriter) handleInSubquery(ctx context.Context, planCtx *exp return v, true } // Build inner join above the aggregation. - join := LogicalJoin{JoinType: InnerJoin}.Init(planCtx.builder.ctx, planCtx.builder.getSelectOffset()) + join := logicalop.LogicalJoin{JoinType: logicalop.InnerJoin}.Init(planCtx.builder.ctx, planCtx.builder.getSelectOffset()) join.SetChildren(planCtx.plan, agg) join.SetSchema(expression.MergeSchema(planCtx.plan.Schema(), agg.Schema())) join.SetOutputNames(make([]*types.FieldName, planCtx.plan.Schema().Len()+agg.Schema().Len())) @@ -1283,7 +1283,7 @@ func (er *expressionRewriter) handleScalarSubquery(ctx context.Context, planCtx } if planCtx.builder.disableSubQueryPreprocessing || len(coreusage.ExtractCorrelatedCols4LogicalPlan(np)) > 0 || hasCTEConsumerInSubPlan(np) { - planCtx.plan = planCtx.builder.buildApplyWithJoinType(planCtx.plan, np, LeftOuterJoin, noDecorrelate) + planCtx.plan = planCtx.builder.buildApplyWithJoinType(planCtx.plan, np, logicalop.LeftOuterJoin, noDecorrelate) if np.Schema().Len() > 1 { newCols := make([]expression.Expression, 0, np.Schema().Len()) for _, col := range np.Schema().Columns { @@ -2469,9 +2469,9 @@ func (er *expressionRewriter) toColumn(v *ast.ColumnName) { func findFieldNameFromNaturalUsingJoin(p base.LogicalPlan, v *ast.ColumnName) (col *expression.Column, name *types.FieldName, err error) { switch x := p.(type) { - case *logicalop.LogicalLimit, *LogicalSelection, *logicalop.LogicalTopN, *logicalop.LogicalSort, *logicalop.LogicalMaxOneRow: + case *logicalop.LogicalLimit, *logicalop.LogicalSelection, *logicalop.LogicalTopN, *logicalop.LogicalSort, *logicalop.LogicalMaxOneRow: return findFieldNameFromNaturalUsingJoin(p.Children()[0], v) - case *LogicalJoin: + case *logicalop.LogicalJoin: if x.FullSchema != nil { idx, err := expression.FindFieldName(x.FullNames, v) if err != nil { diff --git a/pkg/planner/core/find_best_task.go b/pkg/planner/core/find_best_task.go index 8e819dee9c6e8..6b0e5a854eb0a 100644 --- a/pkg/planner/core/find_best_task.go +++ b/pkg/planner/core/find_best_task.go @@ -981,7 +981,7 @@ func (ds *DataSource) matchPropForIndexMergeAlternatives(path *util.AccessPath, } // path.ShouldBeKeptCurrentFilter record that whether there are some part of the cnf item couldn't be pushed down to tikv already. shouldKeepCurrentFilter := path.KeepIndexMergeORSourceFilter - pushDownCtx := GetPushDownCtx(ds.SCtx()) + pushDownCtx := util.GetPushDownCtx(ds.SCtx()) for _, path := range determinedIndexPartialPaths { // If any partial path contains table filters, we need to keep the whole DNF filter in the Selection. if len(path.TableFilters) > 0 { @@ -1665,7 +1665,7 @@ func (ds *DataSource) convertToPartialIndexScan(physPlanPartInfo *PhysPlanPartIn } if len(indexConds) > 0 { - pushedFilters, remainingFilter := extractFiltersForIndexMerge(GetPushDownCtx(ds.SCtx()), indexConds) + pushedFilters, remainingFilter := extractFiltersForIndexMerge(util.GetPushDownCtx(ds.SCtx()), indexConds) var selectivity float64 if path.CountAfterAccess > 0 { selectivity = path.CountAfterIndex / path.CountAfterAccess @@ -1792,7 +1792,7 @@ func (ds *DataSource) buildIndexMergeTableScan(tableFilters []expression.Express } var currentTopPlan base.PhysicalPlan = ts if len(tableFilters) > 0 { - pushedFilters, remainingFilters := extractFiltersForIndexMerge(GetPushDownCtx(ds.SCtx()), tableFilters) + pushedFilters, remainingFilters := extractFiltersForIndexMerge(util.GetPushDownCtx(ds.SCtx()), tableFilters) pushedFilters1, remainingFilters1 := SplitSelCondsWithVirtualColumn(pushedFilters) pushedFilters = pushedFilters1 remainingFilters = append(remainingFilters, remainingFilters1...) @@ -2259,7 +2259,7 @@ func (is *PhysicalIndexScan) addPushedDownSelection(copTask *CopTask, p *DataSou tableConds, copTask.rootTaskConds = SplitSelCondsWithVirtualColumn(tableConds) var newRootConds []expression.Expression - pctx := GetPushDownCtx(is.SCtx()) + pctx := util.GetPushDownCtx(is.SCtx()) indexConds, newRootConds = expression.PushDownExprs(pctx, indexConds, kv.TiKV) copTask.rootTaskConds = append(copTask.rootTaskConds, newRootConds...) @@ -2767,7 +2767,7 @@ func (ds *DataSource) convertToBatchPointGet(prop *property.PhysicalProperty, ca func (ts *PhysicalTableScan) addPushedDownSelectionToMppTask(mpp *MppTask, stats *property.StatsInfo) *MppTask { filterCondition, rootTaskConds := SplitSelCondsWithVirtualColumn(ts.filterCondition) var newRootConds []expression.Expression - filterCondition, newRootConds = expression.PushDownExprs(GetPushDownCtx(ts.SCtx()), filterCondition, ts.StoreType) + filterCondition, newRootConds = expression.PushDownExprs(util.GetPushDownCtx(ts.SCtx()), filterCondition, ts.StoreType) mpp.rootTaskConds = append(rootTaskConds, newRootConds...) ts.filterCondition = filterCondition @@ -2783,7 +2783,7 @@ func (ts *PhysicalTableScan) addPushedDownSelectionToMppTask(mpp *MppTask, stats func (ts *PhysicalTableScan) addPushedDownSelection(copTask *CopTask, stats *property.StatsInfo) { ts.filterCondition, copTask.rootTaskConds = SplitSelCondsWithVirtualColumn(ts.filterCondition) var newRootConds []expression.Expression - ts.filterCondition, newRootConds = expression.PushDownExprs(GetPushDownCtx(ts.SCtx()), ts.filterCondition, ts.StoreType) + ts.filterCondition, newRootConds = expression.PushDownExprs(util.GetPushDownCtx(ts.SCtx()), ts.filterCondition, ts.StoreType) copTask.rootTaskConds = append(copTask.rootTaskConds, newRootConds...) // Add filter condition to table plan now. diff --git a/pkg/planner/core/flat_plan.go b/pkg/planner/core/flat_plan.go index d77a4f9d30438..62ad6b59fb654 100644 --- a/pkg/planner/core/flat_plan.go +++ b/pkg/planner/core/flat_plan.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/planner/core/base" + "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/util/logutil" "github.com/pingcap/tidb/pkg/util/texttree" "go.uber.org/zap" @@ -276,7 +277,7 @@ func (f *FlatPhysicalPlan) flattenRecursively(p base.Plan, info *operatorCtx, ta label[1-plan.InnerChildIdx] = ProbeSide } case *PhysicalMergeJoin: - if plan.JoinType == RightOuterJoin { + if plan.JoinType == logicalop.RightOuterJoin { label[0] = BuildSide label[1] = ProbeSide } else { diff --git a/pkg/planner/core/hint_utils.go b/pkg/planner/core/hint_utils.go index c4fa9bcd8ba40..62e63f6fe5813 100644 --- a/pkg/planner/core/hint_utils.go +++ b/pkg/planner/core/hint_utils.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/planner/core/base" + "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" h "github.com/pingcap/tidb/pkg/util/hint" ) @@ -579,7 +580,7 @@ func extractOrderedPhysicalJoinGroup(p PhysicalJoin, visitedIDs map[int]struct{} jt := p.GetJoinType() // They are the only join types supported by current join reorder. - if jt != InnerJoin && jt != LeftOuterJoin && jt != RightOuterJoin { + if jt != logicalop.InnerJoin && jt != logicalop.LeftOuterJoin && jt != logicalop.RightOuterJoin { return nil } diff --git a/pkg/planner/core/index_join_path.go b/pkg/planner/core/index_join_path.go index eeea1baceec7b..d34821de013c6 100644 --- a/pkg/planner/core/index_join_path.go +++ b/pkg/planner/core/index_join_path.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/cardinality" "github.com/pingcap/tidb/pkg/planner/context" + "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/statistics" @@ -584,7 +585,7 @@ func indexJoinPathRemoveUselessEQIn(buildTmp *indexJoinPathTmp, idxCols []*expre // getBestIndexJoinPathResult tries to iterate all possible access paths of the inner child and builds // index join path for each access path. It returns the best index join path result and the mapping. func getBestIndexJoinPathResult( - join *LogicalJoin, + join *logicalop.LogicalJoin, innerChild *DataSource, innerJoinKeys, outerJoinKeys []*expression.Column, checkPathValid func(path *util.AccessPath) bool) (*indexJoinPathResult, []int) { diff --git a/pkg/planner/core/indexmerge_path.go b/pkg/planner/core/indexmerge_path.go index 56e1c9aa47070..ce49e872016bf 100644 --- a/pkg/planner/core/indexmerge_path.go +++ b/pkg/planner/core/indexmerge_path.go @@ -138,7 +138,7 @@ func (ds *DataSource) generateNormalIndexPartialPaths4DNF( ) (paths []*util.AccessPath, needSelection bool, usedMap []bool) { paths = make([]*util.AccessPath, 0, len(dnfItems)) usedMap = make([]bool, len(dnfItems)) - pushDownCtx := GetPushDownCtx(ds.SCtx()) + pushDownCtx := util.GetPushDownCtx(ds.SCtx()) for offset, item := range dnfItems { cnfItems := expression.SplitCNFItems(item) pushedDownCNFItems := make([]expression.Expression, 0, len(cnfItems)) @@ -207,7 +207,7 @@ func (ds *DataSource) generateNormalIndexPartialPaths4DNF( // } func (ds *DataSource) generateIndexMergeOrPaths(filters []expression.Expression) error { usedIndexCount := len(ds.PossibleAccessPaths) - pushDownCtx := GetPushDownCtx(ds.SCtx()) + pushDownCtx := util.GetPushDownCtx(ds.SCtx()) for k, cond := range filters { sf, ok := cond.(*expression.ScalarFunction) if !ok || sf.FuncName.L != ast.LogicOr { @@ -524,7 +524,7 @@ func (ds *DataSource) generateIndexMergeAndPaths(normalPathCnt int, usedAccessMa finalFilters := make([]expression.Expression, 0) partialFilters := make([]expression.Expression, 0, len(partialPaths)) hashCodeSet := make(map[string]struct{}) - pushDownCtx := GetPushDownCtx(ds.SCtx()) + pushDownCtx := util.GetPushDownCtx(ds.SCtx()) for _, path := range partialPaths { // Classify filters into coveredConds and notCoveredConds. coveredConds := make([]expression.Expression, 0, len(path.AccessConds)+len(path.IndexFilters)) @@ -730,7 +730,7 @@ func (ds *DataSource) generateIndexMerge4NormalIndex(regularPathCount int, index // PushDownExprs() will append extra warnings, which is annoying. So we reset warnings here. warnings := stmtCtx.GetWarnings() extraWarnings := stmtCtx.GetExtraWarnings() - _, remaining := expression.PushDownExprs(GetPushDownCtx(ds.SCtx()), indexMergeConds, kv.UnSpecified) + _, remaining := expression.PushDownExprs(util.GetPushDownCtx(ds.SCtx()), indexMergeConds, kv.UnSpecified) stmtCtx.SetWarnings(warnings) stmtCtx.SetExtraWarnings(extraWarnings) if len(remaining) > 0 { diff --git a/pkg/planner/core/logical_apply.go b/pkg/planner/core/logical_apply.go index 52bb857c35f59..100b636c3f362 100644 --- a/pkg/planner/core/logical_apply.go +++ b/pkg/planner/core/logical_apply.go @@ -32,7 +32,7 @@ import ( // LogicalApply gets one row from outer executor and gets one row from inner executor according to outer row. type LogicalApply struct { - LogicalJoin + logicalop.LogicalJoin CorCols []*expression.CorrelatedColumn // NoDecorrelate is from /*+ no_decorrelate() */ hint. @@ -73,10 +73,10 @@ func (la *LogicalApply) ReplaceExprColumns(replace map[string]*expression.Column // PruneColumns implements base.LogicalPlan.<2nd> interface. func (la *LogicalApply) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error) { - leftCols, rightCols := la.extractUsedCols(parentUsedCols) + leftCols, rightCols := la.ExtractUsedCols(parentUsedCols) allowEliminateApply := fixcontrol.GetBoolWithDefault(la.SCtx().GetSessionVars().GetOptimizerFixControlMap(), fixcontrol.Fix45822, true) var err error - if allowEliminateApply && rightCols == nil && la.JoinType == LeftOuterJoin { + if allowEliminateApply && rightCols == nil && la.JoinType == logicalop.LeftOuterJoin { logicaltrace.ApplyEliminateTraceStep(la.Children()[1], opt) resultPlan := la.Children()[0] // reEnter the new child's column pruning, returning child[0] as a new child here. @@ -99,7 +99,7 @@ func (la *LogicalApply) PruneColumns(parentUsedCols []*expression.Column, opt *o if err != nil { return nil, err } - la.mergeSchema() + la.MergeSchema() return la, nil } @@ -134,7 +134,7 @@ func (la *LogicalApply) DeriveStats(childStats []*property.StatsInfo, selfSchema for id, c := range leftProfile.ColNDVs { la.StatsInfo().ColNDVs[id] = c } - if la.JoinType == LeftOuterSemiJoin || la.JoinType == AntiLeftOuterSemiJoin { + if la.JoinType == logicalop.LeftOuterSemiJoin || la.JoinType == logicalop.AntiLeftOuterSemiJoin { la.StatsInfo().ColNDVs[selfSchema.Columns[selfSchema.Len()-1].UniqueID] = 2.0 } else { for i := childSchema[0].Len(); i < selfSchema.Len(); i++ { @@ -149,7 +149,7 @@ func (la *LogicalApply) DeriveStats(childStats []*property.StatsInfo, selfSchema func (la *LogicalApply) ExtractColGroups(colGroups [][]*expression.Column) [][]*expression.Column { var outerSchema *expression.Schema // Apply doesn't have RightOuterJoin. - if la.JoinType == LeftOuterJoin || la.JoinType == LeftOuterSemiJoin || la.JoinType == AntiLeftOuterSemiJoin { + if la.JoinType == logicalop.LeftOuterJoin || la.JoinType == logicalop.LeftOuterSemiJoin || la.JoinType == logicalop.AntiLeftOuterSemiJoin { outerSchema = la.Children()[0].Schema() } if len(colGroups) == 0 || outerSchema == nil { @@ -221,12 +221,12 @@ func (la *LogicalApply) ExtractFD() *fd.FDSet { } } switch la.JoinType { - case InnerJoin: - return la.extractFDForInnerJoin(eqCond) - case LeftOuterJoin, RightOuterJoin: - return la.extractFDForOuterJoin(eqCond) - case SemiJoin: - return la.extractFDForSemiJoin(eqCond) + case logicalop.InnerJoin: + return la.ExtractFDForInnerJoin(eqCond) + case logicalop.LeftOuterJoin, logicalop.RightOuterJoin: + return la.ExtractFDForOuterJoin(eqCond) + case logicalop.SemiJoin: + return la.ExtractFDForSemiJoin(eqCond) default: return &fd.FDSet{HashCodeToUniqueID: make(map[string]int)} } @@ -240,7 +240,7 @@ func (la *LogicalApply) ExtractFD() *fd.FDSet { // CanPullUpAgg checks if an apply can pull an aggregation up. func (la *LogicalApply) CanPullUpAgg() bool { - if la.JoinType != InnerJoin && la.JoinType != LeftOuterJoin { + if la.JoinType != logicalop.InnerJoin && la.JoinType != logicalop.LeftOuterJoin { return false } if len(la.EqualConditions)+len(la.LeftConditions)+len(la.RightConditions)+len(la.OtherConditions) > 0 { @@ -280,7 +280,7 @@ func (la *LogicalApply) DeCorColFromEqExpr(expr expression.Expression) expressio } func (la *LogicalApply) getGroupNDVs(colGroups [][]*expression.Column, childStats []*property.StatsInfo) []property.GroupNDV { - if len(colGroups) > 0 && (la.JoinType == LeftOuterSemiJoin || la.JoinType == AntiLeftOuterSemiJoin || la.JoinType == LeftOuterJoin) { + if len(colGroups) > 0 && (la.JoinType == logicalop.LeftOuterSemiJoin || la.JoinType == logicalop.AntiLeftOuterSemiJoin || la.JoinType == logicalop.LeftOuterJoin) { return childStats[0].GroupNDVs } return nil diff --git a/pkg/planner/core/logical_cte.go b/pkg/planner/core/logical_cte.go index 9888eccf8e6be..1e1b2cb091b16 100644 --- a/pkg/planner/core/logical_cte.go +++ b/pkg/planner/core/logical_cte.go @@ -184,7 +184,7 @@ func (p *LogicalCTE) DeriveStats(_ []*property.StatsInfo, selfSchema *expression // Build push-downed predicates. if len(p.Cte.pushDownPredicates) > 0 { newCond := expression.ComposeDNFCondition(p.SCtx().GetExprCtx(), p.Cte.pushDownPredicates...) - newSel := LogicalSelection{Conditions: []expression.Expression{newCond}}.Init(p.SCtx(), p.Cte.seedPartLogicalPlan.QueryBlockOffset()) + newSel := logicalop.LogicalSelection{Conditions: []expression.Expression{newCond}}.Init(p.SCtx(), p.Cte.seedPartLogicalPlan.QueryBlockOffset()) newSel.SetChildren(p.Cte.seedPartLogicalPlan) p.Cte.seedPartLogicalPlan = newSel p.Cte.optFlag |= flagPredicatePushDown diff --git a/pkg/planner/core/logical_datasource.go b/pkg/planner/core/logical_datasource.go index cdaa2401401e4..77dbbcaeda94a 100644 --- a/pkg/planner/core/logical_datasource.go +++ b/pkg/planner/core/logical_datasource.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/cardinality" "github.com/pingcap/tidb/pkg/planner/core/base" + "github.com/pingcap/tidb/pkg/planner/core/constraint" "github.com/pingcap/tidb/pkg/planner/core/cost" "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" ruleutil "github.com/pingcap/tidb/pkg/planner/core/rule/util" @@ -151,12 +152,12 @@ func (ds *DataSource) ExplainInfo() string { // PredicatePushDown implements base.LogicalPlan.<1st> interface. func (ds *DataSource) PredicatePushDown(predicates []expression.Expression, opt *optimizetrace.LogicalOptimizeOp) ([]expression.Expression, base.LogicalPlan) { predicates = expression.PropagateConstant(ds.SCtx().GetExprCtx(), predicates) - predicates = DeleteTrueExprs(ds, predicates) + predicates = constraint.DeleteTrueExprs(ds, predicates) // Add tidb_shard() prefix to the condtion for shard index in some scenarios // TODO: remove it to the place building logical plan predicates = ds.AddPrefix4ShardIndexes(ds.SCtx(), predicates) ds.AllConds = predicates - ds.PushedDownConds, predicates = expression.PushDownExprs(GetPushDownCtx(ds.SCtx()), predicates, kv.UnSpecified) + ds.PushedDownConds, predicates = expression.PushDownExprs(util.GetPushDownCtx(ds.SCtx()), predicates, kv.UnSpecified) appendDataSourcePredicatePushDownTraceStep(ds, opt) return predicates, ds } @@ -476,13 +477,13 @@ func (ds *DataSource) ExtractFD() *fd.FDSet { // handle the datasource conditions (maybe pushed down from upper layer OP) if len(ds.AllConds) != 0 { // extract the not null attributes from selection conditions. - notnullColsUniqueIDs := ExtractNotNullFromConds(ds.AllConds, ds) + notnullColsUniqueIDs := util.ExtractNotNullFromConds(ds.AllConds, ds) // extract the constant cols from selection conditions. - constUniqueIDs := ExtractConstantCols(ds.AllConds, ds.SCtx(), fds) + constUniqueIDs := util.ExtractConstantCols(ds.AllConds, ds.SCtx(), fds) // extract equivalence cols. - equivUniqueIDs := ExtractEquivalenceCols(ds.AllConds, ds.SCtx(), fds) + equivUniqueIDs := util.ExtractEquivalenceCols(ds.AllConds, ds.SCtx(), fds) // apply conditions to FD. fds.MakeNotNull(notnullColsUniqueIDs) diff --git a/pkg/planner/core/logical_plan_builder.go b/pkg/planner/core/logical_plan_builder.go index a62e7574deb75..41ca87fbfd103 100644 --- a/pkg/planner/core/logical_plan_builder.go +++ b/pkg/planner/core/logical_plan_builder.go @@ -367,14 +367,14 @@ func (b *PlanBuilder) buildAggregation(ctx context.Context, p base.LogicalPlan, names = append(names, p.OutputNames()[i]) } var ( - join *LogicalJoin + join *logicalop.LogicalJoin isJoin bool isSelectionJoin bool ) - join, isJoin = p.(*LogicalJoin) - selection, isSelection := p.(*LogicalSelection) + join, isJoin = p.(*logicalop.LogicalJoin) + selection, isSelection := p.(*logicalop.LogicalSelection) if isSelection { - join, isSelectionJoin = selection.Children()[0].(*LogicalJoin) + join, isSelectionJoin = selection.Children()[0].(*logicalop.LogicalJoin) } if (isJoin && join.FullSchema != nil) || (isSelectionJoin && join.FullSchema != nil) { for i, col := range join.FullSchema.Columns { @@ -493,84 +493,6 @@ func (b *PlanBuilder) buildResultSetNode(ctx context.Context, node ast.ResultSet } } -// extractTableAlias returns table alias of the base.LogicalPlan's columns. -// It will return nil when there are multiple table alias, because the alias is only used to check if -// the base.LogicalPlan Match some optimizer hints, and hints are not expected to take effect in this case. -func extractTableAlias(p base.Plan, parentOffset int) *h.HintedTable { - if len(p.OutputNames()) > 0 && p.OutputNames()[0].TblName.L != "" { - firstName := p.OutputNames()[0] - for _, name := range p.OutputNames() { - if name.TblName.L != firstName.TblName.L || - (name.DBName.L != "" && firstName.DBName.L != "" && name.DBName.L != firstName.DBName.L) { // DBName can be nil, see #46160 - return nil - } - } - qbOffset := p.QueryBlockOffset() - var blockAsNames []ast.HintTable - if p := p.SCtx().GetSessionVars().PlannerSelectBlockAsName.Load(); p != nil { - blockAsNames = *p - } - // For sub-queries like `(select * from t) t1`, t1 should belong to its surrounding select block. - if qbOffset != parentOffset && blockAsNames != nil && blockAsNames[qbOffset].TableName.L != "" { - qbOffset = parentOffset - } - dbName := firstName.DBName - if dbName.L == "" { - dbName = model.NewCIStr(p.SCtx().GetSessionVars().CurrentDB) - } - return &h.HintedTable{DBName: dbName, TblName: firstName.TblName, SelectOffset: qbOffset} - } - return nil -} - -func setPreferredJoinTypeFromOneSide(preferJoinType uint, isLeft bool) (resJoinType uint) { - if preferJoinType == 0 { - return - } - if preferJoinType&h.PreferINLJ > 0 { - preferJoinType &= ^h.PreferINLJ - if isLeft { - resJoinType |= h.PreferLeftAsINLJInner - } else { - resJoinType |= h.PreferRightAsINLJInner - } - } - if preferJoinType&h.PreferINLHJ > 0 { - preferJoinType &= ^h.PreferINLHJ - if isLeft { - resJoinType |= h.PreferLeftAsINLHJInner - } else { - resJoinType |= h.PreferRightAsINLHJInner - } - } - if preferJoinType&h.PreferINLMJ > 0 { - preferJoinType &= ^h.PreferINLMJ - if isLeft { - resJoinType |= h.PreferLeftAsINLMJInner - } else { - resJoinType |= h.PreferRightAsINLMJInner - } - } - if preferJoinType&h.PreferHJBuild > 0 { - preferJoinType &= ^h.PreferHJBuild - if isLeft { - resJoinType |= h.PreferLeftAsHJBuild - } else { - resJoinType |= h.PreferRightAsHJBuild - } - } - if preferJoinType&h.PreferHJProbe > 0 { - preferJoinType &= ^h.PreferHJProbe - if isLeft { - resJoinType |= h.PreferLeftAsHJProbe - } else { - resJoinType |= h.PreferRightAsHJProbe - } - } - resJoinType |= preferJoinType - return -} - func (ds *DataSource) setPreferredStoreType(hintInfo *h.PlanHints) { if hintInfo == nil { return @@ -658,7 +580,7 @@ func (b *PlanBuilder) buildJoin(ctx context.Context, joinNode *ast.Join) (base.L handleMap2 := b.handleHelper.popMap() b.handleHelper.mergeAndPush(handleMap1, handleMap2) - joinPlan := LogicalJoin{StraightJoin: joinNode.StraightJoin || b.inStraightJoin}.Init(b.ctx, b.getSelectOffset()) + joinPlan := logicalop.LogicalJoin{StraightJoin: joinNode.StraightJoin || b.inStraightJoin}.Init(b.ctx, b.getSelectOffset()) joinPlan.SetChildren(leftPlan, rightPlan) joinPlan.SetSchema(expression.MergeSchema(leftPlan.Schema(), rightPlan.Schema())) joinPlan.SetOutputNames(make([]*types.FieldName, leftPlan.Schema().Len()+rightPlan.Schema().Len())) @@ -670,15 +592,15 @@ func (b *PlanBuilder) buildJoin(ctx context.Context, joinNode *ast.Join) (base.L case ast.LeftJoin: // left outer join need to be checked elimination b.optFlag = b.optFlag | flagEliminateOuterJoin - joinPlan.JoinType = LeftOuterJoin + joinPlan.JoinType = logicalop.LeftOuterJoin util.ResetNotNullFlag(joinPlan.Schema(), leftPlan.Schema().Len(), joinPlan.Schema().Len()) case ast.RightJoin: // right outer join need to be checked elimination b.optFlag = b.optFlag | flagEliminateOuterJoin - joinPlan.JoinType = RightOuterJoin + joinPlan.JoinType = logicalop.RightOuterJoin util.ResetNotNullFlag(joinPlan.Schema(), 0, leftPlan.Schema().Len()) default: - joinPlan.JoinType = InnerJoin + joinPlan.JoinType = logicalop.InnerJoin } // Merge sub-plan's FullSchema into this join plan. @@ -687,14 +609,14 @@ func (b *PlanBuilder) buildJoin(ctx context.Context, joinNode *ast.Join) (base.L lFullSchema, rFullSchema *expression.Schema lFullNames, rFullNames types.NameSlice ) - if left, ok := leftPlan.(*LogicalJoin); ok && left.FullSchema != nil { + if left, ok := leftPlan.(*logicalop.LogicalJoin); ok && left.FullSchema != nil { lFullSchema = left.FullSchema lFullNames = left.FullNames } else { lFullSchema = leftPlan.Schema() lFullNames = leftPlan.OutputNames() } - if right, ok := rightPlan.(*LogicalJoin); ok && right.FullSchema != nil { + if right, ok := rightPlan.(*logicalop.LogicalJoin); ok && right.FullSchema != nil { rFullSchema = right.FullSchema rFullNames = right.FullNames } else { @@ -756,13 +678,13 @@ func (b *PlanBuilder) buildJoin(ctx context.Context, joinNode *ast.Join) (base.L onCondition := expression.SplitCNFItems(onExpr) // Keep these expressions as a LogicalSelection upon the inner join, in order to apply // possible decorrelate optimizations. The ON clause is actually treated as a WHERE clause now. - if joinPlan.JoinType == InnerJoin { - sel := LogicalSelection{Conditions: onCondition}.Init(b.ctx, b.getSelectOffset()) + if joinPlan.JoinType == logicalop.InnerJoin { + sel := logicalop.LogicalSelection{Conditions: onCondition}.Init(b.ctx, b.getSelectOffset()) sel.SetChildren(joinPlan) return sel, nil } joinPlan.AttachOnConds(onCondition) - } else if joinPlan.JoinType == InnerJoin { + } else if joinPlan.JoinType == logicalop.InnerJoin { // If a inner join without "ON" or "USING" clause, it's a cartesian // product over the join tables. joinPlan.CartesianJoin = true @@ -779,7 +701,7 @@ func (b *PlanBuilder) buildJoin(ctx context.Context, joinNode *ast.Join) (base.L // appears in "leftPlan". // 2. the rest columns in "leftPlan", in the order they appears in "leftPlan". // 3. the rest columns in "rightPlan", in the order they appears in "rightPlan". -func (b *PlanBuilder) buildUsingClause(p *LogicalJoin, leftPlan, rightPlan base.LogicalPlan, join *ast.Join) error { +func (b *PlanBuilder) buildUsingClause(p *logicalop.LogicalJoin, leftPlan, rightPlan base.LogicalPlan, join *ast.Join) error { filter := make(map[string]bool, len(join.Using)) for _, col := range join.Using { filter[col.Name.L] = true @@ -803,7 +725,7 @@ func (b *PlanBuilder) buildUsingClause(p *LogicalJoin, leftPlan, rightPlan base. // All the common columns // Every column in the first (left) table that is not a common column // Every column in the second (right) table that is not a common column -func (b *PlanBuilder) buildNaturalJoin(p *LogicalJoin, leftPlan, rightPlan base.LogicalPlan, join *ast.Join) error { +func (b *PlanBuilder) buildNaturalJoin(p *logicalop.LogicalJoin, leftPlan, rightPlan base.LogicalPlan, join *ast.Join) error { err := b.coalesceCommonColumns(p, leftPlan, rightPlan, join.Tp, nil) if err != nil { return err @@ -817,7 +739,7 @@ func (b *PlanBuilder) buildNaturalJoin(p *LogicalJoin, leftPlan, rightPlan base. } // coalesceCommonColumns is used by buildUsingClause and buildNaturalJoin. The filter is used by buildUsingClause. -func (b *PlanBuilder) coalesceCommonColumns(p *LogicalJoin, leftPlan, rightPlan base.LogicalPlan, joinTp ast.JoinType, filter map[string]bool) error { +func (b *PlanBuilder) coalesceCommonColumns(p *logicalop.LogicalJoin, leftPlan, rightPlan base.LogicalPlan, joinTp ast.JoinType, filter map[string]bool) error { lsc := leftPlan.Schema().Clone() rsc := rightPlan.Schema().Clone() if joinTp == ast.LeftJoin { @@ -996,7 +918,7 @@ func (b *PlanBuilder) buildSelection(ctx context.Context, p base.LogicalPlan, wh conditions := splitWhere(where) expressions := make([]expression.Expression, 0, len(conditions)) - selection := LogicalSelection{}.Init(b.ctx, b.getSelectOffset()) + selection := logicalop.LogicalSelection{}.Init(b.ctx, b.getSelectOffset()) for _, cond := range conditions { expr, np, err := b.rewrite(ctx, cond, p, aggMapper, false) if err != nil { @@ -1278,9 +1200,9 @@ func (b *PlanBuilder) preprocessUserVarTypes(ctx context.Context, p base.Logical // underlying join. func findColFromNaturalUsingJoin(p base.LogicalPlan, col *expression.Column) (name *types.FieldName) { switch x := p.(type) { - case *logicalop.LogicalLimit, *LogicalSelection, *logicalop.LogicalTopN, *logicalop.LogicalSort, *logicalop.LogicalMaxOneRow: + case *logicalop.LogicalLimit, *logicalop.LogicalSelection, *logicalop.LogicalTopN, *logicalop.LogicalSort, *logicalop.LogicalMaxOneRow: return findColFromNaturalUsingJoin(p.Children()[0], col) - case *LogicalJoin: + case *logicalop.LogicalJoin: if x.FullSchema != nil { idx := x.FullSchema.ColumnIndex(col) return x.FullNames[idx] @@ -1800,14 +1722,14 @@ func (b *PlanBuilder) buildSetOpr(ctx context.Context, setOpr *ast.SetOprStmt) ( func (b *PlanBuilder) buildSemiJoinForSetOperator( leftOriginPlan base.LogicalPlan, rightPlan base.LogicalPlan, - joinType JoinType) (leftPlan base.LogicalPlan, err error) { + joinType logicalop.JoinType) (leftPlan base.LogicalPlan, err error) { leftPlan, err = b.buildDistinct(leftOriginPlan, leftOriginPlan.Schema().Len()) if err != nil { return nil, err } b.optFlag |= flagConvertOuterToInnerJoin - joinPlan := LogicalJoin{JoinType: joinType}.Init(b.ctx, b.getSelectOffset()) + joinPlan := logicalop.LogicalJoin{JoinType: joinType}.Init(b.ctx, b.getSelectOffset()) joinPlan.SetChildren(leftPlan, rightPlan) joinPlan.SetSchema(leftPlan.Schema()) joinPlan.SetOutputNames(make([]*types.FieldName, leftPlan.Schema().Len())) @@ -1873,7 +1795,7 @@ func (b *PlanBuilder) buildIntersect(ctx context.Context, selects []ast.Node) (b if rightPlan.Schema().Len() != columnNums { return nil, nil, plannererrors.ErrWrongNumberOfColumnsInSelect.GenWithStackByArgs() } - leftPlan, err = b.buildSemiJoinForSetOperator(leftPlan, rightPlan, SemiJoin) + leftPlan, err = b.buildSemiJoinForSetOperator(leftPlan, rightPlan, logicalop.SemiJoin) if err != nil { return nil, nil, err } @@ -1897,7 +1819,7 @@ func (b *PlanBuilder) buildExcept(ctx context.Context, selects []base.LogicalPla if err != nil { return nil, err } - leftPlan, err = b.buildSemiJoinForSetOperator(leftPlan, rightPlan, AntiSemiJoin) + leftPlan, err = b.buildSemiJoinForSetOperator(leftPlan, rightPlan, logicalop.AntiSemiJoin) if err != nil { return nil, err } @@ -2287,9 +2209,9 @@ func (a *havingWindowAndOrderbyExprResolver) resolveFromPlan(v *ast.ColumnNameEx // schema of selection will be `[t1.a]`, thus we need to recursively // retrieve the `t2.a` from the underlying join. switch x := p.(type) { - case *logicalop.LogicalLimit, *LogicalSelection, *logicalop.LogicalTopN, *logicalop.LogicalSort, *logicalop.LogicalMaxOneRow: + case *logicalop.LogicalLimit, *logicalop.LogicalSelection, *logicalop.LogicalTopN, *logicalop.LogicalSort, *logicalop.LogicalMaxOneRow: return a.resolveFromPlan(v, p.Children()[0], resolveFieldsFirst) - case *LogicalJoin: + case *logicalop.LogicalJoin: if len(x.FullNames) != 0 { idx, err = expression.FindFieldName(x.FullNames, v.Name) schemaCols, outputNames = x.FullSchema.Columns, x.FullNames @@ -3560,7 +3482,7 @@ func (b *PlanBuilder) resolveGbyExprs(ctx context.Context, p base.LogicalPlan, g } func (*PlanBuilder) unfoldWildStar(p base.LogicalPlan, selectFields []*ast.SelectField) (resultList []*ast.SelectField, err error) { - join, isJoin := p.(*LogicalJoin) + join, isJoin := p.(*logicalop.LogicalJoin) for i, field := range selectFields { if field.WildCard == nil { resultList = append(resultList, field) @@ -5129,16 +5051,16 @@ func (b *PlanBuilder) buildProjUponView(_ context.Context, dbName model.CIStr, t // buildApplyWithJoinType builds apply plan with outerPlan and innerPlan, which apply join with particular join type for // every row from outerPlan and the whole innerPlan. -func (b *PlanBuilder) buildApplyWithJoinType(outerPlan, innerPlan base.LogicalPlan, tp JoinType, markNoDecorrelate bool) base.LogicalPlan { +func (b *PlanBuilder) buildApplyWithJoinType(outerPlan, innerPlan base.LogicalPlan, tp logicalop.JoinType, markNoDecorrelate bool) base.LogicalPlan { b.optFlag = b.optFlag | flagPredicatePushDown | flagBuildKeyInfo | flagDecorrelate | flagConvertOuterToInnerJoin - ap := LogicalApply{LogicalJoin: LogicalJoin{JoinType: tp}, NoDecorrelate: markNoDecorrelate}.Init(b.ctx, b.getSelectOffset()) + ap := LogicalApply{LogicalJoin: logicalop.LogicalJoin{JoinType: tp}, NoDecorrelate: markNoDecorrelate}.Init(b.ctx, b.getSelectOffset()) ap.SetChildren(outerPlan, innerPlan) ap.SetOutputNames(make([]*types.FieldName, outerPlan.Schema().Len()+innerPlan.Schema().Len())) copy(ap.OutputNames(), outerPlan.OutputNames()) ap.SetSchema(expression.MergeSchema(outerPlan.Schema(), innerPlan.Schema())) setIsInApplyForCTE(innerPlan, ap.Schema()) // Note that, tp can only be LeftOuterJoin or InnerJoin, so we don't consider other outer joins. - if tp == LeftOuterJoin { + if tp == logicalop.LeftOuterJoin { b.optFlag = b.optFlag | flagEliminateOuterJoin util.ResetNotNullFlag(ap.Schema(), outerPlan.Schema().Len(), ap.Schema().Len()) } @@ -5193,9 +5115,9 @@ func (b *PlanBuilder) buildMaxOneRow(p base.LogicalPlan) base.LogicalPlan { return maxOneRow } -func (b *PlanBuilder) buildSemiJoin(outerPlan, innerPlan base.LogicalPlan, onCondition []expression.Expression, asScalar, not, forceRewrite bool) (*LogicalJoin, error) { +func (b *PlanBuilder) buildSemiJoin(outerPlan, innerPlan base.LogicalPlan, onCondition []expression.Expression, asScalar, not, forceRewrite bool) (*logicalop.LogicalJoin, error) { b.optFlag |= flagConvertOuterToInnerJoin - joinPlan := LogicalJoin{}.Init(b.ctx, b.getSelectOffset()) + joinPlan := logicalop.LogicalJoin{}.Init(b.ctx, b.getSelectOffset()) for i, expr := range onCondition { onCondition[i] = expr.Decorrelate(outerPlan.Schema()) } @@ -5212,16 +5134,16 @@ func (b *PlanBuilder) buildSemiJoin(outerPlan, innerPlan base.LogicalPlan, onCon joinPlan.SetOutputNames(append(joinPlan.OutputNames(), types.EmptyName)) joinPlan.SetSchema(newSchema) if not { - joinPlan.JoinType = AntiLeftOuterSemiJoin + joinPlan.JoinType = logicalop.AntiLeftOuterSemiJoin } else { - joinPlan.JoinType = LeftOuterSemiJoin + joinPlan.JoinType = logicalop.LeftOuterSemiJoin } } else { joinPlan.SetSchema(outerPlan.Schema().Clone()) if not { - joinPlan.JoinType = AntiSemiJoin + joinPlan.JoinType = logicalop.AntiSemiJoin } else { - joinPlan.JoinType = SemiJoin + joinPlan.JoinType = logicalop.SemiJoin } } // Apply forces to choose hash join currently, so don't worry the hints will take effect if the semi join is in one apply. @@ -6911,47 +6833,6 @@ func getInnerFromParenthesesAndUnaryPlus(expr ast.ExprNode) ast.ExprNode { return expr } -// containDifferentJoinTypes checks whether `PreferJoinType` contains different -// join types. -func containDifferentJoinTypes(preferJoinType uint) bool { - preferJoinType &= ^h.PreferNoHashJoin - preferJoinType &= ^h.PreferNoMergeJoin - preferJoinType &= ^h.PreferNoIndexJoin - preferJoinType &= ^h.PreferNoIndexHashJoin - preferJoinType &= ^h.PreferNoIndexMergeJoin - - inlMask := h.PreferRightAsINLJInner ^ h.PreferLeftAsINLJInner - inlhjMask := h.PreferRightAsINLHJInner ^ h.PreferLeftAsINLHJInner - inlmjMask := h.PreferRightAsINLMJInner ^ h.PreferLeftAsINLMJInner - hjRightBuildMask := h.PreferRightAsHJBuild ^ h.PreferLeftAsHJProbe - hjLeftBuildMask := h.PreferLeftAsHJBuild ^ h.PreferRightAsHJProbe - - mppMask := h.PreferShuffleJoin ^ h.PreferBCJoin - mask := inlMask ^ inlhjMask ^ inlmjMask ^ hjRightBuildMask ^ hjLeftBuildMask - onesCount := bits.OnesCount(preferJoinType & ^mask & ^mppMask) - if onesCount > 1 || onesCount == 1 && preferJoinType&mask > 0 { - return true - } - - cnt := 0 - if preferJoinType&inlMask > 0 { - cnt++ - } - if preferJoinType&inlhjMask > 0 { - cnt++ - } - if preferJoinType&inlmjMask > 0 { - cnt++ - } - if preferJoinType&hjLeftBuildMask > 0 { - cnt++ - } - if preferJoinType&hjRightBuildMask > 0 { - cnt++ - } - return cnt > 1 -} - func hasMPPJoinHints(preferJoinType uint) bool { return (preferJoinType&h.PreferBCJoin > 0) || (preferJoinType&h.PreferShuffleJoin > 0) } diff --git a/pkg/planner/core/logical_plans.go b/pkg/planner/core/logical_plans.go index a0d3b1fa368c1..a1ddecb4e62ca 100644 --- a/pkg/planner/core/logical_plans.go +++ b/pkg/planner/core/logical_plans.go @@ -15,19 +15,15 @@ package core import ( - "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" - fd "github.com/pingcap/tidb/pkg/planner/funcdep" - "github.com/pingcap/tidb/pkg/planner/util" - "github.com/pingcap/tidb/pkg/util/intset" ) var ( - _ base.LogicalPlan = &LogicalJoin{} + _ base.LogicalPlan = &logicalop.LogicalJoin{} _ base.LogicalPlan = &LogicalAggregation{} _ base.LogicalPlan = &logicalop.LogicalProjection{} - _ base.LogicalPlan = &LogicalSelection{} + _ base.LogicalPlan = &logicalop.LogicalSelection{} _ base.LogicalPlan = &LogicalApply{} _ base.LogicalPlan = &logicalop.LogicalMaxOneRow{} _ base.LogicalPlan = &logicalop.LogicalTableDual{} @@ -49,101 +45,3 @@ var ( _ base.LogicalPlan = &logicalop.LogicalCTETable{} _ base.LogicalPlan = &logicalop.LogicalSequence{} ) - -// ExtractNotNullFromConds extracts not-null columns from conditions. -func ExtractNotNullFromConds(conditions []expression.Expression, p base.LogicalPlan) intset.FastIntSet { - // extract the column NOT NULL rejection characteristic from selection condition. - // CNF considered only, DNF doesn't have its meanings (cause that condition's eval may don't take effect) - // - // Take this case: select * from t where (a = 1) and (b is null): - // - // If we wanna where phrase eval to true, two pre-condition: {a=1} and {b is null} both need to be true. - // Hence, we assert that: - // - // 1: `a` must not be null since `NULL = 1` is evaluated as NULL. - // 2: `b` must be null since only `NULL is NULL` is evaluated as true. - // - // As a result, `a` will be extracted as not-null column to abound the FDSet. - notnullColsUniqueIDs := intset.NewFastIntSet() - for _, condition := range conditions { - var cols []*expression.Column - cols = expression.ExtractColumnsFromExpressions(cols, []expression.Expression{condition}, nil) - if util.IsNullRejected(p.SCtx(), p.Schema(), condition) { - for _, col := range cols { - notnullColsUniqueIDs.Insert(int(col.UniqueID)) - } - } - } - return notnullColsUniqueIDs -} - -// ExtractConstantCols extracts constant columns from conditions. -func ExtractConstantCols(conditions []expression.Expression, sctx base.PlanContext, fds *fd.FDSet) intset.FastIntSet { - // extract constant cols - // eg: where a=1 and b is null and (1+c)=5. - // TODO: Some columns can only be determined to be constant from multiple constraints (e.g. x <= 1 AND x >= 1) - var ( - constObjs []expression.Expression - constUniqueIDs = intset.NewFastIntSet() - ) - constObjs = expression.ExtractConstantEqColumnsOrScalar(sctx.GetExprCtx(), constObjs, conditions) - for _, constObj := range constObjs { - switch x := constObj.(type) { - case *expression.Column: - constUniqueIDs.Insert(int(x.UniqueID)) - case *expression.ScalarFunction: - hashCode := string(x.HashCode()) - if uniqueID, ok := fds.IsHashCodeRegistered(hashCode); ok { - constUniqueIDs.Insert(uniqueID) - } else { - scalarUniqueID := int(sctx.GetSessionVars().AllocPlanColumnID()) - fds.RegisterUniqueID(string(x.HashCode()), scalarUniqueID) - constUniqueIDs.Insert(scalarUniqueID) - } - } - } - return constUniqueIDs -} - -// ExtractEquivalenceCols extracts equivalence columns from conditions. -func ExtractEquivalenceCols(conditions []expression.Expression, sctx base.PlanContext, fds *fd.FDSet) [][]intset.FastIntSet { - var equivObjsPair [][]expression.Expression - equivObjsPair = expression.ExtractEquivalenceColumns(equivObjsPair, conditions) - equivUniqueIDs := make([][]intset.FastIntSet, 0, len(equivObjsPair)) - for _, equivObjPair := range equivObjsPair { - // lhs of equivalence. - var ( - lhsUniqueID int - rhsUniqueID int - ) - switch x := equivObjPair[0].(type) { - case *expression.Column: - lhsUniqueID = int(x.UniqueID) - case *expression.ScalarFunction: - hashCode := string(x.HashCode()) - if uniqueID, ok := fds.IsHashCodeRegistered(hashCode); ok { - lhsUniqueID = uniqueID - } else { - scalarUniqueID := int(sctx.GetSessionVars().AllocPlanColumnID()) - fds.RegisterUniqueID(string(x.HashCode()), scalarUniqueID) - lhsUniqueID = scalarUniqueID - } - } - // rhs of equivalence. - switch x := equivObjPair[1].(type) { - case *expression.Column: - rhsUniqueID = int(x.UniqueID) - case *expression.ScalarFunction: - hashCode := string(x.HashCode()) - if uniqueID, ok := fds.IsHashCodeRegistered(hashCode); ok { - rhsUniqueID = uniqueID - } else { - scalarUniqueID := int(sctx.GetSessionVars().AllocPlanColumnID()) - fds.RegisterUniqueID(string(x.HashCode()), scalarUniqueID) - rhsUniqueID = scalarUniqueID - } - } - equivUniqueIDs = append(equivUniqueIDs, []intset.FastIntSet{intset.NewFastIntSet(lhsUniqueID), intset.NewFastIntSet(rhsUniqueID)}) - } - return equivUniqueIDs -} diff --git a/pkg/planner/core/logical_plans_test.go b/pkg/planner/core/logical_plans_test.go index ffa5fdac91440..2820f4825823f 100644 --- a/pkg/planner/core/logical_plans_test.go +++ b/pkg/planner/core/logical_plans_test.go @@ -163,7 +163,7 @@ func TestImplicitCastNotNullFlag(t *testing.T) { p, err = logicalOptimize(context.TODO(), flagPredicatePushDown|flagJoinReOrder|flagPrunColumns|flagEliminateProjection, p.(base.LogicalPlan)) require.NoError(t, err) // AggFuncs[0] is count; AggFuncs[1] is bit_and, args[0] is return type of the implicit cast - castNotNullFlag := (p.(*logicalop.LogicalProjection).Children()[0].(*LogicalSelection).Children()[0].(*LogicalAggregation).AggFuncs[1].Args[0].GetType(s.ctx.GetExprCtx().GetEvalCtx()).GetFlag()) & mysql.NotNullFlag + castNotNullFlag := (p.(*logicalop.LogicalProjection).Children()[0].(*logicalop.LogicalSelection).Children()[0].(*LogicalAggregation).AggFuncs[1].Args[0].GetType(s.ctx.GetExprCtx().GetEvalCtx()).GetFlag()) & mysql.NotNullFlag var nullableFlag uint = 0 require.Equal(t, nullableFlag, castNotNullFlag) } @@ -181,8 +181,8 @@ func TestEliminateProjectionUnderUnion(t *testing.T) { p, err = logicalOptimize(context.TODO(), flagPredicatePushDown|flagJoinReOrder|flagPrunColumns|flagEliminateProjection, p.(base.LogicalPlan)) require.NoError(t, err) // after folding constants, the null flag should keep the same with the old one's (i.e., the schema's). - schemaNullFlag := p.(*logicalop.LogicalProjection).Children()[0].(*LogicalJoin).Children()[1].Children()[1].(*logicalop.LogicalProjection).Schema().Columns[0].RetType.GetFlag() & mysql.NotNullFlag - exprNullFlag := p.(*logicalop.LogicalProjection).Children()[0].(*LogicalJoin).Children()[1].Children()[1].(*logicalop.LogicalProjection).Exprs[0].GetType(s.ctx.GetExprCtx().GetEvalCtx()).GetFlag() & mysql.NotNullFlag + schemaNullFlag := p.(*logicalop.LogicalProjection).Children()[0].(*logicalop.LogicalJoin).Children()[1].Children()[1].(*logicalop.LogicalProjection).Schema().Columns[0].RetType.GetFlag() & mysql.NotNullFlag + exprNullFlag := p.(*logicalop.LogicalProjection).Children()[0].(*logicalop.LogicalJoin).Children()[1].Children()[1].(*logicalop.LogicalProjection).Exprs[0].GetType(s.ctx.GetExprCtx().GetEvalCtx()).GetFlag() & mysql.NotNullFlag require.Equal(t, exprNullFlag, schemaNullFlag) } @@ -210,7 +210,7 @@ func TestJoinPredicatePushDown(t *testing.T) { require.NoError(t, err, comment) proj, ok := p.(*logicalop.LogicalProjection) require.True(t, ok, comment) - join, ok := proj.Children()[0].(*LogicalJoin) + join, ok := proj.Children()[0].(*logicalop.LogicalJoin) require.True(t, ok, comment) leftPlan, ok := join.Children()[0].(*DataSource) require.True(t, ok, comment) @@ -251,14 +251,14 @@ func TestOuterWherePredicatePushDown(t *testing.T) { require.NoError(t, err, comment) proj, ok := p.(*logicalop.LogicalProjection) require.True(t, ok, comment) - selection, ok := proj.Children()[0].(*LogicalSelection) + selection, ok := proj.Children()[0].(*logicalop.LogicalSelection) require.True(t, ok, comment) selCond := expression.StringifyExpressionsWithCtx(ectx, selection.Conditions) testdata.OnRecord(func() { output[i].Sel = selCond }) require.Equal(t, output[i].Sel, selCond, comment) - join, ok := selection.Children()[0].(*LogicalJoin) + join, ok := selection.Children()[0].(*logicalop.LogicalJoin) require.True(t, ok, comment) leftPlan, ok := join.Children()[0].(*DataSource) require.True(t, ok, comment) @@ -300,9 +300,9 @@ func TestSimplifyOuterJoin(t *testing.T) { output[i].Best = planString }) require.Equal(t, output[i].Best, planString, comment) - join, ok := p.(base.LogicalPlan).Children()[0].(*LogicalJoin) + join, ok := p.(base.LogicalPlan).Children()[0].(*logicalop.LogicalJoin) if !ok { - join, ok = p.(base.LogicalPlan).Children()[0].Children()[0].(*LogicalJoin) + join, ok = p.(base.LogicalPlan).Children()[0].Children()[0].(*logicalop.LogicalJoin) require.True(t, ok, comment) } testdata.OnRecord(func() { @@ -337,7 +337,7 @@ func TestAntiSemiJoinConstFalse(t *testing.T) { p, err = logicalOptimize(context.TODO(), flagDecorrelate|flagPredicatePushDown|flagPrunColumns|flagPrunColumnsAgain, p.(base.LogicalPlan)) require.NoError(t, err, comment) require.Equal(t, ca.best, ToString(p), comment) - join, _ := p.(base.LogicalPlan).Children()[0].(*LogicalJoin) + join, _ := p.(base.LogicalPlan).Children()[0].(*logicalop.LogicalJoin) require.Equal(t, ca.joinType, join.JoinType.String(), comment) } } @@ -369,7 +369,7 @@ func TestDeriveNotNullConds(t *testing.T) { output[i].Plan = ToString(p) }) require.Equal(t, output[i].Plan, ToString(p), comment) - join := p.(base.LogicalPlan).Children()[0].(*LogicalJoin) + join := p.(base.LogicalPlan).Children()[0].(*logicalop.LogicalJoin) left := join.Children()[0].(*DataSource) right := join.Children()[1].(*DataSource) leftConds := expression.StringifyExpressionsWithCtx(ectx, left.PushedDownConds) @@ -479,9 +479,9 @@ func TestDupRandJoinCondsPushDown(t *testing.T) { require.NoError(t, err, comment) proj, ok := p.(*logicalop.LogicalProjection) require.True(t, ok, comment) - join, ok := proj.Children()[0].(*LogicalJoin) + join, ok := proj.Children()[0].(*logicalop.LogicalJoin) require.True(t, ok, comment) - leftPlan, ok := join.Children()[0].(*LogicalSelection) + leftPlan, ok := join.Children()[0].(*logicalop.LogicalSelection) require.True(t, ok, comment) leftCond := expression.StringifyExpressionsWithCtx(s.ctx.GetExprCtx().GetEvalCtx(), leftPlan.Conditions) // Condition with mutable function cannot be de-duplicated when push down join conds. @@ -764,7 +764,7 @@ func TestCS3389(t *testing.T) { agg, isAgg := child.(*LogicalAggregation) require.True(t, isAgg) child = agg.Children()[0] - _, isJoin := child.(*LogicalJoin) + _, isJoin := child.(*logicalop.LogicalJoin) require.True(t, isJoin) } @@ -2116,7 +2116,7 @@ func TestConflictedJoinTypeHints(t *testing.T) { require.NoError(t, err) proj, ok := p.(*logicalop.LogicalProjection) require.True(t, ok) - join, ok := proj.Children()[0].(*LogicalJoin) + join, ok := proj.Children()[0].(*logicalop.LogicalJoin) require.True(t, ok) require.Nil(t, join.HintInfo) require.Equal(t, uint(0), join.PreferJoinType) @@ -2143,10 +2143,10 @@ func TestSimplyOuterJoinWithOnlyOuterExpr(t *testing.T) { require.NoError(t, err) proj, ok := p.(*logicalop.LogicalProjection) require.True(t, ok) - join, ok := proj.Children()[0].(*LogicalJoin) + join, ok := proj.Children()[0].(*logicalop.LogicalJoin) require.True(t, ok) // previous wrong JoinType is InnerJoin - require.Equal(t, RightOuterJoin, join.JoinType) + require.Equal(t, logicalop.RightOuterJoin, join.JoinType) } func TestResolvingCorrelatedAggregate(t *testing.T) { diff --git a/pkg/planner/core/operator/logicalop/BUILD.bazel b/pkg/planner/core/operator/logicalop/BUILD.bazel index f68f7b4be2f1a..3a1d6c7f4b030 100644 --- a/pkg/planner/core/operator/logicalop/BUILD.bazel +++ b/pkg/planner/core/operator/logicalop/BUILD.bazel @@ -5,12 +5,14 @@ go_library( srcs = [ "base_logical_plan.go", "logical_cte_table.go", + "logical_join.go", "logical_limit.go", "logical_lock.go", "logical_max_one_row.go", "logical_mem_table.go", "logical_projection.go", "logical_schema_producer.go", + "logical_selection.go", "logical_sequence.go", "logical_show.go", "logical_show_ddl_jobs.go", @@ -33,6 +35,8 @@ go_library( "//pkg/parser/mysql", "//pkg/planner/cardinality", "//pkg/planner/core/base", + "//pkg/planner/core/constraint", + "//pkg/planner/core/cost", "//pkg/planner/core/operator/baseimpl", "//pkg/planner/core/rule/util", "//pkg/planner/funcdep", @@ -45,6 +49,7 @@ go_library( "//pkg/statistics", "//pkg/types", "//pkg/util/dbterror/plannererrors", + "//pkg/util/hint", "//pkg/util/intset", "//pkg/util/plancodec", "//pkg/util/size", diff --git a/pkg/planner/core/logical_join.go b/pkg/planner/core/operator/logicalop/logical_join.go similarity index 84% rename from pkg/planner/core/logical_join.go rename to pkg/planner/core/operator/logicalop/logical_join.go index 99dd7b4b97aa5..2f75c62e38f3e 100644 --- a/pkg/planner/core/logical_join.go +++ b/pkg/planner/core/operator/logicalop/logical_join.go @@ -12,22 +12,21 @@ // See the License for the specific language governing permissions and // limitations under the License. -package core +package logicalop import ( "bytes" "fmt" "math" + "math/bits" - "github.com/pingcap/failpoint" + "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/expression" - "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/cardinality" "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/core/cost" - "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" ruleutil "github.com/pingcap/tidb/pkg/planner/core/rule/util" "github.com/pingcap/tidb/pkg/planner/funcdep" "github.com/pingcap/tidb/pkg/planner/property" @@ -94,7 +93,7 @@ func (tp JoinType) String() string { // LogicalJoin is the logical join plan. type LogicalJoin struct { - logicalop.LogicalSchemaProducer + LogicalSchemaProducer JoinType JoinType Reordered bool @@ -147,7 +146,7 @@ type LogicalJoin struct { // Init initializes LogicalJoin. func (p LogicalJoin) Init(ctx base.PlanContext, offset int) *LogicalJoin { - p.BaseLogicalPlan = logicalop.NewBaseLogicalPlan(ctx, plancodec.TypeJoin, &p, offset) + p.BaseLogicalPlan = NewBaseLogicalPlan(ctx, plancodec.TypeJoin, &p, offset) return &p } @@ -206,7 +205,7 @@ func (p *LogicalJoin) PredicatePushDown(predicates []expression.Expression, opt predicates = p.outerJoinPropConst(predicates) dual := Conds2TableDual(p, predicates) if dual != nil { - appendTableDualTraceStep(p, dual, predicates, opt) + AppendTableDualTraceStep(p, dual, predicates, opt) return ret, dual } // Handle where conditions @@ -225,7 +224,7 @@ func (p *LogicalJoin) PredicatePushDown(predicates []expression.Expression, opt predicates = p.outerJoinPropConst(predicates) dual := Conds2TableDual(p, predicates) if dual != nil { - appendTableDualTraceStep(p, dual, predicates, opt) + AppendTableDualTraceStep(p, dual, predicates, opt) return ret, dual } // Handle where conditions @@ -252,7 +251,7 @@ func (p *LogicalJoin) PredicatePushDown(predicates []expression.Expression, opt // Return table dual when filter is constant false or null. dual := Conds2TableDual(p, tempCond) if dual != nil { - appendTableDualTraceStep(p, dual, tempCond, opt) + AppendTableDualTraceStep(p, dual, tempCond, opt) return ret, dual } equalCond, leftPushCond, rightPushCond, otherCond = p.extractOnCondition(tempCond, true, true) @@ -267,7 +266,7 @@ func (p *LogicalJoin) PredicatePushDown(predicates []expression.Expression, opt // Return table dual when filter is constant false or null. dual := Conds2TableDual(p, predicates) if dual != nil { - appendTableDualTraceStep(p, dual, predicates, opt) + AppendTableDualTraceStep(p, dual, predicates, opt) return ret, dual } // `predicates` should only contain left conditions or constant filters. @@ -294,7 +293,7 @@ func (p *LogicalJoin) PredicatePushDown(predicates []expression.Expression, opt // PruneColumns implements the base.LogicalPlan.<2nd> interface. func (p *LogicalJoin) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error) { - leftCols, rightCols := p.extractUsedCols(parentUsedCols) + leftCols, rightCols := p.ExtractUsedCols(parentUsedCols) var err error p.Children()[0], err = p.Children()[0].PruneColumns(leftCols, opt) @@ -307,7 +306,7 @@ func (p *LogicalJoin) PruneColumns(parentUsedCols []*expression.Column, opt *opt return nil, err } - p.mergeSchema() + p.MergeSchema() if p.JoinType == LeftOuterSemiJoin || p.JoinType == AntiLeftOuterSemiJoin { joinCol := p.Schema().Columns[len(p.Schema().Columns)-1] parentUsedCols = append(parentUsedCols, joinCol) @@ -366,9 +365,9 @@ func (p *LogicalJoin) BuildKeyInfo(selfSchema *expression.Schema, childSchema [] // PushDownTopN implements the base.LogicalPlan.<5th> interface. func (p *LogicalJoin) PushDownTopN(topNLogicalPlan base.LogicalPlan, opt *optimizetrace.LogicalOptimizeOp) base.LogicalPlan { - var topN *logicalop.LogicalTopN + var topN *LogicalTopN if topNLogicalPlan != nil { - topN = topNLogicalPlan.(*logicalop.LogicalTopN) + topN = topNLogicalPlan.(*LogicalTopN) } switch p.JoinType { case LeftOuterJoin, LeftOuterSemiJoin, AntiLeftOuterSemiJoin: @@ -600,96 +599,7 @@ func (p *LogicalJoin) PreparePossibleProperties(_ *expression.Schema, childrenPr // If the hint is not matched, it will get other candidates. // If the hint is not figured, we will pick all candidates. func (p *LogicalJoin) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) { - failpoint.Inject("MockOnlyEnableIndexHashJoin", func(val failpoint.Value) { - if val.(bool) && !p.SCtx().GetSessionVars().InRestrictedSQL { - indexJoins, _ := tryToGetIndexJoin(p, prop) - failpoint.Return(indexJoins, true, nil) - } - }) - - if !isJoinHintSupportedInMPPMode(p.PreferJoinType) { - if hasMPPJoinHints(p.PreferJoinType) { - // If there are MPP hints but has some conflicts join method hints, all the join hints are invalid. - p.SCtx().GetSessionVars().StmtCtx.SetHintWarning("The MPP join hints are in conflict, and you can only specify join method hints that are currently supported by MPP mode now") - p.PreferJoinType = 0 - } else { - // If there are no MPP hints but has some conflicts join method hints, the MPP mode will be blocked. - p.SCtx().GetSessionVars().RaiseWarningWhenMPPEnforced("MPP mode may be blocked because you have used hint to specify a join algorithm which is not supported by mpp now.") - if prop.IsFlashProp() { - return nil, false, nil - } - } - } - if prop.MPPPartitionTp == property.BroadcastType { - return nil, false, nil - } - joins := make([]base.PhysicalPlan, 0, 8) - canPushToTiFlash := p.CanPushToCop(kv.TiFlash) - if p.SCtx().GetSessionVars().IsMPPAllowed() && canPushToTiFlash { - if (p.PreferJoinType & utilhint.PreferShuffleJoin) > 0 { - if shuffleJoins := tryToGetMppHashJoin(p, prop, false); len(shuffleJoins) > 0 { - return shuffleJoins, true, nil - } - } - if (p.PreferJoinType & utilhint.PreferBCJoin) > 0 { - if bcastJoins := tryToGetMppHashJoin(p, prop, true); len(bcastJoins) > 0 { - return bcastJoins, true, nil - } - } - if preferMppBCJ(p) { - mppJoins := tryToGetMppHashJoin(p, prop, true) - joins = append(joins, mppJoins...) - } else { - mppJoins := tryToGetMppHashJoin(p, prop, false) - joins = append(joins, mppJoins...) - } - } else { - hasMppHints := false - var errMsg string - if (p.PreferJoinType & utilhint.PreferShuffleJoin) > 0 { - errMsg = "The join can not push down to the MPP side, the shuffle_join() hint is invalid" - hasMppHints = true - } - if (p.PreferJoinType & utilhint.PreferBCJoin) > 0 { - errMsg = "The join can not push down to the MPP side, the broadcast_join() hint is invalid" - hasMppHints = true - } - if hasMppHints { - p.SCtx().GetSessionVars().StmtCtx.SetHintWarning(errMsg) - } - } - if prop.IsFlashProp() { - return joins, true, nil - } - - if !p.IsNAAJ() { - // naaj refuse merge join and index join. - mergeJoins := GetMergeJoin(p, prop, p.Schema(), p.StatsInfo(), p.Children()[0].StatsInfo(), p.Children()[1].StatsInfo()) - if (p.PreferJoinType&utilhint.PreferMergeJoin) > 0 && len(mergeJoins) > 0 { - return mergeJoins, true, nil - } - joins = append(joins, mergeJoins...) - - indexJoins, forced := tryToGetIndexJoin(p, prop) - if forced { - return indexJoins, true, nil - } - joins = append(joins, indexJoins...) - } - - hashJoins, forced := getHashJoins(p, prop) - if forced && len(hashJoins) > 0 { - return hashJoins, true, nil - } - joins = append(joins, hashJoins...) - - if p.PreferJoinType > 0 { - // If we reach here, it means we have a hint that doesn't work. - // It might be affected by the required property, so we enforce - // this property and try the hint again. - return joins, false, nil - } - return joins, true, nil + return utilfuncp.ExhaustPhysicalPlans4LogicalJoin(p, prop) } // ExtractCorrelatedCols implements the base.LogicalPlan.<15th> interface. @@ -726,11 +636,11 @@ func (p *LogicalJoin) ExtractCorrelatedCols() []*expression.CorrelatedColumn { func (p *LogicalJoin) ExtractFD() *funcdep.FDSet { switch p.JoinType { case InnerJoin: - return p.extractFDForInnerJoin(nil) + return p.ExtractFDForInnerJoin(nil) case LeftOuterJoin, RightOuterJoin: - return p.extractFDForOuterJoin(nil) + return p.ExtractFDForOuterJoin(nil) case SemiJoin: - return p.extractFDForSemiJoin(nil) + return p.ExtractFDForSemiJoin(nil) default: return &funcdep.FDSet{HashCodeToUniqueID: make(map[string]int)} } @@ -805,7 +715,8 @@ func (p *LogicalJoin) Shallow() *LogicalJoin { return join.Init(p.SCtx(), p.QueryBlockOffset()) } -func (p *LogicalJoin) extractFDForSemiJoin(filtersFromApply []expression.Expression) *funcdep.FDSet { +// ExtractFDForSemiJoin extracts FD for semi join. +func (p *LogicalJoin) ExtractFDForSemiJoin(filtersFromApply []expression.Expression) *funcdep.FDSet { // 1: since semi join will keep the part or all rows of the outer table, it's outer FD can be saved. // 2: the un-projected column will be left for the upper layer projection or already be pruned from bottom up. outerFD, _ := p.Children()[0].ExtractFD(), p.Children()[1].ExtractFD() @@ -814,9 +725,9 @@ func (p *LogicalJoin) extractFDForSemiJoin(filtersFromApply []expression.Express eqCondSlice := expression.ScalarFuncs2Exprs(p.EqualConditions) allConds := append(eqCondSlice, p.OtherConditions...) allConds = append(allConds, filtersFromApply...) - notNullColsFromFilters := ExtractNotNullFromConds(allConds, p) + notNullColsFromFilters := util.ExtractNotNullFromConds(allConds, p) - constUniqueIDs := ExtractConstantCols(p.LeftConditions, p.SCtx(), fds) + constUniqueIDs := util.ExtractConstantCols(p.LeftConditions, p.SCtx(), fds) fds.MakeNotNull(notNullColsFromFilters) fds.AddConstants(constUniqueIDs) @@ -824,7 +735,8 @@ func (p *LogicalJoin) extractFDForSemiJoin(filtersFromApply []expression.Express return fds } -func (p *LogicalJoin) extractFDForInnerJoin(filtersFromApply []expression.Expression) *funcdep.FDSet { +// ExtractFDForInnerJoin extracts FD for inner join. +func (p *LogicalJoin) ExtractFDForInnerJoin(filtersFromApply []expression.Expression) *funcdep.FDSet { leftFD, rightFD := p.Children()[0].ExtractFD(), p.Children()[1].ExtractFD() fds := leftFD fds.MakeCartesianProduct(rightFD) @@ -833,11 +745,11 @@ func (p *LogicalJoin) extractFDForInnerJoin(filtersFromApply []expression.Expres // some join eq conditions are stored in the OtherConditions. allConds := append(eqCondSlice, p.OtherConditions...) allConds = append(allConds, filtersFromApply...) - notNullColsFromFilters := ExtractNotNullFromConds(allConds, p) + notNullColsFromFilters := util.ExtractNotNullFromConds(allConds, p) - constUniqueIDs := ExtractConstantCols(allConds, p.SCtx(), fds) + constUniqueIDs := util.ExtractConstantCols(allConds, p.SCtx(), fds) - equivUniqueIDs := ExtractEquivalenceCols(allConds, p.SCtx(), fds) + equivUniqueIDs := util.ExtractEquivalenceCols(allConds, p.SCtx(), fds) fds.MakeNotNull(notNullColsFromFilters) fds.AddConstants(constUniqueIDs) @@ -865,7 +777,8 @@ func (p *LogicalJoin) extractFDForInnerJoin(filtersFromApply []expression.Expres return fds } -func (p *LogicalJoin) extractFDForOuterJoin(filtersFromApply []expression.Expression) *funcdep.FDSet { +// ExtractFDForOuterJoin extracts FD for outer join. +func (p *LogicalJoin) ExtractFDForOuterJoin(filtersFromApply []expression.Expression) *funcdep.FDSet { outerFD, innerFD := p.Children()[0].ExtractFD(), p.Children()[1].ExtractFD() innerCondition := p.RightConditions outerCondition := p.LeftConditions @@ -888,13 +801,13 @@ func (p *LogicalJoin) extractFDForOuterJoin(filtersFromApply []expression.Expres allConds = append(allConds, innerCondition...) allConds = append(allConds, outerCondition...) allConds = append(allConds, filtersFromApply...) - notNullColsFromFilters := ExtractNotNullFromConds(allConds, p) + notNullColsFromFilters := util.ExtractNotNullFromConds(allConds, p) filterFD := &funcdep.FDSet{HashCodeToUniqueID: make(map[string]int)} - constUniqueIDs := ExtractConstantCols(allConds, p.SCtx(), filterFD) + constUniqueIDs := util.ExtractConstantCols(allConds, p.SCtx(), filterFD) - equivUniqueIDs := ExtractEquivalenceCols(allConds, p.SCtx(), filterFD) + equivUniqueIDs := util.ExtractEquivalenceCols(allConds, p.SCtx(), filterFD) filterFD.AddConstants(constUniqueIDs) equivOuterUniqueIDs := intset.NewFastIntSet() @@ -1115,8 +1028,8 @@ func (p *LogicalJoin) ExtractJoinKeys(childIdx int) *expression.Schema { return expression.NewSchema(joinKeys...) } -// extractUsedCols extracts all the needed columns. -func (p *LogicalJoin) extractUsedCols(parentUsedCols []*expression.Column) (leftCols []*expression.Column, rightCols []*expression.Column) { +// ExtractUsedCols extracts all the needed columns. +func (p *LogicalJoin) ExtractUsedCols(parentUsedCols []*expression.Column) (leftCols []*expression.Column, rightCols []*expression.Column) { for _, eqCond := range p.EqualConditions { parentUsedCols = append(parentUsedCols, expression.ExtractColumns(eqCond)...) } @@ -1145,12 +1058,12 @@ func (p *LogicalJoin) extractUsedCols(parentUsedCols []*expression.Column) (left } // MergeSchema merge the schema of left and right child of join. -func (p *LogicalJoin) mergeSchema() { - p.SetSchema(buildLogicalJoinSchema(p.JoinType, p)) +func (p *LogicalJoin) MergeSchema() { + p.SetSchema(BuildLogicalJoinSchema(p.JoinType, p)) } // pushDownTopNToChild will push a topN to one child of join. The idx stands for join child index. 0 is for left child. -func (p *LogicalJoin) pushDownTopNToChild(topN *logicalop.LogicalTopN, idx int, opt *optimizetrace.LogicalOptimizeOp) base.LogicalPlan { +func (p *LogicalJoin) pushDownTopNToChild(topN *LogicalTopN, idx int, opt *optimizetrace.LogicalOptimizeOp) base.LogicalPlan { if topN == nil { return p.Children()[idx].PushDownTopN(nil, opt) } @@ -1164,7 +1077,7 @@ func (p *LogicalJoin) pushDownTopNToChild(topN *logicalop.LogicalTopN, idx int, } } - newTopN := logicalop.LogicalTopN{ + newTopN := LogicalTopN{ Count: topN.Count + topN.Offset, ByItems: make([]*util.ByItems, len(topN.ByItems)), PreferLimitToCop: topN.PreferLimitToCop, @@ -1205,7 +1118,7 @@ func addCandidateSelection(currentPlan base.LogicalPlan, currentChildIdx int, pa parentPlan.SetChild(currentChildIdx, selection) } selection.SetChildren(currentPlan) - appendAddSelectionTraceStep(parentPlan, currentPlan, selection, opt) + AppendAddSelectionTraceStep(parentPlan, currentPlan, selection, opt) if parentPlan == nil { return newRoot } @@ -1377,8 +1290,8 @@ func (p *LogicalJoin) SetPreferredJoinTypeAndOrder(hintInfo *utilhint.PlanHints) return } - lhsAlias := extractTableAlias(p.Children()[0], p.QueryBlockOffset()) - rhsAlias := extractTableAlias(p.Children()[1], p.QueryBlockOffset()) + lhsAlias := util.ExtractTableAlias(p.Children()[0], p.QueryBlockOffset()) + rhsAlias := util.ExtractTableAlias(p.Children()[1], p.QueryBlockOffset()) if hintInfo.IfPreferMergeJoin(lhsAlias) { p.PreferJoinType |= utilhint.PreferMergeJoin p.LeftPreferJoinType |= utilhint.PreferMergeJoin @@ -1574,7 +1487,7 @@ func (p *LogicalJoin) updateEQCond() { needRProj = needRProj || !rOk } - var lProj, rProj *logicalop.LogicalProjection + var lProj, rProj *LogicalProjection if needLProj { lProj = p.getProj(0) } @@ -1630,13 +1543,13 @@ func (p *LogicalJoin) updateEQCond() { } } -func (p *LogicalJoin) getProj(idx int) *logicalop.LogicalProjection { +func (p *LogicalJoin) getProj(idx int) *LogicalProjection { child := p.Children()[idx] - proj, ok := child.(*logicalop.LogicalProjection) + proj, ok := child.(*LogicalProjection) if ok { return proj } - proj = logicalop.LogicalProjection{Exprs: make([]expression.Expression, 0, child.Schema().Len())}.Init(p.SCtx(), child.QueryBlockOffset()) + proj = LogicalProjection{Exprs: make([]expression.Expression, 0, child.Schema().Len())}.Init(p.SCtx(), child.QueryBlockOffset()) for _, col := range child.Schema().Columns { proj.Exprs = append(proj.Exprs, col) } @@ -1670,3 +1583,257 @@ func (p *LogicalJoin) outerJoinPropConst(predicates []expression.Expression) []e p.AttachOnConds(joinConds) return predicates } + +func mergeOnClausePredicates(p *LogicalJoin, predicates []expression.Expression) []expression.Expression { + combinedCond := make([]expression.Expression, 0, + len(p.LeftConditions)+len(p.RightConditions)+ + len(p.EqualConditions)+len(p.OtherConditions)+ + len(predicates)) + combinedCond = append(combinedCond, p.LeftConditions...) + combinedCond = append(combinedCond, p.RightConditions...) + combinedCond = append(combinedCond, expression.ScalarFuncs2Exprs(p.EqualConditions)...) + combinedCond = append(combinedCond, p.OtherConditions...) + combinedCond = append(combinedCond, predicates...) + return combinedCond +} + +func appendTopNPushDownJoinTraceStep(p *LogicalJoin, topN *LogicalTopN, idx int, opt *optimizetrace.LogicalOptimizeOp) { + ectx := p.SCtx().GetExprCtx().GetEvalCtx() + action := func() string { + buffer := bytes.NewBufferString(fmt.Sprintf("%v_%v is added and pushed into %v_%v's ", + topN.TP(), topN.ID(), p.TP(), p.ID())) + if idx == 0 { + buffer.WriteString("left ") + } else { + buffer.WriteString("right ") + } + buffer.WriteString("table") + return buffer.String() + } + reason := func() string { + buffer := bytes.NewBufferString(fmt.Sprintf("%v_%v's joinType is %v, and all ByItems[", p.TP(), p.ID(), p.JoinType.String())) + for i, item := range topN.ByItems { + if i > 0 { + buffer.WriteString(",") + } + buffer.WriteString(item.StringWithCtx(ectx, errors.RedactLogDisable)) + } + buffer.WriteString("] contained in ") + if idx == 0 { + buffer.WriteString("left ") + } else { + buffer.WriteString("right ") + } + buffer.WriteString("table") + return buffer.String() + } + opt.AppendStepToCurrent(p.ID(), p.TP(), reason, action) +} + +// AppendAddSelectionTraceStep appends a trace step for adding a selection operator. +func AppendAddSelectionTraceStep(p base.LogicalPlan, child base.LogicalPlan, sel *LogicalSelection, opt *optimizetrace.LogicalOptimizeOp) { + reason := func() string { + return "" + } + action := func() string { + return fmt.Sprintf("add %v_%v to connect %v_%v and %v_%v", sel.TP(), sel.ID(), p.TP(), p.ID(), child.TP(), child.ID()) + } + opt.AppendStepToCurrent(sel.ID(), sel.TP(), reason, action) +} + +// containDifferentJoinTypes checks whether `PreferJoinType` contains different +// join types. +func containDifferentJoinTypes(preferJoinType uint) bool { + preferJoinType &= ^utilhint.PreferNoHashJoin + preferJoinType &= ^utilhint.PreferNoMergeJoin + preferJoinType &= ^utilhint.PreferNoIndexJoin + preferJoinType &= ^utilhint.PreferNoIndexHashJoin + preferJoinType &= ^utilhint.PreferNoIndexMergeJoin + + inlMask := utilhint.PreferRightAsINLJInner ^ utilhint.PreferLeftAsINLJInner + inlhjMask := utilhint.PreferRightAsINLHJInner ^ utilhint.PreferLeftAsINLHJInner + inlmjMask := utilhint.PreferRightAsINLMJInner ^ utilhint.PreferLeftAsINLMJInner + hjRightBuildMask := utilhint.PreferRightAsHJBuild ^ utilhint.PreferLeftAsHJProbe + hjLeftBuildMask := utilhint.PreferLeftAsHJBuild ^ utilhint.PreferRightAsHJProbe + + mppMask := utilhint.PreferShuffleJoin ^ utilhint.PreferBCJoin + mask := inlMask ^ inlhjMask ^ inlmjMask ^ hjRightBuildMask ^ hjLeftBuildMask + onesCount := bits.OnesCount(preferJoinType & ^mask & ^mppMask) + if onesCount > 1 || onesCount == 1 && preferJoinType&mask > 0 { + return true + } + + cnt := 0 + if preferJoinType&inlMask > 0 { + cnt++ + } + if preferJoinType&inlhjMask > 0 { + cnt++ + } + if preferJoinType&inlmjMask > 0 { + cnt++ + } + if preferJoinType&hjLeftBuildMask > 0 { + cnt++ + } + if preferJoinType&hjRightBuildMask > 0 { + cnt++ + } + return cnt > 1 +} + +func setPreferredJoinTypeFromOneSide(preferJoinType uint, isLeft bool) (resJoinType uint) { + if preferJoinType == 0 { + return + } + if preferJoinType&utilhint.PreferINLJ > 0 { + preferJoinType &= ^utilhint.PreferINLJ + if isLeft { + resJoinType |= utilhint.PreferLeftAsINLJInner + } else { + resJoinType |= utilhint.PreferRightAsINLJInner + } + } + if preferJoinType&utilhint.PreferINLHJ > 0 { + preferJoinType &= ^utilhint.PreferINLHJ + if isLeft { + resJoinType |= utilhint.PreferLeftAsINLHJInner + } else { + resJoinType |= utilhint.PreferRightAsINLHJInner + } + } + if preferJoinType&utilhint.PreferINLMJ > 0 { + preferJoinType &= ^utilhint.PreferINLMJ + if isLeft { + resJoinType |= utilhint.PreferLeftAsINLMJInner + } else { + resJoinType |= utilhint.PreferRightAsINLMJInner + } + } + if preferJoinType&utilhint.PreferHJBuild > 0 { + preferJoinType &= ^utilhint.PreferHJBuild + if isLeft { + resJoinType |= utilhint.PreferLeftAsHJBuild + } else { + resJoinType |= utilhint.PreferRightAsHJBuild + } + } + if preferJoinType&utilhint.PreferHJProbe > 0 { + preferJoinType &= ^utilhint.PreferHJProbe + if isLeft { + resJoinType |= utilhint.PreferLeftAsHJProbe + } else { + resJoinType |= utilhint.PreferRightAsHJProbe + } + } + resJoinType |= preferJoinType + return +} + +// DeriveOtherConditions given a LogicalJoin, check the OtherConditions to see if we can derive more +// conditions for left/right child pushdown. +func DeriveOtherConditions( + p *LogicalJoin, leftSchema *expression.Schema, rightSchema *expression.Schema, + deriveLeft bool, deriveRight bool) ( + leftCond []expression.Expression, rightCond []expression.Expression) { + isOuterSemi := (p.JoinType == LeftOuterSemiJoin) || (p.JoinType == AntiLeftOuterSemiJoin) + ctx := p.SCtx() + exprCtx := ctx.GetExprCtx() + for _, expr := range p.OtherConditions { + if deriveLeft { + leftRelaxedCond := expression.DeriveRelaxedFiltersFromDNF(exprCtx, expr, leftSchema) + if leftRelaxedCond != nil { + leftCond = append(leftCond, leftRelaxedCond) + } + notNullExpr := deriveNotNullExpr(ctx, expr, leftSchema) + if notNullExpr != nil { + leftCond = append(leftCond, notNullExpr) + } + } + if deriveRight { + rightRelaxedCond := expression.DeriveRelaxedFiltersFromDNF(exprCtx, expr, rightSchema) + if rightRelaxedCond != nil { + rightCond = append(rightCond, rightRelaxedCond) + } + // For LeftOuterSemiJoin and AntiLeftOuterSemiJoin, we can actually generate + // `col is not null` according to expressions in `OtherConditions` now, but we + // are putting column equal condition converted from `in (subq)` into + // `OtherConditions`(@sa https://github.com/pingcap/tidb/pull/9051), then it would + // cause wrong results, so we disable this optimization for outer semi joins now. + // TODO enable this optimization for outer semi joins later by checking whether + // condition in `OtherConditions` is converted from `in (subq)`. + if isOuterSemi { + continue + } + notNullExpr := deriveNotNullExpr(ctx, expr, rightSchema) + if notNullExpr != nil { + rightCond = append(rightCond, notNullExpr) + } + } + } + return +} + +// deriveNotNullExpr generates a new expression `not(isnull(col))` given `col1 op col2`, +// in which `col` is in specified schema. Caller guarantees that only one of `col1` or +// `col2` is in schema. +func deriveNotNullExpr(ctx base.PlanContext, expr expression.Expression, schema *expression.Schema) expression.Expression { + binop, ok := expr.(*expression.ScalarFunction) + if !ok || len(binop.GetArgs()) != 2 { + return nil + } + arg0, lOK := binop.GetArgs()[0].(*expression.Column) + arg1, rOK := binop.GetArgs()[1].(*expression.Column) + if !lOK || !rOK { + return nil + } + childCol := schema.RetrieveColumn(arg0) + if childCol == nil { + childCol = schema.RetrieveColumn(arg1) + } + if util.IsNullRejected(ctx, schema, expr) && !mysql.HasNotNullFlag(childCol.RetType.GetFlag()) { + return expression.BuildNotNullExpr(ctx.GetExprCtx(), childCol) + } + return nil +} + +// Conds2TableDual builds a LogicalTableDual if cond is constant false or null. +func Conds2TableDual(p base.LogicalPlan, conds []expression.Expression) base.LogicalPlan { + if len(conds) != 1 { + return nil + } + con, ok := conds[0].(*expression.Constant) + if !ok { + return nil + } + sc := p.SCtx().GetSessionVars().StmtCtx + if expression.MaybeOverOptimized4PlanCache(p.SCtx().GetExprCtx(), []expression.Expression{con}) { + return nil + } + if isTrue, err := con.Value.ToBool(sc.TypeCtxOrDefault()); (err == nil && isTrue == 0) || con.Value.IsNull() { + dual := LogicalTableDual{}.Init(p.SCtx(), p.QueryBlockOffset()) + dual.SetSchema(p.Schema()) + return dual + } + return nil +} + +// BuildLogicalJoinSchema builds the schema for join operator. +func BuildLogicalJoinSchema(joinType JoinType, join base.LogicalPlan) *expression.Schema { + leftSchema := join.Children()[0].Schema() + switch joinType { + case SemiJoin, AntiSemiJoin: + return leftSchema.Clone() + case LeftOuterSemiJoin, AntiLeftOuterSemiJoin: + newSchema := leftSchema.Clone() + newSchema.Append(join.Schema().Columns[join.Schema().Len()-1]) + return newSchema + } + newSchema := expression.MergeSchema(leftSchema, join.Children()[1].Schema()) + if joinType == LeftOuterJoin { + util.ResetNotNullFlag(newSchema, leftSchema.Len(), newSchema.Len()) + } else if joinType == RightOuterJoin { + util.ResetNotNullFlag(newSchema, 0, leftSchema.Len()) + } + return newSchema +} diff --git a/pkg/planner/core/logical_selection.go b/pkg/planner/core/operator/logicalop/logical_selection.go similarity index 73% rename from pkg/planner/core/logical_selection.go rename to pkg/planner/core/operator/logicalop/logical_selection.go index 98b7bf58080b4..304ea704508f7 100644 --- a/pkg/planner/core/logical_selection.go +++ b/pkg/planner/core/operator/logicalop/logical_selection.go @@ -12,30 +12,33 @@ // See the License for the specific language governing permissions and // limitations under the License. -package core +package logicalop import ( "bytes" + "fmt" "slices" + "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/planner/core/base" + "github.com/pingcap/tidb/pkg/planner/core/constraint" "github.com/pingcap/tidb/pkg/planner/core/cost" - "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" ruleutil "github.com/pingcap/tidb/pkg/planner/core/rule/util" fd "github.com/pingcap/tidb/pkg/planner/funcdep" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/planner/util/optimizetrace" + "github.com/pingcap/tidb/pkg/planner/util/utilfuncp" "github.com/pingcap/tidb/pkg/util/intset" "github.com/pingcap/tidb/pkg/util/plancodec" ) // LogicalSelection represents a where or having predicate. type LogicalSelection struct { - logicalop.BaseLogicalPlan + BaseLogicalPlan // Originally the WHERE or ON condition is parsed into a single expression, // but after we converted to CNF(Conjunctive normal form), it can be @@ -45,7 +48,7 @@ type LogicalSelection struct { // Init initializes LogicalSelection. func (p LogicalSelection) Init(ctx base.PlanContext, qbOffset int) *LogicalSelection { - p.BaseLogicalPlan = logicalop.NewBaseLogicalPlan(ctx, plancodec.TypeSel, &p, qbOffset) + p.BaseLogicalPlan = NewBaseLogicalPlan(ctx, plancodec.TypeSel, &p, qbOffset) return &p } @@ -93,8 +96,8 @@ func (p *LogicalSelection) HashCode() []byte { // PredicatePushDown implements base.LogicalPlan.<1st> interface. func (p *LogicalSelection) PredicatePushDown(predicates []expression.Expression, opt *optimizetrace.LogicalOptimizeOp) ([]expression.Expression, base.LogicalPlan) { - predicates = DeleteTrueExprs(p, predicates) - p.Conditions = DeleteTrueExprs(p, p.Conditions) + predicates = constraint.DeleteTrueExprs(p, predicates) + p.Conditions = constraint.DeleteTrueExprs(p, p.Conditions) var child base.LogicalPlan var retConditions []expression.Expression var originConditions []expression.Expression @@ -108,7 +111,7 @@ func (p *LogicalSelection) PredicatePushDown(predicates []expression.Expression, // Return table dual when filter is constant false or null. dual := Conds2TableDual(p, p.Conditions) if dual != nil { - appendTableDualTraceStep(p, dual, p.Conditions, opt) + AppendTableDualTraceStep(p, dual, p.Conditions, opt) return nil, dual } return nil, p @@ -160,17 +163,17 @@ func (p *LogicalSelection) BuildKeyInfo(selfSchema *expression.Schema, childSche // DeriveTopN implements the base.LogicalPlan.<6th> interface. func (p *LogicalSelection) DeriveTopN(opt *optimizetrace.LogicalOptimizeOp) base.LogicalPlan { s := p.Self().(*LogicalSelection) - windowIsTopN, limitValue := windowIsTopN(s) + windowIsTopN, limitValue := utilfuncp.WindowIsTopN(s) if windowIsTopN { - child := s.Children()[0].(*logicalop.LogicalWindow) - grandChild := child.Children()[0].(*DataSource) + child := s.Children()[0].(*LogicalWindow) + grandChild := child.Children()[0] // Build order by for derived Limit byItems := make([]*util.ByItems, 0, len(child.OrderBy)) for _, col := range child.OrderBy { byItems = append(byItems, &util.ByItems{Expr: col.Col, Desc: col.Desc}) } // Build derived Limit - derivedTopN := logicalop.LogicalTopN{Count: limitValue, ByItems: byItems, PartitionBy: child.GetPartitionBy()}.Init(grandChild.SCtx(), grandChild.QueryBlockOffset()) + derivedTopN := LogicalTopN{Count: limitValue, ByItems: byItems, PartitionBy: child.GetPartitionBy()}.Init(grandChild.SCtx(), grandChild.QueryBlockOffset()) derivedTopN.SetChildren(grandChild) /* return select->datasource->topN->window */ child.SetChildren(derivedTopN) @@ -190,7 +193,7 @@ func (p *LogicalSelection) PullUpConstantPredicates() []expression.Expression { var result []expression.Expression for _, candidatePredicate := range p.Conditions { // the candidate predicate should be a constant and compare predicate - match := validCompareConstantPredicate(p.SCtx().GetExprCtx().GetEvalCtx(), candidatePredicate) + match := expression.ValidCompareConstantPredicate(p.SCtx().GetExprCtx().GetEvalCtx(), candidatePredicate) if match { result = append(result, candidatePredicate) } @@ -219,7 +222,7 @@ func (*LogicalSelection) PreparePossibleProperties(_ *expression.Schema, childre // ExhaustPhysicalPlans implements base.LogicalPlan.<14th> interface. func (p *LogicalSelection) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) { - return exhaustPhysicalPlans4LogicalSelection(p, prop) + return utilfuncp.ExhaustPhysicalPlans4LogicalSelection(p, prop) } // ExtractCorrelatedCols implements base.LogicalPlan.<15th> interface. @@ -264,13 +267,13 @@ func (p *LogicalSelection) ExtractFD() *fd.FDSet { } // extract the not null attributes from selection conditions. - notnullColsUniqueIDs.UnionWith(ExtractNotNullFromConds(p.Conditions, p)) + notnullColsUniqueIDs.UnionWith(util.ExtractNotNullFromConds(p.Conditions, p)) // extract the constant cols from selection conditions. - constUniqueIDs := ExtractConstantCols(p.Conditions, p.SCtx(), fds) + constUniqueIDs := util.ExtractConstantCols(p.Conditions, p.SCtx(), fds) // extract equivalence cols. - equivUniqueIDs := ExtractEquivalenceCols(p.Conditions, p.SCtx(), fds) + equivUniqueIDs := util.ExtractEquivalenceCols(p.Conditions, p.SCtx(), fds) // apply operator's characteristic's FD setting. fds.MakeNotNull(notnullColsUniqueIDs) @@ -298,9 +301,77 @@ func (p *LogicalSelection) ConvertOuterToInnerJoin(predicates []expression.Expre // *************************** end implementation of logicalPlan interface *************************** -// utility function to check whether we can push down Selection to TiKV or TiFlash -func (p *LogicalSelection) canPushDown(storeTp kv.StoreType) bool { +// CanPushDown is utility function to check whether we can push down Selection to TiKV or TiFlash +func (p *LogicalSelection) CanPushDown(storeTp kv.StoreType) bool { return !expression.ContainVirtualColumn(p.Conditions) && p.CanPushToCop(storeTp) && - expression.CanExprsPushDown(GetPushDownCtx(p.SCtx()), p.Conditions, storeTp) + expression.CanExprsPushDown(util.GetPushDownCtx(p.SCtx()), p.Conditions, storeTp) +} + +func splitSetGetVarFunc(filters []expression.Expression) ([]expression.Expression, []expression.Expression) { + canBePushDown := make([]expression.Expression, 0, len(filters)) + canNotBePushDown := make([]expression.Expression, 0, len(filters)) + for _, expr := range filters { + if expression.HasGetSetVarFunc(expr) { + canNotBePushDown = append(canNotBePushDown, expr) + } else { + canBePushDown = append(canBePushDown, expr) + } + } + return canBePushDown, canNotBePushDown +} + +// AppendTableDualTraceStep appends a trace step for replacing a plan with a dual table. +func AppendTableDualTraceStep(replaced base.LogicalPlan, dual base.LogicalPlan, conditions []expression.Expression, opt *optimizetrace.LogicalOptimizeOp) { + action := func() string { + return fmt.Sprintf("%v_%v is replaced by %v_%v", replaced.TP(), replaced.ID(), dual.TP(), dual.ID()) + } + ectx := replaced.SCtx().GetExprCtx().GetEvalCtx() + reason := func() string { + buffer := bytes.NewBufferString("The conditions[") + for i, cond := range conditions { + if i > 0 { + buffer.WriteString(",") + } + buffer.WriteString(cond.StringWithCtx(ectx, errors.RedactLogDisable)) + } + buffer.WriteString("] are constant false or null") + return buffer.String() + } + opt.AppendStepToCurrent(dual.ID(), dual.TP(), reason, action) +} + +func appendSelectionPredicatePushDownTraceStep(p *LogicalSelection, conditions []expression.Expression, opt *optimizetrace.LogicalOptimizeOp) { + action := func() string { + return fmt.Sprintf("%v_%v is removed", p.TP(), p.ID()) + } + reason := func() string { + return "" + } + if len(conditions) > 0 { + evalCtx := p.SCtx().GetExprCtx().GetEvalCtx() + reason = func() string { + buffer := bytes.NewBufferString("The conditions[") + for i, cond := range conditions { + if i > 0 { + buffer.WriteString(",") + } + buffer.WriteString(cond.StringWithCtx(evalCtx, errors.RedactLogDisable)) + } + fmt.Fprintf(buffer, "] in %v_%v are pushed down", p.TP(), p.ID()) + return buffer.String() + } + } + opt.AppendStepToCurrent(p.ID(), p.TP(), reason, action) +} + +func appendDerivedTopNTrace(topN base.LogicalPlan, opt *optimizetrace.LogicalOptimizeOp) { + child := topN.Children()[0] + action := func() string { + return fmt.Sprintf("%v_%v top N added below %v_%v ", topN.TP(), topN.ID(), child.TP(), child.ID()) + } + reason := func() string { + return fmt.Sprintf("%v filter on row number", topN.TP()) + } + opt.AppendStepToCurrent(topN.ID(), topN.TP(), reason, action) } diff --git a/pkg/planner/core/optimizer.go b/pkg/planner/core/optimizer.go index 1fd7657758403..bd016f5ae082c 100644 --- a/pkg/planner/core/optimizer.go +++ b/pkg/planner/core/optimizer.go @@ -39,6 +39,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/core/base" + "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/planner/core/operator/physicalop" "github.com/pingcap/tidb/pkg/planner/core/rule" "github.com/pingcap/tidb/pkg/planner/property" @@ -1155,8 +1156,8 @@ func transformPhysicalPlan(p base.PhysicalPlan, f func(p base.PhysicalPlan) base } func existsCartesianProduct(p base.LogicalPlan) bool { - if join, ok := p.(*LogicalJoin); ok && len(join.EqualConditions) == 0 { - return join.JoinType == InnerJoin || join.JoinType == LeftOuterJoin || join.JoinType == RightOuterJoin + if join, ok := p.(*logicalop.LogicalJoin); ok && len(join.EqualConditions) == 0 { + return join.JoinType == logicalop.InnerJoin || join.JoinType == logicalop.LeftOuterJoin || join.JoinType == logicalop.RightOuterJoin } for _, child := range p.Children() { if existsCartesianProduct(child) { diff --git a/pkg/planner/core/physical_plans.go b/pkg/planner/core/physical_plans.go index 1ab7ac2104052..57d63b2dea4fd 100644 --- a/pkg/planner/core/physical_plans.go +++ b/pkg/planner/core/physical_plans.go @@ -1275,13 +1275,13 @@ type PhysicalJoin interface { base.PhysicalPlan PhysicalJoinImplement() getInnerChildIdx() int - GetJoinType() JoinType + GetJoinType() logicalop.JoinType } type basePhysicalJoin struct { physicalSchemaProducer - JoinType JoinType + JoinType logicalop.JoinType LeftConditions expression.CNFExprs RightConditions expression.CNFExprs @@ -1303,7 +1303,7 @@ type basePhysicalJoin struct { RightNAJoinKeys []*expression.Column } -func (p *basePhysicalJoin) GetJoinType() JoinType { +func (p *basePhysicalJoin) GetJoinType() logicalop.JoinType { return p.JoinType } @@ -1449,7 +1449,7 @@ type PhysicalHashJoin struct { // CanUseHashJoinV2 returns true if current join is supported by hash join v2 func (p *PhysicalHashJoin) CanUseHashJoinV2() bool { switch p.JoinType { - case LeftOuterJoin, RightOuterJoin, InnerJoin: + case logicalop.LeftOuterJoin, logicalop.RightOuterJoin, logicalop.InnerJoin: // null aware join is not supported yet if len(p.LeftNAJoinKeys) > 0 { return false @@ -1541,7 +1541,7 @@ func (p *PhysicalHashJoin) RightIsBuildSide() bool { } // NewPhysicalHashJoin creates a new PhysicalHashJoin from LogicalJoin. -func NewPhysicalHashJoin(p *LogicalJoin, innerIdx int, useOuterToBuild bool, newStats *property.StatsInfo, prop ...*property.PhysicalProperty) *PhysicalHashJoin { +func NewPhysicalHashJoin(p *logicalop.LogicalJoin, innerIdx int, useOuterToBuild bool, newStats *property.StatsInfo, prop ...*property.PhysicalProperty) *PhysicalHashJoin { leftJoinKeys, rightJoinKeys, isNullEQ, _ := p.GetJoinKeys() leftNAJoinKeys, rightNAJoinKeys := p.GetNAJoinKeys() baseJoin := basePhysicalJoin{ @@ -2639,7 +2639,7 @@ func (p *PhysicalShowDDLJobs) MemoryUsage() (sum int64) { } // BuildMergeJoinPlan builds a PhysicalMergeJoin from the given fields. Currently, it is only used for test purpose. -func BuildMergeJoinPlan(ctx base.PlanContext, joinType JoinType, leftKeys, rightKeys []*expression.Column) *PhysicalMergeJoin { +func BuildMergeJoinPlan(ctx base.PlanContext, joinType logicalop.JoinType, leftKeys, rightKeys []*expression.Column) *PhysicalMergeJoin { baseJoin := basePhysicalJoin{ JoinType: joinType, DefaultValues: []types.Datum{types.NewDatum(1), types.NewDatum(1)}, diff --git a/pkg/planner/core/plan.go b/pkg/planner/core/plan.go index 53d3910706a99..e239bcba95300 100644 --- a/pkg/planner/core/plan.go +++ b/pkg/planner/core/plan.go @@ -251,14 +251,14 @@ func HasMaxOneRow(p base.LogicalPlan, childMaxOneRow []bool) bool { return false } switch x := p.(type) { - case *logicalop.LogicalLock, *logicalop.LogicalLimit, *logicalop.LogicalSort, *LogicalSelection, + case *logicalop.LogicalLock, *logicalop.LogicalLimit, *logicalop.LogicalSort, *logicalop.LogicalSelection, *LogicalApply, *logicalop.LogicalProjection, *logicalop.LogicalWindow, *LogicalAggregation: return childMaxOneRow[0] case *logicalop.LogicalMaxOneRow: return true - case *LogicalJoin: + case *logicalop.LogicalJoin: switch x.JoinType { - case SemiJoin, AntiSemiJoin, LeftOuterSemiJoin, AntiLeftOuterSemiJoin: + case logicalop.SemiJoin, logicalop.AntiSemiJoin, logicalop.LeftOuterSemiJoin, logicalop.AntiLeftOuterSemiJoin: return childMaxOneRow[0] default: return childMaxOneRow[0] && childMaxOneRow[1] diff --git a/pkg/planner/core/plan_cost_ver1.go b/pkg/planner/core/plan_cost_ver1.go index 2b8cb7d9e0a81..f6bfc1c236195 100644 --- a/pkg/planner/core/plan_cost_ver1.go +++ b/pkg/planner/core/plan_cost_ver1.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/cardinality" "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/core/cost" + "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util/costusage" "github.com/pingcap/tidb/pkg/planner/util/optimizetrace" @@ -475,8 +476,8 @@ func (p *PhysicalIndexJoin) GetCost(outerCnt, innerCnt, outerCost, innerCost flo cpuCost += innerCPUCost / innerConcurrency // Cost of probing hash table in main thread. numPairs := outerCnt * innerCnt - if p.JoinType == SemiJoin || p.JoinType == AntiSemiJoin || - p.JoinType == LeftOuterSemiJoin || p.JoinType == AntiLeftOuterSemiJoin { + if p.JoinType == logicalop.SemiJoin || p.JoinType == logicalop.AntiSemiJoin || + p.JoinType == logicalop.LeftOuterSemiJoin || p.JoinType == logicalop.AntiLeftOuterSemiJoin { if len(p.OtherConditions) > 0 { numPairs *= 0.5 } else { @@ -562,8 +563,8 @@ func (p *PhysicalIndexHashJoin) GetCost(outerCnt, innerCnt, outerCost, innerCost cpuCost += outerCPUCost / concurrency // Cost of probing hash table concurrently. numPairs := outerCnt * innerCnt - if p.JoinType == SemiJoin || p.JoinType == AntiSemiJoin || - p.JoinType == LeftOuterSemiJoin || p.JoinType == AntiLeftOuterSemiJoin { + if p.JoinType == logicalop.SemiJoin || p.JoinType == logicalop.AntiSemiJoin || + p.JoinType == logicalop.LeftOuterSemiJoin || p.JoinType == logicalop.AntiLeftOuterSemiJoin { if len(p.OtherConditions) > 0 { numPairs *= 0.5 } else { @@ -653,8 +654,8 @@ func (p *PhysicalIndexMergeJoin) GetCost(outerCnt, innerCnt, outerCost, innerCos cpuCost += innerCPUCost / innerConcurrency // Cost of merge join in inner worker. numPairs := outerCnt * innerCnt - if p.JoinType == SemiJoin || p.JoinType == AntiSemiJoin || - p.JoinType == LeftOuterSemiJoin || p.JoinType == AntiLeftOuterSemiJoin { + if p.JoinType == logicalop.SemiJoin || p.JoinType == logicalop.AntiSemiJoin || + p.JoinType == logicalop.LeftOuterSemiJoin || p.JoinType == logicalop.AntiLeftOuterSemiJoin { if len(p.OtherConditions) > 0 { numPairs *= 0.5 } else { @@ -724,8 +725,8 @@ func (p *PhysicalApply) GetCost(lCount, rCount, lCost, rCost float64) float64 { rCount *= cost.SelectionFactor } if len(p.EqualConditions)+len(p.OtherConditions)+len(p.NAEqualConditions) > 0 { - if p.JoinType == SemiJoin || p.JoinType == AntiSemiJoin || - p.JoinType == LeftOuterSemiJoin || p.JoinType == AntiLeftOuterSemiJoin { + if p.JoinType == logicalop.SemiJoin || p.JoinType == logicalop.AntiSemiJoin || + p.JoinType == logicalop.LeftOuterSemiJoin || p.JoinType == logicalop.AntiLeftOuterSemiJoin { cpuCost += lCount * rCount * sessVars.GetCPUFactor() * 0.5 } else { cpuCost += lCount * rCount * sessVars.GetCPUFactor() @@ -767,7 +768,7 @@ func (p *PhysicalMergeJoin) GetCost(lCnt, rCnt float64, costFlag uint64) float64 innerKeys := p.RightJoinKeys innerSchema := p.Children()[1].Schema() innerStats := p.Children()[1].StatsInfo() - if p.JoinType == RightOuterJoin { + if p.JoinType == logicalop.RightOuterJoin { outerCnt = rCnt innerCnt = lCnt innerKeys = p.LeftJoinKeys @@ -779,8 +780,8 @@ func (p *PhysicalMergeJoin) GetCost(lCnt, rCnt float64, costFlag uint64) float64 p.LeftJoinKeys, p.RightJoinKeys, p.Children()[0].Schema(), p.Children()[1].Schema(), p.LeftNAJoinKeys, p.RightNAJoinKeys) - if p.JoinType == SemiJoin || p.JoinType == AntiSemiJoin || - p.JoinType == LeftOuterSemiJoin || p.JoinType == AntiLeftOuterSemiJoin { + if p.JoinType == logicalop.SemiJoin || p.JoinType == logicalop.AntiSemiJoin || + p.JoinType == logicalop.LeftOuterSemiJoin || p.JoinType == logicalop.AntiLeftOuterSemiJoin { if len(p.OtherConditions) > 0 { numPairs *= 0.5 } else { @@ -862,8 +863,8 @@ func (p *PhysicalHashJoin) GetCost(lCnt, rCnt float64, _ bool, costFlag uint64, // to the end of those pairs; since we have no idea about when we can // terminate the iteration, we assume that we need to iterate half of // those pairs in average. - if p.JoinType == SemiJoin || p.JoinType == AntiSemiJoin || - p.JoinType == LeftOuterSemiJoin || p.JoinType == AntiLeftOuterSemiJoin { + if p.JoinType == logicalop.SemiJoin || p.JoinType == logicalop.AntiSemiJoin || + p.JoinType == logicalop.LeftOuterSemiJoin || p.JoinType == logicalop.AntiLeftOuterSemiJoin { if len(p.OtherConditions) > 0 { numPairs *= 0.5 } else { diff --git a/pkg/planner/core/plan_to_pb.go b/pkg/planner/core/plan_to_pb.go index 8ec4af77d7861..a823a2d03042a 100644 --- a/pkg/planner/core/plan_to_pb.go +++ b/pkg/planner/core/plan_to_pb.go @@ -23,6 +23,8 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/planner/core/base" + "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" + util2 "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/table/tables" "github.com/pingcap/tidb/pkg/util" @@ -92,7 +94,7 @@ func (p *PhysicalHashAgg) ToPB(ctx *base.BuildPBContext, storeType kv.StoreType) aggExec := &tipb.Aggregation{ GroupBy: groupByExprs, } - pushDownCtx := GetPushDownCtx(p.SCtx()) + pushDownCtx := util2.GetPushDownCtx(p.SCtx()) for _, aggFunc := range p.AggFuncs { agg, err := aggregation.AggFuncToPBExpr(pushDownCtx, aggFunc, storeType) if err != nil { @@ -134,7 +136,7 @@ func (p *PhysicalHashAgg) ToPB(ctx *base.BuildPBContext, storeType kv.StoreType) func (p *PhysicalStreamAgg) ToPB(ctx *base.BuildPBContext, storeType kv.StoreType) (*tipb.Executor, error) { client := ctx.GetClient() evalCtx := ctx.GetExprCtx().GetEvalCtx() - pushDownCtx := GetPushDownCtxFromBuildPBContext(ctx) + pushDownCtx := util2.GetPushDownCtxFromBuildPBContext(ctx) groupByExprs, err := expression.ExpressionsToPBList(evalCtx, p.GroupByItems, client) if err != nil { return nil, err @@ -557,7 +559,7 @@ func (p *PhysicalHashJoin) ToPB(ctx *base.BuildPBContext, storeType kv.StoreType var otherEqConditionsFromIn expression.CNFExprs /// For anti join, equal conditions from `in` clause requires additional processing, /// for example, treat `null` as true. - if p.JoinType == AntiSemiJoin || p.JoinType == AntiLeftOuterSemiJoin || p.JoinType == LeftOuterSemiJoin { + if p.JoinType == logicalop.AntiSemiJoin || p.JoinType == logicalop.AntiLeftOuterSemiJoin || p.JoinType == logicalop.LeftOuterSemiJoin { for _, condition := range p.OtherConditions { if expression.IsEQCondFromIn(condition) { otherEqConditionsFromIn = append(otherEqConditionsFromIn, condition) @@ -579,17 +581,17 @@ func (p *PhysicalHashJoin) ToPB(ctx *base.BuildPBContext, storeType kv.StoreType pbJoinType := tipb.JoinType_TypeInnerJoin switch p.JoinType { - case LeftOuterJoin: + case logicalop.LeftOuterJoin: pbJoinType = tipb.JoinType_TypeLeftOuterJoin - case RightOuterJoin: + case logicalop.RightOuterJoin: pbJoinType = tipb.JoinType_TypeRightOuterJoin - case SemiJoin: + case logicalop.SemiJoin: pbJoinType = tipb.JoinType_TypeSemiJoin - case AntiSemiJoin: + case logicalop.AntiSemiJoin: pbJoinType = tipb.JoinType_TypeAntiSemiJoin - case LeftOuterSemiJoin: + case logicalop.LeftOuterSemiJoin: pbJoinType = tipb.JoinType_TypeLeftOuterSemiJoin - case AntiLeftOuterSemiJoin: + case logicalop.AntiLeftOuterSemiJoin: pbJoinType = tipb.JoinType_TypeAntiLeftOuterSemiJoin } diff --git a/pkg/planner/core/resolve_indices.go b/pkg/planner/core/resolve_indices.go index e9885894eda66..a930a7b31a0a2 100644 --- a/pkg/planner/core/resolve_indices.go +++ b/pkg/planner/core/resolve_indices.go @@ -17,6 +17,7 @@ package core import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/expression" + "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/planner/core/operator/physicalop" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/util/disjointset" @@ -135,7 +136,7 @@ func (p *PhysicalHashJoin) ResolveIndicesItself() (err error) { colsNeedResolving := p.schema.Len() // The last output column of this two join is the generated column to indicate whether the row is matched or not. - if p.JoinType == LeftOuterSemiJoin || p.JoinType == AntiLeftOuterSemiJoin { + if p.JoinType == logicalop.LeftOuterSemiJoin || p.JoinType == logicalop.AntiLeftOuterSemiJoin { colsNeedResolving-- } // To avoid that two plan shares the same column slice. @@ -228,7 +229,7 @@ func (p *PhysicalMergeJoin) ResolveIndices() (err error) { colsNeedResolving := p.schema.Len() // The last output column of this two join is the generated column to indicate whether the row is matched or not. - if p.JoinType == LeftOuterSemiJoin || p.JoinType == AntiLeftOuterSemiJoin { + if p.JoinType == logicalop.LeftOuterSemiJoin || p.JoinType == logicalop.AntiLeftOuterSemiJoin { colsNeedResolving-- } // To avoid that two plan shares the same column slice. @@ -324,7 +325,7 @@ func (p *PhysicalIndexJoin) ResolveIndices() (err error) { colsNeedResolving := p.schema.Len() // The last output column of this two join is the generated column to indicate whether the row is matched or not. - if p.JoinType == LeftOuterSemiJoin || p.JoinType == AntiLeftOuterSemiJoin { + if p.JoinType == logicalop.LeftOuterSemiJoin || p.JoinType == logicalop.AntiLeftOuterSemiJoin { colsNeedResolving-- } // To avoid that two plan shares the same column slice. diff --git a/pkg/planner/core/rule_aggregation_elimination.go b/pkg/planner/core/rule_aggregation_elimination.go index 02e806535a9f5..14f0fea535d82 100644 --- a/pkg/planner/core/rule_aggregation_elimination.go +++ b/pkg/planner/core/rule_aggregation_elimination.go @@ -158,11 +158,11 @@ func appendDistinctEliminateTraceStep(agg *LogicalAggregation, uniqueKey express // link: issue#44795 func CheckCanConvertAggToProj(agg *LogicalAggregation) bool { var mayNullSchema *expression.Schema - if join, ok := agg.Children()[0].(*LogicalJoin); ok { - if join.JoinType == LeftOuterJoin { + if join, ok := agg.Children()[0].(*logicalop.LogicalJoin); ok { + if join.JoinType == logicalop.LeftOuterJoin { mayNullSchema = join.Children()[1].Schema() } - if join.JoinType == RightOuterJoin { + if join.JoinType == logicalop.RightOuterJoin { mayNullSchema = join.Children()[0].Schema() } if mayNullSchema == nil { diff --git a/pkg/planner/core/rule_aggregation_push_down.go b/pkg/planner/core/rule_aggregation_push_down.go index bf55073000b33..93cf8a1a98983 100644 --- a/pkg/planner/core/rule_aggregation_push_down.go +++ b/pkg/planner/core/rule_aggregation_push_down.go @@ -104,7 +104,7 @@ func (*AggregationPushDownSolver) getAggFuncChildIdx(aggFunc *aggregation.AggFun // collectAggFuncs collects all aggregate functions and splits them into two parts: "leftAggFuncs" and "rightAggFuncs" whose // arguments are all from left child or right child separately. If some aggregate functions have the arguments that have // columns both from left and right children, the whole aggregation is forbidden to push down. -func (a *AggregationPushDownSolver) collectAggFuncs(agg *LogicalAggregation, join *LogicalJoin) (valid bool, leftAggFuncs, rightAggFuncs []*aggregation.AggFuncDesc) { +func (a *AggregationPushDownSolver) collectAggFuncs(agg *LogicalAggregation, join *logicalop.LogicalJoin) (valid bool, leftAggFuncs, rightAggFuncs []*aggregation.AggFuncDesc) { valid = true leftChild := join.Children()[0] rightChild := join.Children()[1] @@ -115,21 +115,21 @@ func (a *AggregationPushDownSolver) collectAggFuncs(agg *LogicalAggregation, joi index := a.getAggFuncChildIdx(aggFunc, leftChild.Schema(), rightChild.Schema()) switch index { case 0: - if join.JoinType == RightOuterJoin && !a.checkAllArgsColumn(aggFunc) { + if join.JoinType == logicalop.RightOuterJoin && !a.checkAllArgsColumn(aggFunc) { return false, nil, nil } leftAggFuncs = append(leftAggFuncs, aggFunc) case 1: - if join.JoinType == LeftOuterJoin && !a.checkAllArgsColumn(aggFunc) { + if join.JoinType == logicalop.LeftOuterJoin && !a.checkAllArgsColumn(aggFunc) { return false, nil, nil } rightAggFuncs = append(rightAggFuncs, aggFunc) case 2: // arguments are constant switch join.JoinType { - case LeftOuterJoin: + case logicalop.LeftOuterJoin: leftAggFuncs = append(leftAggFuncs, aggFunc) - case RightOuterJoin: + case logicalop.RightOuterJoin: rightAggFuncs = append(rightAggFuncs, aggFunc) default: // either left or right is fine, ideally we'd better put this to the hash build side @@ -147,7 +147,7 @@ func (a *AggregationPushDownSolver) collectAggFuncs(agg *LogicalAggregation, joi // query should be "SELECT SUM(B.agg) FROM A, (SELECT SUM(id) as agg, c1, c2, c3 FROM B GROUP BY id, c1, c2, c3) as B // WHERE A.c1 = B.c1 AND A.c2 != B.c2 GROUP BY B.c3". As you see, all the columns appearing in join-conditions should be // treated as group by columns in join subquery. -func (a *AggregationPushDownSolver) collectGbyCols(agg *LogicalAggregation, join *LogicalJoin) (leftGbyCols, rightGbyCols []*expression.Column) { +func (a *AggregationPushDownSolver) collectGbyCols(agg *LogicalAggregation, join *logicalop.LogicalJoin) (leftGbyCols, rightGbyCols []*expression.Column) { leftChild := join.Children()[0] ctx := agg.SCtx() for _, gbyExpr := range agg.GroupByItems { @@ -186,7 +186,7 @@ func (a *AggregationPushDownSolver) collectGbyCols(agg *LogicalAggregation, join return } -func (a *AggregationPushDownSolver) splitAggFuncsAndGbyCols(agg *LogicalAggregation, join *LogicalJoin) (valid bool, +func (a *AggregationPushDownSolver) splitAggFuncsAndGbyCols(agg *LogicalAggregation, join *logicalop.LogicalJoin) (valid bool, leftAggFuncs, rightAggFuncs []*aggregation.AggFuncDesc, leftGbyCols, rightGbyCols []*expression.Column) { valid, leftAggFuncs, rightAggFuncs = a.collectAggFuncs(agg, join) @@ -215,8 +215,8 @@ func (*AggregationPushDownSolver) addGbyCol(ctx base.PlanContext, gbyCols []*exp } // checkValidJoin checks if this join should be pushed across. -func (*AggregationPushDownSolver) checkValidJoin(join *LogicalJoin) bool { - return join.JoinType == InnerJoin || join.JoinType == LeftOuterJoin || join.JoinType == RightOuterJoin +func (*AggregationPushDownSolver) checkValidJoin(join *logicalop.LogicalJoin) bool { + return join.JoinType == logicalop.InnerJoin || join.JoinType == logicalop.LeftOuterJoin || join.JoinType == logicalop.RightOuterJoin } // decompose splits an aggregate function to two parts: a final mode function and a partial mode function. Currently @@ -254,13 +254,13 @@ func (*AggregationPushDownSolver) decompose(ctx base.PlanContext, aggFunc *aggre // process it temporarily. If not, We will add additional group by columns and first row functions. We make a new aggregation operator. // If the pushed aggregation is grouped by unique key, it's no need to push it down. func (a *AggregationPushDownSolver) tryToPushDownAgg(oldAgg *LogicalAggregation, aggFuncs []*aggregation.AggFuncDesc, gbyCols []*expression.Column, - join *LogicalJoin, childIdx int, blockOffset int, opt *optimizetrace.LogicalOptimizeOp) (_ base.LogicalPlan, err error) { + join *logicalop.LogicalJoin, childIdx int, blockOffset int, opt *optimizetrace.LogicalOptimizeOp) (_ base.LogicalPlan, err error) { child := join.Children()[childIdx] if aggregation.IsAllFirstRow(aggFuncs) { return child, nil } // If the join is multiway-join, we forbid pushing down. - if _, ok := join.Children()[childIdx].(*LogicalJoin); ok { + if _, ok := join.Children()[childIdx].(*logicalop.LogicalJoin); ok { return child, nil } tmpSchema := expression.NewSchema(gbyCols...) @@ -269,8 +269,8 @@ func (a *AggregationPushDownSolver) tryToPushDownAgg(oldAgg *LogicalAggregation, return child, nil } } - nullGenerating := (join.JoinType == LeftOuterJoin && childIdx == 1) || - (join.JoinType == RightOuterJoin && childIdx == 0) + nullGenerating := (join.JoinType == logicalop.LeftOuterJoin && childIdx == 1) || + (join.JoinType == logicalop.RightOuterJoin && childIdx == 0) agg, err := a.makeNewAgg(join.SCtx(), aggFuncs, gbyCols, oldAgg.PreferAggType, oldAgg.PreferAggToCop, blockOffset, nullGenerating) if err != nil { return nil, err @@ -283,7 +283,7 @@ func (a *AggregationPushDownSolver) tryToPushDownAgg(oldAgg *LogicalAggregation, Value: types.NewDatum(0), RetType: types.NewFieldType(mysql.TypeLong)}} } - if (childIdx == 0 && join.JoinType == RightOuterJoin) || (childIdx == 1 && join.JoinType == LeftOuterJoin) { + if (childIdx == 0 && join.JoinType == logicalop.RightOuterJoin) || (childIdx == 1 && join.JoinType == logicalop.LeftOuterJoin) { var existsDefaultValues bool join.DefaultValues, existsDefaultValues = a.getDefaultValues(agg) if !existsDefaultValues { @@ -491,7 +491,7 @@ func (a *AggregationPushDownSolver) aggPushDown(p base.LogicalPlan, opt *optimiz // For example, we can optimize 'select sum(a.id) from t as a,t as b where a.id = b.id;' as // 'select sum(agg) from (select sum(id) as agg,id from t group by id) as a, t as b where a.id = b.id;' // by pushing down sum aggregation functions. - if join, ok1 := child.(*LogicalJoin); ok1 && a.checkValidJoin(join) && p.SCtx().GetSessionVars().AllowAggPushDown { + if join, ok1 := child.(*logicalop.LogicalJoin); ok1 && a.checkValidJoin(join) && p.SCtx().GetSessionVars().AllowAggPushDown { if valid, leftAggFuncs, rightAggFuncs, leftGbyCols, rightGbyCols := a.splitAggFuncsAndGbyCols(agg, join); valid { var lChild, rChild base.LogicalPlan // If there exist count or sum functions in left join path, we can't push any @@ -516,9 +516,9 @@ func (a *AggregationPushDownSolver) aggPushDown(p base.LogicalPlan, opt *optimiz } join.SetChildren(lChild, rChild) join.SetSchema(expression.MergeSchema(lChild.Schema(), rChild.Schema())) - if join.JoinType == LeftOuterJoin { + if join.JoinType == logicalop.LeftOuterJoin { util.ResetNotNullFlag(join.Schema(), lChild.Schema().Len(), join.Schema().Len()) - } else if join.JoinType == RightOuterJoin { + } else if join.JoinType == logicalop.RightOuterJoin { util.ResetNotNullFlag(join.Schema(), 0, lChild.Schema().Len()) } ruleutil.BuildKeyInfoPortal(join) @@ -691,7 +691,7 @@ func (*AggregationPushDownSolver) Name() string { return "aggregation_push_down" } -func appendAggPushDownAcrossJoinTraceStep(oldAgg, newAgg *LogicalAggregation, aggFuncs []*aggregation.AggFuncDesc, join *LogicalJoin, +func appendAggPushDownAcrossJoinTraceStep(oldAgg, newAgg *LogicalAggregation, aggFuncs []*aggregation.AggFuncDesc, join *logicalop.LogicalJoin, childIdx int, opt *optimizetrace.LogicalOptimizeOp) { evalCtx := oldAgg.SCtx().GetExprCtx().GetEvalCtx() reason := func() string { diff --git a/pkg/planner/core/rule_constant_propagation.go b/pkg/planner/core/rule_constant_propagation.go index c9a883dfe5a8f..fb5273e49a530 100644 --- a/pkg/planner/core/rule_constant_propagation.go +++ b/pkg/planner/core/rule_constant_propagation.go @@ -17,8 +17,6 @@ package core import ( "context" - "github.com/pingcap/tidb/pkg/expression" - "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/util/optimizetrace" ) @@ -87,26 +85,3 @@ func (cp *ConstantPropagationSolver) execOptimize(currentPlan base.LogicalPlan, func (*ConstantPropagationSolver) Name() string { return "constant_propagation" } - -// validComparePredicate checks if the predicate is an expression like [column '>'|'>='|'<'|'<='|'=' constant]. -// return param1: return true, if the predicate is a compare constant predicate. -// return param2: return the column side of predicate. -func validCompareConstantPredicate(ctx expression.EvalContext, candidatePredicate expression.Expression) bool { - scalarFunction, ok := candidatePredicate.(*expression.ScalarFunction) - if !ok { - return false - } - if scalarFunction.FuncName.L != ast.GT && scalarFunction.FuncName.L != ast.GE && - scalarFunction.FuncName.L != ast.LT && scalarFunction.FuncName.L != ast.LE && - scalarFunction.FuncName.L != ast.EQ { - return false - } - column, _ := expression.ValidCompareConstantPredicateHelper(ctx, scalarFunction, true) - if column == nil { - column, _ = expression.ValidCompareConstantPredicateHelper(ctx, scalarFunction, false) - } - if column == nil { - return false - } - return true -} diff --git a/pkg/planner/core/rule_decorrelate.go b/pkg/planner/core/rule_decorrelate.go index 634f435b3e006..15b977a3af870 100644 --- a/pkg/planner/core/rule_decorrelate.go +++ b/pkg/planner/core/rule_decorrelate.go @@ -137,7 +137,7 @@ func (s *DecorrelateSolver) Optimize(ctx context.Context, p base.LogicalPlan, op appendApplySimplifiedTraceStep(apply, join, opt) } else if apply.NoDecorrelate { goto NoOptimize - } else if sel, ok := innerPlan.(*LogicalSelection); ok { + } else if sel, ok := innerPlan.(*logicalop.LogicalSelection); ok { // If the inner plan is a selection, we add this condition to join predicates. // Notice that no matter what kind of join is, it's always right. newConds := make([]expression.Expression, 0, len(sel.Conditions)) @@ -166,7 +166,7 @@ func (s *DecorrelateSolver) Optimize(ctx context.Context, p base.LogicalPlan, op break } } - if allConst && apply.JoinType == LeftOuterJoin { + if allConst && apply.JoinType == logicalop.LeftOuterJoin { // If the projection just references some constant. We cannot directly pull it up when the APPLY is an outer join. // e.g. select (select 1 from t1 where t1.a=t2.a) from t2; When the t1.a=t2.a is false the join's output is NULL. // But if we pull the projection upon the APPLY. It will return 1 since the projection is evaluated after the join. @@ -198,7 +198,7 @@ func (s *DecorrelateSolver) Optimize(ctx context.Context, p base.LogicalPlan, op innerPlan = proj.Children()[0] apply.SetChildren(outerPlan, innerPlan) - if apply.JoinType != SemiJoin && apply.JoinType != LeftOuterSemiJoin && apply.JoinType != AntiSemiJoin && apply.JoinType != AntiLeftOuterSemiJoin { + if apply.JoinType != logicalop.SemiJoin && apply.JoinType != logicalop.LeftOuterSemiJoin && apply.JoinType != logicalop.AntiSemiJoin && apply.JoinType != logicalop.AntiLeftOuterSemiJoin { proj.SetSchema(apply.Schema()) proj.Exprs = append(expression.Column2Exprs(outerPlan.Schema().Clone().Columns), proj.Exprs...) apply.SetSchema(expression.MergeSchema(outerPlan.Schema(), innerPlan.Schema())) @@ -216,7 +216,7 @@ func (s *DecorrelateSolver) Optimize(ctx context.Context, p base.LogicalPlan, op // The presence of 'limit' in 'exists' will make the plan not optimal, so we need to decorrelate the 'limit' of subquery in optimization. // e.g. select count(*) from test t1 where exists (select value from test t2 where t1.id = t2.id limit 1); When using 'limit' in subquery, the plan will not optimal. // If apply is not SemiJoin, the output of it might be expanded even though we are `limit 1`. - if apply.JoinType != SemiJoin && apply.JoinType != LeftOuterSemiJoin && apply.JoinType != AntiSemiJoin && apply.JoinType != AntiLeftOuterSemiJoin { + if apply.JoinType != logicalop.SemiJoin && apply.JoinType != logicalop.LeftOuterSemiJoin && apply.JoinType != logicalop.AntiSemiJoin && apply.JoinType != logicalop.AntiLeftOuterSemiJoin { goto NoOptimize } // If subquery has some filter condition, we will not optimize limit. @@ -233,7 +233,7 @@ func (s *DecorrelateSolver) Optimize(ctx context.Context, p base.LogicalPlan, op } else if agg, ok := innerPlan.(*LogicalAggregation); ok { if apply.CanPullUpAgg() && agg.canPullUp() { innerPlan = agg.Children()[0] - apply.JoinType = LeftOuterJoin + apply.JoinType = logicalop.LeftOuterJoin apply.SetChildren(outerPlan, innerPlan) agg.SetSchema(apply.Schema()) agg.GroupByItems = expression.Column2Exprs(outerPlan.Schema().Keys[0]) @@ -290,7 +290,7 @@ func (s *DecorrelateSolver) Optimize(ctx context.Context, p base.LogicalPlan, op } // We can pull up the equal conditions below the aggregation as the join key of the apply, if only // the equal conditions contain the correlated column of this apply. - if sel, ok := agg.Children()[0].(*LogicalSelection); ok && apply.JoinType == LeftOuterJoin { + if sel, ok := agg.Children()[0].(*logicalop.LogicalSelection); ok && apply.JoinType == logicalop.LeftOuterJoin { var ( eqCondWithCorCol []*expression.ScalarFunction remainedExpr []expression.Expression @@ -393,7 +393,7 @@ func (*DecorrelateSolver) Name() string { return "decorrelate" } -func appendApplySimplifiedTraceStep(p *LogicalApply, j *LogicalJoin, opt *optimizetrace.LogicalOptimizeOp) { +func appendApplySimplifiedTraceStep(p *LogicalApply, j *logicalop.LogicalJoin, opt *optimizetrace.LogicalOptimizeOp) { action := func() string { return fmt.Sprintf("%v_%v simplified into %v_%v", plancodec.TypeApply, p.ID(), plancodec.TypeJoin, j.ID()) } @@ -403,7 +403,7 @@ func appendApplySimplifiedTraceStep(p *LogicalApply, j *LogicalJoin, opt *optimi opt.AppendStepToCurrent(p.ID(), p.TP(), reason, action) } -func appendRemoveSelectionTraceStep(p base.LogicalPlan, s *LogicalSelection, opt *optimizetrace.LogicalOptimizeOp) { +func appendRemoveSelectionTraceStep(p base.LogicalPlan, s *logicalop.LogicalSelection, opt *optimizetrace.LogicalOptimizeOp) { action := func() string { return fmt.Sprintf("%v_%v removed from plan tree", s.TP(), s.ID()) } @@ -470,7 +470,7 @@ func appendPullUpAggTraceStep(p *LogicalApply, np base.LogicalPlan, agg *Logical } reason := func() string { return fmt.Sprintf("%v_%v's functions haven't any group by items and %v_%v's join type isn't %v or %v, and hasn't any conditions", - agg.TP(), agg.ID(), p.TP(), p.ID(), InnerJoin.String(), LeftOuterJoin.String()) + agg.TP(), agg.ID(), p.TP(), p.ID(), logicalop.InnerJoin.String(), logicalop.LeftOuterJoin.String()) } opt.AppendStepToCurrent(agg.ID(), agg.TP(), reason, action) } @@ -485,7 +485,7 @@ func appendAddProjTraceStep(p *LogicalApply, proj *logicalop.LogicalProjection, opt.AppendStepToCurrent(proj.ID(), proj.TP(), reason, action) } -func appendModifyAggTraceStep(outerPlan base.LogicalPlan, p *LogicalApply, agg *LogicalAggregation, sel *LogicalSelection, +func appendModifyAggTraceStep(outerPlan base.LogicalPlan, p *LogicalApply, agg *LogicalAggregation, sel *logicalop.LogicalSelection, appendedGroupByCols *expression.Schema, appendedAggFuncs []*aggregation.AggFuncDesc, eqCondWithCorCol []*expression.ScalarFunction, opt *optimizetrace.LogicalOptimizeOp) { evalCtx := outerPlan.SCtx().GetExprCtx().GetEvalCtx() diff --git a/pkg/planner/core/rule_derive_topn_from_window.go b/pkg/planner/core/rule_derive_topn_from_window.go index 194e150ef6e3f..4192389b8e9e0 100644 --- a/pkg/planner/core/rule_derive_topn_from_window.go +++ b/pkg/planner/core/rule_derive_topn_from_window.go @@ -16,7 +16,6 @@ package core import ( "context" - "fmt" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/kv" @@ -30,17 +29,6 @@ import ( type DeriveTopNFromWindow struct { } -func appendDerivedTopNTrace(topN base.LogicalPlan, opt *optimizetrace.LogicalOptimizeOp) { - child := topN.Children()[0] - action := func() string { - return fmt.Sprintf("%v_%v top N added below %v_%v ", topN.TP(), topN.ID(), child.TP(), child.ID()) - } - reason := func() string { - return fmt.Sprintf("%v filter on row number", topN.TP()) - } - opt.AppendStepToCurrent(topN.ID(), topN.TP(), reason, action) -} - // checkPartitionBy mainly checks if partition by of window function is a prefix of // data order (clustered index) of the data source. TiFlash is allowed only for empty partition by. func checkPartitionBy(p *logicalop.LogicalWindow, d *DataSource) bool { @@ -74,7 +62,8 @@ func checkPartitionBy(p *logicalop.LogicalWindow, d *DataSource) bool { current row is only frame applicable to row number - Child is a data source with no tiflash option. */ -func windowIsTopN(p *LogicalSelection) (bool, uint64) { +func windowIsTopN(lp base.LogicalPlan) (bool, uint64) { + p := lp.(*logicalop.LogicalSelection) // Check if child is window function. child, isLogicalWindow := p.Children()[0].(*logicalop.LogicalWindow) if !isLogicalWindow { diff --git a/pkg/planner/core/rule_eliminate_projection.go b/pkg/planner/core/rule_eliminate_projection.go index 0a082bc106ac5..be0e0b0f37ff1 100644 --- a/pkg/planner/core/rule_eliminate_projection.go +++ b/pkg/planner/core/rule_eliminate_projection.go @@ -183,10 +183,10 @@ func (pe *ProjectionEliminator) eliminate(p base.LogicalPlan, replace map[string // replace logical plan schema switch x := p.(type) { - case *LogicalJoin: - x.SetSchema(buildLogicalJoinSchema(x.JoinType, x)) + case *logicalop.LogicalJoin: + x.SetSchema(logicalop.BuildLogicalJoinSchema(x.JoinType, x)) case *LogicalApply: - x.SetSchema(buildLogicalJoinSchema(x.JoinType, x)) + x.SetSchema(logicalop.BuildLogicalJoinSchema(x.JoinType, x)) default: for _, dst := range p.Schema().Columns { ruleutil.ResolveColumnAndReplace(dst, replace) diff --git a/pkg/planner/core/rule_generate_column_substitute.go b/pkg/planner/core/rule_generate_column_substitute.go index d4b0115225687..97274eb4b03c0 100644 --- a/pkg/planner/core/rule_generate_column_substitute.go +++ b/pkg/planner/core/rule_generate_column_substitute.go @@ -186,7 +186,7 @@ func (gc *GcSubstituter) substitute(ctx context.Context, lp base.LogicalPlan, ex var tp types.EvalType ectx := lp.SCtx().GetExprCtx().GetEvalCtx() switch x := lp.(type) { - case *LogicalSelection: + case *logicalop.LogicalSelection: for _, cond := range x.Conditions { substituteExpression(cond, lp, exprToColumn, x.Schema(), opt) } diff --git a/pkg/planner/core/rule_generate_column_substitute_test.go b/pkg/planner/core/rule_generate_column_substitute_test.go index 774d9cf79e5d0..f86aefd88f63e 100644 --- a/pkg/planner/core/rule_generate_column_substitute_test.go +++ b/pkg/planner/core/rule_generate_column_substitute_test.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/planner/core/base" + "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/testkit" "github.com/stretchr/testify/require" ) @@ -276,7 +277,7 @@ func BenchmarkSubstituteExpression(b *testing.B) { b.ResetTimer() b.StartTimer() for i := 0; i < b.N; i++ { - core.SubstituteExpression(selection.(*core.LogicalSelection).Conditions[0], selection, m, selection.Schema(), nil) + core.SubstituteExpression(selection.(*logicalop.LogicalSelection).Conditions[0], selection, m, selection.Schema(), nil) } b.StopTimer() } diff --git a/pkg/planner/core/rule_join_elimination.go b/pkg/planner/core/rule_join_elimination.go index 16b9183f25352..334d8efbbe621 100644 --- a/pkg/planner/core/rule_join_elimination.go +++ b/pkg/planner/core/rule_join_elimination.go @@ -39,12 +39,12 @@ type OuterJoinEliminator struct { // 2. outer join elimination with duplicate agnostic aggregate functions: For example left outer join. // If the parent only use the columns from left table with 'distinct' label. The left outer join can // be eliminated. -func (o *OuterJoinEliminator) tryToEliminateOuterJoin(p *LogicalJoin, aggCols []*expression.Column, parentCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, bool, error) { +func (o *OuterJoinEliminator) tryToEliminateOuterJoin(p *logicalop.LogicalJoin, aggCols []*expression.Column, parentCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, bool, error) { var innerChildIdx int switch p.JoinType { - case LeftOuterJoin: + case logicalop.LeftOuterJoin: innerChildIdx = 1 - case RightOuterJoin: + case logicalop.RightOuterJoin: innerChildIdx = 0 default: return p, false, nil @@ -97,7 +97,7 @@ func (o *OuterJoinEliminator) tryToEliminateOuterJoin(p *LogicalJoin, aggCols [] } // extract join keys as a schema for inner child of a outer join -func (*OuterJoinEliminator) extractInnerJoinKeys(join *LogicalJoin, innerChildIdx int) *expression.Schema { +func (*OuterJoinEliminator) extractInnerJoinKeys(join *logicalop.LogicalJoin, innerChildIdx int) *expression.Schema { joinKeys := make([]*expression.Column, 0, len(join.EqualConditions)) for _, eqCond := range join.EqualConditions { joinKeys = append(joinKeys, eqCond.GetArgs()[innerChildIdx].(*expression.Column)) @@ -203,7 +203,7 @@ func (o *OuterJoinEliminator) doOptimize(p base.LogicalPlan, aggCols []*expressi } var err error var isEliminated bool - for join, isJoin := p.(*LogicalJoin); isJoin; join, isJoin = p.(*LogicalJoin) { + for join, isJoin := p.(*logicalop.LogicalJoin); isJoin; join, isJoin = p.(*logicalop.LogicalJoin) { p, isEliminated, err = o.tryToEliminateOuterJoin(join, aggCols, parentCols, opt) if err != nil { return p, err @@ -262,7 +262,7 @@ func (*OuterJoinEliminator) Name() string { return "outer_join_eliminate" } -func appendOuterJoinEliminateTraceStep(join *LogicalJoin, outerPlan base.LogicalPlan, parentCols []*expression.Column, +func appendOuterJoinEliminateTraceStep(join *logicalop.LogicalJoin, outerPlan base.LogicalPlan, parentCols []*expression.Column, innerJoinKeys *expression.Schema, opt *optimizetrace.LogicalOptimizeOp) { ectx := join.SCtx().GetExprCtx().GetEvalCtx() reason := func() string { @@ -289,7 +289,7 @@ func appendOuterJoinEliminateTraceStep(join *LogicalJoin, outerPlan base.Logical opt.AppendStepToCurrent(join.ID(), join.TP(), reason, action) } -func appendOuterJoinEliminateAggregationTraceStep(join *LogicalJoin, outerPlan base.LogicalPlan, aggCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) { +func appendOuterJoinEliminateAggregationTraceStep(join *logicalop.LogicalJoin, outerPlan base.LogicalPlan, aggCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) { ectx := join.SCtx().GetExprCtx().GetEvalCtx() reason := func() string { buffer := bytes.NewBufferString("The columns[") diff --git a/pkg/planner/core/rule_join_reorder.go b/pkg/planner/core/rule_join_reorder.go index 8ee1672787c88..93896039eb145 100644 --- a/pkg/planner/core/rule_join_reorder.go +++ b/pkg/planner/core/rule_join_reorder.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" + "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/planner/util/optimizetrace" h "github.com/pingcap/tidb/pkg/util/hint" "github.com/pingcap/tidb/pkg/util/plancodec" @@ -46,7 +47,7 @@ func extractJoinGroup(p base.LogicalPlan) *joinGroupResult { joinTypes []*joinTypeWithExtMsg hasOuterJoin bool ) - join, isJoin := p.(*LogicalJoin) + join, isJoin := p.(*logicalop.LogicalJoin) if isJoin && join.PreferJoinOrder { // When there is a leading hint, the hint may not take effect for other reasons. // For example, the join type is cross join or straight join, or exists the join algorithm hint, etc. @@ -55,8 +56,8 @@ func extractJoinGroup(p base.LogicalPlan) *joinGroupResult { } // If the variable `tidb_opt_advanced_join_hint` is false and the join node has the join method hint, we will not split the current join node to join reorder process. if !isJoin || (join.PreferJoinType > uint(0) && !p.SCtx().GetSessionVars().EnableAdvancedJoinHint) || join.StraightJoin || - (join.JoinType != InnerJoin && join.JoinType != LeftOuterJoin && join.JoinType != RightOuterJoin) || - ((join.JoinType == LeftOuterJoin || join.JoinType == RightOuterJoin) && join.EqualConditions == nil) { + (join.JoinType != logicalop.InnerJoin && join.JoinType != logicalop.LeftOuterJoin && join.JoinType != logicalop.RightOuterJoin) || + ((join.JoinType == logicalop.LeftOuterJoin || join.JoinType == logicalop.RightOuterJoin) && join.EqualConditions == nil) { if joinOrderHintInfo != nil { // The leading hint can not work for some reasons. So clear it in the join node. join.HintInfo = nil @@ -68,7 +69,7 @@ func extractJoinGroup(p base.LogicalPlan) *joinGroupResult { } } // If the session var is set to off, we will still reject the outer joins. - if !p.SCtx().GetSessionVars().EnableOuterJoinReorder && (join.JoinType == LeftOuterJoin || join.JoinType == RightOuterJoin) { + if !p.SCtx().GetSessionVars().EnableOuterJoinReorder && (join.JoinType == logicalop.LeftOuterJoin || join.JoinType == logicalop.RightOuterJoin) { return &joinGroupResult{ group: []base.LogicalPlan{p}, joinOrderHintInfo: joinOrderHintInfo, @@ -88,16 +89,16 @@ func extractJoinGroup(p base.LogicalPlan) *joinGroupResult { rightHasHint = true } } - hasOuterJoin = hasOuterJoin || (join.JoinType != InnerJoin) + hasOuterJoin = hasOuterJoin || (join.JoinType != logicalop.InnerJoin) // If the left child has the hint, it means there are some join method hints want to specify the join method based on the left child. // For example: `select .. from t1 join t2 join (select .. from t3 join t4) t5 where ..;` If there are some join method hints related to `t5`, we can't split `t5` into `t3` and `t4`. // So we don't need to split the left child part. The right child part is the same. - if join.JoinType != RightOuterJoin && !leftHasHint { + if join.JoinType != logicalop.RightOuterJoin && !leftHasHint { lhsJoinGroupResult := extractJoinGroup(join.Children()[0]) lhsGroup, lhsEqualConds, lhsOtherConds, lhsJoinTypes, lhsJoinOrderHintInfo, lhsJoinMethodHintInfo, lhsHasOuterJoin := lhsJoinGroupResult.group, lhsJoinGroupResult.eqEdges, lhsJoinGroupResult.otherConds, lhsJoinGroupResult.joinTypes, lhsJoinGroupResult.joinOrderHintInfo, lhsJoinGroupResult.joinMethodHintInfo, lhsJoinGroupResult.hasOuterJoin noExpand := false // If the filters of the outer join is related with multiple leaves of the outer join side. We don't reorder it for now. - if join.JoinType == LeftOuterJoin { + if join.JoinType == logicalop.LeftOuterJoin { extractedCols := make([]*expression.Column, 0, 8) extractedCols = expression.ExtractColumnsFromExpressions(extractedCols, join.OtherConditions, nil) extractedCols = expression.ExtractColumnsFromExpressions(extractedCols, join.LeftConditions, nil) @@ -136,12 +137,12 @@ func extractJoinGroup(p base.LogicalPlan) *joinGroupResult { } // You can see the comments in the upside part which we try to split the left child part. It's the same here. - if join.JoinType != LeftOuterJoin && !rightHasHint { + if join.JoinType != logicalop.LeftOuterJoin && !rightHasHint { rhsJoinGroupResult := extractJoinGroup(join.Children()[1]) rhsGroup, rhsEqualConds, rhsOtherConds, rhsJoinTypes, rhsJoinOrderHintInfo, rhsJoinMethodHintInfo, rhsHasOuterJoin := rhsJoinGroupResult.group, rhsJoinGroupResult.eqEdges, rhsJoinGroupResult.otherConds, rhsJoinGroupResult.joinTypes, rhsJoinGroupResult.joinOrderHintInfo, rhsJoinGroupResult.joinMethodHintInfo, rhsJoinGroupResult.hasOuterJoin noExpand := false // If the filters of the outer join is related with multiple leaves of the outer join side. We don't reorder it for now. - if join.JoinType == RightOuterJoin { + if join.JoinType == logicalop.RightOuterJoin { extractedCols := make([]*expression.Column, 0, 8) extractedCols = expression.ExtractColumnsFromExpressions(extractedCols, join.OtherConditions, nil) extractedCols = expression.ExtractColumnsFromExpressions(extractedCols, join.RightConditions, nil) @@ -184,7 +185,7 @@ func extractJoinGroup(p base.LogicalPlan) *joinGroupResult { tmpOtherConds = append(tmpOtherConds, join.OtherConditions...) tmpOtherConds = append(tmpOtherConds, join.LeftConditions...) tmpOtherConds = append(tmpOtherConds, join.RightConditions...) - if join.JoinType == LeftOuterJoin || join.JoinType == RightOuterJoin || join.JoinType == LeftOuterSemiJoin || join.JoinType == AntiLeftOuterSemiJoin { + if join.JoinType == logicalop.LeftOuterJoin || join.JoinType == logicalop.RightOuterJoin || join.JoinType == logicalop.LeftOuterSemiJoin || join.JoinType == logicalop.AntiLeftOuterSemiJoin { for range join.EqualConditions { abType := &joinTypeWithExtMsg{JoinType: join.JoinType} // outer join's other condition should be bound with the connecting edge. @@ -222,7 +223,7 @@ type jrNode struct { } type joinTypeWithExtMsg struct { - JoinType + logicalop.JoinType outerBindCondition []expression.Expression } @@ -259,7 +260,7 @@ func (s *JoinReOrderSolver) optimizeRecursive(ctx base.PlanContext, p base.Logic // Not support outer join reorder when using the DP algorithm isSupportDP := true for _, joinType := range joinTypes { - if joinType.JoinType != InnerJoin { + if joinType.JoinType != logicalop.InnerJoin { isSupportDP = false break } @@ -414,7 +415,7 @@ func (s *baseSingleGroupJoinOrderSolver) generateLeadingJoinGroup(curJoinGroup [ for _, hintTbl := range hintInfo.LeadingJoinOrder { match := false for i, joinGroup := range leftJoinGroup { - tableAlias := extractTableAlias(joinGroup, joinGroup.QueryBlockOffset()) + tableAlias := util.ExtractTableAlias(joinGroup, joinGroup.QueryBlockOffset()) if tableAlias == nil { continue } @@ -502,7 +503,7 @@ func (s *baseSingleGroupJoinOrderSolver) baseNodeCumCost(groupNode base.LogicalP // checkConnection used to check whether two nodes have equal conditions or not. func (s *baseSingleGroupJoinOrderSolver) checkConnection(leftPlan, rightPlan base.LogicalPlan) (leftNode, rightNode base.LogicalPlan, usedEdges []*expression.ScalarFunction, joinType *joinTypeWithExtMsg) { - joinType = &joinTypeWithExtMsg{JoinType: InnerJoin} + joinType = &joinTypeWithExtMsg{JoinType: logicalop.InnerJoin} leftNode, rightNode = leftPlan, rightPlan for idx, edge := range s.eqEdges { lCol := edge.GetArgs()[0].(*expression.Column) @@ -512,7 +513,7 @@ func (s *baseSingleGroupJoinOrderSolver) checkConnection(leftPlan, rightPlan bas usedEdges = append(usedEdges, edge) } else if rightPlan.Schema().Contains(lCol) && leftPlan.Schema().Contains(rCol) { joinType = s.joinTypes[idx] - if joinType.JoinType != InnerJoin { + if joinType.JoinType != logicalop.InnerJoin { rightNode, leftNode = leftPlan, rightPlan usedEdges = append(usedEdges, edge) } else { @@ -576,7 +577,7 @@ func (s *baseSingleGroupJoinOrderSolver) makeJoin(leftPlan, rightPlan base.Logic return expression.ExprFromSchema(expr, mergedSchema) }) - if joinType.JoinType == LeftOuterJoin || joinType.JoinType == RightOuterJoin || joinType.JoinType == LeftOuterSemiJoin || joinType.JoinType == AntiLeftOuterSemiJoin { + if joinType.JoinType == logicalop.LeftOuterJoin || joinType.JoinType == logicalop.RightOuterJoin || joinType.JoinType == logicalop.LeftOuterSemiJoin || joinType.JoinType == logicalop.AntiLeftOuterSemiJoin { // the original outer join's other conditions has been bound to the outer join Edge, // these remained other condition here shouldn't be appended to it because on-mismatch // logic will produce more append-null rows which is banned in original semantic. @@ -632,7 +633,7 @@ func (s *baseSingleGroupJoinOrderSolver) makeBushyJoin(cartesianJoinGroup []base } // other conditions may be possible to exist across different cartesian join group, resolving cartesianJoin first then adding another selection. if len(s.otherConds) > 0 { - additionSelection := LogicalSelection{ + additionSelection := logicalop.LogicalSelection{ Conditions: s.otherConds, }.Init(cartesianJoinGroup[0].SCtx(), cartesianJoinGroup[0].QueryBlockOffset()) additionSelection.SetChildren(cartesianJoinGroup[0]) @@ -641,13 +642,13 @@ func (s *baseSingleGroupJoinOrderSolver) makeBushyJoin(cartesianJoinGroup []base return cartesianJoinGroup[0] } -func (s *baseSingleGroupJoinOrderSolver) newCartesianJoin(lChild, rChild base.LogicalPlan) *LogicalJoin { +func (s *baseSingleGroupJoinOrderSolver) newCartesianJoin(lChild, rChild base.LogicalPlan) *logicalop.LogicalJoin { offset := lChild.QueryBlockOffset() if offset != rChild.QueryBlockOffset() { offset = -1 } - join := LogicalJoin{ - JoinType: InnerJoin, + join := logicalop.LogicalJoin{ + JoinType: logicalop.InnerJoin, Reordered: true, }.Init(s.ctx, offset) join.SetSchema(expression.MergeSchema(lChild.Schema(), rChild.Schema())) @@ -657,7 +658,7 @@ func (s *baseSingleGroupJoinOrderSolver) newCartesianJoin(lChild, rChild base.Lo } func (s *baseSingleGroupJoinOrderSolver) newJoinWithEdges(lChild, rChild base.LogicalPlan, - eqEdges []*expression.ScalarFunction, otherConds, leftConds, rightConds []expression.Expression, joinType JoinType) base.LogicalPlan { + eqEdges []*expression.ScalarFunction, otherConds, leftConds, rightConds []expression.Expression, joinType logicalop.JoinType) base.LogicalPlan { newJoin := s.newCartesianJoin(lChild, rChild) newJoin.EqualConditions = eqEdges newJoin.OtherConditions = otherConds @@ -670,7 +671,7 @@ func (s *baseSingleGroupJoinOrderSolver) newJoinWithEdges(lChild, rChild base.Lo // setNewJoinWithHint sets the join method hint for the join node. // Before the join reorder process, we split the join node and collect the join method hint. // And we record the join method hint and reset the hint after we have finished the join reorder process. -func (s *baseSingleGroupJoinOrderSolver) setNewJoinWithHint(newJoin *LogicalJoin) { +func (s *baseSingleGroupJoinOrderSolver) setNewJoinWithHint(newJoin *logicalop.LogicalJoin) { lChild := newJoin.Children()[0] rChild := newJoin.Children()[1] if joinMethodHint, ok := s.joinMethodHintInfo[lChild.ID()]; ok { diff --git a/pkg/planner/core/rule_join_reorder_dp.go b/pkg/planner/core/rule_join_reorder_dp.go index 06d5f28f825c3..804a5714f88fd 100644 --- a/pkg/planner/core/rule_join_reorder_dp.go +++ b/pkg/planner/core/rule_join_reorder_dp.go @@ -20,12 +20,13 @@ import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/planner/core/base" + "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/util/dbterror/plannererrors" ) type joinReorderDPSolver struct { *baseSingleGroupJoinOrderSolver - newJoin func(lChild, rChild base.LogicalPlan, eqConds []*expression.ScalarFunction, otherConds, leftConds, rightConds []expression.Expression, joinType JoinType) base.LogicalPlan + newJoin func(lChild, rChild base.LogicalPlan, eqConds []*expression.ScalarFunction, otherConds, leftConds, rightConds []expression.Expression, joinType logicalop.JoinType) base.LogicalPlan } type joinGroupEqEdge struct { @@ -253,7 +254,7 @@ func (s *joinReorderDPSolver) newJoinWithEdge(leftPlan, rightPlan base.LogicalPl eqConds = append(eqConds, newSf) } } - join := s.newJoin(leftPlan, rightPlan, eqConds, otherConds, nil, nil, InnerJoin) + join := s.newJoin(leftPlan, rightPlan, eqConds, otherConds, nil, nil, logicalop.InnerJoin) _, err := join.RecursiveDeriveStats(nil) return join, err } @@ -275,7 +276,7 @@ func (s *joinReorderDPSolver) makeBushyJoin(cartesianJoinGroup []base.LogicalPla otherConds, usedOtherConds = expression.FilterOutInPlace(otherConds, func(expr expression.Expression) bool { return expression.ExprFromSchema(expr, mergedSchema) }) - resultJoinGroup = append(resultJoinGroup, s.newJoin(cartesianJoinGroup[i], cartesianJoinGroup[i+1], nil, usedOtherConds, nil, nil, InnerJoin)) + resultJoinGroup = append(resultJoinGroup, s.newJoin(cartesianJoinGroup[i], cartesianJoinGroup[i+1], nil, usedOtherConds, nil, nil, logicalop.InnerJoin)) } cartesianJoinGroup = resultJoinGroup } diff --git a/pkg/planner/core/rule_join_reorder_dp_test.go b/pkg/planner/core/rule_join_reorder_dp_test.go index 2748a977d9f1f..b859971fee316 100644 --- a/pkg/planner/core/rule_join_reorder_dp_test.go +++ b/pkg/planner/core/rule_join_reorder_dp_test.go @@ -34,7 +34,7 @@ type mockLogicalJoin struct { logicalop.LogicalSchemaProducer involvedNodeSet int statsMap map[int]*property.StatsInfo - JoinType JoinType + JoinType logicalop.JoinType } func (mj mockLogicalJoin) init(ctx base.PlanContext) *mockLogicalJoin { @@ -50,8 +50,8 @@ func (mj *mockLogicalJoin) RecursiveDeriveStats(_ [][]*expression.Column) (*prop return mj.statsMap[mj.involvedNodeSet], nil } -func newMockJoin(ctx base.PlanContext, statsMap map[int]*property.StatsInfo) func(lChild, rChild base.LogicalPlan, _ []*expression.ScalarFunction, _, _, _ []expression.Expression, joinType JoinType) base.LogicalPlan { - return func(lChild, rChild base.LogicalPlan, _ []*expression.ScalarFunction, _, _, _ []expression.Expression, joinType JoinType) base.LogicalPlan { +func newMockJoin(ctx base.PlanContext, statsMap map[int]*property.StatsInfo) func(lChild, rChild base.LogicalPlan, _ []*expression.ScalarFunction, _, _, _ []expression.Expression, joinType logicalop.JoinType) base.LogicalPlan { + return func(lChild, rChild base.LogicalPlan, _ []*expression.ScalarFunction, _, _, _ []expression.Expression, joinType logicalop.JoinType) base.LogicalPlan { retJoin := mockLogicalJoin{}.init(ctx) retJoin.SetSchema(expression.MergeSchema(lChild.Schema(), rChild.Schema())) retJoin.statsMap = statsMap diff --git a/pkg/planner/core/rule_max_min_eliminate.go b/pkg/planner/core/rule_max_min_eliminate.go index 58a509a05ebf6..ca2d37ae3c9b9 100644 --- a/pkg/planner/core/rule_max_min_eliminate.go +++ b/pkg/planner/core/rule_max_min_eliminate.go @@ -49,11 +49,11 @@ func (a *MaxMinEliminator) Optimize(_ context.Context, p base.LogicalPlan, opt * func (*MaxMinEliminator) composeAggsByInnerJoin(originAgg *LogicalAggregation, aggs []*LogicalAggregation, opt *optimizetrace.LogicalOptimizeOp) (plan base.LogicalPlan) { plan = aggs[0] sctx := plan.SCtx() - joins := make([]*LogicalJoin, 0) + joins := make([]*logicalop.LogicalJoin, 0) for i := 1; i < len(aggs); i++ { - join := LogicalJoin{JoinType: InnerJoin}.Init(sctx, plan.QueryBlockOffset()) + join := logicalop.LogicalJoin{JoinType: logicalop.InnerJoin}.Init(sctx, plan.QueryBlockOffset()) join.SetChildren(plan, aggs[i]) - join.SetSchema(buildLogicalJoinSchema(InnerJoin, join)) + join.SetSchema(logicalop.BuildLogicalJoinSchema(logicalop.InnerJoin, join)) join.CartesianJoin = true plan = join joins = append(joins, join) @@ -67,7 +67,7 @@ func (*MaxMinEliminator) composeAggsByInnerJoin(originAgg *LogicalAggregation, a // 2. the path can keep order for `col` after pushing down the conditions. func (a *MaxMinEliminator) checkColCanUseIndex(plan base.LogicalPlan, col *expression.Column, conditions []expression.Expression) bool { switch p := plan.(type) { - case *LogicalSelection: + case *logicalop.LogicalSelection: conditions = append(conditions, p.Conditions...) return a.checkColCanUseIndex(p.Children()[0], col, conditions) case *DataSource: @@ -111,10 +111,10 @@ func (a *MaxMinEliminator) checkColCanUseIndex(plan base.LogicalPlan, col *expre // because we have restricted the subPlan in `checkColCanUseIndex`. func (a *MaxMinEliminator) cloneSubPlans(plan base.LogicalPlan) base.LogicalPlan { switch p := plan.(type) { - case *LogicalSelection: + case *logicalop.LogicalSelection: newConditions := make([]expression.Expression, len(p.Conditions)) copy(newConditions, p.Conditions) - sel := LogicalSelection{Conditions: newConditions}.Init(p.SCtx(), p.QueryBlockOffset()) + sel := logicalop.LogicalSelection{Conditions: newConditions}.Init(p.SCtx(), p.QueryBlockOffset()) sel.SetChildren(a.cloneSubPlans(p.Children()[0])) return sel case *DataSource: @@ -179,13 +179,13 @@ func (*MaxMinEliminator) eliminateSingleMaxMin(agg *LogicalAggregation, opt *opt child := agg.Children()[0] ctx := agg.SCtx() - var sel *LogicalSelection + var sel *logicalop.LogicalSelection var sort *logicalop.LogicalSort // If there's no column in f.GetArgs()[0], we still need limit and read data from real table because the result should be NULL if the input is empty. if len(expression.ExtractColumns(f.Args[0])) > 0 { // If it can be NULL, we need to filter NULL out first. if !mysql.HasNotNullFlag(f.Args[0].GetType(ctx.GetExprCtx().GetEvalCtx()).GetFlag()) { - sel = LogicalSelection{}.Init(ctx, agg.QueryBlockOffset()) + sel = logicalop.LogicalSelection{}.Init(ctx, agg.QueryBlockOffset()) isNullFunc := expression.NewFunctionInternal(ctx.GetExprCtx(), ast.IsNull, types.NewFieldType(mysql.TypeTiny), f.Args[0]) notNullFunc := expression.NewFunctionInternal(ctx.GetExprCtx(), ast.UnaryNot, types.NewFieldType(mysql.TypeTiny), isNullFunc) sel.Conditions = []expression.Expression{notNullFunc} @@ -266,7 +266,7 @@ func (*MaxMinEliminator) Name() string { return "max_min_eliminate" } -func appendEliminateSingleMaxMinTrace(agg *LogicalAggregation, sel *LogicalSelection, sort *logicalop.LogicalSort, limit *logicalop.LogicalLimit, opt *optimizetrace.LogicalOptimizeOp) { +func appendEliminateSingleMaxMinTrace(agg *LogicalAggregation, sel *logicalop.LogicalSelection, sort *logicalop.LogicalSort, limit *logicalop.LogicalLimit, opt *optimizetrace.LogicalOptimizeOp) { action := func() string { buffer := bytes.NewBufferString("") if sel != nil { @@ -291,7 +291,7 @@ func appendEliminateSingleMaxMinTrace(agg *LogicalAggregation, sel *LogicalSelec opt.AppendStepToCurrent(agg.ID(), agg.TP(), reason, action) } -func appendEliminateMultiMinMaxTraceStep(originAgg *LogicalAggregation, aggs []*LogicalAggregation, joins []*LogicalJoin, opt *optimizetrace.LogicalOptimizeOp) { +func appendEliminateMultiMinMaxTraceStep(originAgg *LogicalAggregation, aggs []*LogicalAggregation, joins []*logicalop.LogicalJoin, opt *optimizetrace.LogicalOptimizeOp) { action := func() string { buffer := bytes.NewBufferString(fmt.Sprintf("%v_%v splited into [", originAgg.TP(), originAgg.ID())) for i, agg := range aggs { diff --git a/pkg/planner/core/rule_outer_to_inner_join.go b/pkg/planner/core/rule_outer_to_inner_join.go index 4ef8860014560..47cd20fd151e1 100644 --- a/pkg/planner/core/rule_outer_to_inner_join.go +++ b/pkg/planner/core/rule_outer_to_inner_join.go @@ -17,24 +17,10 @@ package core import ( "context" - "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/util/optimizetrace" ) -func mergeOnClausePredicates(p *LogicalJoin, predicates []expression.Expression) []expression.Expression { - combinedCond := make([]expression.Expression, 0, - len(p.LeftConditions)+len(p.RightConditions)+ - len(p.EqualConditions)+len(p.OtherConditions)+ - len(predicates)) - combinedCond = append(combinedCond, p.LeftConditions...) - combinedCond = append(combinedCond, p.RightConditions...) - combinedCond = append(combinedCond, expression.ScalarFuncs2Exprs(p.EqualConditions)...) - combinedCond = append(combinedCond, p.OtherConditions...) - combinedCond = append(combinedCond, predicates...) - return combinedCond -} - // ConvertOuterToInnerJoin converts outer to inner joins if the unmtaching rows are filtered. type ConvertOuterToInnerJoin struct { } diff --git a/pkg/planner/core/rule_predicate_push_down.go b/pkg/planner/core/rule_predicate_push_down.go index 3df2c9723a4bc..9f8afe5839f3a 100644 --- a/pkg/planner/core/rule_predicate_push_down.go +++ b/pkg/planner/core/rule_predicate_push_down.go @@ -22,8 +22,8 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/core/base" + "github.com/pingcap/tidb/pkg/planner/core/constraint" "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/planner/util/optimizetrace" @@ -56,145 +56,22 @@ func addSelection(p base.LogicalPlan, child base.LogicalPlan, conditions []expre } conditions = expression.PropagateConstant(p.SCtx().GetExprCtx(), conditions) // Return table dual when filter is constant false or null. - dual := Conds2TableDual(child, conditions) + dual := logicalop.Conds2TableDual(child, conditions) if dual != nil { p.Children()[chIdx] = dual - appendTableDualTraceStep(child, dual, conditions, opt) + logicalop.AppendTableDualTraceStep(child, dual, conditions, opt) return } - conditions = DeleteTrueExprs(p, conditions) + conditions = constraint.DeleteTrueExprs(p, conditions) if len(conditions) == 0 { p.Children()[chIdx] = child return } - selection := LogicalSelection{Conditions: conditions}.Init(p.SCtx(), p.QueryBlockOffset()) + selection := logicalop.LogicalSelection{Conditions: conditions}.Init(p.SCtx(), p.QueryBlockOffset()) selection.SetChildren(child) p.Children()[chIdx] = selection - appendAddSelectionTraceStep(p, child, selection, opt) -} - -func splitSetGetVarFunc(filters []expression.Expression) ([]expression.Expression, []expression.Expression) { - canBePushDown := make([]expression.Expression, 0, len(filters)) - canNotBePushDown := make([]expression.Expression, 0, len(filters)) - for _, expr := range filters { - if expression.HasGetSetVarFunc(expr) { - canNotBePushDown = append(canNotBePushDown, expr) - } else { - canBePushDown = append(canBePushDown, expr) - } - } - return canBePushDown, canNotBePushDown -} - -// DeriveOtherConditions given a LogicalJoin, check the OtherConditions to see if we can derive more -// conditions for left/right child pushdown. -func DeriveOtherConditions( - p *LogicalJoin, leftSchema *expression.Schema, rightSchema *expression.Schema, - deriveLeft bool, deriveRight bool) ( - leftCond []expression.Expression, rightCond []expression.Expression) { - isOuterSemi := (p.JoinType == LeftOuterSemiJoin) || (p.JoinType == AntiLeftOuterSemiJoin) - ctx := p.SCtx() - exprCtx := ctx.GetExprCtx() - for _, expr := range p.OtherConditions { - if deriveLeft { - leftRelaxedCond := expression.DeriveRelaxedFiltersFromDNF(exprCtx, expr, leftSchema) - if leftRelaxedCond != nil { - leftCond = append(leftCond, leftRelaxedCond) - } - notNullExpr := deriveNotNullExpr(ctx, expr, leftSchema) - if notNullExpr != nil { - leftCond = append(leftCond, notNullExpr) - } - } - if deriveRight { - rightRelaxedCond := expression.DeriveRelaxedFiltersFromDNF(exprCtx, expr, rightSchema) - if rightRelaxedCond != nil { - rightCond = append(rightCond, rightRelaxedCond) - } - // For LeftOuterSemiJoin and AntiLeftOuterSemiJoin, we can actually generate - // `col is not null` according to expressions in `OtherConditions` now, but we - // are putting column equal condition converted from `in (subq)` into - // `OtherConditions`(@sa https://github.com/pingcap/tidb/pull/9051), then it would - // cause wrong results, so we disable this optimization for outer semi joins now. - // TODO enable this optimization for outer semi joins later by checking whether - // condition in `OtherConditions` is converted from `in (subq)`. - if isOuterSemi { - continue - } - notNullExpr := deriveNotNullExpr(ctx, expr, rightSchema) - if notNullExpr != nil { - rightCond = append(rightCond, notNullExpr) - } - } - } - return -} - -// deriveNotNullExpr generates a new expression `not(isnull(col))` given `col1 op col2`, -// in which `col` is in specified schema. Caller guarantees that only one of `col1` or -// `col2` is in schema. -func deriveNotNullExpr(ctx base.PlanContext, expr expression.Expression, schema *expression.Schema) expression.Expression { - binop, ok := expr.(*expression.ScalarFunction) - if !ok || len(binop.GetArgs()) != 2 { - return nil - } - arg0, lOK := binop.GetArgs()[0].(*expression.Column) - arg1, rOK := binop.GetArgs()[1].(*expression.Column) - if !lOK || !rOK { - return nil - } - childCol := schema.RetrieveColumn(arg0) - if childCol == nil { - childCol = schema.RetrieveColumn(arg1) - } - if util.IsNullRejected(ctx, schema, expr) && !mysql.HasNotNullFlag(childCol.RetType.GetFlag()) { - return expression.BuildNotNullExpr(ctx.GetExprCtx(), childCol) - } - return nil -} - -// Conds2TableDual builds a LogicalTableDual if cond is constant false or null. -func Conds2TableDual(p base.LogicalPlan, conds []expression.Expression) base.LogicalPlan { - if len(conds) != 1 { - return nil - } - con, ok := conds[0].(*expression.Constant) - if !ok { - return nil - } - sc := p.SCtx().GetSessionVars().StmtCtx - if expression.MaybeOverOptimized4PlanCache(p.SCtx().GetExprCtx(), []expression.Expression{con}) { - return nil - } - if isTrue, err := con.Value.ToBool(sc.TypeCtxOrDefault()); (err == nil && isTrue == 0) || con.Value.IsNull() { - dual := logicalop.LogicalTableDual{}.Init(p.SCtx(), p.QueryBlockOffset()) - dual.SetSchema(p.Schema()) - return dual - } - return nil -} - -// DeleteTrueExprs deletes the surely true expressions -func DeleteTrueExprs(p base.LogicalPlan, conds []expression.Expression) []expression.Expression { - newConds := make([]expression.Expression, 0, len(conds)) - for _, cond := range conds { - con, ok := cond.(*expression.Constant) - if !ok { - newConds = append(newConds, cond) - continue - } - if expression.MaybeOverOptimized4PlanCache(p.SCtx().GetExprCtx(), []expression.Expression{con}) { - newConds = append(newConds, cond) - continue - } - sc := p.SCtx().GetSessionVars().StmtCtx - if isTrue, err := con.Value.ToBool(sc.TypeCtx()); err == nil && isTrue == 1 { - continue - } - newConds = append(newConds, cond) - } - return newConds + logicalop.AppendAddSelectionTraceStep(p, child, selection, opt) } // Name implements base.LogicalOptRule.<1st> interface. @@ -202,49 +79,6 @@ func (*PPDSolver) Name() string { return "predicate_push_down" } -func appendTableDualTraceStep(replaced base.LogicalPlan, dual base.LogicalPlan, conditions []expression.Expression, opt *optimizetrace.LogicalOptimizeOp) { - action := func() string { - return fmt.Sprintf("%v_%v is replaced by %v_%v", replaced.TP(), replaced.ID(), dual.TP(), dual.ID()) - } - ectx := replaced.SCtx().GetExprCtx().GetEvalCtx() - reason := func() string { - buffer := bytes.NewBufferString("The conditions[") - for i, cond := range conditions { - if i > 0 { - buffer.WriteString(",") - } - buffer.WriteString(cond.StringWithCtx(ectx, errors.RedactLogDisable)) - } - buffer.WriteString("] are constant false or null") - return buffer.String() - } - opt.AppendStepToCurrent(dual.ID(), dual.TP(), reason, action) -} - -func appendSelectionPredicatePushDownTraceStep(p *LogicalSelection, conditions []expression.Expression, opt *optimizetrace.LogicalOptimizeOp) { - action := func() string { - return fmt.Sprintf("%v_%v is removed", p.TP(), p.ID()) - } - reason := func() string { - return "" - } - if len(conditions) > 0 { - evalCtx := p.SCtx().GetExprCtx().GetEvalCtx() - reason = func() string { - buffer := bytes.NewBufferString("The conditions[") - for i, cond := range conditions { - if i > 0 { - buffer.WriteString(",") - } - buffer.WriteString(cond.StringWithCtx(evalCtx, errors.RedactLogDisable)) - } - fmt.Fprintf(buffer, "] in %v_%v are pushed down", p.TP(), p.ID()) - return buffer.String() - } - } - opt.AppendStepToCurrent(p.ID(), p.TP(), reason, action) -} - func appendDataSourcePredicatePushDownTraceStep(ds *DataSource, opt *optimizetrace.LogicalOptimizeOp) { if len(ds.PushedDownConds) < 1 { return @@ -267,16 +101,6 @@ func appendDataSourcePredicatePushDownTraceStep(ds *DataSource, opt *optimizetra opt.AppendStepToCurrent(ds.ID(), ds.TP(), reason, action) } -func appendAddSelectionTraceStep(p base.LogicalPlan, child base.LogicalPlan, sel *LogicalSelection, opt *optimizetrace.LogicalOptimizeOp) { - reason := func() string { - return "" - } - action := func() string { - return fmt.Sprintf("add %v_%v to connect %v_%v and %v_%v", sel.TP(), sel.ID(), p.TP(), p.ID(), child.TP(), child.ID()) - } - opt.AppendStepToCurrent(sel.ID(), sel.TP(), reason, action) -} - func (ds *DataSource) addExprPrefixCond(sc base.PlanContext, path *util.AccessPath, conds []expression.Expression) ([]expression.Expression, error) { idxCols, idxColLens := diff --git a/pkg/planner/core/rule_result_reorder.go b/pkg/planner/core/rule_result_reorder.go index 28eca5ffc5850..3b31e579b1cfd 100644 --- a/pkg/planner/core/rule_result_reorder.go +++ b/pkg/planner/core/rule_result_reorder.go @@ -103,7 +103,7 @@ func (rs *ResultReorder) injectSort(lp base.LogicalPlan) base.LogicalPlan { func (*ResultReorder) isInputOrderKeeper(lp base.LogicalPlan) bool { switch lp.(type) { - case *LogicalSelection, *logicalop.LogicalProjection, *logicalop.LogicalLimit, *logicalop.LogicalTableDual: + case *logicalop.LogicalSelection, *logicalop.LogicalProjection, *logicalop.LogicalLimit, *logicalop.LogicalTableDual: return true } return false @@ -112,7 +112,7 @@ func (*ResultReorder) isInputOrderKeeper(lp base.LogicalPlan) bool { // extractHandleCols does the best effort to get the handle column. func (rs *ResultReorder) extractHandleCol(lp base.LogicalPlan) *expression.Column { switch x := lp.(type) { - case *LogicalSelection, *logicalop.LogicalLimit: + case *logicalop.LogicalSelection, *logicalop.LogicalLimit: handleCol := rs.extractHandleCol(lp.Children()[0]) if handleCol == nil { return nil // fail to extract handle column from the child, just return nil. diff --git a/pkg/planner/core/rule_semi_join_rewrite.go b/pkg/planner/core/rule_semi_join_rewrite.go index d29d16946d7f1..44d194bcda57c 100644 --- a/pkg/planner/core/rule_semi_join_rewrite.go +++ b/pkg/planner/core/rule_semi_join_rewrite.go @@ -64,16 +64,16 @@ func (smj *SemiJoinRewriter) recursivePlan(p base.LogicalPlan) (base.LogicalPlan newChildren = append(newChildren, newChild) } p.SetChildren(newChildren...) - join, ok := p.(*LogicalJoin) + join, ok := p.(*logicalop.LogicalJoin) // If it's not a join, or not a (outer) semi join. We just return it since no optimization is needed. // Actually the check of the preferRewriteSemiJoin is a superset of checking the join type. We remain them for a better understanding. - if !ok || !(join.JoinType == SemiJoin || join.JoinType == LeftOuterSemiJoin) || (join.PreferJoinType&h.PreferRewriteSemiJoin == 0) { + if !ok || !(join.JoinType == logicalop.SemiJoin || join.JoinType == logicalop.LeftOuterSemiJoin) || (join.PreferJoinType&h.PreferRewriteSemiJoin == 0) { return p, nil } // The preferRewriteSemiJoin flag only be used here. We should reset it in order to not affect other parts. join.PreferJoinType &= ^h.PreferRewriteSemiJoin - if join.JoinType == LeftOuterSemiJoin { + if join.JoinType == logicalop.LeftOuterSemiJoin { p.SCtx().GetSessionVars().StmtCtx.SetHintWarning("SEMI_JOIN_REWRITE() is inapplicable for LeftOuterSemiJoin.") return p, nil } @@ -94,7 +94,7 @@ func (smj *SemiJoinRewriter) recursivePlan(p base.LogicalPlan) (base.LogicalPlan // But the aggregation we added may block the predicate push down since we've not maintained the functional dependency to pass the equiv class to guide the push down. // So we create a selection before we build the aggregation. if len(join.RightConditions) > 0 { - sel := LogicalSelection{Conditions: make([]expression.Expression, len(join.RightConditions))}.Init(p.SCtx(), innerChild.QueryBlockOffset()) + sel := logicalop.LogicalSelection{Conditions: make([]expression.Expression, len(join.RightConditions))}.Init(p.SCtx(), innerChild.QueryBlockOffset()) copy(sel.Conditions, join.RightConditions) sel.SetChildren(innerChild) innerChild = sel @@ -120,8 +120,8 @@ func (smj *SemiJoinRewriter) recursivePlan(p base.LogicalPlan) (base.LogicalPlan subAgg.SetSchema(expression.NewSchema(aggOutputCols...)) subAgg.buildSelfKeyInfo(subAgg.Schema()) - innerJoin := LogicalJoin{ - JoinType: InnerJoin, + innerJoin := logicalop.LogicalJoin{ + JoinType: logicalop.InnerJoin, HintInfo: join.HintInfo, PreferJoinType: join.PreferJoinType, PreferJoinOrder: join.PreferJoinOrder, diff --git a/pkg/planner/core/rule_topn_push_down.go b/pkg/planner/core/rule_topn_push_down.go index cf28cf0cfaf6c..1cb645e8b97ec 100644 --- a/pkg/planner/core/rule_topn_push_down.go +++ b/pkg/planner/core/rule_topn_push_down.go @@ -15,11 +15,9 @@ package core import ( - "bytes" "context" "fmt" - "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/planner/util/optimizetrace" @@ -58,39 +56,6 @@ func (*PushDownTopNOptimizer) Name() string { return "topn_push_down" } -func appendTopNPushDownJoinTraceStep(p *LogicalJoin, topN *logicalop.LogicalTopN, idx int, opt *optimizetrace.LogicalOptimizeOp) { - ectx := p.SCtx().GetExprCtx().GetEvalCtx() - action := func() string { - buffer := bytes.NewBufferString(fmt.Sprintf("%v_%v is added and pushed into %v_%v's ", - topN.TP(), topN.ID(), p.TP(), p.ID())) - if idx == 0 { - buffer.WriteString("left ") - } else { - buffer.WriteString("right ") - } - buffer.WriteString("table") - return buffer.String() - } - reason := func() string { - buffer := bytes.NewBufferString(fmt.Sprintf("%v_%v's joinType is %v, and all ByItems[", p.TP(), p.ID(), p.JoinType.String())) - for i, item := range topN.ByItems { - if i > 0 { - buffer.WriteString(",") - } - buffer.WriteString(item.StringWithCtx(ectx, errors.RedactLogDisable)) - } - buffer.WriteString("] contained in ") - if idx == 0 { - buffer.WriteString("left ") - } else { - buffer.WriteString("right ") - } - buffer.WriteString("table") - return buffer.String() - } - opt.AppendStepToCurrent(p.ID(), p.TP(), reason, action) -} - func appendNewTopNTraceStep(topN *logicalop.LogicalTopN, union *LogicalUnionAll, opt *optimizetrace.LogicalOptimizeOp) { reason := func() string { return "" diff --git a/pkg/planner/core/runtime_filter_generator.go b/pkg/planner/core/runtime_filter_generator.go index bfa4d4091e47d..dba6e0b59d49e 100644 --- a/pkg/planner/core/runtime_filter_generator.go +++ b/pkg/planner/core/runtime_filter_generator.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/core/base" + "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/logutil" @@ -162,8 +163,8 @@ func (generator *RuntimeFilterGenerator) assignRuntimeFilter(physicalTableScan * func (*RuntimeFilterGenerator) matchRFJoinType(hashJoinPlan *PhysicalHashJoin) bool { if hashJoinPlan.RightIsBuildSide() { // case1: build side is on the right - if hashJoinPlan.JoinType == LeftOuterJoin || hashJoinPlan.JoinType == AntiSemiJoin || - hashJoinPlan.JoinType == LeftOuterSemiJoin || hashJoinPlan.JoinType == AntiLeftOuterSemiJoin { + if hashJoinPlan.JoinType == logicalop.LeftOuterJoin || hashJoinPlan.JoinType == logicalop.AntiSemiJoin || + hashJoinPlan.JoinType == logicalop.LeftOuterSemiJoin || hashJoinPlan.JoinType == logicalop.AntiLeftOuterSemiJoin { logutil.BgLogger().Debug("Join type does not match RF pattern when build side is on the right", zap.Int32("PlanNodeId", int32(hashJoinPlan.ID())), zap.String("JoinType", hashJoinPlan.JoinType.String())) @@ -171,7 +172,7 @@ func (*RuntimeFilterGenerator) matchRFJoinType(hashJoinPlan *PhysicalHashJoin) b } } else { // case2: build side is on the left - if hashJoinPlan.JoinType == RightOuterJoin { + if hashJoinPlan.JoinType == logicalop.RightOuterJoin { logutil.BgLogger().Debug("Join type does not match RF pattern when build side is on the left", zap.Int32("PlanNodeId", int32(hashJoinPlan.ID())), zap.String("JoinType", hashJoinPlan.JoinType.String())) diff --git a/pkg/planner/core/stringer.go b/pkg/planner/core/stringer.go index 517714f0dbfe7..2273b4aed9659 100644 --- a/pkg/planner/core/stringer.go +++ b/pkg/planner/core/stringer.go @@ -73,7 +73,7 @@ func fdToString(in base.LogicalPlan, strs []string, idxs []int) ([]string, []int strs = append(strs, "{"+x.FDs().String()+"}") case *LogicalApply: strs = append(strs, "{"+x.FDs().String()+"}") - case *LogicalJoin: + case *logicalop.LogicalJoin: strs = append(strs, "{"+x.FDs().String()+"}") default: } @@ -139,19 +139,19 @@ func toString(in base.Plan, strs []string, idxs []int) ([]string, []int) { idxs = idxs[:last] id := "MergeJoin" switch x.JoinType { - case SemiJoin: + case logicalop.SemiJoin: id = "MergeSemiJoin" - case AntiSemiJoin: + case logicalop.AntiSemiJoin: id = "MergeAntiSemiJoin" - case LeftOuterSemiJoin: + case logicalop.LeftOuterSemiJoin: id = "MergeLeftOuterSemiJoin" - case AntiLeftOuterSemiJoin: + case logicalop.AntiLeftOuterSemiJoin: id = "MergeAntiLeftOuterSemiJoin" - case LeftOuterJoin: + case logicalop.LeftOuterJoin: id = "MergeLeftOuterJoin" - case RightOuterJoin: + case logicalop.RightOuterJoin: id = "MergeRightOuterJoin" - case InnerJoin: + case logicalop.InnerJoin: id = "MergeInnerJoin" } str = id + "{" + strings.Join(children, "->") + "}" @@ -189,7 +189,7 @@ func toString(in base.Plan, strs []string, idxs []int) ([]string, []int) { str = "ShowDDLJobs" case *logicalop.LogicalSort, *PhysicalSort: str = "Sort" - case *LogicalJoin: + case *logicalop.LogicalJoin: last := len(idxs) - 1 idx := idxs[last] children := strs[idx:] @@ -232,7 +232,7 @@ func toString(in base.Plan, strs []string, idxs []int) ([]string, []int) { str = fmt.Sprintf("DataScan(%s)", x.TableInfo.Name) } } - case *LogicalSelection: + case *logicalop.LogicalSelection: str = fmt.Sprintf("Sel(%s)", expression.StringifyExpressionsWithCtx(ectx, x.Conditions)) case *PhysicalSelection: str = fmt.Sprintf("Sel(%s)", expression.StringifyExpressionsWithCtx(ectx, x.Conditions)) diff --git a/pkg/planner/core/task.go b/pkg/planner/core/task.go index 9fa8b67801cdb..83018679d0d87 100644 --- a/pkg/planner/core/task.go +++ b/pkg/planner/core/task.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/core/cost" "github.com/pingcap/tidb/pkg/planner/core/operator/baseimpl" + "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/types" @@ -395,8 +396,8 @@ func (p *PhysicalHashJoin) attach2TaskForMpp(tasks ...base.Task) base.Task { // for outer join, it should always be the outer side of the join // for semi join, it should be the left side(the same as left out join) outerTaskIndex := 1 - p.InnerChildIdx - if p.JoinType != InnerJoin { - if p.JoinType == RightOuterJoin { + if p.JoinType != logicalop.InnerJoin { + if p.JoinType == logicalop.RightOuterJoin { outerTaskIndex = 1 } else { outerTaskIndex = 0 @@ -911,7 +912,7 @@ func (p *PhysicalTopN) canExpressionConvertedToPB(storeTp kv.StoreType) bool { for _, item := range p.ByItems { exprs = append(exprs, item.Expr) } - return expression.CanExprsPushDown(GetPushDownCtx(p.SCtx()), exprs, storeTp) + return expression.CanExprsPushDown(util.GetPushDownCtx(p.SCtx()), exprs, storeTp) } // containVirtualColumn checks whether TopN.ByItems contains virtual generated columns. @@ -1023,12 +1024,12 @@ func (p *PhysicalExpand) Attach2Task(tasks ...base.Task) base.Task { func (p *PhysicalProjection) Attach2Task(tasks ...base.Task) base.Task { t := tasks[0].Copy() if cop, ok := t.(*CopTask); ok { - if (len(cop.rootTaskConds) == 0 && len(cop.idxMergePartPlans) == 0) && expression.CanExprsPushDown(GetPushDownCtx(p.SCtx()), p.Exprs, cop.getStoreType()) { + if (len(cop.rootTaskConds) == 0 && len(cop.idxMergePartPlans) == 0) && expression.CanExprsPushDown(util.GetPushDownCtx(p.SCtx()), p.Exprs, cop.getStoreType()) { copTask := attachPlan2Task(p, cop) return copTask } } else if mpp, ok := t.(*MppTask); ok { - if expression.CanExprsPushDown(GetPushDownCtx(p.SCtx()), p.Exprs, kv.TiFlash) { + if expression.CanExprsPushDown(util.GetPushDownCtx(p.SCtx()), p.Exprs, kv.TiFlash) { p.SetChildren(mpp.p) mpp.p = p return mpp @@ -1088,7 +1089,7 @@ func (p *PhysicalUnionAll) Attach2Task(tasks ...base.Task) base.Task { // Attach2Task implements PhysicalPlan interface. func (sel *PhysicalSelection) Attach2Task(tasks ...base.Task) base.Task { if mppTask, _ := tasks[0].(*MppTask); mppTask != nil { // always push to mpp task. - if expression.CanExprsPushDown(GetPushDownCtx(sel.SCtx()), sel.Conditions, kv.TiFlash) { + if expression.CanExprsPushDown(util.GetPushDownCtx(sel.SCtx()), sel.Conditions, kv.TiFlash) { return attachPlan2Task(sel, mppTask.Copy()) } } @@ -1102,7 +1103,7 @@ func CheckAggCanPushCop(sctx base.PlanContext, aggFuncs []*aggregation.AggFuncDe sc := sctx.GetSessionVars().StmtCtx ret := true reason := "" - pushDownCtx := GetPushDownCtx(sctx) + pushDownCtx := util.GetPushDownCtx(sctx) for _, aggFunc := range aggFuncs { // if the aggFunc contain VirtualColumn or CorrelatedColumn, it can not be pushed down. if expression.ContainVirtualColumn(aggFunc.Args) || expression.ContainCorrelatedColumn(aggFunc.Args) { @@ -1115,7 +1116,7 @@ func CheckAggCanPushCop(sctx base.PlanContext, aggFuncs []*aggregation.AggFuncDe ret = false break } - if !expression.CanExprsPushDownWithExtraInfo(GetPushDownCtx(sctx), aggFunc.Args, storeType, aggFunc.Name == ast.AggFuncSum) { + if !expression.CanExprsPushDownWithExtraInfo(util.GetPushDownCtx(sctx), aggFunc.Args, storeType, aggFunc.Name == ast.AggFuncSum) { reason = "arguments of AggFunc `" + aggFunc.Name + "` contains unsupported exprs" ret = false break @@ -1126,7 +1127,7 @@ func CheckAggCanPushCop(sctx base.PlanContext, aggFuncs []*aggregation.AggFuncDe for _, item := range aggFunc.OrderByItems { exprs = append(exprs, item.Expr) } - if !expression.CanExprsPushDownWithExtraInfo(GetPushDownCtx(sctx), exprs, storeType, false) { + if !expression.CanExprsPushDownWithExtraInfo(util.GetPushDownCtx(sctx), exprs, storeType, false) { reason = "arguments of AggFunc `" + aggFunc.Name + "` contains unsupported exprs in order-by clause" ret = false break @@ -1143,7 +1144,7 @@ func CheckAggCanPushCop(sctx base.PlanContext, aggFuncs []*aggregation.AggFuncDe reason = "groupByItems contain virtual columns, which is not supported now" ret = false } - if ret && !expression.CanExprsPushDown(GetPushDownCtx(sctx), groupByItems, storeType) { + if ret && !expression.CanExprsPushDown(util.GetPushDownCtx(sctx), groupByItems, storeType) { reason = "groupByItems contain unsupported exprs" ret = false } diff --git a/pkg/planner/core/util.go b/pkg/planner/core/util.go index 2b522ea963499..7ad81e06cd5d3 100644 --- a/pkg/planner/core/util.go +++ b/pkg/planner/core/util.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/core/operator/baseimpl" + "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/planner/core/operator/physicalop" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/sessionctx" @@ -210,40 +211,21 @@ func (s *baseSchemaProducer) MemoryUsage() (sum int64) { return } -func buildLogicalJoinSchema(joinType JoinType, join base.LogicalPlan) *expression.Schema { - leftSchema := join.Children()[0].Schema() - switch joinType { - case SemiJoin, AntiSemiJoin: - return leftSchema.Clone() - case LeftOuterSemiJoin, AntiLeftOuterSemiJoin: - newSchema := leftSchema.Clone() - newSchema.Append(join.Schema().Columns[join.Schema().Len()-1]) - return newSchema - } - newSchema := expression.MergeSchema(leftSchema, join.Children()[1].Schema()) - if joinType == LeftOuterJoin { - util.ResetNotNullFlag(newSchema, leftSchema.Len(), newSchema.Len()) - } else if joinType == RightOuterJoin { - util.ResetNotNullFlag(newSchema, 0, leftSchema.Len()) - } - return newSchema -} - // BuildPhysicalJoinSchema builds the schema of PhysicalJoin from it's children's schema. -func BuildPhysicalJoinSchema(joinType JoinType, join base.PhysicalPlan) *expression.Schema { +func BuildPhysicalJoinSchema(joinType logicalop.JoinType, join base.PhysicalPlan) *expression.Schema { leftSchema := join.Children()[0].Schema() switch joinType { - case SemiJoin, AntiSemiJoin: + case logicalop.SemiJoin, logicalop.AntiSemiJoin: return leftSchema.Clone() - case LeftOuterSemiJoin, AntiLeftOuterSemiJoin: + case logicalop.LeftOuterSemiJoin, logicalop.AntiLeftOuterSemiJoin: newSchema := leftSchema.Clone() newSchema.Append(join.Schema().Columns[join.Schema().Len()-1]) return newSchema } newSchema := expression.MergeSchema(leftSchema, join.Children()[1].Schema()) - if joinType == LeftOuterJoin { + if joinType == logicalop.LeftOuterJoin { util.ResetNotNullFlag(newSchema, leftSchema.Len(), newSchema.Len()) - } else if joinType == RightOuterJoin { + } else if joinType == logicalop.RightOuterJoin { util.ResetNotNullFlag(newSchema, 0, leftSchema.Len()) } return newSchema @@ -428,13 +410,3 @@ func EncodeUniqueIndexValuesForKey(ctx sessionctx.Context, tblInfo *model.TableI } return encodedIdxVals, nil } - -// GetPushDownCtx creates a PushDownContext from PlanContext -func GetPushDownCtx(pctx base.PlanContext) expression.PushDownContext { - return GetPushDownCtxFromBuildPBContext(pctx.GetBuildPBCtx()) -} - -// GetPushDownCtxFromBuildPBContext creates a PushDownContext from BuildPBContext -func GetPushDownCtxFromBuildPBContext(bctx *base.BuildPBContext) expression.PushDownContext { - return expression.NewPushDownContext(bctx.GetExprCtx().GetEvalCtx(), bctx.GetClient(), bctx.InExplainStmt, bctx.WarnHandler, bctx.ExtraWarnghandler, bctx.GroupConcatMaxLen) -} diff --git a/pkg/planner/memo/expr_iterator_test.go b/pkg/planner/memo/expr_iterator_test.go index 026539df8b398..ab62157adc674 100644 --- a/pkg/planner/memo/expr_iterator_test.go +++ b/pkg/planner/memo/expr_iterator_test.go @@ -34,17 +34,17 @@ func TestNewExprIterFromGroupElem(t *testing.T) { do := domain.GetDomain(ctx) do.StatsHandle().Close() }() - g0 := NewGroupWithSchema(NewGroupExpr(plannercore.LogicalSelection{}.Init(ctx, 0)), schema) + g0 := NewGroupWithSchema(NewGroupExpr(logicalop.LogicalSelection{}.Init(ctx, 0)), schema) g0.Insert(NewGroupExpr(logicalop.LogicalLimit{}.Init(ctx, 0))) g0.Insert(NewGroupExpr(logicalop.LogicalProjection{}.Init(ctx, 0))) g0.Insert(NewGroupExpr(logicalop.LogicalLimit{}.Init(ctx, 0))) - g1 := NewGroupWithSchema(NewGroupExpr(plannercore.LogicalSelection{}.Init(ctx, 0)), schema) + g1 := NewGroupWithSchema(NewGroupExpr(logicalop.LogicalSelection{}.Init(ctx, 0)), schema) g1.Insert(NewGroupExpr(logicalop.LogicalLimit{}.Init(ctx, 0))) g1.Insert(NewGroupExpr(logicalop.LogicalProjection{}.Init(ctx, 0))) g1.Insert(NewGroupExpr(logicalop.LogicalLimit{}.Init(ctx, 0))) - expr := NewGroupExpr(plannercore.LogicalJoin{}.Init(ctx, 0)) + expr := NewGroupExpr(logicalop.LogicalJoin{}.Init(ctx, 0)) expr.Children = append(expr.Children, g0) expr.Children = append(expr.Children, g1) g2 := NewGroupWithSchema(expr, schema) @@ -88,13 +88,13 @@ func TestExprIterNext(t *testing.T) { g0.Insert(NewGroupExpr(logicalop.LogicalLimit{Count: 2}.Init(ctx, 0))) g0.Insert(NewGroupExpr(logicalop.LogicalProjection{Exprs: []expression.Expression{expression.NewNull()}}.Init(ctx, 0))) - g1 := NewGroupWithSchema(NewGroupExpr(plannercore.LogicalSelection{Conditions: []expression.Expression{expression.NewNull()}}.Init(ctx, 0)), schema) + g1 := NewGroupWithSchema(NewGroupExpr(logicalop.LogicalSelection{Conditions: []expression.Expression{expression.NewNull()}}.Init(ctx, 0)), schema) g1.Insert(NewGroupExpr(logicalop.LogicalLimit{Count: 3}.Init(ctx, 0))) - g1.Insert(NewGroupExpr(plannercore.LogicalSelection{Conditions: []expression.Expression{expression.NewOne()}}.Init(ctx, 0))) + g1.Insert(NewGroupExpr(logicalop.LogicalSelection{Conditions: []expression.Expression{expression.NewOne()}}.Init(ctx, 0))) g1.Insert(NewGroupExpr(logicalop.LogicalLimit{Count: 4}.Init(ctx, 0))) - g1.Insert(NewGroupExpr(plannercore.LogicalSelection{Conditions: []expression.Expression{expression.NewZero()}}.Init(ctx, 0))) + g1.Insert(NewGroupExpr(logicalop.LogicalSelection{Conditions: []expression.Expression{expression.NewZero()}}.Init(ctx, 0))) - expr := NewGroupExpr(plannercore.LogicalJoin{}.Init(ctx, 0)) + expr := NewGroupExpr(logicalop.LogicalJoin{}.Init(ctx, 0)) expr.Children = append(expr.Children, g0) expr.Children = append(expr.Children, g1) g2 := NewGroupWithSchema(expr, schema) @@ -141,23 +141,23 @@ func TestExprIterReset(t *testing.T) { g0.Insert(NewGroupExpr(logicalop.LogicalLimit{Count: 2}.Init(ctx, 0))) g0.Insert(NewGroupExpr(logicalop.LogicalProjection{Exprs: []expression.Expression{expression.NewNull()}}.Init(ctx, 0))) - sel1 := NewGroupExpr(plannercore.LogicalSelection{Conditions: []expression.Expression{expression.NewNull()}}.Init(ctx, 0)) - sel2 := NewGroupExpr(plannercore.LogicalSelection{Conditions: []expression.Expression{expression.NewOne()}}.Init(ctx, 0)) - sel3 := NewGroupExpr(plannercore.LogicalSelection{Conditions: []expression.Expression{expression.NewZero()}}.Init(ctx, 0)) + sel1 := NewGroupExpr(logicalop.LogicalSelection{Conditions: []expression.Expression{expression.NewNull()}}.Init(ctx, 0)) + sel2 := NewGroupExpr(logicalop.LogicalSelection{Conditions: []expression.Expression{expression.NewOne()}}.Init(ctx, 0)) + sel3 := NewGroupExpr(logicalop.LogicalSelection{Conditions: []expression.Expression{expression.NewZero()}}.Init(ctx, 0)) g1 := NewGroupWithSchema(sel1, schema) g1.Insert(NewGroupExpr(logicalop.LogicalLimit{Count: 3}.Init(ctx, 0))) g1.Insert(sel2) g1.Insert(NewGroupExpr(logicalop.LogicalLimit{Count: 4}.Init(ctx, 0))) g1.Insert(sel3) - g2 := NewGroupWithSchema(NewGroupExpr(plannercore.LogicalSelection{Conditions: []expression.Expression{expression.NewNull()}}.Init(ctx, 0)), schema) + g2 := NewGroupWithSchema(NewGroupExpr(logicalop.LogicalSelection{Conditions: []expression.Expression{expression.NewNull()}}.Init(ctx, 0)), schema) g2.Insert(NewGroupExpr(logicalop.LogicalLimit{Count: 3}.Init(ctx, 0))) - g2.Insert(NewGroupExpr(plannercore.LogicalSelection{Conditions: []expression.Expression{expression.NewOne()}}.Init(ctx, 0))) + g2.Insert(NewGroupExpr(logicalop.LogicalSelection{Conditions: []expression.Expression{expression.NewOne()}}.Init(ctx, 0))) g2.Insert(NewGroupExpr(logicalop.LogicalLimit{Count: 4}.Init(ctx, 0))) - g2.Insert(NewGroupExpr(plannercore.LogicalSelection{Conditions: []expression.Expression{expression.NewZero()}}.Init(ctx, 0))) + g2.Insert(NewGroupExpr(logicalop.LogicalSelection{Conditions: []expression.Expression{expression.NewZero()}}.Init(ctx, 0))) // link join with Group 0 and 1 - expr := NewGroupExpr(plannercore.LogicalJoin{}.Init(ctx, 0)) + expr := NewGroupExpr(logicalop.LogicalJoin{}.Init(ctx, 0)) expr.Children = append(expr.Children, g0) expr.Children = append(expr.Children, g1) g3 := NewGroupWithSchema(expr, schema) @@ -212,12 +212,12 @@ func TestExprIterWithEngineType(t *testing.T) { do := domain.GetDomain(ctx) do.StatsHandle().Close() }() - g1 := NewGroupWithSchema(NewGroupExpr(plannercore.LogicalSelection{Conditions: []expression.Expression{expression.NewOne()}}.Init(ctx, 0)), schema).SetEngineType(pattern.EngineTiFlash) + g1 := NewGroupWithSchema(NewGroupExpr(logicalop.LogicalSelection{Conditions: []expression.Expression{expression.NewOne()}}.Init(ctx, 0)), schema).SetEngineType(pattern.EngineTiFlash) g1.Insert(NewGroupExpr(logicalop.LogicalLimit{Count: 1}.Init(ctx, 0))) g1.Insert(NewGroupExpr(logicalop.LogicalProjection{Exprs: []expression.Expression{expression.NewOne()}}.Init(ctx, 0))) g1.Insert(NewGroupExpr(logicalop.LogicalLimit{Count: 2}.Init(ctx, 0))) - g2 := NewGroupWithSchema(NewGroupExpr(plannercore.LogicalSelection{Conditions: []expression.Expression{expression.NewOne()}}.Init(ctx, 0)), schema).SetEngineType(pattern.EngineTiKV) + g2 := NewGroupWithSchema(NewGroupExpr(logicalop.LogicalSelection{Conditions: []expression.Expression{expression.NewOne()}}.Init(ctx, 0)), schema).SetEngineType(pattern.EngineTiKV) g2.Insert(NewGroupExpr(logicalop.LogicalLimit{Count: 2}.Init(ctx, 0))) g2.Insert(NewGroupExpr(logicalop.LogicalProjection{Exprs: []expression.Expression{expression.NewOne()}}.Init(ctx, 0))) g2.Insert(NewGroupExpr(logicalop.LogicalLimit{Count: 3}.Init(ctx, 0))) @@ -230,7 +230,7 @@ func TestExprIterWithEngineType(t *testing.T) { tikvGather.Children = append(tikvGather.Children, g2) g3.Insert(tikvGather) - join := NewGroupExpr(plannercore.LogicalJoin{}.Init(ctx, 0)) + join := NewGroupExpr(logicalop.LogicalJoin{}.Init(ctx, 0)) join.Children = append(join.Children, g3, g3) g4 := NewGroupWithSchema(join, schema).SetEngineType(pattern.EngineTiDB) diff --git a/pkg/planner/memo/group_test.go b/pkg/planner/memo/group_test.go index a599cb27d6cc7..32ec633d48050 100644 --- a/pkg/planner/memo/group_test.go +++ b/pkg/planner/memo/group_test.go @@ -71,7 +71,7 @@ func TestGroupDeleteAll(t *testing.T) { do := domain.GetDomain(ctx) do.StatsHandle().Close() }() - expr := NewGroupExpr(plannercore.LogicalSelection{}.Init(ctx, 0)) + expr := NewGroupExpr(logicalop.LogicalSelection{}.Init(ctx, 0)) g := NewGroupWithSchema(expr, expression.NewSchema()) require.True(t, g.Insert(NewGroupExpr(logicalop.LogicalLimit{}.Init(ctx, 0)))) require.True(t, g.Insert(NewGroupExpr(logicalop.LogicalProjection{}.Init(ctx, 0)))) @@ -115,7 +115,7 @@ func TestGroupFingerPrint(t *testing.T) { // Plan tree should be: DataSource -> Selection -> Projection proj, ok := logic1.(*logicalop.LogicalProjection) require.True(t, ok) - sel, ok := logic1.Children()[0].(*plannercore.LogicalSelection) + sel, ok := logic1.Children()[0].(*logicalop.LogicalSelection) require.True(t, ok) group1 := Convert2Group(logic1) oldGroupExpr := group1.Equivalents.Front().Value.(*GroupExpr) @@ -142,7 +142,7 @@ func TestGroupFingerPrint(t *testing.T) { // Insert two LogicalSelections with same conditions but different order. require.Len(t, sel.Conditions, 2) - newSelection := plannercore.LogicalSelection{ + newSelection := logicalop.LogicalSelection{ Conditions: make([]expression.Expression, 2)}.Init(sel.SCtx(), sel.QueryBlockOffset()) newSelection.Conditions[0], newSelection.Conditions[1] = sel.Conditions[1], sel.Conditions[0] newGroupExpr4 := NewGroupExpr(sel) @@ -262,7 +262,7 @@ func TestBuildKeyInfo(t *testing.T) { require.Len(t, group2.Prop.Schema.Keys, 1) // case 3: build key info for new Group - newSel := plannercore.LogicalSelection{}.Init(ctx, 0) + newSel := logicalop.LogicalSelection{}.Init(ctx, 0) newExpr1 := NewGroupExpr(newSel) newExpr1.SetChildren(group2) newGroup1 := NewGroupWithSchema(newExpr1, group2.Prop.Schema) diff --git a/pkg/planner/pattern/pattern.go b/pkg/planner/pattern/pattern.go index 6ac6de67bcb32..c21503139cb22 100644 --- a/pkg/planner/pattern/pattern.go +++ b/pkg/planner/pattern/pattern.go @@ -79,13 +79,13 @@ func GetOperand(p base.LogicalPlan) Operand { switch p.(type) { case *plannercore.LogicalApply: return OperandApply - case *plannercore.LogicalJoin: + case *logicalop.LogicalJoin: return OperandJoin case *plannercore.LogicalAggregation: return OperandAggregation case *logicalop.LogicalProjection: return OperandProjection - case *plannercore.LogicalSelection: + case *logicalop.LogicalSelection: return OperandSelection case *logicalop.LogicalMaxOneRow: return OperandMaxOneRow diff --git a/pkg/planner/pattern/pattern_test.go b/pkg/planner/pattern/pattern_test.go index b579b95c8d8ba..707a65b66a73f 100644 --- a/pkg/planner/pattern/pattern_test.go +++ b/pkg/planner/pattern/pattern_test.go @@ -23,10 +23,10 @@ import ( ) func TestGetOperand(t *testing.T) { - require.Equal(t, OperandJoin, GetOperand(&plannercore.LogicalJoin{})) + require.Equal(t, OperandJoin, GetOperand(&logicalop.LogicalJoin{})) require.Equal(t, OperandAggregation, GetOperand(&plannercore.LogicalAggregation{})) require.Equal(t, OperandProjection, GetOperand(&logicalop.LogicalProjection{})) - require.Equal(t, OperandSelection, GetOperand(&plannercore.LogicalSelection{})) + require.Equal(t, OperandSelection, GetOperand(&logicalop.LogicalSelection{})) require.Equal(t, OperandApply, GetOperand(&plannercore.LogicalApply{})) require.Equal(t, OperandMaxOneRow, GetOperand(&logicalop.LogicalMaxOneRow{})) require.Equal(t, OperandTableDual, GetOperand(&logicalop.LogicalTableDual{})) diff --git a/pkg/planner/util/BUILD.bazel b/pkg/planner/util/BUILD.bazel index 57f53ce1d9e18..8500f83dc55bd 100644 --- a/pkg/planner/util/BUILD.bazel +++ b/pkg/planner/util/BUILD.bazel @@ -6,6 +6,7 @@ go_library( "byitem.go", "explain_misc.go", "expression.go", + "funcdep_misc.go", "handle_cols.go", "misc.go", "null_misc.go", @@ -21,6 +22,7 @@ go_library( "//pkg/parser/mysql", "//pkg/planner/context", "//pkg/planner/core/base", + "//pkg/planner/funcdep", "//pkg/planner/property", "//pkg/sessionctx/stmtctx", "//pkg/tablecodec", @@ -28,6 +30,8 @@ go_library( "//pkg/util/chunk", "//pkg/util/codec", "//pkg/util/collate", + "//pkg/util/hint", + "//pkg/util/intset", "//pkg/util/ranger", "//pkg/util/size", "@com_github_pingcap_errors//:errors", diff --git a/pkg/planner/util/funcdep_misc.go b/pkg/planner/util/funcdep_misc.go new file mode 100644 index 0000000000000..ed2fc849fb4b6 --- /dev/null +++ b/pkg/planner/util/funcdep_misc.go @@ -0,0 +1,123 @@ +// Copyright 2024 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package util + +import ( + "github.com/pingcap/tidb/pkg/expression" + "github.com/pingcap/tidb/pkg/planner/core/base" + "github.com/pingcap/tidb/pkg/planner/funcdep" + "github.com/pingcap/tidb/pkg/util/intset" +) + +// ExtractNotNullFromConds extracts not-null columns from conditions. +func ExtractNotNullFromConds(conditions []expression.Expression, p base.LogicalPlan) intset.FastIntSet { + // extract the column NOT NULL rejection characteristic from selection condition. + // CNF considered only, DNF doesn't have its meanings (cause that condition's eval may don't take effect) + // + // Take this case: select * from t where (a = 1) and (b is null): + // + // If we wanna where phrase eval to true, two pre-condition: {a=1} and {b is null} both need to be true. + // Hence, we assert that: + // + // 1: `a` must not be null since `NULL = 1` is evaluated as NULL. + // 2: `b` must be null since only `NULL is NULL` is evaluated as true. + // + // As a result, `a` will be extracted as not-null column to abound the FDSet. + notnullColsUniqueIDs := intset.NewFastIntSet() + for _, condition := range conditions { + var cols []*expression.Column + cols = expression.ExtractColumnsFromExpressions(cols, []expression.Expression{condition}, nil) + if IsNullRejected(p.SCtx(), p.Schema(), condition) { + for _, col := range cols { + notnullColsUniqueIDs.Insert(int(col.UniqueID)) + } + } + } + return notnullColsUniqueIDs +} + +// ExtractConstantCols extracts constant columns from conditions. +func ExtractConstantCols(conditions []expression.Expression, sctx base.PlanContext, + fds *funcdep.FDSet) intset.FastIntSet { + // extract constant cols + // eg: where a=1 and b is null and (1+c)=5. + // TODO: Some columns can only be determined to be constant from multiple constraints (e.g. x <= 1 AND x >= 1) + var ( + constObjs []expression.Expression + constUniqueIDs = intset.NewFastIntSet() + ) + constObjs = expression.ExtractConstantEqColumnsOrScalar(sctx.GetExprCtx(), constObjs, conditions) + for _, constObj := range constObjs { + switch x := constObj.(type) { + case *expression.Column: + constUniqueIDs.Insert(int(x.UniqueID)) + case *expression.ScalarFunction: + hashCode := string(x.HashCode()) + if uniqueID, ok := fds.IsHashCodeRegistered(hashCode); ok { + constUniqueIDs.Insert(uniqueID) + } else { + scalarUniqueID := int(sctx.GetSessionVars().AllocPlanColumnID()) + fds.RegisterUniqueID(string(x.HashCode()), scalarUniqueID) + constUniqueIDs.Insert(scalarUniqueID) + } + } + } + return constUniqueIDs +} + +// ExtractEquivalenceCols extracts equivalence columns from conditions. +func ExtractEquivalenceCols(conditions []expression.Expression, sctx base.PlanContext, + fds *funcdep.FDSet) [][]intset.FastIntSet { + var equivObjsPair [][]expression.Expression + equivObjsPair = expression.ExtractEquivalenceColumns(equivObjsPair, conditions) + equivUniqueIDs := make([][]intset.FastIntSet, 0, len(equivObjsPair)) + for _, equivObjPair := range equivObjsPair { + // lhs of equivalence. + var ( + lhsUniqueID int + rhsUniqueID int + ) + switch x := equivObjPair[0].(type) { + case *expression.Column: + lhsUniqueID = int(x.UniqueID) + case *expression.ScalarFunction: + hashCode := string(x.HashCode()) + if uniqueID, ok := fds.IsHashCodeRegistered(hashCode); ok { + lhsUniqueID = uniqueID + } else { + scalarUniqueID := int(sctx.GetSessionVars().AllocPlanColumnID()) + fds.RegisterUniqueID(string(x.HashCode()), scalarUniqueID) + lhsUniqueID = scalarUniqueID + } + } + // rhs of equivalence. + switch x := equivObjPair[1].(type) { + case *expression.Column: + rhsUniqueID = int(x.UniqueID) + case *expression.ScalarFunction: + hashCode := string(x.HashCode()) + if uniqueID, ok := fds.IsHashCodeRegistered(hashCode); ok { + rhsUniqueID = uniqueID + } else { + scalarUniqueID := int(sctx.GetSessionVars().AllocPlanColumnID()) + fds.RegisterUniqueID(string(x.HashCode()), scalarUniqueID) + rhsUniqueID = scalarUniqueID + } + } + equivUniqueIDs = append(equivUniqueIDs, []intset.FastIntSet{intset.NewFastIntSet( + lhsUniqueID), intset.NewFastIntSet(rhsUniqueID)}) + } + return equivUniqueIDs +} diff --git a/pkg/planner/util/misc.go b/pkg/planner/util/misc.go index da87cb886661d..300f45d004517 100644 --- a/pkg/planner/util/misc.go +++ b/pkg/planner/util/misc.go @@ -23,10 +23,13 @@ import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" "github.com/pingcap/tidb/pkg/types" + h "github.com/pingcap/tidb/pkg/util/hint" "github.com/pingcap/tidb/pkg/util/ranger" ) @@ -304,3 +307,45 @@ func DeriveLimitStats(childProfile *property.StatsInfo, limitCount float64) *pro } return stats } + +// ExtractTableAlias returns table alias of the base.LogicalPlan's columns. +// It will return nil when there are multiple table alias, because the alias is only used to check if +// the base.LogicalPlan Match some optimizer hints, and hints are not expected to take effect in this case. +func ExtractTableAlias(p base.Plan, parentOffset int) *h.HintedTable { + if len(p.OutputNames()) > 0 && p.OutputNames()[0].TblName.L != "" { + firstName := p.OutputNames()[0] + for _, name := range p.OutputNames() { + if name.TblName.L != firstName.TblName.L || + (name.DBName.L != "" && firstName.DBName.L != "" && + name.DBName.L != firstName.DBName.L) { // DBName can be nil, see #46160 + return nil + } + } + qbOffset := p.QueryBlockOffset() + var blockAsNames []ast.HintTable + if p := p.SCtx().GetSessionVars().PlannerSelectBlockAsName.Load(); p != nil { + blockAsNames = *p + } + // For sub-queries like `(select * from t) t1`, t1 should belong to its surrounding select block. + if qbOffset != parentOffset && blockAsNames != nil && blockAsNames[qbOffset].TableName.L != "" { + qbOffset = parentOffset + } + dbName := firstName.DBName + if dbName.L == "" { + dbName = model.NewCIStr(p.SCtx().GetSessionVars().CurrentDB) + } + return &h.HintedTable{DBName: dbName, TblName: firstName.TblName, SelectOffset: qbOffset} + } + return nil +} + +// GetPushDownCtx creates a PushDownContext from PlanContext +func GetPushDownCtx(pctx base.PlanContext) expression.PushDownContext { + return GetPushDownCtxFromBuildPBContext(pctx.GetBuildPBCtx()) +} + +// GetPushDownCtxFromBuildPBContext creates a PushDownContext from BuildPBContext +func GetPushDownCtxFromBuildPBContext(bctx *base.BuildPBContext) expression.PushDownContext { + return expression.NewPushDownContext(bctx.GetExprCtx().GetEvalCtx(), bctx.GetClient(), + bctx.InExplainStmt, bctx.WarnHandler, bctx.ExtraWarnghandler, bctx.GroupConcatMaxLen) +} diff --git a/pkg/planner/util/utilfuncp/func_pointer_misc.go b/pkg/planner/util/utilfuncp/func_pointer_misc.go index 956bce7e3f83d..6159d7cc5a5d0 100644 --- a/pkg/planner/util/utilfuncp/func_pointer_misc.go +++ b/pkg/planner/util/utilfuncp/func_pointer_misc.go @@ -147,6 +147,14 @@ var ExhaustPhysicalPlans4LogicalLock func(lp base.LogicalPlan, prop *property.Ph var ExhaustPhysicalPlans4LogicalUnionScan func(lp base.LogicalPlan, prop *property.PhysicalProperty) ( []base.PhysicalPlan, bool, error) +// ExhaustPhysicalPlans4LogicalSelection will be called by LogicalSelection in logicalOp pkg. +var ExhaustPhysicalPlans4LogicalSelection func(lp base.LogicalPlan, prop *property.PhysicalProperty) ( + []base.PhysicalPlan, bool, error) + +// ExhaustPhysicalPlans4LogicalJoin will be called by LogicalJoin in logicalOp pkg. +var ExhaustPhysicalPlans4LogicalJoin func(lp base.LogicalPlan, prop *property.PhysicalProperty) ( + []base.PhysicalPlan, bool, error) + // *************************************** physical op related ******************************************* // GetEstimatedProbeCntFromProbeParents will be called by BasePhysicalPlan in physicalOp pkg. @@ -159,3 +167,7 @@ var GetActualProbeCntFromProbeParents func(pps []base.PhysicalPlan, statsColl *e // AttachPlan2Task will be called by BasePhysicalPlan in physicalOp pkg. var AttachPlan2Task func(p base.PhysicalPlan, t base.Task) base.Task + +// WindowIsTopN is used in DeriveTopNFromWindow rule. +// todo: @arenatlx: remove it after logical_datasource is migrated to logicalop. +var WindowIsTopN func(p base.LogicalPlan) (bool, uint64) diff --git a/pkg/util/ranger/BUILD.bazel b/pkg/util/ranger/BUILD.bazel index 88afae84c1132..22f01dba8c0d8 100644 --- a/pkg/util/ranger/BUILD.bazel +++ b/pkg/util/ranger/BUILD.bazel @@ -58,6 +58,7 @@ go_test( "//pkg/parser/mysql", "//pkg/planner/core", "//pkg/planner/core/base", + "//pkg/planner/core/operator/logicalop", "//pkg/session", "//pkg/sessionctx", "//pkg/testkit", diff --git a/pkg/util/ranger/bench_test.go b/pkg/util/ranger/bench_test.go index 7881aaf1e5def..69bba53273d29 100644 --- a/pkg/util/ranger/bench_test.go +++ b/pkg/util/ranger/bench_test.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/tidb/pkg/expression" plannercore "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/planner/core/base" + "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/session" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/testkit" @@ -115,7 +116,7 @@ WHERE ctx := context.Background() p, err := plannercore.BuildLogicalPlanForTest(ctx, sctx, stmts[0], ret.InfoSchema) require.NoError(b, err) - selection := p.(base.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) + selection := p.(base.LogicalPlan).Children()[0].(*logicalop.LogicalSelection) tbl := selection.Children()[0].(*plannercore.DataSource).TableInfo require.NotNil(b, selection) conds := make([]expression.Expression, len(selection.Conditions)) diff --git a/pkg/util/ranger/ranger_test.go b/pkg/util/ranger/ranger_test.go index 60c3d485eb721..a4c0956980c7e 100644 --- a/pkg/util/ranger/ranger_test.go +++ b/pkg/util/ranger/ranger_test.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" plannercore "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/planner/core/base" + "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/session" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/testkit" @@ -273,7 +274,7 @@ func TestTableRange(t *testing.T) { require.NoError(t, err) p, err := plannercore.BuildLogicalPlanForTest(ctx, sctx, stmts[0], ret.InfoSchema) require.NoError(t, err) - selection := p.(base.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) + selection := p.(base.LogicalPlan).Children()[0].(*logicalop.LogicalSelection) conds := make([]expression.Expression, len(selection.Conditions)) for i, cond := range selection.Conditions { conds[i] = expression.PushDownNot(sctx.GetExprCtx(), cond) @@ -472,7 +473,7 @@ create table t( require.NoError(t, err) p, err := plannercore.BuildLogicalPlanForTest(ctx, sctx, stmts[0], ret.InfoSchema) require.NoError(t, err) - selection := p.(base.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) + selection := p.(base.LogicalPlan).Children()[0].(*logicalop.LogicalSelection) tbl := selection.Children()[0].(*plannercore.DataSource).TableInfo require.NotNil(t, selection) conds := make([]expression.Expression, len(selection.Conditions)) @@ -835,7 +836,7 @@ func TestColumnRange(t *testing.T) { require.NoError(t, err) p, err := plannercore.BuildLogicalPlanForTest(ctx, sctx, stmts[0], ret.InfoSchema) require.NoError(t, err) - sel := p.(base.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) + sel := p.(base.LogicalPlan).Children()[0].(*logicalop.LogicalSelection) ds, ok := sel.Children()[0].(*plannercore.DataSource) require.True(t, ok) conds := make([]expression.Expression, len(sel.Conditions)) @@ -994,7 +995,7 @@ func TestIndexRangeForYear(t *testing.T) { require.NoError(t, err) p, err := plannercore.BuildLogicalPlanForTest(ctx, sctx, stmts[0], ret.InfoSchema) require.NoError(t, err) - selection := p.(base.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) + selection := p.(base.LogicalPlan).Children()[0].(*logicalop.LogicalSelection) tbl := selection.Children()[0].(*plannercore.DataSource).TableInfo require.NotNil(t, selection) conds := make([]expression.Expression, len(selection.Conditions)) @@ -1064,7 +1065,7 @@ func TestPrefixIndexRangeScan(t *testing.T) { require.NoError(t, err) p, err := plannercore.BuildLogicalPlanForTest(ctx, sctx, stmts[0], ret.InfoSchema) require.NoError(t, err) - selection := p.(base.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) + selection := p.(base.LogicalPlan).Children()[0].(*logicalop.LogicalSelection) tbl := selection.Children()[0].(*plannercore.DataSource).TableInfo require.NotNil(t, selection) conds := make([]expression.Expression, len(selection.Conditions)) @@ -1412,7 +1413,7 @@ create table t( require.NoError(t, err) p, err := plannercore.BuildLogicalPlanForTest(ctx, sctx, stmts[0], ret.InfoSchema) require.NoError(t, err) - selection := p.(base.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) + selection := p.(base.LogicalPlan).Children()[0].(*logicalop.LogicalSelection) tbl := selection.Children()[0].(*plannercore.DataSource).TableInfo require.NotNil(t, selection) conds := make([]expression.Expression, len(selection.Conditions)) @@ -1654,7 +1655,7 @@ func TestTableShardIndex(t *testing.T) { require.NoError(t, err) p, err := plannercore.BuildLogicalPlanForTest(ctx, sctx, stmts[0], ret.InfoSchema) require.NoError(t, err) - selection := p.(base.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) + selection := p.(base.LogicalPlan).Children()[0].(*logicalop.LogicalSelection) conds := make([]expression.Expression, len(selection.Conditions)) for i, cond := range selection.Conditions { conds[i] = expression.PushDownNot(sctx.GetExprCtx(), cond) @@ -1836,7 +1837,7 @@ func TestShardIndexFuncSuites(t *testing.T) { } } -func getSelectionFromQuery(t *testing.T, sctx sessionctx.Context, sql string) *plannercore.LogicalSelection { +func getSelectionFromQuery(t *testing.T, sctx sessionctx.Context, sql string) *logicalop.LogicalSelection { ctx := context.Background() stmts, err := session.Parse(sctx, sql) require.NoError(t, err) @@ -1846,7 +1847,7 @@ func getSelectionFromQuery(t *testing.T, sctx sessionctx.Context, sql string) *p require.NoError(t, err) p, err := plannercore.BuildLogicalPlanForTest(ctx, sctx, stmts[0], ret.InfoSchema) require.NoError(t, err) - selection, isSelection := p.(base.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) + selection, isSelection := p.(base.LogicalPlan).Children()[0].(*logicalop.LogicalSelection) require.True(t, isSelection) return selection } @@ -2297,7 +2298,7 @@ create table t( require.NoError(t, err, fmt.Sprintf("error %v, for resolve name, expr %s", err, tt.exprStr)) p, err := plannercore.BuildLogicalPlanForTest(ctx, sctx, stmts[0], ret.InfoSchema) require.NoError(t, err, fmt.Sprintf("error %v, for build plan, expr %s", err, tt.exprStr)) - selection := p.(base.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) + selection := p.(base.LogicalPlan).Children()[0].(*logicalop.LogicalSelection) tbl := selection.Children()[0].(*plannercore.DataSource).TableInfo require.NotNil(t, selection, fmt.Sprintf("expr:%v", tt.exprStr)) conds := make([]expression.Expression, len(selection.Conditions))