diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel
index 8afa3cf93faf..4dcf79ad2754 100644
--- a/pkg/BUILD.bazel
+++ b/pkg/BUILD.bazel
@@ -343,6 +343,7 @@ ALL_TESTS = [
"//pkg/workload/cli:cli_test",
"//pkg/workload/faker:faker_test",
"//pkg/workload/movr:movr_test",
+ "//pkg/workload/schemachange:schemachange_test",
"//pkg/workload/tpcc:tpcc_test",
"//pkg/workload/workloadimpl:workloadimpl_test",
"//pkg/workload/workloadsql:workloadsql_test",
diff --git a/pkg/sql/catalog/lease/helpers_test.go b/pkg/sql/catalog/lease/helpers_test.go
index d98746ed21b0..c728186d4e67 100644
--- a/pkg/sql/catalog/lease/helpers_test.go
+++ b/pkg/sql/catalog/lease/helpers_test.go
@@ -269,7 +269,8 @@ func (m *Manager) Publish(
updates := func(_ *kv.Txn, descs map[descpb.ID]catalog.MutableDescriptor) error {
desc, ok := descs[id]
if !ok {
- return errors.AssertionFailedf("required descriptor with ID %d not provided to update closure", id)
+ return errors.AssertionFailedf(
+ "required descriptor with ID %d not provided to update closure", id)
}
return update(desc)
}
diff --git a/pkg/sql/drop_view.go b/pkg/sql/drop_view.go
index 3c1b605de4d3..1a1ec77a1933 100644
--- a/pkg/sql/drop_view.go
+++ b/pkg/sql/drop_view.go
@@ -196,7 +196,7 @@ func (p *planner) dropViewImpl(
dependencyDesc, err := p.Descriptors().GetMutableTableVersionByID(ctx, depID, p.txn)
if err != nil {
return cascadeDroppedViews,
- errors.Errorf("error resolving dependency relation ID %d: %v", depID, err)
+ errors.Wrapf(err, "error resolving dependency relation ID %d", depID)
}
// The dependency is also being deleted, so we don't have to remove the
// references.
diff --git a/pkg/workload/schemachange/BUILD.bazel b/pkg/workload/schemachange/BUILD.bazel
index ad6895bb4472..79a7fd5c05d2 100644
--- a/pkg/workload/schemachange/BUILD.bazel
+++ b/pkg/workload/schemachange/BUILD.bazel
@@ -1,4 +1,4 @@
-load("@io_bazel_rules_go//go:def.bzl", "go_library")
+load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
load("//build:STRINGER.bzl", "stringer")
go_library(
@@ -43,3 +43,28 @@ stringer(
src = "operation_generator.go",
typ = "opType",
)
+
+go_test(
+ name = "schemachange_test",
+ srcs = [
+ "main_test.go",
+ "schema_change_external_test.go",
+ ],
+ deps = [
+ "//pkg/base",
+ "//pkg/ccl",
+ "//pkg/security",
+ "//pkg/security/securitytest",
+ "//pkg/server",
+ "//pkg/testutils/serverutils",
+ "//pkg/testutils/skip",
+ "//pkg/testutils/sqlutils",
+ "//pkg/testutils/testcluster",
+ "//pkg/util/leaktest",
+ "//pkg/util/randutil",
+ "//pkg/workload",
+ "//pkg/workload/histogram",
+ "@com_github_stretchr_testify//require",
+ "@org_golang_x_sync//errgroup",
+ ],
+)
diff --git a/pkg/workload/schemachange/error_screening.go b/pkg/workload/schemachange/error_screening.go
index f1f3dc73df8c..ac1422c87fc1 100644
--- a/pkg/workload/schemachange/error_screening.go
+++ b/pkg/workload/schemachange/error_screening.go
@@ -15,6 +15,7 @@ import (
"strings"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
+ "github.com/cockroachdb/errors"
"github.com/jackc/pgx"
)
@@ -74,7 +75,7 @@ func tableHasRows(tx *pgx.Tx, tableName *tree.TableName) (bool, error) {
func scanBool(tx *pgx.Tx, query string, args ...interface{}) (b bool, err error) {
err = tx.QueryRow(query, args...).Scan(&b)
- return b, err
+ return b, errors.Wrapf(err, "scanBool: %q %q", query, args)
}
func schemaExists(tx *pgx.Tx, schemaName string) (bool, error) {
@@ -98,6 +99,8 @@ func tableHasDependencies(tx *pgx.Tx, tableName *tree.TableName) (bool, error) {
ns.oid = c.relnamespace
WHERE c.relname = $1 AND ns.nspname = $2
)
+ AND fd.descriptor_id != fd.dependedonby_id
+ AND fd.dependedonby_type != 'sequence'
)
`, tableName.Object(), tableName.Schema())
}
@@ -133,6 +136,7 @@ func columnIsDependedOn(tx *pgx.Tx, tableName *tree.TableName, columnName string
AS fd
WHERE fd.descriptor_id
= $1::REGCLASS
+ AND fd.dependedonby_type != 'sequence'
)
UNION (
SELECT unnest(confkey) AS column_id
@@ -306,7 +310,7 @@ func violatesUniqueConstraintsHelper(
func scanStringArrayRows(tx *pgx.Tx, query string, args ...interface{}) ([][]string, error) {
rows, err := tx.Query(query, args...)
if err != nil {
- return nil, err
+ return nil, errors.Wrapf(err, "scanStringArrayRows: %q %q", query, args)
}
defer rows.Close()
@@ -315,12 +319,12 @@ func scanStringArrayRows(tx *pgx.Tx, query string, args ...interface{}) ([][]str
var columnNames []string
err := rows.Scan(&columnNames)
if err != nil {
- return nil, err
+ return nil, errors.Wrapf(err, "scan: %q, args %q, scanArgs %q", query, columnNames, args)
}
results = append(results, columnNames)
}
- return results, err
+ return results, nil
}
func indexExists(tx *pgx.Tx, tableName *tree.TableName, indexName string) (bool, error) {
@@ -366,7 +370,7 @@ func columnsStoredInPrimaryIdx(
func scanStringArray(tx *pgx.Tx, query string, args ...interface{}) (b []string, err error) {
err = tx.QueryRow(query, args...).Scan(&b)
- return b, err
+ return b, errors.Wrapf(err, "scanStringArray %q %q", query, args)
}
// canApplyUniqueConstraint checks if the rows in a table are unique with respect
@@ -470,17 +474,36 @@ func constraintIsUnique(
`, tableName.String(), constraintName))
}
+func columnIsStoredComputed(
+ tx *pgx.Tx, tableName *tree.TableName, columnName string,
+) (bool, error) {
+ // Note that we COALESCE because the column may not exist.
+ return scanBool(tx, `
+SELECT COALESCE(
+ (
+ SELECT attgenerated
+ FROM pg_catalog.pg_attribute
+ WHERE attrelid = $1:::REGCLASS AND attname = $2
+ )
+ = 's',
+ false
+ );
+`, tableName.String(), columnName)
+}
+
func columnIsComputed(tx *pgx.Tx, tableName *tree.TableName, columnName string) (bool, error) {
+ // Note that we COALESCE because the column may not exist.
return scanBool(tx, `
- SELECT (
- SELECT is_generated
- FROM information_schema.columns
- WHERE table_schema = $1
- AND table_name = $2
- AND column_name = $3
- )
- = 'YES'
-`, tableName.Schema(), tableName.Object(), columnName)
+SELECT COALESCE(
+ (
+ SELECT attgenerated
+ FROM pg_catalog.pg_attribute
+ WHERE attrelid = $1:::REGCLASS AND attname = $2
+ )
+ != '',
+ false
+ );
+`, tableName.String(), columnName)
}
func constraintExists(tx *pgx.Tx, constraintName string) (bool, error) {
@@ -605,3 +628,145 @@ func violatesFkConstraintsHelper(
)
`, parentTableSchema, parentTableName, parentColumn, childValue))
}
+
+func columnIsInDroppingIndex(
+ tx *pgx.Tx, tableName *tree.TableName, columnName string,
+) (bool, error) {
+ return scanBool(tx, `
+SELECT EXISTS(
+ SELECT index_id
+ FROM (
+ SELECT DISTINCT index_id
+ FROM crdb_internal.index_columns
+ WHERE descriptor_id = $1::REGCLASS AND column_name = $2
+ ) AS indexes
+ JOIN crdb_internal.schema_changes AS sc ON sc.target_id
+ = indexes.index_id
+ AND table_id = $1::REGCLASS
+ AND type = 'INDEX'
+ AND direction = 'DROP'
+ );
+`, tableName.String(), columnName)
+}
+
+// A pair of CTE definitions that expect the first argument to be a table name.
+const descriptorsAndConstraintMutationsCTE = `descriptors AS (
+ SELECT crdb_internal.pb_to_json(
+ 'cockroach.sql.sqlbase.Descriptor',
+ descriptor
+ )->'table' AS d
+ FROM system.descriptor
+ WHERE id = $1::REGCLASS
+ ),
+ constraint_mutations AS (
+ SELECT mut
+ FROM (
+ SELECT json_array_elements(
+ d->'mutations'
+ ) AS mut
+ FROM descriptors
+ )
+ WHERE (mut->'constraint') IS NOT NULL
+ )`
+
+func constraintInDroppingState(
+ tx *pgx.Tx, tableName *tree.TableName, constraintName string,
+) (bool, error) {
+ // TODO(ajwerner): Figure out how to plumb the column name into this query.
+ return scanBool(tx, `
+ WITH `+descriptorsAndConstraintMutationsCTE+`
+SELECT true
+ IN (
+ SELECT (t.f).value @> json_set('{"validity": "Dropping"}', ARRAY['name'], to_json($2:::STRING))
+ FROM (
+ SELECT json_each(mut->'constraint') AS f
+ FROM constraint_mutations
+ ) AS t
+ );
+`, tableName.String(), constraintName)
+}
+
+func columnNotNullConstraintInMutation(
+ tx *pgx.Tx, tableName *tree.TableName, columnName string,
+) (bool, error) {
+ return scanBool(tx, `
+ WITH `+descriptorsAndConstraintMutationsCTE+`,
+ col AS (
+ SELECT (c->>'id')::INT8 AS id
+ FROM (
+ SELECT json_array_elements(d->'columns') AS c
+ FROM descriptors
+ )
+ WHERE c->>'name' = $2
+ )
+SELECT EXISTS(
+ SELECT *
+ FROM constraint_mutations
+ JOIN col ON mut->'constraint'->>'constraintType' = 'NOT_NULL'
+ AND (mut->'constraint'->>'notNullColumn')::INT8 = id
+ );
+`, tableName.String(), columnName)
+}
+
+func schemaContainsTypesWithCrossSchemaReferences(tx *pgx.Tx, schemaName string) (bool, error) {
+ return scanBool(tx, `
+ WITH database_id AS (
+ SELECT id
+ FROM system.namespace
+ WHERE "parentID" = 0
+ AND "parentSchemaID" = 0
+ AND name = current_database()
+ ),
+ schema_id AS (
+ SELECT nsp.id
+ FROM system.namespace AS nsp
+ JOIN database_id ON "parentID" = database_id.id
+ AND "parentSchemaID" = 0
+ AND name = $1
+ ),
+ descriptor_ids AS (
+ SELECT nsp.id
+ FROM system.namespace AS nsp,
+ schema_id,
+ database_id
+ WHERE nsp."parentID" = database_id.id
+ AND nsp."parentSchemaID" = schema_id.id
+ ),
+ descriptors AS (
+ SELECT crdb_internal.pb_to_json(
+ 'cockroach.sql.sqlbase.Descriptor',
+ descriptor
+ ) AS descriptor
+ FROM system.descriptor AS descriptors
+ JOIN descriptor_ids ON descriptors.id
+ = descriptor_ids.id
+ ),
+ types AS (
+ SELECT descriptor
+ FROM descriptors
+ WHERE (descriptor->'type') IS NOT NULL
+ ),
+ table_references AS (
+ SELECT json_array_elements(
+ descriptor->'table'->'dependedOnBy'
+ ) AS ref
+ FROM descriptors
+ WHERE (descriptor->'table') IS NOT NULL
+ ),
+ dependent AS (
+ SELECT (ref->>'id')::INT8 AS id FROM table_references
+ ),
+ referenced_descriptors AS (
+ SELECT json_array_elements_text(
+ descriptor->'type'->'referencingDescriptorIds'
+ )::INT8 AS id
+ FROM types
+ )
+SELECT EXISTS(
+ SELECT *
+ FROM system.namespace
+ WHERE id IN (SELECT id FROM referenced_descriptors)
+ AND "parentSchemaID" NOT IN (SELECT id FROM schema_id)
+ AND id NOT IN (SELECT id FROM dependent)
+ );`, schemaName)
+}
diff --git a/pkg/workload/schemachange/main_test.go b/pkg/workload/schemachange/main_test.go
new file mode 100644
index 000000000000..5b698407e14d
--- /dev/null
+++ b/pkg/workload/schemachange/main_test.go
@@ -0,0 +1,34 @@
+// Copyright 2020 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 schemachange_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())
+}
+
+//go:generate ../../util/leaktest/add-leaktest.sh *_test.go
diff --git a/pkg/workload/schemachange/operation_generator.go b/pkg/workload/schemachange/operation_generator.go
index 96b88deca705..0cdc05caf043 100644
--- a/pkg/workload/schemachange/operation_generator.go
+++ b/pkg/workload/schemachange/operation_generator.go
@@ -56,9 +56,9 @@ type operationGenerator struct {
// are added to expectedCommitErrors.
candidateExpectedCommitErrors errorCodeSet
- // opsInTxn is a list of previous ops in the current transaction implemented
- // as a map for fast lookups.
- opsInTxn map[opType]bool
+ // opsInTxn is a list of previous ops in the current transaction to check
+ // for DDLs after writes.
+ opsInTxn []opType
}
func makeOperationGenerator(params *operationGeneratorParams) *operationGenerator {
@@ -67,7 +67,6 @@ func makeOperationGenerator(params *operationGeneratorParams) *operationGenerato
expectedExecErrors: makeExpectedErrorSet(),
expectedCommitErrors: makeExpectedErrorSet(),
candidateExpectedCommitErrors: makeExpectedErrorSet(),
- opsInTxn: map[opType]bool{},
}
}
@@ -80,15 +79,18 @@ func (og *operationGenerator) resetOpState() {
// Reset internal state used per transaction
func (og *operationGenerator) resetTxnState() {
og.expectedCommitErrors.reset()
-
- for k := range og.opsInTxn {
- delete(og.opsInTxn, k)
- }
+ og.opsInTxn = nil
}
//go:generate stringer -type=opType
type opType int
+// isDDL returns true if the operation mutates the system config span and thus
+// cannot follow a write elsewhere.
+func (ot opType) isDDL() bool {
+ return ot != insertRow && ot != validate
+}
+
const (
addColumn opType = iota // ALTER TABLE
ADD [COLUMN]
addConstraint // ALTER TABLE ADD CONSTRAINT
@@ -175,8 +177,8 @@ func init() {
var opWeights = []int{
addColumn: 1,
addConstraint: 0, // TODO(spaskob): unimplemented
- addForeignKeyConstraint: 1,
- addUniqueConstraint: 1,
+ addForeignKeyConstraint: 0,
+ addUniqueConstraint: 0,
createIndex: 1,
createSequence: 1,
createTable: 1,
@@ -184,7 +186,7 @@ var opWeights = []int{
createView: 1,
createEnum: 1,
createSchema: 1,
- dropColumn: 1,
+ dropColumn: 0,
dropColumnDefault: 1,
dropColumnNotNull: 1,
dropColumnStored: 1,
@@ -202,7 +204,7 @@ var opWeights = []int{
setColumnDefault: 1,
setColumnNotNull: 1,
setColumnType: 1,
- insertRow: 1,
+ insertRow: 0,
validate: 2, // validate twice more often
}
@@ -211,52 +213,47 @@ var opWeights = []int{
// change constructed. Constructing a random schema change may require a few
// stochastic attempts and if verbosity is >= 2 the unsuccessful attempts are
// recorded in `log` to help with debugging of the workload.
-func (og *operationGenerator) randOp(tx *pgx.Tx) (stmt string, noops []string, err error) {
- savepointCount := 0
+func (og *operationGenerator) randOp(tx *pgx.Tx) (stmt string, err error) {
+
for {
op := opType(og.params.ops.Int())
og.resetOpState()
+ stmt, err = opFuncs[op](og, tx)
+ if err != nil {
+ if errors.Is(err, pgx.ErrNoRows) {
+ continue
+ }
- // Savepoints are used to prevent an infinite loop from occurring as a result of the transaction
- // becoming aborted by an op function. If a transaction becomes aborted, no op function will be able
- // to complete without error because all op functions rely on the passed transaction to introspect the database.
- // With sufficient verbosity, any failed op functions will be logged as NOOPs below for debugging purposes.
- //
- // For example, functions such as getTableColumns() can abort the transaction if called with
- // a non existing table. getTableColumns() is used by op functions such as createIndex().
- // Op functions normally ensure that a table exists by checking system tables before passing the
- // table to getTableColumns(). However, due to bugs such as #57494, system tables
- // may be inaccurate and can cause getTableColumns() to be called with a non-existing table. This
- // will result in an aborted transaction. Wrapping calls to op functions with savepoints will protect
- // the transaction from being aborted by spurious errors such as in the above example.
- savepointCount++
- if _, err := tx.Exec(fmt.Sprintf(`SAVEPOINT s%d`, savepointCount)); err != nil {
- return "", noops, err
+ return "", err
}
+ // Screen for schema change after write in the same transaction.
+ og.checkIfOpViolatesDDLAfterWrite(op)
- stmt, err := opFuncs[op](og, tx)
-
- if err == nil {
- // Screen for schema change after write in the same transaction.
- if op != insertRow && op != validate {
- if _, previous := og.opsInTxn[insertRow]; previous {
- og.expectedExecErrors.add(pgcode.FeatureNotSupported)
- }
- }
+ // Add candidateExpectedCommitErrors to expectedCommitErrors
+ og.expectedCommitErrors.merge(og.candidateExpectedCommitErrors)
+ break
+ }
- // Add candidateExpectedCommitErrors to expectedCommitErrors
- og.expectedCommitErrors.merge(og.candidateExpectedCommitErrors)
+ return stmt, err
+}
- og.opsInTxn[op] = true
+func (og *operationGenerator) checkIfOpViolatesDDLAfterWrite(ot opType) {
+ if ot.isDDL() && og.haveInsertBeforeAnyDDLs() {
+ og.expectedExecErrors.add(pgcode.FeatureNotSupported)
+ }
+ og.opsInTxn = append(og.opsInTxn, ot)
+}
- return stmt, noops, err
+func (og *operationGenerator) haveInsertBeforeAnyDDLs() bool {
+ for _, ot := range og.opsInTxn {
+ if ot.isDDL() {
+ break
}
- if _, err := tx.Exec(fmt.Sprintf(`ROLLBACK TO SAVEPOINT s%d`, savepointCount)); err != nil {
- return "", noops, err
+ if ot == insertRow {
+ return true
}
-
- noops = append(noops, fmt.Sprintf("NOOP: %s -> %v", op, err))
}
+ return false
}
func (og *operationGenerator) addColumn(tx *pgx.Tx) (string, error) {
@@ -299,15 +296,18 @@ func (og *operationGenerator) addColumn(tx *pgx.Tx) (string, error) {
if err != nil {
return "", err
}
- tableHasRows, err := tableHasRows(tx, tableName)
- if err != nil {
- return "", err
+ var hasRows bool
+ if tableExists {
+ hasRows, err = tableHasRows(tx, tableName)
+ if err != nil {
+ return "", err
+ }
}
codesWithConditions{
{code: pgcode.DuplicateColumn, condition: columnExistsOnTable},
{code: pgcode.UndefinedObject, condition: !typeExists},
- {code: pgcode.NotNullViolation, condition: tableHasRows && def.Nullable.Nullability == tree.NotNull},
+ {code: pgcode.NotNullViolation, condition: hasRows && def.Nullable.Nullability == tree.NotNull},
}.add(og.expectedExecErrors)
return fmt.Sprintf(`ALTER TABLE %s ADD COLUMN %s`, tableName, tree.Serialize(def)), nil
@@ -501,7 +501,11 @@ func (og *operationGenerator) createIndex(tx *pgx.Tx) (string, error) {
def.Columns[i].Direction = tree.Direction(og.randIntn(1 + int(tree.Descending)))
if def.Inverted {
- if !colinfo.ColumnTypeIsInvertedIndexable(columnNames[i].typ) {
+ // 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.
+ invertedIndexableType := colinfo.ColumnTypeIsInvertedIndexable(columnNames[i].typ)
+ if (invertedIndexableType && i < len(def.Columns)-1) ||
+ (!invertedIndexableType && i == len(def.Columns)-1) {
nonIndexableType = true
}
} else {
@@ -514,11 +518,22 @@ func (og *operationGenerator) createIndex(tx *pgx.Tx) (string, error) {
// If there are extra columns not used in the index, randomly use them
// as stored columns.
duplicateStore := false
+ virtualComputedStored := 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)
+ // Virtual computed columns are not allowed to be indexed
+ if columnNames[i].generated && !virtualComputedStored {
+ isStored, err := columnIsStoredComputed(tx, tableName, columnNames[i].name)
+ if err != nil {
+ return "", err
+ }
+ if !isStored {
+ virtualComputedStored = true
+ }
+ }
}
// If the column is already used in the primary key, then attempting to store
@@ -552,8 +567,6 @@ func (og *operationGenerator) createIndex(tx *pgx.Tx) (string, error) {
{code: pgcode.DuplicateRelation, condition: indexExists},
// Inverted indexes do not support stored columns.
{code: pgcode.InvalidSQLStatementName, condition: len(def.Storing) > 0 && def.Inverted},
- // Inverted indexes do not support indexing more than one column.
- {code: pgcode.InvalidSQLStatementName, condition: len(def.Columns) > 1 && def.Inverted},
// Inverted indexes cannot be unique.
{code: pgcode.InvalidSQLStatementName, condition: def.Unique && def.Inverted},
// If there is data in the table such that a unique index cannot be created,
@@ -563,6 +576,7 @@ 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.Uncategorized, condition: virtualComputedStored},
}.add(og.expectedExecErrors)
}
@@ -617,7 +631,9 @@ func (og *operationGenerator) createSequence(tx *pgx.Tx) (string, error) {
Name: tree.SeqOptOwnedBy,
ColumnItemVal: &tree.ColumnItem{TableName: table.ToUnresolvedObjectName(), ColumnName: "IrrelevantColumnName"}},
)
- og.expectedExecErrors.add(pgcode.UndefinedTable)
+ if !(sequenceExists && ifNotExists) { // IF NOT EXISTS prevents the error
+ og.expectedExecErrors.add(pgcode.UndefinedTable)
+ }
} else {
column, err := og.randColumn(tx, *table, og.pctExisting(true))
if err != nil {
@@ -974,8 +990,13 @@ func (og *operationGenerator) dropColumn(tx *pgx.Tx) (string, error) {
if err != nil {
return "", err
}
+ columnIsInDroppingIndex, err := columnIsInDroppingIndex(tx, tableName, columnName)
+ if err != nil {
+ return "", err
+ }
codesWithConditions{
+ {code: pgcode.ObjectNotInPrerequisiteState, condition: columnIsInDroppingIndex},
{code: pgcode.UndefinedColumn, condition: !columnExists},
{code: pgcode.InvalidColumnReference, condition: colIsPrimaryKey},
{code: pgcode.DependentObjectsStillExist, condition: columnIsDependedOn},
@@ -1070,13 +1091,13 @@ func (og *operationGenerator) dropColumnStored(tx *pgx.Tx) (string, error) {
return "", err
}
- columnIsComputed, err := columnIsComputed(tx, tableName, columnName)
+ columnIsStored, err := columnIsStoredComputed(tx, tableName, columnName)
if err != nil {
return "", err
}
codesWithConditions{
- {code: pgcode.InvalidColumnDefinition, condition: !columnIsComputed},
+ {code: pgcode.InvalidColumnDefinition, condition: !columnIsStored},
{code: pgcode.UndefinedColumn, condition: !columnExists},
}.add(og.expectedExecErrors)
@@ -1125,6 +1146,14 @@ func (og *operationGenerator) dropConstraint(tx *pgx.Tx) (string, error) {
og.expectedExecErrors.add(pgcode.FeatureNotSupported)
}
+ constraintBeingDropped, err := constraintInDroppingState(tx, tableName, constraintName)
+ if err != nil {
+ return "", err
+ }
+ if constraintBeingDropped {
+ og.expectedExecErrors.add(pgcode.FeatureNotSupported)
+ }
+
return fmt.Sprintf(`ALTER TABLE %s DROP CONSTRAINT "%s"`, tableName, constraintName), nil
}
@@ -1558,6 +1587,13 @@ func (og *operationGenerator) setColumnNotNull(tx *pgx.Tx) (string, error) {
if err != nil {
return "", err
}
+ constraintBeingAdded, err := columnNotNullConstraintInMutation(tx, tableName, columnName)
+ if err != nil {
+ return "", err
+ }
+ if constraintBeingAdded {
+ og.expectedExecErrors.add(pgcode.ObjectNotInPrerequisiteState)
+ }
if !columnExists {
og.expectedExecErrors.add(pgcode.UndefinedColumn)
@@ -1660,6 +1696,17 @@ func (og *operationGenerator) insertRow(tx *pgx.Tx) (string, error) {
if err != nil {
return "", errors.Wrapf(err, "error getting table columns for insert row")
}
+
+ // Filter out computed columns.
+ {
+ truncated := cols[:0]
+ for _, c := range cols {
+ if !c.generated {
+ truncated = append(truncated, c)
+ }
+ }
+ cols = truncated
+ }
colNames := []string{}
rows := [][]string{}
for _, col := range cols {
@@ -1690,6 +1737,11 @@ func (og *operationGenerator) insertRow(tx *pgx.Tx) (string, error) {
return "", err
}
+ // TODO(ajwerner): Errors can occur if computed columns are referenced. It's
+ // hard to classify all the ways this can cause problems. One source of
+ // problems is that the expression may overflow the width of a computed column
+ // that has a smaller width than the inputs.
+
codesWithConditions{
{code: pgcode.UniqueViolation, condition: uniqueConstraintViolation},
{code: pgcode.ForeignKeyViolation, condition: foreignKeyViolation},
@@ -1740,21 +1792,25 @@ func (og *operationGenerator) validate(tx *pgx.Tx) (string, error) {
}
type column struct {
- name string
- typ *types.T
- nullable bool
+ name string
+ typ *types.T
+ nullable bool
+ generated bool
}
func (og *operationGenerator) getTableColumns(
tx *pgx.Tx, tableName string, shuffle bool,
) ([]column, error) {
q := fmt.Sprintf(`
- SELECT column_name, data_type, is_nullable
- FROM [SHOW COLUMNS FROM %s]
+SELECT column_name,
+ data_type,
+ is_nullable,
+ generation_expression != '' AS is_generated
+ FROM [SHOW COLUMNS FROM %s];
`, tableName)
rows, err := tx.Query(q)
if err != nil {
- return nil, err
+ return nil, errors.Wrapf(err, "getting table columns from %s", tableName)
}
defer rows.Close()
var typNames []string
@@ -1762,7 +1818,7 @@ func (og *operationGenerator) getTableColumns(
for rows.Next() {
var c column
var typName string
- err := rows.Scan(&c.name, &typName, &c.nullable)
+ err := rows.Scan(&c.name, &typName, &c.nullable, &c.generated)
if err != nil {
return nil, err
}
@@ -1841,7 +1897,7 @@ ORDER BY random()
var col column
var typ string
if err := tx.QueryRow(q).Scan(&col.name, &typ, &col.nullable); err != nil {
- return column{}, err
+ return column{}, errors.Wrapf(err, "randColumnWithMeta: %q", q)
}
var err error
@@ -2190,7 +2246,7 @@ func (og *operationGenerator) randView(
}
q := fmt.Sprintf(`
- SELECT schema_name, table_name
+ SELECT table_name
FROM [SHOW TABLES]
WHERE table_name LIKE 'view%%'
AND schema_name = '%s'
@@ -2344,9 +2400,14 @@ func (og *operationGenerator) dropSchema(tx *pgx.Tx) (string, error) {
if err != nil {
return "", err
}
+ crossReferences, err := schemaContainsTypesWithCrossSchemaReferences(tx, schemaName)
+ if err != nil {
+ return "", err
+ }
codesWithConditions{
{pgcode.UndefinedSchema, !schemaExists},
{pgcode.InvalidSchemaName, schemaName == tree.PublicSchema},
+ {pgcode.FeatureNotSupported, crossReferences},
}.add(og.expectedExecErrors)
return fmt.Sprintf(`DROP SCHEMA "%s" CASCADE`, schemaName), nil
@@ -2386,7 +2447,7 @@ func (og *operationGenerator) newUniqueSeqNum() int64 {
func (og *operationGenerator) typeFromTypeName(tx *pgx.Tx, typeName string) (*types.T, error) {
stmt, err := parser.ParseOne(fmt.Sprintf("SELECT 'placeholder'::%s", typeName))
if err != nil {
- return nil, err
+ return nil, errors.Wrapf(err, "typeFromTypeName: %s", typeName)
}
typ, err := tree.ResolveType(
context.Background(),
@@ -2394,7 +2455,7 @@ func (og *operationGenerator) typeFromTypeName(tx *pgx.Tx, typeName string) (*ty
&txTypeResolver{tx: tx},
)
if err != nil {
- return nil, err
+ return nil, errors.Wrapf(err, "ResolveType: %v", typeName)
}
return typ, nil
}
diff --git a/pkg/workload/schemachange/schema_change_external_test.go b/pkg/workload/schemachange/schema_change_external_test.go
new file mode 100644
index 000000000000..d96396875622
--- /dev/null
+++ b/pkg/workload/schemachange/schema_change_external_test.go
@@ -0,0 +1,110 @@
+// Copyright 2020 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 schemachange_test
+
+import (
+ "context"
+ gosql "database/sql"
+ "fmt"
+ "io/ioutil"
+ "net/url"
+ "strconv"
+ "testing"
+ "time"
+
+ "github.com/cockroachdb/cockroach/pkg/base"
+ _ "github.com/cockroachdb/cockroach/pkg/ccl"
+ "github.com/cockroachdb/cockroach/pkg/testutils/skip"
+ "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
+ "github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
+ "github.com/cockroachdb/cockroach/pkg/util/leaktest"
+ "github.com/cockroachdb/cockroach/pkg/workload"
+ "github.com/cockroachdb/cockroach/pkg/workload/histogram"
+ "github.com/stretchr/testify/require"
+ "golang.org/x/sync/errgroup"
+)
+
+func TestWorkload(t *testing.T) {
+ defer leaktest.AfterTest(t)()
+
+ skip.UnderStressRace(t, "times out")
+
+ dir, err := ioutil.TempDir("", t.Name())
+ require.NoError(t, err)
+ ctx := context.Background()
+ tc := testcluster.StartTestCluster(t, 3, base.TestClusterArgs{
+ ServerArgs: base.TestServerArgs{
+ ExternalIODir: dir,
+ },
+ })
+
+ defer tc.Stopper().Stop(ctx)
+ m, err := workload.Get("schemachange")
+ require.NoError(t, err)
+ wl := m.New().(interface {
+ workload.Opser
+ workload.Flagser
+ })
+ tdb := sqlutils.MakeSQLRunner(tc.ServerConn(0))
+ reg := histogram.NewRegistry(20 * time.Second)
+ tdb.Exec(t, "CREATE USER testuser")
+ tdb.Exec(t, "CREATE DATABASE schemachange")
+ tdb.Exec(t, "GRANT admin TO testuser")
+ tdb.Exec(t, "SET CLUSTER SETTING sql.trace.log_statement_execute = true")
+
+ // Grab a backup and also print the namespace and descriptor tables upon
+ // failure.
+ // It's not clear how helpful this actually is but it doesn't hurt.
+ printRows := func(rows *gosql.Rows) {
+ t.Helper()
+ mat, err := sqlutils.RowsToStrMatrix(rows)
+ require.NoError(t, err)
+ fmt.Printf("rows:\n%s", sqlutils.MatrixToStr(mat))
+ }
+ defer func() {
+ if !t.Failed() {
+ return
+ }
+ printRows(tdb.Query(t, "SELECT id, encode(descriptor, 'hex') FROM system.descriptor"))
+ printRows(tdb.Query(t, "SELECT * FROM system.namespace"))
+ tdb.Exec(t, "BACKUP DATABASE schemachange TO 'nodelocal://0/backup'")
+ t.Logf("backup in %s", dir)
+ }()
+
+ pgURL, cleanup := sqlutils.PGUrl(t, tc.Server(0).ServingSQLAddr(), t.Name(), url.User("testuser"))
+ defer cleanup()
+
+ const concurrency = 2
+ require.NoError(t, wl.Flags().Parse([]string{
+ "--concurrency", strconv.Itoa(concurrency),
+ "--verbose", "2",
+ }))
+
+ ql, err := wl.Ops(ctx, []string{pgURL.String()}, reg)
+ require.NoError(t, err)
+
+ const N = 100
+ workerFn := func(ctx context.Context, fn func(ctx context.Context) error) func() error {
+ return func() error {
+ for i := 0; i < N; i++ {
+ if err := fn(ctx); err != nil || ctx.Err() != nil {
+ return err
+ }
+ }
+ return nil
+ }
+ }
+ g, gCtx := errgroup.WithContext(ctx)
+ for i := 0; i < concurrency; i++ {
+ g.Go(workerFn(gCtx, ql.WorkerFns[i]))
+ }
+ require.NoError(t, g.Wait())
+}
diff --git a/pkg/workload/schemachange/schemachange.go b/pkg/workload/schemachange/schemachange.go
index 9e3c98ba1866..7f1d775e8f6e 100644
--- a/pkg/workload/schemachange/schemachange.go
+++ b/pkg/workload/schemachange/schemachange.go
@@ -340,22 +340,18 @@ func (w *schemaChangeWorker) runInTxn(tx *pgx.Tx) error {
break
}
- op, noops, err := w.opGen.randOp(tx)
- if w.logger.verbose >= 2 {
- for _, noop := range noops {
- w.logger.writeLog(noop)
- }
- }
-
- w.logger.addExpectedErrors(w.opGen.expectedExecErrors, w.opGen.expectedCommitErrors)
+ op, err := w.opGen.randOp(tx)
- if err != nil {
+ if pgErr := (pgx.PgError{}); errors.As(err, &pgErr) && pgcode.MakeCode(pgErr.Code) == pgcode.SerializationFailure {
+ return errors.Mark(err, errRunInTxnRbkSentinel)
+ } else if err != nil {
return errors.Mark(
errors.Wrap(err, "***UNEXPECTED ERROR; Failed to generate a random operation"),
errRunInTxnFatalSentinel,
)
}
+ w.logger.addExpectedErrors(w.opGen.expectedExecErrors, w.opGen.expectedCommitErrors)
w.logger.writeLog(op)
if !w.dryRun {
start := timeutil.Now()
@@ -412,7 +408,7 @@ func (w *schemaChangeWorker) run(_ context.Context) error {
}
// Release log entry locks if holding all.
- w.releaseLocksIfHeld()
+ defer w.releaseLocksIfHeld()
// Run between 1 and maxOpsPerWorker schema change operations.
start := timeutil.Now()