From f724300340d53df47a23cea06fb83a83354bff41 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Fri, 23 Aug 2019 15:00:31 -0700 Subject: [PATCH] exec: randomize types when testing columnar operators against processors This commit adds type randomization when testing columnar operators against processors. A few edge cases have been uncovered in the vectorize engine when handling special values, and those are now fixed (we might need to revisit the fixes though). Additionally, the generation of random datums has been adjusted to take into account the requested width for integers and floats. Without the adjustment, we observe different behavior between the row and the vectorized engines since the former always upcasts to the maximum width type. For example, if a special math.MaxInt64 value was generated, but the requested type was Int4, then the row engine would use the full int64 value, but the vectorized engine would use only the int32 value. Release note: None --- pkg/sql/distsqlrun/column_exec_setup.go | 6 + pkg/sql/distsqlrun/columnar_operators_test.go | 268 ++++++++++++------ pkg/sql/exec/execgen/cmd/execgen/overloads.go | 36 ++- pkg/sql/exec/hashjoiner.go | 6 +- pkg/sql/sqlbase/testutils.go | 73 ++++- 5 files changed, 278 insertions(+), 111 deletions(-) diff --git a/pkg/sql/distsqlrun/column_exec_setup.go b/pkg/sql/distsqlrun/column_exec_setup.go index c6affc348edd..20709ee5e7ed 100644 --- a/pkg/sql/distsqlrun/column_exec_setup.go +++ b/pkg/sql/distsqlrun/column_exec_setup.go @@ -673,6 +673,12 @@ func (r *newColOperatorResult) planFilterExpr( if err != nil { return columnTypes, err } + if helper.expr == tree.DNull { + // The filter expression is tree.DNull meaning that it is always false, so + // we put a zero operator. + r.op = exec.NewZeroOp(r.op) + return columnTypes, nil + } var filterColumnTypes []types.T r.op, _, filterColumnTypes, selectionMem, err = planSelectionOperators(flowCtx.NewEvalCtx(), helper.expr, columnTypes, r.op) if err != nil { diff --git a/pkg/sql/distsqlrun/columnar_operators_test.go b/pkg/sql/distsqlrun/columnar_operators_test.go index d2614dd9bfb9..9675cd759e24 100644 --- a/pkg/sql/distsqlrun/columnar_operators_test.go +++ b/pkg/sql/distsqlrun/columnar_operators_test.go @@ -15,10 +15,13 @@ import ( "fmt" "math/rand" "sort" + "strings" "testing" + "github.com/cockroachdb/cockroach/pkg/col/coltypes" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/distsqlpb" + "github.com/cockroachdb/cockroach/pkg/sql/exec/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -26,38 +29,55 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/randutil" ) +const nullProbability = 0.2 +const randTypesProbability = 0.5 + func TestSorterAgainstProcessor(t *testing.T) { defer leaktest.AfterTest(t)() st := cluster.MakeTestingClusterSettings() evalCtx := tree.MakeTestingEvalContext(st) defer evalCtx.Stop(context.Background()) - rng, _ := randutil.NewPseudoRand() + seed := rand.Int() + rng := rand.New(rand.NewSource(int64(seed))) + nRuns := 10 nRows := 100 maxCols := 5 maxNum := 10 - nullProbability := 0.2 - typs := make([]types.T, maxCols) - for i := range typs { - typs[i] = *types.Int + intTyps := make([]types.T, maxCols) + for i := range intTyps { + intTyps[i] = *types.Int } - for nCols := 1; nCols <= maxCols; nCols++ { - inputTypes := typs[:nCols] - - rows := sqlbase.MakeRandIntRowsInRange(rng, nRows, nCols, maxNum, nullProbability) - // Note: we're only generating column orderings on all nCols columns since - // if there are columns not in the ordering, the results are not fully - // deterministic. - orderingCols := generateColumnOrdering(rng, nCols, nCols) - sorterSpec := &distsqlpb.SorterSpec{ - OutputOrdering: distsqlpb.Ordering{Columns: orderingCols}, - } - pspec := &distsqlpb.ProcessorSpec{ - Input: []distsqlpb.InputSyncSpec{{ColumnTypes: inputTypes}}, - Core: distsqlpb.ProcessorCoreUnion{Sorter: sorterSpec}, - } - if err := verifyColOperator(false /* anyOrder */, [][]types.T{inputTypes}, []sqlbase.EncDatumRows{rows}, inputTypes, pspec); err != nil { - t.Fatal(err) + + for run := 0; run < nRuns; run++ { + for nCols := 1; nCols <= maxCols; nCols++ { + var ( + rows sqlbase.EncDatumRows + inputTypes []types.T + ) + if rng.Float64() < randTypesProbability { + inputTypes = generateRandomSupportedTypes(rng, nCols) + rows = sqlbase.RandEncDatumRowsOfTypes(rng, nRows, inputTypes) + } else { + inputTypes = intTyps[:nCols] + rows = sqlbase.MakeRandIntRowsInRange(rng, nRows, nCols, maxNum, nullProbability) + } + + // Note: we're only generating column orderings on all nCols columns since + // if there are columns not in the ordering, the results are not fully + // deterministic. + orderingCols := generateColumnOrdering(rng, nCols, nCols) + sorterSpec := &distsqlpb.SorterSpec{ + OutputOrdering: distsqlpb.Ordering{Columns: orderingCols}, + } + pspec := &distsqlpb.ProcessorSpec{ + Input: []distsqlpb.InputSyncSpec{{ColumnTypes: inputTypes}}, + Core: distsqlpb.ProcessorCoreUnion{Sorter: sorterSpec}, + } + if err := verifyColOperator(false /* anyOrder */, [][]types.T{inputTypes}, []sqlbase.EncDatumRows{rows}, inputTypes, pspec); err != nil { + fmt.Printf("--- seed = %d nCols = %d types = %v ---\n", seed, nCols, inputTypes) + t.Fatal(err) + } } } } @@ -68,44 +88,59 @@ func TestSortChunksAgainstProcessor(t *testing.T) { st := cluster.MakeTestingClusterSettings() evalCtx := tree.MakeTestingEvalContext(st) defer evalCtx.Stop(context.Background()) - rng, _ := randutil.NewPseudoRand() + seed := rand.Int() + rng := rand.New(rand.NewSource(int64(seed))) + nRuns := 5 nRows := 100 maxCols := 5 maxNum := 10 - nullProbability := 0.2 - typs := make([]types.T, maxCols) - for i := range typs { - typs[i] = *types.Int + intTyps := make([]types.T, maxCols) + for i := range intTyps { + intTyps[i] = *types.Int } - for nCols := 1; nCols <= maxCols; nCols++ { - inputTypes := typs[:nCols] - // Note: we're only generating column orderings on all nCols columns since - // if there are columns not in the ordering, the results are not fully - // deterministic. - orderingCols := generateColumnOrdering(rng, nCols, nCols) - for matchLen := 1; matchLen <= nCols; matchLen++ { - rows := sqlbase.MakeRandIntRowsInRange(rng, nRows, nCols, maxNum, nullProbability) - matchedCols := distsqlpb.ConvertToColumnOrdering(distsqlpb.Ordering{Columns: orderingCols[:matchLen]}) - // Presort the input on first matchLen columns. - sort.Slice(rows, func(i, j int) bool { - cmp, err := rows[i].Compare(inputTypes, &da, matchedCols, &evalCtx, rows[j]) - if err != nil { - t.Fatal(err) + + for run := 0; run < nRuns; run++ { + for nCols := 1; nCols <= maxCols; nCols++ { + for matchLen := 1; matchLen <= nCols; matchLen++ { + var ( + rows sqlbase.EncDatumRows + inputTypes []types.T + ) + if rng.Float64() < randTypesProbability { + inputTypes = generateRandomSupportedTypes(rng, nCols) + rows = sqlbase.RandEncDatumRowsOfTypes(rng, nRows, inputTypes) + } else { + inputTypes = intTyps[:nCols] + rows = sqlbase.MakeRandIntRowsInRange(rng, nRows, nCols, maxNum, nullProbability) } - return cmp < 0 - }) - sorterSpec := &distsqlpb.SorterSpec{ - OutputOrdering: distsqlpb.Ordering{Columns: orderingCols}, - OrderingMatchLen: uint32(matchLen), - } - pspec := &distsqlpb.ProcessorSpec{ - Input: []distsqlpb.InputSyncSpec{{ColumnTypes: inputTypes}}, - Core: distsqlpb.ProcessorCoreUnion{Sorter: sorterSpec}, - } - if err := verifyColOperator(false /* anyOrder */, [][]types.T{inputTypes}, []sqlbase.EncDatumRows{rows}, inputTypes, pspec); err != nil { - t.Fatal(err) + // Note: we're only generating column orderings on all nCols columns since + // if there are columns not in the ordering, the results are not fully + // deterministic. + orderingCols := generateColumnOrdering(rng, nCols, nCols) + matchedCols := distsqlpb.ConvertToColumnOrdering(distsqlpb.Ordering{Columns: orderingCols[:matchLen]}) + // Presort the input on first matchLen columns. + sort.Slice(rows, func(i, j int) bool { + cmp, err := rows[i].Compare(inputTypes, &da, matchedCols, &evalCtx, rows[j]) + if err != nil { + t.Fatal(err) + } + return cmp < 0 + }) + + sorterSpec := &distsqlpb.SorterSpec{ + OutputOrdering: distsqlpb.Ordering{Columns: orderingCols}, + OrderingMatchLen: uint32(matchLen), + } + pspec := &distsqlpb.ProcessorSpec{ + Input: []distsqlpb.InputSyncSpec{{ColumnTypes: inputTypes}}, + Core: distsqlpb.ProcessorCoreUnion{Sorter: sorterSpec}, + } + if err := verifyColOperator(false /* anyOrder */, [][]types.T{inputTypes}, []sqlbase.EncDatumRows{rows}, inputTypes, pspec); err != nil { + fmt.Printf("--- seed = %d nCols = %d types = %v ---\n", seed, nCols, inputTypes) + t.Fatal(err) + } } } } @@ -115,8 +150,6 @@ func TestHashJoinerAgainstProcessor(t *testing.T) { defer leaktest.AfterTest(t)() evalCtx := tree.MakeTestingEvalContext(cluster.MakeTestingClusterSettings()) defer evalCtx.Stop(context.Background()) - seed := rand.Int() - rng := rand.New(rand.NewSource(int64(seed))) type hjTestSpec struct { joinType sqlbase.JoinType @@ -141,16 +174,17 @@ func TestHashJoinerAgainstProcessor(t *testing.T) { }, } + seed := rand.Int() + rng := rand.New(rand.NewSource(int64(seed))) nRuns := 3 nRows := 10 maxCols := 3 maxNum := 5 - nullProbability := 0.1 - typs := make([]types.T, maxCols) - for i := range typs { - // TODO(yuzefovich): randomize the types of the columns. - typs[i] = *types.Int + intTyps := make([]types.T, maxCols) + for i := range intTyps { + intTyps[i] = *types.Int } + for run := 1; run < nRuns; run++ { for _, testSpec := range testSpecs { for nCols := 1; nCols <= maxCols; nCols++ { @@ -160,10 +194,29 @@ func TestHashJoinerAgainstProcessor(t *testing.T) { triedWithOnExpr = true } for !triedWithoutOnExpr || !triedWithOnExpr { - inputTypes := typs[:nCols] + var ( + lRows, rRows sqlbase.EncDatumRows + lEqCols, rEqCols []uint32 + inputTypes []types.T + usingRandomTypes bool + ) + if rng.Float64() < randTypesProbability { + inputTypes = generateRandomSupportedTypes(rng, nCols) + lRows = sqlbase.RandEncDatumRowsOfTypes(rng, nRows, inputTypes) + rRows = sqlbase.RandEncDatumRowsOfTypes(rng, nRows, inputTypes) + lEqCols = generateEqualityColumns(rng, nCols, nEqCols) + // Since random types might not be comparable, we use the same + // equality columns for both inputs. + rEqCols = lEqCols + usingRandomTypes = true + } else { + inputTypes = intTyps[:nCols] + lRows = sqlbase.MakeRandIntRowsInRange(rng, nRows, nCols, maxNum, nullProbability) + rRows = sqlbase.MakeRandIntRowsInRange(rng, nRows, nCols, maxNum, nullProbability) + lEqCols = generateEqualityColumns(rng, nCols, nEqCols) + rEqCols = generateEqualityColumns(rng, nCols, nEqCols) + } - lRows := sqlbase.MakeRandIntRowsInRange(rng, nRows, nCols, maxNum, nullProbability) - rRows := sqlbase.MakeRandIntRowsInRange(rng, nRows, nCols, maxNum, nullProbability) outputTypes := append(inputTypes, inputTypes...) if testSpec.joinType == sqlbase.JoinType_LEFT_SEMI { outputTypes = inputTypes @@ -175,11 +228,12 @@ func TestHashJoinerAgainstProcessor(t *testing.T) { var onExpr distsqlpb.Expression if triedWithoutOnExpr { - onExpr = generateOnExpr(rng, nCols, nEqCols, maxNum) + colTypes := append(inputTypes, inputTypes...) + onExpr = generateOnExpr(rng, nCols, nEqCols, colTypes, usingRandomTypes) } hjSpec := &distsqlpb.HashJoinerSpec{ - LeftEqColumns: generateEqualityColumns(rng, nCols, nEqCols), - RightEqColumns: generateEqualityColumns(rng, nCols, nEqCols), + LeftEqColumns: lEqCols, + RightEqColumns: rEqCols, OnExpr: onExpr, Type: testSpec.joinType, } @@ -197,6 +251,8 @@ func TestHashJoinerAgainstProcessor(t *testing.T) { ); err != nil { fmt.Printf("--- join type = %s onExpr = %q seed = %d run = %d ---\n", testSpec.joinType.String(), onExpr.Expr, seed, run) + fmt.Printf("--- lEqCols = %v rEqCols = %v ---\n", lEqCols, rEqCols) + fmt.Printf("--- inputTypes = %v ---\n", inputTypes) t.Fatal(err) } if onExpr.Expr == "" { @@ -230,8 +286,6 @@ func TestMergeJoinerAgainstProcessor(t *testing.T) { var da sqlbase.DatumAlloc evalCtx := tree.MakeTestingEvalContext(cluster.MakeTestingClusterSettings()) defer evalCtx.Stop(context.Background()) - seed := rand.Int() - rng := rand.New(rand.NewSource(int64(seed))) type mjTestSpec struct { joinType sqlbase.JoinType @@ -263,16 +317,17 @@ func TestMergeJoinerAgainstProcessor(t *testing.T) { }, } + seed := rand.Int() + rng := rand.New(rand.NewSource(int64(seed))) nRuns := 3 nRows := 10 maxCols := 3 maxNum := 5 - nullProbability := 0.1 - typs := make([]types.T, maxCols) - for i := range typs { - // TODO (georgeutsin): Randomize the types of the columns. - typs[i] = *types.Int + intTyps := make([]types.T, maxCols) + for i := range intTyps { + intTyps[i] = *types.Int } + for run := 1; run < nRuns; run++ { for _, testSpec := range testSpecs { for nCols := 1; nCols <= maxCols; nCols++ { @@ -282,16 +337,33 @@ func TestMergeJoinerAgainstProcessor(t *testing.T) { triedWithOnExpr = true } for !triedWithoutOnExpr || !triedWithOnExpr { - inputTypes := typs[:nCols] - lOrderingCols := generateColumnOrdering(rng, nCols, nOrderingCols) - rOrderingCols := generateColumnOrdering(rng, nCols, nOrderingCols) + var ( + lRows, rRows sqlbase.EncDatumRows + inputTypes []types.T + lOrderingCols, rOrderingCols []distsqlpb.Ordering_Column + usingRandomTypes bool + ) + if rng.Float64() < randTypesProbability { + inputTypes = generateRandomSupportedTypes(rng, nCols) + lRows = sqlbase.RandEncDatumRowsOfTypes(rng, nRows, inputTypes) + rRows = sqlbase.RandEncDatumRowsOfTypes(rng, nRows, inputTypes) + lOrderingCols = generateColumnOrdering(rng, nCols, nOrderingCols) + // We use the same ordering columns in the same order because the + // columns can be not comparable in different order. + rOrderingCols = lOrderingCols + usingRandomTypes = true + } else { + inputTypes = intTyps[:nCols] + lRows = sqlbase.MakeRandIntRowsInRange(rng, nRows, nCols, maxNum, nullProbability) + rRows = sqlbase.MakeRandIntRowsInRange(rng, nRows, nCols, maxNum, nullProbability) + lOrderingCols = generateColumnOrdering(rng, nCols, nOrderingCols) + rOrderingCols = generateColumnOrdering(rng, nCols, nOrderingCols) + } // Set the directions of both columns to be the same. for i, lCol := range lOrderingCols { rOrderingCols[i].Direction = lCol.Direction } - lRows := sqlbase.MakeRandIntRowsInRange(rng, nRows, nCols, maxNum, nullProbability) - rRows := sqlbase.MakeRandIntRowsInRange(rng, nRows, nCols, maxNum, nullProbability) lMatchedCols := distsqlpb.ConvertToColumnOrdering(distsqlpb.Ordering{Columns: lOrderingCols}) rMatchedCols := distsqlpb.ConvertToColumnOrdering(distsqlpb.Ordering{Columns: rOrderingCols}) sort.Slice(lRows, func(i, j int) bool { @@ -320,7 +392,8 @@ func TestMergeJoinerAgainstProcessor(t *testing.T) { var onExpr distsqlpb.Expression if triedWithoutOnExpr { - onExpr = generateOnExpr(rng, nCols, nOrderingCols, maxNum) + colTypes := append(inputTypes, inputTypes...) + onExpr = generateOnExpr(rng, nCols, nOrderingCols, colTypes, usingRandomTypes) } mjSpec := &distsqlpb.MergeJoinerSpec{ OnExpr: onExpr, @@ -380,9 +453,11 @@ func generateColumnOrdering( // comparison which can be either comparing a column from the left against a // column from the right or comparing a column from either side against a // constant. -// TODO(yuzefovich): update this once LEFT SEMI or LEFT ANTI is supported with -// ON expression. -func generateOnExpr(rng *rand.Rand, nCols int, nEqCols int, maxNum int) distsqlpb.Expression { +// If forceConstComparison is true, then the comparison against the constant +// will be used. +func generateOnExpr( + rng *rand.Rand, nCols int, nEqCols int, colTypes []types.T, forceConstComparison bool, +) distsqlpb.Expression { var comparison string r := rng.Float64() if r < 0.25 { @@ -398,14 +473,19 @@ func generateOnExpr(rng *rand.Rand, nCols int, nEqCols int, maxNum int) distsqlp // only one interesting case when a column from either side is compared // against a constant. The second conditional is us choosing to compare // against a constant. - if nCols == nEqCols || rng.Float64() < 0.33 { - colIdx := rng.Intn(nCols) + 1 - constVal := rng.Intn(maxNum) + if nCols == nEqCols || rng.Float64() < 0.33 || forceConstComparison { + colIdx := rng.Intn(nCols) if rng.Float64() >= 0.5 { // Use right side. colIdx += nCols } - return distsqlpb.Expression{Expr: fmt.Sprintf("@%d %s %d", colIdx, comparison, constVal)} + constDatum := sqlbase.RandDatum(rng, &colTypes[colIdx], true /* nullOk */) + constDatumString := constDatum.String() + if strings.Contains(constDatumString, "NaN") || strings.Contains(constDatumString, "Inf") { + // We need to surround special values with quotes. + constDatumString = fmt.Sprintf("'%s'", constDatumString) + } + return distsqlpb.Expression{Expr: fmt.Sprintf("@%d %s %s", colIdx+1, comparison, constDatumString)} } // We will compare a column from the left against a column from the right. leftColIdx := rng.Intn(nCols) + 1 @@ -420,8 +500,6 @@ func TestWindowFunctionsAgainstProcessor(t *testing.T) { nRows := 10 maxCols := 4 maxNum := 5 - // TODO(yuzefovich): use non-zero null probability once sorter handles nulls. - nullProbability := 0.0 typs := make([]types.T, maxCols) for i := range typs { // TODO(yuzefovich): randomize the types of the columns once we support @@ -481,6 +559,20 @@ func TestWindowFunctionsAgainstProcessor(t *testing.T) { } } +// generateRandomSupportedTypes generates nCols random types that are supported +// by the vectorized engine. +func generateRandomSupportedTypes(rng *rand.Rand, nCols int) []types.T { + typs := make([]types.T, 0, nCols) + for len(typs) < nCols { + typ := sqlbase.RandType(rng) + converted := typeconv.FromColumnType(typ) + if converted != coltypes.Unhandled { + typs = append(typs, *typ) + } + } + return typs +} + // generateOrderingGivenPartitionBy produces a random ordering of up to // nOrderingCols columns on a table with nCols columns such that only columns // not present in partitionBy are used. This is useful to simulate how diff --git a/pkg/sql/exec/execgen/cmd/execgen/overloads.go b/pkg/sql/exec/execgen/cmd/execgen/overloads.go index dcc583f9b353..484176b26026 100644 --- a/pkg/sql/exec/execgen/cmd/execgen/overloads.go +++ b/pkg/sql/exec/execgen/cmd/execgen/overloads.go @@ -372,13 +372,18 @@ func (bytesCustomizer) getCmpOpCompareFunc() compareFunc { } } +// hashByteSliceString is a templated code for hashing a byte slice. It is +// meant to be used as a format string for fmt.Sprintf with the first argument +// being the "target" (i.e. what variable to assign the hash to) and the second +// argument being the "value" (i.e. what is the name of a byte slice variable). +const hashByteSliceString = ` + sh := (*reflect.SliceHeader)(unsafe.Pointer(&%[2]s)) + %[1]s = memhash(unsafe.Pointer(sh.Data), %[1]s, uintptr(len(%[2]s))) +` + func (bytesCustomizer) getHashAssignFunc() assignFunc { return func(op overload, target, v, _ string) string { - return fmt.Sprintf(` - sh := (*reflect.SliceHeader)(unsafe.Pointer(&%[1]s)) - %[2]s = memhash(unsafe.Pointer(sh.Data), %[2]s, uintptr(len(%[1]s))) - - `, v, target) + return fmt.Sprintf(hashByteSliceString, target, v) } } @@ -397,20 +402,23 @@ func (decimalCustomizer) getBinOpAssignFunc() assignFunc { func (decimalCustomizer) getHashAssignFunc() assignFunc { return func(op overload, target, v, _ string) string { - return fmt.Sprintf(` - d, err := %[2]s.Float64() - if err != nil { - execerror.NonVectorizedPanic(err) - } - - %[1]s = f64hash(noescape(unsafe.Pointer(&d)), %[1]s) - `, target, v) + return fmt.Sprintf(`b := []byte(%[1]s.String())`, v) + + fmt.Sprintf(hashByteSliceString, target, "b") } } func (c floatCustomizer) getHashAssignFunc() assignFunc { return func(op overload, target, v, _ string) string { - return fmt.Sprintf("%[1]s = f%[3]dhash(noescape(unsafe.Pointer(&%[2]s)), %[1]s)", target, v, c.width) + // TODO(yuzefovich): think through whether this is appropriate way to hash + // NaNs. + return fmt.Sprintf( + ` + f := %[2]s + if math.IsNaN(float64(f)) { + f = 0 + } + %[1]s = f%[3]dhash(noescape(unsafe.Pointer(&f)), %[1]s) +`, target, v, c.width) } } diff --git a/pkg/sql/exec/hashjoiner.go b/pkg/sql/exec/hashjoiner.go index f82912bdc98f..8e032e981f80 100644 --- a/pkg/sql/exec/hashjoiner.go +++ b/pkg/sql/exec/hashjoiner.go @@ -366,7 +366,7 @@ type hashTable struct { // head and thereby should not be traversed. head []bool - // vals stores the union of the equality and output columns of the left + // vals stores the union of the equality and output columns of the build // table. A key tuple is defined as the elements in each row of vals that // makes up the equality columns. The ID of a key at any index of vals is // index + 1. @@ -383,13 +383,13 @@ type hashTable struct { // outTypes stores the types of the output columns. outTypes []coltypes.T - // size returns the total number of keyCols the hashTable currently stores. + // size returns the total number of tuples the hashTable currently stores. size uint64 // bucketSize returns the number of buckets the hashTable employs. This is // equivalent to the size of first. bucketSize uint64 - // keyCols stores the equality columns on the probe table for a single batch. + // keys stores the equality columns on the probe table for a single batch. keys []coldata.Vec // buckets is used to store the computed hash value of each key in a single // batch. diff --git a/pkg/sql/sqlbase/testutils.go b/pkg/sql/sqlbase/testutils.go index 9baab49a3ecd..7af7e2f4db94 100644 --- a/pkg/sql/sqlbase/testutils.go +++ b/pkg/sql/sqlbase/testutils.go @@ -112,19 +112,39 @@ func RandDatumWithNullChance(rng *rand.Rand, typ *types.T, nullChance int) tree. } // Sometimes pick from a predetermined list of known interesting datums. if rng.Intn(10) == 0 { - specials := randInterestingDatums[typ.Family()] - if len(specials) > 0 { - return specials[rng.Intn(len(specials))] + if special := randInterestingDatum(rng, typ); special != nil { + return special } } switch typ.Family() { case types.BoolFamily: return tree.MakeDBool(rng.Intn(2) == 1) case types.IntFamily: - // int64(rng.Uint64()) to get negative numbers, too - return tree.NewDInt(tree.DInt(int64(rng.Uint64()))) + switch typ.Width() { + case 64: + // int64(rng.Uint64()) to get negative numbers, too + return tree.NewDInt(tree.DInt(int64(rng.Uint64()))) + case 32: + // int32(rng.Uint64()) to get negative numbers, too + return tree.NewDInt(tree.DInt(int32(rng.Uint64()))) + case 16: + // int16(rng.Uint64()) to get negative numbers, too + return tree.NewDInt(tree.DInt(int16(rng.Uint64()))) + case 8: + // int8(rng.Uint64()) to get negative numbers, too + return tree.NewDInt(tree.DInt(int8(rng.Uint64()))) + default: + panic(fmt.Sprintf("int with an unexpected width %d", typ.Width())) + } case types.FloatFamily: - return tree.NewDFloat(tree.DFloat(rng.NormFloat64())) + switch typ.Width() { + case 64: + return tree.NewDFloat(tree.DFloat(rng.NormFloat64())) + case 32: + return tree.NewDFloat(tree.DFloat(float32(rng.NormFloat64()))) + default: + panic(fmt.Sprintf("float with an unexpected width %d", typ.Width())) + } case types.DecimalFamily: d := &tree.DDecimal{} // int64(rng.Uint64()) to get negative numbers, too @@ -234,6 +254,10 @@ var ( tree.NewDInt(tree.DInt(0)), tree.NewDInt(tree.DInt(-1)), tree.NewDInt(tree.DInt(1)), + tree.NewDInt(tree.DInt(math.MaxInt8)), + tree.NewDInt(tree.DInt(math.MinInt8)), + tree.NewDInt(tree.DInt(math.MaxInt16)), + tree.NewDInt(tree.DInt(math.MinInt16)), tree.NewDInt(tree.DInt(math.MaxInt32)), tree.NewDInt(tree.DInt(math.MinInt32)), tree.NewDInt(tree.DInt(math.MaxInt64)), @@ -377,6 +401,43 @@ func init() { } } +// randInterestingDatum returns an interesting Datum of type typ. If there are +// no such Datums, it returns nil. Note that it pays attention to the width of +// the requested type for Int and Float type families. +func randInterestingDatum(rng *rand.Rand, typ *types.T) tree.Datum { + specials := randInterestingDatums[typ.Family()] + if len(specials) == 0 { + return nil + } + special := specials[rng.Intn(len(specials))] + switch typ.Family() { + case types.IntFamily: + switch typ.Width() { + case 64: + return special + case 32: + return tree.NewDInt(tree.DInt(int32(tree.MustBeDInt(special)))) + case 16: + return tree.NewDInt(tree.DInt(int16(tree.MustBeDInt(special)))) + case 8: + return tree.NewDInt(tree.DInt(int8(tree.MustBeDInt(special)))) + default: + panic(fmt.Sprintf("int with an unexpected width %d", typ.Width())) + } + case types.FloatFamily: + switch typ.Width() { + case 64: + return special + case 32: + return tree.NewDFloat(tree.DFloat(float32(*special.(*tree.DFloat)))) + default: + panic(fmt.Sprintf("float with an unexpected width %d", typ.Width())) + } + default: + return special + } +} + // RandCollationLocale returns a random element of collationLocales. func RandCollationLocale(rng *rand.Rand) *string { return &collationLocales[rng.Intn(len(collationLocales))]