Skip to content

Commit

Permalink
sql/schemachanger: add DML injection tests for rollback scenarios
Browse files Browse the repository at this point in the history
Previously, our DML injection tests only covered happy paths
for schema changes. We recently ran into a bug where the old
primary index was corrupted after DML and left in a bad state
visible only after a rollback. To address this, this patch
adds tests for this specific scenario and adds the ability
to test rollbacks with DML injection.

Epic: none

Release note: None
  • Loading branch information
fqazi committed Mar 29, 2023
1 parent 710752e commit 6c4208d
Show file tree
Hide file tree
Showing 40 changed files with 8,434 additions and 20 deletions.
10 changes: 10 additions & 0 deletions pkg/ccl/schemachangerccl/backup_base_generated_test.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

10 changes: 10 additions & 0 deletions pkg/ccl/schemachangerccl/backup_base_mixed_generated_test.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

85 changes: 65 additions & 20 deletions pkg/sql/schemachanger/sctest/cumulative.go
Original file line number Diff line number Diff line change
Expand Up @@ -72,6 +72,8 @@ type stageExecStmt struct {
expectedOutput string
observedOutput string
schemaChangeErrorRegex *regexp.Regexp
// schemaChangeErrorRegexRollback will cause a rollback.
schemaChangeErrorRegexRollback *regexp.Regexp
}

// HasSchemaChangeError indicates if a schema change error will be observed,
Expand All @@ -80,6 +82,13 @@ func (e *stageExecStmt) HasSchemaChangeError() bool {
return e.schemaChangeErrorRegex != nil
}

func (e *stageExecStmt) HasAnySchemaChangeError() *regexp.Regexp {
if e.schemaChangeErrorRegex != nil {
return e.schemaChangeErrorRegex
}
return e.schemaChangeErrorRegexRollback
}

// stageKeyOrdinalLatest targets the latest ordinal in a stage.
const stageKeyOrdinalLatest = math.MaxUint16

Expand All @@ -89,14 +98,16 @@ type stageKey struct {
minOrdinal int
maxOrdinal int
phase scop.Phase
rollback bool
}

