Skip to content

Commit

Permalink
randgen: add PopulateTableWithRandomData
Browse files Browse the repository at this point in the history
PopulateRandTable populates the caller's table with random data. This helper
function aims to make it easier for engineers to develop randomized tests that
leverage randgen / sqlsmith.

I considered adding random insert statements into sqlsmith's randtables setup,
however the high probably of a faulty insert statement would cause the whole
setup to fail. See #75159

In the future, I'd like to develop a new helper function
PopulateDatabaseWithRandData which calls PopulateTableWithRandData on each
table in the order of the fk dependency graph.

Informs #72345

Release note: None
  • Loading branch information
msbutler committed Feb 9, 2022
1 parent a3cfd63 commit 6236b0a
Show file tree
Hide file tree
Showing 6 changed files with 262 additions and 5 deletions.
3 changes: 0 additions & 3 deletions pkg/internal/sqlsmith/setup.go
Original file line number Diff line number Diff line change
Expand Up @@ -103,9 +103,6 @@ func randTablesN(r *rand.Rand, n int) string {
sb.WriteString(stmt.String())
sb.WriteString(";\n")
}

// TODO(mjibson): add random INSERTs.

return sb.String()
}

Expand Down
20 changes: 18 additions & 2 deletions pkg/sql/randgen/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,23 @@ go_library(

go_test(
name = "randgen_test",
srcs = ["mutator_test.go"],
srcs = [
"main_test.go",
"mutator_test.go",
"schema_test.go",
],
embed = [":randgen"],
deps = ["//pkg/util/randutil"],
deps = [
"//pkg/base",
"//pkg/security",
"//pkg/security/securitytest",
"//pkg/server",
"//pkg/sql/sem/tree",
"//pkg/testutils/serverutils",
"//pkg/testutils/sqlutils",
"//pkg/testutils/testcluster",
"//pkg/util/leaktest",
"//pkg/util/randutil",
"@com_github_stretchr_testify//require",
],
)
33 changes: 33 additions & 0 deletions pkg/sql/randgen/main_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
// Copyright 2019 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package randgen

import (
"os"
"testing"

"github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/security/securitytest"
"github.com/cockroachdb/cockroach/pkg/server"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
)

//go:generate ../../util/leaktest/add-leaktest.sh *_test.go

func TestMain(m *testing.M) {
security.SetAssetLoader(securitytest.EmbeddedAssets)
randutil.SeedForTests()
serverutils.InitTestServerFactory(server.TestServerFactory)
serverutils.InitTestClusterFactory(testcluster.TestClusterFactory)
os.Exit(m.Run())
}
2 changes: 2 additions & 0 deletions pkg/sql/randgen/mutator_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,10 +14,12 @@ import (
"strings"
"testing"

"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
)

func TestPostgresMutator(t *testing.T) {
defer leaktest.AfterTest(t)()
q := `
CREATE TABLE t (s STRING FAMILY fam1, b BYTES, FAMILY fam2 (b), PRIMARY KEY (s ASC, b DESC), INDEX (s) STORING (b))
PARTITION BY LIST (s)
Expand Down
133 changes: 133 additions & 0 deletions pkg/sql/randgen/schema.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/rowenc"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/types"
Expand Down Expand Up @@ -186,6 +187,138 @@ func RandCreateTableWithColumnIndexNumberGenerator(
return res[0].(*tree.CreateTable)
}

func parseCreateStatement(createStmtSQL string) (*tree.CreateTable, error) {
var p parser.Parser
stmts, err := p.Parse(createStmtSQL)
if err != nil {
return nil, err
}
if len(stmts) != 1 {
return nil, errors.Errorf("parsed CreateStatement string yielded more than one parsed statment")
}
tableStmt, ok := stmts[0].AST.(*tree.CreateTable)
if !ok {
return nil, errors.Errorf("AST could not be cast to *tree.CreateTable")
}
return tableStmt, nil
}

// generateInsertStmtVals generates random data for a string builder thats
// used after the VALUES keyword in an INSERT statement.
func generateInsertStmtVals(rng *rand.Rand, colTypes []*types.T, nullable []bool) string {
var valBuilder strings.Builder
valBuilder.WriteString("(")
comma := ""
for j := 0; j < len(colTypes); j++ {
valBuilder.WriteString(comma)
var d tree.Datum
if rand.Intn(10) < 4 {
// 40% of the time, use a corner case value
d = randInterestingDatum(rng, colTypes[j])
}
if colTypes[j] == types.RegType {
// RandDatum is naive to the constraint that a RegType < len(types.OidToType),
// at least before linking and user defined types are added.
d = tree.NewDOid(tree.DInt(rand.Intn(len(types.OidToType))))
}
if d == nil {
d = RandDatum(rng, colTypes[j], nullable[j])
}
valBuilder.WriteString(tree.AsStringWithFlags(d, tree.FmtParsable))
comma = ", "
}
valBuilder.WriteString(")")
return valBuilder.String()
}

// TODO(butler): develop new helper function PopulateDatabaseWithRandData which calls
// PopulateTableWithRandData on each table in the order of the fk
// dependency graph.

// PopulateTableWithRandData populates the provided table by executing exactly
// `numInserts` statements. numRowsInserted <= numInserts because inserting into
// an arbitrary table can fail for reasons which include:
// - UNIQUE or CHECK constraint violation. RandDatum is naive to these constraints.
// - Out of range error for a computed INT2 or INT4 column.
//
// If numRowsInserted == 0, PopulateTableWithRandomData or RandDatum couldn't
// handle this table's schema. Consider increasing numInserts or filing a bug.
func PopulateTableWithRandData(
rng *rand.Rand, db *gosql.DB, tableName string, numInserts int,
) (numRowsInserted int, err error) {
var createStmtSQL string
res := db.QueryRow(fmt.Sprintf("SELECT create_statement FROM [SHOW CREATE TABLE %s]", tableName))
err = res.Scan(&createStmtSQL)
if err != nil {
return 0, errors.Wrapf(err, "table does not exist in db")
}
createStmt, err := parseCreateStatement(createStmtSQL)
if err != nil {
return 0, errors.Wrapf(err, "failed to determine table schema")
}

// Find columns subject to a foreign key constraint
var hasFK = map[string]bool{}
for _, def := range createStmt.Defs {
if fk, ok := def.(*tree.ForeignKeyConstraintTableDef); ok {
for _, col := range fk.FromCols {
hasFK[col.String()] = true
}
}
}

// Populate helper objects for insert statement creation and error out if a
// column's constraints will make it impossible to execute random insert
// statements.

colTypes := make([]*types.T, 0)
nullable := make([]bool, 0)
var colNameBuilder strings.Builder
comma := ""
for _, def := range createStmt.Defs {
if col, ok := def.(*tree.ColumnTableDef); ok {
if _, ok := hasFK[col.Name.String()]; ok {
// Given that this function only populates an individual table without
// considering other tables in the database, populating a column with a
// foreign key reference with actual data can be nearly impossible. To
// make inserts pass more frequently, this function skips populating
// columns with a foreign key reference. Sadly, if these columns with
// FKs are also NOT NULL, 0 rows will get inserted.

// TODO(butler): get the unique values from each foreign key reference and
// populate the column by sampling the FK's unique values.
if col.Nullable.Nullability == tree.Null {
continue
}
}
if col.Computed.Computed || col.Hidden {
// Cannot insert values into hidden or computed columns, so skip adding
// them to the list of columns to insert data into.
continue
}
colTypes = append(colTypes, tree.MustBeStaticallyKnownType(col.Type.(*types.T)))
nullable = append(nullable, col.Nullable.Nullability == tree.Null)

colNameBuilder.WriteString(comma)
colNameBuilder.WriteString(col.Name.String())
comma = ", "
}
}

for i := 0; i < numInserts; i++ {
insertVals := generateInsertStmtVals(rng, colTypes, nullable)
insertStmt := fmt.Sprintf("INSERT INTO %s (%s) VALUES %s;",
tableName,
colNameBuilder.String(),
insertVals)
_, err := db.Exec(insertStmt)
if err == nil {
numRowsInserted++
}
}
return numRowsInserted, nil
}

// GenerateRandInterestingTable takes a gosql.DB connection and creates
// a table with all the types in randInterestingDatums and rows of the
// interesting datums.
Expand Down
76 changes: 76 additions & 0 deletions pkg/sql/randgen/schema_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,76 @@
// Copyright 2022 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package randgen

import (
"context"
"fmt"
"strings"
"testing"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
"github.com/stretchr/testify/require"
)

// TestPopulateTableWithRandData generates some random tables and passes if it
// at least one of those tables will be successfully populated.
func TestPopulateTableWithRandData(t *testing.T) {
defer leaktest.AfterTest(t)()

ctx := context.Background()
s, dbConn, _ := serverutils.StartServer(t, base.TestServerArgs{})
defer s.Stopper().Stop(ctx)

rng, _ := randutil.NewTestRand()

sqlDB := sqlutils.MakeSQLRunner(dbConn)
sqlDB.Exec(t, "CREATE DATABASE rand")

// Turn off auto stats collection to prevent out of memory errors on stress tests
sqlDB.Exec(t, "SET CLUSTER SETTING sql.stats.automatic_collection.enabled = false")

tablePrefix := "table"
numTables := 10

stmts := RandCreateTables(rng, tablePrefix, numTables,
PartialIndexMutator,
ForeignKeyMutator,
)

var sb strings.Builder
for _, stmt := range stmts {
sb.WriteString(tree.SerializeForDisplay(stmt))
sb.WriteString(";\n")
}
sqlDB.Exec(t, sb.String())

// To prevent the test from being flaky, pass the test if PopulateTableWithRandomData
// inserts at least one row in at least one table.
success := false
for i := 1; i <= numTables; i++ {
tableName := tablePrefix + fmt.Sprint(i)
numRows := 30
numRowsInserted, err := PopulateTableWithRandData(rng, dbConn, tableName, numRows)
require.NoError(t, err)
res := sqlDB.QueryStr(t, fmt.Sprintf("SELECT count(*) FROM %s", tableName))
require.Equal(t, fmt.Sprint(numRowsInserted), res[0][0])
if numRowsInserted > 0 {
success = true
break
}
}
require.Equal(t, true, success)
}

0 comments on commit 6236b0a

Please sign in to comment.