diff --git a/go/test/endtoend/utils/cmp.go b/go/test/endtoend/utils/cmp.go index 5d03e305154..bf66fc6ef2b 100644 --- a/go/test/endtoend/utils/cmp.go +++ b/go/test/endtoend/utils/cmp.go @@ -233,6 +233,8 @@ func (mcmp *MySQLCompare) ExecWithColumnCompare(query string) *sqltypes.Result { // - MySQL and Vitess did not find an error, but their results are matching // // The result set and error produced by Vitess are returned to the caller. +// If the Vitess and MySQL error are both nil, but the results do not match, +// the mismatched results are instead returned as an error, as well as the Vitess result set func (mcmp *MySQLCompare) ExecAllowAndCompareError(query string) (*sqltypes.Result, error) { mcmp.t.Helper() vtQr, vtErr := mcmp.VtConn.ExecuteFetch(query, 1000, true) @@ -242,7 +244,7 @@ func (mcmp *MySQLCompare) ExecAllowAndCompareError(query string) (*sqltypes.Resu // Since we allow errors, we don't want to compare results if one of the client failed. // Vitess and MySQL should always be agreeing whether the query returns an error or not. if vtErr == nil && mysqlErr == nil { - compareVitessAndMySQLResults(mcmp.t, query, mcmp.VtConn, vtQr, mysqlQr, false) + vtErr = compareVitessAndMySQLResults(mcmp.t, query, mcmp.VtConn, vtQr, mysqlQr, false) } return vtQr, vtErr } diff --git a/go/test/endtoend/utils/mysql.go b/go/test/endtoend/utils/mysql.go index 6249d639a4d..6e85ec6bdf7 100644 --- a/go/test/endtoend/utils/mysql.go +++ b/go/test/endtoend/utils/mysql.go @@ -18,6 +18,7 @@ package utils import ( "context" + "errors" "fmt" "os" "path" @@ -154,24 +155,27 @@ func prepareMySQLWithSchema(params mysql.ConnParams, sql string) error { return nil } -func compareVitessAndMySQLResults(t *testing.T, query string, vtConn *mysql.Conn, vtQr, mysqlQr *sqltypes.Result, compareColumns bool) { +func compareVitessAndMySQLResults(t *testing.T, query string, vtConn *mysql.Conn, vtQr, mysqlQr *sqltypes.Result, compareColumns bool) error { if vtQr == nil && mysqlQr == nil { - return + return nil } if vtQr == nil { t.Error("Vitess result is 'nil' while MySQL's is not.") - return + return errors.New("Vitess result is 'nil' while MySQL's is not.\n") } if mysqlQr == nil { t.Error("MySQL result is 'nil' while Vitess' is not.") - return + return errors.New("MySQL result is 'nil' while Vitess' is not.\n") } + + var errStr string if compareColumns { vtColCount := len(vtQr.Fields) myColCount := len(mysqlQr.Fields) if vtColCount > 0 && myColCount > 0 { if vtColCount != myColCount { t.Errorf("column count does not match: %d vs %d", vtColCount, myColCount) + errStr += fmt.Sprintf("column count does not match: %d vs %d\n", vtColCount, myColCount) } var vtCols []string @@ -180,26 +184,27 @@ func compareVitessAndMySQLResults(t *testing.T, query string, vtConn *mysql.Conn vtCols = append(vtCols, vtField.Name) myCols = append(myCols, mysqlQr.Fields[i].Name) } - assert.Equal(t, myCols, vtCols, "column names do not match - the expected values are what mysql produced") + if !assert.Equal(t, myCols, vtCols, "column names do not match - the expected values are what mysql produced") { + errStr += "column names do not match - the expected values are what mysql produced\n" + errStr += fmt.Sprintf("Not equal: \nexpected: %v\nactual: %v\n", myCols, vtCols) + } } } stmt, err := sqlparser.Parse(query) if err != nil { t.Error(err) - return + return err } orderBy := false if selStmt, isSelStmt := stmt.(sqlparser.SelectStatement); isSelStmt { orderBy = selStmt.GetOrderBy() != nil } - if orderBy && sqltypes.ResultsEqual([]sqltypes.Result{*vtQr}, []sqltypes.Result{*mysqlQr}) { - return - } else if sqltypes.ResultsEqualUnordered([]sqltypes.Result{*vtQr}, []sqltypes.Result{*mysqlQr}) { - return + if (orderBy && sqltypes.ResultsEqual([]sqltypes.Result{*vtQr}, []sqltypes.Result{*mysqlQr})) || sqltypes.ResultsEqualUnordered([]sqltypes.Result{*vtQr}, []sqltypes.Result{*mysqlQr}) { + return nil } - errStr := "Query (" + query + ") results mismatched.\nVitess Results:\n" + errStr += "Query (" + query + ") results mismatched.\nVitess Results:\n" for _, row := range vtQr.Rows { errStr += fmt.Sprintf("%s\n", row) } @@ -212,6 +217,7 @@ func compareVitessAndMySQLResults(t *testing.T, query string, vtConn *mysql.Conn errStr += fmt.Sprintf("query plan: \n%s\n", qr.Rows[0][0].ToString()) } t.Error(errStr) + return errors.New(errStr) } func compareVitessAndMySQLErrors(t *testing.T, vtErr, mysqlErr error) { diff --git a/go/test/endtoend/vtgate/queries/aggregation/fuzz_test.go b/go/test/endtoend/vtgate/queries/aggregation/fuzz_test.go deleted file mode 100644 index ba6fca839c4..00000000000 --- a/go/test/endtoend/vtgate/queries/aggregation/fuzz_test.go +++ /dev/null @@ -1,213 +0,0 @@ -/* -Copyright 2023 The Vitess Authors. - -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 aggregation - -import ( - "fmt" - "math/rand" - "strings" - "testing" - "time" - - "golang.org/x/exp/maps" - - "vitess.io/vitess/go/vt/log" -) - -type ( - column struct { - name string - typ string - } - tableT struct { - name string - columns []column - } -) - -func TestFuzzAggregations(t *testing.T) { - t.Skip("dont run on CI for now") - // This test randomizes values and queries, and checks that mysql returns the same values that Vitess does - mcmp, closer := start(t) - defer closer() - - noOfRows := rand.Intn(20) - var values []string - for i := 0; i < noOfRows; i++ { - values = append(values, fmt.Sprintf("(%d, 'name%d', 'value%d', %d)", i, i, i, i)) - } - t1Insert := fmt.Sprintf("insert into t1 (t1_id, name, value, shardKey) values %s;", strings.Join(values, ",")) - values = nil - noOfRows = rand.Intn(20) - for i := 0; i < noOfRows; i++ { - values = append(values, fmt.Sprintf("(%d, %d)", i, i)) - } - t2Insert := fmt.Sprintf("insert into t2 (id, shardKey) values %s;", strings.Join(values, ",")) - - mcmp.Exec(t1Insert) - mcmp.Exec(t2Insert) - - t.Cleanup(func() { - if t.Failed() { - fmt.Println(t1Insert) - fmt.Println(t2Insert) - } - }) - - schema := map[string]tableT{ - "t1": {name: "t1", columns: []column{ - {name: "t1_id", typ: "bigint"}, - {name: "name", typ: "varchar"}, - {name: "value", typ: "varchar"}, - {name: "shardKey", typ: "bigint"}, - }}, - "t2": {name: "t2", columns: []column{ - {name: "id", typ: "bigint"}, - {name: "shardKey", typ: "bigint"}, - }}, - } - - endBy := time.Now().Add(1 * time.Second) - schemaTables := maps.Values(schema) - - var queryCount int - for time.Now().Before(endBy) || t.Failed() { - tables := createTables(schemaTables) - query := randomQuery(tables, 3, 3) - mcmp.Exec(query) - if t.Failed() { - fmt.Println(query) - } - queryCount++ - } - log.Info("Queries successfully executed: %d", queryCount) -} - -func randomQuery(tables []tableT, maxAggrs, maxGroupBy int) string { - randomCol := func(tblIdx int) (string, string) { - tbl := tables[tblIdx] - col := randomEl(tbl.columns) - return fmt.Sprintf("tbl%d.%s", tblIdx, col.name), col.typ - } - predicates := createPredicates(tables, randomCol) - aggregates := createAggregations(tables, maxAggrs, randomCol) - grouping := createGroupBy(tables, maxGroupBy, randomCol) - sel := "select /*vt+ PLANNER=Gen4 */ " + strings.Join(aggregates, ", ") + " from " - - var tbls []string - for i, s := range tables { - tbls = append(tbls, fmt.Sprintf("%s as tbl%d", s.name, i)) - } - sel += strings.Join(tbls, ", ") - - if len(predicates) > 0 { - sel += " where " - sel += strings.Join(predicates, " and ") - } - if len(grouping) > 0 { - sel += " group by " - sel += strings.Join(grouping, ", ") - } - // we do it this way so we don't have to do only `only_full_group_by` queries - var noOfOrderBy int - if len(grouping) > 0 { - // panic on rand function call if value is 0 - noOfOrderBy = rand.Intn(len(grouping)) - } - if noOfOrderBy > 0 { - noOfOrderBy = 0 // TODO turning on ORDER BY here causes lots of failures to happen - } - if noOfOrderBy > 0 { - var orderBy []string - for noOfOrderBy > 0 { - noOfOrderBy-- - if rand.Intn(2) == 0 || len(grouping) == 0 { - orderBy = append(orderBy, randomEl(aggregates)) - } else { - orderBy = append(orderBy, randomEl(grouping)) - } - } - sel += " order by " - sel += strings.Join(orderBy, ", ") - } - return sel -} - -func createGroupBy(tables []tableT, maxGB int, randomCol func(tblIdx int) (string, string)) (grouping []string) { - noOfGBs := rand.Intn(maxGB) - for i := 0; i < noOfGBs; i++ { - tblIdx := rand.Intn(len(tables)) - col, _ := randomCol(tblIdx) - grouping = append(grouping, col) - } - return -} - -func createAggregations(tables []tableT, maxAggrs int, randomCol func(tblIdx int) (string, string)) (aggregates []string) { - aggregations := []func(string) string{ - func(_ string) string { return "count(*)" }, - func(e string) string { return fmt.Sprintf("count(%s)", e) }, - //func(e string) string { return fmt.Sprintf("sum(%s)", e) }, - //func(e string) string { return fmt.Sprintf("avg(%s)", e) }, - //func(e string) string { return fmt.Sprintf("min(%s)", e) }, - //func(e string) string { return fmt.Sprintf("max(%s)", e) }, - } - - noOfAggrs := rand.Intn(maxAggrs) + 1 - for i := 0; i < noOfAggrs; i++ { - tblIdx := rand.Intn(len(tables)) - e, _ := randomCol(tblIdx) - aggregates = append(aggregates, randomEl(aggregations)(e)) - } - return aggregates -} - -func createTables(schemaTables []tableT) []tableT { - noOfTables := rand.Intn(2) + 1 - var tables []tableT - - for i := 0; i < noOfTables; i++ { - tables = append(tables, randomEl(schemaTables)) - } - return tables -} - -func createPredicates(tables []tableT, randomCol func(tblIdx int) (string, string)) (predicates []string) { - for idx1 := range tables { - for idx2 := range tables { - if idx1 == idx2 { - continue - } - noOfPredicates := rand.Intn(2) - - for noOfPredicates > 0 { - col1, t1 := randomCol(idx1) - col2, t2 := randomCol(idx2) - if t1 != t2 { - continue - } - predicates = append(predicates, fmt.Sprintf("%s = %s", col1, col2)) - noOfPredicates-- - } - } - } - return predicates -} - -func randomEl[K any](in []K) K { - return in[rand.Intn(len(in))] -} diff --git a/go/test/endtoend/vtgate/queries/random/main_test.go b/go/test/endtoend/vtgate/queries/random/main_test.go new file mode 100644 index 00000000000..bfef910f036 --- /dev/null +++ b/go/test/endtoend/vtgate/queries/random/main_test.go @@ -0,0 +1,94 @@ +/* +Copyright 2023 The Vitess Authors. + +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 random + +import ( + _ "embed" + "flag" + "fmt" + "os" + "testing" + + "vitess.io/vitess/go/test/endtoend/utils" + + "vitess.io/vitess/go/mysql" + "vitess.io/vitess/go/test/endtoend/cluster" +) + +var ( + clusterInstance *cluster.LocalProcessCluster + vtParams mysql.ConnParams + mysqlParams mysql.ConnParams + keyspaceName = "ks_random" + cell = "test_random" + + //go:embed schema.sql + schemaSQL string + + //go:embed vschema.json + vschema string +) + +func TestMain(m *testing.M) { + defer cluster.PanicHandler(nil) + flag.Parse() + + exitCode := func() int { + clusterInstance = cluster.NewCluster(cell, "localhost") + defer clusterInstance.Teardown() + + // Start topo server + err := clusterInstance.StartTopo() + if err != nil { + return 1 + } + + // Start keyspace + keyspace := &cluster.Keyspace{ + Name: keyspaceName, + SchemaSQL: schemaSQL, + VSchema: vschema, + } + clusterInstance.VtGateExtraArgs = []string{"--schema_change_signal"} + clusterInstance.VtTabletExtraArgs = []string{"--queryserver-config-schema-change-signal", "--queryserver-config-schema-change-signal-interval", "0.1"} + err = clusterInstance.StartKeyspace(*keyspace, []string{"-80", "80-"}, 0, false) + if err != nil { + return 1 + } + + clusterInstance.VtGateExtraArgs = append(clusterInstance.VtGateExtraArgs, "--enable_system_settings=true") + // Start vtgate + err = clusterInstance.StartVtgate() + if err != nil { + return 1 + } + + vtParams = clusterInstance.GetVTParams(keyspaceName) + + // create mysql instance and connection parameters + conn, closer, err := utils.NewMySQL(clusterInstance, keyspaceName, schemaSQL) + if err != nil { + fmt.Println(err) + return 1 + } + defer closer() + mysqlParams = conn + + return m.Run() + }() + os.Exit(exitCode) +} diff --git a/go/test/endtoend/vtgate/queries/random/query_gen.go b/go/test/endtoend/vtgate/queries/random/query_gen.go new file mode 100644 index 00000000000..d673a904189 --- /dev/null +++ b/go/test/endtoend/vtgate/queries/random/query_gen.go @@ -0,0 +1,490 @@ +/* +Copyright 2023 The Vitess Authors. + +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 random + +import ( + "fmt" + "math/rand" + "time" + + "golang.org/x/exp/slices" + + "vitess.io/vitess/go/slices2" + "vitess.io/vitess/go/vt/log" + "vitess.io/vitess/go/vt/sqlparser" +) + +// this file contains the structs and functions to generate random queries + +// if true then known failing query types are still generated by randomQuery() +const testFailingQueries = false + +type ( + column struct { + tableName string + name string + typ string + } + tableT struct { + // the tableT struct can be used to represent the schema of a table or a derived table + // in the former case name will be a sqlparser.TableName, in the latter a sqlparser.DerivedTable + // in order to create a query with a derived table, its AST form is retrieved from name + // once the derived table is aliased, name is replaced by a sqlparser.TableName with that alias + name sqlparser.SimpleTableExpr + cols []column + } +) + +var _ sqlparser.ExprGenerator = (*tableT)(nil) + +func (t *tableT) typeExpr(typ string) sqlparser.Expr { + tableCopy := t.clone() + + for len(tableCopy.cols) > 0 { + idx := rand.Intn(len(tableCopy.cols)) + randCol := tableCopy.cols[idx] + if randCol.typ == typ { + newTableName := "" + if tName, ok := tableCopy.name.(sqlparser.TableName); ok { + newTableName = sqlparser.String(tName.Name) + } + return sqlparser.NewColNameWithQualifier(randCol.name, sqlparser.NewTableName(newTableName)) + } + + // delete randCol from table.columns + tableCopy.cols[idx] = tableCopy.cols[len(tableCopy.cols)-1] + tableCopy.cols = tableCopy.cols[:len(tableCopy.cols)-1] + } + + return nil +} + +func (t *tableT) IntExpr() sqlparser.Expr { + // TODO: better way to check if int type? + return t.typeExpr("bigint") +} + +func (t *tableT) StringExpr() sqlparser.Expr { + return t.typeExpr("varchar") +} + +// setName sets the alias for t, as well as setting the tableName for all columns in cols +func (t *tableT) setName(newName string) { + t.name = sqlparser.NewTableName(newName) + for i := range t.cols { + t.cols[i].tableName = newName + } +} + +// setColumns sets the columns of t, and automatically assigns tableName +// this makes it unnatural (but still possible as cols is exportable) to modify tableName +func (t *tableT) setColumns(col ...column) { + t.cols = nil + t.addColumns(col...) +} + +// addColumns adds columns to t, and automatically assigns tableName +// this makes it unnatural (but still possible as cols is exportable) to modify tableName +func (t *tableT) addColumns(col ...column) { + for i := range col { + // only change the Col's tableName if t is of type tableName + if tName, ok := t.name.(sqlparser.TableName); ok { + col[i].tableName = sqlparser.String(tName.Name) + } + + t.cols = append(t.cols, col[i]) + } +} + +// clone returns a deep copy of t +func (t *tableT) clone() *tableT { + return &tableT{ + name: t.name, + cols: slices.Clone(t.cols), + } +} + +// getColumnName returns tableName.name +func (c *column) getColumnName() string { + return fmt.Sprintf("%s.%s", c.tableName, c.name) +} + +func randomQuery(schemaTables []tableT, maxAggrs, maxGroupBy int) *sqlparser.Select { + sel := &sqlparser.Select{} + sel.SetComments(sqlparser.Comments{"/*vt+ PLANNER=Gen4 */"}) + + // select distinct (fails with group by bigint) + isDistinct := rand.Intn(2) < 1 + if isDistinct { + sel.MakeDistinct() + } + + // create both tables and join at the same time since both occupy the from clause + tables, isJoin := createTablesAndJoin(schemaTables, sel) + + var ( + groupBy sqlparser.GroupBy + groupSelectExprs sqlparser.SelectExprs + grouping []column + ) + // TODO: distinct makes vitess think there is grouping on aggregation columns + if testFailingQueries || !isDistinct { + groupBy, groupSelectExprs, grouping = createGroupBy(tables, maxGroupBy) + sel.AddSelectExprs(groupSelectExprs) + sel.GroupBy = groupBy + + } + + aggrExprs, aggregates := createAggregations(tables, maxAggrs) + sel.AddSelectExprs(aggrExprs) + + // can add both aggregate and grouping columns to order by + // TODO: order fails with distinct and outer joins + isOrdered := rand.Intn(2) < 1 && (!isDistinct || testFailingQueries) && (!isJoin || testFailingQueries) && testFailingQueries + // TODO: order by fails a lot; probably related to the previously passing query + // TODO: should be fixed soon + if isOrdered { + sel.OrderBy = createOrderBy(groupBy, aggrExprs) + } + + // where + sel.AddWhere(sqlparser.AndExpressions(createWherePredicates(tables, false)...)) + + // random predicate expression + // TODO: random expressions cause a lot of failures + if rand.Intn(2) < 1 && testFailingQueries { + predRandomExpr := getRandomExpr(tables) + sel.AddWhere(predRandomExpr) + } + + // having + isHaving := rand.Intn(2) < 1 + if isHaving { + sel.AddHaving(sqlparser.AndExpressions(createHavingPredicates(tables)...)) + if rand.Intn(2) < 1 && testFailingQueries { + // TODO: having can only contain aggregate or grouping columns in mysql, works fine in vitess + // TODO: Can fix this by putting only the table with the grouping and aggregates column (newTable) + sel.AddHaving(sqlparser.AndExpressions(createWherePredicates(tables, false)...)) + } + } + // TODO: use sqlparser.ExprGenerator to generate a random expression with aggregation functions + + // only add a limit if the grouping columns are ordered + // TODO: limit fails a lot + if rand.Intn(2) < 1 && (isOrdered || len(groupBy) == 0) && testFailingQueries { + sel.Limit = createLimit() + } + + var newTable tableT + // add random expression to select + // TODO: random expressions cause a lot of failures + isRandomExpr := rand.Intn(2) < 1 && testFailingQueries + var ( + randomExpr sqlparser.Expr + typ string + ) + // TODO: selecting a random expression potentially with columns creates + // TODO: only_full_group_by related errors in Vitess + if testFailingQueries { + randomExpr = getRandomExpr(tables) + } else { + randomExpr = getRandomExpr(nil) + } + + // make sure we have at least one select expression + if isRandomExpr || len(sel.SelectExprs) == 0 { + // TODO: select distinct [literal] fails + sel.Distinct = false + + sel.SelectExprs = append(sel.SelectExprs, sqlparser.NewAliasedExpr(randomExpr, "crandom0")) + newTable.addColumns(column{ + name: "crandom0", + typ: typ, + }) + + // make sure to add the random expression to group by for only_full_group_by + sel.AddGroupBy(randomExpr) + } + + // add them to newTable + newTable.addColumns(grouping...) + newTable.addColumns(aggregates...) + + // add new table to schemaTables + newTable.name = sqlparser.NewDerivedTable(false, sel) + schemaTables = append(schemaTables, newTable) + + // derived tables (partially unsupported) + // TODO: derived tables fails a lot + if rand.Intn(10) < 1 && testFailingQueries { + sel = randomQuery(schemaTables, 3, 3) + } + + return sel +} + +func createTablesAndJoin(schemaTables []tableT, sel *sqlparser.Select) ([]tableT, bool) { + var tables []tableT + // add at least one of original emp/dept tables for now because derived tables have nil columns + tables = append(tables, schemaTables[rand.Intn(2)]) + + sel.From = append(sel.From, newAliasedTable(tables[0], "tbl0")) + tables[0].setName("tbl0") + + numTables := rand.Intn(len(schemaTables)) + for i := 0; i < numTables; i++ { + tables = append(tables, randomEl(schemaTables)) + sel.From = append(sel.From, newAliasedTable(tables[i+1], fmt.Sprintf("tbl%d", i+1))) + tables[i+1].setName(fmt.Sprintf("tbl%d", i+1)) + } + + // TODO: outer joins produce mismatched results + isJoin := rand.Intn(2) < 1 && testFailingQueries + if isJoin { + newTable := randomEl(schemaTables) + tables = append(tables, newTable) + + // create the join before aliasing + newJoinTableExpr := createJoin(tables, sel) + + // alias + tables[numTables+1].setName(fmt.Sprintf("tbl%d", numTables+1)) + + // create the condition after aliasing + newJoinTableExpr.Condition = sqlparser.NewJoinCondition(sqlparser.AndExpressions(createWherePredicates(tables, true)...), nil) + sel.From[numTables] = newJoinTableExpr + } + + return tables, isJoin +} + +// creates a left join (without the condition) between the last table in sel and newTable +// tables should have one more table than sel +func createJoin(tables []tableT, sel *sqlparser.Select) *sqlparser.JoinTableExpr { + n := len(sel.From) + if len(tables) != n+1 { + log.Fatalf("sel has %d tables and tables has %d tables", len(sel.From), n) + } + + return sqlparser.NewJoinTableExpr(sel.From[n-1], sqlparser.LeftJoinType, newAliasedTable(tables[n], fmt.Sprintf("tbl%d", n)), nil) +} + +// returns the grouping columns as three types: sqlparser.GroupBy, sqlparser.SelectExprs, []column +func createGroupBy(tables []tableT, maxGB int) (groupBy sqlparser.GroupBy, groupSelectExprs sqlparser.SelectExprs, grouping []column) { + numGBs := rand.Intn(maxGB) + for i := 0; i < numGBs; i++ { + tblIdx := rand.Intn(len(tables)) + col := randomEl(tables[tblIdx].cols) + // TODO: grouping by a date column sometimes errors + if col.typ == "date" && !testFailingQueries { + continue + } + groupBy = append(groupBy, newColumn(col)) + + // add to select + if rand.Intn(2) < 1 { + groupSelectExprs = append(groupSelectExprs, newAliasedColumn(col, fmt.Sprintf("cgroup%d", i))) + // TODO: alias in a separate function to properly generate the having clause + col.name = fmt.Sprintf("cgroup%d", i) + grouping = append(grouping, col) + } + } + + return +} + +// returns the aggregation columns as three types: sqlparser.SelectExprs, []column +func createAggregations(tables []tableT, maxAggrs int) (aggrExprs sqlparser.SelectExprs, aggregates []column) { + aggregations := []func(col column) sqlparser.Expr{ + func(_ column) sqlparser.Expr { return &sqlparser.CountStar{} }, + func(col column) sqlparser.Expr { return &sqlparser.Count{Args: sqlparser.Exprs{newColumn(col)}} }, + func(col column) sqlparser.Expr { return &sqlparser.Sum{Arg: newColumn(col)} }, + // func(col column) sqlparser.Expr { return &sqlparser.Avg{Arg: newAggregateExpr(col)} }, + func(col column) sqlparser.Expr { return &sqlparser.Min{Arg: newColumn(col)} }, + func(col column) sqlparser.Expr { return &sqlparser.Max{Arg: newColumn(col)} }, + } + + numAggrs := rand.Intn(maxAggrs) + for i := 0; i < numAggrs; i++ { + tblIdx, aggrIdx := rand.Intn(len(tables)), rand.Intn(len(aggregations)) + col := randomEl(tables[tblIdx].cols) + // TODO: aggregating on a date column sometimes errors + if col.typ == "date" && !testFailingQueries { + i-- + continue + } + + newAggregate := aggregations[aggrIdx](col) + // TODO: collating on strings sometimes errors + if col.typ == "varchar" && !testFailingQueries { + switch newAggregate.(type) { + case *sqlparser.Min, *sqlparser.Max: + i-- + continue + } + } + + // TODO: type of sum() is incorrect (int64 vs decimal) in certain queries + if _, ok := newAggregate.(*sqlparser.Sum); ok && !testFailingQueries { + i-- + continue + } + + aggrExprs = append(aggrExprs, sqlparser.NewAliasedExpr(newAggregate, fmt.Sprintf("caggr%d", i))) + + if aggrIdx <= 1 /* CountStar and Count */ { + col.typ = "bigint" + } else if _, ok := newAggregate.(*sqlparser.Avg); ok && col.getColumnName() == "bigint" { + col.typ = "decimal" + } + + col.name = fmt.Sprintf("caggr%d", i) + aggregates = append(aggregates, col) + } + return +} + +// orders on all non-aggregate SelectExprs and independently at random on all aggregate SelectExprs of sel +func createOrderBy(groupBy sqlparser.GroupBy, aggrExprs sqlparser.SelectExprs) (orderBy sqlparser.OrderBy) { + // always order on grouping columns + for i := range groupBy { + orderBy = append(orderBy, sqlparser.NewOrder(groupBy[i], getRandomOrderDirection())) + } + + // randomly order on aggregation columns + for i := range aggrExprs { + if aliasedExpr, ok := aggrExprs[i].(*sqlparser.AliasedExpr); ok && rand.Intn(2) < 1 { + orderBy = append(orderBy, sqlparser.NewOrder(aliasedExpr.Expr, getRandomOrderDirection())) + } + } + + return +} + +// compares two random columns (usually of the same type) +// returns a random expression if there are no other predicates and isJoin is true +// returns the predicates as a sqlparser.Exprs (slice of sqlparser.Expr's) +func createWherePredicates(tables []tableT, isJoin bool) (predicates sqlparser.Exprs) { + // if creating predicates for a join, + // then make sure predicates are created for the last two tables (which are being joined) + incr := 0 + if isJoin && len(tables) > 2 { + incr += len(tables) - 2 + } + + for idx1 := range tables { + for idx2 := range tables { + // fmt.Printf("predicate tables:\n%v\n idx1: %d idx2: %d, incr: %d", tables, idx1, idx2, incr) + if idx1 >= idx2 || idx1 < incr || idx2 < incr { + continue + } + noOfPredicates := rand.Intn(2) + if isJoin { + noOfPredicates++ + } + + for i := 0; noOfPredicates > 0; i++ { + col1 := randomEl(tables[idx1].cols) + col2 := randomEl(tables[idx2].cols) + + // prevent infinite loops + if i > 50 { + predicates = append(predicates, sqlparser.NewComparisonExpr(getRandomComparisonExprOperator(), newColumn(col1), newColumn(col2), nil)) + break + } + + if col1.typ != col2.typ { + continue + } + + predicates = append(predicates, sqlparser.NewComparisonExpr(getRandomComparisonExprOperator(), newColumn(col1), newColumn(col2), nil)) + noOfPredicates-- + } + } + } + + // make sure the join predicate is never empty + if len(predicates) == 0 && isJoin { + predRandomExpr := getRandomExpr(tables) + predicates = append(predicates, predRandomExpr) + } + + return +} + +// creates predicates for the having clause comparing a column to a random expression +func createHavingPredicates(tables []tableT) (havingPredicates sqlparser.Exprs) { + aggrSelectExprs, _ := createAggregations(tables, 2) + for i := range aggrSelectExprs { + if lhs, ok := aggrSelectExprs[i].(*sqlparser.AliasedExpr); ok { + // TODO: HAVING can only contain aggregate or grouping columns in mysql, works fine in vitess + // TODO: Can fix this by putting only the table with the grouping and aggregates column (newTable) + // TODO: but random expressions without the columns also fails + if testFailingQueries { + predRandomExpr := getRandomExpr(tables) + havingPredicates = append(havingPredicates, sqlparser.NewComparisonExpr(getRandomComparisonExprOperator(), lhs.Expr, predRandomExpr, nil)) + } else if rhs, ok1 := randomEl(aggrSelectExprs).(*sqlparser.AliasedExpr); ok1 { + havingPredicates = append(havingPredicates, sqlparser.NewComparisonExpr(getRandomComparisonExprOperator(), lhs.Expr, rhs.Expr, nil)) + } + } + } + return +} + +// creates sel.Limit +func createLimit() *sqlparser.Limit { + limitNum := rand.Intn(10) + if rand.Intn(2) < 1 { + offset := rand.Intn(10) + return sqlparser.NewLimit(offset, limitNum) + } + + return sqlparser.NewLimitWithoutOffset(limitNum) +} + +// returns a random expression and its type +func getRandomExpr(tables []tableT) sqlparser.Expr { + seed := time.Now().UnixNano() + g := sqlparser.NewGenerator(seed, 2, slices2.Map(tables, func(t tableT) sqlparser.ExprGenerator { return &t })...) + return g.Expression() +} + +func newAliasedTable(tbl tableT, alias string) *sqlparser.AliasedTableExpr { + return sqlparser.NewAliasedTableExpr(tbl.name, alias) +} + +func newAliasedColumn(col column, alias string) *sqlparser.AliasedExpr { + return sqlparser.NewAliasedExpr(newColumn(col), alias) +} + +func newColumn(col column) *sqlparser.ColName { + return sqlparser.NewColNameWithQualifier(col.name, sqlparser.NewTableName(col.tableName)) +} + +func getRandomComparisonExprOperator() sqlparser.ComparisonExprOperator { + // =, <, >, <=, >=, !=, <=> + return randomEl([]sqlparser.ComparisonExprOperator{0, 1, 2, 3, 4, 5, 6}) +} + +func getRandomOrderDirection() sqlparser.OrderDirection { + // asc, desc + return randomEl([]sqlparser.OrderDirection{0, 1}) +} + +func randomEl[K any](in []K) K { + return in[rand.Intn(len(in))] +} diff --git a/go/test/endtoend/vtgate/queries/random/random_expr_test.go b/go/test/endtoend/vtgate/queries/random/random_expr_test.go new file mode 100644 index 00000000000..b4e1cfb9c2d --- /dev/null +++ b/go/test/endtoend/vtgate/queries/random/random_expr_test.go @@ -0,0 +1,52 @@ +/* +Copyright 2023 The Vitess Authors. + +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 random + +import ( + "testing" + "time" + + "vitess.io/vitess/go/slices2" + "vitess.io/vitess/go/vt/sqlparser" +) + +// This test tests that generating a random expression with a schema does not panic +func TestRandomExprWithTables(t *testing.T) { + schemaTables := []tableT{ + {name: sqlparser.NewTableName("emp")}, + {name: sqlparser.NewTableName("dept")}, + } + schemaTables[0].addColumns([]column{ + {name: "empno", typ: "bigint"}, + {name: "ename", typ: "varchar"}, + {name: "job", typ: "varchar"}, + {name: "mgr", typ: "bigint"}, + {name: "hiredate", typ: "date"}, + {name: "sal", typ: "bigint"}, + {name: "comm", typ: "bigint"}, + {name: "deptno", typ: "bigint"}, + }...) + schemaTables[1].addColumns([]column{ + {name: "deptno", typ: "bigint"}, + {name: "dname", typ: "varchar"}, + {name: "loc", typ: "varchar"}, + }...) + + seed := time.Now().UnixNano() + g := sqlparser.NewGenerator(seed, 3, slices2.Map(schemaTables, func(t tableT) sqlparser.ExprGenerator { return &t })...) + g.Expression() +} diff --git a/go/test/endtoend/vtgate/queries/random/random_test.go b/go/test/endtoend/vtgate/queries/random/random_test.go new file mode 100644 index 00000000000..a51b919e0dc --- /dev/null +++ b/go/test/endtoend/vtgate/queries/random/random_test.go @@ -0,0 +1,310 @@ +/* +Copyright 2023 The Vitess Authors. + +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 random + +import ( + "fmt" + "strings" + "testing" + "time" + + "vitess.io/vitess/go/mysql" + "vitess.io/vitess/go/vt/sqlparser" + + "github.com/stretchr/testify/require" + + "vitess.io/vitess/go/test/endtoend/cluster" + "vitess.io/vitess/go/test/endtoend/utils" +) + +// this test uses the AST defined in the sqlparser package to randomly generate queries + +// if true then execution will always stop on a "must fix" error: a mismatched results or EOF +const stopOnMustFixError = true + +func start(t *testing.T) (utils.MySQLCompare, func()) { + mcmp, err := utils.NewMySQLCompare(t, vtParams, mysqlParams) + require.NoError(t, err) + + deleteAll := func() { + _, _ = utils.ExecAllowError(t, mcmp.VtConn, "set workload = oltp") + + tables := []string{"dept", "emp"} + for _, table := range tables { + _, _ = mcmp.ExecAndIgnore("delete from " + table) + } + } + + deleteAll() + + // disable only_full_group_by + // mcmp.Exec("set sql_mode=''") + + // insert data + mcmp.Exec("INSERT INTO emp(empno, ename, job, mgr, hiredate, sal, comm, deptno) VALUES (7369,'SMITH','CLERK',7902,'1980-12-17',800,NULL,20), (7499,'ALLEN','SALESMAN',7698,'1981-02-20',1600,300,30), (7521,'WARD','SALESMAN',7698,'1981-02-22',1250,500,30), (7566,'JONES','MANAGER',7839,'1981-04-02',2975,NULL,20), (7654,'MARTIN','SALESMAN',7698,'1981-09-28',1250,1400,30), (7698,'BLAKE','MANAGER',7839,'1981-05-01',2850,NULL,30), (7782,'CLARK','MANAGER',7839,'1981-06-09',2450,NULL,10), (7788,'SCOTT','ANALYST',7566,'1982-12-09',3000,NULL,20), (7839,'KING','PRESIDENT',NULL,'1981-11-17',5000,NULL,10), (7844,'TURNER','SALESMAN',7698,'1981-09-08',1500,0,30), (7876,'ADAMS','CLERK',7788,'1983-01-12',1100,NULL,20), (7900,'JAMES','CLERK',7698,'1981-12-03',950,NULL,30), (7902,'FORD','ANALYST',7566,'1981-12-03',3000,NULL,20), (7934,'MILLER','CLERK',7782,'1982-01-23',1300,NULL,10)") + mcmp.Exec("INSERT INTO dept(deptno, dname, loc) VALUES ('10','ACCOUNTING','NEW YORK'), ('20','RESEARCH','DALLAS'), ('30','SALES','CHICAGO'), ('40','OPERATIONS','BOSTON')") + + return mcmp, func() { + deleteAll() + mcmp.Close() + cluster.PanicHandler(t) + } +} + +func helperTest(t *testing.T, query string) { + t.Helper() + t.Run(query, func(t *testing.T) { + mcmp, closer := start(t) + defer closer() + + result, err := mcmp.ExecAllowAndCompareError(query) + fmt.Println(result) + fmt.Println(err) + }) +} + +func TestMustFix(t *testing.T) { + t.Skip("Skip CI") + + require.NoError(t, utils.WaitForAuthoritative(t, keyspaceName, "emp", clusterInstance.VtgateProcess.ReadVSchema)) + require.NoError(t, utils.WaitForAuthoritative(t, keyspaceName, "dept", clusterInstance.VtgateProcess.ReadVSchema)) + + // mismatched results + // sum values returned as int64 instead of decimal + helperTest(t, "select /*vt+ PLANNER=Gen4 */ sum(tbl1.sal) as caggr1 from emp as tbl0, emp as tbl1 group by tbl1.ename order by tbl1.ename asc") + + // mismatched results + // limit >= 9 works + helperTest(t, "select /*vt+ PLANNER=Gen4 */ tbl0.ename as cgroup1 from emp as tbl0 group by tbl0.job, tbl0.ename having sum(tbl0.mgr) = sum(tbl0.mgr) order by tbl0.job desc, tbl0.ename asc limit 8") + + // mismatched results + helperTest(t, "select /*vt+ PLANNER=Gen4 */ distinct count(*) as caggr1 from dept as tbl0, emp as tbl1 group by tbl1.sal having max(tbl1.comm) != true") + + // mismatched results + helperTest(t, "select /*vt+ PLANNER=Gen4 */ distinct sum(tbl1.loc) as caggr0 from dept as tbl0, dept as tbl1 group by tbl1.deptno having max(tbl1.dname) <= 1") + + // mismatched results + helperTest(t, "select /*vt+ PLANNER=Gen4 */ distinct max(tbl0.dname) as caggr0, 'cattle' as crandom0 from dept as tbl0, emp as tbl1 where tbl0.deptno != tbl1.sal group by tbl1.comm") + + // mismatched results + helperTest(t, "select /*vt+ PLANNER=Gen4 */ count(*) as caggr0, 1 as crandom0 from dept as tbl0, emp as tbl1 where 1 = 0") + + // mismatched results + helperTest(t, "select /*vt+ PLANNER=Gen4 */ count(*) as caggr0, 1 as crandom0 from dept as tbl0, emp as tbl1 where 'octopus'") + + // similar to previous two + // mismatched results + helperTest(t, "select /*vt+ PLANNER=Gen4 */ distinct 'octopus' as crandom0 from dept as tbl0, emp as tbl1 where tbl0.deptno = tbl1.empno having count(*) = count(*)") + + // mismatched results + // previously failing, then succeeding query, now failing again + helperTest(t, "select /*vt+ PLANNER=Gen4 */ count(tbl0.deptno) from dept as tbl0, emp as tbl1 group by tbl1.job order by tbl1.job limit 3") + + // mismatched results (group by + right join) + // left instead of right works + // swapping tables and predicates and changing to left fails + helperTest(t, "select /*vt+ PLANNER=Gen4 */ max(tbl0.deptno) from dept as tbl0 right join emp as tbl1 on tbl0.deptno = tbl1.empno and tbl0.deptno = tbl1.deptno group by tbl0.deptno") + + // mismatched results (count + right join) + // left instead of right works + // swapping tables and predicates and changing to left fails + helperTest(t, "select /*vt+ PLANNER=Gen4 */ count(tbl1.comm) from emp as tbl1 right join emp as tbl2 on tbl1.mgr = tbl2.sal") + + // EOF + helperTest(t, "select /*vt+ PLANNER=Gen4 */ count(*) from dept as tbl0, (select count(*) from emp as tbl0, emp as tbl1 limit 18) as tbl1") + + // EOF + helperTest(t, "select /*vt+ PLANNER=Gen4 */ count(*), count(*) from (select count(*) from dept as tbl0 group by tbl0.deptno) as tbl0") +} + +func TestKnownFailures(t *testing.T) { + t.Skip("Skip CI") + + require.NoError(t, utils.WaitForAuthoritative(t, keyspaceName, "emp", clusterInstance.VtgateProcess.ReadVSchema)) + require.NoError(t, utils.WaitForAuthoritative(t, keyspaceName, "dept", clusterInstance.VtgateProcess.ReadVSchema)) + + // logs more stuff + //clusterInstance.EnableGeneralLog() + + // cannot compare strings, collation is unknown or unsupported (collation ID: 0) + helperTest(t, "select /*vt+ PLANNER=Gen4 */ max(tbl1.dname) as caggr1 from dept as tbl0, dept as tbl1 group by tbl1.dname order by tbl1.dname asc") + + // vitess error: + // mysql error: Incorrect DATE value: 'tuna' + helperTest(t, "select /*vt+ PLANNER=Gen4 */ min(tbl0.empno) as caggr0 from emp as tbl0 where case 'gator' when false then 314 else 'weevil' end > tbl0.job having min(tbl0.hiredate) <=> 'tuna'") + + // vitess error: + // mysql error: Unknown column 'tbl0.deptno' in 'having clause' + helperTest(t, "select /*vt+ PLANNER=Gen4 */ count(*) as caggr0 from dept as tbl0 having tbl0.deptno") + + // coercion should not try to coerce this value: DATE("1980-12-17") + helperTest(t, "select /*vt+ PLANNER=Gen4 */ distinct tbl1.hiredate as cgroup0, count(tbl1.mgr) as caggr0 from emp as tbl1 group by tbl1.hiredate, tbl1.ename") + + // only_full_group_by enabled (vitess sometimes (?) produces the correct result assuming only_full_group_by is disabled) + // vitess error: nil + // mysql error: In aggregated query without GROUP BY, expression #1 of SELECT list contains nonaggregated column 'ks_random.tbl0.ENAME'; this is incompatible with sql_mode=only_full_group_by + helperTest(t, "select /*vt+ PLANNER=Gen4 */ tbl0.ename, min(tbl0.comm) from emp as tbl0 left join emp as tbl1 on tbl0.empno = tbl1.comm and tbl0.empno = tbl1.empno") + + // only_full_group_by enabled + // vitess error: nil + // mysql error: Expression #1 of ORDER BY clause is not in SELECT list, references column 'ks_random.tbl2.DNAME' which is not in SELECT list; this is incompatible with DISTINCT + helperTest(t, "select /*vt+ PLANNER=Gen4 */ distinct count(*) as caggr0 from dept as tbl2 group by tbl2.dname order by tbl2.dname asc") + + // vttablet: rpc error: code = NotFound desc = Unknown column 'cgroup0' in 'field list' (errno 1054) (sqlstate 42S22) (CallerID: userData1) + helperTest(t, "select /*vt+ PLANNER=Gen4 */ tbl1.ename as cgroup0, max(tbl0.comm) as caggr0 from emp as tbl0, emp as tbl1 group by cgroup0") + + // vttablet: rpc error: code = NotFound desc = Unknown column '347' in 'group statement' + helperTest(t, "select /*vt+ PLANNER=Gen4 */ distinct 347 as crandom0 from emp as tbl0") + + // vttablet: rpc error: code = InvalidArgument desc = Can't group on 'count(*)' (errno 1056) (sqlstate 42000) (CallerID: userData1) + helperTest(t, "select /*vt+ PLANNER=Gen4 */ distinct count(*) from dept as tbl0 group by tbl0.deptno") + + // [BUG] push projection does not yet support: *planbuilder.memorySort (errno 1815) (sqlstate HY000) + helperTest(t, "select /*vt+ PLANNER=Gen4 */ count(*) from dept as tbl1 join (select count(*) from emp as tbl0, dept as tbl1 group by tbl1.loc) as tbl2") + + // unsupported + // unsupported: in scatter query: complex aggregate expression (errno 1235) (sqlstate 42000) + helperTest(t, "select /*vt+ PLANNER=Gen4 */ (select count(*) from emp as tbl0) from emp as tbl0") + + // unsupported + // unsupported: using aggregation on top of a *planbuilder.filter plan + helperTest(t, "select /*vt+ PLANNER=Gen4 */ count(tbl1.dname) as caggr1 from dept as tbl0 left join dept as tbl1 on tbl1.dname > tbl1.loc where tbl1.loc <=> tbl1.dname group by tbl1.dname order by tbl1.dname asc") + + // unsupported + // unsupported: using aggregation on top of a *planbuilder.orderedAggregate plan + helperTest(t, "select /*vt+ PLANNER=Gen4 */ count(*) from (select count(*) from dept as tbl0) as tbl0") + + // unsupported + // unsupported: using aggregation on top of a *planbuilder.orderedAggregate plan + helperTest(t, "select /*vt+ PLANNER=Gen4 */ count(*), count(*) from (select count(*) from dept as tbl0) as tbl0, dept as tbl1") + + // unsupported + // unsupported: in scatter query: aggregation function 'avg(tbl0.deptno)' + helperTest(t, "select /*vt+ PLANNER=Gen4 */ avg(tbl0.deptno) from dept as tbl0") +} + +func TestRandom(t *testing.T) { + mcmp, closer := start(t) + defer closer() + + require.NoError(t, utils.WaitForAuthoritative(t, keyspaceName, "emp", clusterInstance.VtgateProcess.ReadVSchema)) + require.NoError(t, utils.WaitForAuthoritative(t, keyspaceName, "dept", clusterInstance.VtgateProcess.ReadVSchema)) + + // specify the schema (that is defined in schema.sql) + schemaTables := []tableT{ + {name: sqlparser.NewTableName("emp")}, + {name: sqlparser.NewTableName("dept")}, + } + schemaTables[0].addColumns([]column{ + {name: "empno", typ: "bigint"}, + {name: "ename", typ: "varchar"}, + {name: "job", typ: "varchar"}, + {name: "mgr", typ: "bigint"}, + {name: "hiredate", typ: "date"}, + {name: "sal", typ: "bigint"}, + {name: "comm", typ: "bigint"}, + {name: "deptno", typ: "bigint"}, + }...) + schemaTables[1].addColumns([]column{ + {name: "deptno", typ: "bigint"}, + {name: "dname", typ: "varchar"}, + {name: "loc", typ: "varchar"}, + }...) + + endBy := time.Now().Add(1 * time.Second) + + var queryCount int + for time.Now().Before(endBy) && (!t.Failed() || testFailingQueries) { + query := sqlparser.String(randomQuery(schemaTables, 3, 3)) + _, vtErr := mcmp.ExecAllowAndCompareError(query) + + // this assumes all queries are valid mysql queries + if vtErr != nil { + fmt.Println(query) + fmt.Println(vtErr) + + if stopOnMustFixError { + // EOF + if sqlError, ok := vtErr.(*mysql.SQLError); ok && strings.Contains(sqlError.Message, "EOF") { + break + } + // mismatched results + if strings.Contains(vtErr.Error(), "results mismatched") { + break + } + } + + // restart the mysql and vitess connections in case something bad happened + closer() + mcmp, closer = start(t) + } + queryCount++ + } + fmt.Printf("Queries successfully executed: %d\n", queryCount) +} + +func TestBuggyQueries(t *testing.T) { + mcmp, closer := start(t) + defer closer() + + require.NoError(t, utils.WaitForAuthoritative(t, keyspaceName, "emp", clusterInstance.VtgateProcess.ReadVSchema)) + require.NoError(t, utils.WaitForAuthoritative(t, keyspaceName, "dept", clusterInstance.VtgateProcess.ReadVSchema)) + + mcmp.AssertMatches("select /*vt+ PLANNER=Gen4 */ count(*), count(*), count(*) from dept as tbl0, emp as tbl1 where tbl0.deptno = tbl1.deptno group by tbl1.empno order by tbl1.empno", + `[[INT64(1) INT64(1) INT64(1)] [INT64(1) INT64(1) INT64(1)] [INT64(1) INT64(1) INT64(1)] [INT64(1) INT64(1) INT64(1)] [INT64(1) INT64(1) INT64(1)] [INT64(1) INT64(1) INT64(1)] [INT64(1) INT64(1) INT64(1)] [INT64(1) INT64(1) INT64(1)] [INT64(1) INT64(1) INT64(1)] [INT64(1) INT64(1) INT64(1)] [INT64(1) INT64(1) INT64(1)] [INT64(1) INT64(1) INT64(1)] [INT64(1) INT64(1) INT64(1)] [INT64(1) INT64(1) INT64(1)]]`) + //mcmp.AssertMatches("select /*vt+ PLANNER=Gen4 */ count(tbl0.deptno) from dept as tbl0, emp as tbl1 group by tbl1.job order by tbl1.job limit 3", + // `[[INT64(8)] [INT64(16)] [INT64(12)]]`) + mcmp.AssertMatches("select /*vt+ PLANNER=Gen4 */ count(*), count(*) from emp as tbl0 group by tbl0.empno order by tbl0.empno", + `[[INT64(1) INT64(1)] [INT64(1) INT64(1)] [INT64(1) INT64(1)] [INT64(1) INT64(1)] [INT64(1) INT64(1)] [INT64(1) INT64(1)] [INT64(1) INT64(1)] [INT64(1) INT64(1)] [INT64(1) INT64(1)] [INT64(1) INT64(1)] [INT64(1) INT64(1)] [INT64(1) INT64(1)] [INT64(1) INT64(1)] [INT64(1) INT64(1)]]`) + mcmp.AssertMatches("select /*vt+ PLANNER=Gen4 */ distinct count(*), tbl0.loc from dept as tbl0 group by tbl0.loc", + `[[INT64(1) VARCHAR("BOSTON")] [INT64(1) VARCHAR("CHICAGO")] [INT64(1) VARCHAR("DALLAS")] [INT64(1) VARCHAR("NEW YORK")]]`) + mcmp.AssertMatches("select /*vt+ PLANNER=Gen4 */ distinct count(*) from dept as tbl0 group by tbl0.loc", + `[[INT64(1)]]`) + mcmp.AssertMatches("select /*vt+ PLANNER=Gen4 */ sum(tbl1.comm) from emp as tbl0, emp as tbl1", + `[[DECIMAL(30800)]]`) + mcmp.AssertMatches("select /*vt+ PLANNER=Gen4 */ tbl1.mgr, tbl1.mgr, count(*) from emp as tbl1 group by tbl1.mgr", + `[[NULL NULL INT64(1)] [INT64(7566) INT64(7566) INT64(2)] [INT64(7698) INT64(7698) INT64(5)] [INT64(7782) INT64(7782) INT64(1)] [INT64(7788) INT64(7788) INT64(1)] [INT64(7839) INT64(7839) INT64(3)] [INT64(7902) INT64(7902) INT64(1)]]`) + mcmp.AssertMatches("select /*vt+ PLANNER=Gen4 */ tbl1.mgr, tbl1.mgr, count(*) from emp as tbl0, emp as tbl1 group by tbl1.mgr", + `[[NULL NULL INT64(14)] [INT64(7566) INT64(7566) INT64(28)] [INT64(7698) INT64(7698) INT64(70)] [INT64(7782) INT64(7782) INT64(14)] [INT64(7788) INT64(7788) INT64(14)] [INT64(7839) INT64(7839) INT64(42)] [INT64(7902) INT64(7902) INT64(14)]]`) + mcmp.AssertMatches("select /*vt+ PLANNER=Gen4 */ count(*), count(*), count(tbl0.comm) from emp as tbl0, emp as tbl1 join dept as tbl2", + `[[INT64(784) INT64(784) INT64(224)]]`) + mcmp.AssertMatches("select /*vt+ PLANNER=Gen4 */ count(*), count(*) from (select count(*) from dept as tbl0 group by tbl0.deptno) as tbl0, dept as tbl1", + `[[INT64(16) INT64(16)]]`) + mcmp.AssertMatches("select /*vt+ PLANNER=Gen4 */ count(*) from (select count(*) from dept as tbl0 group by tbl0.deptno) as tbl0", + `[[INT64(4)]]`) + mcmp.AssertMatches("select /*vt+ PLANNER=Gen4 */ min(tbl0.loc) from dept as tbl0", + `[[VARCHAR("BOSTON")]]`) + mcmp.AssertMatches("select /*vt+ PLANNER=Gen4 */ tbl1.empno, max(tbl1.job) from dept as tbl0, emp as tbl1 group by tbl1.empno", + `[[INT64(7369) VARCHAR("CLERK")] [INT64(7499) VARCHAR("SALESMAN")] [INT64(7521) VARCHAR("SALESMAN")] [INT64(7566) VARCHAR("MANAGER")] [INT64(7654) VARCHAR("SALESMAN")] [INT64(7698) VARCHAR("MANAGER")] [INT64(7782) VARCHAR("MANAGER")] [INT64(7788) VARCHAR("ANALYST")] [INT64(7839) VARCHAR("PRESIDENT")] [INT64(7844) VARCHAR("SALESMAN")] [INT64(7876) VARCHAR("CLERK")] [INT64(7900) VARCHAR("CLERK")] [INT64(7902) VARCHAR("ANALYST")] [INT64(7934) VARCHAR("CLERK")]]`) + mcmp.AssertMatches("select /*vt+ PLANNER=Gen4 */ tbl1.ename, max(tbl0.comm) from emp as tbl0, emp as tbl1 group by tbl1.ename", + `[[VARCHAR("ADAMS") INT64(1400)] [VARCHAR("ALLEN") INT64(1400)] [VARCHAR("BLAKE") INT64(1400)] [VARCHAR("CLARK") INT64(1400)] [VARCHAR("FORD") INT64(1400)] [VARCHAR("JAMES") INT64(1400)] [VARCHAR("JONES") INT64(1400)] [VARCHAR("KING") INT64(1400)] [VARCHAR("MARTIN") INT64(1400)] [VARCHAR("MILLER") INT64(1400)] [VARCHAR("SCOTT") INT64(1400)] [VARCHAR("SMITH") INT64(1400)] [VARCHAR("TURNER") INT64(1400)] [VARCHAR("WARD") INT64(1400)]]`) + mcmp.AssertMatches("select /*vt+ PLANNER=Gen4 */ tbl0.dname, tbl0.dname, min(tbl0.deptno) from dept as tbl0, dept as tbl1 group by tbl0.dname, tbl0.dname", + `[[VARCHAR("ACCOUNTING") VARCHAR("ACCOUNTING") INT64(10)] [VARCHAR("OPERATIONS") VARCHAR("OPERATIONS") INT64(40)] [VARCHAR("RESEARCH") VARCHAR("RESEARCH") INT64(20)] [VARCHAR("SALES") VARCHAR("SALES") INT64(30)]]`) + mcmp.AssertMatches("select /*vt+ PLANNER=Gen4 */ tbl0.dname, min(tbl1.deptno) from dept as tbl0, dept as tbl1 group by tbl0.dname, tbl1.dname", + `[[VARCHAR("ACCOUNTING") INT64(10)] [VARCHAR("ACCOUNTING") INT64(40)] [VARCHAR("ACCOUNTING") INT64(20)] [VARCHAR("ACCOUNTING") INT64(30)] [VARCHAR("OPERATIONS") INT64(10)] [VARCHAR("OPERATIONS") INT64(40)] [VARCHAR("OPERATIONS") INT64(20)] [VARCHAR("OPERATIONS") INT64(30)] [VARCHAR("RESEARCH") INT64(10)] [VARCHAR("RESEARCH") INT64(40)] [VARCHAR("RESEARCH") INT64(20)] [VARCHAR("RESEARCH") INT64(30)] [VARCHAR("SALES") INT64(10)] [VARCHAR("SALES") INT64(40)] [VARCHAR("SALES") INT64(20)] [VARCHAR("SALES") INT64(30)]]`) + mcmp.AssertMatches("select /*vt+ PLANNER=Gen4 */ max(tbl0.hiredate) from emp as tbl0", + `[[DATE("1983-01-12")]]`) + mcmp.AssertMatches("select /*vt+ PLANNER=Gen4 */ min(tbl0.deptno) as caggr0, count(*) as caggr1 from dept as tbl0 left join dept as tbl1 on tbl1.loc = tbl1.dname", + `[[INT64(10) INT64(4)]]`) + mcmp.AssertMatches("select /*vt+ PLANNER=Gen4 */ count(tbl1.loc) as caggr0 from dept as tbl1 left join dept as tbl2 on tbl1.loc = tbl2.loc where (tbl2.deptno)", + `[[INT64(4)]]`) + mcmp.AssertMatches("select /*vt+ PLANNER=Gen4 */ sum(tbl1.ename), min(tbl0.empno) from emp as tbl0, emp as tbl1 left join dept as tbl2 on tbl1.job = tbl2.loc and tbl1.comm = tbl2.deptno where ('trout') and tbl0.deptno = tbl1.comm", + `[[NULL NULL]]`) + mcmp.AssertMatches("select /*vt+ PLANNER=Gen4 */ distinct max(tbl0.deptno), count(tbl0.job) from emp as tbl0, dept as tbl1 left join dept as tbl2 on tbl1.dname = tbl2.loc and tbl1.dname = tbl2.loc where (tbl2.loc) and tbl0.deptno = tbl1.deptno", + `[[NULL INT64(0)]]`) + +} diff --git a/go/test/endtoend/vtgate/queries/random/schema.sql b/go/test/endtoend/vtgate/queries/random/schema.sql new file mode 100644 index 00000000000..7ef4721a381 --- /dev/null +++ b/go/test/endtoend/vtgate/queries/random/schema.sql @@ -0,0 +1,20 @@ +CREATE TABLE emp ( + EMPNO bigint NOT NULL, + ENAME VARCHAR(10), + JOB VARCHAR(9), + MGR bigint, + HIREDATE DATE, + SAL bigint, + COMM bigint, + DEPTNO bigint, + PRIMARY KEY (EMPNO) +) Engine = InnoDB + COLLATE = utf8mb4_general_ci; + +CREATE TABLE dept ( + DEPTNO bigint, + DNAME VARCHAR(14), + LOC VARCHAR(13), + PRIMARY KEY (DEPTNO) +) Engine = InnoDB + COLLATE = utf8mb4_general_ci; \ No newline at end of file diff --git a/go/test/endtoend/vtgate/queries/random/vschema.json b/go/test/endtoend/vtgate/queries/random/vschema.json new file mode 100644 index 00000000000..21e31d5618c --- /dev/null +++ b/go/test/endtoend/vtgate/queries/random/vschema.json @@ -0,0 +1,26 @@ +{ + "sharded": true, + "vindexes": { + "hash": { + "type": "hash" + } + }, + "tables": { + "emp": { + "column_vindexes": [ + { + "column": "deptno", + "name": "hash" + } + ] + }, + "dept": { + "column_vindexes": [ + { + "column": "deptno", + "name": "hash" + } + ] + } + } +} \ No newline at end of file diff --git a/go/vt/sqlparser/ast_funcs.go b/go/vt/sqlparser/ast_funcs.go index 29d84412d7f..6540105383b 100644 --- a/go/vt/sqlparser/ast_funcs.go +++ b/go/vt/sqlparser/ast_funcs.go @@ -685,6 +685,106 @@ func NewColNameWithQualifier(identifier string, table TableName) *ColName { } } +// NewTableName makes a new TableName +func NewTableName(name string) TableName { + return TableName{ + Name: NewIdentifierCS(name), + } +} + +// NewTableNameWithQualifier makes a new TableName with a qualifier +func NewTableNameWithQualifier(name, qualifier string) TableName { + return TableName{ + Name: NewIdentifierCS(name), + Qualifier: NewIdentifierCS(qualifier), + } +} + +// NewAliasedTableExpr makes a new AliasedTableExpr with an alias +func NewAliasedTableExpr(simpleTableExpr SimpleTableExpr, alias string) *AliasedTableExpr { + return &AliasedTableExpr{ + Expr: simpleTableExpr, + As: NewIdentifierCS(alias), + } +} + +// NewJoinTableExpr makes a new JoinTableExpr +func NewJoinTableExpr(leftExpr TableExpr, join JoinType, rightExpr TableExpr, condition *JoinCondition) *JoinTableExpr { + return &JoinTableExpr{ + LeftExpr: leftExpr, + Join: join, + RightExpr: rightExpr, + Condition: condition, + } +} + +// NewJoinCondition makes a new JoinCondition +func NewJoinCondition(on Expr, using Columns) *JoinCondition { + return &JoinCondition{ + On: on, + Using: using, + } +} + +// NewAliasedExpr makes a new AliasedExpr +func NewAliasedExpr(expr Expr, alias string) *AliasedExpr { + return &AliasedExpr{ + Expr: expr, + As: NewIdentifierCI(alias), + } +} + +// NewOrder makes a new Order +func NewOrder(expr Expr, direction OrderDirection) *Order { + return &Order{ + Expr: expr, + Direction: direction, + } +} + +// NewComparisonExpr makes a new ComparisonExpr +func NewComparisonExpr(operator ComparisonExprOperator, left, right, escape Expr) *ComparisonExpr { + return &ComparisonExpr{ + Operator: operator, + Left: left, + Right: right, + Escape: escape, + } +} + +// NewLimit makes a new Limit +func NewLimit(offset, rowCount int) *Limit { + return &Limit{ + Offset: &Literal{ + Type: IntVal, + Val: fmt.Sprint(offset), + }, + Rowcount: &Literal{ + Type: IntVal, + Val: fmt.Sprint(rowCount), + }, + } +} + +// NewLimitWithoutOffset makes a new Limit without an offset +func NewLimitWithoutOffset(rowCount int) *Limit { + return &Limit{ + Offset: nil, + Rowcount: &Literal{ + Type: IntVal, + Val: fmt.Sprint(rowCount), + }, + } +} + +// NewDerivedTable makes a new DerivedTable +func NewDerivedTable(lateral bool, selectStatement SelectStatement) *DerivedTable { + return &DerivedTable{ + Lateral: lateral, + Select: selectStatement, + } +} + // NewSelect is used to create a select statement func NewSelect(comments Comments, exprs SelectExprs, selectOptions []string, into *SelectInto, from TableExprs, where *Where, groupBy GroupBy, having *Where, windows NamedWindows) *Select { var cache *bool @@ -952,6 +1052,10 @@ func compliantName(in string) string { return buf.String() } +func (node *Select) AddSelectExprs(selectExprs SelectExprs) { + node.SelectExprs = append(node.SelectExprs, selectExprs...) +} + // AddOrder adds an order by element func (node *Select) AddOrder(order *Order) { node.OrderBy = append(node.OrderBy, order) diff --git a/go/vt/sqlparser/precedence_test.go b/go/vt/sqlparser/precedence_test.go index 215c9480823..cb8c1f23805 100644 --- a/go/vt/sqlparser/precedence_test.go +++ b/go/vt/sqlparser/precedence_test.go @@ -216,7 +216,7 @@ func TestRandom(t *testing.T) { // The idea is to generate random queries, and pass them through the parser and then the unparser, and one more time. The result of the first unparse should be the same as the second result. seed := time.Now().UnixNano() fmt.Println(fmt.Sprintf("seed is %d", seed)) // nolint - g := newGenerator(seed, 5) + g := NewGenerator(seed, 5) endBy := time.Now().Add(1 * time.Second) for { @@ -224,7 +224,7 @@ func TestRandom(t *testing.T) { break } // Given a random expression - randomExpr := g.expression() + randomExpr := g.Expression() inputQ := "select " + String(randomExpr) + " from t" // When it's parsed and unparsed diff --git a/go/vt/sqlparser/random_expr.go b/go/vt/sqlparser/random_expr.go index e2725f37a37..9b3b711c87f 100644 --- a/go/vt/sqlparser/random_expr.go +++ b/go/vt/sqlparser/random_expr.go @@ -23,34 +23,43 @@ import ( // This file is used to generate random expressions to be used for testing -func newGenerator(seed int64, maxDepth int) *generator { - g := generator{ - seed: seed, - r: rand.New(rand.NewSource(seed)), - maxDepth: maxDepth, +type ( + ExprGenerator interface { + IntExpr() Expr + StringExpr() Expr + } +) + +func NewGenerator(seed int64, maxDepth int, exprGenerators ...ExprGenerator) *Generator { + g := Generator{ + seed: seed, + r: rand.New(rand.NewSource(seed)), + maxDepth: maxDepth, + exprGenerator: exprGenerators, } return &g } -type generator struct { - seed int64 - r *rand.Rand - depth int - maxDepth int +type Generator struct { + seed int64 + r *rand.Rand + depth int + maxDepth int + exprGenerator []ExprGenerator } // enter should be called whenever we are producing an intermediate node. it should be followed by a `defer g.exit()` -func (g *generator) enter() { +func (g *Generator) enter() { g.depth++ } // exit should be called when exiting an intermediate node -func (g *generator) exit() { +func (g *Generator) exit() { g.depth-- } // atMaxDepth returns true if we have reached the maximum allowed depth or the expression tree -func (g *generator) atMaxDepth() bool { +func (g *Generator) atMaxDepth() bool { return g.depth >= g.maxDepth } @@ -58,19 +67,22 @@ func (g *generator) atMaxDepth() bool { Creates a random expression. It builds an expression tree using the following constructs: - true/false - AND/OR/NOT - - string literalrs, numeric literals (-/+ 1000) + - string literals, numeric literals (-/+ 1000) + - columns of types bigint and varchar - =, >, <, >=, <=, <=>, != - &, |, ^, +, -, *, /, div, %, <<, >> - IN, BETWEEN and CASE - IS NULL, IS NOT NULL, IS TRUE, IS NOT TRUE, IS FALSE, IS NOT FALSE + Returns the random expression (Expr) and its type (string) Note: It's important to update this method so that it produces all expressions that need precedence checking. It's currently missing function calls and string operators */ -func (g *generator) expression() Expr { +func (g *Generator) Expression() Expr { if g.randomBool() { return g.booleanExpr() } + options := []exprF{ func() Expr { return g.intExpr() }, func() Expr { return g.stringExpr() }, @@ -80,7 +92,7 @@ func (g *generator) expression() Expr { return g.randomOf(options) } -func (g *generator) booleanExpr() Expr { +func (g *Generator) booleanExpr() Expr { if g.atMaxDepth() { return g.booleanLiteral() } @@ -102,7 +114,7 @@ func (g *generator) booleanExpr() Expr { return g.randomOf(options) } -func (g *generator) intExpr() Expr { +func (g *Generator) intExpr() Expr { if g.atMaxDepth() { return g.intLiteral() } @@ -113,30 +125,40 @@ func (g *generator) intExpr() Expr { func() Expr { return g.caseExpr(g.intExpr) }, } + for _, generator := range g.exprGenerator { + options = append(options, func() Expr { + expr := generator.IntExpr() + if expr == nil { + return g.intLiteral() + } + return expr + }) + } + return g.randomOf(options) } -func (g *generator) booleanLiteral() Expr { +func (g *Generator) booleanLiteral() Expr { return BoolVal(g.randomBool()) } -func (g *generator) randomBool() bool { +func (g *Generator) randomBool() bool { return g.r.Float32() < 0.5 } -func (g *generator) intLiteral() Expr { - t := fmt.Sprintf("%d", g.r.Intn(1000)-g.r.Intn((1000))) +func (g *Generator) intLiteral() Expr { + t := fmt.Sprintf("%d", g.r.Intn(1000)-g.r.Intn(1000)) return NewIntLiteral(t) } var words = []string{"ox", "ant", "ape", "asp", "bat", "bee", "boa", "bug", "cat", "cod", "cow", "cub", "doe", "dog", "eel", "eft", "elf", "elk", "emu", "ewe", "fly", "fox", "gar", "gnu", "hen", "hog", "imp", "jay", "kid", "kit", "koi", "lab", "man", "owl", "pig", "pug", "pup", "ram", "rat", "ray", "yak", "bass", "bear", "bird", "boar", "buck", "bull", "calf", "chow", "clam", "colt", "crab", "crow", "dane", "deer", "dodo", "dory", "dove", "drum", "duck", "fawn", "fish", "flea", "foal", "fowl", "frog", "gnat", "goat", "grub", "gull", "hare", "hawk", "ibex", "joey", "kite", "kiwi", "lamb", "lark", "lion", "loon", "lynx", "mako", "mink", "mite", "mole", "moth", "mule", "mutt", "newt", "orca", "oryx", "pika", "pony", "puma", "seal", "shad", "slug", "sole", "stag", "stud", "swan", "tahr", "teal", "tick", "toad", "tuna", "wasp", "wolf", "worm", "wren", "yeti", "adder", "akita", "alien", "aphid", "bison", "boxer", "bream", "bunny", "burro", "camel", "chimp", "civet", "cobra", "coral", "corgi", "crane", "dingo", "drake", "eagle", "egret", "filly", "finch", "gator", "gecko", "ghost", "ghoul", "goose", "guppy", "heron", "hippo", "horse", "hound", "husky", "hyena", "koala", "krill", "leech", "lemur", "liger", "llama", "louse", "macaw", "midge", "molly", "moose", "moray", "mouse", "panda", "perch", "prawn", "quail", "racer", "raven", "rhino", "robin", "satyr", "shark", "sheep", "shrew", "skink", "skunk", "sloth", "snail", "snake", "snipe", "squid", "stork", "swift", "swine", "tapir", "tetra", "tiger", "troll", "trout", "viper", "wahoo", "whale", "zebra", "alpaca", "amoeba", "baboon", "badger", "beagle", "bedbug", "beetle", "bengal", "bobcat", "caiman", "cattle", "cicada", "collie", "condor", "cougar", "coyote", "dassie", "donkey", "dragon", "earwig", "falcon", "feline", "ferret", "gannet", "gibbon", "glider", "goblin", "gopher", "grouse", "guinea", "hermit", "hornet", "iguana", "impala", "insect", "jackal", "jaguar", "jennet", "kitten", "kodiak", "lizard", "locust", "maggot", "magpie", "mammal", "mantis", "marlin", "marmot", "marten", "martin", "mayfly", "minnow", "monkey", "mullet", "muskox", "ocelot", "oriole", "osprey", "oyster", "parrot", "pigeon", "piglet", "poodle", "possum", "python", "quagga", "rabbit", "raptor", "rodent", "roughy", "salmon", "sawfly", "serval", "shiner", "shrimp", "spider", "sponge", "tarpon", "thrush", "tomcat", "toucan", "turkey", "turtle", "urchin", "vervet", "walrus", "weasel", "weevil", "wombat", "anchovy", "anemone", "bluejay", "buffalo", "bulldog", "buzzard", "caribou", "catfish", "chamois", "cheetah", "chicken", "chigger", "cowbird", "crappie", "crawdad", "cricket", "dogfish", "dolphin", "firefly", "garfish", "gazelle", "gelding", "giraffe", "gobbler", "gorilla", "goshawk", "grackle", "griffon", "grizzly", "grouper", "haddock", "hagfish", "halibut", "hamster", "herring", "jackass", "javelin", "jawfish", "jaybird", "katydid", "ladybug", "lamprey", "lemming", "leopard", "lioness", "lobster", "macaque", "mallard", "mammoth", "manatee", "mastiff", "meerkat", "mollusk", "monarch", "mongrel", "monitor", "monster", "mudfish", "muskrat", "mustang", "narwhal", "oarfish", "octopus", "opossum", "ostrich", "panther", "peacock", "pegasus", "pelican", "penguin", "phoenix", "piranha", "polecat", "primate", "quetzal", "raccoon", "rattler", "redbird", "redfish", "reptile", "rooster", "sawfish", "sculpin", "seagull", "skylark", "snapper", "spaniel", "sparrow", "sunbeam", "sunbird", "sunfish", "tadpole", "termite", "terrier", "unicorn", "vulture", "wallaby", "walleye", "warthog", "whippet", "wildcat", "aardvark", "airedale", "albacore", "anteater", "antelope", "arachnid", "barnacle", "basilisk", "blowfish", "bluebird", "bluegill", "bonefish", "bullfrog", "cardinal", "chipmunk", "cockatoo", "crayfish", "dinosaur", "doberman", "duckling", "elephant", "escargot", "flamingo", "flounder", "foxhound", "glowworm", "goldfish", "grubworm", "hedgehog", "honeybee", "hookworm", "humpback", "kangaroo", "killdeer", "kingfish", "labrador", "lacewing", "ladybird", "lionfish", "longhorn", "mackerel", "malamute", "marmoset", "mastodon", "moccasin", "mongoose", "monkfish", "mosquito", "pangolin", "parakeet", "pheasant", "pipefish", "platypus", "polliwog", "porpoise", "reindeer", "ringtail", "sailfish", "scorpion", "seahorse", "seasnail", "sheepdog", "shepherd", "silkworm", "squirrel", "stallion", "starfish", "starling", "stingray", "stinkbug", "sturgeon", "terrapin", "titmouse", "tortoise", "treefrog", "werewolf", "woodcock"} -func (g *generator) stringLiteral() Expr { +func (g *Generator) stringLiteral() Expr { return NewStrLiteral(g.randomOfS(words)) } -func (g *generator) stringExpr() Expr { +func (g *Generator) stringExpr() Expr { if g.atMaxDepth() { return g.stringLiteral() } @@ -146,10 +168,20 @@ func (g *generator) stringExpr() Expr { func() Expr { return g.caseExpr(g.stringExpr) }, } + for _, generator := range g.exprGenerator { + options = append(options, func() Expr { + expr := generator.StringExpr() + if expr == nil { + return g.stringLiteral() + } + return expr + }) + } + return g.randomOf(options) } -func (g *generator) likeExpr() Expr { +func (g *Generator) likeExpr() Expr { g.enter() defer g.exit() return &ComparisonExpr{ @@ -161,7 +193,7 @@ func (g *generator) likeExpr() Expr { var comparisonOps = []ComparisonExprOperator{EqualOp, LessThanOp, GreaterThanOp, LessEqualOp, GreaterEqualOp, NotEqualOp, NullSafeEqualOp} -func (g *generator) comparison(f func() Expr) Expr { +func (g *Generator) comparison(f func() Expr) Expr { g.enter() defer g.exit() @@ -173,7 +205,7 @@ func (g *generator) comparison(f func() Expr) Expr { return cmp } -func (g *generator) caseExpr(valueF func() Expr) Expr { +func (g *Generator) caseExpr(valueF func() Expr) Expr { g.enter() defer g.exit() @@ -193,12 +225,13 @@ func (g *generator) caseExpr(valueF func() Expr) Expr { if exp == nil { cond = g.booleanExpr() } else { - cond = g.expression() + cond = g.Expression() } + val := g.Expression() whens = append(whens, &When{ Cond: cond, - Val: g.expression(), + Val: val, }) } @@ -211,7 +244,7 @@ func (g *generator) caseExpr(valueF func() Expr) Expr { var arithmeticOps = []BinaryExprOperator{BitAndOp, BitOrOp, BitXorOp, PlusOp, MinusOp, MultOp, DivOp, IntDivOp, ModOp, ShiftRightOp, ShiftLeftOp} -func (g *generator) arithmetic() Expr { +func (g *Generator) arithmetic() Expr { g.enter() defer g.exit() @@ -226,15 +259,15 @@ func (g *generator) arithmetic() Expr { type exprF func() Expr -func (g *generator) randomOf(options []exprF) Expr { +func (g *Generator) randomOf(options []exprF) Expr { return options[g.r.Intn(len(options))]() } -func (g *generator) randomOfS(options []string) string { +func (g *Generator) randomOfS(options []string) string { return options[g.r.Intn(len(options))] } -func (g *generator) andExpr() Expr { +func (g *Generator) andExpr() Expr { g.enter() defer g.exit() return &AndExpr{ @@ -243,7 +276,7 @@ func (g *generator) andExpr() Expr { } } -func (g *generator) orExpr() Expr { +func (g *Generator) orExpr() Expr { g.enter() defer g.exit() return &OrExpr{ @@ -252,7 +285,7 @@ func (g *generator) orExpr() Expr { } } -func (g *generator) xorExpr() Expr { +func (g *Generator) xorExpr() Expr { g.enter() defer g.exit() return &XorExpr{ @@ -261,13 +294,13 @@ func (g *generator) xorExpr() Expr { } } -func (g *generator) notExpr() Expr { +func (g *Generator) notExpr() Expr { g.enter() defer g.exit() return &NotExpr{g.booleanExpr()} } -func (g *generator) inExpr() Expr { +func (g *Generator) inExpr() Expr { g.enter() defer g.exit() @@ -289,7 +322,7 @@ func (g *generator) inExpr() Expr { } } -func (g *generator) between() Expr { +func (g *Generator) between() Expr { g.enter() defer g.exit() @@ -308,7 +341,7 @@ func (g *generator) between() Expr { } } -func (g *generator) isExpr() Expr { +func (g *Generator) isExpr() Expr { g.enter() defer g.exit() diff --git a/go/vt/sqlparser/rewriter_test.go b/go/vt/sqlparser/rewriter_test.go index dadd2c501df..9adae1b4a81 100644 --- a/go/vt/sqlparser/rewriter_test.go +++ b/go/vt/sqlparser/rewriter_test.go @@ -25,8 +25,8 @@ import ( ) func BenchmarkVisitLargeExpression(b *testing.B) { - gen := newGenerator(1, 5) - exp := gen.expression() + gen := NewGenerator(1, 5) + exp := gen.Expression() depth := 0 for i := 0; i < b.N; i++ { diff --git a/go/vt/sqlparser/walker_test.go b/go/vt/sqlparser/walker_test.go index f8bf2b4792a..5359235afa5 100644 --- a/go/vt/sqlparser/walker_test.go +++ b/go/vt/sqlparser/walker_test.go @@ -26,7 +26,7 @@ import ( func BenchmarkWalkLargeExpression(b *testing.B) { for i := 0; i < 10; i++ { b.Run(fmt.Sprintf("%d", i), func(b *testing.B) { - exp := newGenerator(int64(i*100), 5).expression() + exp := NewGenerator(int64(i*100), 5).Expression() count := 0 for i := 0; i < b.N; i++ { err := Walk(func(node SQLNode) (kontinue bool, err error) { @@ -42,7 +42,7 @@ func BenchmarkWalkLargeExpression(b *testing.B) { func BenchmarkRewriteLargeExpression(b *testing.B) { for i := 1; i < 7; i++ { b.Run(fmt.Sprintf("%d", i), func(b *testing.B) { - exp := newGenerator(int64(i*100), i).expression() + exp := NewGenerator(int64(i*100), i).Expression() count := 0 for i := 0; i < b.N; i++ { _ = Rewrite(exp, func(_ *Cursor) bool { diff --git a/test/config.json b/test/config.json index 0070720f1a0..222f38de078 100644 --- a/test/config.json +++ b/test/config.json @@ -621,6 +621,15 @@ "RetryMax": 1, "Tags": [] }, + "vtgate_queries_random": { + "File": "unused.go", + "Args": ["vitess.io/vitess/go/test/endtoend/vtgate/queries/random"], + "Command": [], + "Manual": false, + "Shard": "vtgate_queries", + "RetryMax": 1, + "Tags": [] + }, "vtgate_concurrentdml": { "File": "unused.go", "Args": ["vitess.io/vitess/go/test/endtoend/vtgate/concurrentdml"],