Skip to content

Commit

Permalink
sql: First round of cleanup of schemachange/random-load
Browse files Browse the repository at this point in the history
This issue addresses several issues uncovered in running the randomized
schema changer. Specifically:

- Makes several errors pgcodes, so that they can be properly added to
  the expected errors list in the randomized schema changer.
- Detects cases where the region column (crdb_region) is used multiple
  times in an index definition.
- Allows for column type changes, which must have the experimental flag
  enable_experimental_alter_column_type_general flag set.

It also disables the testing of setColumnType (tracked with cockroachdb#66662) as
well as making a column nullable/non-nullable due to a timing hole
(tracked with cockroachdb#66663).

Release note: None.
  • Loading branch information
ajstorm committed Jun 22, 2021
1 parent 19865dd commit 46baa75
Show file tree
Hide file tree
Showing 7 changed files with 148 additions and 19 deletions.
1 change: 1 addition & 0 deletions pkg/cmd/roachtest/schemachange_random_load.go
Original file line number Diff line number Diff line change
Expand Up @@ -181,6 +181,7 @@ func runSchemaChangeRandomLoad(ctx context.Context, t *test, c Cluster, maxOps,
// saveArtifacts saves important test artifacts in the artifacts directory.
func saveArtifacts(ctx context.Context, t *test, c Cluster, storeDirectory string) {
db := c.Conn(ctx, 1)
defer db.Close()

// Save a backup file called schemachange to the store directory.
_, err := db.Exec("BACKUP DATABASE schemachange to 'nodelocal://1/schemachange'")
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvclient/kvcoord/testdata/savepoints
Original file line number Diff line number Diff line change
Expand Up @@ -385,7 +385,7 @@ injected error

rollback x
----
(*telemetrykeys.withTelemetry) unimplemented: cannot rollback to savepoint after error
(*pgerror.withCandidateCode) unimplemented: cannot rollback to savepoint after error

subtest end

Expand Down
10 changes: 5 additions & 5 deletions pkg/sql/catalog/tabledesc/validate.go
Original file line number Diff line number Diff line change
Expand Up @@ -598,13 +598,13 @@ func (desc *wrapper) ValidateSelf(vea catalog.ValidationErrorAccumulator) {
if alterPKMutation == m.MutationID {
vea.Report(unimplemented.NewWithIssue(
45615,
"cannot perform other schema changes in the same transaction as a primary key change",
))
"cannot perform other schema changes in the same transaction as a primary key change"),
)
} else {
vea.Report(unimplemented.NewWithIssue(
45615,
"cannot perform a schema change operation while a primary key change is in progress",
))
"cannot perform a schema change operation while a primary key change is in progress"),
)
}
return
}
Expand Down Expand Up @@ -969,7 +969,7 @@ func (desc *wrapper) validateTableIndexes(columnNames map[string]descpb.ColumnID
idx.GetName(), name, colID, idx.IndexDesc().KeyColumnIDs[i])
}
if validateIndexDup.Contains(colID) {
return fmt.Errorf("index %q contains duplicate column %q", idx.GetName(), name)
return pgerror.Newf(pgcode.FeatureNotSupported, "index %q contains duplicate column %q", idx.GetName(), name)
}
validateIndexDup.Add(colID)
}
Expand Down
2 changes: 2 additions & 0 deletions pkg/util/errorutil/unimplemented/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -7,6 +7,8 @@ go_library(
visibility = ["//visibility:public"],
deps = [
"//pkg/build",
"//pkg/sql/pgwire/pgcode",
"//pkg/sql/pgwire/pgerror",
"@com_github_cockroachdb_errors//:errors",
],
)
Expand Down
5 changes: 5 additions & 0 deletions pkg/util/errorutil/unimplemented/unimplemented.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,8 @@ import (
"fmt"

"github.com/cockroachdb/cockroach/pkg/build"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/errors"
)

Expand Down Expand Up @@ -96,5 +98,8 @@ func unimplementedInternal(
// perform telemetry.
err = errors.WithTelemetry(err, detail)
}

