Skip to content

Commit

Permalink
Merge #77956
Browse files Browse the repository at this point in the history
77956: sql,roachtest: fix sqlsmith setup to return multiple stmts r=yuzefovich,mgartner a=rafiss

fixes #77902
fixes #77887

Rather than doing the setup in one large batch, it now returns multiple
strings. This is because batch statements are now treated as an implicit
txn, which is not always desired.

Release note: None

Co-authored-by: Rafi Shamim <[email protected]>
  • Loading branch information
craig[bot] and rafiss committed Mar 17, 2022
2 parents e9123b6 + dbb0200 commit e882432
Show file tree
Hide file tree
Showing 9 changed files with 85 additions and 63 deletions.
6 changes: 4 additions & 2 deletions pkg/ccl/backupccl/backup_rand_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -51,8 +51,10 @@ func TestBackupRestoreRandomDataRoundtrips(t *testing.T) {
sqlDB.Exec(t, "CREATE DATABASE rand")

setup := sqlsmith.Setups[sqlsmith.RandTableSetupName](rng)
if _, err := tc.Conns[0].Exec(setup); err != nil {
t.Fatal(err)
for _, stmt := range setup {
if _, err := tc.Conns[0].Exec(stmt); err != nil {
t.Fatal(err)
}
}

tables := sqlDB.Query(t, `SELECT name FROM crdb_internal.tables WHERE
Expand Down
29 changes: 18 additions & 11 deletions pkg/cmd/roachtest/tests/sqlsmith.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,12 +37,12 @@ func registerSQLSmith(r registry.Registry) {
"empty": sqlsmith.Setups["empty"],
"seed": sqlsmith.Setups["seed"],
sqlsmith.RandTableSetupName: sqlsmith.Setups[sqlsmith.RandTableSetupName],
"tpch-sf1": func(r *rand.Rand) string {
return `RESTORE TABLE tpch.* FROM 'gs://cockroach-fixtures/workload/tpch/scalefactor=1/backup?AUTH=implicit' WITH into_db = 'defaultdb';`
"tpch-sf1": func(r *rand.Rand) []string {
return []string{`RESTORE TABLE tpch.* FROM 'gs://cockroach-fixtures/workload/tpch/scalefactor=1/backup?AUTH=implicit' WITH into_db = 'defaultdb';`}
},
"tpcc": func(r *rand.Rand) string {
"tpcc": func(r *rand.Rand) []string {
const version = "version=2.1.0,fks=true,interleaved=false,seed=1,warehouses=1"
var sb strings.Builder
var stmts []string
for _, t := range []string{
"customer",
"district",
Expand All @@ -54,9 +54,14 @@ func registerSQLSmith(r registry.Registry) {
"stock",
"warehouse",
} {
fmt.Fprintf(&sb, "RESTORE TABLE tpcc.%s FROM 'gs://cockroach-fixtures/workload/tpcc/%[2]s/%[1]s?AUTH=implicit' WITH into_db = 'defaultdb';\n", t, version)
stmts = append(
stmts,
fmt.Sprintf("RESTORE TABLE tpcc.%s FROM 'gs://cockroach-fixtures/workload/tpcc/%[2]s/%[1]s?AUTH=implicit' WITH into_db = 'defaultdb';",
t, version,
),
)
}
return sb.String()
return stmts
},
}
settings := map[string]sqlsmith.SettingFunc{
Expand Down Expand Up @@ -113,11 +118,13 @@ func registerSQLSmith(r registry.Registry) {
}
conn := allConns[0]
t.Status("executing setup")
t.L().Printf("setup:\n%s", setup)
if _, err := conn.Exec(setup); err != nil {
t.Fatal(err)
} else {
logStmt(setup)
t.L().Printf("setup:\n%s", strings.Join(setup, "\n"))
for _, stmt := range setup {
if _, err := conn.Exec(stmt); err != nil {
t.Fatal(err)
} else {
logStmt(stmt)
}
}

if settingName == "multi-region" {
Expand Down
12 changes: 7 additions & 5 deletions pkg/cmd/roachtest/tests/tlp.go
Original file line number Diff line number Diff line change
Expand Up @@ -117,11 +117,13 @@ func runOneTLP(
setup := sqlsmith.Setups[sqlsmith.RandTableSetupName](rnd)

t.Status("executing setup")
t.L().Printf("setup:\n%s", setup)
if _, err := conn.Exec(setup); err != nil {
t.Fatal(err)
} else {
logStmt(setup)
t.L().Printf("setup:\n%s", strings.Join(setup, "\n"))
for _, stmt := range setup {
if _, err := conn.Exec(stmt); err != nil {
t.Fatal(err)
} else {
logStmt(stmt)
}
}

setStmtTimeout := fmt.Sprintf("SET statement_timeout='%s';", statementTimeout.String())
Expand Down
5 changes: 4 additions & 1 deletion pkg/cmd/roachtest/tests/tpc_utils.go
Original file line number Diff line number Diff line change
Expand Up @@ -77,7 +77,10 @@ func loadTPCHDataset(

t.L().Printf("restoring tpch scale factor %d\n", sf)
tpchURL := fmt.Sprintf("gs://cockroach-fixtures/workload/tpch/scalefactor=%d/backup?AUTH=implicit", sf)
query := fmt.Sprintf(`CREATE DATABASE IF NOT EXISTS tpch; RESTORE tpch.* FROM '%s' WITH into_db = 'tpch';`, tpchURL)
if _, err := db.ExecContext(ctx, `CREATE DATABASE IF NOT EXISTS tpch;`); err != nil {
return err
}
query := fmt.Sprintf(`RESTORE tpch.* FROM '%s' WITH into_db = 'tpch';`, tpchURL)
_, err := db.ExecContext(ctx, query)
return err
}
Expand Down
10 changes: 6 additions & 4 deletions pkg/cmd/smithtest/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -204,8 +204,10 @@ func (s WorkerSetup) run(ctx context.Context, rnd *rand.Rand) error {
fmt.Println("worker started")

initSQL := sqlsmith.Setups[sqlsmith.RandSetup(rnd)](rnd)
if _, err := pgdb.Exec(ctx, initSQL); err != nil {
return errors.Wrap(err, "init")
for _, stmt := range initSQL {
if _, err := pgdb.Exec(ctx, stmt); err != nil {
return errors.Wrap(err, "init")
}
}

setting := sqlsmith.Settings[sqlsmith.RandSetting(rnd)](rnd)
Expand Down Expand Up @@ -288,7 +290,7 @@ func (s WorkerSetup) run(ctx context.Context, rnd *rand.Rand) error {
// failure de-duplicates, reduces, and files errors. It generally returns nil
// indicating that this was successfully filed and we should continue looking
// for errors.
func (s WorkerSetup) failure(ctx context.Context, initSQL, stmt string, err error) error {
func (s WorkerSetup) failure(ctx context.Context, initSQL []string, stmt string, err error) error {
var message, stack string
var pqerr pgconn.PgError
if errors.As(err, &pqerr) {
Expand All @@ -315,7 +317,7 @@ func (s WorkerSetup) failure(ctx context.Context, initSQL, stmt string, err erro
return nil
}
fmt.Println("found", message)
input := fmt.Sprintf("%s\n\n%s;", initSQL, stmt)
input := fmt.Sprintf("%s\n\n%s;", strings.Join(initSQL, "\n"), stmt)
fmt.Printf("SQL:\n%s\n\n", input)

// Run reducer.
Expand Down
14 changes: 9 additions & 5 deletions pkg/compose/compare/compare/compare_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -126,7 +126,9 @@ func TestCompare(t *testing.T) {
t.Logf("starting test: %s", confName)
rng, _ := randutil.NewTestRand()
setup := config.setup(rng)
setup, _ = randgen.ApplyString(rng, setup, config.setupMutators...)
for i := range setup {
setup[i], _ = randgen.ApplyString(rng, setup[i], config.setupMutators...)
}

conns := map[string]cmpconn.Conn{}
for _, testCn := range config.conns {
Expand All @@ -149,10 +151,12 @@ func TestCompare(t *testing.T) {
t.Fatalf("%s: %v", testCn.name, err)
}
}
connSetup, _ := randgen.ApplyString(rng, setup, testCn.mutators...)
if err := conn.Exec(ctx, connSetup); err != nil {
t.Log(connSetup)
t.Fatalf("%s: %v", testCn.name, err)
for i := range setup {
stmt, _ := randgen.ApplyString(rng, setup[i], testCn.mutators...)
if err := conn.Exec(ctx, stmt); err != nil {
t.Log(stmt)
t.Fatalf("%s: %v", testCn.name, err)
}
}
conns[testCn.name] = conn
}
Expand Down
38 changes: 15 additions & 23 deletions pkg/internal/sqlsmith/setup.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,15 +14,14 @@ import (
"fmt"
"math/rand"
"sort"
"strings"

"github.com/cockroachdb/cockroach/pkg/sql/randgen"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
)

// Setup generates a SQL query that can be executed to initialize a database
// for smithing.
type Setup func(*rand.Rand) string
type Setup func(*rand.Rand) []string

// RandTableSetupName is the name of the table setup that creates random tables.
const RandTableSetupName = "rand-tables"
Expand All @@ -40,11 +39,8 @@ var Setups = map[string]Setup{
// wrapCommonSetup wraps setup steps common to all SQLSmith setups around the
// specific setup passed in.
func wrapCommonSetup(setupFn Setup) Setup {
return func(r *rand.Rand) string {
s := setupFn(r)
var sb strings.Builder
sb.WriteString(s)
return sb.String()
return func(r *rand.Rand) []string {
return setupFn(r)
}
}

Expand All @@ -64,46 +60,42 @@ func RandSetup(r *rand.Rand) string {
}

func stringSetup(s string) Setup {
return func(*rand.Rand) string {
return s
return func(*rand.Rand) []string {
return []string{s}
}
}

// randTables is a Setup function that creates 1-5 random tables.
func randTables(r *rand.Rand) string {
func randTables(r *rand.Rand) []string {
return randTablesN(r, r.Intn(5)+1)
}

// randTablesN is a Setup function that creates n random tables.
func randTablesN(r *rand.Rand, n int) string {
var sb strings.Builder
func randTablesN(r *rand.Rand, n int) []string {
var stmts []string
// Since we use the stats mutator, disable auto stats generation.
sb.WriteString(`
SET CLUSTER SETTING sql.stats.automatic_collection.enabled = false;
SET CLUSTER SETTING sql.stats.histogram_collection.enabled = false;
`)
stmts = append(stmts, `SET CLUSTER SETTING sql.stats.automatic_collection.enabled = false;`)
stmts = append(stmts, `SET CLUSTER SETTING sql.stats.histogram_collection.enabled = false;`)

// Create the random tables.
stmts := randgen.RandCreateTables(r, "table", n,
createTableStatements := randgen.RandCreateTables(r, "table", n,
randgen.StatisticsMutator,
randgen.PartialIndexMutator,
randgen.ForeignKeyMutator,
)

for _, stmt := range stmts {
sb.WriteString(tree.SerializeForDisplay(stmt))
sb.WriteString(";\n")
for _, stmt := range createTableStatements {
stmts = append(stmts, tree.SerializeForDisplay(stmt))
}

// Create some random types as well.
numTypes := r.Intn(5) + 1
for i := 0; i < numTypes; i++ {
name := fmt.Sprintf("rand_typ_%d", i)
stmt := randgen.RandCreateType(r, name, letters)
sb.WriteString(stmt.String())
sb.WriteString(";\n")
stmts = append(stmts, stmt.String())
}
return sb.String()
return stmts
}

const (
Expand Down
23 changes: 15 additions & 8 deletions pkg/internal/sqlsmith/sqlsmith_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ import (
"context"
"flag"
"fmt"
"strings"
"testing"

"github.com/cockroachdb/cockroach/pkg/base"
Expand Down Expand Up @@ -45,9 +46,11 @@ func TestSetups(t *testing.T) {
rnd, _ := randutil.NewTestRand()

sql := setup(rnd)
if _, err := sqlDB.Exec(sql); err != nil {
t.Log(sql)
t.Fatal(err)
for _, stmt := range sql {
if _, err := sqlDB.Exec(stmt); err != nil {
t.Log(stmt)
t.Fatal(err)
}
}
})
}
Expand Down Expand Up @@ -85,9 +88,11 @@ func TestRandTableInserts(t *testing.T) {
rnd, _ := randutil.NewTestRand()

setup := randTablesN(rnd, 10)
if _, err := sqlDB.Exec(setup); err != nil {
t.Log(setup)
t.Fatal(err)
for _, stmt := range setup {
if _, err := sqlDB.Exec(stmt); err != nil {
t.Log(stmt)
t.Fatal(err)
}
}

insertOnly := simpleOption("insert only", func(s *Smither) {
Expand Down Expand Up @@ -168,8 +173,10 @@ func TestGenerateParse(t *testing.T) {
settings := setting(rnd)
t.Log("setting:", settingName, settings.Options)
setupSQL := setup(rnd)
t.Log(setupSQL)
db.Exec(t, setupSQL)
t.Log(strings.Join(setupSQL, "\n"))
for _, stmt := range setupSQL {
db.Exec(t, stmt)
}

smither, err := NewSmither(sqlDB, rnd, settings.Options...)
if err != nil {
Expand Down
11 changes: 7 additions & 4 deletions pkg/sql/tests/rsg_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -560,11 +560,14 @@ func TestRandomSyntaxSQLSmith(t *testing.T) {
setups := []string{sqlsmith.RandTableSetupName, "seed"}
for _, s := range setups {
randTables := sqlsmith.Setups[s](r.Rnd)
if err := db.exec(t, ctx, randTables); err != nil {
return err
for _, stmt := range randTables {
if err := db.exec(t, ctx, stmt); err != nil {
return err
}
tableStmts = append(tableStmts, stmt)
t.Logf("%s;", stmt)
}
tableStmts = append(tableStmts, randTables)
t.Logf("%s;", randTables)

}
var err error
smither, err = sqlsmith.NewSmither(db.db, r.Rnd, sqlsmith.DisableMutations())
Expand Down

0 comments on commit e882432

Please sign in to comment.