Skip to content

Commit

Permalink
colexec: add random projection tests for all types
Browse files Browse the repository at this point in the history
This test randomly runs binary comparisons against all types with random
data, verifying that the result of Datum.Compare matches the result of
the exec projection.

This found the bug with date infinity matching tracked in cockroachdb#40354, and
immediately found a bug with the timestamp implementation in the
previous commit, so I'm fairly sure it's a useful random test to have
around.

Release note: None
  • Loading branch information
jordanlewis authored and yuzefovich committed Oct 28, 2019
1 parent 523e77f commit b007f71
Show file tree
Hide file tree
Showing 9 changed files with 111 additions and 14 deletions.
8 changes: 5 additions & 3 deletions pkg/col/colserde/arrowbatchconverter_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -119,9 +119,11 @@ func assertEqualBatches(t *testing.T, expected, actual coldata.Batch) {
func TestArrowBatchConverterRejectsUnsupportedTypes(t *testing.T) {
defer leaktest.AfterTest(t)()

typs := []coltypes.T{coltypes.Decimal, coltypes.Timestamp}
_, err := NewArrowBatchConverter(typs)
require.Error(t, err)
unsupportedTypes := []coltypes.T{coltypes.Decimal, coltypes.Timestamp}
for _, typ := range unsupportedTypes {
_, err := NewArrowBatchConverter([]coltypes.T{typ})
require.Error(t, err)
}
}

func TestArrowBatchConverterRandom(t *testing.T) {
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/colexec/builtin_funcs.go
Original file line number Diff line number Diff line change
Expand Up @@ -63,7 +63,7 @@ func (b *defaultBuiltinFuncOperator) Next(ctx context.Context) coldata.Batch {

for j := range b.argumentCols {
col := batch.ColVec(b.argumentCols[j])
b.row[j] = PhysicalTypeColElemToDatum(col, rowIdx, b.da, b.columnTypes[b.argumentCols[j]])
b.row[j] = PhysicalTypeColElemToDatum(col, rowIdx, b.da, &b.columnTypes[b.argumentCols[j]])
hasNulls = hasNulls || b.row[j] == tree.DNull
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/colexec/cfetcher.go
Original file line number Diff line number Diff line change
Expand Up @@ -788,7 +788,7 @@ func (rf *cFetcher) pushState(state fetcherState) {
// getDatumAt returns the converted datum object at the given (colIdx, rowIdx).
// This function is meant for tracing and should not be used in hot paths.
func (rf *cFetcher) getDatumAt(colIdx int, rowIdx uint16, typ types.T) tree.Datum {
return PhysicalTypeColElemToDatum(rf.machine.colvecs[colIdx], rowIdx, rf.table.da, typ)
return PhysicalTypeColElemToDatum(rf.machine.colvecs[colIdx], rowIdx, rf.table.da, &typ)
}

// processValue processes the state machine's current value component, setting
Expand Down
3 changes: 1 addition & 2 deletions pkg/sql/colexec/execgen/cmd/execgen/overloads.go
Original file line number Diff line number Diff line change
Expand Up @@ -1024,9 +1024,8 @@ func (c timestampCustomizer) getCmpOpCompareFunc() compareFunc {
func (timestampCustomizer) getHashAssignFunc() assignFunc {
return func(op overload, target, v, _ string) string {
return fmt.Sprintf(`
s, ns := %[2]s.Second(), %[2]s.Nanosecond()
s := %[2]s.UnixNano()
%[1]s = memhash64(noescape(unsafe.Pointer(&s)), %[1]s)
%[1]s = memhash64(noescape(unsafe.Pointer(&ns)), %[1]s)
`, target, v)
}
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/colexec/materializer.go
Original file line number Diff line number Diff line change
Expand Up @@ -165,7 +165,7 @@ func (m *Materializer) next() (sqlbase.EncDatumRow, *execinfrapb.ProducerMetadat
typs := m.OutputTypes()
for colIdx := 0; colIdx < len(typs); colIdx++ {
col := m.batch.ColVec(colIdx)
m.row[colIdx].Datum = PhysicalTypeColElemToDatum(col, rowIdx, m.da, typs[colIdx])
m.row[colIdx].Datum = PhysicalTypeColElemToDatum(col, rowIdx, m.da, &typs[colIdx])
}
return m.ProcessRowHelper(m.row), nil
}
Expand Down
5 changes: 3 additions & 2 deletions pkg/sql/colexec/mem_estimation.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ package colexec

import (
"fmt"
"time"
"unsafe"

"github.com/cockroachdb/cockroach/pkg/col/coltypes"
Expand All @@ -24,6 +25,7 @@ const (
sizeOfInt32 = int(unsafe.Sizeof(int32(0)))
sizeOfInt64 = int(unsafe.Sizeof(int64(0)))
sizeOfFloat64 = int(unsafe.Sizeof(float64(0)))
sizeOfTime = int(unsafe.Sizeof(time.Time{}))
)

// EstimateBatchSizeBytes returns an estimated amount of bytes needed to
Expand Down Expand Up @@ -56,8 +58,7 @@ func EstimateBatchSizeBytes(vecTypes []coltypes.T, batchLength int) int {
// to hold the arbitrary precision decimal objects.
acc += 50
case coltypes.Timestamp:
// time.Time has 2 int64s and a pointer.
acc += sizeOfInt64 * 3
acc += sizeOfTime
default:
execerror.VectorizedInternalPanic(fmt.Sprintf("unhandled type %s", t))
}
Expand Down
82 changes: 82 additions & 0 deletions pkg/sql/colexec/projection_ops_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,9 +19,14 @@ import (

"github.com/cockroachdb/cockroach/pkg/col/coldata"
"github.com/cockroachdb/cockroach/pkg/col/coltypes"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/colexec/typeconv"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
"github.com/stretchr/testify/assert"
)

func TestProjPlusInt64Int64ConstOp(t *testing.T) {
Expand Down Expand Up @@ -173,6 +178,83 @@ func TestGetProjectionConstMixedTypeOperator(t *testing.T) {
}
}

// TestRandomComparisons runs binary comparisons against all scalar types
// (supported by the vectorized engine) with random non-null data verifying
// that the result of Datum.Compare matches the result of the exec projection.
func TestRandomComparisons(t *testing.T) {
defer leaktest.AfterTest(t)()
const numTuples = 2048
rng, _ := randutil.NewPseudoRand()
evalCtx := tree.NewTestingEvalContext(cluster.MakeTestingClusterSettings())
ctx := evalCtx.Ctx()
expected := make([]bool, numTuples)
var da sqlbase.DatumAlloc
lDatums := make([]tree.Datum, numTuples)
rDatums := make([]tree.Datum, numTuples)
for _, ct := range types.Scalar {
if ct.Family() == types.DateFamily {
// TODO(jordan): #40354 tracks failure to compare infinite dates.
continue
}
typ := typeconv.FromColumnType(ct)
if typ == coltypes.Unhandled {
continue
}
typs := []coltypes.T{typ, typ, coltypes.Bool}
bytesFixedLength := 0
if ct.Family() == types.UuidFamily {
bytesFixedLength = 16
}
b := coldata.NewMemBatchWithSize(typs, numTuples)
lVec := b.ColVec(0)
rVec := b.ColVec(1)
ret := b.ColVec(2)
RandomVec(rng, typ, bytesFixedLength, lVec, numTuples, 0)
RandomVec(rng, typ, bytesFixedLength, rVec, numTuples, 0)
for i := range lDatums {
lDatums[i] = PhysicalTypeColElemToDatum(lVec, uint16(i), da, ct)
rDatums[i] = PhysicalTypeColElemToDatum(rVec, uint16(i), da, ct)
}
for _, cmpOp := range []tree.ComparisonOperator{tree.EQ, tree.NE, tree.LT, tree.LE, tree.GT, tree.GE} {
for i := range lDatums {
cmp := lDatums[i].Compare(evalCtx, rDatums[i])
var b bool
switch cmpOp {
case tree.EQ:
b = cmp == 0
case tree.NE:
b = cmp != 0
case tree.LT:
b = cmp < 0
case tree.LE:
b = cmp <= 0
case tree.GT:
b = cmp > 0
case tree.GE:
b = cmp >= 0
}
expected[i] = b
}
input := newChunkingBatchSource(typs, []coldata.Vec{lVec, rVec, ret}, numTuples)
op, err := GetProjectionOperator(ct, ct, cmpOp, input, 0, 1, 2)
if err != nil {
t.Fatal(err)
}
op.Init()
var idx uint16
for batch := op.Next(ctx); batch.Length() > 0; batch = op.Next(ctx) {
for i := uint16(0); i < batch.Length(); i++ {
absIdx := idx + i
assert.Equal(t, expected[absIdx], batch.ColVec(2).Bool()[i],
"expected %s %s %s (%s[%d]) to be %t found %t", lDatums[absIdx], cmpOp, rDatums[absIdx], ct, absIdx,
expected[absIdx], ret.Bool()[i])
}
idx += batch.Length()
}
}
}
}

func TestGetProjectionOperator(t *testing.T) {
defer leaktest.AfterTest(t)()
ct := types.Int2
Expand Down
19 changes: 16 additions & 3 deletions pkg/sql/colexec/random_testutils.go
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,16 @@ func randomTypes(rng *rand.Rand, n int) []coltypes.T {
// RandomVec populates vec with n random values of typ, setting each value to
// null with a probability of nullProbability. It is assumed that n is in bounds
// of the given vec.
func RandomVec(rng *rand.Rand, typ coltypes.T, vec coldata.Vec, n int, nullProbability float64) {
// bytesFixedLength (when greater than zero) specifies the fixed length of the
// bytes slice to be generated. It is used only if typ == coltypes.Bytes.
func RandomVec(
rng *rand.Rand,
typ coltypes.T,
bytesFixedLength int,
vec coldata.Vec,
n int,
nullProbability float64,
) {
switch typ {
case coltypes.Bool:
bools := vec.Bool()
Expand All @@ -54,7 +63,11 @@ func RandomVec(rng *rand.Rand, typ coltypes.T, vec coldata.Vec, n int, nullProba
case coltypes.Bytes:
bytes := vec.Bytes()
for i := 0; i < n; i++ {
randBytes := make([]byte, rng.Intn(maxVarLen))
bytesLen := bytesFixedLength
if bytesLen <= 0 {
bytesLen = rng.Intn(maxVarLen)
}
randBytes := make([]byte, bytesLen)
// Read always returns len(bytes[i]) and nil.
_, _ = rand.Read(randBytes)
bytes.Set(i, randBytes)
Expand Down Expand Up @@ -116,7 +129,7 @@ func RandomBatch(
length = capacity
}
for i, typ := range typs {
RandomVec(rng, typ, batch.ColVec(i), length, nullProbability)
RandomVec(rng, typ, 0 /* bytesFixedLength */, batch.ColVec(i), length, nullProbability)
}
batch.SetLength(uint16(length))
return batch
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/colexec/vec_elem_to_datum.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@ import (
// semtype ct. Note that this function handles nulls as well, so there is no
// need for a separate null check.
func PhysicalTypeColElemToDatum(
col coldata.Vec, rowIdx uint16, da sqlbase.DatumAlloc, ct types.T,
col coldata.Vec, rowIdx uint16, da sqlbase.DatumAlloc, ct *types.T,
) tree.Datum {
if col.MaybeHasNulls() {
if col.Nulls().NullAt(rowIdx) {
Expand Down

0 comments on commit b007f71

Please sign in to comment.