Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
89778: schemachanger: Enable adding/dropping path of check constraints r=Xiang-Gu a=Xiang-Gu

See each commit message for deatils.

Commit 1 (easy to review): Fixed a careless bug in existing code;

Commit 2 (easy to review): Added a bool field, `FromHashShardedIndex`, in the protobuf
definition of the `CheckConstraint` element.

Commit 3 (meaty commit): Enable adding/dropping path of check constraints.
```
ABSENT <==> WRITE_ONLY <==> VALIDATED <==> PUBLIC
```

Informs #89665


90956: builtins: add timestamp to crdb_internal.scan/list_sql_keys_in_range r=stevendanna a=adityamaru

This change adds a third return column to the builtins
that corresponds to the timestamp at which the value for
a particular key was written.

Release note (sql change): `crdb_internal.scan` and
`crdb_internal.list_sql_keys_in_range` return
the timestamp at which the value for a key was written, in
addition to the raw key and value.

91024: backupccl: ensure restore2TB/nodes=10/with-pause pauses at least once r=msbutler a=msbutler

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

91055: test: fix error on test r=maryliag a=maryliag

The PR #90862 introduced an error on generated code. This PR fixes the missing parameters used for tests.

Epic: None

Release note: None

91061: bazci: fix issue posting r=nicktrav a=rickystewart

This block got accidentally deleted in
`61161542c910d82c86e4c8ccb6fd2946bba9ab8d`.

Release note: None
Epic: None

Co-authored-by: Xiang Gu <[email protected]>
Co-authored-by: adityamaru <[email protected]>
Co-authored-by: Michael Butler <[email protected]>
Co-authored-by: maryliag <[email protected]>
Co-authored-by: Ricky Stewart <[email protected]>
  • Loading branch information
