Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
38767: exec: fix planning of count operator r=yuzefovich a=yuzefovich

Previously, when planning a count operator, we would add it to the
flow and would ignore any post-operator planning (like projections).
Now, this is fixed.

Additionally, this commit fixes slicing within projections operators -
previously, we would always slice up to BatchSize, but the underlying
memory not always has sufficient capacity (for example, count operator
uses a batch with a capacity of 1) which would cause an index out of
bounds.

Fixes: #38752.

Release note: None

38881: exec: fix NaN comparison logic r=solongordon a=solongordon

I added special NaN handling for float comparisons. In SQL, NaNs are
treated as less than any other float value.

Thankfully I'm not seeing a performance hit when I run our sort
benchmarks with float64 values.

Fixes #38751

Release note: None

38891: c-deps: bump rocksdb for macOS build fix r=ajkr a=ajkr

Pick up cockroachdb/rocksdb#39

Release note: None

Co-authored-by: Yahor Yuzefovich <yahor@cockroachlabs.com>
Co-authored-by: Solon Gordon <solon@cockroachlabs.com>
Co-authored-by: Andrew Kryczka <andrew.kryczka2@gmail.com>
  • Loading branch information
4 people committed Jul 16, 2019
4 parents 07732c1 + 7214042 + 9bec27f + 8669dc7 commit 8809566
Show file tree
Hide file tree
Showing 10 changed files with 81 additions and 17 deletions.
2 changes: 1 addition & 1 deletion c-deps/rocksdb
Submodule rocksdb updated 1 files
+2 −1 env/env_posix.cc
4 changes: 3 additions & 1 deletion pkg/sql/distsqlrun/column_exec_setup.go
Original file line number Diff line number Diff line change
Expand Up @@ -142,7 +142,9 @@ func newColOperator(
aggSpec.Aggregations[0].FilterColIdx == nil &&
aggSpec.Aggregations[0].Func == distsqlpb.AggregatorSpec_COUNT_ROWS &&
!aggSpec.Aggregations[0].Distinct {
return exec.NewCountOp(inputs[0]), []types.T{types.Int64}, nil
op, err = exec.NewCountOp(inputs[0]), nil
columnTypes = []semtypes.T{*semtypes.Int}
break
}

var groupCols, orderedCols util.FastIntSet
Expand Down
7 changes: 7 additions & 0 deletions pkg/sql/exec/execgen/cmd/execgen/overloads.go
Original file line number Diff line number Diff line change
Expand Up @@ -336,6 +336,13 @@ func (c floatCustomizer) getHashAssignFunc() assignFunc {
}
}

func (c floatCustomizer) getCmpOpCompareFunc() compareFunc {
// Float comparisons need special handling for NaN.
return func(l, r string) string {
return fmt.Sprintf("compareFloats(float64(%s), float64(%s))", l, r)
}
}

