Skip to content

Commit

Permalink
remove keyspace qualifier from output, added tests
Browse files Browse the repository at this point in the history
Signed-off-by: Harshit Gangal <harshit@planetscale.com>
  • Loading branch information
harshit-gangal committed Jan 25, 2024
1 parent c4d34c1 commit ae2d1f5
Show file tree
Hide file tree
Showing 10 changed files with 118 additions and 28 deletions.
22 changes: 20 additions & 2 deletions go/vt/sqlparser/ast_funcs.go
Original file line number Diff line number Diff line change
Expand Up @@ -2150,8 +2150,8 @@ func (s SelectExprs) AllAggregation() bool {
return true
}

// RemoveKeyspace removes the Qualifier.Qualifier on all ColNames in the AST
func RemoveKeyspace(in SQLNode) {
// RemoveKeyspaceInCol removes the Qualifier.Qualifier on all ColNames in the AST
func RemoveKeyspaceInCol(in SQLNode) {
// Walk will only return an error if we return an error from the inner func. safe to ignore here
_ = Walk(func(node SQLNode) (kontinue bool, err error) {
if col, ok := node.(*ColName); ok && col.Qualifier.Qualifier.NotEmpty() {
Expand All @@ -2175,6 +2175,24 @@ func RemoveKeyspaceInTables(in SQLNode) {
})
}

// RemoveKeyspace removes the Qualifier.Qualifier on all ColNames and Qualifier on all TableNames in the AST
func RemoveKeyspace(in SQLNode) {
Rewrite(in, nil, func(cursor *Cursor) bool {
switch expr := cursor.Node().(type) {
case *ColName:
if expr.Qualifier.Qualifier.NotEmpty() {
expr.Qualifier.Qualifier = NewIdentifierCS("")
}
case TableName:
if expr.Qualifier.NotEmpty() {
expr.Qualifier = NewIdentifierCS("")
cursor.Replace(expr)
}
}
return true
})
}

func convertStringToInt(integer string) int {
val, _ := strconv.Atoi(integer)
return val
Expand Down
4 changes: 2 additions & 2 deletions go/vt/vtgate/planbuilder/operators/SQL_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -309,7 +309,7 @@ func (ts *tableSorter) Swap(i, j int) {
func removeKeyspaceFromSelectExpr(expr sqlparser.SelectExpr) {
switch expr := expr.(type) {
case *sqlparser.AliasedExpr:
sqlparser.RemoveKeyspace(expr.Expr)
sqlparser.RemoveKeyspaceInCol(expr.Expr)
case *sqlparser.StarExpr:
expr.TableName.Qualifier = sqlparser.NewIdentifierCS("")
}
Expand Down Expand Up @@ -567,7 +567,7 @@ func buildFilter(op *Filter, qb *queryBuilder) {
func buildDerived(op *Horizon, qb *queryBuilder) {
buildQuery(op.Source, qb)

sqlparser.RemoveKeyspace(op.Query)
sqlparser.RemoveKeyspaceInCol(op.Query)

stmt := qb.stmt
qb.stmt = nil
Expand Down
2 changes: 1 addition & 1 deletion go/vt/vtgate/planbuilder/operators/ast_to_op.go
Original file line number Diff line number Diff line change
Expand Up @@ -72,7 +72,7 @@ func addWherePredicates(ctx *plancontext.PlanningContext, expr sqlparser.Expr, o
outerID := TableID(op)
exprs := sqlparser.SplitAndExpression(nil, expr)
for _, expr := range exprs {
sqlparser.RemoveKeyspace(expr)
sqlparser.RemoveKeyspaceInCol(expr)
subq := sqc.handleSubquery(ctx, expr, outerID)
if subq != nil {
continue
Expand Down
4 changes: 2 additions & 2 deletions go/vt/vtgate/planbuilder/operators/join.go
Original file line number Diff line number Diff line change
Expand Up @@ -92,7 +92,7 @@ func createOuterJoin(tableExpr *sqlparser.JoinTableExpr, lhs, rhs Operator) Oper
panic(vterrors.VT12001("subquery in outer join predicate"))
}
predicate := tableExpr.Condition.On
sqlparser.RemoveKeyspace(predicate)
sqlparser.RemoveKeyspaceInCol(predicate)
return &Join{LHS: lhs, RHS: rhs, LeftJoin: true, Predicate: predicate}
}

Expand All @@ -115,7 +115,7 @@ func createInnerJoin(ctx *plancontext.PlanningContext, tableExpr *sqlparser.Join
sqc := &SubQueryBuilder{}
outerID := TableID(op)
joinPredicate := tableExpr.Condition.On
sqlparser.RemoveKeyspace(joinPredicate)
sqlparser.RemoveKeyspaceInCol(joinPredicate)
exprs := sqlparser.SplitAndExpression(nil, joinPredicate)
for _, pred := range exprs {
subq := sqc.handleSubquery(ctx, pred, outerID)
Expand Down
2 changes: 1 addition & 1 deletion go/vt/vtgate/planbuilder/operators/subquery_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -201,7 +201,7 @@ func (sqb *SubQueryBuilder) inspectWhere(
outerID: sqb.outerID,
}
for _, predicate := range sqlparser.SplitAndExpression(nil, in.Expr) {
sqlparser.RemoveKeyspace(predicate)
sqlparser.RemoveKeyspaceInCol(predicate)
subq := sqb.handleSubquery(ctx, predicate, sqb.totalID)
if subq != nil {
continue
Expand Down
2 changes: 1 addition & 1 deletion go/vt/vtgate/planbuilder/operators/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -115,7 +115,7 @@ func addColumn(ctx *plancontext.PlanningContext, op ColNameColumns, e sqlparser.
if !ok {
panic(vterrors.VT09018(fmt.Sprintf("cannot add '%s' expression to a table/vindex", sqlparser.String(e))))
}
sqlparser.RemoveKeyspace(col)
sqlparser.RemoveKeyspaceInCol(col)
cols := op.GetColNames()
colAsExpr := func(c *sqlparser.ColName) sqlparser.Expr { return c }
if offset, found := canReuseColumn(ctx, cols, e, colAsExpr); found {
Expand Down
2 changes: 1 addition & 1 deletion go/vt/vtgate/planbuilder/single_sharded_shortcut.go
Original file line number Diff line number Diff line change
Expand Up @@ -105,7 +105,7 @@ func getTableNames(semTable *semantics.SemTable) ([]sqlparser.TableName, error)
func removeKeyspaceFromSelectExpr(expr sqlparser.SelectExpr) {
switch expr := expr.(type) {
case *sqlparser.AliasedExpr:
sqlparser.RemoveKeyspace(expr.Expr)
sqlparser.RemoveKeyspaceInCol(expr.Expr)
case *sqlparser.StarExpr:
expr.TableName.Qualifier = sqlparser.NewIdentifierCS("")
}
Expand Down
65 changes: 65 additions & 0 deletions go/vt/vtgate/planbuilder/testdata/other_read_cases.json
Original file line number Diff line number Diff line change
Expand Up @@ -81,5 +81,70 @@
"main.t"
]
}
},
{
"comment": "explain - routed table with same name",
"query": "explain select 1, second_user.user.id from second_user.user",
"plan": {
"QueryType": "EXPLAIN",
"Original": "explain select 1, second_user.user.id from second_user.user",
"Instructions": {
"OperatorType": "Send",
"Keyspace": {
"Name": "user",
"Sharded": true
},
"TargetDestination": "AnyShard()",
"Query": "explain select 1, `user`.id from `user`",
"SingleShardOnly": true
},
"TablesUsed": [
"user.user"
]
}
},
{
"comment": "explain - routed table with different name",
"query": "explain select 1, second_user.foo.id, foo.col from second_user.foo",
"plan": {
"QueryType": "EXPLAIN",
"Original": "explain select 1, second_user.foo.id, foo.col from second_user.foo",
"Instructions": {
"OperatorType": "Send",
"Keyspace": {
"Name": "user",
"Sharded": true
},
"TargetDestination": "AnyShard()",
"Query": "explain select 1, foo.id, foo.col from `user` as foo",
"SingleShardOnly": true
},
"TablesUsed": [
"user.foo"
]
}
},
{
"comment": "explain - routed table with join on different table on routed keyspace",
"query": "explain select 1, second_user.foo.id, foo.col from second_user.foo join user.user join user.music",
"plan": {
"QueryType": "EXPLAIN",
"Original": "explain select 1, second_user.foo.id, foo.col from second_user.foo join user.user join user.music",
"Instructions": {
"OperatorType": "Send",
"Keyspace": {
"Name": "user",
"Sharded": true
},
"TargetDestination": "AnyShard()",
"Query": "explain select 1, foo.id, foo.col from `user` as foo join `user` join music",
"SingleShardOnly": true
},
"TablesUsed": [
"user.foo",
"user.user",
"user.music"
]
}
}
]
5 changes: 5 additions & 0 deletions go/vt/vtgate/planbuilder/testdata/unsupported_cases.json
Original file line number Diff line number Diff line change
Expand Up @@ -279,6 +279,11 @@
"query": "delete from user where x = (@val := 42)",
"plan": "VT12001: unsupported: Assignment expression"
},
{
"comment": "explain - routed table with join on different keyspace table",
"query": "explain select 1, second_user.foo.id, foo.col from second_user.foo join user.user join main.unsharded",
"plan": "VT03031: EXPLAIN is only supported for single keyspace"
},
{
"comment": "extremum on input from both sides",
"query": "insert into music(user_id, id) select foo, bar from music on duplicate key update id = id+1",
Expand Down
38 changes: 20 additions & 18 deletions go/vt/vtgate/planbuilder/vexplain.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,6 @@ import (
"vitess.io/vitess/go/vt/vtgate/engine"
"vitess.io/vitess/go/vt/vtgate/planbuilder/operators"
"vitess.io/vitess/go/vt/vtgate/planbuilder/plancontext"
"vitess.io/vitess/go/vt/vtgate/semantics"
)

func buildVExplainPlan(ctx context.Context, vexplainStmt *sqlparser.VExplainStmt, reservedVars *sqlparser.ReservedVars, vschema plancontext.VSchema, enableOnlineDDL, enableDirectDDL bool) (*planResult, error) {
Expand Down Expand Up @@ -112,36 +111,35 @@ func buildExplainStmtPlan(
reservedVars *sqlparser.ReservedVars,
vschema plancontext.VSchema,
) (*planResult, error) {
ksName := ""
if ks, _ := vschema.DefaultKeyspace(); ks != nil {
ksName = ks.Name
switch explain.Statement.(type) {
case sqlparser.SelectStatement, *sqlparser.Update, *sqlparser.Delete, *sqlparser.Insert:
return explainSelectPlan(explain, reservedVars, vschema)
default:
return buildOtherReadAndAdmin(sqlparser.String(explain), vschema)
}

semTable, err := semantics.Analyze(explain.Statement, ksName, vschema)
}

func explainDefaultPlan(explain *sqlparser.ExplainStmt, reservedVars *sqlparser.ReservedVars, vschema plancontext.VSchema) (*planResult, error) {
ctx, err := plancontext.CreatePlanningContext(explain.Statement, reservedVars, vschema, Gen4)
if err != nil {
return nil, err
}

ks := semTable.SingleKeyspace()
ks := ctx.SemTable.SingleKeyspace()
if ks == nil {
return nil, vterrors.VT03031()
}

if err = queryRewrite(semTable, reservedVars, explain.Statement); err != nil {
if err = queryRewrite(ctx.SemTable, reservedVars, explain.Statement); err != nil {
return nil, err
}

destination, keyspace, _, err := vschema.TargetDestination(ks.Name)
if err != nil {
return nil, err
}

if destination == nil {
destination = key.DestinationAnyShard{}
}
// Remove keyspace qualifier from columns and tables.
sqlparser.RemoveKeyspace(explain.Statement)

var tables []string
for _, table := range semTable.Tables {
for _, table := range ctx.SemTable.Tables {
name, err := table.Name()
if err != nil {
// this is just for reporting which tables we are touching
Expand All @@ -152,9 +150,13 @@ func buildExplainStmtPlan(
}

return newPlanResult(&engine.Send{
Keyspace: keyspace,
TargetDestination: destination,
Keyspace: ks,
TargetDestination: key.DestinationAnyShard{},
Query: sqlparser.String(explain),
SingleShardOnly: true,
}, tables...), nil
}

func explainSelectPlan(explain *sqlparser.ExplainStmt, reservedVars *sqlparser.ReservedVars, vschema plancontext.VSchema) (*planResult, error) {
return explainDefaultPlan(explain, reservedVars, vschema)
}

0 comments on commit ae2d1f5

Please sign in to comment.