// makeStageKey constructs a stage key targeting a single ordinal.
func makeStageKey(phase scop.Phase, ordinal int) stageKey {
func makeStageKey(phase scop.Phase, ordinal int, rollback bool) stageKey {
return stageKey{
phase: phase,
minOrdinal: ordinal,
maxOrdinal: ordinal,
rollback: rollback,
}
}

Expand Down Expand Up @@ -151,6 +162,7 @@ func (m *stageExecStmtMap) getExecStmts(targetKey stageKey) []*stageExecStmt {
}
for _, key := range m.entries {
if key.stageKey.phase == targetKey.phase &&
key.stageKey.rollback == targetKey.rollback &&
targetKey.minOrdinal >= key.stageKey.minOrdinal &&
targetKey.minOrdinal <= key.stageKey.maxOrdinal {
stmts = append(stmts, key.stmt)
Expand All @@ -162,6 +174,12 @@ func (m *stageExecStmtMap) getExecStmts(targetKey stageKey) []*stageExecStmt {
// AssertMapIsUsed asserts that all injected DML statements injected
// at various stages.
func (m *stageExecStmtMap) AssertMapIsUsed(t *testing.T) {
// If there is any rollback error, then not all stages will be used.
for _, e := range m.entries {
if e.stmt.schemaChangeErrorRegexRollback != nil {
return
}
}
if len(m.entries) != len(m.usedMap) {
for _, entry := range m.entries {
if _, ok := m.usedMap[entry.stmt]; !ok {
Expand Down Expand Up @@ -213,16 +231,17 @@ func (m *stageExecStmtMap) GetInjectionRanges(
panic("unknown phase type for latest")
}
}
if !key.stmt.HasSchemaChangeError() {
if !key.stmt.HasSchemaChangeError() && !key.rollback {
forcedSplitEntries = append(forcedSplitEntries, key)
} else {
} else if !key.rollback {
for i := key.minOrdinal; i <= key.maxOrdinal; i++ {
forcedSplitEntries = append(forcedSplitEntries,
stageKeyEntry{
stageKey: stageKey{
minOrdinal: i,
maxOrdinal: i,
phase: key.phase,
rollback: key.rollback,
},
stmt: key.stmt,
},
Expand Down Expand Up @@ -307,19 +326,25 @@ func (m *stageExecStmtMap) ParseStageExec(t *testing.T, d *datadriven.TestData)
// parseStageCommon common fields between stage-exec and stage-query, which
// support the following keys:
// - phase - The phase in which this statement/query should be injected, of the
// string scop.Phase.
// string scop.Phase. Note: PreCommitPhase with stage 1 can be used to
// inject failures that will only happen for DML injection testing.
// - stage / stageStart / stageEnd - The ordinal for the stage where this
// statement should be injected. stageEnd accepts the special value
// latest which will map to the highest observed stage.
// - schemaChangeExecError a schema change execution error will be encountered
// by injecting at this stage.
// - schemaChangeExecErrorForRollback a schema change execution error that will
// be encountered at a future stage, leading to a rollback.
// - statements can refer to builtin variable names with a $
// - $stageKey - A unique identifier for stages and phases
// - $successfulStageCount - Number of stages of the that have been successfully
// executed with injections.
// executed with injections
func (m *stageExecStmtMap) parseStageCommon(
t *testing.T, d *datadriven.TestData, execType stageExecType,
) {
var key stageKey
var schemaChangeErrorRegex *regexp.Regexp
var schemaChangeErrorRegexRollback *regexp.Regexp
stmts := strings.Split(d.Input, ";")
require.NotEmpty(t, stmts)
// Remove any trailing empty lines.
Expand Down Expand Up @@ -369,18 +394,27 @@ func (m *stageExecStmtMap) parseStageCommon(
}
case "schemaChangeExecError":
schemaChangeErrorRegex = regexp.MustCompile(strings.Join(cmdArgs.Vals, " "))
require.Nil(t, schemaChangeErrorRegexRollback, "future and current stage errors cannot be set concurrently")
case "schemaChangeExecErrorForRollback":
schemaChangeErrorRegexRollback = regexp.MustCompile(strings.Join(cmdArgs.Vals, " "))
require.Nil(t, schemaChangeErrorRegex, "rollback and current stage errors cannot be set concurrently")
case "rollback":
rollback, err := strconv.ParseBool(cmdArgs.Vals[0])
require.NoError(t, err)
key.rollback = rollback
default:
require.Failf(t, "unknown key encountered", "key was %s", cmdArgs.Key)
}
}
entry := stageKeyEntry{
stageKey: key,
stmt: &stageExecStmt{
execType: execType,
stmts: stmts,
observedOutput: "",
expectedOutput: d.Expected,
schemaChangeErrorRegex: schemaChangeErrorRegex,
execType: execType,
stmts: stmts,
observedOutput: "",
expectedOutput: d.Expected,
schemaChangeErrorRegex: schemaChangeErrorRegex,
schemaChangeErrorRegexRollback: schemaChangeErrorRegexRollback,
},
}
m.entries = append(m.entries, entry)
Expand Down Expand Up @@ -814,31 +848,36 @@ func ExecuteWithDMLInjection(t *testing.T, relPath string, newCluster NewCluster
}
testDMLInjectionCase = func(t *testing.T, setup, stmts []parser.Statement, injection stageKey) {
var schemaChangeErrorRegex *regexp.Regexp
var lastRollbackStageKey *stageKey
usedStages := make(map[int]struct{})
successfulStages := 0
var tdb *sqlutils.SQLRunner
_, db, cleanup := newCluster(t, &scexec.TestingKnobs{
BeforeStage: func(p scplan.Plan, stageIdx int) error {
// FIXME: Support rollback detection
s := p.Stages[stageIdx]
if !p.CurrentState.InRollback &&
!p.InRollback &&
injection.phase == p.Stages[stageIdx].Phase &&
if (injection.phase == p.Stages[stageIdx].Phase &&
p.Stages[stageIdx].Ordinal >= injection.minOrdinal &&
p.Stages[stageIdx].Ordinal <= injection.maxOrdinal {
p.Stages[stageIdx].Ordinal <= injection.maxOrdinal) ||
(p.InRollback || p.CurrentState.InRollback) || /* Rollbacks are always injected */
(p.Stages[stageIdx].Phase == scop.PreCommitPhase) {
jobErrorMutex.Lock()
defer jobErrorMutex.Unlock()
key := makeStageKey(s.Phase, s.Ordinal)
key := makeStageKey(s.Phase, s.Ordinal, p.InRollback || p.CurrentState.InRollback)
if _, ok := usedStages[key.AsInt()]; !ok {
successfulStages++
// Rollbacks don't count towards the successful count
if !p.InRollback && !p.CurrentState.InRollback &&
p.Stages[stageIdx].Phase != scop.PreCommitPhase {
successfulStages++
} else {
lastRollbackStageKey = &key
}
injectStmts := injectionFunc(key, tdb, successfulStages)
regexSetOnce := false
schemaChangeErrorRegex = nil
for _, injectStmt := range injectStmts {
if injectStmt != nil &&
injectStmt.HasSchemaChangeError() {
injectStmt.HasAnySchemaChangeError() != nil {
require.Falsef(t, regexSetOnce, "multiple statements are expecting errors in the same phase.")
schemaChangeErrorRegex = injectStmt.schemaChangeErrorRegex
schemaChangeErrorRegex = injectStmt.HasAnySchemaChangeError()
regexSetOnce = true
t.Logf("Expecting schema change error: %v", schemaChangeErrorRegex)
}
Expand Down Expand Up @@ -867,6 +906,12 @@ func ExecuteWithDMLInjection(t *testing.T, relPath string, newCluster NewCluster
require.NoError(t, executeSchemaChangeTxn(
context.Background(), t, setup, stmts, db, nil, nil, onError,
))
// Re-inject anything from the rollback once the job transaction
// commits, this enforces any sanity checks one last time in
// the final descriptor state.
if lastRollbackStageKey != nil {
injectionFunc(*lastRollbackStageKey, tdb, successfulStages)
}
require.Equal(t, errorDetected, schemaChangeErrorRegex != nil)
}
cumulativeTest(t, relPath, testFunc)
Expand Down
50 changes: 50 additions & 0 deletions pkg/sql/schemachanger/sctest_generated_test.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

10 changes: 10 additions & 0 deletions pkg/sql/schemachanger/sctest_mixed_generated_test.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

Loading

0 comments on commit 6c4208d

Please sign in to comment.