Skip to content

Commit

Permalink
exec: add unit test for AND operator
Browse files Browse the repository at this point in the history
This commit adds a unit test for AND operator. It also extends our
testing infrastructure to be able to specify the types schema (this
was needed because nil values are treated as Int64 by default which
is incorrect for tests of AND operator).

Release note: None
  • Loading branch information
yuzefovich committed Sep 11, 2019
1 parent 10193ad commit 37b76c1
Show file tree
Hide file tree
Showing 6 changed files with 155 additions and 34 deletions.
2 changes: 1 addition & 1 deletion pkg/sql/exec/aggregator_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -263,7 +263,7 @@ func TestAggregatorOneFunc(t *testing.T) {
t.Fatal(err)
}

tupleSource := newOpTestInput(uint16(tc.batchSize), tc.input)
tupleSource := newOpTestInput(uint16(tc.batchSize), tc.input, nil /* typs */)
a, err := NewOrderedAggregator(
tupleSource,
tc.colTypes,
Expand Down
94 changes: 94 additions & 0 deletions pkg/sql/exec/and_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,94 @@
// Copyright 2019 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package exec

import (
"testing"

"github.com/cockroachdb/cockroach/pkg/col/coltypes"
)

func TestAndOp(t *testing.T) {
tcs := []struct {
tuples []tuple
expected []tuple
}{
// All variations of pairs separately first.
{
tuples: tuples{{false, true}},
expected: tuples{{false}},
},
{
tuples: tuples{{false, nil}},
expected: tuples{{false}},
},
{
tuples: tuples{{false, false}},
expected: tuples{{false}},
},
{
tuples: tuples{{true, true}},
expected: tuples{{true}},
},
{
tuples: tuples{{true, false}},
expected: tuples{{false}},
},
{
tuples: tuples{{true, nil}},
expected: tuples{{nil}},
},
{
tuples: tuples{{nil, true}},
expected: tuples{{nil}},
},
{
tuples: tuples{{nil, false}},
expected: tuples{{false}},
},
{
tuples: tuples{{nil, nil}},
expected: tuples{{nil}},
},
// Now all variations of pairs combined together to make sure that nothing
// funky going on with multiple tuples.
{
tuples: tuples{
{false, true}, {false, nil}, {false, false},
{true, true}, {true, false}, {true, nil},
{nil, true}, {nil, false}, {nil, nil},
},
expected: tuples{
{false}, {false}, {false},
{true}, {false}, {nil},
{nil}, {false}, {nil},
},
},
}

for _, tc := range tcs {
runTestsWithTyps(
t,
[]tuples{tc.tuples},
[]coltypes.T{coltypes.Bool, coltypes.Bool},
tc.expected,
orderedVerifier,
[]int{2},
func(input []Operator) (Operator, error) {
return NewAndOp(
input[0],
0, /* leftIdx */
1, /* rightIdx */
2, /* outputIdx */
), nil
})
}
}
2 changes: 1 addition & 1 deletion pkg/sql/exec/orderedsynchronizer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -178,7 +178,7 @@ func TestOrderedSyncRandomInput(t *testing.T) {
}
inputs := make([]Operator, numInputs)
for i := range inputs {
inputs[i] = newOpTestInput(batchSize, sources[i])
inputs[i] = newOpTestInput(batchSize, sources[i], nil /* typs */)
}

op := OrderedSynchronizer{
Expand Down
12 changes: 6 additions & 6 deletions pkg/sql/exec/routers_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -97,7 +97,7 @@ func TestRouterOutputAddBatch(t *testing.T) {
o := newRouterOutputOpWithBlockedThresholdAndBatchSize(
[]coltypes.T{coltypes.Int64}, unblockEventsChan, tc.blockedThreshold, tc.outputBatchSize,
)
in := newOpTestInput(tc.inputBatchSize, data)
in := newOpTestInput(tc.inputBatchSize, data, nil /* typs */)
out := newOpTestOutput(o, []int{0}, data[:len(tc.selection)])
in.Init()
for {
Expand Down Expand Up @@ -177,7 +177,7 @@ func TestRouterOutputNext(t *testing.T) {
var wg sync.WaitGroup
batchChan := make(chan coldata.Batch)
o := newRouterOutputOp([]coltypes.T{coltypes.Int64}, unblockedEventsChan)
in := newOpTestInput(coldata.BatchSize, data)
in := newOpTestInput(coldata.BatchSize, data, nil /* typs */)
in.Init()
wg.Add(1)
go func() {
Expand Down Expand Up @@ -257,7 +257,7 @@ func TestRouterOutputNext(t *testing.T) {
o := newRouterOutputOpWithBlockedThresholdAndBatchSize(
[]coltypes.T{coltypes.Int64}, ch, blockThreshold, coldata.BatchSize,
)
in := newOpTestInput(smallBatchSize, data)
in := newOpTestInput(smallBatchSize, data, nil /* typs */)
out := newOpTestOutput(o, []int{0}, expected)
in.Init()

Expand Down Expand Up @@ -322,7 +322,7 @@ func TestRouterOutputRandom(t *testing.T) {
"blockedThreshold=%d/outputSize=%d/totalInputSize=%d", blockedThreshold, outputSize, len(data),
)
t.Run(testName, func(t *testing.T) {
runTestsWithFn(t, []tuples{data}, func(t *testing.T, inputs []Operator) {
runTestsWithFn(t, []tuples{data}, nil /* typs */, func(t *testing.T, inputs []Operator) {
var wg sync.WaitGroup
unblockedEventsChans := make(chan struct{}, 2)
o := newRouterOutputOpWithBlockedThresholdAndBatchSize(
Expand Down Expand Up @@ -448,7 +448,7 @@ func TestHashRouterComputesDestination(t *testing.T) {
valsYetToSee[int64(i)] = struct{}{}
}

in := newOpTestInput(coldata.BatchSize, data)
in := newOpTestInput(coldata.BatchSize, data, nil /* typs */)
in.Init()

var (
Expand Down Expand Up @@ -696,7 +696,7 @@ func TestHashRouterRandom(t *testing.T) {
// same data to the same number of outputs.
var expectedDistribution []int
t.Run(testName, func(t *testing.T) {
runTestsWithFn(t, []tuples{data}, func(t *testing.T, inputs []Operator) {
runTestsWithFn(t, []tuples{data}, nil /* typs */, func(t *testing.T, inputs []Operator) {
unblockEventsChan := make(chan struct{}, 2*numOutputs)
outputs := make([]routerOutput, numOutputs)
outputsAsOps := make([]Operator, numOutputs)
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/exec/sort_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -247,7 +247,7 @@ func TestAllSpooler(t *testing.T) {
},
}
for _, tc := range tcs {
runTestsWithFn(t, []tuples{tc.tuples}, func(t *testing.T, input []Operator) {
runTestsWithFn(t, []tuples{tc.tuples}, nil /* typs */, func(t *testing.T, input []Operator) {
allSpooler := newAllSpooler(input[0], tc.typ)
allSpooler.init()
allSpooler.spool(context.Background())
Expand Down
77 changes: 52 additions & 25 deletions pkg/sql/exec/utils_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -89,7 +89,21 @@ func runTests(
cols []int,
constructor func(inputs []Operator) (Operator, error),
) {
runTestsWithFn(t, tups, func(t *testing.T, inputs []Operator) {
runTestsWithTyps(t, tups, nil /* typs */, expected, verifier, cols, constructor)
}

// runTestsWithTyps is the same as runTests with an ability to specify the
// types of the input tuples.
func runTestsWithTyps(
t *testing.T,
tups []tuples,
typs []coltypes.T,
expected tuples,
verifier verifier,
cols []int,
constructor func(inputs []Operator) (Operator, error),
) {
runTestsWithFn(t, tups, typs, func(t *testing.T, inputs []Operator) {
op, err := constructor(inputs)
if err != nil {
t.Fatal(err)
Expand All @@ -112,7 +126,7 @@ func runTests(
for round := 0; round < 2; round++ {
inputSources := make([]Operator, len(tups))
for i, tup := range tups {
inputSources[i] = newOpTestInput(1 /* batchSize */, tup)
inputSources[i] = newOpTestInput(1 /* batchSize */, tup, typs)
}
op, err := constructor(inputSources)
if err != nil {
Expand Down Expand Up @@ -162,10 +176,17 @@ func runTests(
// testing facility than runTests, because it can't get a handle on the operator
// under test and therefore can't perform as many extra checks. You should
// always prefer using runTests over runTestsWithFn.
// tups is the set of input tuples.
// test is a function that takes a list of input Operators and performs testing
// - tups is the set of input tuples.
// - typs is the type schema of the input tuples. This can be left nil in which
// case the types will be determined at the runtime looking at the first
// input tuple, and if the determination doesn't succeed for a value of the
// tuple (likely because it's a nil), then that column will be assumed by
// default of type Int64.
// - test is a function that takes a list of input Operators and performs testing
// with t.
func runTestsWithFn(t *testing.T, tups []tuples, test func(t *testing.T, inputs []Operator)) {
func runTestsWithFn(
t *testing.T, tups []tuples, typs []coltypes.T, test func(t *testing.T, inputs []Operator),
) {
rng, _ := randutil.NewPseudoRand()

for _, batchSize := range []uint16{1, 2, 3, 16, 1024} {
Expand All @@ -174,11 +195,11 @@ func runTestsWithFn(t *testing.T, tups []tuples, test func(t *testing.T, inputs
inputSources := make([]Operator, len(tups))
if useSel {
for i, tup := range tups {
inputSources[i] = newOpTestSelInput(rng, batchSize, tup)
inputSources[i] = newOpTestSelInput(rng, batchSize, tup, typs)
}
} else {
for i, tup := range tups {
inputSources[i] = newOpTestInput(batchSize, tup)
inputSources[i] = newOpTestInput(batchSize, tup, typs)
}
}
test(t, inputSources)
Expand Down Expand Up @@ -252,23 +273,27 @@ type opTestInput struct {

var _ Operator = &opTestInput{}

// newOpTestInput returns a new opTestInput with the given input tuples. The
// input tuples are translated into types automatically, using simple rules
// (e.g. integers always become Int64).
func newOpTestInput(batchSize uint16, tuples tuples) *opTestInput {
// newOpTestInput returns a new opTestInput with the given input tuples and the
// given type schema. If typs is nil, the input tuples are translated into
// types automatically, using simple rules (e.g. integers always become Int64).
func newOpTestInput(batchSize uint16, tuples tuples, typs []coltypes.T) *opTestInput {
ret := &opTestInput{
batchSize: batchSize,
tuples: tuples,
typs: typs,
}
return ret
}

func newOpTestSelInput(rng *rand.Rand, batchSize uint16, tuples tuples) *opTestInput {
func newOpTestSelInput(
rng *rand.Rand, batchSize uint16, tuples tuples, typs []coltypes.T,
) *opTestInput {
ret := &opTestInput{
useSel: true,
rng: rng,
batchSize: batchSize,
tuples: tuples,
typs: typs,
}
return ret
}
Expand All @@ -278,21 +303,23 @@ func (s *opTestInput) Init() {
execerror.VectorizedInternalPanic("empty tuple source")
}

typs := make([]coltypes.T, len(s.tuples[0]))
for i := range typs {
// Default type for test cases is Int64 in case the entire column is null
// and the type is indeterminate.
typs[i] = coltypes.Int64
for _, tup := range s.tuples {
if tup[i] != nil {
typs[i] = coltypes.FromGoType(tup[i])
break
if s.typs == nil {
// The type schema was not provided, so we need to determine it based on
// the input tuple.
s.typs = make([]coltypes.T, len(s.tuples[0]))
for i := range s.typs {
// Default type for test cases is Int64 in case the entire column is null
// and the type is indeterminate.
s.typs[i] = coltypes.Int64
for _, tup := range s.tuples {
if tup[i] != nil {
s.typs[i] = coltypes.FromGoType(tup[i])
break
}
}
}
}

s.typs = typs
s.batch = coldata.NewMemBatch(typs)
s.batch = coldata.NewMemBatch(s.typs)

s.selection = make([]uint16, coldata.BatchSize)
for i := range s.selection {
Expand Down Expand Up @@ -836,7 +863,7 @@ func TestOpTestInputOutput(t *testing.T) {
{1, 5, 0},
},
}
runTestsWithFn(t, inputs, func(t *testing.T, sources []Operator) {
runTestsWithFn(t, inputs, nil /* typs */, func(t *testing.T, sources []Operator) {
out := newOpTestOutput(sources[0], []int{0, 1, 2}, inputs[0])

if err := out.Verify(); err != nil {
Expand Down

0 comments on commit 37b76c1

Please sign in to comment.