Skip to content

Commit

Permalink
Merge pull request #57677 from ajwerner/backport20.2-57648
Browse files Browse the repository at this point in the history
release-20.2: sql: add force flag overloads to crdb_internal.unsafe_.* builtins
  • Loading branch information
ajwerner authored Dec 8, 2020
2 parents 44fe5a3 + 03cbd1e commit 89dd2d1
Show file tree
Hide file tree
Showing 5 changed files with 390 additions and 35 deletions.
8 changes: 5 additions & 3 deletions pkg/sql/faketreeeval/evalctx.go
Original file line number Diff line number Diff line change
Expand Up @@ -88,13 +88,15 @@ type DummyEvalPlanner struct{}

// UnsafeUpsertDescriptor is part of the EvalPlanner interface.
func (ep *DummyEvalPlanner) UnsafeUpsertDescriptor(
ctx context.Context, descID int64, encodedDescriptor []byte,
ctx context.Context, descID int64, encodedDescriptor []byte, force bool,
) error {
return errors.WithStack(errEvalPlanner)
}

// UnsafeDeleteDescriptor is part of the EvalPlanner interface.
func (ep *DummyEvalPlanner) UnsafeDeleteDescriptor(ctx context.Context, descID int64) error {
func (ep *DummyEvalPlanner) UnsafeDeleteDescriptor(
ctx context.Context, descID int64, force bool,
) error {
return errors.WithStack(errEvalPlanner)
}

Expand All @@ -107,7 +109,7 @@ func (ep *DummyEvalPlanner) UnsafeUpsertNamespaceEntry(

// UnsafeDeleteNamespaceEntry is part of the EvalPlanner interface.
func (ep *DummyEvalPlanner) UnsafeDeleteNamespaceEntry(
ctx context.Context, parentID, parentSchemaID int64, name string, descID int64,
ctx context.Context, parentID, parentSchemaID int64, name string, descID int64, force bool,
) error {
return errors.WithStack(errEvalPlanner)
}
Expand Down
99 changes: 73 additions & 26 deletions pkg/sql/repair.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/typedesc"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/errors"
)
Expand Down Expand Up @@ -55,7 +56,7 @@ import (
// all descriptors at the end of the transaction to ensure that this operation
// didn't break a reference to this descriptor.
func (p *planner) UnsafeUpsertDescriptor(
ctx context.Context, descID int64, encodedDesc []byte,
ctx context.Context, descID int64, encodedDesc []byte, force bool,
) error {
const method = "crdb_internal.unsafe_upsert_descriptor()"
if err := checkPlannerStateForRepairFunctions(ctx, p, method); err != nil {
Expand All @@ -69,9 +70,18 @@ func (p *planner) UnsafeUpsertDescriptor(
}

// Fetch the existing descriptor.

existing, err := p.Descriptors().GetMutableDescriptorByID(ctx, id, p.txn)
var forceNoticeString string // for the event
if !errors.Is(err, catalog.ErrDescriptorNotFound) && err != nil {
return err
if force {
notice := pgnotice.NewWithSeverityf("WARNING",
"failed to retrieve existing descriptor, continuing with force flag: %v", err)
p.SendClientNotice(ctx, notice)
forceNoticeString = notice.Error()
} else {
return err
}
}

// Validate that existing is sane and store its hex serialization into
Expand Down Expand Up @@ -130,6 +140,7 @@ func (p *planner) UnsafeUpsertDescriptor(
return err
}
}

return MakeEventLogger(p.execCfg).InsertEventRecord(
ctx,
p.txn,
Expand All @@ -140,10 +151,14 @@ func (p *planner) UnsafeUpsertDescriptor(
ID descpb.ID `json:"id"`
ExistingDescriptor string `json:"existing_descriptor,omitempty"`
Descriptor string `json:"descriptor,omitempty"`
Force bool `json:"force,omitempty"`
ValidationErrors string `json:"validation_errors,omitempty"`
}{
ID: id,
ExistingDescriptor: existingStr,
Descriptor: hex.EncodeToString(encodedDesc),
Force: force,
ValidationErrors: forceNoticeString,
})
}

Expand Down Expand Up @@ -308,7 +323,11 @@ func (p *planner) UnsafeUpsertNamespaceEntry(
// will ensure that the entry does not correspond to a non-dropped descriptor
// and that the entry exists with the provided ID.
func (p *planner) UnsafeDeleteNamespaceEntry(
ctx context.Context, parentIDInt, parentSchemaIDInt int64, name string, descIDInt int64,
ctx context.Context,
parentIDInt, parentSchemaIDInt int64,
name string,
descIDInt int64,
force bool,
) error {
const method = "crdb_internal.unsafe_delete_namespace_entry()"
if err := checkPlannerStateForRepairFunctions(ctx, p, method); err != nil {
Expand Down Expand Up @@ -336,8 +355,16 @@ func (p *planner) UnsafeDeleteNamespaceEntry(
}
}
desc, err := p.Descriptors().GetMutableDescriptorByID(ctx, descID, p.txn)
var forceNoticeString string // for the event
if err != nil && !errors.Is(err, catalog.ErrDescriptorNotFound) {
return errors.Wrapf(err, "failed to retrieve descriptor %d", descID)
if force {
notice := pgnotice.NewWithSeverityf("WARNING",
"failed to retrieve existing descriptor, continuing with force flag: %v", err)
p.SendClientNotice(ctx, notice)
forceNoticeString = notice.Error()
} else {
return errors.Wrapf(err, "failed to retrieve descriptor %d", descID)
}
}
if err == nil && !desc.Dropped() {
return pgerror.Newf(pgcode.ObjectNotInPrerequisiteState,
Expand All @@ -353,16 +380,20 @@ func (p *planner) UnsafeDeleteNamespaceEntry(
int32(descID),
int32(p.EvalContext().NodeID.SQLInstanceID()),
&struct {
ParentID descpb.ID `json:"parent_id,omitempty"`
ParentSchemaID descpb.ID `json:"parent_schema_id,omitempty"`
Name string `json:"name"`
ID descpb.ID `json:"id"`
ExistingID descpb.ID `json:"existing_id,omitempty"`
ParentID descpb.ID `json:"parent_id,omitempty"`
ParentSchemaID descpb.ID `json:"parent_schema_id,omitempty"`
Name string `json:"name"`
ID descpb.ID `json:"id"`
ExistingID descpb.ID `json:"existing_id,omitempty"`
Force bool `json:"force,omitempty"`
ValidationErrors string `json:"validation_errors,omitempty"`
}{
ParentID: parentID,
ParentSchemaID: parentSchemaID,
ID: descID,
Name: name,
ParentID: parentID,
ParentSchemaID: parentSchemaID,
ID: descID,
Name: name,
Force: force,
ValidationErrors: forceNoticeString,
})
}

Expand All @@ -373,37 +404,53 @@ func (p *planner) UnsafeDeleteNamespaceEntry(
// This method will perform very minimal validation. An error will be returned
// if no such descriptor exists. This method can very easily introduce
// corruption, beware.
func (p *planner) UnsafeDeleteDescriptor(ctx context.Context, descID int64) error {
func (p *planner) UnsafeDeleteDescriptor(ctx context.Context, descID int64, force bool) error {
const method = "crdb_internal.unsafe_delete_descriptor()"
if err := checkPlannerStateForRepairFunctions(ctx, p, method); err != nil {
return err
}
id := descpb.ID(descID)
mut, err := p.Descriptors().GetMutableDescriptorByID(ctx, id, p.txn)
var forceNoticeString string // for the event
if err != nil {
return err
if !errors.Is(err, catalog.ErrDescriptorNotFound) && force {
notice := pgnotice.NewWithSeverityf("WARNING",
"failed to retrieve existing descriptor, continuing with force flag: %v", err)
p.SendClientNotice(ctx, notice)
forceNoticeString = notice.Error()
} else {
return err
}
}
descKey := catalogkeys.MakeDescMetadataKey(p.execCfg.Codec, id)
if err := p.txn.Del(ctx, descKey); err != nil {
return err
}
ev := struct {
ParentID descpb.ID `json:"parent_id,omitempty"`
ParentSchemaID descpb.ID `json:"parent_schema_id,omitempty"`
Name string `json:"name"`
ID descpb.ID `json:"id"`
Force bool `json:"force,omitempty"`
ValidationErrors string `json:"validation_errors,omitempty"`
}{
ID: id,
Force: force,
ValidationErrors: forceNoticeString,
}
if mut != nil {
ev.ParentID = mut.GetParentID()
ev.ParentSchemaID = mut.GetParentSchemaID()
ev.Name = mut.GetName()
}
return MakeEventLogger(p.execCfg).InsertEventRecord(
ctx,
p.txn,
EventLogUnsafeDeleteDescriptor,
int32(descID),
int32(p.EvalContext().NodeID.SQLInstanceID()),
&struct {
ParentID descpb.ID `json:"parent_id,omitempty"`
ParentSchemaID descpb.ID `json:"parent_schema_id,omitempty"`
Name string `json:"name"`
ID descpb.ID `json:"id"`
}{
ParentID: mut.GetParentID(),
ParentSchemaID: mut.GetParentSchemaID(),
ID: mut.GetID(),
Name: mut.GetName(),
})
ev,
)
}

func checkPlannerStateForRepairFunctions(ctx context.Context, p *planner, method string) error {
Expand Down
69 changes: 67 additions & 2 deletions pkg/sql/sem/builtins/builtins.go
Original file line number Diff line number Diff line change
Expand Up @@ -4248,7 +4248,27 @@ may increase either contention or retry errors, or both.`,
Fn: func(ctx *tree.EvalContext, args tree.Datums) (tree.Datum, error) {
if err := ctx.Planner.UnsafeUpsertDescriptor(ctx.Context,
int64(*args[0].(*tree.DInt)),
[]byte(*args[1].(*tree.DBytes))); err != nil {
[]byte(*args[1].(*tree.DBytes)),
false /* force */); err != nil {
return nil, err
}
return tree.DBoolTrue, nil
},
Info: "This function is used only by CockroachDB's developers for testing purposes.",
Volatility: tree.VolatilityVolatile,
},
tree.Overload{
Types: tree.ArgTypes{
{"id", types.Int},
{"desc", types.Bytes},
{"force", types.Bool},
},
ReturnType: tree.FixedReturnType(types.Bool),
Fn: func(ctx *tree.EvalContext, args tree.Datums) (tree.Datum, error) {
if err := ctx.Planner.UnsafeUpsertDescriptor(ctx.Context,
int64(*args[0].(*tree.DInt)),
[]byte(*args[1].(*tree.DBytes)),
bool(*args[2].(*tree.DBool))); err != nil {
return nil, err
}
return tree.DBoolTrue, nil
Expand All @@ -4271,6 +4291,25 @@ may increase either contention or retry errors, or both.`,
Fn: func(ctx *tree.EvalContext, args tree.Datums) (tree.Datum, error) {
if err := ctx.Planner.UnsafeDeleteDescriptor(ctx.Context,
int64(*args[0].(*tree.DInt)),
false, /* force */
); err != nil {
return nil, err
}
return tree.DBoolTrue, nil
},
Info: "This function is used only by CockroachDB's developers for testing purposes.",
Volatility: tree.VolatilityVolatile,
},
tree.Overload{
Types: tree.ArgTypes{
{"id", types.Int},
{"force", types.Bool},
},
ReturnType: tree.FixedReturnType(types.Bool),
Fn: func(ctx *tree.EvalContext, args tree.Datums) (tree.Datum, error) {
if err := ctx.Planner.UnsafeDeleteDescriptor(ctx.Context,
int64(*args[0].(*tree.DInt)),
bool(*args[1].(*tree.DBool)),
); err != nil {
return nil, err
}
Expand Down Expand Up @@ -4356,7 +4395,33 @@ may increase either contention or retry errors, or both.`,
int64(*args[0].(*tree.DInt)), // parentID
int64(*args[1].(*tree.DInt)), // parentSchemaID
string(*args[2].(*tree.DString)), // name
int64(*args[3].(*tree.DInt)), // descID
int64(*args[3].(*tree.DInt)), // id
false, // force
); err != nil {
return nil, err
}
return tree.DBoolTrue, nil
},
Info: "This function is used only by CockroachDB's developers for testing purposes.",
Volatility: tree.VolatilityVolatile,
},
tree.Overload{
Types: tree.ArgTypes{
{"parent_id", types.Int},
{"parent_schema_id", types.Int},
{"name", types.String},
{"desc_id", types.Int},
{"force", types.Bool},
},
ReturnType: tree.FixedReturnType(types.Bool),
Fn: func(ctx *tree.EvalContext, args tree.Datums) (tree.Datum, error) {
if err := ctx.Planner.UnsafeDeleteNamespaceEntry(
ctx.Context,
int64(*args[0].(*tree.DInt)), // parentID
int64(*args[1].(*tree.DInt)), // parentSchemaID
string(*args[2].(*tree.DString)), // name
int64(*args[3].(*tree.DInt)), // id
bool(*args[4].(*tree.DBool)), // force
); err != nil {
return nil, err
}
Expand Down
22 changes: 18 additions & 4 deletions pkg/sql/sem/tree/eval.go
Original file line number Diff line number Diff line change
Expand Up @@ -2990,19 +2990,33 @@ type EvalPlanner interface {

// UnsafeUpsertDescriptor is a used to repair descriptors in dire
// circumstances. See the comment on the planner implementation.
UnsafeUpsertDescriptor(ctx context.Context, descID int64, encodedDescriptor []byte) error
UnsafeUpsertDescriptor(
ctx context.Context, descID int64, encodedDescriptor []byte, force bool,
) error

// UnsafeDeleteDescriptor is a used to repair descriptors in dire
// circumstances. See the comment on the planner implementation.
UnsafeDeleteDescriptor(ctx context.Context, descID int64) error
UnsafeDeleteDescriptor(ctx context.Context, descID int64, force bool) error

// UnsafeUpsertNamespaceEntry is a used to repair namespace entries in dire
// circumstances. See the comment on the planner implementation.
UnsafeUpsertNamespaceEntry(ctx context.Context, parentID, parentSchemaID int64, name string, descID int64, force bool) error
UnsafeUpsertNamespaceEntry(
ctx context.Context,
parentID, parentSchemaID int64,
name string,
descID int64,
force bool,
) error

// UnsafeDeleteNamespaceEntry is a used to repair namespace entries in dire
// circumstances. See the comment on the planner implementation.
UnsafeDeleteNamespaceEntry(ctx context.Context, parentID, parentSchemaID int64, name string, descID int64) error
UnsafeDeleteNamespaceEntry(
ctx context.Context,
parentID, parentSchemaID int64,
name string,
descID int64,
force bool,
) error
}

// EvalSessionAccessor is a limited interface to access session variables.
Expand Down
Loading

0 comments on commit 89dd2d1

Please sign in to comment.