Skip to content

Commit

Permalink
Merge #88535 #89124
Browse files Browse the repository at this point in the history
88535: sql: reduce job times for randomized syntax change tests r=fqazi a=fqazi

Fixes: #86366
Fixes #87569

Previously, the exponential back off time for randomized syntax change tests was 20 seconds, which was inappropriate for some randomized tests, since transaction retry errors could lead to schema changes taking a really long time. This could cause the test to fail with timeout errors, since we would incorrectly think that connections were hung. To address this, this patch
makes the exponential back off configurable for the purpose of testing.

Release note: None
Release justification: low risk change to improve test stability

89124: builtins: allow null in enum_* functions r=[rafiss] a=HonoreDB

Our enum functions don't quite have parity with their Postgres equivalents. Even though all they need is the type of their argument, they still evaluate it and error if the result is null. This PR fixes that, not in the ambitious way suggested in #78358 of persisting type info on nulls, but in the more limited way of changing the function implementations to act on their arguments after type resolution but before evaluation.

Informs #78358

Release note (sql change): enum_first, enum_last, and enum_range may now take null arguments as long as their type can be inferred from the expression.
Example: select num_first(null::switch);

Co-authored-by: Faizan Qazi <[email protected]>
Co-authored-by: Aaron Zinger <[email protected]>
  • Loading branch information
3 people committed Oct 3, 2022
3 parents ca31896 + 5bd588e + a93491d commit b5e54ff
Show file tree
Hide file tree
Showing 4 changed files with 66 additions and 30 deletions.
17 changes: 17 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/enums
Original file line number Diff line number Diff line change
Expand Up @@ -143,6 +143,23 @@ SELECT enum_first('mysql'::dbs), enum_last('spanner'::dbs)
----
postgres cockroach

query T
SELECT enum_first(null::dbs)
----
postgres

query T
SELECT enum_first(val) FROM unnest(array_append(enum_range(null::dbs),null)) val
----
postgres
postgres
postgres
postgres
postgres

statement error input expression must always resolve to the same enum type
SELECT enum_first(null)

query T
SELECT enum_range('cockroach'::dbs)
----
Expand Down
10 changes: 9 additions & 1 deletion pkg/sql/schema_changer.go
Original file line number Diff line number Diff line change
Expand Up @@ -68,6 +68,14 @@ import (
"github.com/cockroachdb/logtags"
)

var schemaChangeJobMaxRetryBackoff = settings.RegisterDurationSetting(
settings.TenantWritable,
"schemachanger.job.max_retry_backoff",
"the exponential back off when retrying jobs for schema changes",
20*time.Second,
settings.PositiveDuration,
)

const (
// RunningStatusWaitingForMVCCGC is used for the GC job when it has cleared
// the data but is waiting for MVCC GC to remove the data.
Expand Down Expand Up @@ -2594,7 +2602,7 @@ func (r schemaChangeResumer) Resume(ctx context.Context, execCtx interface{}) er
}
opts := retry.Options{
InitialBackoff: 20 * time.Millisecond,
MaxBackoff: 20 * time.Second,
MaxBackoff: schemaChangeJobMaxRetryBackoff.Get(p.ExecCfg().SV()),
Multiplier: 1.5,
}

