From 894f152fa6d58220f2ebd36ceba1c7a07c7e52d7 Mon Sep 17 00:00:00 2001 From: rimadeodhar Date: Thu, 18 Jan 2024 16:10:28 -0800 Subject: [PATCH 1/2] sql: Allow dropping unused enum values when enum is referenced by UDF This commit extends the current validation code executed while dropping an enum value to also check for the value being referenced in a UDF. Prior to dropping an enum value, we confirm that the value is unused in tables, constraints, indices etc. However, when we added support for UDFs, we did not extend the scope of this to also check for an enum value being referenced in a UDF. This introduced a bug where we were unable to drop an unused enum value if some other value from within the enum was being referenced by a UDF. For example, if an enum 'e' contained values {'1', '2', '3'}, with the value '1' being referenced in a UDF, we were unable to drop the values '2' and '3' as well. This PR fixes this bug by expanding the usage check for enum values to also include UDFs. With this fix, we can drop an enum value as long as the value itself is unreferenced. Additionally, the PR also refactors some of the common code into its own methods to avoid code duplication. Finally, it extends the existing test suite to test for multiple scenarios when an enum value is dropped. Fixes: https://github.com/cockroachdb/cockroach/issues/115612 --- pkg/sql/BUILD.bazel | 1 + pkg/sql/type_change.go | 551 +++++++++++++++++++++--------------- pkg/sql/type_change_test.go | 230 +++++++++++++++ 3 files changed, 550 insertions(+), 232 deletions(-) diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index 27f3b78bb5bc..c43f98eb9ec5 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -754,6 +754,7 @@ go_test( deps = [ "//pkg/base", "//pkg/build/bazel", + "//pkg/ccl", "//pkg/ccl/changefeedccl/schemafeed/schematestutils", "//pkg/cloud/impl:cloudimpl", "//pkg/clusterversion", diff --git a/pkg/sql/type_change.go b/pkg/sql/type_change.go index 960dc22a25d9..67c101270556 100644 --- a/pkg/sql/type_change.go +++ b/pkg/sql/type_change.go @@ -34,7 +34,10 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + plpgsql "github.com/cockroachdb/cockroach/pkg/sql/plpgsql/parser" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" + "github.com/cockroachdb/cockroach/pkg/sql/sem/plpgsqltree" + "github.com/cockroachdb/cockroach/pkg/sql/sem/plpgsqltree/utils" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/util/intsets" @@ -623,6 +626,63 @@ func doesArrayContainEnumValues(s string, member *descpb.TypeDescriptor_EnumMemb return false } +func visitExprToCheckEnumValueUsage( + expr tree.Expr, typeID descpb.ID, member *descpb.TypeDescriptor_EnumMember, +) (foundUsage bool, recurse bool, newExpr tree.Expr, err error) { + foundUsage = false + switch t := expr.(type) { + // Case for types being used regularly, which are serialized like '\x80':::@100053. + case *tree.AnnotateTypeExpr: + // Check if this expr's type is the one we're dropping the enum value from. + typeOid, ok := t.Type.(*tree.OIDTypeReference) + if !ok { + return foundUsage, true, expr, nil + } + id := typedesc.UserDefinedTypeOIDToID(typeOid.OID) + if id != typeID { + return foundUsage, true, expr, nil + } + + // Check if this expr uses the enum value we're dropping. + strVal, ok := t.Expr.(*tree.StrVal) + if !ok { + return foundUsage, true, expr, nil + } + physicalRep := []byte(strVal.RawString()) + if bytes.Equal(physicalRep, member.PhysicalRepresentation) { + foundUsage = true + } + return foundUsage, false, expr, nil + + // Case for types used in string arrays, serialized like '{a, b, c}':::STRING::@100053. + case *tree.CastExpr: + typeOid, ok := t.Type.(*tree.OIDTypeReference) + if !ok { + return foundUsage, true, expr, nil + } + id := typedesc.UserDefinedTypeOIDToID(typeOid.OID) + // -1 since the type of this CastExpr is the array type. + id = id - 1 + if id != typeID { + return foundUsage, true, expr, nil + } + + // Extract the array and check if it contains the enum member. + annotateType, ok := t.Expr.(*tree.AnnotateTypeExpr) + if !ok { + return foundUsage, true, expr, nil + } + strVal, ok := annotateType.Expr.(*tree.StrVal) + if !ok { + return foundUsage, true, expr, nil + } + foundUsage = doesArrayContainEnumValues(strVal.RawString(), member) + return foundUsage, false, expr, nil + default: + return foundUsage, true, expr, nil + } +} + // findUsagesOfEnumValue takes an expr, type ID and a enum member of that type, // and checks if the expr uses that enum member. func findUsagesOfEnumValue( @@ -632,60 +692,13 @@ func findUsagesOfEnumValue( if err != nil { return false, err } - var foundUsage bool + var foundUsage, foundUsageInCurrentWalk bool visitFunc := func(expr tree.Expr) (recurse bool, newExpr tree.Expr, err error) { - switch t := expr.(type) { - // Case for types being used regularly, which are serialized like '\x80':::@100053. - case *tree.AnnotateTypeExpr: - // Check if this expr's type is the one we're dropping the enum value from. - typeOid, ok := t.Type.(*tree.OIDTypeReference) - if !ok { - return true, expr, nil - } - id := typedesc.UserDefinedTypeOIDToID(typeOid.OID) - if id != typeID { - return true, expr, nil - } - - // Check if this expr uses the enum value we're dropping. - strVal, ok := t.Expr.(*tree.StrVal) - if !ok { - return true, expr, nil - } - physicalRep := []byte(strVal.RawString()) - if bytes.Equal(physicalRep, member.PhysicalRepresentation) { - foundUsage = true - } - return false, expr, nil - - // Case for types used in string arrays, serialized like '{a, b, c}':::STRING::@100053. - case *tree.CastExpr: - typeOid, ok := t.Type.(*tree.OIDTypeReference) - if !ok { - return true, expr, nil - } - id := typedesc.UserDefinedTypeOIDToID(typeOid.OID) - // -1 since the type of this CastExpr is the array type. - id = id - 1 - if id != typeID { - return true, expr, nil - } - - // Extract the array and check if it contains the enum member. - annotateType, ok := t.Expr.(*tree.AnnotateTypeExpr) - if !ok { - return true, expr, nil - } - strVal, ok := annotateType.Expr.(*tree.StrVal) - if !ok { - return true, expr, nil - } - foundUsage = doesArrayContainEnumValues(strVal.RawString(), member) - return false, expr, nil - default: - return true, expr, nil - } + foundUsageInCurrentWalk, recurse, newExpr, err = visitExprToCheckEnumValueUsage(expr, typeID, member) + // Set foundUsage to true if enum usage is detected in any expression in the AST walk. + foundUsage = foundUsage || foundUsageInCurrentWalk + return recurse, newExpr, err } _, err = tree.SimpleVisit(expr, visitFunc) @@ -700,35 +713,12 @@ func findUsagesOfEnumValue( func findUsagesOfEnumValueInViewQuery( viewQuery string, member *descpb.TypeDescriptor_EnumMember, typeID descpb.ID, ) (bool, error) { - var foundUsage bool + var foundUsage, foundUsageInCurrentWalk bool visitFunc := func(expr tree.Expr) (recurse bool, newExpr tree.Expr, err error) { - annotateType, ok := expr.(*tree.AnnotateTypeExpr) - if !ok { - return true, expr, nil - } - - // Check if this expr's type is the one we're dropping the enum value from. - typeOid, ok := annotateType.Type.(*tree.OIDTypeReference) - if !ok { - return true, expr, nil - } - id := typedesc.UserDefinedTypeOIDToID(typeOid.OID) - if id != typeID { - return true, expr, nil - } - - // Check if this expr uses the enum value we're dropping. - strVal, ok := annotateType.Expr.(*tree.StrVal) - if !ok { - return true, expr, nil - } - physicalRep := []byte(strVal.RawString()) - if bytes.Equal(physicalRep, member.PhysicalRepresentation) { - foundUsage = true - return false, expr, nil - } - - return false, expr, nil + foundUsageInCurrentWalk, recurse, newExpr, err = visitExprToCheckEnumValueUsage(expr, typeID, member) + // Set foundUsage to true if enum usage is detected in any expression in the AST walk. + foundUsage = foundUsage || foundUsageInCurrentWalk + return recurse, newExpr, err } stmt, err := parser.ParseOne(viewQuery) @@ -742,205 +732,294 @@ func findUsagesOfEnumValueInViewQuery( return foundUsage, nil } -// canRemoveEnumValue returns an error if the enum value is in use and therefore -// can't be removed. -func (t *typeSchemaChanger) canRemoveEnumValue( - ctx context.Context, +// canRemoveEnumValueFromUDF checks if the enum value is being used +// within the function body. As of today, CockroachDB does not support +// default values for input arguments. However, when we add that support, +// we should augment this method to also check if the enum value is being +// used within the function input arguments. +func (t *typeSchemaChanger) canRemoveEnumValueFromUDF( typeDesc *typedesc.Mutable, - txn isql.Txn, member *descpb.TypeDescriptor_EnumMember, - descsCol *descs.Collection, + udfDesc catalog.FunctionDescriptor, ) error { - for _, ID := range typeDesc.ReferencingDescriptorIDs { - desc, err := descsCol.ByID(txn.KV()).WithoutNonPublic().Get().Table(ctx, ID) + var foundUsage, foundUsageInCurrentWalk bool + visitFunc := func(expr tree.Expr) (recurse bool, newExpr tree.Expr, err error) { + foundUsageInCurrentWalk, recurse, newExpr, err = visitExprToCheckEnumValueUsage(expr, typeDesc.ID, member) + // Set foundUsage to true if enum usage is detected in any expression in the AST walk. + foundUsage = foundUsage || foundUsageInCurrentWalk + return recurse, newExpr, err + } + switch udfDesc.GetLanguage() { + case catpb.Function_SQL: + parsedStmts, err := parser.Parse(udfDesc.GetFunctionBody()) if err != nil { - return errors.Wrapf(err, - "could not validate enum value removal for %q", member.LogicalRepresentation) + return err } - if desc.IsView() { - foundUsage, err := findUsagesOfEnumValueInViewQuery(desc.GetViewQuery(), member, typeDesc.ID) + for _, stmt := range parsedStmts { + _, err = tree.SimpleStmtVisit(stmt.AST, visitFunc) if err != nil { - return err + return errors.Wrapf(err, "failed to parse UDF %s", udfDesc.GetName()) } if foundUsage { return pgerror.Newf(pgcode.DependentObjectsStillExist, - "could not remove enum value %q as it is being used in view %q", - member.LogicalRepresentation, desc.GetName()) + "could not remove enum value %q as it is being used in a UDF %q", + member.LogicalRepresentation, udfDesc.GetName()) } } + case catpb.Function_PLPGSQL: + stmt, err := plpgsql.Parse(udfDesc.GetFunctionBody()) + if err != nil { + return errors.Wrapf(err, "failed to parse UDF %s", udfDesc.GetName()) + } + v := utils.SQLStmtVisitor{Fn: visitFunc} + plpgsqltree.Walk(&v, stmt.AST) + if v.Err != nil { + return errors.Wrapf(v.Err, "failed to parse UDF %s", udfDesc.GetName()) + } + if foundUsage { + return pgerror.Newf(pgcode.DependentObjectsStillExist, + "could not remove enum value %q as it is being used in a UDF %q", + member.LogicalRepresentation, udfDesc.GetName()) + } + } + return nil +} - var query strings.Builder - colSelectors := tabledesc.ColumnsSelectors(desc.PublicColumns()) - columns := tree.AsStringWithFlags(&colSelectors, tree.FmtSerializable) - query.WriteString(fmt.Sprintf("SELECT %s FROM [%d as t] WHERE", columns, ID)) - firstClause := true - validationQueryConstructed := false - - // Note that we examine all indexes as opposed to non-drop indexes so we - // do not remove a partitioning value which is in use on an index which - // is in the process of being dropped but gets re-added due to a failure - // in that schema change. - for _, idx := range desc.AllIndexes() { - if pred := idx.GetPredicate(); pred != "" { - foundUsage, err := findUsagesOfEnumValue(pred, member, typeDesc.ID) - if err != nil { - return err - } - if foundUsage { - return pgerror.Newf(pgcode.DependentObjectsStillExist, - "could not remove enum value %q as it is being used in a predicate of index %s", - member.LogicalRepresentation, &tree.TableIndexName{ - Table: tree.MakeUnqualifiedTableName(tree.Name(desc.GetName())), - Index: tree.UnrestrictedName(idx.GetName()), - }) - } - } - keyColumns := make([]catalog.Column, 0, idx.NumKeyColumns()) - for i := 0; i < idx.NumKeyColumns(); i++ { - col, err := catalog.MustFindColumnByID(desc, idx.GetKeyColumnID(i)) - if err != nil { - return errors.WithAssertionFailure(err) - } - keyColumns = append(keyColumns, col) - } - foundUsage, err := findUsagesOfEnumValueInPartitioning( - idx.GetPartitioning(), t.execCfg.Codec, keyColumns, desc, idx, member, nil, typeDesc, - ) +func (t *typeSchemaChanger) canRemoveEnumValueFromTable( + ctx context.Context, + typeDesc *typedesc.Mutable, + txn isql.Txn, + member *descpb.TypeDescriptor_EnumMember, + descsCol *descs.Collection, + desc catalog.TableDescriptor, + ID descpb.ID, +) error { + if desc.IsView() { + foundUsage, err := findUsagesOfEnumValueInViewQuery(desc.GetViewQuery(), member, typeDesc.ID) + if err != nil { + return err + } + if foundUsage { + return pgerror.Newf(pgcode.DependentObjectsStillExist, + "could not remove enum value %q as it is being used in view %q", + member.LogicalRepresentation, desc.GetName()) + } + } + + var query strings.Builder + colSelectors := tabledesc.ColumnsSelectors(desc.PublicColumns()) + columns := tree.AsStringWithFlags(&colSelectors, tree.FmtSerializable) + query.WriteString(fmt.Sprintf("SELECT %s FROM [%d as t] WHERE", columns, ID)) + firstClause := true + validationQueryConstructed := false + + // Note that we examine all indexes as opposed to non-drop indexes so we + // do not remove a partitioning value which is in use on an index which + // is in the process of being dropped but gets re-added due to a failure + // in that schema change. + for _, idx := range desc.AllIndexes() { + if pred := idx.GetPredicate(); pred != "" { + foundUsage, err := findUsagesOfEnumValue(pred, member, typeDesc.ID) if err != nil { return err } if foundUsage { return pgerror.Newf(pgcode.DependentObjectsStillExist, - "could not remove enum value %q as it is being used in the partitioning of index %s", + "could not remove enum value %q as it is being used in a predicate of index %s", member.LogicalRepresentation, &tree.TableIndexName{ Table: tree.MakeUnqualifiedTableName(tree.Name(desc.GetName())), Index: tree.UnrestrictedName(idx.GetName()), }) } } + keyColumns := make([]catalog.Column, 0, idx.NumKeyColumns()) + for i := 0; i < idx.NumKeyColumns(); i++ { + col, err := catalog.MustFindColumnByID(desc, idx.GetKeyColumnID(i)) + if err != nil { + return errors.WithAssertionFailure(err) + } + keyColumns = append(keyColumns, col) + } + foundUsage, err := findUsagesOfEnumValueInPartitioning( + idx.GetPartitioning(), t.execCfg.Codec, keyColumns, desc, idx, member, nil, typeDesc, + ) + if err != nil { + return err + } + if foundUsage { + return pgerror.Newf(pgcode.DependentObjectsStillExist, + "could not remove enum value %q as it is being used in the partitioning of index %s", + member.LogicalRepresentation, &tree.TableIndexName{ + Table: tree.MakeUnqualifiedTableName(tree.Name(desc.GetName())), + Index: tree.UnrestrictedName(idx.GetName()), + }) + } + } - // Examine all check constraints. - for _, chk := range desc.CheckConstraints() { - foundUsage, err := findUsagesOfEnumValue(chk.GetExpr(), member, typeDesc.ID) + // Examine all check constraints. + for _, chk := range desc.CheckConstraints() { + foundUsage, err := findUsagesOfEnumValue(chk.GetExpr(), member, typeDesc.ID) + if err != nil { + return err + } + if foundUsage { + return pgerror.Newf(pgcode.DependentObjectsStillExist, + "could not remove enum value %q as it is being used in a check constraint of %q", + member.LogicalRepresentation, desc.GetName()) + } + } + + for _, col := range desc.PublicColumns() { + // If this column has a default expression, check if it uses the enum member being dropped. + if col.HasDefault() { + foundUsage, err := findUsagesOfEnumValue(col.GetDefaultExpr(), member, typeDesc.ID) if err != nil { return err } if foundUsage { return pgerror.Newf(pgcode.DependentObjectsStillExist, - "could not remove enum value %q as it is being used in a check constraint of %q", + "could not remove enum value %q as it is being used in a default expresion of %q", member.LogicalRepresentation, desc.GetName()) } } - for _, col := range desc.PublicColumns() { - // If this column has a default expression, check if it uses the enum member being dropped. - if col.HasDefault() { - foundUsage, err := findUsagesOfEnumValue(col.GetDefaultExpr(), member, typeDesc.ID) - if err != nil { - return err - } - if foundUsage { - return pgerror.Newf(pgcode.DependentObjectsStillExist, - "could not remove enum value %q as it is being used in a default expresion of %q", - member.LogicalRepresentation, desc.GetName()) - } + // If this column is computed, check if it uses the enum member being dropped. + if col.IsComputed() { + foundUsage, err := findUsagesOfEnumValue(col.GetComputeExpr(), member, typeDesc.ID) + if err != nil { + return err } + if foundUsage { + return pgerror.Newf(pgcode.DependentObjectsStillExist, + "could not remove enum value %q as it is being used in a computed column of %q", + member.LogicalRepresentation, desc.GetName()) + } + } - // If this column is computed, check if it uses the enum member being dropped. - if col.IsComputed() { - foundUsage, err := findUsagesOfEnumValue(col.GetComputeExpr(), member, typeDesc.ID) - if err != nil { - return err - } - if foundUsage { - return pgerror.Newf(pgcode.DependentObjectsStillExist, - "could not remove enum value %q as it is being used in a computed column of %q", - member.LogicalRepresentation, desc.GetName()) - } + // If this column has an ON UPDATE expression, check if it uses the enum + // member being dropped. + if col.HasOnUpdate() { + foundUsage, err := findUsagesOfEnumValue(col.GetOnUpdateExpr(), member, typeDesc.ID) + if err != nil { + return err } + if foundUsage { + return pgerror.Newf(pgcode.DependentObjectsStillExist, + "could not remove enum value %q as it is being used in an ON UPDATE expression"+ + " of %q", + member.LogicalRepresentation, desc.GetName()) + } + } - // If this column has an ON UPDATE expression, check if it uses the enum - // member being dropped. - if col.HasOnUpdate() { - foundUsage, err := findUsagesOfEnumValue(col.GetOnUpdateExpr(), member, typeDesc.ID) + if col.GetType().UserDefined() { + tid := typedesc.GetUserDefinedTypeDescID(col.GetType()) + if typeDesc.ID == tid { + if !firstClause { + query.WriteString(" OR") + } + sqlPhysRep, err := convertToSQLStringRepresentation(member.PhysicalRepresentation) if err != nil { return err } - if foundUsage { - return pgerror.Newf(pgcode.DependentObjectsStillExist, - "could not remove enum value %q as it is being used in an ON UPDATE expression"+ - " of %q", - member.LogicalRepresentation, desc.GetName()) - } + colName := col.ColName() + query.WriteString(fmt.Sprintf( + " t.%s = %s", + colName.String(), + sqlPhysRep, + )) + firstClause = false + validationQueryConstructed = true } + } + } + query.WriteString(" LIMIT 1") + + // NB: A type descriptor reference does not imply at-least one column in the + // table is of the type whose value is being removed. The notable exception + // being REGIONAL BY TABLE multi-region tables. In this case, no valid query + // is constructed and there's nothing to execute. Instead, their validation + // is handled as a special case below. + if validationQueryConstructed { + // We need to override the internal executor's current database (which would + // be unset by default) when executing the query constructed above. This is + // because the enum value may be used in a view expression, which is + // name resolved in the context of the type's database. + dbDesc, err := descsCol.ByID(txn.KV()).WithoutNonPublic().Get().Database(ctx, typeDesc.ParentID) + const validationErr = "could not validate removal of enum value %q" + if err != nil { + return errors.Wrapf(err, validationErr, member.LogicalRepresentation) + } + override := sessiondata.InternalExecutorOverride{ + User: username.RootUserName(), + Database: dbDesc.GetName(), + } + rows, err := txn.QueryRowEx(ctx, "count-value-usage", txn.KV(), override, query.String()) + if err != nil { + return errors.Wrapf(err, validationErr, member.LogicalRepresentation) + } + // Check if the above query returned a result. If it did, then the + // enum value is being used by some place. + if len(rows) > 0 { + return pgerror.Newf(pgcode.DependentObjectsStillExist, + "could not remove enum value %q as it is being used by %q in row: %s", + member.LogicalRepresentation, desc.GetName(), labeledRowValues(desc.PublicColumns(), rows)) + } + } - if col.GetType().UserDefined() { - tid := typedesc.GetUserDefinedTypeDescID(col.GetType()) - if typeDesc.ID == tid { - if !firstClause { - query.WriteString(" OR") - } - sqlPhysRep, err := convertToSQLStringRepresentation(member.PhysicalRepresentation) - if err != nil { - return err - } - colName := col.ColName() - query.WriteString(fmt.Sprintf( - " t.%s = %s", - colName.String(), - sqlPhysRep, - )) - firstClause = false - validationQueryConstructed = true - } - } + // If the type descriptor is a multi-region enum and the table descriptor + // belongs to a regional (by table) table, we disallow dropping the region + // if it is being used as the homed region for that table. + if typeDesc.Kind == descpb.TypeDescriptor_MULTIREGION_ENUM && desc.IsLocalityRegionalByTable() { + homedRegion, err := desc.GetRegionalByTableRegion() + if err != nil { + return err } - query.WriteString(" LIMIT 1") - - // NB: A type descriptor reference does not imply at-least one column in the - // table is of the type whose value is being removed. The notable exception - // being REGIONAL BY TABLE multi-region tables. In this case, no valid query - // is constructed and there's nothing to execute. Instead, their validation - // is handled as a special case below. - if validationQueryConstructed { - // We need to override the internal executor's current database (which would - // be unset by default) when executing the query constructed above. This is - // because the enum value may be used in a view expression, which is - // name resolved in the context of the type's database. - dbDesc, err := descsCol.ByID(txn.KV()).WithoutNonPublic().Get().Database(ctx, typeDesc.ParentID) - const validationErr = "could not validate removal of enum value %q" - if err != nil { - return errors.Wrapf(err, validationErr, member.LogicalRepresentation) - } - override := sessiondata.InternalExecutorOverride{ - User: username.RootUserName(), - Database: dbDesc.GetName(), - } - rows, err := txn.QueryRowEx(ctx, "count-value-usage", txn.KV(), override, query.String()) - if err != nil { - return errors.Wrapf(err, validationErr, member.LogicalRepresentation) - } - // Check if the above query returned a result. If it did, then the - // enum value is being used by some place. - if len(rows) > 0 { - return pgerror.Newf(pgcode.DependentObjectsStillExist, - "could not remove enum value %q as it is being used by %q in row: %s", - member.LogicalRepresentation, desc.GetName(), labeledRowValues(desc.PublicColumns(), rows)) - } + if catpb.RegionName(member.LogicalRepresentation) == homedRegion { + return errors.Newf("could not remove enum value %q as it is the home region for table %q", + member.LogicalRepresentation, desc.GetName()) + } + } + return nil +} + +// canRemoveEnumValue returns an error if the enum value is in use and therefore +// can't be removed. An enum value can be referenced in a UDF and a relation. This method +// should be updated if any other data element is added/updated to reference an enum +// value. +func (t *typeSchemaChanger) canRemoveEnumValue( + ctx context.Context, + typeDesc *typedesc.Mutable, + txn isql.Txn, + member *descpb.TypeDescriptor_EnumMember, + descsCol *descs.Collection, +) error { + descGetter := descsCol.ByID(txn.KV()).WithoutNonPublic().Get() + for _, id := range typeDesc.ReferencingDescriptorIDs { + desc, err := descGetter.Desc(ctx, id) + if err != nil { + return errors.Wrapf(err, + "could not validate enum value removal for %q", member.LogicalRepresentation) } - // If the type descriptor is a multi-region enum and the table descriptor - // belongs to a regional (by table) table, we disallow dropping the region - // if it is being used as the homed region for that table. - if typeDesc.Kind == descpb.TypeDescriptor_MULTIREGION_ENUM && desc.IsLocalityRegionalByTable() { - homedRegion, err := desc.GetRegionalByTableRegion() + // An enum value can be used within a table and a UDF. + switch desc := desc.(type) { + case catalog.TableDescriptor: + err = t.canRemoveEnumValueFromTable(ctx, typeDesc, txn, member, descsCol, desc, id) if err != nil { return err } - if catpb.RegionName(member.LogicalRepresentation) == homedRegion { - return errors.Newf("could not remove enum value %q as it is the home region for table %q", - member.LogicalRepresentation, desc.GetName()) + case catalog.FunctionDescriptor: + err = t.canRemoveEnumValueFromUDF(typeDesc, member, desc) + if err != nil { + return err } + default: + // Enum value is being referenced by some other type of descriptor, return error. + // This should never happen. + return errors.Newf( + "enum value being referenced by descriptor type %s, name %s, unable to delete", + desc.DescriptorType(), + desc.GetName()) } } @@ -1079,10 +1158,18 @@ func (t *typeSchemaChanger) canRemoveEnumValueFromArrayUsages( const validationErr = "could not validate removal of enum value %q" for i := 0; i < arrayTypeDesc.NumReferencingDescriptors(); i++ { id := arrayTypeDesc.GetReferencingDescriptorID(i) - desc, err := descsCol.ByIDWithLeased(txn.KV()).WithoutNonPublic().Get().Table(ctx, id) + desc, err := descsCol.ByIDWithLeased(txn.KV()).WithoutNonPublic().Get().Desc(ctx, id) if err != nil { return errors.Wrapf(err, validationErr, member.LogicalRepresentation) } + // We install a backreference to both the type descriptor and + // its array alias even when referenced within a UDF. We only need to + // execute the following code when its referenced with a relation. So check for descriptor + // type and skip if it is not a relation. + tblDesc, isTable := desc.(catalog.TableDescriptor) + if !isTable { + continue + } var unionUnnests strings.Builder var query strings.Builder @@ -1094,7 +1181,7 @@ func (t *typeSchemaChanger) canRemoveEnumValueFromArrayUsages( // ... // ) WHERE unnest = 'enum_value' firstClause := true - for _, col := range desc.PublicColumns() { + for _, col := range tblDesc.PublicColumns() { if !col.GetType().UserDefined() { continue } diff --git a/pkg/sql/type_change_test.go b/pkg/sql/type_change_test.go index 47fdad6bd852..19c9fa92ba94 100644 --- a/pkg/sql/type_change_test.go +++ b/pkg/sql/type_change_test.go @@ -13,8 +13,10 @@ package sql_test import ( "context" "fmt" + "strings" "testing" + "github.com/cockroachdb/cockroach/pkg/ccl" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" @@ -503,3 +505,231 @@ WHERE }) } } + +func TestAddDropEnumValues(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + defer ccl.TestingEnableEnterprise()() + ctx := context.Background() + + params, _ := createTestServerParams() + // Decrease the adopt loop interval so that retries happen quickly. + params.Knobs.JobsTestingKnobs = jobs.NewTestingKnobsWithShortIntervals() + + s, sqlDB, _ := serverutils.StartServer(t, params) + defer s.Stopper().Stop(ctx) + + // Set up schema necessary to execute all unit test cases below. We + // execute the schema setup statements within a single, implicit txn + // to ensure all the necessary schema elements required for the subsequent + // test cases have been set up successfully. + if _, err := sqlDB.Exec(` +CREATE DATABASE d; +USE d; +CREATE TYPE e1 AS ENUM('check', 'enum', 'value', 'reference', 'cases', 'unused value'); +-- case 1: enum value referenced within a UDF with language = SQL. +CREATE FUNCTION f1() RETURNS e1 LANGUAGE SQL AS $$ SELECT 'check'::e1 $$; +-- case 2: enum value referenced within a UDF with language = PLPGSQL. +CREATE FUNCTION f2() RETURNS e1 AS $$ + begin + select 'enum'::e1; + end $$ + language plpgsql; +-- case 3: array type alias for enum referenced within a UDF with language = SQL. +CREATE TYPE e2 AS ENUM('check', 'array', 'type', 'usage', 'cases', 'within', 'udfs', 'unused value'); +CREATE FUNCTION f3() RETURNS _e2 LANGUAGE SQL AS $$ SELECT '{check, array, type}'::_e2; $$; +CREATE FUNCTION f4() RETURNS e2[] LANGUAGE SQL AS $$ SELECT ARRAY['usage'::e2, 'cases'::e2]; $$; +-- case 4: array type alias for enum referenced within a UDF with language = PLPGSQL. +CREATE FUNCTION f5() RETURNS e2[] AS $$ + declare + b e2[] := ARRAY['within'::e2, 'udfs'::e2]; + begin + return b; + end $$ + language plpgsql; +-- case 5: enum value referenced with a view query. +CREATE TYPE e3 AS ENUM('check', 'enum', 'type', 'usage', 'cases', 'within', 'views', 'unused value'); +CREATE VIEW v1 AS (SELECT 'check'::e3); +-- case 7: array type alias for enum referenced within a view query +CREATE VIEW v2 AS (SELECT '{enum, type, usage}'::_e3); +-- case 6: enum value referenced within a table. +CREATE TYPE e4 AS ENUM('check', 'enum', 'type', 'usage', 'cases', 'within', 'tables', 'invalid value', 'unused value'); +CREATE TABLE t1(use_enum e4 CHECK (use_enum != 'invalid value'::e4), use_enum_computed e4 AS (IF (use_enum = 'check'::e4, 'type'::e4, 'cases'::e4)) STORED); +INSERT INTO t1 VALUES('check'::e4); +INSERT INTO t1 VALUES('enum'::e4); +-- case 8: array type alias for enum referenced within a table +CREATE TABLE t2(use_enum_arr e4[]); +INSERT INTO t2 VALUES(ARRAY['usage'::e4, 'cases'::e4]); +INSERT INTO t2 VALUES('{within, tables}'::_e4); +-- Add default column, on update expr, indexes +CREATE TYPE e5 AS ENUM('usage', 'in', 'default column', 'update expr'); +CREATE TABLE t3(id int, enum_arr_default e5[] DEFAULT ARRAY['default column'::e5]); +CREATE TABLE t4(id int, enum_val e5 DEFAULT 'usage'::e5 ON UPDATE 'update expr'::e5); +`); err != nil { + t.Fatal(err) + } + + testCases := []struct { + query string + success bool + err string + }{ + { + `ALTER TYPE e1 DROP VALUE 'check'`, + false, + "could not remove enum value \"check\" as it is being used in a UDF \"f1\"", + }, + { + `ALTER TYPE e1 DROP VALUE 'enum'`, + false, + "could not remove enum value \"enum\" as it is being used in a UDF \"f2\"", + }, + { + `ALTER TYPE e1 DROP VALUE 'unused value'`, + true, + "", + }, + { + `DROP FUNCTION f1; ALTER TYPE e1 DROP VALUE 'check'`, + true, + "", + }, + { + `DROP FUNCTION f2; ALTER TYPE e1 DROP VALUE 'enum'`, + true, + "", + }, + { + `ALTER TYPE e2 DROP VALUE 'check'`, + false, + "could not remove enum value \"check\" as it is being used in a UDF \"f3\"", + }, + { + `ALTER TYPE e2 DROP VALUE 'usage'`, + false, + "could not remove enum value \"usage\" as it is being used in a UDF \"f4\"", + }, + { + `ALTER TYPE e2 DROP VALUE 'within'`, + false, + "could not remove enum value \"within\" as it is being used in a UDF \"f5\"", + }, + { + `ALTER TYPE e2 DROP VALUE 'unused value'`, + true, + "", + }, + { + `DROP FUNCTION f3; ALTER TYPE e2 DROP VALUE 'check'`, + true, + "", + }, + { + `DROP FUNCTION f4; ALTER TYPE e2 DROP VALUE 'usage'`, + true, + "", + }, + { + `DROP FUNCTION f5; ALTER TYPE e2 DROP VALUE 'within'`, + true, + "", + }, + { + `ALTER TYPE e3 DROP VALUE 'check'`, + false, + "could not remove enum value \"check\" as it is being used in view \"v1\"", + }, + { + `ALTER TYPE e3 DROP VALUE 'enum'`, + false, + "could not remove enum value \"enum\" as it is being used in view \"v2\"", + }, + { + `ALTER TYPE e3 DROP VALUE 'unused value'`, + true, + "", + }, + { + `DROP VIEW v1; ALTER TYPE e3 DROP VALUE 'check'`, + true, + "", + }, + { + `DROP VIEW v2; ALTER TYPE e3 DROP VALUE 'enum'`, + true, + "", + }, + { + `ALTER TYPE e4 DROP VALUE 'enum'`, + false, + "could not remove enum value \"enum\" as it is being used by \"t1\" in row: use_enum='enum', use_enum_computed='cases'", + }, + { + `ALTER TYPE e4 DROP VALUE 'usage'`, + false, + "could not remove enum value \"usage\" as it is being used by table \"d.public.t2\"", + }, + { + `ALTER TYPE e4 DROP VALUE 'type'`, + false, + "could not remove enum value \"type\" as it is being used in a computed column of \"t1\"", + }, + { + `ALTER TYPE e4 DROP VALUE 'tables'`, + false, + "could not remove enum value \"tables\" as it is being used by table \"d.public.t2\"", + }, + { + `ALTER TYPE e4 DROP VALUE 'invalid value'`, + false, + "could not remove enum value \"invalid value\" as it is being used in a check constraint of \"t1\"", + }, + { + `ALTER TYPE e4 DROP VALUE 'unused value'`, + true, + "", + }, + { + `ALTER TYPE e5 DROP VALUE 'default column'`, + false, + "could not remove enum value \"default column\" as it is being used in a default expresion of \"t3\"", + }, + { + `ALTER TYPE e5 DROP VALUE 'update expr'`, + false, + " could not remove enum value \"update expr\" as it is being used in an ON UPDATE expression of \"t4\"", + }, + { + `DROP TABLE t1; ALTER TYPE e4 DROP VALUE 'enum'; ALTER TYPE e4 DROP VALUE 'type'`, + true, + "", + }, + { + `DROP TABLE t2; ALTER TYPE e4 DROP VALUE 'usage'; ALTER TYPE e4 DROP VALUE 'tables'`, + true, + "", + }, + { + `DROP TABLE t3; ALTER TYPE e5 DROP VALUE 'default column'`, + true, + "", + }, + { + `DROP TABLE t4; ALTER TYPE e5 DROP VALUE 'update expr'`, + true, + "", + }, + } + + for i, tc := range testCases { + _, err := sqlDB.Exec(tc.query) + if tc.success { + require.NoErrorf(t, err, "#%d: unexpected error while executing query: %v", i, err) + } else { + require.Errorf(t, err, "#%d: expected error %s, but got no error", i, tc.err) + if !strings.Contains(err.Error(), tc.err) { + t.Fatalf("#%d: expected error %s, got error %s", i, tc.err, err) + } + } + } +} From 622cd1c76a02f651645293226af00499f3acd5a5 Mon Sep 17 00:00:00 2001 From: rimadeodhar Date: Fri, 16 Feb 2024 12:45:43 -0800 Subject: [PATCH 2/2] schemaworkload: Enable testing for alterTypeDropValue --- pkg/workload/schemachange/optype.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/workload/schemachange/optype.go b/pkg/workload/schemachange/optype.go index 9d81f0115383..069fa3e24298 100644 --- a/pkg/workload/schemachange/optype.go +++ b/pkg/workload/schemachange/optype.go @@ -272,7 +272,7 @@ var opWeights = []int{ dropSequence: 1, dropTable: 1, dropView: 1, - alterTypeDropValue: 0, // Disabled and tracked with #114844, #113859, and #115612. + alterTypeDropValue: 1, dropSchema: 1, alterDatabasePrimaryRegion: 0, // Disabled and tracked with #83831 alterTableRenameColumn: 1,