crdb_internal.scan(span: bytes[]) → tuple{bytes AS key, bytes AS value} | Returns the raw keys and values from the specified span
+crdb_internal.scan(span: bytes[]) → tuple{bytes AS key, bytes AS value, string AS ts} | Returns the raw keys and values from the specified span
| Stable |
-crdb_internal.scan(start_key: bytes, end_key: bytes) → tuple{bytes AS key, bytes AS value} | Returns the raw keys and values from the specified span
+crdb_internal.scan(start_key: bytes, end_key: bytes) → tuple{bytes AS key, bytes AS value, string AS ts} | Returns the raw keys and values with their timestamp from the specified span
| Stable |
crdb_internal.testing_callback(name: string) → int | For internal CRDB testing only. The function calls a callback identified by name registered with the server by the test.
| Volatile |
@@ -3081,7 +3081,7 @@ active for the current transaction.
| Volatile |
crdb_internal.lease_holder(key: bytes) → int | This function is used to fetch the leaseholder corresponding to a request key
| Volatile |
-crdb_internal.list_sql_keys_in_range(range_id: int) → tuple{string AS key, string AS value} | Returns all SQL K/V pairs within the requested range.
+crdb_internal.list_sql_keys_in_range(range_id: int) → tuple{string AS key, string AS value, string AS ts} | Returns all SQL K/V pairs within the requested range.
| Volatile |
crdb_internal.locality_value(key: string) → string | Returns the value of the specified locality key.
| Stable |
diff --git a/pkg/sql/sem/builtins/fixed_oids.go b/pkg/sql/sem/builtins/fixed_oids.go
index e99911d76694..bb2f7b3abfdb 100644
--- a/pkg/sql/sem/builtins/fixed_oids.go
+++ b/pkg/sql/sem/builtins/fixed_oids.go
@@ -424,7 +424,7 @@ var builtinOidsBySignature = map[string]oid.Oid{
`crdb_internal.kv_set_queue_active(queue_name: string, active: bool) -> bool`: 1383,
`crdb_internal.kv_set_queue_active(queue_name: string, active: bool, store_id: int) -> bool`: 1384,
`crdb_internal.lease_holder(key: bytes) -> int`: 1316,
- `crdb_internal.list_sql_keys_in_range(range_id: int) -> tuple{string AS key, string AS value}`: 348,
+ `crdb_internal.list_sql_keys_in_range(range_id: int) -> tuple{string AS key, string AS value, string AS ts}`: 348,
`crdb_internal.locality_value(key: string) -> string`: 1292,
`crdb_internal.merge_statement_stats(input: jsonb[]) -> jsonb`: 1277,
`crdb_internal.merge_stats_metadata(input: jsonb[]) -> jsonb`: 1279,
@@ -466,8 +466,8 @@ var builtinOidsBySignature = map[string]oid.Oid{
`crdb_internal.revalidate_unique_constraints_in_table(table_name: string) -> void`: 1380,
`crdb_internal.round_decimal_values(val: decimal, scale: int) -> decimal`: 1342,
`crdb_internal.round_decimal_values(val: decimal[], scale: int) -> decimal[]`: 1343,
- `crdb_internal.scan(start_key: bytes, end_key: bytes) -> tuple{bytes AS key, bytes AS value}`: 315,
- `crdb_internal.scan(span: bytes[]) -> tuple{bytes AS key, bytes AS value}`: 316,
+ `crdb_internal.scan(start_key: bytes, end_key: bytes) -> tuple{bytes AS key, bytes AS value, string AS ts}`: 315,
+ `crdb_internal.scan(span: bytes[]) -> tuple{bytes AS key, bytes AS value, string AS ts}`: 316,
`crdb_internal.schedule_sql_stats_compaction() -> bool`: 1377,
`crdb_internal.serialize_session() -> bytes`: 1370,
`crdb_internal.set_compaction_concurrency(node_id: int, store_id: int, compaction_concurrency: int) -> bool`: 1389,
diff --git a/pkg/sql/sem/builtins/generator_builtins.go b/pkg/sql/sem/builtins/generator_builtins.go
index af5509acd9f1..d149d441315b 100644
--- a/pkg/sql/sem/builtins/generator_builtins.go
+++ b/pkg/sql/sem/builtins/generator_builtins.go
@@ -136,7 +136,7 @@ var generators = map[string]builtinDefinition{
EndKey: endKey,
}), nil
},
- "Returns the raw keys and values from the specified span",
+ "Returns the raw keys and values with their timestamp from the specified span",
volatility.Stable,
),
makeGeneratorOverload(
@@ -2065,13 +2065,13 @@ const spanKeyIteratorChunkBytes = 8 << 20 // 8MiB
var rangeKeyIteratorType = types.MakeLabeledTuple(
// TODO(rohany): These could be bytes if we don't want to display the
// prettified versions of the key and value.
- []*types.T{types.String, types.String},
- []string{"key", "value"},
+ []*types.T{types.String, types.String, types.String},
+ []string{"key", "value", "ts"},
)
var spanKeyIteratorType = types.MakeLabeledTuple(
- []*types.T{types.Bytes, types.Bytes},
- []string{"key", "value"},
+ []*types.T{types.Bytes, types.Bytes, types.String},
+ []string{"key", "value", "ts"},
)
// spanKeyIterator is a ValueGenerator that iterates over all
@@ -2092,7 +2092,7 @@ type spanKeyIterator struct {
// index maintains the current position of the iterator in kvs.
index int
// A buffer to avoid allocating an array on every call to Values().
- buf [2]tree.Datum
+ buf [3]tree.Datum
}
func newSpanKeyIterator(evalCtx *eval.Context, span roachpb.Span) *spanKeyIterator {
@@ -2166,6 +2166,7 @@ func (sp *spanKeyIterator) Values() (tree.Datums, error) {
kv := sp.kvs[sp.index]
sp.buf[0] = tree.NewDBytes(tree.DBytes(kv.Key))
sp.buf[1] = tree.NewDBytes(tree.DBytes(kv.Value.RawBytes))
+ sp.buf[2] = tree.NewDString(kv.Value.Timestamp.String())
return sp.buf[:], nil
}
@@ -2234,6 +2235,7 @@ func (rk *rangeKeyIterator) Values() (tree.Datums, error) {
kv := rk.kvs[rk.index]
rk.buf[0] = tree.NewDString(kv.Key.String())
rk.buf[1] = tree.NewDString(kv.Value.PrettyPrint())
+ rk.buf[2] = tree.NewDString(kv.Value.Timestamp.String())
return rk.buf[:], nil
}
From 08ee64b828aead3295f03f6f0f62f86bda3cd62d Mon Sep 17 00:00:00 2001
From: Michael Butler
Date: Mon, 31 Oct 2022 17:15:54 -0400
Subject: [PATCH 2/7] backupccl: ensure restore2TB/nodes=10/with-pause pauses
at least once
This patch adds a check that will cause the roachtest to fail if the job was
not paused at least once.
Release note: None
Epic: none
---
pkg/cmd/roachtest/tests/restore.go | 7 ++++++-
1 file changed, 6 insertions(+), 1 deletion(-)
diff --git a/pkg/cmd/roachtest/tests/restore.go b/pkg/cmd/roachtest/tests/restore.go
index eb1e38fb84fe..2810cc9d0c2f 100644
--- a/pkg/cmd/roachtest/tests/restore.go
+++ b/pkg/cmd/roachtest/tests/restore.go
@@ -531,6 +531,7 @@ func registerRestore(r registry.Registry) {
jobIDCh := make(chan jobspb.JobID)
jobCompleteCh := make(chan struct{}, 1)
+ maxPauses := 3
m.Go(func(ctx context.Context) error {
// Wait until the restore job has been created.
conn, err := c.ConnE(ctx, t.L(), c.Node(1)[0])
@@ -547,7 +548,6 @@ func registerRestore(r registry.Registry) {
//
// Limit the number of pauses to 3 to ensure that the test doesn't get
// into a pause-resume-slowdown spiral that eventually times out.
- maxPauses := 3
pauseJobTick := time.NewTicker(time.Minute * 15)
defer pauseJobTick.Stop()
for {
@@ -653,6 +653,11 @@ func registerRestore(r registry.Registry) {
return nil
})
m.Wait()
+ // All failures from the above go routines surface via a t.Fatal() within
+ // the m.Wait( ) call above; therefore, at this point, the restore job
+ // should have succeeded. This final check ensures this test is actually
+ // doing its job: causing the restore job to pause at least once.
+ require.NotEqual(t, 3, maxPauses, "the job should have paused at least once")
},
})
}
From 9e1a77be2aac449f043e65e166a8c58c9ffd3934 Mon Sep 17 00:00:00 2001
From: maryliag
Date: Tue, 1 Nov 2022 09:30:22 -0400
Subject: [PATCH 3/7] test: fix error on test
The PR #90862 introduced an error on generated code.
This PR fixes the missing parameters used for tests.
Epic: None
Release note: None
---
.../src/databaseDetailsPage/databaseDetailsPage.stories.tsx | 5 +++++
.../src/databaseTablePage/databaseTablePage.stories.tsx | 6 ++++++
.../cluster-ui/src/databasesPage/databasesPage.stories.tsx | 4 ++++
3 files changed, 15 insertions(+)
diff --git a/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.stories.tsx b/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.stories.tsx
index 13d4a25ba917..33f6452a4807 100644
--- a/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.stories.tsx
+++ b/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.stories.tsx
@@ -32,6 +32,7 @@ const history = H.createHashHistory();
const withLoadingIndicator: DatabaseDetailsPageProps = {
loading: true,
loaded: false,
+ lastError: undefined,
name: randomName(),
tables: [],
viewMode: ViewMode.Tables,
@@ -61,6 +62,7 @@ const withLoadingIndicator: DatabaseDetailsPageProps = {
const withoutData: DatabaseDetailsPageProps = {
loading: false,
loaded: true,
+ lastError: null,
name: randomName(),
tables: [],
viewMode: ViewMode.Tables,
@@ -98,6 +100,7 @@ function createTable(): DatabaseDetailsPageDataTable {
details: {
loading: false,
loaded: true,
+ lastError: null,
columnCount: _.random(5, 42),
indexCount: _.random(1, 6),
userCount: roles.length,
@@ -112,6 +115,7 @@ function createTable(): DatabaseDetailsPageDataTable {
stats: {
loading: false,
loaded: true,
+ lastError: null,
replicationSizeInBytes: _.random(1000.0) * 1024 ** _.random(1, 2),
rangeCount: _.random(50, 500),
nodesByRegionString:
@@ -123,6 +127,7 @@ function createTable(): DatabaseDetailsPageDataTable {
const withData: DatabaseDetailsPageProps = {
loading: false,
loaded: true,
+ lastError: null,
name: randomName(),
tables: [createTable()],
viewMode: ViewMode.Tables,
diff --git a/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/databaseTablePage.stories.tsx b/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/databaseTablePage.stories.tsx
index 25a11b5a73b7..9650a2cb09f7 100644
--- a/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/databaseTablePage.stories.tsx
+++ b/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/databaseTablePage.stories.tsx
@@ -29,6 +29,7 @@ const withLoadingIndicator: DatabaseTablePageProps = {
details: {
loading: true,
loaded: false,
+ lastError: undefined,
createStatement: "",
replicaCount: 0,
indexNames: [],
@@ -41,12 +42,14 @@ const withLoadingIndicator: DatabaseTablePageProps = {
stats: {
loading: true,
loaded: false,
+ lastError: undefined,
sizeInBytes: 0,
rangeCount: 0,
},
indexStats: {
loading: true,
loaded: false,
+ lastError: undefined,
stats: [],
lastReset: moment("2021-09-04T13:55:00Z"),
},
@@ -74,6 +77,7 @@ const withData: DatabaseTablePageProps = {
details: {
loading: false,
loaded: true,
+ lastError: null,
createStatement: `
CREATE TABLE public.${name} (
id UUID NOT NULL,
@@ -102,6 +106,7 @@ const withData: DatabaseTablePageProps = {
stats: {
loading: false,
loaded: true,
+ lastError: null,
sizeInBytes: 44040192,
rangeCount: 4200,
nodesByRegionString:
@@ -110,6 +115,7 @@ const withData: DatabaseTablePageProps = {
indexStats: {
loading: false,
loaded: true,
+ lastError: null,
stats: [
{
totalReads: 0,
diff --git a/pkg/ui/workspaces/cluster-ui/src/databasesPage/databasesPage.stories.tsx b/pkg/ui/workspaces/cluster-ui/src/databasesPage/databasesPage.stories.tsx
index 023180a437c6..cec9f6bb4965 100644
--- a/pkg/ui/workspaces/cluster-ui/src/databasesPage/databasesPage.stories.tsx
+++ b/pkg/ui/workspaces/cluster-ui/src/databasesPage/databasesPage.stories.tsx
@@ -22,6 +22,7 @@ const history = H.createHashHistory();
const withLoadingIndicator: DatabasesPageProps = {
loading: true,
loaded: false,
+ lastError: undefined,
automaticStatsCollectionEnabled: true,
databases: [],
sortSetting: {
@@ -46,6 +47,7 @@ const withLoadingIndicator: DatabasesPageProps = {
const withoutData: DatabasesPageProps = {
loading: false,
loaded: true,
+ lastError: null,
automaticStatsCollectionEnabled: true,
databases: [],
sortSetting: {
@@ -70,6 +72,7 @@ const withoutData: DatabasesPageProps = {
const withData: DatabasesPageProps = {
loading: false,
loaded: true,
+ lastError: null,
showNodeRegionsColumn: true,
automaticStatsCollectionEnabled: true,
sortSetting: {
@@ -80,6 +83,7 @@ const withData: DatabasesPageProps = {
return {
loading: false,
loaded: true,
+ lastError: null,
name: randomName(),
sizeInBytes: _.random(1000.0) * 1024 ** _.random(1, 2),
tableCount: _.random(5, 100),
From c98c82b0f49f5b91981fdb0fc2076909b8a850dc Mon Sep 17 00:00:00 2001
From: Xiang Gu
Date: Tue, 11 Oct 2022 14:08:42 -0400
Subject: [PATCH 4/7] scmutationexec: fixed a careless bug
1. Fixed a careless bug when removing a check constraint;
2. a minor rewrite to `isColumn` to make it more compact.
---
pkg/sql/schemachanger/scexec/scmutationexec/references.go | 2 +-
pkg/sql/schemachanger/scplan/internal/rules/helpers.go | 7 ++-----
2 files changed, 3 insertions(+), 6 deletions(-)
diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/references.go b/pkg/sql/schemachanger/scexec/scmutationexec/references.go
index 4133ac0586e7..97c21af2313e 100644
--- a/pkg/sql/schemachanger/scexec/scmutationexec/references.go
+++ b/pkg/sql/schemachanger/scexec/scmutationexec/references.go
@@ -75,7 +75,7 @@ func (m *visitor) RemoveCheckConstraint(ctx context.Context, op scop.RemoveCheck
}
for i, m := range tbl.Mutations {
if c := m.GetConstraint(); c != nil &&
- c.ConstraintType != descpb.ConstraintToUpdate_CHECK &&
+ c.ConstraintType == descpb.ConstraintToUpdate_CHECK &&
c.Check.ConstraintID == op.ConstraintID {
tbl.Mutations = append(tbl.Mutations[:i], tbl.Mutations[i+1:]...)
found = true
diff --git a/pkg/sql/schemachanger/scplan/internal/rules/helpers.go b/pkg/sql/schemachanger/scplan/internal/rules/helpers.go
index 435d3eb87ab9..a44d1001aa7e 100644
--- a/pkg/sql/schemachanger/scplan/internal/rules/helpers.go
+++ b/pkg/sql/schemachanger/scplan/internal/rules/helpers.go
@@ -267,11 +267,8 @@ func isIndex(e scpb.Element) bool {
}
func isColumn(e scpb.Element) bool {
- switch e.(type) {
- case *scpb.Column:
- return true
- }
- return false
+ _, ok := e.(*scpb.Column)
+ return ok
}
func isSimpleDependent(e scpb.Element) bool {
From c6bc7b47c2517ab6543d02fb71d42f512598d8ce Mon Sep 17 00:00:00 2001
From: Xiang Gu
Date: Tue, 11 Oct 2022 13:28:23 -0400
Subject: [PATCH 5/7] schemachanger: added a field in CheckConstraint element
protobuf
We added a field bool field, `FromHashShardedColumn`, in the
CheckConstraint element protobuf. This will be needed in the future
when we translate it to a descpb.CheckConstraint when doing check
constraint related work.
---
pkg/sql/schemachanger/scbuild/testdata/drop_index | 2 +-
pkg/sql/schemachanger/scdecomp/decomp.go | 9 +++++----
pkg/sql/schemachanger/scdecomp/testdata/type | 1 +
pkg/sql/schemachanger/scpb/elements.proto | 2 ++
pkg/sql/schemachanger/scpb/uml/table.puml | 1 +
5 files changed, 10 insertions(+), 5 deletions(-)
diff --git a/pkg/sql/schemachanger/scbuild/testdata/drop_index b/pkg/sql/schemachanger/scbuild/testdata/drop_index
index 9bcc62a46db3..2d6a186b669f 100644
--- a/pkg/sql/schemachanger/scbuild/testdata/drop_index
+++ b/pkg/sql/schemachanger/scbuild/testdata/drop_index
@@ -68,6 +68,6 @@ DROP INDEX idx3 CASCADE
- [[IndexData:{DescID: 104, IndexID: 6}, ABSENT], PUBLIC]
{indexId: 6, tableId: 104}
- [[CheckConstraint:{DescID: 104, ConstraintID: 2}, ABSENT], PUBLIC]
- {columnIds: [5], constraintId: 2, expr: 'crdb_internal_i_shard_16 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8, 8:::INT8, 9:::INT8, 10:::INT8, 11:::INT8, 12:::INT8, 13:::INT8, 14:::INT8, 15:::INT8)', referencedColumnIds: [5], tableId: 104}
+ {columnIds: [5], constraintId: 2, expr: 'crdb_internal_i_shard_16 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8, 8:::INT8, 9:::INT8, 10:::INT8, 11:::INT8, 12:::INT8, 13:::INT8, 14:::INT8, 15:::INT8)', fromHashShardedColumn: true, referencedColumnIds: [5], tableId: 104}
- [[ConstraintName:{DescID: 104, Name: check_crdb_internal_i_shard_16, ConstraintID: 2}, ABSENT], PUBLIC]
{constraintId: 2, name: check_crdb_internal_i_shard_16, tableId: 104}
diff --git a/pkg/sql/schemachanger/scdecomp/decomp.go b/pkg/sql/schemachanger/scdecomp/decomp.go
index 4e9f1b230f17..920384264637 100644
--- a/pkg/sql/schemachanger/scdecomp/decomp.go
+++ b/pkg/sql/schemachanger/scdecomp/decomp.go
@@ -606,10 +606,11 @@ func (w *walkCtx) walkCheckConstraint(
}
// TODO(postamar): proper handling of constraint status
w.ev(scpb.Status_PUBLIC, &scpb.CheckConstraint{
- TableID: tbl.GetID(),
- ConstraintID: c.ConstraintID,
- ColumnIDs: catalog.MakeTableColSet(c.ColumnIDs...).Ordered(),
- Expression: *expr,
+ TableID: tbl.GetID(),
+ ConstraintID: c.ConstraintID,
+ ColumnIDs: catalog.MakeTableColSet(c.ColumnIDs...).Ordered(),
+ Expression: *expr,
+ FromHashShardedColumn: c.FromHashShardedColumn,
})
w.ev(scpb.Status_PUBLIC, &scpb.ConstraintName{
TableID: tbl.GetID(),
diff --git a/pkg/sql/schemachanger/scdecomp/testdata/type b/pkg/sql/schemachanger/scdecomp/testdata/type
index cef07f70ac7a..f309198890b6 100644
--- a/pkg/sql/schemachanger/scdecomp/testdata/type
+++ b/pkg/sql/schemachanger/scdecomp/testdata/type
@@ -179,6 +179,7 @@ ElementState:
- 5
constraintId: 2
expr: s::STRING = name
+ fromHashShardedColumn: false
referencedColumnIds:
- 3
- 5
diff --git a/pkg/sql/schemachanger/scpb/elements.proto b/pkg/sql/schemachanger/scpb/elements.proto
index cbc54303a5a6..d9cf0004d84a 100644
--- a/pkg/sql/schemachanger/scpb/elements.proto
+++ b/pkg/sql/schemachanger/scpb/elements.proto
@@ -321,6 +321,8 @@ message CheckConstraint {
uint32 constraint_id = 2 [(gogoproto.customname) = "ConstraintID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.ConstraintID"];
repeated uint32 column_ids = 3 [(gogoproto.customname) = "ColumnIDs", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.ColumnID"];
Expression embedded_expr = 4 [(gogoproto.nullable) = false, (gogoproto.embed) = true];
+ // FromHashShardedColumn indicates whether this check constraint comes from a hash sharded column.
+ bool from_hash_sharded_column = 5;
}
message ForeignKeyConstraint {
diff --git a/pkg/sql/schemachanger/scpb/uml/table.puml b/pkg/sql/schemachanger/scpb/uml/table.puml
index 078ba3a2f21e..26c28e439b71 100644
--- a/pkg/sql/schemachanger/scpb/uml/table.puml
+++ b/pkg/sql/schemachanger/scpb/uml/table.puml
@@ -82,6 +82,7 @@ CheckConstraint : TableID
CheckConstraint : ConstraintID
CheckConstraint : []ColumnIDs
CheckConstraint : Expression
+CheckConstraint : FromHashShardedColumn
object ForeignKeyConstraint
From a7f3ae671d31acf6c1ef8fcb91a2125e68b0eb53 Mon Sep 17 00:00:00 2001
From: Xiang Gu
Date: Tue, 11 Oct 2022 14:12:35 -0400
Subject: [PATCH 6/7] schemachanger: Enable adding/dropping path of check
constraints
We enable the adding/dropping path of check constraints in declarative
schema changer (previously it changes between PUBLIC and ABSENT status
with a NotImplemented operation).
Namely, we introduced a new element status `validating` and reused
`validated` such that the status transitions of a check constraint
element is
ABSENT <==> WRITE_ONLY <==> VALIDATED <==> PUBLIC
and it's subject to the 2-version invariant.
Correspondingly, we emit operations for each transition as follows:
Adding path:
- ABSENT ==> WRITE_ONLY: Add a check constraint (ADD direction) to the mutation slice (Mutation Type op)
- WRITE_ONLY ==> VALIDATED: validate a check constraint (Validation Type op)
- VALIDATED ==> PUBLIC: complete the check constraint adding mutation (Mutation Type op)
Dropping path:
- PUBLIC ==> VALIDATED: Add a check constraint (DROP direction) to the mutation slice (Mutation Type op)
- VALIDATED ==> WRITE_ONLY: equivalent status
- (*) WRITE_ONLY ==> ABSENT: complete the check constraint dropping mutation (Mutation Type op)
A check constraint in
- WRITE_ONLY means it will be enforced for all new UPDATE/INSERT.
- VALIDATED means validation has succeeded but it's not yet public.
N.B.: * means it's information publishing/destroying stage and hence
should be non-revertible.
---
pkg/sql/catalog/tabledesc/table.go | 6 +
.../schemachanger/scexec/exec_validation.go | 29 +-
.../scexec/scmutationexec/helpers.go | 16 +
.../scexec/scmutationexec/references.go | 87 +++++
pkg/sql/schemachanger/scop/mutation.go | 27 ++
.../scop/mutation_visitor_generated.go | 18 +
pkg/sql/schemachanger/scpb/scpb.proto | 2 +
.../internal/opgen/opgen_check_constraint.go | 54 ++-
.../internal/rules/dep_drop_constraint.go | 25 +-
.../scplan/internal/rules/helpers.go | 17 +-
.../scplan/internal/rules/op_drop.go | 5 +
.../scplan/internal/rules/testdata/deprules | 358 +++++++++++-------
.../scplan/internal/rules/testdata/oprules | 8 +-
.../schemachanger/scplan/testdata/drop_index | 36 +-
.../schemachanger/scplan/testdata/drop_table | 43 +--
.../end_to_end/drop_index_hash_sharded_index | 71 ++--
.../explain/drop_index_hash_sharded_index | 22 +-
.../drop_index_hash_sharded_index | 68 ++--
18 files changed, 644 insertions(+), 248 deletions(-)
diff --git a/pkg/sql/catalog/tabledesc/table.go b/pkg/sql/catalog/tabledesc/table.go
index b843af3e2bca..300846f41190 100644
--- a/pkg/sql/catalog/tabledesc/table.go
+++ b/pkg/sql/catalog/tabledesc/table.go
@@ -655,6 +655,12 @@ func IndexNamePlaceholder(id descpb.IndexID) string {
return fmt.Sprintf("crdb_internal_index_%d_name_placeholder", id)
}
+// ConstraintNamePlaceholder constructs a placeholder name for a constraint based
+// on its id.
+func ConstraintNamePlaceholder(id descpb.ConstraintID) string {
+ return fmt.Sprintf("crdb_internal_constraint_%d_name_placeholder", id)
+}
+
// RenameColumnInTable will rename the column in tableDesc from oldName to
// newName, including in expressions as well as shard columns.
// The function is recursive because of this, but there should only be one level
diff --git a/pkg/sql/schemachanger/scexec/exec_validation.go b/pkg/sql/schemachanger/scexec/exec_validation.go
index c1d68f9d550e..72256621bb15 100644
--- a/pkg/sql/schemachanger/scexec/exec_validation.go
+++ b/pkg/sql/schemachanger/scexec/exec_validation.go
@@ -56,7 +56,34 @@ func executeValidateUniqueIndex(
func executeValidateCheckConstraint(
ctx context.Context, deps Dependencies, op *scop.ValidateCheckConstraint,
) error {
- return errors.Errorf("executeValidateCheckConstraint is not implemented")
+ descs, err := deps.Catalog().MustReadImmutableDescriptors(ctx, op.TableID)
+ if err != nil {
+ return err
+ }
+ desc := descs[0]
+ table, err := catalog.AsTableDescriptor(desc)
+ if err != nil {
+ return err
+ }
+ constraint, err := table.FindConstraintWithID(op.ConstraintID)
+ if err != nil {
+ return err
+ }
+ if constraint.CheckConstraint == nil {
+ return errors.Newf("constraint ID %v does not identify a check constraint in table %v.",
+ op.ConstraintID, op.TableID)
+ }
+
+ // Execute the validation operation as a root user.
+ execOverride := sessiondata.InternalExecutorOverride{
+ User: username.RootUserName(),
+ }
+ err = deps.Validator().ValidateCheckConstraint(ctx, table, constraint, execOverride)
+ if err != nil {
+ return scerrors.SchemaChangerUserError(err)
+ }
+ constraint.CheckConstraint.Validity = descpb.ConstraintValidity_Validated
+ return nil
}
func executeValidationOps(ctx context.Context, deps Dependencies, ops []scop.Op) (err error) {
diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/helpers.go b/pkg/sql/schemachanger/scexec/scmutationexec/helpers.go
index 2336d4dbeaaf..fc68b4f2c4a3 100644
--- a/pkg/sql/schemachanger/scexec/scmutationexec/helpers.go
+++ b/pkg/sql/schemachanger/scexec/scmutationexec/helpers.go
@@ -211,6 +211,22 @@ func enqueueDropColumnMutation(tbl *tabledesc.Mutable, col *descpb.ColumnDescrip
return nil
}
+func enqueueAddCheckConstraintMutation(
+ tbl *tabledesc.Mutable, ck *descpb.TableDescriptor_CheckConstraint,
+) error {
+ tbl.AddCheckMutation(ck, descpb.DescriptorMutation_ADD)
+ tbl.NextMutationID--
+ return nil
+}
+
+func enqueueDropCheckConstraintMutation(
+ tbl *tabledesc.Mutable, ck *descpb.TableDescriptor_CheckConstraint,
+) error {
+ tbl.AddCheckMutation(ck, descpb.DescriptorMutation_DROP)
+ tbl.NextMutationID--
+ return nil
+}
+
func enqueueAddIndexMutation(
tbl *tabledesc.Mutable, idx *descpb.IndexDescriptor, state descpb.DescriptorMutation_State,
) error {
diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/references.go b/pkg/sql/schemachanger/scexec/scmutationexec/references.go
index 97c21af2313e..ed2598ba024c 100644
--- a/pkg/sql/schemachanger/scexec/scmutationexec/references.go
+++ b/pkg/sql/schemachanger/scexec/scmutationexec/references.go
@@ -15,6 +15,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
+ "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop"
"github.com/cockroachdb/cockroach/pkg/sql/sem/catid"
"github.com/cockroachdb/cockroach/pkg/util/iterutil"
@@ -89,6 +90,92 @@ func (m *visitor) RemoveCheckConstraint(ctx context.Context, op scop.RemoveCheck
return nil
}
+func (m *visitor) MakeAbsentCheckConstraintWriteOnly(
+ ctx context.Context, op scop.MakeAbsentCheckConstraintWriteOnly,
+) error {
+ tbl, err := m.checkOutTable(ctx, op.TableID)
+ if err != nil || tbl.Dropped() {
+ return err
+ }
+ if op.ConstraintID >= tbl.NextConstraintID {
+ tbl.NextConstraintID = op.ConstraintID + 1
+ }
+
+ // We should have already validated that the check constraint
+ // is syntactically valid in the builder, so we just need to
+ // enqueue it to the descriptor's mutation slice.
+ ck := &descpb.TableDescriptor_CheckConstraint{
+ Expr: string(op.Expr),
+ Name: tabledesc.ConstraintNamePlaceholder(op.ConstraintID),
+ Validity: descpb.ConstraintValidity_Validating,
+ ColumnIDs: op.ColumnIDs,
+ FromHashShardedColumn: op.FromHashShardedColumn,
+ ConstraintID: op.ConstraintID,
+ }
+ return enqueueAddCheckConstraintMutation(tbl, ck)
+}
+
+func (m *visitor) MakeValidatedCheckConstraintPublic(
+ ctx context.Context, op scop.MakeValidatedCheckConstraintPublic,
+) error {
+ tbl, err := m.checkOutTable(ctx, op.TableID)
+ if err != nil || tbl.Dropped() {
+ return err
+ }
+
+ var found bool
+ for idx, mutation := range tbl.Mutations {
+ if c := mutation.GetConstraint(); c != nil &&
+ c.ConstraintType == descpb.ConstraintToUpdate_CHECK &&
+ c.Check.ConstraintID == op.ConstraintID {
+ tbl.Checks = append(tbl.Checks, &c.Check)
+
+ // Remove the mutation from the mutation slice. The `MakeMutationComplete`
+ // call will also mark the above added check as VALIDATED.
+ // If this is a rollback of a drop, we are trying to add the check constraint
+ // back, so swap the direction before making it complete.
+ mutation.Direction = descpb.DescriptorMutation_ADD
+ err = tbl.MakeMutationComplete(mutation)
+ if err != nil {
+ return err
+ }
+ tbl.Mutations = append(tbl.Mutations[:idx], tbl.Mutations[idx+1:]...)
+
+ found = true
+ break
+ }
+ }
+
+ if !found {
+ return errors.AssertionFailedf("failed to find check constraint %d in table %q (%d)",
+ op.ConstraintID, tbl.GetName(), tbl.GetID())
+ }
+
+ if len(tbl.Mutations) == 0 {
+ tbl.Mutations = nil
+ }
+
+ return nil
+}
+
+func (m *visitor) MakePublicCheckConstraintValidated(
+ ctx context.Context, op scop.MakePublicCheckConstraintValidated,
+) error {
+ tbl, err := m.checkOutTable(ctx, op.TableID)
+ if err != nil {
+ return err
+ }
+ for i, ck := range tbl.Checks {
+ if ck.ConstraintID == op.ConstraintID {
+ tbl.Checks = append(tbl.Checks[:i], tbl.Checks[i+1:]...)
+ ck.Validity = descpb.ConstraintValidity_Dropping
+ return enqueueDropCheckConstraintMutation(tbl, ck)
+ }
+ }
+
+ return errors.AssertionFailedf("failed to find check constraint %d in descriptor %v", op.ConstraintID, tbl)
+}
+
func (m *visitor) RemoveForeignKeyBackReference(
ctx context.Context, op scop.RemoveForeignKeyBackReference,
) error {
diff --git a/pkg/sql/schemachanger/scop/mutation.go b/pkg/sql/schemachanger/scop/mutation.go
index 07722c73ea6f..064fd99cdaa7 100644
--- a/pkg/sql/schemachanger/scop/mutation.go
+++ b/pkg/sql/schemachanger/scop/mutation.go
@@ -292,6 +292,33 @@ type RemoveCheckConstraint struct {
ConstraintID descpb.ConstraintID
}
+// MakeAbsentCheckConstraintWriteOnly adds a non-existent check constraint
+// to the table in the WRITE_ONLY state.
+type MakeAbsentCheckConstraintWriteOnly struct {
+ mutationOp
+ TableID descpb.ID
+ ConstraintID descpb.ConstraintID
+ ColumnIDs []descpb.ColumnID
+ scpb.Expression
+ FromHashShardedColumn bool
+}
+
+// MakePublicCheckConstraintValidated moves a public
+// check constraint to VALIDATED.
+type MakePublicCheckConstraintValidated struct {
+ mutationOp
+ TableID descpb.ID
+ ConstraintID descpb.ConstraintID
+}
+
+// MakeValidatedCheckConstraintPublic moves a new, validated check
+// constraint from mutation to public.
+type MakeValidatedCheckConstraintPublic struct {
+ mutationOp
+ TableID descpb.ID
+ ConstraintID descpb.ConstraintID
+}
+
// RemoveForeignKeyConstraint removes a foreign key from the origin table.
type RemoveForeignKeyConstraint struct {
mutationOp
diff --git a/pkg/sql/schemachanger/scop/mutation_visitor_generated.go b/pkg/sql/schemachanger/scop/mutation_visitor_generated.go
index 7d25415b50db..82dad2179355 100644
--- a/pkg/sql/schemachanger/scop/mutation_visitor_generated.go
+++ b/pkg/sql/schemachanger/scop/mutation_visitor_generated.go
@@ -55,6 +55,9 @@ type MutationVisitor interface {
RemoveOwnerBackReferenceInSequence(context.Context, RemoveOwnerBackReferenceInSequence) error
RemoveSequenceOwner(context.Context, RemoveSequenceOwner) error
RemoveCheckConstraint(context.Context, RemoveCheckConstraint) error
+ MakeAbsentCheckConstraintWriteOnly(context.Context, MakeAbsentCheckConstraintWriteOnly) error
+ MakePublicCheckConstraintValidated(context.Context, MakePublicCheckConstraintValidated) error
+ MakeValidatedCheckConstraintPublic(context.Context, MakeValidatedCheckConstraintPublic) error
RemoveForeignKeyConstraint(context.Context, RemoveForeignKeyConstraint) error
RemoveForeignKeyBackReference(context.Context, RemoveForeignKeyBackReference) error
RemoveSchemaParent(context.Context, RemoveSchemaParent) error
@@ -262,6 +265,21 @@ func (op RemoveCheckConstraint) Visit(ctx context.Context, v MutationVisitor) er
return v.RemoveCheckConstraint(ctx, op)
}
+// Visit is part of the MutationOp interface.
+func (op MakeAbsentCheckConstraintWriteOnly) Visit(ctx context.Context, v MutationVisitor) error {
+ return v.MakeAbsentCheckConstraintWriteOnly(ctx, op)
+}
+
+// Visit is part of the MutationOp interface.
+func (op MakePublicCheckConstraintValidated) Visit(ctx context.Context, v MutationVisitor) error {
+ return v.MakePublicCheckConstraintValidated(ctx, op)
+}
+
+// Visit is part of the MutationOp interface.
+func (op MakeValidatedCheckConstraintPublic) Visit(ctx context.Context, v MutationVisitor) error {
+ return v.MakeValidatedCheckConstraintPublic(ctx, op)
+}
+
// Visit is part of the MutationOp interface.
func (op RemoveForeignKeyConstraint) Visit(ctx context.Context, v MutationVisitor) error {
return v.RemoveForeignKeyConstraint(ctx, op)
diff --git a/pkg/sql/schemachanger/scpb/scpb.proto b/pkg/sql/schemachanger/scpb/scpb.proto
index c402942a3233..7b4ffb3536f9 100644
--- a/pkg/sql/schemachanger/scpb/scpb.proto
+++ b/pkg/sql/schemachanger/scpb/scpb.proto
@@ -43,10 +43,12 @@ enum Status {
DROPPED = 5;
// Intermediate states on the column and index dropping and adding paths.
+ // WRITE_ONLY is also used on constraint adding and dropping paths.
WRITE_ONLY = 6;
DELETE_ONLY = 7;
// Intermediate states on the index dropping and adding paths.
+ // VALIDATED is also used on constraint adding and dropping paths.
VALIDATED = 8;
MERGED = 9;
MERGE_ONLY = 10;
diff --git a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_check_constraint.go b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_check_constraint.go
index df2104452e0c..f75e0267551c 100644
--- a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_check_constraint.go
+++ b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_check_constraint.go
@@ -19,16 +19,64 @@ func init() {
opRegistry.register((*scpb.CheckConstraint)(nil),
toPublic(
scpb.Status_ABSENT,
+ to(scpb.Status_WRITE_ONLY,
+ emit(func(this *scpb.CheckConstraint) *scop.MakeAbsentCheckConstraintWriteOnly {
+ return &scop.MakeAbsentCheckConstraintWriteOnly{
+ TableID: this.TableID,
+ ConstraintID: this.ConstraintID,
+ ColumnIDs: this.ColumnIDs,
+ Expression: this.Expression,
+ FromHashShardedColumn: this.FromHashShardedColumn,
+ }
+ }),
+ emit(func(this *scpb.CheckConstraint) *scop.UpdateTableBackReferencesInTypes {
+ if len(this.UsesTypeIDs) == 0 {
+ return nil
+ }
+ return &scop.UpdateTableBackReferencesInTypes{
+ TypeIDs: this.UsesTypeIDs,
+ BackReferencedTableID: this.TableID,
+ }
+ }),
+ emit(func(this *scpb.CheckConstraint) *scop.UpdateBackReferencesInSequences {
+ if len(this.UsesSequenceIDs) == 0 {
+ return nil
+ }
+ return &scop.UpdateBackReferencesInSequences{
+ SequenceIDs: this.UsesSequenceIDs,
+ BackReferencedTableID: this.TableID,
+ }
+ }),
+ ),
+ to(scpb.Status_VALIDATED,
+ emit(func(this *scpb.CheckConstraint) *scop.ValidateCheckConstraint {
+ return &scop.ValidateCheckConstraint{
+ TableID: this.TableID,
+ ConstraintID: this.ConstraintID,
+ }
+ }),
+ ),
to(scpb.Status_PUBLIC,
- emit(func(this *scpb.CheckConstraint) *scop.NotImplemented {
- return notImplemented(this)
+ emit(func(this *scpb.CheckConstraint) *scop.MakeValidatedCheckConstraintPublic {
+ return &scop.MakeValidatedCheckConstraintPublic{
+ TableID: this.TableID,
+ ConstraintID: this.ConstraintID,
+ }
}),
),
),
toAbsent(
scpb.Status_PUBLIC,
+ to(scpb.Status_VALIDATED,
+ emit(func(this *scpb.CheckConstraint) *scop.MakePublicCheckConstraintValidated {
+ return &scop.MakePublicCheckConstraintValidated{
+ TableID: this.TableID,
+ ConstraintID: this.ConstraintID,
+ }
+ }),
+ ),
+ equiv(scpb.Status_WRITE_ONLY),
to(scpb.Status_ABSENT,
- // TODO(postamar): remove revertibility constraint when possible
revertible(false),
emit(func(this *scpb.CheckConstraint) *scop.RemoveCheckConstraint {
return &scop.RemoveCheckConstraint{
diff --git a/pkg/sql/schemachanger/scplan/internal/rules/dep_drop_constraint.go b/pkg/sql/schemachanger/scplan/internal/rules/dep_drop_constraint.go
index 36790a4bd9f4..10770d53328e 100644
--- a/pkg/sql/schemachanger/scplan/internal/rules/dep_drop_constraint.go
+++ b/pkg/sql/schemachanger/scplan/internal/rules/dep_drop_constraint.go
@@ -18,31 +18,34 @@ import (
// These rules ensure that constraint-dependent elements, like an constraint's
// name, etc. disappear once the constraint reaches a suitable state.
+// TODO (xiang): The dep rules here are not complete, as they are aimed specifically
+// for check constraints only. Complete them when we properly support the
+// other constraints: UniqueWithoutIndex, ForeignKey, Unique, and Not Null.
func init() {
registerDepRuleForDrop(
- "constraint dependent absent right before constraint",
- scgraph.SameStagePrecedence,
- "dependent", "constraint",
- scpb.Status_ABSENT, scpb.Status_ABSENT,
+ "constraint no longer public before dependents",
+ scgraph.Precedence,
+ "constraint", "dependent",
+ scpb.Status_VALIDATED, scpb.Status_ABSENT,
func(from, to nodeVars) rel.Clauses {
return rel.Clauses{
- from.typeFilter(isConstraintDependent),
- to.typeFilter(isConstraint, not(isIndex)),
+ from.typeFilter(isSupportedNonIndexBackedConstraint),
+ to.typeFilter(isConstraintDependent),
joinOnConstraintID(from, to, "table-id", "constraint-id"),
}
},
)
registerDepRuleForDrop(
- "constraint dependent absent right before constraint",
- scgraph.SameStagePrecedence,
- "dependent", "constraint",
- scpb.Status_VALIDATED, scpb.Status_ABSENT,
+ "dependents removed before constraint",
+ scgraph.Precedence,
+ "dependents", "constraint",
+ scpb.Status_ABSENT, scpb.Status_ABSENT,
func(from, to nodeVars) rel.Clauses {
return rel.Clauses{
from.typeFilter(isConstraintDependent),
- to.typeFilter(isConstraint, isIndex),
+ to.typeFilter(isSupportedNonIndexBackedConstraint),
joinOnConstraintID(from, to, "table-id", "constraint-id"),
}
},
diff --git a/pkg/sql/schemachanger/scplan/internal/rules/helpers.go b/pkg/sql/schemachanger/scplan/internal/rules/helpers.go
index a44d1001aa7e..c0afb98905c1 100644
--- a/pkg/sql/schemachanger/scplan/internal/rules/helpers.go
+++ b/pkg/sql/schemachanger/scplan/internal/rules/helpers.go
@@ -255,7 +255,7 @@ func isSubjectTo2VersionInvariant(e scpb.Element) bool {
// TODO(ajwerner): This should include constraints and enum values but it
// currently does not because we do not support dropping them unless we're
// dropping the descriptor and we do not support adding them.
- return isIndex(e) || isColumn(e)
+ return isIndex(e) || isColumn(e) || isSupportedNonIndexBackedConstraint(e)
}
func isIndex(e scpb.Element) bool {
@@ -369,6 +369,15 @@ func isIndexDependent(e scpb.Element) bool {
return false
}
+// A non-index-backed constraint is one of {Check, FK, UniqueWithoutIndex}. We only
+// support Check for now.
+// TODO (xiang): Expand this predicate to include other non-index-backed constraints
+// when we properly support adding/dropping them in the new schema changer.
+func isSupportedNonIndexBackedConstraint(e scpb.Element) bool {
+ _, ok := e.(*scpb.CheckConstraint)
+ return ok
+}
+
func isConstraint(e scpb.Element) bool {
switch e.(type) {
case *scpb.PrimaryIndex, *scpb.SecondaryIndex, *scpb.TemporaryIndex:
@@ -414,12 +423,6 @@ func or(predicates ...elementTypePredicate) elementTypePredicate {
}
}
-func not(predicate elementTypePredicate) elementTypePredicate {
- return func(e scpb.Element) bool {
- return !predicate(e)
- }
-}
-
// registerDepRuleForDrop is a convenience function which calls
// registerDepRule with the cross-product of (ToAbsent,Transient)^2 target
// states, which can't easily be composed.
diff --git a/pkg/sql/schemachanger/scplan/internal/rules/op_drop.go b/pkg/sql/schemachanger/scplan/internal/rules/op_drop.go
index c600f9da3097..a2339902bc48 100644
--- a/pkg/sql/schemachanger/scplan/internal/rules/op_drop.go
+++ b/pkg/sql/schemachanger/scplan/internal/rules/op_drop.go
@@ -168,6 +168,7 @@ func init() {
(*scpb.View)(nil),
),
constraint.Type(
+ (*scpb.CheckConstraint)(nil),
(*scpb.UniqueWithoutIndexConstraint)(nil),
),
@@ -177,6 +178,10 @@ func init() {
relation.targetStatus(scpb.ToAbsent),
constraint.joinTargetNode(),
constraint.targetStatus(scpb.ToAbsent),
+ constraint.currentStatus(
+ // Only skip ops in the transition from PUBLIC on relation drop.
+ scpb.Status_PUBLIC,
+ ),
),
)
diff --git a/pkg/sql/schemachanger/scplan/internal/rules/testdata/deprules b/pkg/sql/schemachanger/scplan/internal/rules/testdata/deprules
index 3fcf7de7d03a..ca50ae885cbb 100644
--- a/pkg/sql/schemachanger/scplan/internal/rules/testdata/deprules
+++ b/pkg/sql/schemachanger/scplan/internal/rules/testdata/deprules
@@ -92,6 +92,11 @@ nodeNotExistsWithStatusIn_VALIDATED_TRANSIENT_WRITE_ONLY_MERGE_ONLY_TRANSIENT_ME
- $n[Type] = '*screl.Node'
- $n[Target] = $sharedTarget
- $n[CurrentStatus] IN [VALIDATED, TRANSIENT_WRITE_ONLY, MERGE_ONLY, TRANSIENT_MERGE_ONLY, MERGED, TRANSIENT_MERGED]
+nodeNotExistsWithStatusIn_WRITE_ONLY($sharedTarget):
+ not-join:
+ - $n[Type] = '*screl.Node'
+ - $n[Target] = $sharedTarget
+ - $n[CurrentStatus] IN [WRITE_ONLY]
sourceIndexIsSet($index):
- $index[SourceIndexID] != 0
toAbsent($target1, $target2):
@@ -106,6 +111,103 @@ transient($target1, $target2):
deprules
----
+- name: 'CheckConstraint transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED'
+ from: prev-node
+ kind: PreviousTransactionPrecedence
+ to: next-node
+ query:
+ - $prev[Type] = '*scpb.CheckConstraint'
+ - $next[Type] = '*scpb.CheckConstraint'
+ - $prev[DescID] = $_
+ - $prev[Self] = $next
+ - $prev-target[Self] = $next-target
+ - $prev-target[TargetStatus] = ABSENT
+ - $prev-node[CurrentStatus] = PUBLIC
+ - $next-node[CurrentStatus] = VALIDATED
+ - descriptorIsNotBeingDropped($prev)
+ - joinTargetNode($prev, $prev-target, $prev-node)
+ - joinTargetNode($next, $next-target, $next-node)
+- name: 'CheckConstraint transitions to ABSENT uphold 2-version invariant: VALIDATED->ABSENT'
+ from: prev-node
+ kind: PreviousTransactionPrecedence
+ to: next-node
+ query:
+ - $prev[Type] = '*scpb.CheckConstraint'
+ - $next[Type] = '*scpb.CheckConstraint'
+ - $prev[DescID] = $_
+ - $prev[Self] = $next
+ - $prev-target[Self] = $next-target
+ - $prev-target[TargetStatus] = ABSENT
+ - $prev-node[CurrentStatus] = VALIDATED
+ - $next-node[CurrentStatus] = ABSENT
+ - descriptorIsNotBeingDropped($prev)
+ - nodeNotExistsWithStatusIn_WRITE_ONLY($prev-target)
+ - joinTargetNode($prev, $prev-target, $prev-node)
+ - joinTargetNode($next, $next-target, $next-node)
+- name: 'CheckConstraint transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->VALIDATED'
+ from: prev-node
+ kind: PreviousTransactionPrecedence
+ to: next-node
+ query:
+ - $prev[Type] = '*scpb.CheckConstraint'
+ - $next[Type] = '*scpb.CheckConstraint'
+ - $prev[DescID] = $_
+ - $prev[Self] = $next
+ - $prev-target[Self] = $next-target
+ - $prev-target[TargetStatus] = ABSENT
+ - $prev-node[CurrentStatus] = WRITE_ONLY
+ - $next-node[CurrentStatus] = VALIDATED
+ - descriptorIsNotBeingDropped($prev)
+ - joinTargetNode($prev, $prev-target, $prev-node)
+ - joinTargetNode($next, $next-target, $next-node)
+- name: 'CheckConstraint transitions to PUBLIC uphold 2-version invariant: ABSENT->WRITE_ONLY'
+ from: prev-node
+ kind: PreviousTransactionPrecedence
+ to: next-node
+ query:
+ - $prev[Type] = '*scpb.CheckConstraint'
+ - $next[Type] = '*scpb.CheckConstraint'
+ - $prev[DescID] = $_
+ - $prev[Self] = $next
+ - $prev-target[Self] = $next-target
+ - $prev-target[TargetStatus] = PUBLIC
+ - $prev-node[CurrentStatus] = ABSENT
+ - $next-node[CurrentStatus] = WRITE_ONLY
+ - descriptorIsNotBeingDropped($prev)
+ - joinTargetNode($prev, $prev-target, $prev-node)
+ - joinTargetNode($next, $next-target, $next-node)
+- name: 'CheckConstraint transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC'
+ from: prev-node
+ kind: PreviousTransactionPrecedence
+ to: next-node
+ query:
+ - $prev[Type] = '*scpb.CheckConstraint'
+ - $next[Type] = '*scpb.CheckConstraint'
+ - $prev[DescID] = $_
+ - $prev[Self] = $next
+ - $prev-target[Self] = $next-target
+ - $prev-target[TargetStatus] = PUBLIC
+ - $prev-node[CurrentStatus] = VALIDATED
+ - $next-node[CurrentStatus] = PUBLIC
+ - descriptorIsNotBeingDropped($prev)
+ - joinTargetNode($prev, $prev-target, $prev-node)
+ - joinTargetNode($next, $next-target, $next-node)
+- name: 'CheckConstraint transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED'
+ from: prev-node
+ kind: PreviousTransactionPrecedence
+ to: next-node
+ query:
+ - $prev[Type] = '*scpb.CheckConstraint'
+ - $next[Type] = '*scpb.CheckConstraint'
+ - $prev[DescID] = $_
+ - $prev[Self] = $next
+ - $prev-target[Self] = $next-target
+ - $prev-target[TargetStatus] = PUBLIC
+ - $prev-node[CurrentStatus] = WRITE_ONLY
+ - $next-node[CurrentStatus] = VALIDATED
+ - descriptorIsNotBeingDropped($prev)
+ - joinTargetNode($prev, $prev-target, $prev-node)
+ - joinTargetNode($next, $next-target, $next-node)
- name: 'Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT'
from: prev-node
kind: PreviousTransactionPrecedence
@@ -1409,125 +1511,71 @@ deprules
- $column-node[CurrentStatus] = ABSENT
- joinTargetNode($column-type, $column-type-target, $column-type-node)
- joinTargetNode($column, $column-target, $column-node)
-- name: constraint dependent absent right before constraint
- from: dependent-node
+- name: constraint dependent public right before constraint
+ from: constraint-node
kind: SameStagePrecedence
- to: constraint-node
+ to: dependent-node
query:
+ - $constraint[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex', '*scpb.TemporaryIndex', '*scpb.UniqueWithoutIndexConstraint', '*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint']
- $dependent[Type] IN ['*scpb.ConstraintName', '*scpb.ConstraintComment']
- - $constraint[Type] IN ['*scpb.UniqueWithoutIndexConstraint', '*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint']
- - joinOnConstraintID($dependent, $constraint, $table-id, $constraint-id)
- - transient($dependent-target, $constraint-target)
- - $dependent-node[CurrentStatus] = TRANSIENT_ABSENT
- - $constraint-node[CurrentStatus] = TRANSIENT_ABSENT
- - joinTargetNode($dependent, $dependent-target, $dependent-node)
+ - joinOnConstraintID($constraint, $dependent, $table-id, $constraint-id)
+ - toPublicOrTransient($constraint-target, $dependent-target)
+ - $constraint-node[CurrentStatus] = PUBLIC
+ - $dependent-node[CurrentStatus] = PUBLIC
- joinTargetNode($constraint, $constraint-target, $constraint-node)
-- name: constraint dependent absent right before constraint
- from: dependent-node
- kind: SameStagePrecedence
- to: constraint-node
+ - joinTargetNode($dependent, $dependent-target, $dependent-node)
+- name: constraint no longer public before dependents
+ from: constraint-node
+ kind: Precedence
+ to: dependent-node
query:
+ - $constraint[Type] = '*scpb.CheckConstraint'
- $dependent[Type] IN ['*scpb.ConstraintName', '*scpb.ConstraintComment']
- - $constraint[Type] IN ['*scpb.UniqueWithoutIndexConstraint', '*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint']
- - joinOnConstraintID($dependent, $constraint, $table-id, $constraint-id)
- - toAbsent($dependent-target, $constraint-target)
+ - joinOnConstraintID($constraint, $dependent, $table-id, $constraint-id)
+ - toAbsent($constraint-target, $dependent-target)
+ - $constraint-node[CurrentStatus] = VALIDATED
- $dependent-node[CurrentStatus] = ABSENT
- - $constraint-node[CurrentStatus] = ABSENT
- - joinTargetNode($dependent, $dependent-target, $dependent-node)
- joinTargetNode($constraint, $constraint-target, $constraint-node)
-- name: constraint dependent absent right before constraint
- from: dependent-node
- kind: SameStagePrecedence
- to: constraint-node
+ - joinTargetNode($dependent, $dependent-target, $dependent-node)
+- name: constraint no longer public before dependents
+ from: constraint-node
+ kind: Precedence
+ to: dependent-node
query:
+ - $constraint[Type] = '*scpb.CheckConstraint'
- $dependent[Type] IN ['*scpb.ConstraintName', '*scpb.ConstraintComment']
- - $constraint[Type] IN ['*scpb.UniqueWithoutIndexConstraint', '*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint']
- - joinOnConstraintID($dependent, $constraint, $table-id, $constraint-id)
+ - joinOnConstraintID($constraint, $dependent, $table-id, $constraint-id)
+ - $constraint-target[TargetStatus] = ABSENT
+ - $constraint-node[CurrentStatus] = VALIDATED
- $dependent-target[TargetStatus] = TRANSIENT_ABSENT
- $dependent-node[CurrentStatus] = TRANSIENT_ABSENT
- - $constraint-target[TargetStatus] = ABSENT
- - $constraint-node[CurrentStatus] = ABSENT
- - joinTargetNode($dependent, $dependent-target, $dependent-node)
- joinTargetNode($constraint, $constraint-target, $constraint-node)
-- name: constraint dependent absent right before constraint
- from: dependent-node
- kind: SameStagePrecedence
- to: constraint-node
- query:
- - $dependent[Type] IN ['*scpb.ConstraintName', '*scpb.ConstraintComment']
- - $constraint[Type] IN ['*scpb.UniqueWithoutIndexConstraint', '*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint']
- - joinOnConstraintID($dependent, $constraint, $table-id, $constraint-id)
- - $dependent-target[TargetStatus] = ABSENT
- - $dependent-node[CurrentStatus] = ABSENT
- - $constraint-target[TargetStatus] = TRANSIENT_ABSENT
- - $constraint-node[CurrentStatus] = TRANSIENT_ABSENT
- joinTargetNode($dependent, $dependent-target, $dependent-node)
- - joinTargetNode($constraint, $constraint-target, $constraint-node)
-- name: constraint dependent absent right before constraint
- from: dependent-node
- kind: SameStagePrecedence
- to: constraint-node
- query:
- - $dependent[Type] IN ['*scpb.ConstraintName', '*scpb.ConstraintComment']
- - $constraint[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex', '*scpb.TemporaryIndex']
- - joinOnConstraintID($dependent, $constraint, $table-id, $constraint-id)
- - toAbsent($dependent-target, $constraint-target)
- - $dependent-node[CurrentStatus] = VALIDATED
- - $constraint-node[CurrentStatus] = ABSENT
- - joinTargetNode($dependent, $dependent-target, $dependent-node)
- - joinTargetNode($constraint, $constraint-target, $constraint-node)
-- name: constraint dependent absent right before constraint
- from: dependent-node
- kind: SameStagePrecedence
- to: constraint-node
+- name: constraint no longer public before dependents
+ from: constraint-node
+ kind: Precedence
+ to: dependent-node
query:
+ - $constraint[Type] = '*scpb.CheckConstraint'
- $dependent[Type] IN ['*scpb.ConstraintName', '*scpb.ConstraintComment']
- - $constraint[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex', '*scpb.TemporaryIndex']
- - joinOnConstraintID($dependent, $constraint, $table-id, $constraint-id)
- - transient($dependent-target, $constraint-target)
- - $dependent-node[CurrentStatus] = TRANSIENT_VALIDATED
- - $constraint-node[CurrentStatus] = TRANSIENT_ABSENT
- - joinTargetNode($dependent, $dependent-target, $dependent-node)
+ - joinOnConstraintID($constraint, $dependent, $table-id, $constraint-id)
+ - transient($constraint-target, $dependent-target)
+ - $constraint-node[CurrentStatus] = TRANSIENT_VALIDATED
+ - $dependent-node[CurrentStatus] = TRANSIENT_ABSENT
- joinTargetNode($constraint, $constraint-target, $constraint-node)
-- name: constraint dependent absent right before constraint
- from: dependent-node
- kind: SameStagePrecedence
- to: constraint-node
- query:
- - $dependent[Type] IN ['*scpb.ConstraintName', '*scpb.ConstraintComment']
- - $constraint[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex', '*scpb.TemporaryIndex']
- - joinOnConstraintID($dependent, $constraint, $table-id, $constraint-id)
- - $dependent-target[TargetStatus] = TRANSIENT_ABSENT
- - $dependent-node[CurrentStatus] = TRANSIENT_VALIDATED
- - $constraint-target[TargetStatus] = ABSENT
- - $constraint-node[CurrentStatus] = ABSENT
- joinTargetNode($dependent, $dependent-target, $dependent-node)
- - joinTargetNode($constraint, $constraint-target, $constraint-node)
-- name: constraint dependent absent right before constraint
- from: dependent-node
- kind: SameStagePrecedence
- to: constraint-node
- query:
- - $dependent[Type] IN ['*scpb.ConstraintName', '*scpb.ConstraintComment']
- - $constraint[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex', '*scpb.TemporaryIndex']
- - joinOnConstraintID($dependent, $constraint, $table-id, $constraint-id)
- - $dependent-target[TargetStatus] = ABSENT
- - $dependent-node[CurrentStatus] = VALIDATED
- - $constraint-target[TargetStatus] = TRANSIENT_ABSENT
- - $constraint-node[CurrentStatus] = TRANSIENT_ABSENT
- - joinTargetNode($dependent, $dependent-target, $dependent-node)
- - joinTargetNode($constraint, $constraint-target, $constraint-node)
-- name: constraint dependent public right before constraint
+- name: constraint no longer public before dependents
from: constraint-node
- kind: SameStagePrecedence
+ kind: Precedence
to: dependent-node
query:
- - $constraint[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex', '*scpb.TemporaryIndex', '*scpb.UniqueWithoutIndexConstraint', '*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint']
+ - $constraint[Type] = '*scpb.CheckConstraint'
- $dependent[Type] IN ['*scpb.ConstraintName', '*scpb.ConstraintComment']
- joinOnConstraintID($constraint, $dependent, $table-id, $constraint-id)
- - toPublicOrTransient($constraint-target, $dependent-target)
- - $constraint-node[CurrentStatus] = PUBLIC
- - $dependent-node[CurrentStatus] = PUBLIC
+ - $constraint-target[TargetStatus] = TRANSIENT_ABSENT
+ - $constraint-node[CurrentStatus] = TRANSIENT_VALIDATED
+ - $dependent-target[TargetStatus] = ABSENT
+ - $dependent-node[CurrentStatus] = ABSENT
- joinTargetNode($constraint, $constraint-target, $constraint-node)
- joinTargetNode($dependent, $dependent-target, $dependent-node)
- name: dependents removed before column
@@ -1538,10 +1586,9 @@ deprules
- $dependent[Type] IN ['*scpb.ColumnName', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SequenceOwner', '*scpb.ColumnComment', '*scpb.IndexColumn']
- $column[Type] = '*scpb.Column'
- joinOnColumnID($dependent, $column, $table-id, $col-id)
- - $dependent-target[TargetStatus] = ABSENT
+ - toAbsent($dependent-target, $column-target)
- $dependent-node[CurrentStatus] = ABSENT
- - $column-target[TargetStatus] = TRANSIENT_ABSENT
- - $column-node[CurrentStatus] = TRANSIENT_ABSENT
+ - $column-node[CurrentStatus] = ABSENT
- joinTargetNode($dependent, $dependent-target, $dependent-node)
- joinTargetNode($column, $column-target, $column-node)
- name: dependents removed before column
@@ -1552,10 +1599,9 @@ deprules
- $dependent[Type] IN ['*scpb.ColumnName', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SequenceOwner', '*scpb.ColumnComment', '*scpb.IndexColumn']
- $column[Type] = '*scpb.Column'
- joinOnColumnID($dependent, $column, $table-id, $col-id)
- - $dependent-target[TargetStatus] = TRANSIENT_ABSENT
+ - transient($dependent-target, $column-target)
- $dependent-node[CurrentStatus] = TRANSIENT_ABSENT
- - $column-target[TargetStatus] = ABSENT
- - $column-node[CurrentStatus] = ABSENT
+ - $column-node[CurrentStatus] = TRANSIENT_ABSENT
- joinTargetNode($dependent, $dependent-target, $dependent-node)
- joinTargetNode($column, $column-target, $column-node)
- name: dependents removed before column
@@ -1566,9 +1612,10 @@ deprules
- $dependent[Type] IN ['*scpb.ColumnName', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SequenceOwner', '*scpb.ColumnComment', '*scpb.IndexColumn']
- $column[Type] = '*scpb.Column'
- joinOnColumnID($dependent, $column, $table-id, $col-id)
- - transient($dependent-target, $column-target)
+ - $dependent-target[TargetStatus] = TRANSIENT_ABSENT
- $dependent-node[CurrentStatus] = TRANSIENT_ABSENT
- - $column-node[CurrentStatus] = TRANSIENT_ABSENT
+ - $column-target[TargetStatus] = ABSENT
+ - $column-node[CurrentStatus] = ABSENT
- joinTargetNode($dependent, $dependent-target, $dependent-node)
- joinTargetNode($column, $column-target, $column-node)
- name: dependents removed before column
@@ -1579,11 +1626,66 @@ deprules
- $dependent[Type] IN ['*scpb.ColumnName', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SequenceOwner', '*scpb.ColumnComment', '*scpb.IndexColumn']
- $column[Type] = '*scpb.Column'
- joinOnColumnID($dependent, $column, $table-id, $col-id)
- - toAbsent($dependent-target, $column-target)
+ - $dependent-target[TargetStatus] = ABSENT
- $dependent-node[CurrentStatus] = ABSENT
- - $column-node[CurrentStatus] = ABSENT
+ - $column-target[TargetStatus] = TRANSIENT_ABSENT
+ - $column-node[CurrentStatus] = TRANSIENT_ABSENT
- joinTargetNode($dependent, $dependent-target, $dependent-node)
- joinTargetNode($column, $column-target, $column-node)
+- name: dependents removed before constraint
+ from: dependents-node
+ kind: Precedence
+ to: constraint-node
+ query:
+ - $dependents[Type] IN ['*scpb.ConstraintName', '*scpb.ConstraintComment']
+ - $constraint[Type] = '*scpb.CheckConstraint'
+ - joinOnConstraintID($dependents, $constraint, $table-id, $constraint-id)
+ - $dependents-target[TargetStatus] = ABSENT
+ - $dependents-node[CurrentStatus] = ABSENT
+ - $constraint-target[TargetStatus] = TRANSIENT_ABSENT
+ - $constraint-node[CurrentStatus] = TRANSIENT_ABSENT
+ - joinTargetNode($dependents, $dependents-target, $dependents-node)
+ - joinTargetNode($constraint, $constraint-target, $constraint-node)
+- name: dependents removed before constraint
+ from: dependents-node
+ kind: Precedence
+ to: constraint-node
+ query:
+ - $dependents[Type] IN ['*scpb.ConstraintName', '*scpb.ConstraintComment']
+ - $constraint[Type] = '*scpb.CheckConstraint'
+ - joinOnConstraintID($dependents, $constraint, $table-id, $constraint-id)
+ - $dependents-target[TargetStatus] = TRANSIENT_ABSENT
+ - $dependents-node[CurrentStatus] = TRANSIENT_ABSENT
+ - $constraint-target[TargetStatus] = ABSENT
+ - $constraint-node[CurrentStatus] = ABSENT
+ - joinTargetNode($dependents, $dependents-target, $dependents-node)
+ - joinTargetNode($constraint, $constraint-target, $constraint-node)
+- name: dependents removed before constraint
+ from: dependents-node
+ kind: Precedence
+ to: constraint-node
+ query:
+ - $dependents[Type] IN ['*scpb.ConstraintName', '*scpb.ConstraintComment']
+ - $constraint[Type] = '*scpb.CheckConstraint'
+ - joinOnConstraintID($dependents, $constraint, $table-id, $constraint-id)
+ - transient($dependents-target, $constraint-target)
+ - $dependents-node[CurrentStatus] = TRANSIENT_ABSENT
+ - $constraint-node[CurrentStatus] = TRANSIENT_ABSENT
+ - joinTargetNode($dependents, $dependents-target, $dependents-node)
+ - joinTargetNode($constraint, $constraint-target, $constraint-node)
+- name: dependents removed before constraint
+ from: dependents-node
+ kind: Precedence
+ to: constraint-node
+ query:
+ - $dependents[Type] IN ['*scpb.ConstraintName', '*scpb.ConstraintComment']
+ - $constraint[Type] = '*scpb.CheckConstraint'
+ - joinOnConstraintID($dependents, $constraint, $table-id, $constraint-id)
+ - toAbsent($dependents-target, $constraint-target)
+ - $dependents-node[CurrentStatus] = ABSENT
+ - $constraint-node[CurrentStatus] = ABSENT
+ - joinTargetNode($dependents, $dependents-target, $dependents-node)
+ - joinTargetNode($constraint, $constraint-target, $constraint-node)
- name: dependents removed before index
from: dependent-node
kind: Precedence
@@ -1592,8 +1694,9 @@ deprules
- $dependent[Type] IN ['*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.SecondaryIndexPartial', '*scpb.IndexComment', '*scpb.IndexColumn']
- $index[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex', '*scpb.TemporaryIndex']
- joinOnIndexID($dependent, $index, $table-id, $index-id)
- - transient($dependent-target, $index-target)
- - $dependent-node[CurrentStatus] = TRANSIENT_ABSENT
+ - $dependent-target[TargetStatus] = ABSENT
+ - $dependent-node[CurrentStatus] = ABSENT
+ - $index-target[TargetStatus] = TRANSIENT_ABSENT
- $index-node[CurrentStatus] = TRANSIENT_ABSENT
- joinTargetNode($dependent, $dependent-target, $dependent-node)
- joinTargetNode($index, $index-target, $index-node)
@@ -1605,8 +1708,9 @@ deprules
- $dependent[Type] IN ['*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.SecondaryIndexPartial', '*scpb.IndexComment', '*scpb.IndexColumn']
- $index[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex', '*scpb.TemporaryIndex']
- joinOnIndexID($dependent, $index, $table-id, $index-id)
- - toAbsent($dependent-target, $index-target)
- - $dependent-node[CurrentStatus] = ABSENT
+ - $dependent-target[TargetStatus] = TRANSIENT_ABSENT
+ - $dependent-node[CurrentStatus] = TRANSIENT_ABSENT
+ - $index-target[TargetStatus] = ABSENT
- $index-node[CurrentStatus] = ABSENT
- joinTargetNode($dependent, $dependent-target, $dependent-node)
- joinTargetNode($index, $index-target, $index-node)
@@ -1618,10 +1722,9 @@ deprules
- $dependent[Type] IN ['*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.SecondaryIndexPartial', '*scpb.IndexComment', '*scpb.IndexColumn']
- $index[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex', '*scpb.TemporaryIndex']
- joinOnIndexID($dependent, $index, $table-id, $index-id)
- - $dependent-target[TargetStatus] = TRANSIENT_ABSENT
+ - transient($dependent-target, $index-target)
- $dependent-node[CurrentStatus] = TRANSIENT_ABSENT
- - $index-target[TargetStatus] = ABSENT
- - $index-node[CurrentStatus] = ABSENT
+ - $index-node[CurrentStatus] = TRANSIENT_ABSENT
- joinTargetNode($dependent, $dependent-target, $dependent-node)
- joinTargetNode($index, $index-target, $index-node)
- name: dependents removed before index
@@ -1632,10 +1735,9 @@ deprules
- $dependent[Type] IN ['*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.SecondaryIndexPartial', '*scpb.IndexComment', '*scpb.IndexColumn']
- $index[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex', '*scpb.TemporaryIndex']
- joinOnIndexID($dependent, $index, $table-id, $index-id)
- - $dependent-target[TargetStatus] = ABSENT
+ - toAbsent($dependent-target, $index-target)
- $dependent-node[CurrentStatus] = ABSENT
- - $index-target[TargetStatus] = TRANSIENT_ABSENT
- - $index-node[CurrentStatus] = TRANSIENT_ABSENT
+ - $index-node[CurrentStatus] = ABSENT
- joinTargetNode($dependent, $dependent-target, $dependent-node)
- joinTargetNode($index, $index-target, $index-node)
- name: descriptor DROPPED in transaction before removal
@@ -1670,7 +1772,7 @@ deprules
to: dependent-node
query:
- $descriptor[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType']
- - $dependent[Type] IN ['*scpb.ColumnFamily', '*scpb.UniqueWithoutIndexConstraint', '*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint', '*scpb.TableComment', '*scpb.RowLevelTTL', '*scpb.TableZoneConfig', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalitySecondaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.ColumnName', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SequenceOwner', '*scpb.ColumnComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.SecondaryIndexPartial', '*scpb.IndexComment', '*scpb.IndexColumn', '*scpb.ConstraintName', '*scpb.ConstraintComment', '*scpb.Namespace', '*scpb.Owner', '*scpb.UserPrivileges', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseComment', '*scpb.SchemaParent', '*scpb.SchemaComment', '*scpb.ObjectParent', '*scpb.EnumTypeValue']
+ - $dependent[Type] IN ['*scpb.ColumnFamily', '*scpb.UniqueWithoutIndexConstraint', '*scpb.ForeignKeyConstraint', '*scpb.TableComment', '*scpb.RowLevelTTL', '*scpb.TableZoneConfig', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalitySecondaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.ColumnName', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SequenceOwner', '*scpb.ColumnComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.SecondaryIndexPartial', '*scpb.IndexComment', '*scpb.IndexColumn', '*scpb.ConstraintName', '*scpb.ConstraintComment', '*scpb.Namespace', '*scpb.Owner', '*scpb.UserPrivileges', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseComment', '*scpb.SchemaParent', '*scpb.SchemaComment', '*scpb.ObjectParent', '*scpb.EnumTypeValue']
- joinOnDescID($descriptor, $dependent, $desc-id)
- toAbsent($descriptor-target, $dependent-target)
- $descriptor-node[CurrentStatus] = DROPPED
@@ -1684,7 +1786,7 @@ deprules
to: referencing-via-attr-node
query:
- $referenced-descriptor[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType']
- - $referencing-via-attr[Type] IN ['*scpb.ColumnFamily', '*scpb.UniqueWithoutIndexConstraint', '*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint', '*scpb.TableComment', '*scpb.RowLevelTTL', '*scpb.TableZoneConfig', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalitySecondaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.ColumnName', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SequenceOwner', '*scpb.ColumnComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.SecondaryIndexPartial', '*scpb.IndexComment', '*scpb.IndexColumn', '*scpb.ConstraintName', '*scpb.ConstraintComment', '*scpb.Namespace', '*scpb.Owner', '*scpb.UserPrivileges', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseComment', '*scpb.SchemaParent', '*scpb.SchemaComment', '*scpb.ObjectParent', '*scpb.EnumTypeValue']
+ - $referencing-via-attr[Type] IN ['*scpb.ColumnFamily', '*scpb.UniqueWithoutIndexConstraint', '*scpb.ForeignKeyConstraint', '*scpb.TableComment', '*scpb.RowLevelTTL', '*scpb.TableZoneConfig', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalitySecondaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.ColumnName', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SequenceOwner', '*scpb.ColumnComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.SecondaryIndexPartial', '*scpb.IndexComment', '*scpb.IndexColumn', '*scpb.ConstraintName', '*scpb.ConstraintComment', '*scpb.Namespace', '*scpb.Owner', '*scpb.UserPrivileges', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseComment', '*scpb.SchemaParent', '*scpb.SchemaComment', '*scpb.ObjectParent', '*scpb.EnumTypeValue']
- joinReferencedDescID($referencing-via-attr, $referenced-descriptor, $desc-id)
- toAbsent($referenced-descriptor-target, $referencing-via-attr-target)
- $referenced-descriptor-node[CurrentStatus] = DROPPED
@@ -1699,7 +1801,7 @@ deprules
- $referenced-descriptor[Type] = '*scpb.Sequence'
- $referenced-descriptor[DescID] = $seqID
- $referencing-via-expr[ReferencedSequenceIDs] CONTAINS $seqID
- - $referencing-via-expr[Type] IN ['*scpb.CheckConstraint', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SecondaryIndexPartial']
+ - $referencing-via-expr[Type] IN ['*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SecondaryIndexPartial']
- toAbsent($referenced-descriptor-target, $referencing-via-expr-target)
- $referenced-descriptor-node[CurrentStatus] = DROPPED
- $referencing-via-expr-node[CurrentStatus] = ABSENT
@@ -1713,7 +1815,7 @@ deprules
- $referenced-descriptor[Type] IN ['*scpb.EnumType', '*scpb.AliasType']
- $referenced-descriptor[DescID] = $fromDescID
- $referencing-via-type[ReferencedTypeIDs] CONTAINS $fromDescID
- - $referencing-via-type[Type] IN ['*scpb.CheckConstraint', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SecondaryIndexPartial']
+ - $referencing-via-type[Type] IN ['*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SecondaryIndexPartial']
- toAbsent($referenced-descriptor-target, $referencing-via-type-target)
- $referenced-descriptor-node[CurrentStatus] = DROPPED
- $referencing-via-type-node[CurrentStatus] = ABSENT
@@ -1807,8 +1909,9 @@ deprules
- $index[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex', '*scpb.TemporaryIndex']
- $dependent[Type] IN ['*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.SecondaryIndexPartial', '*scpb.IndexComment', '*scpb.IndexColumn']
- joinOnIndexID($index, $dependent, $table-id, $index-id)
- - toAbsent($index-target, $dependent-target)
- - $index-node[CurrentStatus] = VALIDATED
+ - $index-target[TargetStatus] = TRANSIENT_ABSENT
+ - $index-node[CurrentStatus] = TRANSIENT_VALIDATED
+ - $dependent-target[TargetStatus] = ABSENT
- $dependent-node[CurrentStatus] = ABSENT
- joinTargetNode($index, $index-target, $index-node)
- joinTargetNode($dependent, $dependent-target, $dependent-node)
@@ -1834,9 +1937,9 @@ deprules
- $index[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex', '*scpb.TemporaryIndex']
- $dependent[Type] IN ['*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.SecondaryIndexPartial', '*scpb.IndexComment', '*scpb.IndexColumn']
- joinOnIndexID($index, $dependent, $table-id, $index-id)
- - transient($index-target, $dependent-target)
- - $index-node[CurrentStatus] = TRANSIENT_VALIDATED
- - $dependent-node[CurrentStatus] = TRANSIENT_ABSENT
+ - toAbsent($index-target, $dependent-target)
+ - $index-node[CurrentStatus] = VALIDATED
+ - $dependent-node[CurrentStatus] = ABSENT
- joinTargetNode($index, $index-target, $index-node)
- joinTargetNode($dependent, $dependent-target, $dependent-node)
- name: index no longer public before dependents
@@ -1847,10 +1950,9 @@ deprules
- $index[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex', '*scpb.TemporaryIndex']
- $dependent[Type] IN ['*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.SecondaryIndexPartial', '*scpb.IndexComment', '*scpb.IndexColumn']
- joinOnIndexID($index, $dependent, $table-id, $index-id)
- - $index-target[TargetStatus] = TRANSIENT_ABSENT
+ - transient($index-target, $dependent-target)
- $index-node[CurrentStatus] = TRANSIENT_VALIDATED
- - $dependent-target[TargetStatus] = ABSENT
- - $dependent-node[CurrentStatus] = ABSENT
+ - $dependent-node[CurrentStatus] = TRANSIENT_ABSENT
- joinTargetNode($index, $index-target, $index-node)
- joinTargetNode($dependent, $dependent-target, $dependent-node)
- name: index removed before garbage collection
diff --git a/pkg/sql/schemachanger/scplan/internal/rules/testdata/oprules b/pkg/sql/schemachanger/scplan/internal/rules/testdata/oprules
index 340e78d09504..d77e5bc50f7b 100644
--- a/pkg/sql/schemachanger/scplan/internal/rules/testdata/oprules
+++ b/pkg/sql/schemachanger/scplan/internal/rules/testdata/oprules
@@ -92,6 +92,11 @@ nodeNotExistsWithStatusIn_VALIDATED_TRANSIENT_WRITE_ONLY_MERGE_ONLY_TRANSIENT_ME
- $n[Type] = '*screl.Node'
- $n[Target] = $sharedTarget
- $n[CurrentStatus] IN [VALIDATED, TRANSIENT_WRITE_ONLY, MERGE_ONLY, TRANSIENT_MERGE_ONLY, MERGED, TRANSIENT_MERGED]
+nodeNotExistsWithStatusIn_WRITE_ONLY($sharedTarget):
+ not-join:
+ - $n[Type] = '*screl.Node'
+ - $n[Target] = $sharedTarget
+ - $n[CurrentStatus] IN [WRITE_ONLY]
sourceIndexIsSet($index):
- $index[SourceIndexID] != 0
toAbsent($target1, $target2):
@@ -149,12 +154,13 @@ oprules
from: constraint-node
query:
- $relation[Type] IN ['*scpb.Table', '*scpb.View']
- - $constraint[Type] = '*scpb.UniqueWithoutIndexConstraint'
+ - $constraint[Type] IN ['*scpb.CheckConstraint', '*scpb.UniqueWithoutIndexConstraint']
- joinOnDescID($relation, $constraint, $relation-id)
- joinTarget($relation, $relation-target)
- $relation-target[TargetStatus] = ABSENT
- joinTargetNode($constraint, $constraint-target, $constraint-node)
- $constraint-target[TargetStatus] = ABSENT
+ - $constraint-node[CurrentStatus] = PUBLIC
- name: skip element removal ops on descriptor drop
from: dep-node
query:
diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_index b/pkg/sql/schemachanger/scplan/testdata/drop_index
index 83280682b170..77673ec28c50 100644
--- a/pkg/sql/schemachanger/scplan/testdata/drop_index
+++ b/pkg/sql/schemachanger/scplan/testdata/drop_index
@@ -369,12 +369,13 @@ DROP INDEX idx2 CASCADE
ops
DROP INDEX idx3 CASCADE
----
-StatementPhase stage 1 of 1 with 5 MutationType ops
+StatementPhase stage 1 of 1 with 6 MutationType ops
transitions:
[[Column:{DescID: 104, ColumnID: 5}, ABSENT], PUBLIC] -> WRITE_ONLY
[[ColumnName:{DescID: 104, Name: crdb_internal_i_shard_16, ColumnID: 5}, ABSENT], PUBLIC] -> ABSENT
[[SecondaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 0}, ABSENT], PUBLIC] -> VALIDATED
[[IndexName:{DescID: 104, Name: idx3, IndexID: 6}, ABSENT], PUBLIC] -> ABSENT
+ [[CheckConstraint:{DescID: 104, ConstraintID: 2}, ABSENT], PUBLIC] -> VALIDATED
ops:
*scop.MakePublicSecondaryIndexWriteOnly
IndexID: 6
@@ -383,6 +384,9 @@ StatementPhase stage 1 of 1 with 5 MutationType ops
IndexID: 6
Name: crdb_internal_index_6_name_placeholder
TableID: 104
+ *scop.MakePublicCheckConstraintValidated
+ ConstraintID: 2
+ TableID: 104
*scop.MakePublicColumnWriteOnly
ColumnID: 5
TableID: 104
@@ -406,19 +410,15 @@ StatementPhase stage 1 of 1 with 5 MutationType ops
ColumnID: 5
Name: crdb_internal_column_5_name_placeholder
TableID: 104
-PreCommitPhase stage 1 of 1 with 4 MutationType ops
+PreCommitPhase stage 1 of 1 with 3 MutationType ops
transitions:
[[IndexColumn:{DescID: 104, ColumnID: 5, IndexID: 6}, ABSENT], PUBLIC] -> ABSENT
[[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 6}, ABSENT], PUBLIC] -> ABSENT
[[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 6}, ABSENT], PUBLIC] -> ABSENT
- [[CheckConstraint:{DescID: 104, ConstraintID: 2}, ABSENT], PUBLIC] -> ABSENT
[[ConstraintName:{DescID: 104, Name: check_crdb_internal_i_shard_16, ConstraintID: 2}, ABSENT], PUBLIC] -> ABSENT
ops:
*scop.NotImplemented
ElementType: scpb.ConstraintName
- *scop.RemoveCheckConstraint
- ConstraintID: 2
- TableID: 104
*scop.SetJobStateOnDescriptor
DescriptorID: 104
Initialize: true
@@ -429,19 +429,23 @@ PreCommitPhase stage 1 of 1 with 4 MutationType ops
- 104
JobID: 1
NonCancelable: true
- RunningStatus: PostCommitNonRevertiblePhase stage 1 of 2 with 2 MutationType ops pending
+ RunningStatus: PostCommitNonRevertiblePhase stage 1 of 2 with 3 MutationType ops pending
Statements:
- statement: DROP INDEX idx3 CASCADE
redactedstatement: DROP INDEX ‹defaultdb›.public.‹t1›@‹idx3› CASCADE
statementtag: DROP INDEX
-PostCommitNonRevertiblePhase stage 1 of 2 with 4 MutationType ops
+PostCommitNonRevertiblePhase stage 1 of 2 with 5 MutationType ops
transitions:
[[Column:{DescID: 104, ColumnID: 5}, ABSENT], WRITE_ONLY] -> DELETE_ONLY
[[SecondaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 0}, ABSENT], VALIDATED] -> DELETE_ONLY
+ [[CheckConstraint:{DescID: 104, ConstraintID: 2}, ABSENT], VALIDATED] -> ABSENT
ops:
*scop.MakeWriteOnlyColumnDeleteOnly
ColumnID: 5
TableID: 104
+ *scop.RemoveCheckConstraint
+ ConstraintID: 2
+ TableID: 104
*scop.MakeWriteOnlyIndexDeleteOnly
IndexID: 6
TableID: 104
@@ -509,6 +513,18 @@ PostCommitNonRevertiblePhase stage 2 of 2 with 6 MutationType ops
deps
DROP INDEX idx3 CASCADE
----
+- from: [CheckConstraint:{DescID: 104, ConstraintID: 2}, PUBLIC]
+ to: [CheckConstraint:{DescID: 104, ConstraintID: 2}, VALIDATED]
+ kind: PreviousTransactionPrecedence
+ rule: CheckConstraint transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED
+- from: [CheckConstraint:{DescID: 104, ConstraintID: 2}, VALIDATED]
+ to: [CheckConstraint:{DescID: 104, ConstraintID: 2}, ABSENT]
+ kind: PreviousTransactionPrecedence
+ rule: CheckConstraint transitions to ABSENT uphold 2-version invariant: VALIDATED->ABSENT
+- from: [CheckConstraint:{DescID: 104, ConstraintID: 2}, VALIDATED]
+ to: [ConstraintName:{DescID: 104, Name: check_crdb_internal_i_shard_16, ConstraintID: 2}, ABSENT]
+ kind: Precedence
+ rule: constraint no longer public before dependents
- from: [Column:{DescID: 104, ColumnID: 5}, DELETE_ONLY]
to: [Column:{DescID: 104, ColumnID: 5}, ABSENT]
kind: PreviousTransactionPrecedence
@@ -543,8 +559,8 @@ DROP INDEX idx3 CASCADE
rules: [dependents removed before column; column type removed right before column when not dropping relation]
- from: [ConstraintName:{DescID: 104, Name: check_crdb_internal_i_shard_16, ConstraintID: 2}, ABSENT]
to: [CheckConstraint:{DescID: 104, ConstraintID: 2}, ABSENT]
- kind: SameStagePrecedence
- rule: constraint dependent absent right before constraint
+ kind: Precedence
+ rule: dependents removed before constraint
- from: [IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 6}, ABSENT]
to: [SecondaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 0}, ABSENT]
kind: Precedence
diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_table b/pkg/sql/schemachanger/scplan/testdata/drop_table
index 4c4f85d007f0..47c0319b1fbb 100644
--- a/pkg/sql/schemachanger/scplan/testdata/drop_table
+++ b/pkg/sql/schemachanger/scplan/testdata/drop_table
@@ -164,6 +164,20 @@ PreCommitPhase stage 1 of 1 with 43 MutationType ops
TypeIDs:
- 107
- 108
+ *scop.RemoveForeignKeyBackReference
+ OriginConstraintID: 2
+ OriginTableID: 109
+ ReferencedTableID: 104
+ *scop.RemoveForeignKeyConstraint
+ ConstraintID: 2
+ TableID: 109
+ *scop.RemoveForeignKeyBackReference
+ OriginConstraintID: 3
+ OriginTableID: 109
+ ReferencedTableID: 105
+ *scop.RemoveForeignKeyConstraint
+ ConstraintID: 3
+ TableID: 109
*scop.MarkDescriptorAsDropped
DescriptorID: 110
*scop.RemoveAllTableComments
@@ -232,20 +246,6 @@ PreCommitPhase stage 1 of 1 with 43 MutationType ops
SourceElementID: 1
SubWorkID: 1
TableID: 109
- *scop.RemoveForeignKeyBackReference
- OriginConstraintID: 2
- OriginTableID: 109
- ReferencedTableID: 104
- *scop.RemoveForeignKeyConstraint
- ConstraintID: 2
- TableID: 109
- *scop.RemoveForeignKeyBackReference
- OriginConstraintID: 3
- OriginTableID: 109
- ReferencedTableID: 105
- *scop.RemoveForeignKeyConstraint
- ConstraintID: 3
- TableID: 109
*scop.DrainDescriptorName
Namespace:
DatabaseID: 100
@@ -740,18 +740,6 @@ DROP TABLE defaultdb.shipments CASCADE;
to: [Column:{DescID: 111, ColumnID: 4294967295}, ABSENT]
kind: Precedence
rule: dependents removed before column
-- from: [ConstraintComment:{DescID: 109, ConstraintID: 2, Comment: customer is not god}, ABSENT]
- to: [ForeignKeyConstraint:{DescID: 109, ConstraintID: 2, ReferencedDescID: 104}, ABSENT]
- kind: SameStagePrecedence
- rule: constraint dependent absent right before constraint
-- from: [ConstraintName:{DescID: 109, Name: fk_customers, ConstraintID: 2}, ABSENT]
- to: [ForeignKeyConstraint:{DescID: 109, ConstraintID: 2, ReferencedDescID: 104}, ABSENT]
- kind: SameStagePrecedence
- rule: constraint dependent absent right before constraint
-- from: [ConstraintName:{DescID: 109, Name: fk_orders, ConstraintID: 3}, ABSENT]
- to: [ForeignKeyConstraint:{DescID: 109, ConstraintID: 3, ReferencedDescID: 105}, ABSENT]
- kind: SameStagePrecedence
- rule: constraint dependent absent right before constraint
- from: [IndexColumn:{DescID: 109, ColumnID: 1, IndexID: 1}, ABSENT]
to: [Column:{DescID: 109, ColumnID: 1}, ABSENT]
kind: Precedence
@@ -1208,6 +1196,7 @@ StatementPhase stage 1 of 1 with 1 MutationType op
[[Column:{DescID: 114, ColumnID: 4294967294}, ABSENT], PUBLIC] -> WRITE_ONLY
[[PrimaryIndex:{DescID: 114, IndexID: 1, ConstraintID: 1}, ABSENT], PUBLIC] -> VALIDATED
[[SecondaryIndex:{DescID: 114, IndexID: 2, ConstraintID: 0}, ABSENT], PUBLIC] -> VALIDATED
+ [[CheckConstraint:{DescID: 114, ReferencedTypeIDs: [112 113], ConstraintID: 2}, ABSENT], PUBLIC] -> VALIDATED
ops:
*scop.MarkDescriptorAsSyntheticallyDropped
DescriptorID: 114
@@ -1247,7 +1236,7 @@ PreCommitPhase stage 1 of 1 with 21 MutationType ops
[[SecondaryIndexPartial:{DescID: 114, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT
[[SecondaryIndex:{DescID: 114, IndexID: 2, ConstraintID: 0}, ABSENT], VALIDATED] -> ABSENT
[[IndexName:{DescID: 114, Name: i, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT
- [[CheckConstraint:{DescID: 114, ReferencedTypeIDs: [112 113], ConstraintID: 2}, ABSENT], PUBLIC] -> ABSENT
+ [[CheckConstraint:{DescID: 114, ReferencedTypeIDs: [112 113], ConstraintID: 2}, ABSENT], VALIDATED] -> ABSENT
[[ConstraintName:{DescID: 114, Name: check, ConstraintID: 2}, ABSENT], PUBLIC] -> ABSENT
ops:
*scop.MarkDescriptorAsDropped
diff --git a/pkg/sql/schemachanger/testdata/end_to_end/drop_index_hash_sharded_index b/pkg/sql/schemachanger/testdata/end_to_end/drop_index_hash_sharded_index
index 8b7ca3b1e780..5530cb242fb8 100644
--- a/pkg/sql/schemachanger/testdata/end_to_end/drop_index_hash_sharded_index
+++ b/pkg/sql/schemachanger/testdata/end_to_end/drop_index_hash_sharded_index
@@ -12,19 +12,21 @@ begin transaction #1
# begin StatementPhase
checking for feature: DROP INDEX
increment telemetry for sql.schema.drop_index
-## StatementPhase stage 1 of 1 with 5 MutationType ops
+## StatementPhase stage 1 of 1 with 6 MutationType ops
upsert descriptor #104
- ...
- - 3
- constraintId: 2
+ table:
+ - checks:
+ - - columnIds:
+ - - 3
+ - constraintId: 2
- expr: crdb_internal_j_shard_16 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, 4:::INT8,
- 5:::INT8, 6:::INT8, 7:::INT8, 8:::INT8, 9:::INT8, 10:::INT8, 11:::INT8, 12:::INT8,
- 13:::INT8, 14:::INT8, 15:::INT8)
- + expr: crdb_internal_column_3_name_placeholder IN (0:::INT8, 1:::INT8, 2:::INT8,
- + 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8, 8:::INT8, 9:::INT8, 10:::INT8,
- + 11:::INT8, 12:::INT8, 13:::INT8, 14:::INT8, 15:::INT8)
- fromHashShardedColumn: true
- name: check_crdb_internal_j_shard_16
+ - fromHashShardedColumn: true
+ - name: check_crdb_internal_j_shard_16
+ + checks: []
+ columns:
+ - id: 1
...
oid: 20
width: 64
@@ -102,6 +104,23 @@ upsert descriptor #104
+ version: 3
+ mutationId: 2
+ state: WRITE_ONLY
+ + - constraint:
+ + check:
+ + columnIds:
+ + - 3
+ + constraintId: 2
+ + expr: crdb_internal_column_3_name_placeholder IN (0:::INT8, 1:::INT8, 2:::INT8,
+ + 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8, 8:::INT8, 9:::INT8, 10:::INT8,
+ + 11:::INT8, 12:::INT8, 13:::INT8, 14:::INT8, 15:::INT8)
+ + fromHashShardedColumn: true
+ + name: check_crdb_internal_j_shard_16
+ + validity: Dropping
+ + foreignKey: {}
+ + name: check_crdb_internal_j_shard_16
+ + uniqueWithoutIndexConstraint: {}
+ + direction: DROP
+ + mutationId: 2
+ + state: WRITE_ONLY
+ - column:
+ computeExpr: mod(fnv32(crdb_internal.datums_to_bytes(j)), 16:::INT8)
+ hidden: true
@@ -124,21 +143,8 @@ upsert descriptor #104
+ version: "9"
# end StatementPhase
# begin PreCommitPhase
-## PreCommitPhase stage 1 of 1 with 4 MutationType ops
+## PreCommitPhase stage 1 of 1 with 3 MutationType ops
upsert descriptor #104
- table:
- - checks:
- - - columnIds:
- - - 3
- - constraintId: 2
- - expr: crdb_internal_column_3_name_placeholder IN (0:::INT8, 1:::INT8, 2:::INT8,
- - 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8, 8:::INT8, 9:::INT8, 10:::INT8,
- - 11:::INT8, 12:::INT8, 13:::INT8, 14:::INT8, 15:::INT8)
- - fromHashShardedColumn: true
- - name: check_crdb_internal_j_shard_16
- + checks: []
- columns:
- - id: 1
...
createAsOfTime:
wallTime: "1640995200000000000"
@@ -170,12 +176,29 @@ notified job registry to adopt jobs: [1]
begin transaction #2
commit transaction #2
begin transaction #3
-## PostCommitNonRevertiblePhase stage 1 of 2 with 4 MutationType ops
+## PostCommitNonRevertiblePhase stage 1 of 2 with 5 MutationType ops
upsert descriptor #104
...
version: 3
mutationId: 2
- state: WRITE_ONLY
+ - - constraint:
+ - check:
+ - columnIds:
+ - - 3
+ - constraintId: 2
+ - expr: crdb_internal_column_3_name_placeholder IN (0:::INT8, 1:::INT8, 2:::INT8,
+ - 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8, 8:::INT8, 9:::INT8, 10:::INT8,
+ - 11:::INT8, 12:::INT8, 13:::INT8, 14:::INT8, 15:::INT8)
+ - fromHashShardedColumn: true
+ - name: check_crdb_internal_j_shard_16
+ - validity: Dropping
+ - foreignKey: {}
+ - name: check_crdb_internal_j_shard_16
+ - uniqueWithoutIndexConstraint: {}
+ - direction: DROP
+ - mutationId: 2
+ - state: WRITE_ONLY
+ state: DELETE_ONLY
- column:
computeExpr: mod(fnv32(crdb_internal.datums_to_bytes(j)), 16:::INT8)
diff --git a/pkg/sql/schemachanger/testdata/explain/drop_index_hash_sharded_index b/pkg/sql/schemachanger/testdata/explain/drop_index_hash_sharded_index
index d4f32531d49a..7206779d6aa7 100644
--- a/pkg/sql/schemachanger/testdata/explain/drop_index_hash_sharded_index
+++ b/pkg/sql/schemachanger/testdata/explain/drop_index_hash_sharded_index
@@ -8,37 +8,39 @@ EXPLAIN (ddl) DROP INDEX idx CASCADE;
Schema change plan for DROP INDEX ‹defaultdb›.‹public›.‹t›@‹idx› CASCADE;
├── StatementPhase
│ └── Stage 1 of 1 in StatementPhase
- │ ├── 4 elements transitioning toward ABSENT
+ │ ├── 5 elements transitioning toward ABSENT
│ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 104, ColumnID: 3}
│ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104, Name: crdb_internal_j_shard_16, ColumnID: 3}
│ │ ├── PUBLIC → VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0}
- │ │ └── PUBLIC → ABSENT IndexName:{DescID: 104, Name: idx, IndexID: 2}
- │ └── 5 Mutation operations
+ │ │ ├── PUBLIC → ABSENT IndexName:{DescID: 104, Name: idx, IndexID: 2}
+ │ │ └── PUBLIC → VALIDATED CheckConstraint:{DescID: 104, ConstraintID: 2}
+ │ └── 6 Mutation operations
│ ├── MakePublicSecondaryIndexWriteOnly {"IndexID":2,"TableID":104}
│ ├── SetIndexName {"IndexID":2,"Name":"crdb_internal_in...","TableID":104}
+ │ ├── MakePublicCheckConstraintValidated {"ConstraintID":2,"TableID":104}
│ ├── MakePublicColumnWriteOnly {"ColumnID":3,"TableID":104}
│ ├── LogEvent {"TargetStatus":1}
│ └── SetColumnName {"ColumnID":3,"Name":"crdb_internal_co...","TableID":104}
├── PreCommitPhase
│ └── Stage 1 of 1 in PreCommitPhase
- │ ├── 5 elements transitioning toward ABSENT
+ │ ├── 4 elements transitioning toward ABSENT
│ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2}
│ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2}
│ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2}
- │ │ ├── PUBLIC → ABSENT CheckConstraint:{DescID: 104, ConstraintID: 2}
│ │ └── PUBLIC → ABSENT ConstraintName:{DescID: 104, Name: check_crdb_internal_j_shard_16, ConstraintID: 2}
- │ └── 4 Mutation operations
+ │ └── 3 Mutation operations
│ ├── NotImplemented {"ElementType":"scpb.ConstraintN..."}
- │ ├── RemoveCheckConstraint {"ConstraintID":2,"TableID":104}
│ ├── SetJobStateOnDescriptor {"DescriptorID":104,"Initialize":true}
│ └── CreateSchemaChangerJob {"NonCancelable":true,"RunningStatus":"PostCommitNonRev..."}
└── PostCommitNonRevertiblePhase
├── Stage 1 of 2 in PostCommitNonRevertiblePhase
- │ ├── 2 elements transitioning toward ABSENT
+ │ ├── 3 elements transitioning toward ABSENT
│ │ ├── WRITE_ONLY → DELETE_ONLY Column:{DescID: 104, ColumnID: 3}
- │ │ └── VALIDATED → DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0}
- │ └── 4 Mutation operations
+ │ │ ├── VALIDATED → DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0}
+ │ │ └── VALIDATED → ABSENT CheckConstraint:{DescID: 104, ConstraintID: 2}
+ │ └── 5 Mutation operations
│ ├── MakeWriteOnlyColumnDeleteOnly {"ColumnID":3,"TableID":104}
+ │ ├── RemoveCheckConstraint {"ConstraintID":2,"TableID":104}
│ ├── MakeWriteOnlyIndexDeleteOnly {"IndexID":2,"TableID":104}
│ ├── SetJobStateOnDescriptor {"DescriptorID":104}
│ └── UpdateSchemaChangerJob {"IsNonCancelable":true,"RunningStatus":"PostCommitNonRev..."}
diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_index_hash_sharded_index b/pkg/sql/schemachanger/testdata/explain_verbose/drop_index_hash_sharded_index
index 7476201e73bc..dc7207458dea 100644
--- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_index_hash_sharded_index
+++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_index_hash_sharded_index
@@ -11,7 +11,7 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE;
│ │
│ └── • Stage 1 of 1 in StatementPhase
│ │
-│ ├── • 4 elements transitioning toward ABSENT
+│ ├── • 5 elements transitioning toward ABSENT
│ │ │
│ │ ├── • Column:{DescID: 104, ColumnID: 3}
│ │ │ │ PUBLIC → WRITE_ONLY
@@ -34,13 +34,19 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE;
│ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0}
│ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED"
│ │ │
-│ │ └── • IndexName:{DescID: 104, Name: idx, IndexID: 2}
-│ │ │ PUBLIC → ABSENT
+│ │ ├── • IndexName:{DescID: 104, Name: idx, IndexID: 2}
+│ │ │ │ PUBLIC → ABSENT
+│ │ │ │
+│ │ │ └── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0}
+│ │ │ rule: "index no longer public before dependents"
+│ │ │
+│ │ └── • CheckConstraint:{DescID: 104, ConstraintID: 2}
+│ │ │ PUBLIC → VALIDATED
│ │ │
-│ │ └── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0}
-│ │ rule: "index no longer public before dependents"
+│ │ └── • PreviousTransactionPrecedence dependency from PUBLIC CheckConstraint:{DescID: 104, ConstraintID: 2}
+│ │ rule: "CheckConstraint transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED"
│ │
-│ └── • 5 Mutation operations
+│ └── • 6 Mutation operations
│ │
│ ├── • MakePublicSecondaryIndexWriteOnly
│ │ IndexID: 2
@@ -51,6 +57,10 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE;
│ │ Name: crdb_internal_index_2_name_placeholder
│ │ TableID: 104
│ │
+│ ├── • MakePublicCheckConstraintValidated
+│ │ ConstraintID: 2
+│ │ TableID: 104
+│ │
│ ├── • MakePublicColumnWriteOnly
│ │ ColumnID: 3
│ │ TableID: 104
@@ -81,7 +91,7 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE;
│ │
│ └── • Stage 1 of 1 in PreCommitPhase
│ │
-│ ├── • 5 elements transitioning toward ABSENT
+│ ├── • 4 elements transitioning toward ABSENT
│ │ │
│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2}
│ │ │ │ PUBLIC → ABSENT
@@ -113,24 +123,17 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE;
│ │ │ └── • skip PUBLIC → ABSENT operations
│ │ │ rule: "skip index-column removal ops on index removal"
│ │ │
-│ │ ├── • CheckConstraint:{DescID: 104, ConstraintID: 2}
-│ │ │ │ PUBLIC → ABSENT
-│ │ │ │
-│ │ │ └── • SameStagePrecedence dependency from ABSENT ConstraintName:{DescID: 104, Name: check_crdb_internal_j_shard_16, ConstraintID: 2}
-│ │ │ rule: "constraint dependent absent right before constraint"
-│ │ │
│ │ └── • ConstraintName:{DescID: 104, Name: check_crdb_internal_j_shard_16, ConstraintID: 2}
-│ │ PUBLIC → ABSENT
+│ │ │ PUBLIC → ABSENT
+│ │ │
+│ │ └── • Precedence dependency from VALIDATED CheckConstraint:{DescID: 104, ConstraintID: 2}
+│ │ rule: "constraint no longer public before dependents"
│ │
-│ └── • 4 Mutation operations
+│ └── • 3 Mutation operations
│ │
│ ├── • NotImplemented
│ │ ElementType: scpb.ConstraintName
│ │
-│ ├── • RemoveCheckConstraint
-│ │ ConstraintID: 2
-│ │ TableID: 104
-│ │
│ ├── • SetJobStateOnDescriptor
│ │ DescriptorID: 104
│ │ Initialize: true
@@ -142,7 +145,7 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE;
│ - 104
│ JobID: 1
│ NonCancelable: true
-│ RunningStatus: PostCommitNonRevertiblePhase stage 1 of 2 with 2 MutationType ops pending
+│ RunningStatus: PostCommitNonRevertiblePhase stage 1 of 2 with 3 MutationType ops pending
│ Statements:
│ - statement: DROP INDEX idx CASCADE
│ redactedstatement: DROP INDEX ‹defaultdb›.‹public›.‹t›@‹idx› CASCADE
@@ -152,7 +155,7 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE;
│
├── • Stage 1 of 2 in PostCommitNonRevertiblePhase
│ │
- │ ├── • 2 elements transitioning toward ABSENT
+ │ ├── • 3 elements transitioning toward ABSENT
│ │ │
│ │ ├── • Column:{DescID: 104, ColumnID: 3}
│ │ │ │ WRITE_ONLY → DELETE_ONLY
@@ -160,18 +163,31 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE;
│ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3}
│ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY"
│ │ │
- │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0}
- │ │ │ VALIDATED → DELETE_ONLY
+ │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0}
+ │ │ │ │ VALIDATED → DELETE_ONLY
+ │ │ │ │
+ │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0}
+ │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY"
+ │ │ │
+ │ │ └── • CheckConstraint:{DescID: 104, ConstraintID: 2}
+ │ │ │ VALIDATED → ABSENT
│ │ │
- │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0}
- │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY"
+ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED CheckConstraint:{DescID: 104, ConstraintID: 2}
+ │ │ │ rule: "CheckConstraint transitions to ABSENT uphold 2-version invariant: VALIDATED->ABSENT"
+ │ │ │
+ │ │ └── • Precedence dependency from ABSENT ConstraintName:{DescID: 104, Name: check_crdb_internal_j_shard_16, ConstraintID: 2}
+ │ │ rule: "dependents removed before constraint"
│ │
- │ └── • 4 Mutation operations
+ │ └── • 5 Mutation operations
│ │
│ ├── • MakeWriteOnlyColumnDeleteOnly
│ │ ColumnID: 3
│ │ TableID: 104
│ │
+ │ ├── • RemoveCheckConstraint
+ │ │ ConstraintID: 2
+ │ │ TableID: 104
+ │ │
│ ├── • MakeWriteOnlyIndexDeleteOnly
│ │ IndexID: 2
│ │ TableID: 104
From c5746a33d723732c1be020a1403d601a66ade65d Mon Sep 17 00:00:00 2001
From: Ricky Stewart
Date: Tue, 1 Nov 2022 09:49:54 -0500
Subject: [PATCH 7/7] bazci: fix issue posting
This block got accidentally deleted in
`61161542c910d82c86e4c8ccb6fd2946bba9ab8d`.
Release note: None
Epic: None
---
pkg/cmd/bazci/bazci.go | 13 +++++++++++++
1 file changed, 13 insertions(+)
diff --git a/pkg/cmd/bazci/bazci.go b/pkg/cmd/bazci/bazci.go
index 29d2fac3429f..d3d445aaca1b 100644
--- a/pkg/cmd/bazci/bazci.go
+++ b/pkg/cmd/bazci/bazci.go
@@ -313,6 +313,19 @@ func bazciImpl(cmd *cobra.Command, args []string) (retErr error) {
return
}
args = append(args, fmt.Sprintf("--bes_backend=grpc://127.0.0.1:%d", port))
+ if shouldProcessTestFailures {
+ f, createTempErr := os.CreateTemp(artifactsDir, "test.json.txt")
+ if createTempErr != nil {
+ retErr = createTempErr
+ return
+ }
+ goTestJSONOutputFilePath = f.Name()
+ // Closing the file because we will not use the file pointer.
+ if retErr = f.Close(); retErr != nil {
+ return
+ }
+ args = append(args, "--test_env", goTestJSONOutputFilePath)
+ }
fmt.Println("running bazel w/ args: ", shellescape.QuoteCommand(args))
bazelCmd := exec.Command("bazel", args...)
bazelCmd.Stdout = os.Stdout
| |