Expand Down
66 changes: 37 additions & 29 deletions pkg/sql/sem/builtins/builtins.go
Original file line number Diff line number Diff line change
Expand Up @@ -4220,17 +4220,14 @@ value if you rely on the HLC for accuracy.`,
tree.Overload{
Types: tree.ArgTypes{{"val", types.AnyEnum}},
ReturnType: tree.IdentityReturnType(0),
Fn: func(evalCtx *eval.Context, args tree.Datums) (tree.Datum, error) {
if args[0] == tree.DNull {
return nil, pgerror.Newf(pgcode.NullValueNotAllowed, "argument cannot be NULL")
}
arg := args[0].(*tree.DEnum)
min, ok := arg.MinWriteable()
FnWithExprs: makeEnumTypeFunc(func(enumType *types.T) (tree.Datum, error) {
enum := tree.DEnum{EnumTyp: enumType}
min, ok := enum.MinWriteable()
if !ok {
return nil, errors.Newf("enum %s contains no values", arg.ResolvedType().Name())
return nil, errors.Newf("enum %s contains no values", enumType.Name())
}
return min, nil
},
}),
Info: "Returns the first value of the input enum type.",
Volatility: volatility.Stable,
CalledOnNullInput: true,
Expand All @@ -4242,17 +4239,14 @@ value if you rely on the HLC for accuracy.`,
tree.Overload{
Types: tree.ArgTypes{{"val", types.AnyEnum}},
ReturnType: tree.IdentityReturnType(0),
Fn: func(evalCtx *eval.Context, args tree.Datums) (tree.Datum, error) {
if args[0] == tree.DNull {
return nil, pgerror.Newf(pgcode.NullValueNotAllowed, "argument cannot be NULL")
}
arg := args[0].(*tree.DEnum)
max, ok := arg.MaxWriteable()
FnWithExprs: makeEnumTypeFunc(func(enumType *types.T) (tree.Datum, error) {
enum := tree.DEnum{EnumTyp: enumType}
max, ok := enum.MaxWriteable()
if !ok {
return nil, errors.Newf("enum %s contains no values", arg.ResolvedType().Name())
return nil, errors.Newf("enum %s contains no values", enumType.Name())
}
return max, nil
},
}),
Info: "Returns the last value of the input enum type.",
Volatility: volatility.Stable,
CalledOnNullInput: true,
Expand All @@ -4264,29 +4258,24 @@ value if you rely on the HLC for accuracy.`,
tree.Overload{
Types: tree.ArgTypes{{"val", types.AnyEnum}},
ReturnType: tree.ArrayOfFirstNonNullReturnType(),
Fn: func(evalCtx *eval.Context, args tree.Datums) (tree.Datum, error) {
if args[0] == tree.DNull {
return nil, pgerror.Newf(pgcode.NullValueNotAllowed, "argument cannot be NULL")
}
arg := args[0].(*tree.DEnum)
typ := arg.EnumTyp
arr := tree.NewDArray(typ)
for i := range typ.TypeMeta.EnumData.LogicalRepresentations {
FnWithExprs: makeEnumTypeFunc(func(enumType *types.T) (tree.Datum, error) {
arr := tree.NewDArray(enumType)
for i := range enumType.TypeMeta.EnumData.LogicalRepresentations {
// Read-only members should be excluded.
if typ.TypeMeta.EnumData.IsMemberReadOnly[i] {
if enumType.TypeMeta.EnumData.IsMemberReadOnly[i] {
continue
}
enum := &tree.DEnum{
EnumTyp: typ,
PhysicalRep: typ.TypeMeta.EnumData.PhysicalRepresentations[i],
LogicalRep: typ.TypeMeta.EnumData.LogicalRepresentations[i],
EnumTyp: enumType,
PhysicalRep: enumType.TypeMeta.EnumData.PhysicalRepresentations[i],
LogicalRep: enumType.TypeMeta.EnumData.LogicalRepresentations[i],
}
if err := arr.Append(enum); err != nil {
return nil, err
}
}
return arr, nil
},
}),
Info: "Returns all values of the input enum in an ordered array.",
Volatility: volatility.Stable,
CalledOnNullInput: true,
Expand Down Expand Up @@ -9283,6 +9272,25 @@ func extractTimeSpanFromTimestamp(
}
}

// makeEnumTypeFunc creates a FnWithExprs for a function that takes an enum but only cares about its
// type.
func makeEnumTypeFunc(impl func(t *types.T) (tree.Datum, error)) tree.FnWithExprsOverload {
return eval.FnWithExprsOverload(func(
evalCtx *eval.Context, args tree.Exprs,
) (tree.Datum, error) {
enumType := args[0].(tree.TypedExpr).ResolvedType()
if enumType == types.Unknown || enumType == types.AnyEnum {
return nil, errors.WithHint(pgerror.New(pgcode.InvalidParameterValue, "input expression must always resolve to the same enum type"),
"Try NULL::yourenumtype")
}
// This assertion failure is necessary so that the type resolver knows it needs to hydrate.
if !enumType.IsHydrated() {
return nil, errors.AssertionFailedf("unhydrated type %+v", enumType)
}
return impl(enumType)
})
}

func truncateTime(fromTime *tree.DTime, timeSpan string) (*tree.DTime, error) {
t := timeofday.TimeOfDay(*fromTime)
hour := t.Hour()
Expand Down
3 changes: 3 additions & 0 deletions pkg/sql/tests/rsg_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
"github.com/lib/pq"
"github.com/stretchr/testify/require"
)

var (
Expand Down Expand Up @@ -778,6 +779,8 @@ func testRandomSyntax(
s, rawDB, _ := serverutils.StartServer(t, params)
defer s.Stopper().Stop(ctx)
db := &verifyFormatDB{db: rawDB}
err := db.exec(t, ctx, "SET CLUSTER SETTING schemachanger.job.max_retry_backoff='1s'")
require.NoError(t, err)

yBytes, err := os.ReadFile(testutils.TestDataPath(t, "rsg", "sql.y"))
if err != nil {
Expand Down

0 comments on commit b5e54ff

Please sign in to comment.