Skip to content

Commit

Permalink
Merge #122864
Browse files Browse the repository at this point in the history
122864: sql: add randomized test for plan-gist logic r=yuzefovich a=yuzefovich

This commit adds a randomized test for plan-gist logic. It utilizes sqlsmith to generate random stmts, then runs EXPLAIN (GIST) for it to retrieve the gist followed by decoding that gist, and ensures that no internal errors occur. All generated stmts are also executed with a short timeout to allow for DB state to evolve. Additionally, all gists are accumulated and decoded against an empty DB.

The test is placed in the ccl package to exercise features gated behind CCL license.

The test quickly discovered bug fixed in 1315dda (when the corresponding fix is reverted) but haven't found anything new for plan-gist logic (but did hit a few issues elsewhere which are currently ignored in the test).

Fixes: #117634.

Release note: None

Co-authored-by: Yahor Yuzefovich <[email protected]>
  • Loading branch information
craig[bot] and yuzefovich committed May 20, 2024
2 parents a896503 + 84e1d21 commit a7aad51
Show file tree
Hide file tree
Showing 4 changed files with 208 additions and 8 deletions.
1 change: 1 addition & 0 deletions pkg/ccl/testccl/sqlccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@ go_test(
"//pkg/settings/cluster",
"//pkg/spanconfig",
"//pkg/sql",
"//pkg/sql/catalog",
"//pkg/sql/gcjob",
"//pkg/sql/isql",
"//pkg/sql/lexbase",
Expand Down
189 changes: 189 additions & 0 deletions pkg/ccl/testccl/sqlccl/explain_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,13 +12,17 @@ import (
"context"
"strings"
"testing"
"time"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/internal/sqlsmith"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/lexbase"
"github.com/cockroachdb/cockroach/pkg/sql/sqltestutils"
"github.com/cockroachdb/cockroach/pkg/sql/tests"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
Expand Down Expand Up @@ -126,3 +130,188 @@ func TestExplainRedactDDL(t *testing.T) {

tests.GenerateAndCheckRedactedExplainsForPII(t, smith, numStatements, conn, containsPII)
}

// TestExplainGist is a randomized test for plan-gist logic.
func TestExplainGist(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

skip.UnderDeadlock(t, "the test is too slow")
skip.UnderRace(t, "the test is too slow")

ctx := context.Background()
rng, _ := randutil.NewTestRand()

const numStatements = 500
var gists []string

t.Run("main", func(t *testing.T) {
srv, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{})
defer srv.Stopper().Stop(ctx)
runner := sqlutils.MakeSQLRunner(sqlDB)

// Set up some initial state.
setup := sqlsmith.Setups["seed"](rng)
setup = append(setup, "SET CLUSTER SETTING sql.stats.automatic_collection.enabled = off;")
if rng.Float64() < 0.5 {
// In some cases have stats on the seed table.
setup = append(setup, "ANALYZE seed;")
}
runner.ExecMultiple(t, setup...)

// Given that statement timeout might apply differently between test
// runs with the same seed (e.g. because of different CPU load), we'll
// accumulate all successful statements for ease of reproduction.
var successfulStmts strings.Builder
logStmt := func(stmt string) {
successfulStmts.WriteString(stmt)
successfulStmts.WriteString(";\n")
}
for _, stmt := range setup {
logStmt(stmt)
}

smither, err := sqlsmith.NewSmither(sqlDB, rng, sqlsmith.SimpleNames())
if err != nil {
t.Fatal(err)
}
defer smither.Close()

// Always set the statement timeout - we actually execute the stmts on
// the best effort basis to evolve the state of the DB. Note that we do
// this after having set up the smither since it itself can issue some
// statements.
//
// Note that we don't include this statement into successfulStmts since
// all that are included there must have not timed out.
timeoutStmt := "SET statement_timeout = '0.1s';"
runner.Exec(t, timeoutStmt)

checkErr := func(err error, stmt string) {
if err != nil && strings.Contains(err.Error(), "internal error") {
// Ignore all errors except the internal ones.
for _, knownErr := range []string{
"ALTER TABLE: failed to type check the cast of", // #114316
"invalid datum type given: RECORD, expected RECORD", // #117101
"expected equivalence dependants to be its closure", // #119045
} {
if strings.Contains(err.Error(), knownErr) {
// Don't fail the test on a set of known errors.
return
}
}
t.Log(successfulStmts.String())
t.Fatalf("%v: %s", err, stmt)
}
}

for i := 0; i < numStatements; i++ {
stmt := func() string {
for {
stmt := smither.Generate()
switch stmt {
case "BEGIN TRANSACTION", "COMMIT TRANSACTION", "ROLLBACK TRANSACTION":
// Ignore frequently generated statements that result in
// a syntax error with EXPLAIN.
default:
return stmt
}
}
}()

row := sqlDB.QueryRow("EXPLAIN (GIST) " + stmt)
if err = row.Err(); err != nil {
checkErr(err, stmt)
continue
}
var gist string
err = row.Scan(&gist)
if err != nil {
if !sqltestutils.IsClientSideQueryCanceledErr(err) {
t.Fatal(err)
}
// Short statement timeout might exceed planning time. Let's
// retry this statement with longer timeout.
if ok := func() bool {
runner.Exec(t, "SET statement_timeout = '1s'")
defer runner.Exec(t, timeoutStmt)
row = sqlDB.QueryRow("EXPLAIN (GIST) " + stmt)
if err = row.Err(); err != nil {
checkErr(err, stmt)
return false
}
err = row.Scan(&gist)
if err != nil {
t.Fatalf("when re-running EXPLAIN (GIST) with 1s timeout: %+v", err)
}
return true
}(); !ok {
continue
}
}
_, err = sqlDB.Exec("SELECT crdb_internal.decode_plan_gist($1);", gist)
if err != nil {
// We might be still in the process of cancelling the previous
// DROP operation - ignore this particular error.
if !errors.Is(err, catalog.ErrDescriptorDropped) {
t.Fatal(err)
}
continue
}
// Store the gist to be run against empty DB.
gists = append(gists, gist)

if shouldSkip := func() bool {
// Executing these statements is out of scope for this test
// (skipping them makes reproduction easier).
for _, toSkipPrefix := range []string{"BACKUP", "EXPORT", "IMPORT", "RESTORE"} {
if strings.HasPrefix(stmt, toSkipPrefix) {
return true
}
}
for _, toSkipSubstring := range []string{
"ALTER PRIMARY KEY", // #123017
} {
if strings.Contains(stmt, toSkipSubstring) {
return true
}
}
return false
}(); shouldSkip {
continue
}

// Execute the stmt with short timeout so that the table schema is
// modified. We do so in a separate goroutine to ensure that we fail
// the test if the stmt doesn't respect the timeout (if we didn't
// use a separate goroutine, then the main test goroutine would be
// blocked until either the stmt is completed or is canceled,
// possibly timing out the test run).
errCh := make(chan error)
go func() {
_, err := sqlDB.Exec(stmt)
errCh <- err
}()
select {
case err = <-errCh:
if err != nil {
checkErr(err, stmt)
} else {
logStmt(stmt)
}
case <-time.After(time.Minute):
t.Log(successfulStmts.String())
t.Fatalf("stmt wasn't canceled by statement_timeout of 0.1s - ran at least for 1m: %s", stmt)
}
}
})

t.Run("empty", func(t *testing.T) {
srv, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{})
defer srv.Stopper().Stop(ctx)
runner := sqlutils.MakeSQLRunner(sqlDB)
for _, gist := range gists {
runner.Exec(t, "SELECT crdb_internal.decode_plan_gist($1);", gist)
}
})
}
1 change: 0 additions & 1 deletion pkg/cmd/reduce/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -313,7 +313,6 @@ SELECT '%[1]s';
cmd = exec.CommandContext(ctx, binary,
"demo",
"--empty",
"--disable-demo-license",
"--set=errexit=false",
"--format=tsv",
)
Expand Down
25 changes: 18 additions & 7 deletions pkg/internal/sqlsmith/relational.go
Original file line number Diff line number Diff line change
Expand Up @@ -813,9 +813,8 @@ func makeSelect(s *Smither) (tree.Statement, bool) {
order := make(tree.OrderBy, len(refs))
for i, r := range refs {
var expr tree.Expr = r.item
// PostGIS cannot order box2d types, so we cast to string so the
// order is deterministic.
if s.postgres && r.typ.Family() == types.Box2DFamily {
if !s.isOrderable(r.typ) {
// Cast to string so the order is deterministic.
expr = &tree.CastExpr{Expr: r.item, Type: types.String}
}
order[i] = &tree.Order{
Expand Down Expand Up @@ -1722,15 +1721,28 @@ func (s *Smither) makeHaving(refs colRefs) *tree.Where {
return nil
}

func (s *Smither) isOrderable(typ *types.T) bool {
if s.postgres {
// PostGIS cannot order box2d types.
return typ.Family() != types.Box2DFamily
}
switch typ.Family() {
case types.TSQueryFamily, types.TSVectorFamily:
// We can't order by these types - see #92165.
return false
default:
return true
}
}

func (s *Smither) makeOrderBy(refs colRefs) tree.OrderBy {
if len(refs) == 0 {
return nil
}
var ob tree.OrderBy
for s.coin() {
ref := refs[s.rnd.Intn(len(refs))]
// PostGIS cannot order box2d types.
if s.postgres && ref.typ.Family() == types.Box2DFamily {
if !s.isOrderable(ref.typ) {
continue
}
ob = append(ob, &tree.Order{
Expand All @@ -1748,8 +1760,7 @@ func (s *Smither) makeOrderByWithAllCols(refs colRefs) tree.OrderBy {
}
var ob tree.OrderBy
for _, ref := range refs {
// PostGIS cannot order box2d types.
if s.postgres && ref.typ.Family() == types.Box2DFamily {
if !s.isOrderable(ref.typ) {
continue
}
ob = append(ob, &tree.Order{
Expand Down

0 comments on commit a7aad51

Please sign in to comment.