func (c intCustomizer) getHashAssignFunc() assignFunc {
return func(op overload, target, v, _ string) string {
return fmt.Sprintf("%[1]s = memhash%[3]d(noescape(unsafe.Pointer(&%[2]s)), %[1]s)", target, v, c.width)
Expand Down
14 changes: 7 additions & 7 deletions pkg/sql/exec/execgen/cmd/execgen/projection_ops_gen.go
Original file line number Diff line number Diff line change
Expand Up @@ -61,9 +61,9 @@ func (p {{template "opRConstName" .}}) Next(ctx context.Context) coldata.Batch {
batch.AppendCol(types.{{.RetTyp}})
}
vec := batch.ColVec(p.colIdx)
col := vec.{{.LTyp}}()[:coldata.BatchSize]
col := vec.{{.LTyp}}()
projVec := batch.ColVec(p.outputIdx)
projCol := projVec.{{.RetTyp}}()[:coldata.BatchSize]
projCol := projVec.{{.RetTyp}}()
if sel := batch.Selection(); sel != nil {
for _, i := range sel {
{{(.Assign "projCol[i]" "col[i]" "p.constArg")}}
Expand Down Expand Up @@ -105,9 +105,9 @@ func (p {{template "opLConstName" .}}) Next(ctx context.Context) coldata.Batch {
batch.AppendCol(types.{{.RetTyp}})
}
vec := batch.ColVec(p.colIdx)
col := vec.{{.RTyp}}()[:coldata.BatchSize]
col := vec.{{.RTyp}}()
projVec := batch.ColVec(p.outputIdx)
projCol := projVec.{{.RetTyp}}()[:coldata.BatchSize]
projCol := projVec.{{.RetTyp}}()
if sel := batch.Selection(); sel != nil {
for _, i := range sel {
{{(.Assign "projCol[i]" "p.constArg" "col[i]")}}
Expand Down Expand Up @@ -149,11 +149,11 @@ func (p {{template "opName" .}}) Next(ctx context.Context) coldata.Batch {
batch.AppendCol(types.{{.RetTyp}})
}
projVec := batch.ColVec(p.outputIdx)
projCol := projVec.{{.RetTyp}}()[:coldata.BatchSize]
projCol := projVec.{{.RetTyp}}()
vec1 := batch.ColVec(p.col1Idx)
vec2 := batch.ColVec(p.col2Idx)
col1 := vec1.{{.LTyp}}()[:coldata.BatchSize]
col2 := vec2.{{.RTyp}}()[:coldata.BatchSize]
col1 := vec1.{{.LTyp}}()
col2 := vec2.{{.RTyp}}()
if sel := batch.Selection(); sel != nil {
for _, i := range sel {
{{(.Assign "projCol[i]" "col1[i]" "col2[i]")}}
Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/exec/execgen/cmd/execgen/selection_ops_gen.go
Original file line number Diff line number Diff line change
Expand Up @@ -105,7 +105,7 @@ func (p *{{template "opConstName" .}}) Next(ctx context.Context) coldata.Batch {
}
vec := batch.ColVec(p.colIdx)
col := vec.{{.LTyp}}()[:coldata.BatchSize]
col := vec.{{.LTyp}}()
var idx uint16
n := batch.Length()
if vec.MaybeHasNulls() {
Expand Down Expand Up @@ -143,8 +143,8 @@ func (p *{{template "opName" .}}) Next(ctx context.Context) coldata.Batch {
vec1 := batch.ColVec(p.col1Idx)
vec2 := batch.ColVec(p.col2Idx)
col1 := vec1.{{.LTyp}}()[:coldata.BatchSize]
col2 := vec2.{{.RTyp}}()[:coldata.BatchSize]
col1 := vec1.{{.LTyp}}()
col2 := vec2.{{.RTyp}}()
n := batch.Length()
var idx uint16
Expand Down
34 changes: 34 additions & 0 deletions pkg/sql/exec/float.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,34 @@
// 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 "math"

// compareFloats compares two float values. This function is necessary for NaN
// handling. In SQL, NaN is treated as less than all other float values. In Go,
// any comparison with NaN returns false.
func compareFloats(a, b float64) int {
if a < b {
return -1
}
if a > b {
return 1
}
// Compare bits so that NaN == NaN.
if math.Float64bits(a) == math.Float64bits(b) {
return 0
}
// Either a or b is NaN.
if math.IsNaN(a) {
return -1
}
return 1
}
6 changes: 3 additions & 3 deletions pkg/sql/exec/select_in_tmpl.go
Original file line number Diff line number Diff line change
Expand Up @@ -177,7 +177,7 @@ func (si *selectInOp_TYPE) Next(ctx context.Context) coldata.Batch {
}

vec := batch.ColVec(si.colIdx)
col := vec._TemplateType()[:coldata.BatchSize]
col := vec._TemplateType()
var idx uint16
n := batch.Length()

Expand Down Expand Up @@ -247,10 +247,10 @@ func (pi *projectInOp_TYPE) Next(ctx context.Context) coldata.Batch {
}

vec := batch.ColVec(pi.colIdx)
col := vec._TemplateType()[:coldata.BatchSize]
col := vec._TemplateType()

projVec := batch.ColVec(pi.outputIdx)
projCol := projVec.Bool()[:coldata.BatchSize]
projCol := projVec.Bool()
projNulls := projVec.Nulls()

n := batch.Length()
Expand Down
5 changes: 3 additions & 2 deletions pkg/sql/exec/sort_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ package exec
import (
"context"
"fmt"
"math"
"math/rand"
"sort"
"testing"
Expand Down Expand Up @@ -76,8 +77,8 @@ func TestSort(t *testing.T) {
ordCols: []distsqlpb.Ordering_Column{{ColIdx: 0}},
},
{
tuples: tuples{{3.2}, {2.0}, {2.4}},
expected: tuples{{2.0}, {2.4}, {3.2}},
tuples: tuples{{3.2}, {2.0}, {2.4}, {math.NaN()}, {math.Inf(-1)}, {math.Inf(1)}},
expected: tuples{{math.NaN()}, {math.Inf(-1)}, {2.0}, {2.4}, {3.2}, {math.Inf(1)}},
typ: []types.T{types.Float64},
ordCols: []distsqlpb.Ordering_Column{{ColIdx: 0}},
},
Expand Down
9 changes: 9 additions & 0 deletions pkg/sql/exec/utils_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ package exec
import (
"context"
"fmt"
"math"
"math/rand"
"reflect"
"sort"
Expand Down Expand Up @@ -516,6 +517,14 @@ func tupleEquals(expected tuple, actual tuple) bool {
return false
}
} else {
// Special case for NaN, since it does not equal itself.
if f1, ok := expected[i].(float64); ok {
if f2, ok := actual[i].(float64); ok {
if math.IsNaN(f1) && math.IsNaN(f2) {
continue
}
}
}
if !reflect.DeepEqual(reflect.ValueOf(actual[i]).Convert(reflect.TypeOf(expected[i])).Interface(), expected[i]) {
return false
}
Expand Down
11 changes: 11 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/vectorize
Original file line number Diff line number Diff line change
Expand Up @@ -444,3 +444,14 @@ query II
SELECT * FROM t38753 ORDER BY y;
----
0 NULL

# Regression test for #38752.
query IIBB
SELECT count(*), count(*) + 1, count(*) > 4, count(*) + 1 > 4 FROM b
----
4 5 false true

query I
SELECT * FROM (SELECT count(*) AS x FROM b) WHERE x > 0;
----
4

0 comments on commit 8809566

Please sign in to comment.