Skip to content

Commit

Permalink
ttl: validate ttl_expiration_expression is a TIMESTAMPTZ
Browse files Browse the repository at this point in the history
Previously the only validation is that the expression is valid SQL
syntax. Now the expression must also be a TIMESTAMPTZ which will
prevent errors in the TTL job when comparing to the cutoff.

Validation is also added so that a column cannot be dropped or altered
in type if it is used in the TTL expression. If the column is renamed, the
expression is automatically updated.

Release note: None

Release justification: high priority fix to new functionality
  • Loading branch information
ecwall authored and rafiss committed Aug 18, 2022
1 parent 7c38417 commit 2747b6f
Show file tree
Hide file tree
Showing 13 changed files with 299 additions and 66 deletions.
3 changes: 3 additions & 0 deletions pkg/sql/alter_column_type.go
Original file line number Diff line number Diff line change
Expand Up @@ -78,6 +78,9 @@ func AlterColumnType(
)
}
}
if err := schemaexpr.ValidateTTLExpressionDoesNotDependOnColumn(tableDesc, col); err != nil {
return err
}

typ, err := tree.ResolveType(ctx, t.ToType, params.p.semaCtx.GetTypeResolver())
if err != nil {
Expand Down
36 changes: 24 additions & 12 deletions pkg/sql/alter_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -487,15 +487,17 @@ func (n *alterTableNode) startExec(params runParams) error {
}

if t.Column == colinfo.TTLDefaultExpirationColumnName && n.tableDesc.HasRowLevelTTL() {
return errors.WithHintf(
pgerror.Newf(
pgcode.InvalidTableDefinition,
`cannot drop column %s while row-level TTL is active`,
t.Column,
),
"use ALTER TABLE %s RESET (ttl) instead",
tree.Name(n.tableDesc.GetName()),
)
if ttlInfo := n.tableDesc.GetRowLevelTTL(); ttlInfo.DurationExpr != "" {
return errors.WithHintf(
pgerror.Newf(
pgcode.InvalidTableDefinition,
`cannot drop column %s while row-level TTL is active`,
t.Column,
),
"use ALTER TABLE %[1]s RESET (ttl) or ALTER TABLE %[1]s SET (ttl_expiration_expression = ...) instead",
tree.Name(n.tableDesc.GetName()),
)
}
}

colDroppedViews, err := dropColumnImpl(params, tn, n.tableDesc, t)
Expand Down Expand Up @@ -1666,10 +1668,14 @@ func dropColumnImpl(
}
}

// We cannot remove this column if there are computed columns that use it.
// We cannot remove this column if there are computed columns or a TTL
// expiration expression that use it.
if err := schemaexpr.ValidateColumnHasNoDependents(tableDesc, colToDrop); err != nil {
return nil, err
}
if err := schemaexpr.ValidateTTLExpressionDoesNotDependOnColumn(tableDesc, colToDrop); err != nil {
return nil, err
}

