Skip to content

Commit

Permalink
sql: Add unit tests for planning inside the new schema changer
Browse files Browse the repository at this point in the history
Previously, the new schema changer did not have any unit
tests covering the planning capability. This was inadequate,
because we had no way of detect if plans were regressing with
changes or enhancements. To address this, this patch adds
basic tests to see if the operators/dependencies for a given
command are sane.
Release note: None
  • Loading branch information
fqazi committed May 28, 2021
1 parent 821d315 commit 38531fa
Show file tree
Hide file tree
Showing 6 changed files with 912 additions and 14 deletions.
12 changes: 6 additions & 6 deletions pkg/sql/schemachanger/scgraphviz/graphviz.go
Original file line number Diff line number Diff line change
Expand Up @@ -161,11 +161,11 @@ func htmlLabel(o interface{}) dot.HTML {
return dot.HTML(buf.String())
}

// toMap converts a struct to a map, field by field. If at any point a protobuf
// ToMap converts a struct to a map, field by field. If at any point a protobuf
// message is encountered, it is converted to a map using jsonpb to marshal it
// to json and then marshaling it back to a map. This approach allows zero
// values to be effectively omitted.
func toMap(v interface{}) (interface{}, error) {
func ToMap(v interface{}) (interface{}, error) {
if v == nil {
return nil, nil
}
Expand Down Expand Up @@ -202,7 +202,7 @@ func toMap(v interface{}) (interface{}, error) {
continue
}
var err error
if m[vt.Field(i).Name], err = toMap(vvf.Interface()); err != nil {
if m[vt.Field(i).Name], err = ToMap(vvf.Interface()); err != nil {
return nil, err
}
}
Expand Down Expand Up @@ -231,7 +231,7 @@ var objectTemplate = template.Must(template.New("obj").Funcs(template.FuncMap{
"isStruct": func(v interface{}) bool {
return reflect.ValueOf(v).Kind() == reflect.Struct
},
"toMap": toMap,
"ToMap": ToMap,
}).Parse(`
{{- define "key" -}}
<td>
Expand Down Expand Up @@ -271,7 +271,7 @@ var objectTemplate = template.Must(template.New("obj").Funcs(template.FuncMap{
{{- typeOf . -}}
</td>
<td>
{{- template "mapVal" (toMap $v) -}}
{{- template "mapVal" (ToMap $v) -}}
</td>
{{- else -}}
-{{- template "val" $v -}}
Expand All @@ -291,7 +291,7 @@ var objectTemplate = template.Must(template.New("obj").Funcs(template.FuncMap{
<table class="outer">
{{- template "header" . -}}
<tr><td>
{{- template "mapVal" (toMap .) -}}
{{- template "mapVal" (ToMap .) -}}
</td></tr>
</table>
`))
31 changes: 31 additions & 0 deletions pkg/sql/schemachanger/scplan/main_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,31 @@
// Copyright 2021 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 scplan_test

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"
)

func TestMain(m *testing.M) {
security.SetAssetLoader(securitytest.EmbeddedAssets)
randutil.SeedForTests()
serverutils.InitTestServerFactory(server.TestServerFactory)
serverutils.InitTestClusterFactory(testcluster.TestClusterFactory)
os.Exit(m.Run())
}
23 changes: 15 additions & 8 deletions pkg/sql/schemachanger/scplan/plan.go
Original file line number Diff line number Diff line change
Expand Up @@ -56,6 +56,9 @@ type Params struct {
//
// This doesn't do anything right now.
CreatedDescriptorIDs catalog.DescriptorIDSet
// DisableOpRandomization disables randomization for the final set of
// operations.
DisableOpRandomization bool
}

// A Plan is a schema change plan, primarily containing ops to be executed that
Expand Down Expand Up @@ -110,7 +113,7 @@ func MakePlan(initialStates []*scpb.Node, params Params) (_ Plan, err error) {
}); err != nil {
return Plan{}, err
}
stages := buildStages(initialStates, g)
stages := buildStages(initialStates, g, params)
return Plan{
Params: params,
InitialNodes: initialStates,
Expand All @@ -119,7 +122,7 @@ func MakePlan(initialStates []*scpb.Node, params Params) (_ Plan, err error) {
}, nil
}