6 people committed Nov 1, 2022
6 parents 9c035ab + a7f3ae6 + 0af0ea9 + 08ee64b + 9e1a77b + c5746a3 commit 822f37f
Show file tree
Hide file tree
Showing 31 changed files with 705 additions and 272 deletions.
6 changes: 3 additions & 3 deletions docs/generated/sql/functions.md
Original file line number Diff line number Diff line change
Expand Up @@ -1252,9 +1252,9 @@ the locality flag on node startup. Returns an error if no region is set.</p>
<tbody>
<tr><td><a name="aclexplode"></a><code>aclexplode(aclitems: <a href="string.html">string</a>[]) &rarr; tuple{oid AS grantor, oid AS grantee, string AS privilege_type, bool AS is_grantable}</code></td><td><span class="funcdesc"><p>Produces a virtual table containing aclitem stuff (returns no rows as this feature is unsupported in CockroachDB)</p>
</span></td><td>Stable</td></tr>
<tr><td><a name="crdb_internal.scan"></a><code>crdb_internal.scan(span: <a href="bytes.html">bytes</a>[]) &rarr; tuple{bytes AS key, bytes AS value}</code></td><td><span class="funcdesc"><p>Returns the raw keys and values from the specified span</p>
<tr><td><a name="crdb_internal.scan"></a><code>crdb_internal.scan(span: <a href="bytes.html">bytes</a>[]) &rarr; tuple{bytes AS key, bytes AS value, string AS ts}</code></td><td><span class="funcdesc"><p>Returns the raw keys and values from the specified span</p>
</span></td><td>Stable</td></tr>
<tr><td><a name="crdb_internal.scan"></a><code>crdb_internal.scan(start_key: <a href="bytes.html">bytes</a>, end_key: <a href="bytes.html">bytes</a>) &rarr; tuple{bytes AS key, bytes AS value}</code></td><td><span class="funcdesc"><p>Returns the raw keys and values from the specified span</p>
<tr><td><a name="crdb_internal.scan"></a><code>crdb_internal.scan(start_key: <a href="bytes.html">bytes</a>, end_key: <a href="bytes.html">bytes</a>) &rarr; tuple{bytes AS key, bytes AS value, string AS ts}</code></td><td><span class="funcdesc"><p>Returns the raw keys and values with their timestamp from the specified span</p>
</span></td><td>Stable</td></tr>
<tr><td><a name="crdb_internal.testing_callback"></a><code>crdb_internal.testing_callback(name: <a href="string.html">string</a>) &rarr; <a href="int.html">int</a></code></td><td><span class="funcdesc"><p>For internal CRDB testing only. The function calls a callback identified by <code>name</code> registered with the server by the test.</p>
</span></td><td>Volatile</td></tr>
Expand Down Expand Up @@ -3081,7 +3081,7 @@ active for the current transaction.</p>
</span></td><td>Volatile</td></tr>
<tr><td><a name="crdb_internal.lease_holder"></a><code>crdb_internal.lease_holder(key: <a href="bytes.html">bytes</a>) &rarr; <a href="int.html">int</a></code></td><td><span class="funcdesc"><p>This function is used to fetch the leaseholder corresponding to a request key</p>
</span></td><td>Volatile</td></tr>
<tr><td><a name="crdb_internal.list_sql_keys_in_range"></a><code>crdb_internal.list_sql_keys_in_range(range_id: <a href="int.html">int</a>) &rarr; tuple{string AS key, string AS value}</code></td><td><span class="funcdesc"><p>Returns all SQL K/V pairs within the requested range.</p>
<tr><td><a name="crdb_internal.list_sql_keys_in_range"></a><code>crdb_internal.list_sql_keys_in_range(range_id: <a href="int.html">int</a>) &rarr; tuple{string AS key, string AS value, string AS ts}</code></td><td><span class="funcdesc"><p>Returns all SQL K/V pairs within the requested range.</p>
</span></td><td>Volatile</td></tr>
<tr><td><a name="crdb_internal.locality_value"></a><code>crdb_internal.locality_value(key: <a href="string.html">string</a>) &rarr; <a href="string.html">string</a></code></td><td><span class="funcdesc"><p>Returns the value of the specified locality key.</p>
</span></td><td>Stable</td></tr>
Expand Down
13 changes: 13 additions & 0 deletions pkg/cmd/bazci/bazci.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
7 changes: 6 additions & 1 deletion pkg/cmd/roachtest/tests/restore.go
Original file line number Diff line number Diff line change
Expand Up @@ -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])
Expand All @@ -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 {
Expand Down Expand Up @@ -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")
},
})
}
Expand Down
6 changes: 6 additions & 0 deletions pkg/sql/catalog/tabledesc/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/schemachanger/scbuild/testdata/drop_index
Original file line number Diff line number Diff line change
Expand Up @@ -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}
9 changes: 5 additions & 4 deletions pkg/sql/schemachanger/scdecomp/decomp.go
Original file line number Diff line number Diff line change
Expand Up @@ -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(),
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/schemachanger/scdecomp/testdata/type
Original file line number Diff line number Diff line change
Expand Up @@ -179,6 +179,7 @@ ElementState:
- 5
constraintId: 2
expr: s::STRING = name
fromHashShardedColumn: false
referencedColumnIds:
- 3
- 5
Expand Down
29 changes: 28 additions & 1 deletion pkg/sql/schemachanger/scexec/exec_validation.go
Original file line number Diff line number Diff line change
Expand Up @@ -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) {
Expand Down
16 changes: 16 additions & 0 deletions pkg/sql/schemachanger/scexec/scmutationexec/helpers.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down
89 changes: 88 additions & 1 deletion pkg/sql/schemachanger/scexec/scmutationexec/references.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -75,7 +76,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
Expand All @@ -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 {
Expand Down
27 changes: 27 additions & 0 deletions pkg/sql/schemachanger/scop/mutation.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
18 changes: 18 additions & 0 deletions pkg/sql/schemachanger/scop/mutation_visitor_generated.go

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

2 changes: 2 additions & 0 deletions pkg/sql/schemachanger/scpb/elements.proto
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/schemachanger/scpb/scpb.proto
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/schemachanger/scpb/uml/table.puml
Original file line number Diff line number Diff line change
Expand Up @@ -82,6 +82,7 @@ CheckConstraint : TableID
CheckConstraint : ConstraintID
CheckConstraint : []ColumnIDs
CheckConstraint : Expression
CheckConstraint : FromHashShardedColumn

object ForeignKeyConstraint

Expand Down
Loading

0 comments on commit 822f37f

Please sign in to comment.