diff --git a/executor/builder.go b/executor/builder.go index 2817150a8134a..93528e1611504 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -126,6 +126,8 @@ func (b *executorBuilder) build(p plannercore.Plan) Executor { return b.buildExplain(v) case *plannercore.PointGetPlan: return b.buildPointGet(v) + case *plannercore.BatchPointGetPlan: + return b.buildBatchPointGet(v) case *plannercore.Insert: return b.buildInsert(v) case *plannercore.LoadData: @@ -1308,46 +1310,6 @@ func (b *executorBuilder) buildMaxOneRow(v *plannercore.PhysicalMaxOneRow) Execu } func (b *executorBuilder) buildUnionAll(v *plannercore.PhysicalUnionAll) Executor { - if v.IsPointGetUnion { - startTS, err := b.getStartTS() - if err != nil { - b.err = err - return nil - } - children := v.Children() - // It's OK to type assert here because `v.IsPointGetUnion == true` only if all children are PointGet - pointGet := children[0].(*plannercore.PointGetPlan) - e := &BatchPointGetExec{ - baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ExplainID()), - tblInfo: pointGet.TblInfo, - idxInfo: pointGet.IndexInfo, - startTS: startTS, - } - if pointGet.IndexInfo != nil { - idxVals := make([][]types.Datum, len(children)) - for i, child := range children { - idxVals[i] = child.(*plannercore.PointGetPlan).IndexValues - } - e.idxVals = idxVals - } else { - // `SELECT a FROM t WHERE a IN (1, 1, 2, 1, 2)` should not return duplicated rows - handles := make([]int64, 0, len(children)) - dedup := make(map[int64]struct{}, len(children)) - for _, child := range children { - handle := child.(*plannercore.PointGetPlan).Handle - if _, found := dedup[handle]; found { - continue - } - dedup[handle] = struct{}{} - handles = append(handles, handle) - } - e.handles = handles - } - e.base().initCap = len(children) - e.base().maxChunkSize = len(children) - return e - } - childExecs := make([]Executor, len(v.Children())) for i, child := range v.Children() { childExecs[i] = b.build(child) @@ -2285,6 +2247,41 @@ func (b *executorBuilder) buildSQLBindExec(v *plannercore.SQLBindPlan) Executor return e } +func (b *executorBuilder) buildBatchPointGet(plan *plannercore.BatchPointGetPlan) Executor { + startTS, err := b.getStartTS() + if err != nil { + b.err = err + return nil + } + e := &BatchPointGetExec{ + baseExecutor: newBaseExecutor(b.ctx, plan.Schema(), plan.ExplainID()), + tblInfo: plan.TblInfo, + idxInfo: plan.IndexInfo, + startTS: startTS, + } + var capacity int + if plan.IndexInfo != nil { + e.idxVals = plan.IndexValues + capacity = len(e.idxVals) + } else { + // `SELECT a FROM t WHERE a IN (1, 1, 2, 1, 2)` should not return duplicated rows + handles := make([]int64, 0, len(plan.Handles)) + dedup := make(map[int64]struct{}, len(plan.Handles)) + for _, handle := range plan.Handles { + if _, found := dedup[handle]; found { + continue + } + dedup[handle] = struct{}{} + handles = append(handles, handle) + } + e.handles = handles + capacity = len(e.handles) + } + e.base().initCap = capacity + e.base().maxChunkSize = capacity + return e +} + func getPhysicalTableID(t table.Table) int64 { if p, ok := t.(table.PhysicalTable); ok { return p.GetPhysicalID() diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index f322ad3bf1664..0566415198c85 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -405,6 +405,26 @@ func (e *Execute) rebuildRange(p Plan) error { } } return nil + case *BatchPointGetPlan: + for i, param := range x.HandleParams { + if param != nil { + x.Handles[i], err = param.Datum.ToInt64(sc) + if err != nil { + return err + } + return nil + } + } + for i, params := range x.IndexValueParams { + if len(params) < 1 { + continue + } + for j, param := range params { + if param != nil { + x.IndexValues[i][j] = param.Datum + } + } + } case PhysicalPlan: for _, child := range x.Children() { err = e.rebuildRange(child) diff --git a/planner/core/initialize.go b/planner/core/initialize.go index 99bbde2299ac7..186038a95bc86 100644 --- a/planner/core/initialize.go +++ b/planner/core/initialize.go @@ -14,8 +14,10 @@ package core import ( + "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/types" ) const ( @@ -483,6 +485,15 @@ func (p PhysicalIndexHashJoin) Init(ctx sessionctx.Context) *PhysicalIndexHashJo return &p } +// Init initializes BatchPointGetPlan. +func (p BatchPointGetPlan) Init(ctx sessionctx.Context, stats *property.StatsInfo, schema *expression.Schema, names []*types.FieldName) *BatchPointGetPlan { + p.basePlan = newBasePlan(ctx, "Batch_Point_Get", 0) + p.schema = schema + p.names = names + p.stats = stats + return &p +} + // flattenPushDownPlan converts a plan tree to a list, whose head is the leaf node like table scan. func flattenPushDownPlan(p PhysicalPlan) []PhysicalPlan { plans := make([]PhysicalPlan, 0, 5) diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index ce103daaed282..745ba64f11375 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -52,6 +52,7 @@ var ( _ PhysicalPlan = &PhysicalMergeJoin{} _ PhysicalPlan = &PhysicalUnionScan{} _ PhysicalPlan = &PhysicalWindow{} + _ PhysicalPlan = &BatchPointGetPlan{} ) // PhysicalTableReader is the table reader in tidb. @@ -336,17 +337,6 @@ type PhysicalLimit struct { // PhysicalUnionAll is the physical operator of UnionAll. type PhysicalUnionAll struct { physicalSchemaProducer - // IsPointGetUnion indicates all the children are PointGet and - // all of them reference the same table and use the same `unique key` - IsPointGetUnion bool -} - -// OutputNames returns the outputting names of each column. -func (p *PhysicalUnionAll) OutputNames() []*types.FieldName { - if p.IsPointGetUnion { - return p.children[0].OutputNames() - } - return p.physicalSchemaProducer.OutputNames() } // AggregationType stands for the mode of aggregation plan. diff --git a/planner/core/point_get_plan.go b/planner/core/point_get_plan.go index d16f6570909fe..78fff77ec95e8 100644 --- a/planner/core/point_get_plan.go +++ b/planner/core/point_get_plan.go @@ -143,6 +143,83 @@ func (p *PointGetPlan) OutputNames() []*types.FieldName { return p.outputNames } +// BatchPointGetPlan represents a physical plan which contains a bunch of +// keys reference the same table and use the same `unique key` +type BatchPointGetPlan struct { + baseSchemaProducer + + TblInfo *model.TableInfo + IndexInfo *model.IndexInfo + Handles []int64 + HandleParams []*driver.ParamMarkerExpr + IndexValues [][]types.Datum + IndexValueParams [][]*driver.ParamMarkerExpr +} + +// attach2Task makes the current physical plan as the father of task's physicalPlan and updates the cost of +// current task. If the child's task is cop task, some operator may close this task and return a new rootTask. +func (p *BatchPointGetPlan) attach2Task(...task) task { + return nil +} + +// ToPB converts physical plan to tipb executor. +func (p *BatchPointGetPlan) ToPB(ctx sessionctx.Context) (*tipb.Executor, error) { + return nil, nil +} + +// ExplainInfo returns operator information to be explained. +func (p *BatchPointGetPlan) ExplainInfo() string { + buffer := bytes.NewBufferString("") + tblName := p.TblInfo.Name.O + fmt.Fprintf(buffer, "table:%s", tblName) + if p.IndexInfo != nil { + fmt.Fprintf(buffer, ", index:") + for i, col := range p.IndexInfo.Columns { + buffer.WriteString(col.Name.O) + if i < len(p.IndexInfo.Columns)-1 { + buffer.WriteString(" ") + } + } + } + return buffer.String() +} + +// GetChildReqProps gets the required property by child index. +func (p *BatchPointGetPlan) GetChildReqProps(idx int) *property.PhysicalProperty { + return nil +} + +// StatsCount will return the the RowCount of property.StatsInfo for this plan. +func (p *BatchPointGetPlan) StatsCount() float64 { + return p.statsInfo().RowCount +} + +// statsInfo will return the the RowCount of property.StatsInfo for this plan. +func (p *BatchPointGetPlan) statsInfo() *property.StatsInfo { + return p.stats +} + +// Children gets all the children. +func (p *BatchPointGetPlan) Children() []PhysicalPlan { + return nil +} + +// SetChildren sets the children for the plan. +func (p *BatchPointGetPlan) SetChildren(...PhysicalPlan) {} + +// SetChild sets a specific child for the plan. +func (p *BatchPointGetPlan) SetChild(i int, child PhysicalPlan) {} + +// ResolveIndices resolves the indices for columns. After doing this, the columns can evaluate the rows by their indices. +func (p *BatchPointGetPlan) ResolveIndices() error { + return nil +} + +// OutputNames returns the outputting names of each column. +func (p *BatchPointGetPlan) OutputNames() []*types.FieldName { + return p.names +} + // TryFastPlan tries to use the PointGetPlan for the query. func TryFastPlan(ctx sessionctx.Context, node ast.Node) Plan { switch x := node.(type) { @@ -154,7 +231,7 @@ func TryFastPlan(ctx sessionctx.Context, node ast.Node) Plan { } fp := tryPointGetPlan(ctx, x) if fp != nil { - if checkFastPlanPrivilege(ctx, fp, mysql.SelectPriv) != nil { + if checkFastPlanPrivilege(ctx, fp.dbName, fp.TblInfo.Name.L, mysql.SelectPriv) != nil { return nil } if fp.IsTableDual { @@ -184,10 +261,137 @@ func TryFastPlan(ctx sessionctx.Context, node ast.Node) Plan { return nil } +func newBatchPointGetPlan( + ctx sessionctx.Context, patternInExpr *ast.PatternInExpr, + tryHandle bool, fieldType *types.FieldType, + tbl *model.TableInfo, schema *expression.Schema, + names []*types.FieldName, whereColNames []string, +) Plan { + statsInfo := &property.StatsInfo{RowCount: float64(len(patternInExpr.List))} + if tryHandle && fieldType != nil { + var handles = make([]int64, len(patternInExpr.List)) + var handleParams = make([]*driver.ParamMarkerExpr, len(patternInExpr.List)) + for i, item := range patternInExpr.List { + // SELECT * FROM t WHERE (key) in ((1), (2)) + if p, ok := item.(*ast.ParenthesesExpr); ok { + item = p.Expr + } + var d types.Datum + var param *driver.ParamMarkerExpr + switch x := item.(type) { + case *driver.ValueExpr: + d = x.Datum + case *driver.ParamMarkerExpr: + d = x.Datum + param = x + default: + return nil + } + if d.IsNull() { + return nil + } + intDatum, err := d.ConvertTo(ctx.GetSessionVars().StmtCtx, fieldType) + if err != nil { + return nil + } + // The converted result must be same as original datum + cmp, err := intDatum.CompareDatum(ctx.GetSessionVars().StmtCtx, &d) + if err != nil || cmp != 0 { + return nil + } + handles[i] = intDatum.GetInt64() + handleParams[i] = param + } + return BatchPointGetPlan{ + TblInfo: tbl, + Handles: handles, + HandleParams: handleParams, + }.Init(ctx, statsInfo, schema, names) + } + + // The columns in where clause should be covered by unique index + var matchIdxInfo *model.IndexInfo + permutations := make([]int, len(whereColNames)) + for _, idxInfo := range tbl.Indices { + if !idxInfo.Unique || idxInfo.State != model.StatePublic { + continue + } + if len(idxInfo.Columns) != len(whereColNames) || idxInfo.HasPrefixIndex() { + continue + } + // TODO: not sure is there any function to reuse + matched := true + for whereColIndex, innerCol := range whereColNames { + var found bool + for i, col := range idxInfo.Columns { + if innerCol == col.Name.L { + permutations[whereColIndex] = i + found = true + break + } + } + if !found { + matched = false + break + } + } + if matched { + matchIdxInfo = idxInfo + break + } + } + if matchIdxInfo == nil { + return nil + } + indexValues := make([][]types.Datum, len(patternInExpr.List)) + indexValueParams := make([][]*driver.ParamMarkerExpr, len(patternInExpr.List)) + for i, item := range patternInExpr.List { + // SELECT * FROM t WHERE (key) in ((1), (2)) + if p, ok := item.(*ast.ParenthesesExpr); ok { + item = p.Expr + } + var values []types.Datum + var valuesParams []*driver.ParamMarkerExpr + switch x := item.(type) { + case *ast.RowExpr: + // The `len(values) == len(valuesParams)` should be satisfied in this mode + values = make([]types.Datum, len(x.Values)) + valuesParams = make([]*driver.ParamMarkerExpr, len(x.Values)) + for index, inner := range x.Values { + permIndex := permutations[index] + switch innerX := inner.(type) { + case *driver.ValueExpr: + values[permIndex] = innerX.Datum + case *driver.ParamMarkerExpr: + values[permIndex] = innerX.Datum + valuesParams[permIndex] = innerX + default: + return nil + } + } + case *driver.ValueExpr: + values = []types.Datum{x.Datum} + case *driver.ParamMarkerExpr: + values = []types.Datum{x.Datum} + valuesParams = []*driver.ParamMarkerExpr{x} + default: + return nil + } + indexValues[i] = values + indexValueParams[i] = valuesParams + } + return BatchPointGetPlan{ + TblInfo: tbl, + IndexInfo: matchIdxInfo, + IndexValues: indexValues, + IndexValueParams: indexValueParams, + }.Init(ctx, statsInfo, schema, names) +} + func tryWhereIn2BatchPointGet(ctx sessionctx.Context, selStmt *ast.SelectStmt) Plan { - if selStmt.OrderBy != nil || selStmt.GroupBy != nil || selStmt.Limit != nil || - selStmt.Having != nil || len(selStmt.WindowSpecs) > 0 || - selStmt.LockTp != ast.SelectLockNone { + if selStmt.OrderBy != nil || selStmt.GroupBy != nil || + selStmt.Limit != nil || selStmt.Having != nil || + len(selStmt.WindowSpecs) > 0 || selStmt.LockTp != ast.SelectLockNone { return nil } in, ok := selStmt.Where.(*ast.PatternInExpr) @@ -195,85 +399,93 @@ func tryWhereIn2BatchPointGet(ctx sessionctx.Context, selStmt *ast.SelectStmt) P return nil } - children := make([]PhysicalPlan, 0, len(in.List)) - chReqProps := make([]*property.PhysicalProperty, 0, len(in.List)) - reusedStmt := &ast.SelectStmt{ - SelectStmtOpts: selStmt.SelectStmtOpts, - Distinct: selStmt.Distinct, - From: selStmt.From, - Fields: selStmt.Fields, + tblName, tblAlias := getSingleTableNameAndAlias(selStmt.From) + if tblName == nil { + return nil + } + tbl := tblName.TableInfo + if tbl == nil { + return nil } - switch leftExpr := in.Expr.(type) { - case *ast.ColumnNameExpr: - reusedStmt := &ast.SelectStmt{ - SelectStmtOpts: selStmt.SelectStmtOpts, - Distinct: selStmt.Distinct, - From: selStmt.From, - Fields: selStmt.Fields, - } - for _, row := range in.List { - where := &ast.BinaryOperationExpr{ - Op: opcode.EQ, - L: in.Expr, - R: row, - } - reusedStmt.Where = where - fp := TryFastPlan(ctx, reusedStmt) - if fp == nil { - return nil - } - chReqProps = append(chReqProps, &property.PhysicalProperty{ExpectedCnt: 1}) - children = append(children, fp.(*PointGetPlan)) + // Do not handle partitioned table. + // Table partition implementation translates LogicalPlan from `DataSource` to + // `Union -> DataSource` in the logical plan optimization pass, since BatchPointGetPlan + // bypass the logical plan optimization, it can't support partitioned table. + if tbl.GetPartitionInfo() != nil { + return nil + } + + for _, col := range tbl.Columns { + if col.IsGenerated() || col.State != model.StatePublic { + return nil } + } - case *ast.RowExpr: - if len(leftExpr.Values) < 1 { + schema, names := buildSchemaFromFields(tblName.Schema, tbl, tblAlias, selStmt.Fields.Fields) + if schema == nil { + return nil + } + + var ( + tryHandle bool + fieldType *types.FieldType + whereColNames []string + ) + + // SELECT * FROM t WHERE (key) in ((1), (2)) + colExpr := in.Expr + if p, ok := colExpr.(*ast.ParenthesesExpr); ok { + colExpr = p.Expr + } + switch colName := colExpr.(type) { + case *ast.ColumnNameExpr: + if name := colName.Name.Table.L; name != "" && name != tblAlias.L { return nil } + // Try use handle + if tbl.PKIsHandle { + for _, col := range tbl.Columns { + if mysql.HasPriKeyFlag(col.Flag) { + tryHandle = col.Name.L == colName.Name.Name.L + fieldType = &col.FieldType + whereColNames = append(whereColNames, col.Name.L) + break + } + } + } else { + // Downgrade to use unique index + whereColNames = append(whereColNames, colName.Name.Name.L) + } - eleCount := len(leftExpr.Values) - for _, row := range in.List { - rightExpr, ok := row.(*ast.RowExpr) - if !ok || len(rightExpr.Values) != eleCount { + case *ast.RowExpr: + for _, col := range colName.Values { + c, ok := col.(*ast.ColumnNameExpr) + if !ok { return nil } - where := &ast.BinaryOperationExpr{ - Op: opcode.EQ, - L: leftExpr.Values[0], - R: rightExpr.Values[0], - } - for i := 1; i < eleCount; i++ { - right := &ast.BinaryOperationExpr{ - Op: opcode.EQ, - L: leftExpr.Values[i], - R: rightExpr.Values[i], - } - where = &ast.BinaryOperationExpr{ - Op: opcode.LogicAnd, - L: where, - R: right, - } - } - reusedStmt.Where = where - fp := TryFastPlan(ctx, reusedStmt) - if fp == nil { + if name := c.Name.Table.L; name != "" && name != tblAlias.L { return nil } - chReqProps = append(chReqProps, &property.PhysicalProperty{ExpectedCnt: 1}) - children = append(children, fp.(*PointGetPlan)) + whereColNames = append(whereColNames, c.Name.Name.L) } - default: return nil } - ua := PhysicalUnionAll{ - IsPointGetUnion: true, - }.Init(ctx, children[0].statsInfo().Scale(float64(len(children))), 0, chReqProps...) - ua.SetSchema(children[0].Schema()) - ua.SetChildren(children...) - return ua + p := newBatchPointGetPlan(ctx, in, tryHandle, fieldType, tbl, schema, names, whereColNames) + if p == nil { + return nil + } + + dbName := tblName.Schema.L + if dbName == "" { + dbName = ctx.GetSessionVars().CurrentDB + } + if checkFastPlanPrivilege(ctx, dbName, tbl.Name.L, mysql.SelectPriv) != nil { + return nil + } + return p } // tryPointGetPlan determine if the SelectStmt can use a PointGetPlan. @@ -397,13 +609,13 @@ func newPointGetPlan(ctx sessionctx.Context, dbName string, schema *expression.S return p } -func checkFastPlanPrivilege(ctx sessionctx.Context, fastPlan *PointGetPlan, checkTypes ...mysql.PrivilegeType) error { +func checkFastPlanPrivilege(ctx sessionctx.Context, dbName, tableName string, checkTypes ...mysql.PrivilegeType) error { pm := privilege.GetPrivilegeManager(ctx) if pm == nil { return nil } for _, checkType := range checkTypes { - if !pm.RequestVerification(ctx.GetSessionVars().ActiveRoles, fastPlan.dbName, fastPlan.TblInfo.Name.L, "", checkType) { + if !pm.RequestVerification(ctx.GetSessionVars().ActiveRoles, dbName, tableName, "", checkType) { return errors.New("privilege check fail") } } @@ -611,7 +823,7 @@ func tryUpdatePointPlan(ctx sessionctx.Context, updateStmt *ast.UpdateStmt) Plan if fastSelect == nil { return nil } - if checkFastPlanPrivilege(ctx, fastSelect, mysql.SelectPriv, mysql.UpdatePriv) != nil { + if checkFastPlanPrivilege(ctx, fastSelect.dbName, fastSelect.TblInfo.Name.L, mysql.SelectPriv, mysql.UpdatePriv) != nil { return nil } if fastSelect.IsTableDual { @@ -690,7 +902,7 @@ func tryDeletePointPlan(ctx sessionctx.Context, delStmt *ast.DeleteStmt) Plan { if fastSelect == nil { return nil } - if checkFastPlanPrivilege(ctx, fastSelect, mysql.SelectPriv, mysql.DeletePriv) != nil { + if checkFastPlanPrivilege(ctx, fastSelect.dbName, fastSelect.TblInfo.Name.L, mysql.SelectPriv, mysql.DeletePriv) != nil { return nil } if fastSelect.IsTableDual { diff --git a/planner/core/point_get_plan_test.go b/planner/core/point_get_plan_test.go index bff5118c56b6f..bcbaf312554d2 100644 --- a/planner/core/point_get_plan_test.go +++ b/planner/core/point_get_plan_test.go @@ -221,26 +221,51 @@ func (s *testPointGetSuite) TestWhereIn2BatchPointGet(c *C) { "Point_Get_1 1.00 root table:t, index:a b c", )) tk.MustQuery("explain select * from t where (a, b, c) in ((1, 1, 1), (2, 2, 2))").Check(testkit.Rows( - "Union_3 2.00 root ", - "├─Point_Get_1 1.00 root table:t, index:a b c", - "└─Point_Get_2 1.00 root table:t, index:a b c", + "Batch_Point_Get_1 2.00 root table:t, index:a b c", )) tk.MustQuery("explain select * from t where a in (1, 2, 3, 4, 5)").Check(testkit.Rows( - "Union_6 5.00 root ", - "├─Point_Get_1 1.00 root table:t, handle:1", - "├─Point_Get_2 1.00 root table:t, handle:2", - "├─Point_Get_3 1.00 root table:t, handle:3", - "├─Point_Get_4 1.00 root table:t, handle:4", - "└─Point_Get_5 1.00 root table:t, handle:5", + "Batch_Point_Get_1 5.00 root table:t", )) tk.MustQuery("explain select * from t where a in (1, 2, 3, 1, 2)").Check(testkit.Rows( - "Union_6 5.00 root ", - "├─Point_Get_1 1.00 root table:t, handle:1", - "├─Point_Get_2 1.00 root table:t, handle:2", - "├─Point_Get_3 1.00 root table:t, handle:3", - "├─Point_Get_4 1.00 root table:t, handle:1", - "└─Point_Get_5 1.00 root table:t, handle:2", + "Batch_Point_Get_1 5.00 root table:t", + )) + + tk.MustQuery("explain select * from t where (a) in ((1), (2), (3), (1), (2))").Check(testkit.Rows( + "Batch_Point_Get_1 5.00 root table:t", + )) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b int, c int, unique key idx_ab(a, b))") + tk.MustExec("insert into t values(1, 2, 3), (2, 3, 4), (3, 4, 5), (4, 5, 6)") + tk.MustQuery("select * from t").Check(testkit.Rows( + "1 2 3", + "2 3 4", + "3 4 5", + "4 5 6", + )) + tk.MustQuery("explain select * from t where (a, b) in ((1, 2), (2, 3))").Check(testkit.Rows( + "Batch_Point_Get_1 2.00 root table:t, index:a b", + )) + tk.MustQuery("select * from t where (a, b) in ((1, 2), (2, 3))").Check(testkit.Rows( + "1 2 3", + "2 3 4", + )) + tk.MustQuery("select * from t where (b, a) in ((1, 2), (2, 3))").Check(testkit.Rows()) + tk.MustQuery("select * from t where (b, a) in ((2, 1), (3, 2))").Check(testkit.Rows( + "1 2 3", + "2 3 4", + )) + tk.MustQuery("select * from t where (b, a) in ((2, 1), (3, 2), (2, 1), (5, 4))").Check(testkit.Rows( + "1 2 3", + "2 3 4", + "4 5 6", + )) + tk.MustQuery("select * from t where (b, a) in ((2, 1), (3, 2), (2, 1), (5, 4), (3, 4))").Check(testkit.Rows( + "1 2 3", + "2 3 4", + "4 5 6", )) }