Skip to content

Commit

Permalink
wip
Browse files Browse the repository at this point in the history
Signed-off-by: Florent Poinsard <[email protected]>
  • Loading branch information
frouioui committed Jan 30, 2025
1 parent ba53da7 commit 57f12cf
Show file tree
Hide file tree
Showing 6 changed files with 122 additions and 26 deletions.
8 changes: 6 additions & 2 deletions go/vt/vtgate/planbuilder/operators/expressions.go
Original file line number Diff line number Diff line change
Expand Up @@ -133,14 +133,18 @@ func getFirstSelect(selStmt sqlparser.TableStatement) *sqlparser.Select {
func breakValuesJoinExpressionInLHS(ctx *plancontext.PlanningContext,
expr sqlparser.Expr,
lhs semantics.TableSet,
) (results []*sqlparser.ColName) {
) (result valuesJoinColumn) {
result.Original = expr
result.PureLHS = true
_ = sqlparser.Walk(func(node sqlparser.SQLNode) (kontinue bool, err error) {
col, ok := node.(*sqlparser.ColName)
if !ok {
return true, nil
}
if ctx.SemTable.RecursiveDeps(col) == lhs {
results = append(results, col)
result.LHS = append(result.LHS, col)
} else {
result.PureLHS = false
}
return true, nil
}, expr)
Expand Down
4 changes: 2 additions & 2 deletions go/vt/vtgate/planbuilder/operators/expressions_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -49,8 +49,8 @@ func TestSplitComplexPredicateToLHS(t *testing.T) {
return false, nil
}, ast)

lhsExprs := breakValuesJoinExpressionInLHS(ctx, ast, lID)
nodes := slice.Map(lhsExprs, func(from *sqlparser.ColName) string {
valuesJoinCols := breakValuesJoinExpressionInLHS(ctx, ast, lID)
nodes := slice.Map(valuesJoinCols.LHS, func(from *sqlparser.ColName) string {
return sqlparser.String(from)
})

Expand Down
10 changes: 9 additions & 1 deletion go/vt/vtgate/planbuilder/operators/query_planning.go
Original file line number Diff line number Diff line change
Expand Up @@ -104,7 +104,8 @@ func runRewriters(ctx *plancontext.PlanningContext, root Operator) Operator {
return tryPushUpdate(in)
case *RecurseCTE:
return tryMergeRecurse(ctx, in)

case *Values:
return tryPushValues(in)
default:
return in, NoRewrite
}
Expand All @@ -120,6 +121,13 @@ func runRewriters(ctx *plancontext.PlanningContext, root Operator) Operator {
return FixedPointBottomUp(root, TableID, visitor, stopAtRoute)
}

func tryPushValues(in *Values) (Operator, *ApplyResult) {
if src, ok := in.Source.(*Route); ok {
return Swap(in, src, "pushed values under route")
}
return in, NoRewrite
}

func tryPushDelete(in *Delete) (Operator, *ApplyResult) {
if src, ok := in.Source.(*Route); ok {
return pushDMLUnderRoute(in, src, "pushed delete under route")
Expand Down
29 changes: 27 additions & 2 deletions go/vt/vtgate/planbuilder/operators/route_planning.go
Original file line number Diff line number Diff line change
Expand Up @@ -287,6 +287,31 @@ func requiresSwitchingSides(ctx *plancontext.PlanningContext, op Operator) (requ
return
}

func newJoin(ctx *plancontext.PlanningContext, lhs, rhs Operator, joinType sqlparser.JoinType) JoinOp {
lhsID := TableID(lhs)
if lhsID.NumberOfTables() > 1 || !joinType.IsInner() {
return NewApplyJoin(ctx, lhs, rhs, nil, joinType)
}
lhsTableInfo, err := ctx.SemTable.TableInfoFor(lhsID)
if err != nil {
panic(vterrors.VT13001(err.Error()))
}
lhsTableName, err := lhsTableInfo.Name()
if err != nil {
panic(vterrors.VT13001(err.Error()))
}
bindVariableName := ctx.ReservedVars.ReserveVariable("values")
v := &Values{
unaryOperator: newUnaryOp(rhs),
Name: lhsTableName.Name.String(),
Arg: bindVariableName,
}
return &ValuesJoin{
binaryOperator: newBinaryOp(lhs, v),
bindVarName: bindVariableName,
}
}

func mergeOrJoin(ctx *plancontext.PlanningContext, lhs, rhs Operator, joinPredicates []sqlparser.Expr, joinType sqlparser.JoinType) (Operator, *ApplyResult) {
jm := newJoinMerge(joinPredicates, joinType)
newPlan := jm.mergeJoinInputs(ctx, lhs, rhs, joinPredicates)
Expand All @@ -305,14 +330,14 @@ func mergeOrJoin(ctx *plancontext.PlanningContext, lhs, rhs Operator, joinPredic
return join, Rewrote("use a hash join because we have LIMIT on the LHS")
}

join := NewApplyJoin(ctx, Clone(rhs), Clone(lhs), nil, joinType)
join := newJoin(ctx, Clone(rhs), Clone(lhs), joinType)
for _, pred := range joinPredicates {
join.AddJoinPredicate(ctx, pred)
}
return join, Rewrote("logical join to applyJoin, switching side because LIMIT")
}

join := NewApplyJoin(ctx, Clone(lhs), Clone(rhs), nil, joinType)
join := newJoin(ctx, Clone(lhs), Clone(rhs), joinType)
for _, pred := range joinPredicates {
join.AddJoinPredicate(ctx, pred)
}
Expand Down
95 changes: 77 additions & 18 deletions go/vt/vtgate/planbuilder/operators/values_join.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,13 +27,62 @@ type (

bindVarName string

noColumns
JoinColumns []valuesJoinColumn
JoinPredicates []valuesJoinColumn

// After offset planning
Columns []int
ColumnName []string
}

valuesJoinColumn struct {
Original sqlparser.Expr
LHS []*sqlparser.ColName
PureLHS bool
}
)

var _ Operator = (*ValuesJoin)(nil)
var _ JoinOp = (*ValuesJoin)(nil)

func (vj *ValuesJoin) AddColumn(ctx *plancontext.PlanningContext, reuseExisting bool, addToGroupBy bool, expr *sqlparser.AliasedExpr) int {
if reuseExisting {
if offset := vj.FindCol(ctx, expr.Expr, false); offset >= 0 {
return offset
}
}

vj.JoinColumns = append(vj.JoinColumns, breakValuesJoinExpressionInLHS(ctx, expr.Expr, TableID(vj.LHS)))
vj.ColumnName = append(vj.ColumnName, expr.ColumnName())
return len(vj.JoinColumns) - 1
}

// AddWSColumn is used to add a weight_string column to the operator
func (vj *ValuesJoin) AddWSColumn(ctx *plancontext.PlanningContext, offset int, underRoute bool) int {
panic("oh no")
}

func (vj *ValuesJoin) FindCol(ctx *plancontext.PlanningContext, expr sqlparser.Expr, underRoute bool) int {
for offset, column := range vj.JoinColumns {
if ctx.SemTable.EqualsExpr(column.Original, expr) {
return offset
}
}
return -1
}

func (vj *ValuesJoin) GetColumns(ctx *plancontext.PlanningContext) []*sqlparser.AliasedExpr {
results := make([]*sqlparser.AliasedExpr, len(vj.JoinColumns))
for i, column := range vj.JoinColumns {
results = append(results, sqlparser.NewAliasedExpr(column.Original, vj.ColumnName[i]))
}
return results
}

func (vj *ValuesJoin) GetSelectExprs(ctx *plancontext.PlanningContext) sqlparser.SelectExprs {
return transformColumnsToSelectExprs(ctx, vj)
}

func (vj *ValuesJoin) GetLHS() Operator {
return vj.LHS
}
Expand Down Expand Up @@ -63,23 +112,9 @@ func (vj *ValuesJoin) AddJoinPredicate(ctx *plancontext.PlanningContext, expr sq
return
}
lID := TableID(vj.LHS)
lhsCols := breakValuesJoinExpressionInLHS(ctx, expr, lID)
lhsJoinCols := breakValuesJoinExpressionInLHS(ctx, expr, lID)
vj.RHS = vj.RHS.AddPredicate(ctx, expr)

columns := ctx.ValuesJoinColumns[vj.bindVarName]

outer:
for _, lhsCol := range lhsCols {
for _, ci := range columns {
if ci.Equal(lhsCol.Name) {
// already there, no need to add it again
continue outer
}
}
columns = append(columns, lhsCol.Name)
}

ctx.ValuesJoinColumns[vj.bindVarName] = columns
vj.JoinPredicates = append(vj.JoinPredicates, lhsJoinCols)
}

func (vj *ValuesJoin) Clone(inputs []Operator) Operator {
Expand All @@ -98,5 +133,29 @@ func (vj *ValuesJoin) GetOrdering(ctx *plancontext.PlanningContext) []OrderBy {
}

func (vj *ValuesJoin) planOffsets(ctx *plancontext.PlanningContext) Operator {
panic("implement me")
valuesColumns := ctx.ValuesJoinColumns[vj.bindVarName]
for i, jc := range vj.JoinColumns {
if jc.PureLHS {
offset := vj.LHS.AddColumn(ctx, true, false, sqlparser.NewAliasedExpr(jc.Original, vj.ColumnName[i]))
vj.Columns = append(vj.Columns, ToLeftOffset(offset))
} else {
outer:
for _, lh := range jc.LHS {
_ = vj.LHS.AddColumn(ctx, true, false, aeWrap(lh))
// TODO: can be optimized
for _, ci := range valuesColumns {
if ci.Equal(lh.Name) {
// already there, no need to add it again
continue outer
}
}
valuesColumns = append(valuesColumns, lh.Name)
}

vj.RHS.AddColumn(ctx, true, false)

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Local example using consul on Ubuntu

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Local example using zk2 on Ubuntu

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Local example using etcd on Ubuntu

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / VTop Example

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Region Sharding example using etcd on Ubuntu

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / End-to-End Test (Race)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / End-to-End Test

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (vreplication_partial_movetables_and_materialize)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (backup_pitr_mysqlshell)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (vtgate_reservedconn)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (vtgate_transaction)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (onlineddl_scheduler)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (vtgate_topo)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (vtgate_general_heavy)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (vreplication_multi_tenant)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (vtorc)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (vtgate_readafterwrite)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (mysql80)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (21)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (vtgate_vindex_heavy)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (vtgate_godriver)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (onlineddl_revert)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (13)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (vreplication_foreign_key_stress)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (vtgate_topo_etcd)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (tabletmanager_tablegc)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (vreplication_copy_parallel)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (vtgate_tablet_healthcheck_cache)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (vttablet_prscomplex)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (vreplication_mariadb_to_mysql)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (backup_pitr)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (vtgate_concurrentdml)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (schemadiff_vrepl)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (topo_connection_cache)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (tabletmanager_throttler_topo)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (vtgate_partial_keyspace)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (12)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (vtgate_schema_tracker)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (vtgate_foreignkey_stress)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (vtgate_unsharded)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (vtgate_gen4)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (vreplication_v2)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (15)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (vreplication_cellalias)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (vreplication_across_db_versions)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (mysql_server_vault)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (18)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (backup_pitr_xtrabackup)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (xb_backup)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (vtgate_topo_consul)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (xb_recovery)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Unit Test (mysql80)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (vtgate_vschema)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (tabletmanager_consul)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Unit Test (evalengine_mysql80)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (vtgate_schema)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (vstream)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (vtgate_queries)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Unit Test (mysql57)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Vitess Tester (vtgate)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (vtbackup)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (vtgate_plantests)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (vtctlbackup_sharded_clustertest_heavy)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (ers_prs_newfeatures_heavy)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Unit Test (mysql84)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Unit Test (evalengine_mysql57)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Java Docker Test

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Java Docker Test

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (onlineddl_vrepl)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (vreplication_vtctldclient_vdiff2_movetables_tz)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (vreplication_basic)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (onlineddl_vrepl_stress)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (onlineddl_vrepl_stress_suite)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Code Coverage

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Static Code Checks Etc

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Static Code Checks Etc

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Static Code Checks Etc

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Unit Test (evalengine_mysql84)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run Upgrade Downgrade Test - Query Serving (Queries - 2)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Docker Test Cluster

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Docker Test Cluster

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Docker Test Cluster

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run Upgrade Downgrade Test - Query Serving (Queries)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (onlineddl_vrepl_suite)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run Upgrade Downgrade Test - Backups - E2E

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run Upgrade Downgrade Test - Reparent Old VTTablet

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run Upgrade Downgrade Test - Reparent Old Vtctl

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run Upgrade Downgrade Test - Online DDL flow

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run Upgrade Downgrade Test - Backups - Manual

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run Semi Sync Upgrade Downgrade Test

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run endtoend tests on Cluster (vreplication_migrate)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Unit Test (Evalengine_Race)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Unit Test (Race)

not enough arguments in call to vj.RHS.AddColumn

Check failure on line 155 in go/vt/vtgate/planbuilder/operators/values_join.go

View workflow job for this annotation

GitHub Actions / Run Upgrade Downgrade Test - Query Serving (Schema)

not enough arguments in call to vj.RHS.AddColumn
}

}
ctx.ValuesJoinColumns[vj.bindVarName] = valuesColumns
return vj
}
2 changes: 1 addition & 1 deletion go/vt/vtgate/planbuilder/testdata/onecase.json
Original file line number Diff line number Diff line change
@@ -1,7 +1,7 @@
[
{
"comment": "Add your test case here for debugging and run go test -run=One.",
"query": "",
"query": "select user.id, user_extra.user_id from user, user_extra where user.id = user_extra.toto",
"plan": {
}
}
Expand Down

0 comments on commit 57f12cf

Please sign in to comment.