if tableDesc.GetPrimaryIndex().CollectKeyColumnIDs().Contains(colToDrop.GetID()) {
return nil, sqlerrors.NewColumnReferencedByPrimaryKeyError(colToDrop.GetName())
Expand Down Expand Up @@ -1969,8 +1975,6 @@ func handleTTLStorageParamChange(
if before.HasDurationExpr() {
// Keep the TTL from beforehand, but create the DROP COLUMN job and the
// associated mutation.
tableDesc.RowLevelTTL = before

droppedViews, err := dropColumnImpl(params, tn, tableDesc, &tree.AlterTableDropColumn{
Column: colinfo.TTLDefaultExpirationColumnName,
})
Expand All @@ -1981,6 +1985,7 @@ func handleTTLStorageParamChange(
if len(droppedViews) > 0 {
return pgerror.Newf(pgcode.InvalidParameterValue, "cannot drop TTL automatic column if it is depended on by a view")
}
tableDesc.RowLevelTTL = before

tableDesc.AddModifyRowLevelTTLMutation(
&descpb.ModifyRowLevelTTL{RowLevelTTL: before},
Expand All @@ -1989,6 +1994,13 @@ func handleTTLStorageParamChange(
}
}

// validate ttl_expiration_expression
if after != nil && after.HasExpirationExpr() {
if err := schemaexpr.ValidateTTLExpirationExpression(params.ctx, tableDesc, params.p.SemaCtx(), tn); err != nil {
return err
}
}

return nil
}

Expand Down
78 changes: 78 additions & 0 deletions pkg/sql/catalog/schemaexpr/expr.go
Original file line number Diff line number Diff line change
Expand Up @@ -455,3 +455,81 @@ func SanitizeVarFreeExpr(
}
return typedExpr, nil
}

// ValidateTTLExpressionDoesNotDependOnColumn verifies that the
// ttl_expiration_expression, if any, does not reference the given column.
func ValidateTTLExpressionDoesNotDependOnColumn(
tableDesc catalog.TableDescriptor, col catalog.Column,
) error {
if !tableDesc.HasRowLevelTTL() {
return nil
}
expirationExpr := tableDesc.GetRowLevelTTL().ExpirationExpr
if expirationExpr == "" {
return nil
}
expr, err := parser.ParseExpr(string(expirationExpr))
if err != nil {
// At this point, we should be able to parse the expiration expression.
return errors.WithAssertionFailure(err)
}
referencedCols, err := ExtractColumnIDs(tableDesc, expr)
if err != nil {
return err
}
if referencedCols.Contains(col.GetID()) {
return pgerror.Newf(
pgcode.InvalidColumnReference,
"column %q is referenced by row-level TTL expiration expression %q",
col.ColName(), expirationExpr,
)
}
return nil
}

// ValidateTTLExpirationExpression verifies that the ttl_expiration_expression,
// if any, is valid according to the following rules:
// * type-checks as a TIMESTAMPTZ.
// * is an immutable expression.
// * references valid columns in the table.
func ValidateTTLExpirationExpression(
ctx context.Context,
tableDesc catalog.TableDescriptor,
semaCtx *tree.SemaContext,
tableName *tree.TableName,
) error {
if !tableDesc.HasRowLevelTTL() {
return nil
}

ttl := tableDesc.GetRowLevelTTL()
if !ttl.HasExpirationExpr() {
return nil
}

expr, err := parser.ParseExpr(string(ttl.ExpirationExpr))
if err != nil {
return pgerror.Wrapf(
err,
pgcode.InvalidParameterValue,
`ttl_expiration_expression %q must be a valid expression`,
ttl.ExpirationExpr,
)
}

if _, _, _, err := DequalifyAndValidateExpr(
ctx,
tableDesc,
expr,
types.TimestampTZ,
"ttl_expiration_expression",
semaCtx,
volatility.Immutable,
tableName,
); err != nil {
return pgerror.WithCandidateCode(err, pgcode.InvalidParameterValue)
}

// todo: check dropped column here?
return nil
}
11 changes: 11 additions & 0 deletions pkg/sql/catalog/tabledesc/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -699,6 +699,17 @@ func RenameColumnInTable(
}
}

// Rename the column in the TTL expiration expression.
if tableDesc.HasRowLevelTTL() {
if expirationExpr := tableDesc.GetRowLevelTTL().ExpirationExpr; expirationExpr != "" {
expirationExprStr := string(expirationExpr)
if err := renameInExpr(&expirationExprStr); err != nil {
return err
}
tableDesc.GetRowLevelTTL().ExpirationExpr = catpb.Expression(expirationExprStr)
}
}

// Do all of the above renames inside check constraints, computed expressions,
// and idx predicates that are in mutations.
for i := range tableDesc.Mutations {
Expand Down
28 changes: 28 additions & 0 deletions pkg/sql/catalog/tabledesc/ttl.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,9 +13,13 @@ package tabledesc
import (
"time"

"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/errors"
"github.com/robfig/cron/v3"
)

Expand Down Expand Up @@ -63,6 +67,30 @@ func ValidateRowLevelTTL(ttl *catpb.RowLevelTTL) error {
return nil
}

// ValidateTTLExpirationExpr validates that the ttl_expiration_expression
// only references existing columns.
func ValidateTTLExpirationExpr(
desc catalog.TableDescriptor, expirationExpr catpb.Expression,
) error {
if expirationExpr == "" {
return nil
}
expr, err := parser.ParseExpr(string(expirationExpr))
if err != nil {
return errors.Wrapf(err, "ttl_expiration_expression %q must be a valid expression", expirationExpr)
}
// Ideally, we would also call schemaexpr.ValidateTTLExpirationExpression
// here, but that requires a SemaCtx which we don't have here.
valid, err := schemaexpr.HasValidColumnReferences(desc, expr)
if err != nil {
return err
}
if !valid {
return errors.Newf("row-level TTL expiration expression %q refers to unknown columns", expirationExpr)
}
return nil
}

// ValidateTTLBatchSize validates the batch size of a TTL.
func ValidateTTLBatchSize(key string, val int64) error {
if val <= 0 {
Expand Down
10 changes: 10 additions & 0 deletions pkg/sql/catalog/tabledesc/validate.go
Original file line number Diff line number Diff line change
Expand Up @@ -784,6 +784,16 @@ func (desc *wrapper) ValidateSelf(vea catalog.ValidationErrorAccumulator) {
})

vea.Report(ValidateRowLevelTTL(desc.GetRowLevelTTL()))
if desc.HasRowLevelTTL() {
// ValidateTTLExpirationExpr is called separately from ValidateRowLevelTTL
// because it can only be called on an initialized table descriptor.
// ValidateRowLevelTTL is also used before the table descriptor is fully
// initialized to validate the storage parameters.
if err := ValidateTTLExpirationExpr(desc, desc.GetRowLevelTTL().ExpirationExpr); err != nil {
vea.Report(err)
return
}
}

// Validate that there are no column with both a foreign key ON UPDATE and an
// ON UPDATE expression. This check is made to ensure that we know which ON
Expand Down
6 changes: 5 additions & 1 deletion pkg/sql/create_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -2346,7 +2346,7 @@ func newTableDesc(
creationTime,
privileges,
affected,
&params.p.semaCtx,
params.p.SemaCtx(),
params.EvalContext(),
params.SessionData(),
n.Persistence,
Expand All @@ -2369,6 +2369,10 @@ func newTableDesc(

// Row level TTL tables require a scheduled job to be created as well.
if ttl := ret.RowLevelTTL; ttl != nil {
if err := schemaexpr.ValidateTTLExpirationExpression(params.ctx, ret, params.p.SemaCtx(), &n.Table); err != nil {
return nil, err
}

j, err := CreateRowLevelTTLScheduledJob(
params.ctx,
params.ExecCfg(),
Expand Down
Loading

0 comments on commit 2747b6f

Please sign in to comment.