Skip to content

Commit

Permalink
colexecbase: improve randomized test of casts
Browse files Browse the repository at this point in the history
Release note: None
  • Loading branch information
yuzefovich committed Jul 22, 2021
1 parent 5e1c089 commit 4ad9b52
Show file tree
Hide file tree
Showing 5 changed files with 66 additions and 158 deletions.
2 changes: 2 additions & 0 deletions pkg/sql/colexec/colexecbase/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,7 @@ go_test(
"//pkg/col/coldataext",
"//pkg/col/coldatatestutils",
"//pkg/settings/cluster",
"//pkg/sql/colconv",
"//pkg/sql/colexec",
"//pkg/sql/colexec/colbuilder",
"//pkg/sql/colexec/colexecargs",
Expand All @@ -61,6 +62,7 @@ go_test(
"//pkg/sql/colmem",
"//pkg/sql/execinfra",
"//pkg/sql/execinfrapb",
"//pkg/sql/faketreeeval",
"//pkg/sql/randgen",
"//pkg/sql/sem/tree",
"//pkg/sql/types",
Expand Down
203 changes: 48 additions & 155 deletions pkg/sql/colexec/colexecbase/cast_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,9 +18,11 @@ import (
"github.com/cockroachdb/cockroach/pkg/col/coldata"
"github.com/cockroachdb/cockroach/pkg/col/coldatatestutils"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/colconv"
"github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecbase"
"github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils"
"github.com/cockroachdb/cockroach/pkg/sql/colexecop"
"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
"github.com/cockroachdb/cockroach/pkg/sql/faketreeeval"
"github.com/cockroachdb/cockroach/pkg/sql/randgen"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/types"
Expand All @@ -37,159 +39,69 @@ func TestRandomizedCast(t *testing.T) {
st := cluster.MakeTestingClusterSettings()
evalCtx := tree.MakeTestingEvalContext(st)
defer evalCtx.Stop(ctx)
flowCtx := &execinfra.FlowCtx{
EvalCtx: &evalCtx,
Cfg: &execinfra.ServerConfig{
Settings: st,
},
}
evalCtx.Planner = &faketreeeval.DummyEvalPlanner{}
rng, _ := randutil.NewPseudoRand()

datumAsBool := func(d tree.Datum) interface{} {
return bool(tree.MustBeDBool(d))
}
datumAsInt := func(d tree.Datum) interface{} {
return int(tree.MustBeDInt(d))
}
datumAsInt4 := func(d tree.Datum) interface{} {
return int32(tree.MustBeDInt(d))
}
datumAsInt2 := func(d tree.Datum) interface{} {
return int16(tree.MustBeDInt(d))
}
datumAsFloat := func(d tree.Datum) interface{} {
return float64(tree.MustBeDFloat(d))
}
datumAsDecimal := func(d tree.Datum) interface{} {
return tree.MustBeDDecimal(d).Decimal
}
datumAsColdataextDatum := func(datumVec coldata.DatumVec, d tree.Datum) interface{} {
datumVec.Set(0, d)
return datumVec.Get(0)
}
makeDatumVecAdapter := func(datumVec coldata.DatumVec) func(tree.Datum) interface{} {
return func(d tree.Datum) interface{} {
return datumAsColdataextDatum(datumVec, d)
}
}

collatedStringType := types.MakeCollatedString(types.String, *randgen.RandCollationLocale(rng))
collatedStringVec := testColumnFactory.MakeColumn(collatedStringType, 1 /* n */).(coldata.DatumVec)
getCollatedStringsThatCanBeCastAsBools := func() []tree.Datum {
var res []tree.Datum
for _, validString := range []string{"true", "false", "yes", "no"} {
d, err := tree.NewDCollatedString(validString, collatedStringType.Locale(), &tree.CollationEnvironment{})
if err != nil {
t.Fatal(err)
getValidSupportedCast := func() (from, to *types.T) {
for {
from, to = randgen.RandType(rng), randgen.RandType(rng)
if _, ok := tree.LookupCastVolatility(from, to); ok {
if colexecbase.IsCastSupported(from, to) {
return from, to
}
}
res = append(res, d)
}
return res
}

tc := []struct {
fromTyp *types.T
fromPhysType func(tree.Datum) interface{}
toTyp *types.T
toPhysType func(tree.Datum) interface{}
// getValidSet (when non-nil) is a function that returns a set of valid
// datums of fromTyp type that can be cast to toTyp type. The test
// harness will be randomly choosing a datum from this set. This
// function should be specified when rowenc.RandDatum will take ages
// (if ever) to generate the datum that is valid for a cast.
getValidSet func() []tree.Datum
// Some types casting can fail, so retry if we generate a datum that is
// unable to be cast.
retryGeneration bool
}{
//bool -> t tests
{fromTyp: types.Bool, fromPhysType: datumAsBool, toTyp: types.Bool, toPhysType: datumAsBool},
{fromTyp: types.Bool, fromPhysType: datumAsBool, toTyp: types.Int, toPhysType: datumAsInt},
{fromTyp: types.Bool, fromPhysType: datumAsBool, toTyp: types.Int4, toPhysType: datumAsInt4},
{fromTyp: types.Bool, fromPhysType: datumAsBool, toTyp: types.Int2, toPhysType: datumAsInt2},
{fromTyp: types.Bool, fromPhysType: datumAsBool, toTyp: types.Float, toPhysType: datumAsFloat},
// decimal -> t tests
{fromTyp: types.Decimal, fromPhysType: datumAsDecimal, toTyp: types.Bool, toPhysType: datumAsBool},
{fromTyp: types.Decimal, fromPhysType: datumAsDecimal, toTyp: types.Int, toPhysType: datumAsInt, retryGeneration: true},
{fromTyp: types.Decimal, fromPhysType: datumAsDecimal, toTyp: types.Int4, toPhysType: datumAsInt4, retryGeneration: true},
{fromTyp: types.Decimal, fromPhysType: datumAsDecimal, toTyp: types.Int2, toPhysType: datumAsInt2, retryGeneration: true},
{fromTyp: types.Decimal, fromPhysType: datumAsDecimal, toTyp: types.Float, toPhysType: datumAsFloat, retryGeneration: true},
{fromTyp: types.Decimal, fromPhysType: datumAsDecimal, toTyp: types.Decimal, toPhysType: datumAsDecimal},
// int -> t tests
{fromTyp: types.Int, fromPhysType: datumAsInt, toTyp: types.Bool, toPhysType: datumAsBool},
{fromTyp: types.Int, fromPhysType: datumAsInt, toTyp: types.Int, toPhysType: datumAsInt},
{fromTyp: types.Int, fromPhysType: datumAsInt, toTyp: types.Int4, toPhysType: datumAsInt4, retryGeneration: true},
{fromTyp: types.Int, fromPhysType: datumAsInt, toTyp: types.Int2, toPhysType: datumAsInt2, retryGeneration: true},
{fromTyp: types.Int, fromPhysType: datumAsInt, toTyp: types.Float, toPhysType: datumAsFloat},
{fromTyp: types.Int, fromPhysType: datumAsInt, toTyp: types.Decimal, toPhysType: datumAsDecimal},
// int4 -> t tests
{fromTyp: types.Int4, fromPhysType: datumAsInt4, toTyp: types.Bool, toPhysType: datumAsBool},
{fromTyp: types.Int4, fromPhysType: datumAsInt4, toTyp: types.Int, toPhysType: datumAsInt},
{fromTyp: types.Int4, fromPhysType: datumAsInt4, toTyp: types.Int4, toPhysType: datumAsInt4},
{fromTyp: types.Int4, fromPhysType: datumAsInt4, toTyp: types.Int2, toPhysType: datumAsInt2, retryGeneration: true},
{fromTyp: types.Int4, fromPhysType: datumAsInt4, toTyp: types.Float, toPhysType: datumAsFloat},
{fromTyp: types.Int4, fromPhysType: datumAsInt4, toTyp: types.Decimal, toPhysType: datumAsDecimal},
// int2 -> t tests
{fromTyp: types.Int2, fromPhysType: datumAsInt2, toTyp: types.Bool, toPhysType: datumAsBool},
{fromTyp: types.Int2, fromPhysType: datumAsInt2, toTyp: types.Int, toPhysType: datumAsInt},
{fromTyp: types.Int2, fromPhysType: datumAsInt2, toTyp: types.Int4, toPhysType: datumAsInt},
{fromTyp: types.Int2, fromPhysType: datumAsInt2, toTyp: types.Int2, toPhysType: datumAsInt2},
{fromTyp: types.Int2, fromPhysType: datumAsInt2, toTyp: types.Float, toPhysType: datumAsFloat},
{fromTyp: types.Int2, fromPhysType: datumAsInt2, toTyp: types.Decimal, toPhysType: datumAsDecimal},
// float -> t tests
{fromTyp: types.Float, fromPhysType: datumAsFloat, toTyp: types.Bool, toPhysType: datumAsBool},
// We can sometimes generate a float outside of the range of the integers,
// so we want to retry with generation if that occurs.
{fromTyp: types.Float, fromPhysType: datumAsFloat, toTyp: types.Int, toPhysType: datumAsInt, retryGeneration: true},
{fromTyp: types.Float, fromPhysType: datumAsFloat, toTyp: types.Int4, toPhysType: datumAsInt4, retryGeneration: true},
{fromTyp: types.Float, fromPhysType: datumAsFloat, toTyp: types.Int2, toPhysType: datumAsInt2, retryGeneration: true},
{fromTyp: types.Float, fromPhysType: datumAsFloat, toTyp: types.Float, toPhysType: datumAsFloat},
{fromTyp: types.Float, fromPhysType: datumAsFloat, toTyp: types.Decimal, toPhysType: datumAsDecimal},
// datum-backed type -> t tests
{fromTyp: collatedStringType, fromPhysType: makeDatumVecAdapter(collatedStringVec), toTyp: types.Bool, toPhysType: datumAsBool, getValidSet: getCollatedStringsThatCanBeCastAsBools},
}

for _, c := range tc {
log.Infof(ctx, "%s to %s", c.fromTyp.String(), c.toTyp.String())
n := 100
// Make an input vector of length n.
const numTypePairs = 5
numRows := 1 + rng.Intn(coldata.BatchSize()) + rng.Intn(3)*coldata.BatchSize()
log.Infof(ctx, "num rows = %d", numRows)
for run := 0; run < numTypePairs; run++ {
from, to := getValidSupportedCast()
log.Infof(ctx, "%s to %s", from.String(), to.String())
input := colexectestutils.Tuples{}
output := colexectestutils.Tuples{}
for i := 0; i < n; i++ {
fromConverter := colconv.GetDatumToPhysicalFn(from)
toConverter := colconv.GetDatumToPhysicalFn(to)
errorExpected := false
for i := 0; i < numRows; i++ {
var (
fromDatum, toDatum tree.Datum
err error
)
if c.getValidSet != nil {
validFromDatums := c.getValidSet()
fromDatum = validFromDatums[rng.Intn(len(validFromDatums))]
toDatum, err = tree.PerformCast(&evalCtx, fromDatum, c.toTyp)
} else {
for {
// We don't allow any NULL datums to be generated, so disable
// this ability in the RandDatum function.
fromDatum = randgen.RandDatum(rng, c.fromTyp, false)
toDatum, err = tree.PerformCast(&evalCtx, fromDatum, c.toTyp)
if c.retryGeneration {
for err != nil {
// If we are allowed to retry, make a new datum and cast it on error.
fromDatum = randgen.RandDatum(rng, c.fromTyp, false)
toDatum, err = tree.PerformCast(&evalCtx, fromDatum, c.toTyp)
}
fromDatum = randgen.RandDatum(rng, from, false)
toDatum, err = tree.PerformCast(&evalCtx, fromDatum, to)
if to.String() == "char" && string(*toDatum.(*tree.DString)) == "" {
// There is currently a problem when converting an empty
// string datum to a physical representation, so we skip
// such a datum and retry generation.
// TODO(yuzefovich): figure it out.
continue
}
break
}
var toPhys interface{}
if err != nil {
t.Fatal(err)
errorExpected = true
} else {
toPhys = toConverter(toDatum)
}
input = append(input, colexectestutils.Tuple{c.fromPhysType(fromDatum)})
output = append(output, colexectestutils.Tuple{c.fromPhysType(fromDatum), c.toPhysType(toDatum)})
input = append(input, colexectestutils.Tuple{fromConverter(fromDatum)})
output = append(output, colexectestutils.Tuple{fromConverter(fromDatum), toPhys})
}
// TODO(yuzefovich): once the test harness is updated to allow for
// expected errors to occur, check that the expected errors when casting
// do, in fact, occur rather than "retrying the generation" above.
colexectestutils.RunTestsWithTyps(t, testAllocator, []colexectestutils.Tuples{input}, [][]*types.T{{c.fromTyp}}, output, colexectestutils.OrderedVerifier,
colexectestutils.RunTestsWithoutAllNullsInjectionWithErrorHandler(
t, testAllocator, []colexectestutils.Tuples{input}, [][]*types.T{{from}}, output, colexectestutils.OrderedVerifier,
func(input []colexecop.Operator) (colexecop.Operator, error) {
return createTestCastOperator(ctx, flowCtx, input[0], c.fromTyp, c.toTyp)
})
return colexecbase.GetCastOperator(testAllocator, input[0], 0, 1, from, to)
},
func(err error) {
if !errorExpected {
t.Fatal(err)
}
},
)
}
}

Expand All @@ -199,12 +111,6 @@ func BenchmarkCastOp(b *testing.B) {
st := cluster.MakeTestingClusterSettings()
evalCtx := tree.MakeTestingEvalContext(st)
defer evalCtx.Stop(ctx)
flowCtx := &execinfra.FlowCtx{
EvalCtx: &evalCtx,
Cfg: &execinfra.ServerConfig{
Settings: st,
},
}
rng, _ := randutil.NewPseudoRand()
for _, typePair := range [][]*types.T{
{types.Int, types.Float},
Expand All @@ -231,7 +137,7 @@ func BenchmarkCastOp(b *testing.B) {
coldata.BatchSize(), nullProbability, selectivity,
)
source := colexecop.NewRepeatableBatchSource(testAllocator, batch, typs)
op, err := createTestCastOperator(ctx, flowCtx, source, typePair[0], typePair[1])
op, err := colexecbase.GetCastOperator(testAllocator, source, 0, 1, typePair[0], typePair[1])
require.NoError(b, err)
b.SetBytes(int64(8 * coldata.BatchSize()))
b.ResetTimer()
Expand All @@ -244,16 +150,3 @@ func BenchmarkCastOp(b *testing.B) {
}
}
}

func createTestCastOperator(
ctx context.Context,
flowCtx *execinfra.FlowCtx,
input colexecop.Operator,
fromTyp *types.T,
toTyp *types.T,
) (colexecop.Operator, error) {
return colexectestutils.CreateTestProjectingOperator(
ctx, flowCtx, input, []*types.T{fromTyp},
fmt.Sprintf("@1::%s", toTyp.Name()), false /* canFallbackToRowexec */, testMemAcc,
)
}
2 changes: 2 additions & 0 deletions pkg/sql/colexec/colexecbase/main_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/colexecerror"
"github.com/cockroachdb/cockroach/pkg/sql/colmem"
"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
"github.com/cockroachdb/cockroach/pkg/sql/faketreeeval"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
"github.com/cockroachdb/cockroach/pkg/util/mon"
Expand Down Expand Up @@ -56,6 +57,7 @@ func TestMain(m *testing.M) {
memAcc := testMemMonitor.MakeBoundAccount()
testMemAcc = &memAcc
evalCtx := tree.MakeTestingEvalContext(st)
evalCtx.Planner = &faketreeeval.DummyEvalPlanner{}
testColumnFactory = coldataext.NewExtendedColumnFactory(&evalCtx)
testAllocator = colmem.NewAllocator(ctx, testMemAcc, testColumnFactory)
defer testMemAcc.Close(ctx)
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/colexec/colexectestutils/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ go_library(
"//pkg/util/mon",
"//pkg/util/randutil",
"//pkg/util/timeofday",
"//pkg/util/timeutil",
"@com_github_cockroachdb_apd_v2//:apd",
"@com_github_cockroachdb_errors//:errors",
"@com_github_pmezard_go_difflib//difflib",
Expand Down
16 changes: 13 additions & 3 deletions pkg/sql/colexec/colexectestutils/utils.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
"github.com/cockroachdb/cockroach/pkg/util/timeofday"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
"github.com/pmezard/go-difflib/difflib"
"github.com/stretchr/testify/assert"
Expand Down Expand Up @@ -972,6 +973,10 @@ func (s *opTestInput) Next() coldata.Batch {
colexecerror.InternalError(errors.AssertionFailedf("%v", err))
}
setColVal(vec, outputIdx, j, s.evalCtx)
case types.TimestampTZFamily:
t := timeutil.Unix(rng.Int63n(2000000000), rng.Int63n(1000000))
t.Round(tree.TimeFamilyPrecisionToRoundDuration(vec.Type().Precision()))
setColVal(vec, outputIdx, t, s.evalCtx)
case typeconv.DatumVecCanonicalTypeFamily:
switch vec.Type().Family() {
case types.CollatedStringFamily:
Expand All @@ -988,7 +993,10 @@ func (s *opTestInput) Next() coldata.Batch {
case types.TupleFamily:
setColVal(vec, outputIdx, stringToDatum("(NULL)", vec.Type(), s.evalCtx), s.evalCtx)
default:
colexecerror.InternalError(errors.AssertionFailedf("unexpected datum-backed type: %s", vec.Type()))
// For other datum-backed types we'll be lazy and
// won't set the garbage values. We should already
// have good coverage with other types.
continue
}
default:
if val, ok := quick.Value(reflect.TypeOf(vec.Col()).Elem(), rng); ok {
Expand Down Expand Up @@ -1277,8 +1285,10 @@ func tupleEquals(expected Tuple, actual Tuple, evalCtx *tree.EvalContext) bool {
return false
}
for i := 0; i < len(actual); i++ {
if expected[i] == nil || actual[i] == nil {
if expected[i] != nil || actual[i] != nil {
expectedIsNull := expected[i] == nil || expected[i] == tree.DNull
actualIsNull := actual[i] == nil || actual[i] == tree.DNull
if expectedIsNull || actualIsNull {
if !expectedIsNull || !actualIsNull {
return false
}
} else {
Expand Down

0 comments on commit 4ad9b52

Please sign in to comment.