func buildStages(init []*scpb.Node, g *scgraph.Graph) []Stage {
func buildStages(init []*scpb.Node, g *scgraph.Graph, params Params) []Stage {
// TODO(ajwerner): deal with the case where the target state was
// fulfilled by something that preceded the initial state.
cur := init
Expand Down Expand Up @@ -236,12 +239,15 @@ func buildStages(init []*scpb.Node, g *scgraph.Graph) []Stage {
// be order independent, however we will
// try to execute non-failing ones first.
opsSlice := s.Ops.Slice()
rand.Seed(timeutil.Now().UnixNano())
rand.Shuffle(len(opsSlice), func(i, j int) {
tmp := opsSlice[i]
opsSlice[i] = opsSlice[j]
opsSlice[j] = tmp
})
if !params.DisableOpRandomization {

rand.Seed(timeutil.Now().UnixNano())
rand.Shuffle(len(opsSlice), func(i, j int) {
tmp := opsSlice[i]
opsSlice[i] = opsSlice[j]
opsSlice[j] = tmp
})
}
// Place non-revertible operations at the end
sort.SliceStable(opsSlice, func(i, j int) bool {
if opsSlice[i].Revertible() == opsSlice[j].Revertible() {
Expand All @@ -251,6 +257,7 @@ func buildStages(init []*scpb.Node, g *scgraph.Graph) []Stage {
})
stages = append(stages, s)
cur = s.After

}
return stages
}
213 changes: 213 additions & 0 deletions pkg/sql/schemachanger/scplan/plan_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,213 @@
// Copyright 2021 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 scplan_test

import (
"context"
"fmt"
"path/filepath"
"strings"
"testing"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descs"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/resolver"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scbuild"
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scgraph"
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scgraphviz"
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb"
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb"
"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/log"
"github.com/cockroachdb/datadriven"
"github.com/stretchr/testify/require"
"gopkg.in/yaml.v2"
)

func TestPlanAlterTable(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
ctx := context.Background()

datadriven.Walk(t, filepath.Join("testdata"), func(t *testing.T, path string) {
s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{})
defer s.Stopper().Stop(ctx)

tdb := sqlutils.MakeSQLRunner(sqlDB)
run := func(t *testing.T, d *datadriven.TestData) string {
switch d.Cmd {
case "create-view":
fallthrough
case "create-sequence":
fallthrough
case "create-table":
stmts, err := parser.Parse(d.Input)
require.NoError(t, err)
require.Len(t, stmts, 1)
tableName := ""
switch node := stmts[0].AST.(type) {
case *tree.CreateTable:
tableName = node.Table.String()
case *tree.CreateSequence:
tableName = node.Name.String()
case *tree.CreateView:
tableName = node.Name.String()
default:
t.Fatal("not a CREATE TABLE/SEQUENCE/VIEW statement")
}
tdb.Exec(t, d.Input)

var tableID descpb.ID
tdb.QueryRow(t, `SELECT $1::regclass::int`, tableName).Scan(&tableID)
if tableID == 0 {
t.Fatalf("failed to read ID of new table %s", tableName)
}
t.Logf("created relation with id %d", tableID)

return ""
case "deps":
fallthrough
case "ops":
deps, cleanup := newTestingPlanDeps(s)
defer cleanup()

stmts, err := parser.Parse(d.Input)
require.NoError(t, err)
var outputNodes []*scpb.Node
for i := range stmts {
outputNodes, err = scbuild.Build(ctx, stmts[i].AST, *deps, outputNodes)
require.NoError(t, err)
}

plan, err := scplan.MakePlan(outputNodes,
scplan.Params{
ExecutionPhase: scplan.PostCommitPhase,
DisableOpRandomization: true,
})
require.NoError(t, err)

if d.Cmd == "ops" {
return marshalOps(t, &plan)
}
return marshalDeps(t, &plan)
case "unimplemented":
deps, cleanup := newTestingPlanDeps(s)
defer cleanup()

stmts, err := parser.Parse(d.Input)
require.NoError(t, err)
require.Len(t, stmts, 1)

stmt := stmts[0]
alter, ok := stmt.AST.(*tree.AlterTable)
require.Truef(t, ok, "not an ALTER TABLE statement: %s", stmt.SQL)
_, err = scbuild.Build(ctx, alter, *deps, nil)
require.Truef(t, scbuild.HasNotImplemented(err), "expected unimplemented, got %v", err)
return ""

default:
return fmt.Sprintf("unknown command: %s", d.Cmd)
}
}
datadriven.RunTest(t, path, run)
})
}

// indentText indents text for formatting out marshalled data.
func indentText(input string, tab string) (final string) {
split := strings.Split(input, "\n")
for idx, line := range split {
if len(line) == 0 {
continue
}
final += tab + line
if idx != len(split)-1 {
final = final + "\n"
}
}
return final
}

// marshalDeps marshals dependencies in scplan.Plan to a string.
func marshalDeps(t *testing.T, plan *scplan.Plan) string {
stages := ""
convertNode := func(node *scpb.Node) string {
depMap, err := scgraphviz.ToMap(node)
require.NoError(t, err)
data, err := yaml.Marshal(depMap)
require.NoError(t, err)
return string(data)
}

plan.Graph.ForEachNode(func(n *scpb.Node) error {
return plan.Graph.ForEachDepEdgeFrom(n, func(de *scgraph.DepEdge) error {
stages += fmt.Sprintf("Dependency From:\n%sTo:\n%s",
indentText(convertNode(de.To()), " "),
indentText(convertNode(de.From()), " "))
return nil
})
})
return stages
}

// marshalOps marshals operations in scplan.Plan to a string.
func marshalOps(t *testing.T, plan *scplan.Plan) string {
stages := ""
for stageIdx, stage := range plan.Stages {
stages += fmt.Sprintf("Stage %d\n", stageIdx)
stageOps := ""
for _, op := range stage.Ops.Slice() {
opMap, err := scgraphviz.ToMap(op)
require.NoError(t, err)
data, err := yaml.Marshal(opMap)
require.NoError(t, err)
stageOps += fmt.Sprintf("%T\n%s", op, indentText(string(data), " "))
}
stages += indentText(stageOps, " ")
}
return stages
}

func newTestingPlanDeps(s serverutils.TestServerInterface) (*scbuild.Dependencies, func()) {
execCfg := s.ExecutorConfig().(sql.ExecutorConfig)
ip, cleanup := sql.NewInternalPlanner(
"test",
kv.NewTxn(context.Background(), s.DB(), s.NodeID()),
security.RootUserName(),
&sql.MemoryMetrics{},
&execCfg,
// Setting the database on the session data to "defaultdb" in the obvious
// way doesn't seem to do what we want.
sessiondatapb.SessionData{},
)
planner := ip.(interface {
resolver.SchemaResolver
SemaCtx() *tree.SemaContext
EvalContext() *tree.EvalContext
Descriptors() *descs.Collection
})
buildDeps := scbuild.Dependencies{
Res: planner,
SemaCtx: planner.SemaCtx(),
EvalCtx: planner.EvalContext(),
Descs: planner.Descriptors(),
}
return &buildDeps, cleanup
}
Loading

0 comments on commit 38531fa

Please sign in to comment.