// Wrap with the corresponding PG error for unimplemented.
err = pgerror.WithCandidateCode(err, pgcode.FeatureNotSupported)
return err
}
54 changes: 53 additions & 1 deletion pkg/workload/schemachange/error_screening.go
Original file line number Diff line number Diff line change
Expand Up @@ -65,6 +65,11 @@ func scanBool(tx *pgx.Tx, query string, args ...interface{}) (b bool, err error)
return b, errors.Wrapf(err, "scanBool: %q %q", query, args)
}

func scanString(tx *pgx.Tx, query string, args ...interface{}) (s string, err error) {
err = tx.QueryRow(query, args...).Scan(&s)
return s, errors.Wrapf(err, "scanString: %q %q", query, args)
}

func schemaExists(tx *pgx.Tx, schemaName string) (bool, error) {
return scanBool(tx, `SELECT EXISTS (
SELECT schema_name
Expand Down Expand Up @@ -834,6 +839,53 @@ SELECT
)
}

// getRegionColumn returns the column used for partitioning a REGIONAL BY ROW
// table. This column is either the tree.RegionalByRowRegionDefaultCol column,
// or the column specified in the AS clause. This function asserts if the
// supplied table is not REGIONAL BY ROW.
func getRegionColumn(tx *pgx.Tx, tableName *tree.TableName) (string, error) {
isTableRegionalByRow, err := tableIsRegionalByRow(tx, tableName)
if err != nil {
return "", err
}
if !isTableRegionalByRow {
return "", errors.AssertionFailedf(
"invalid call to get region column of table %s which is not a REGIONAL BY ROW table",
tableName.String())
}

regionCol, err := scanString(
tx,
`
WITH
descriptors
AS (
SELECT
crdb_internal.pb_to_json(
'cockroach.sql.sqlbase.Descriptor',
descriptor
)->'table'
AS d
FROM
system.descriptor
WHERE
id = $1::REGCLASS
)
SELECT
COALESCE (d->'localityConfig'->'regionalByRow'->>'as', $2)
FROM
descriptors;
`,
tableName.String(),
tree.RegionalByRowRegionDefaultCol,
)
if err != nil {
return "", err
}

return regionCol, nil
}

// tableIsRegionalByRow checks whether the given table is a REGIONAL BY ROW table.
func tableIsRegionalByRow(tx *pgx.Tx, tableName *tree.TableName) (bool, error) {
return scanBool(
Expand Down Expand Up @@ -902,7 +954,7 @@ SELECT EXISTS (
)
}

// databaseHasOngoingAlterPKChanges checks whether a given database has any tables
// databaseHasRegionalByRowChange checks whether a given database has any tables
// which are currently undergoing a change to or from REGIONAL BY ROW, or
// REGIONAL BY ROW tables with schema changes on it.
func databaseHasRegionalByRowChange(tx *pgx.Tx) (bool, error) {
Expand Down
93 changes: 81 additions & 12 deletions pkg/workload/schemachange/operation_generator.go
Original file line number Diff line number Diff line change
Expand Up @@ -219,7 +219,7 @@ var opWeights = []int{
renameView: 1,
setColumnDefault: 1,
setColumnNotNull: 1,
setColumnType: 1,
setColumnType: 0, // Disabled and tracked with #66662.
survive: 1,
insertRow: 0,
validate: 2, // validate twice more often
Expand Down Expand Up @@ -473,15 +473,35 @@ func (og *operationGenerator) alterTableLocality(tx *pgx.Tx) (string, error) {
},
func() (string, error) {
columnForAs, err := og.randColumnWithMeta(tx, *tableName, og.alwaysExisting())
columnForAsUsed := false
if err != nil {
return "", err
}
ret := "REGIONAL BY ROW"
if columnForAs.typ.TypeMeta.Name != nil {
if columnForAs.typ.TypeMeta.Name.Basename() == tree.RegionEnum {
ret += "AS " + columnForAs.name
if columnForAs.typ.TypeMeta.Name.Basename() == tree.RegionEnum &&
!columnForAs.nullable {
ret += " AS " + columnForAs.name
columnForAsUsed = true
}
}
// If the table has a crdb_region column, make sure that it's not
// nullable. This is required to handle the case where there's an
// existing crdb_region column, but it is nullable, and therefore
// cannot be used as the implicit partitioning column.
if !columnForAsUsed {
columnNames, err := og.getTableColumns(tx, tableName.String(), true)
if err != nil {
return "", err
}
for _, col := range columnNames {
if col.name == tree.RegionalByRowRegionDefaultCol &&
col.nullable {
og.expectedExecErrors.add(pgcode.InvalidTableDefinition)
}
}
}

return ret, nil
},
}
Expand Down Expand Up @@ -785,13 +805,33 @@ func (og *operationGenerator) createIndex(tx *pgx.Tx) (string, error) {
IfNotExists: og.randIntn(2) == 0, // 50% IF NOT EXISTS
}

regionColumn := ""
tableIsRegionalByRow, err := tableIsRegionalByRow(tx, tableName)
if err != nil {
return "", err
}
if tableIsRegionalByRow {
regionColumn, err = getRegionColumn(tx, tableName)
if err != nil {
return "", err
}
}

// Define columns on which to create an index. Check for types which cannot be indexed.
duplicateRegionColumn := false
nonIndexableType := false
def.Columns = make(tree.IndexElemList, 1+og.randIntn(len(columnNames)))
for i := range def.Columns {
def.Columns[i].Column = tree.Name(columnNames[i].name)
def.Columns[i].Direction = tree.Direction(og.randIntn(1 + int(tree.Descending)))

// When creating an index, the column being used as the region column
// for a REGIONAL BY ROW table can only be included in indexes as the
// first column. If it's not the first column, we need to add an error
// below.
if columnNames[i].name == regionColumn && i != 0 {
duplicateRegionColumn = true
}
if def.Inverted {
// We can have an inverted index on a set of columns if the last column
// is an inverted indexable type and the preceding columns are not.
Expand All @@ -811,11 +851,18 @@ func (og *operationGenerator) createIndex(tx *pgx.Tx) (string, error) {
// as stored columns.
duplicateStore := false
virtualComputedStored := false
regionColStored := false
columnNames = columnNames[len(def.Columns):]
if n := len(columnNames); n > 0 {
def.Storing = make(tree.NameList, og.randIntn(1+n))
for i := range def.Storing {
def.Storing[i] = tree.Name(columnNames[i].name)

// The region column can not be stored.
if tableIsRegionalByRow && columnNames[i].name == regionColumn {
regionColStored = true
}

// Virtual computed columns are not allowed to be indexed
if columnNames[i].generated && !virtualComputedStored {
isStored, err := columnIsStoredComputed(tx, tableName, columnNames[i].name)
Expand Down Expand Up @@ -861,10 +908,6 @@ func (og *operationGenerator) createIndex(tx *pgx.Tx) (string, error) {
if err != nil {
return "", err
}
tableIsRegionalByRow, err := tableIsRegionalByRow(tx, tableName)
if err != nil {
return "", err
}
if databaseHasRegionChange && tableIsRegionalByRow {
og.expectedExecErrors.add(pgcode.ObjectNotInPrerequisiteState)
}
Expand All @@ -885,6 +928,8 @@ func (og *operationGenerator) createIndex(tx *pgx.Tx) (string, error) {
{code: pgcode.UniqueViolation, condition: !uniqueViolationWillNotOccur},
{code: pgcode.DuplicateColumn, condition: duplicateStore},
{code: pgcode.FeatureNotSupported, condition: nonIndexableType},
{code: pgcode.FeatureNotSupported, condition: regionColStored},
{code: pgcode.FeatureNotSupported, condition: duplicateRegionColumn},
{code: pgcode.Uncategorized, condition: virtualComputedStored},
{code: pgcode.FeatureNotSupported, condition: hasAlterPKSchemaChange},
}.add(og.expectedExecErrors)
Expand Down Expand Up @@ -1376,6 +1421,16 @@ func (og *operationGenerator) dropColumnNotNull(tx *pgx.Tx) (string, error) {
if !columnExists {
og.expectedExecErrors.add(pgcode.UndefinedColumn)
}

hasAlterPKSchemaChange, err := tableHasOngoingAlterPKSchemaChanges(tx, tableName)
if err != nil {
return "", err
}
if hasAlterPKSchemaChange {
// Possible timing hole. Don't issue this schema change with a
// background PK change in progress. Tracked with #66663.
return `SELECT 'avoiding timing hole'`, nil
}
return fmt.Sprintf(`ALTER TABLE %s ALTER COLUMN "%s" DROP NOT NULL`, tableName, columnName), nil
}

Expand Down Expand Up @@ -1930,6 +1985,16 @@ func (og *operationGenerator) setColumnNotNull(tx *pgx.Tx) (string, error) {
}
}

hasPKSchemaChanges, err := tableHasOngoingAlterPKSchemaChanges(tx, tableName)
if err != nil {
return "", err
}
if hasPKSchemaChanges {
// Possible timing hole. Don't issue this schema change with a
// background PK change in progress. Tracked with #66663.
return `SELECT 'avoiding timing hole'`, nil
}

return fmt.Sprintf(`ALTER TABLE %s ALTER COLUMN "%s" SET NOT NULL`, tableName, columnName), nil
}

Expand All @@ -1939,13 +2004,15 @@ func (og *operationGenerator) setColumnType(tx *pgx.Tx) (string, error) {
return "", err
}

const setSessionVariableString = `SET enable_experimental_alter_column_type_general = true;`

tableExists, err := tableExists(tx, tableName)
if err != nil {
return "", err
}
if !tableExists {
og.expectedExecErrors.add(pgcode.UndefinedTable)
return fmt.Sprintf(`ALTER TABLE %s ALTER COLUMN IrrelevantColumnName SET DATA TYPE IrrelevantDataType`, tableName), nil
return fmt.Sprintf(`%s ALTER TABLE %s ALTER COLUMN IrrelevantColumnName SET DATA TYPE IrrelevantDataType`, setSessionVariableString, tableName), nil
}

columnForTypeChange, err := og.randColumnWithMeta(tx, *tableName, og.pctExisting(true))
Expand All @@ -1959,8 +2026,8 @@ func (og *operationGenerator) setColumnType(tx *pgx.Tx) (string, error) {
}
if !columnExists {
og.expectedExecErrors.add(pgcode.UndefinedColumn)
return fmt.Sprintf(`ALTER TABLE %s ALTER COLUMN "%s" SET DATA TYPE IrrelevantTypeName`,
tableName, columnForTypeChange.name), nil
return fmt.Sprintf(`%s ALTER TABLE %s ALTER COLUMN "%s" SET DATA TYPE IrrelevantTypeName`,
setSessionVariableString, tableName, columnForTypeChange.name), nil
}

newTypeName, newType, err := og.randType(tx, og.pctExisting(true))
Expand All @@ -1974,6 +2041,8 @@ func (og *operationGenerator) setColumnType(tx *pgx.Tx) (string, error) {
}

if newType != nil {
// Ignoring the error here intentionally, as we want to carry on with
// the operation and not fail it prematurely.
kind, _ := schemachange.ClassifyConversion(context.Background(), columnForTypeChange.typ, newType)
codesWithConditions{
{code: pgcode.CannotCoerce, condition: kind == schemachange.ColumnConversionImpossible},
Expand All @@ -1986,8 +2055,8 @@ func (og *operationGenerator) setColumnType(tx *pgx.Tx) (string, error) {
{code: pgcode.DependentObjectsStillExist, condition: columnHasDependencies},
}.add(og.expectedExecErrors)

return fmt.Sprintf(`ALTER TABLE %s ALTER COLUMN "%s" SET DATA TYPE %s`,
tableName, columnForTypeChange.name, newTypeName.SQLString()), nil
return fmt.Sprintf(`%s ALTER TABLE %s ALTER COLUMN "%s" SET DATA TYPE %s`,
setSessionVariableString, tableName, columnForTypeChange.name, newTypeName.SQLString()), nil
}

func (og *operationGenerator) survive(tx *pgx.Tx) (string, error) {
Expand Down

0 comments on commit 46baa75

Please sign in to comment.