Skip to content
Draft
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view

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

Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ go_library(
"alter_table_rename_constraint.go",
"alter_table_set_rls_mode.go",
"alter_table_set_schema.go",
"alter_table_set_storage_param.go",
"alter_table_validate_constraint.go",
"comment_on.go",
"configure_zone.go",
Expand Down Expand Up @@ -118,6 +119,7 @@ go_library(
"//pkg/sql/sqltelemetry",
"//pkg/sql/storageparam",
"//pkg/sql/storageparam/indexstorageparam",
"//pkg/sql/storageparam/tablestorageparam",
"//pkg/sql/types",
"//pkg/sql/vecindex",
"//pkg/sql/vecindex/vecpb",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@ var supportedAlterTableStatements = map[reflect.Type]supportedAlterTableCommand{
reflect.TypeOf((*tree.AlterTableSetIdentity)(nil)): {fn: alterTableSetIdentity, on: true, checks: isV261Active},
reflect.TypeOf((*tree.AlterTableAddIdentity)(nil)): {fn: alterTableAddIdentity, on: true, checks: isV261Active},
reflect.TypeOf((*tree.AlterTableSetVisible)(nil)): {fn: alterTableAlterColumnSetVisible, on: true, checks: isV261Active},
reflect.TypeOf((*tree.AlterTableSetStorageParams)(nil)): {fn: AlterTableSetStorageParams, on: true, checks: isV261Active},
}

func init() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -326,7 +326,7 @@ func walkColumnDependencies(
*scpb.UniqueWithoutIndexConstraint, *scpb.CheckConstraint,
*scpb.UniqueWithoutIndexConstraintUnvalidated, *scpb.CheckConstraintUnvalidated,
*scpb.RowLevelTTL, *scpb.PolicyUsingExpr, *scpb.PolicyWithCheckExpr,
*scpb.TriggerDeps, *scpb.ColumnGeneratedAsIdentity, *scpb.ColumnHidden:
*scpb.TriggerDeps, *scpb.ColumnGeneratedAsIdentity, *scpb.ColumnHidden, *scpb.TableStorageParam:
fn(e, op, objType)
case *scpb.ColumnType:
if elt.ColumnID == col.ColumnID {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,127 @@
// Copyright 2025 The Cockroach Authors.
//
// Use of this software is governed by the CockroachDB Software License
// included in the /LICENSE file.

package scbuildstmt

import (
"strings"

"github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scerrors"
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/storageparam"
"github.com/cockroachdb/cockroach/pkg/sql/storageparam/tablestorageparam"
"github.com/cockroachdb/errors"
)

// AlterTableSetStorageParams implements ALTER TABLE ... SET {storage_param} in the declarative schema changer.
func AlterTableSetStorageParams(
b BuildCtx,
tn *tree.TableName,
tbl *scpb.Table,
stmt tree.Statement,
t *tree.AlterTableSetStorageParams,
) {
if err := storageparam.StorageParamPreChecks(
b,
b.EvalCtx(),
false, /* isNewObject */
t.StorageParams,
nil, /* resetParams */
); err != nil {
panic(err)
}
for _, param := range t.StorageParams {
val, err := tablestorageparam.ParseAndValidate(b, b.SemaCtx(), b.EvalCtx(), param)
if err != nil {
panic(err) // tried to set an invalid value for param
}
key := param.Key
if strings.HasPrefix(key, "ttl") {
panic(scerrors.NotImplementedErrorf(t, "set ttl not implemented yet"))
}
if key == catpb.RBRUsingConstraintTableSettingName {
panic(scerrors.NotImplementedErrorf(t, "set rbr not implemented yet"))
}
// schema_locked has a dedicated element and will be handled differently
if handleSetSchemaLockIfNeeded(b, tbl, key, val) {
return
}
// Do extra validation for exclude_data_from_backup
validateExcludeDataFromBackup(b, tbl, key)
currElem := b.QueryByID(tbl.TableID).FilterTableStorageParam().Filter(
func(_ scpb.Status, _ scpb.TargetStatus, e *scpb.TableStorageParam) bool {
return e.Name == key
}).MustGetZeroOrOneElement()
if currElem != nil {
b.Drop(currElem)
}
if val != "" {
newElem := scpb.TableStorageParam{
TableID: tbl.TableID,
Name: key,
Value: val,
}
b.Add(&newElem)
}
}
}

func validateExcludeDataFromBackup(b BuildCtx, tbl *scpb.Table, key string) {
if key != "exclude_data_from_backup" {
return
}
if tbl.IsTemporary {
panic(pgerror.Newf(pgcode.FeatureNotSupported,
"cannot set data in a temporary table to be excluded from backup"))
}
// Check that the table does not have any incoming FK references. During a
// backup, the rows of a table with ephemeral data will not be backed up, and
// could result in a violation of FK constraints on restore. To prevent this,
// we only allow a table with no incoming FK references to be marked as
// ephemeral.
if isTableReferencedByFK(b, tbl) {
panic(errors.New("cannot set data in a table with inbound foreign key constraints to be excluded from backup"))
}
}

// isTableReferencedByFK returns the count of inbound foreign key constraints
// where tbl is the referenced table.
func isTableReferencedByFK(b BuildCtx, tbl *scpb.Table) bool {
hasInboundFK := false
backRefs := b.BackReferences(tbl.TableID)
// Check validated foreign key constraints
backRefs.FilterForeignKeyConstraint().ForEach(func(current scpb.Status, target scpb.TargetStatus, e *scpb.ForeignKeyConstraint) {
if e.ReferencedTableID == tbl.TableID {
hasInboundFK = true
}
})
// Check unvalidated foreign key constraints
backRefs.FilterForeignKeyConstraintUnvalidated().ForEach(func(current scpb.Status, target scpb.TargetStatus, e *scpb.ForeignKeyConstraintUnvalidated) {
if e.ReferencedTableID == tbl.TableID {
hasInboundFK = true
}
})
return hasInboundFK
}

func handleSetSchemaLockIfNeeded(b BuildCtx, tbl *scpb.Table, key, val string) bool {
if key != "schema_locked" {
return false
}
currElem := b.QueryByID(tbl.TableID).FilterTableSchemaLocked().MustGetZeroOrOneElement()
// Drop element if schema is currently locked
if currElem != nil && val == "false" {
b.Drop(currElem)
}
// Add element if schema is currently unlocked
if currElem == nil && val == "true" {
b.Add(&scpb.TableSchemaLocked{TableID: tbl.TableID})
}
return true
}
26 changes: 26 additions & 0 deletions pkg/sql/schemachanger/scdecomp/decomp.go
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@ import (
"context"
"fmt"
"reflect"
"strings"

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/geo/geopb"
Expand Down Expand Up @@ -487,6 +488,31 @@ func (w *walkCtx) walkRelation(tbl catalog.TableDescriptor) {
JobIDs: tbl.TableDesc().LDRJobIDs,
})
}
w.walkStorageParams(tbl)
}

// walkStorageParams walks through table storage parameters and creates
// TableStorageParam elements for each non-null parameter.
func (w *walkCtx) walkStorageParams(tbl catalog.TableDescriptor) {
tableID := tbl.GetID()
storageParams, err := tbl.GetStorageParams(false)
if err != nil {
panic(err)
}
for _, param := range storageParams {
key, value, found := strings.Cut(param, "=")
if !found {
continue
}
if key == "schema_locked" {
continue
}
w.ev(scpb.Status_PUBLIC, &scpb.TableStorageParam{
TableID: tableID,
Name: key,
Value: value,
})
}
}

func (w *walkCtx) walkLocality(tbl catalog.TableDescriptor, l *catpb.LocalityConfig) {
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/schemachanger/scexec/scmutationexec/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,7 @@ go_library(
"//pkg/sql/sem/catid",
"//pkg/sql/sem/idxtype",
"//pkg/sql/sem/tree",
"//pkg/sql/storageparam/tablestorageparam",
"//pkg/sql/types",
"//pkg/util/iterutil",
"//pkg/util/protoutil",
Expand Down
23 changes: 23 additions & 0 deletions pkg/sql/schemachanger/scexec/scmutationexec/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@ import (

"github.com/cockroachdb/cockroach/pkg/config/zonepb"
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop"
"github.com/cockroachdb/cockroach/pkg/sql/storageparam/tablestorageparam"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
)

Expand All @@ -31,6 +32,28 @@ func (i *immediateVisitor) SetTableSchemaLocked(
return nil
}

func (i *immediateVisitor) SetTableStorageParam(
ctx context.Context, op scop.SetTableStorageParam,
) error {
tbl, err := i.checkOutTable(ctx, op.TableID)
if err != nil {
return err
}
setter := tablestorageparam.NewSetter(tbl, false)
return setter.SetToStringValue(ctx, op.Name, op.Value)
}

func (i *immediateVisitor) ResetTableStorageParam(
ctx context.Context, op scop.ResetTableStorageParam,
) error {
tbl, err := i.checkOutTable(ctx, op.TableID)
if err != nil {
return err
}
setter := tablestorageparam.NewSetter(tbl, false)
return setter.ResetToStringValue(ctx, op.Name, op.Value)
}

func (d *deferredVisitor) UpdateTTLScheduleMetadata(
ctx context.Context, op scop.UpdateTTLScheduleMetadata,
) error {
Expand Down
12 changes: 12 additions & 0 deletions pkg/sql/schemachanger/scop/immediate_mutation.go
Original file line number Diff line number Diff line change
Expand Up @@ -1240,3 +1240,15 @@ type SetTableSchemaLocked struct {
TableID descpb.ID
Locked bool
}

// SetTableStorageParam will add a new generated as identity to a column.
type SetTableStorageParam struct {
immediateMutationOp
scpb.TableStorageParam
}

// ResetTableStorageParam will remove the generated as identity from a column.
type ResetTableStorageParam struct {
immediateMutationOp
scpb.TableStorageParam
}

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

8 changes: 8 additions & 0 deletions pkg/sql/schemachanger/scpb/elements.proto
Original file line number Diff line number Diff line change
Expand Up @@ -95,6 +95,7 @@ message ElementProto {
Policy policy = 137 [(gogoproto.moretags) = "parent:\"Table\""];
RowLevelSecurityEnabled row_level_security_enabled = 138 [(gogoproto.moretags) = "parent:\"Table\""];
RowLevelSecurityForced row_level_security_forced = 139 [(gogoproto.moretags) = "parent:\"Table\""];
TableStorageParam table_storage_param = 142 [(gogoproto.moretags) = "parent:\"Table\""];

// Multi-region elements.
TableLocalityGlobal table_locality_global = 110 [(gogoproto.moretags) = "parent:\"Table\""];
Expand Down Expand Up @@ -960,6 +961,13 @@ message TableSchemaLocked {
uint32 table_id = 1 [(gogoproto.customname) = "TableID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.DescID"];
}

// TableStorageParam models storage parameter `schema_locked` of a table.
message TableStorageParam {
uint32 table_id = 1 [(gogoproto.customname) = "TableID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.DescID"];
string name = 2;
string value = 3;
}

// LDRJobIDs models the field `ldr_job_ids` of a table descriptor.
message LDRJobIDs {
uint32 table_id = 1 [(gogoproto.customname) = "TableID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.DescID"];
Expand Down
Loading
Loading