diff --git a/Makefile b/Makefile index bc5b8c373e00..f0fc5b52ee7c 100644 --- a/Makefile +++ b/Makefile @@ -802,7 +802,8 @@ EXECGEN_TARGETS = \ pkg/sql/exec/vec_comparators.eg.go \ pkg/sql/exec/vecbuiltins/rank.eg.go \ pkg/sql/exec/vecbuiltins/row_number.eg.go \ - pkg/sql/exec/zerocolumns.eg.go + pkg/sql/exec/zerocolumns.eg.go \ + pkg/sql/exec/overloads_test_utils.eg.go execgen-exclusions = $(addprefix -not -path ,$(EXECGEN_TARGETS)) diff --git a/pkg/sql/exec/.gitignore b/pkg/sql/exec/.gitignore index 3c288fc44050..dd6aaa2cefc6 100644 --- a/pkg/sql/exec/.gitignore +++ b/pkg/sql/exec/.gitignore @@ -19,3 +19,4 @@ vec_comparators.eg.go vecbuiltins/rank.eg.go vecbuiltins/row_number.eg.go zerocolumns.eg.go +overloads_test_utils.eg.go diff --git a/pkg/sql/exec/execgen/cmd/execgen/overloads.go b/pkg/sql/exec/execgen/cmd/execgen/overloads.go index b801cc251093..4345ebdf09f1 100644 --- a/pkg/sql/exec/execgen/cmd/execgen/overloads.go +++ b/pkg/sql/exec/execgen/cmd/execgen/overloads.go @@ -14,6 +14,8 @@ import ( "errors" "fmt" "regexp" + "strings" + "text/template" "github.com/cockroachdb/cockroach/pkg/sql/exec/types" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -103,7 +105,7 @@ var hashOverloads []*overload // Assign produces a Go source string that assigns the "target" variable to the // result of applying the overload to the two inputs, l and r. // -// For example, an overload that implemented the int64 plus operation, when fed +// For example, an overload that implemented the float64 plus operation, when fed // the inputs "x", "a", "b", would produce the string "x = a + b". func (o overload) Assign(target, l, r string) string { if o.AssignFunc != nil { @@ -349,6 +351,120 @@ func (c intCustomizer) getHashAssignFunc() assignFunc { } } +func (c intCustomizer) getBinOpAssignFunc() assignFunc { + return func(op overload, target, l, r string) string { + args := map[string]string{"Target": target, "Left": l, "Right": r} + buf := strings.Builder{} + var t *template.Template + + switch op.BinOp { + + case tree.Plus: + t = template.Must(template.New("").Parse(` + { + result := {{.Left}} + {{.Right}} + if (result < {{.Left}}) != ({{.Right}} < 0) { + panic(tree.ErrIntOutOfRange) + } + {{.Target}} = result + } + `)) + + case tree.Minus: + t = template.Must(template.New("").Parse(` + { + result := {{.Left}} - {{.Right}} + if (result < {{.Left}}) != ({{.Right}} > 0) { + panic(tree.ErrIntOutOfRange) + } + {{.Target}} = result + } + `)) + + case tree.Mult: + // If the inputs are small enough, then we don't have to do any further + // checks. For the sake of legibility, upperBound and lowerBound are both + // not set to their maximal/minimal values. An even more advanced check + // (for positive values) might involve adding together the highest bit + // positions of the inputs, and checking if the sum is less than the + // integer width. + var upperBound, lowerBound string + switch c.width { + case 8: + upperBound = "10" + lowerBound = "-10" + case 16: + upperBound = "math.MaxInt8" + lowerBound = "math.MinInt8" + case 32: + upperBound = "math.MaxInt16" + lowerBound = "math.MinInt16" + case 64: + upperBound = "math.MaxInt32" + lowerBound = "math.MinInt32" + default: + panic(fmt.Sprintf("unhandled integer width %d", c.width)) + } + + args["UpperBound"] = upperBound + args["LowerBound"] = lowerBound + t = template.Must(template.New("").Parse(` + { + result := {{.Left}} * {{.Right}} + if {{.Left}} > {{.UpperBound}} || {{.Left}} < {{.LowerBound}} || {{.Right}} > {{.UpperBound}} || {{.Right}} < {{.LowerBound}} { + if {{.Left}} != 0 && {{.Right}} != 0 { + sameSign := ({{.Left}} < 0) == ({{.Right}} < 0) + if (result < 0) == sameSign { + panic(tree.ErrIntOutOfRange) + } else if result/{{.Right}} != {{.Left}} { + panic(tree.ErrIntOutOfRange) + } + } + } + {{.Target}} = result + } + `)) + + case tree.Div: + var minInt string + switch c.width { + case 8: + minInt = "math.MinInt8" + case 16: + minInt = "math.MinInt16" + case 32: + minInt = "math.MinInt32" + case 64: + minInt = "math.MinInt64" + default: + panic(fmt.Sprintf("unhandled integer width %d", c.width)) + } + + args["MinInt"] = minInt + t = template.Must(template.New("").Parse(` + { + if {{.Right}} == 0 { + panic(tree.ErrDivByZero) + } + result := {{.Left}} / {{.Right}} + if {{.Left}} == {{.MinInt}} && {{.Right}} == -1 { + panic(tree.ErrIntOutOfRange) + } + {{.Target}} = result + } + `)) + + default: + panic(fmt.Sprintf("unhandled binary operator %s", op.BinOp.String())) + } + + if err := t.Execute(&buf, args); err != nil { + panic(err) + } + return buf.String() + } +} + func registerTypeCustomizers() { typeCustomizers = make(map[types.T]typeCustomizer) registerTypeCustomizer(types.Bool, boolCustomizer{}) diff --git a/pkg/sql/exec/execgen/cmd/execgen/overloads_test_utils_gen.go b/pkg/sql/exec/execgen/cmd/execgen/overloads_test_utils_gen.go new file mode 100644 index 000000000000..0593b1b7a2d5 --- /dev/null +++ b/pkg/sql/exec/execgen/cmd/execgen/overloads_test_utils_gen.go @@ -0,0 +1,65 @@ +// 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 main + +import ( + "io" + "text/template" + + "github.com/cockroachdb/cockroach/pkg/sql/exec/types" +) + +const overloadsTestUtilsTemplate = ` +package exec + +import ( + "math" + + "github.com/cockroachdb/apd" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" +) + +{{define "opName"}}perform{{.Name}}{{.LTyp}}{{end}} + +{{/* The outer range is a types.T, and the inner is the overloads associated + with that type. */}} +{{range .}} +{{range .}} + +func {{template "opName" .}}(a, b {{.LTyp.GoTypeName}}) {{.RetTyp.GoTypeName}} { + {{(.Assign "a" "a" "b")}} + return a +} + +{{end}} +{{end}} +` + +// genOverloadsTestUtils creates a file that has a function for each overload +// defined in overloads.go. This is so that we can more easily test each +// overload. +func genOverloadsTestUtils(wr io.Writer) error { + tmpl, err := template.New("overloads_test_utils").Parse(overloadsTestUtilsTemplate) + if err != nil { + return err + } + + typToOverloads := make(map[types.T][]*overload) + for _, overload := range binaryOpOverloads { + typ := overload.LTyp + typToOverloads[typ] = append(typToOverloads[typ], overload) + } + return tmpl.Execute(wr, typToOverloads) +} + +func init() { + registerGenerator(genOverloadsTestUtils, "overloads_test_utils.eg.go") +} diff --git a/pkg/sql/exec/execgen/cmd/execgen/projection_ops_gen.go b/pkg/sql/exec/execgen/cmd/execgen/projection_ops_gen.go index a2aa2ff9ccf7..b0dcee4a9774 100644 --- a/pkg/sql/exec/execgen/cmd/execgen/projection_ops_gen.go +++ b/pkg/sql/exec/execgen/cmd/execgen/projection_ops_gen.go @@ -23,6 +23,7 @@ package exec import ( "bytes" "context" + "math" "github.com/cockroachdb/apd" "github.com/cockroachdb/cockroach/pkg/sql/exec/coldata" diff --git a/pkg/sql/exec/overloads_test.go b/pkg/sql/exec/overloads_test.go new file mode 100644 index 000000000000..4d712faa99cf --- /dev/null +++ b/pkg/sql/exec/overloads_test.go @@ -0,0 +1,121 @@ +// 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" + "testing" + + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/stretchr/testify/assert" +) + +func TestIntegerAddition(t *testing.T) { + // The addition overload is the same for all integer widths, so we only test + // one of them. + assert.PanicsWithValue(t, tree.ErrIntOutOfRange, func() { performPlusInt16(1, math.MaxInt16) }) + assert.PanicsWithValue(t, tree.ErrIntOutOfRange, func() { performPlusInt16(-1, math.MinInt16) }) + assert.PanicsWithValue(t, tree.ErrIntOutOfRange, func() { performPlusInt16(math.MaxInt16, 1) }) + assert.PanicsWithValue(t, tree.ErrIntOutOfRange, func() { performPlusInt16(math.MinInt16, -1) }) + + assert.Equal(t, int16(math.MaxInt16), performPlusInt16(1, math.MaxInt16-1)) + assert.Equal(t, int16(math.MinInt16), performPlusInt16(-1, math.MinInt16+1)) + assert.Equal(t, int16(math.MaxInt16-1), performPlusInt16(-1, math.MaxInt16)) + assert.Equal(t, int16(math.MinInt16+1), performPlusInt16(1, math.MinInt16)) + + assert.Equal(t, int16(22), performPlusInt16(10, 12)) + assert.Equal(t, int16(-22), performPlusInt16(-10, -12)) + assert.Equal(t, int16(2), performPlusInt16(-10, 12)) + assert.Equal(t, int16(-2), performPlusInt16(10, -12)) +} + +func TestIntegerSubtraction(t *testing.T) { + // The subtraction overload is the same for all integer widths, so we only + // test one of them. + assert.PanicsWithValue(t, tree.ErrIntOutOfRange, func() { performMinusInt16(1, -math.MaxInt16) }) + assert.PanicsWithValue(t, tree.ErrIntOutOfRange, func() { performMinusInt16(-2, math.MaxInt16) }) + assert.PanicsWithValue(t, tree.ErrIntOutOfRange, func() { performMinusInt16(math.MaxInt16, -1) }) + assert.PanicsWithValue(t, tree.ErrIntOutOfRange, func() { performMinusInt16(math.MinInt16, 1) }) + + assert.Equal(t, int16(math.MaxInt16), performMinusInt16(1, -math.MaxInt16+1)) + assert.Equal(t, int16(math.MinInt16), performMinusInt16(-1, math.MaxInt16)) + assert.Equal(t, int16(math.MaxInt16-1), performMinusInt16(-1, -math.MaxInt16)) + assert.Equal(t, int16(math.MinInt16+1), performMinusInt16(0, math.MaxInt16)) + + assert.Equal(t, int16(-2), performMinusInt16(10, 12)) + assert.Equal(t, int16(2), performMinusInt16(-10, -12)) + assert.Equal(t, int16(-22), performMinusInt16(-10, 12)) + assert.Equal(t, int16(22), performMinusInt16(10, -12)) +} + +func TestIntegerDivision(t *testing.T) { + assert.PanicsWithValue(t, tree.ErrIntOutOfRange, func() { performDivInt8(math.MinInt8, -1) }) + assert.PanicsWithValue(t, tree.ErrIntOutOfRange, func() { performDivInt16(math.MinInt16, -1) }) + assert.PanicsWithValue(t, tree.ErrIntOutOfRange, func() { performDivInt32(math.MinInt32, -1) }) + assert.PanicsWithValue(t, tree.ErrIntOutOfRange, func() { performDivInt64(math.MinInt64, -1) }) + + assert.PanicsWithValue(t, tree.ErrDivByZero, func() { performDivInt8(10, 0) }) + assert.PanicsWithValue(t, tree.ErrDivByZero, func() { performDivInt16(10, 0) }) + assert.PanicsWithValue(t, tree.ErrDivByZero, func() { performDivInt32(10, 0) }) + assert.PanicsWithValue(t, tree.ErrDivByZero, func() { performDivInt64(10, 0) }) + + assert.Equal(t, int8(-math.MaxInt8), performDivInt8(math.MaxInt8, -1)) + assert.Equal(t, int16(-math.MaxInt16), performDivInt16(math.MaxInt16, -1)) + assert.Equal(t, int32(-math.MaxInt32), performDivInt32(math.MaxInt32, -1)) + assert.Equal(t, int64(-math.MaxInt64), performDivInt64(math.MaxInt64, -1)) + + assert.Equal(t, int16(0), performDivInt16(10, 12)) + assert.Equal(t, int16(0), performDivInt16(-10, -12)) + assert.Equal(t, int16(-1), performDivInt16(-12, 10)) + assert.Equal(t, int16(-1), performDivInt16(12, -10)) +} + +func TestIntegerMultiplication(t *testing.T) { + assert.PanicsWithValue(t, tree.ErrIntOutOfRange, func() { performMultInt8(math.MaxInt8-1, 100) }) + assert.PanicsWithValue(t, tree.ErrIntOutOfRange, func() { performMultInt8(math.MaxInt8-1, 3) }) + assert.PanicsWithValue(t, tree.ErrIntOutOfRange, func() { performMultInt8(math.MinInt8+1, 3) }) + assert.PanicsWithValue(t, tree.ErrIntOutOfRange, func() { performMultInt8(math.MinInt8+1, 100) }) + + assert.PanicsWithValue(t, tree.ErrIntOutOfRange, func() { performMultInt16(math.MaxInt16-1, 100) }) + assert.PanicsWithValue(t, tree.ErrIntOutOfRange, func() { performMultInt16(math.MaxInt16-1, 3) }) + assert.PanicsWithValue(t, tree.ErrIntOutOfRange, func() { performMultInt16(math.MinInt16+1, 3) }) + assert.PanicsWithValue(t, tree.ErrIntOutOfRange, func() { performMultInt16(math.MinInt16+1, 100) }) + + assert.PanicsWithValue(t, tree.ErrIntOutOfRange, func() { performMultInt32(math.MaxInt32-1, 100) }) + assert.PanicsWithValue(t, tree.ErrIntOutOfRange, func() { performMultInt32(math.MaxInt32-1, 3) }) + assert.PanicsWithValue(t, tree.ErrIntOutOfRange, func() { performMultInt32(math.MinInt32+1, 3) }) + assert.PanicsWithValue(t, tree.ErrIntOutOfRange, func() { performMultInt32(math.MinInt32+1, 100) }) + + assert.PanicsWithValue(t, tree.ErrIntOutOfRange, func() { performMultInt64(math.MaxInt64-1, 100) }) + assert.PanicsWithValue(t, tree.ErrIntOutOfRange, func() { performMultInt64(math.MaxInt64-1, 3) }) + assert.PanicsWithValue(t, tree.ErrIntOutOfRange, func() { performMultInt64(math.MinInt64+1, 3) }) + assert.PanicsWithValue(t, tree.ErrIntOutOfRange, func() { performMultInt64(math.MinInt64+1, 100) }) + + assert.PanicsWithValue(t, tree.ErrIntOutOfRange, func() { performMultInt8(math.MinInt8, -1) }) + assert.PanicsWithValue(t, tree.ErrIntOutOfRange, func() { performMultInt16(math.MinInt16, -1) }) + assert.PanicsWithValue(t, tree.ErrIntOutOfRange, func() { performMultInt32(math.MinInt32, -1) }) + assert.PanicsWithValue(t, tree.ErrIntOutOfRange, func() { performMultInt64(math.MinInt64, -1) }) + + assert.Equal(t, int8(-math.MaxInt8), performMultInt8(math.MaxInt8, -1)) + assert.Equal(t, int16(-math.MaxInt16), performMultInt16(math.MaxInt16, -1)) + assert.Equal(t, int32(-math.MaxInt32), performMultInt32(math.MaxInt32, -1)) + assert.Equal(t, int64(-math.MaxInt64), performMultInt64(math.MaxInt64, -1)) + + assert.Equal(t, int8(0), performMultInt8(math.MinInt8, 0)) + assert.Equal(t, int16(0), performMultInt16(math.MinInt16, 0)) + assert.Equal(t, int32(0), performMultInt32(math.MinInt32, 0)) + assert.Equal(t, int64(0), performMultInt64(math.MinInt64, 0)) + + assert.Equal(t, int8(120), performMultInt8(10, 12)) + assert.Equal(t, int16(120), performMultInt16(-10, -12)) + assert.Equal(t, int32(-120), performMultInt32(-12, 10)) + assert.Equal(t, int64(-120), performMultInt64(12, -10)) +} diff --git a/pkg/sql/exec/projection_ops_test.go b/pkg/sql/exec/projection_ops_test.go index a2b061ea69d3..0dfefe163c52 100644 --- a/pkg/sql/exec/projection_ops_test.go +++ b/pkg/sql/exec/projection_ops_test.go @@ -142,7 +142,12 @@ func TestGetProjectionOperator(t *testing.T) { } } -func benchmarkProjPlusInt64Int64Op(b *testing.B, useSelectionVector bool, hasNulls bool) { +func benchmarkProjOp( + b *testing.B, + makeProjOp func(source *RepeatableBatchSource) Operator, + useSelectionVector bool, + hasNulls bool, +) { ctx := context.Background() batch := coldata.NewMemBatch([]types.T{types.Int64, types.Int64, types.Int64}) @@ -173,26 +178,58 @@ func benchmarkProjPlusInt64Int64Op(b *testing.B, useSelectionVector bool, hasNul source := NewRepeatableBatchSource(batch) source.Init() - plusOp := &projPlusInt64Int64Op{ - input: source, - col1Idx: 0, - col2Idx: 1, - outputIdx: 2, - } - plusOp.Init() + op := makeProjOp(source) + op.Init() b.SetBytes(int64(8 * coldata.BatchSize * 2)) for i := 0; i < b.N; i++ { - plusOp.Next(ctx) + op.Next(ctx) } } -func BenchmarkProjPlusInt64Int64Op(b *testing.B) { - for _, useSel := range []bool{true, false} { - for _, hasNulls := range []bool{true, false} { - b.Run(fmt.Sprintf("useSel=%t,hasNulls=%t", useSel, hasNulls), func(b *testing.B) { - benchmarkProjPlusInt64Int64Op(b, useSel, hasNulls) - }) +func BenchmarkProjOp(b *testing.B) { + projOpMap := map[string]func(*RepeatableBatchSource) Operator{ + "projPlusInt64Int64Op": func(source *RepeatableBatchSource) Operator { + return &projPlusInt64Int64Op{ + input: source, + col1Idx: 0, + col2Idx: 1, + outputIdx: 2, + } + }, + "projMinusInt64Int64Op": func(source *RepeatableBatchSource) Operator { + return &projMinusInt64Int64Op{ + input: source, + col1Idx: 0, + col2Idx: 1, + outputIdx: 2, + } + }, + "projMultInt64Int64Op": func(source *RepeatableBatchSource) Operator { + return &projMultInt64Int64Op{ + input: source, + col1Idx: 0, + col2Idx: 1, + outputIdx: 2, + } + }, + "projDivInt64Int64Op": func(source *RepeatableBatchSource) Operator { + return &projDivInt64Int64Op{ + input: source, + col1Idx: 0, + col2Idx: 1, + outputIdx: 2, + } + }, + } + + for projOp, makeProjOp := range projOpMap { + for _, useSel := range []bool{true, false} { + for _, hasNulls := range []bool{true, false} { + b.Run(fmt.Sprintf("op=%s/useSel=%t/hasNulls=%t", projOp, useSel, hasNulls), func(b *testing.B) { + benchmarkProjOp(b, makeProjOp, useSel, hasNulls) + }) + } } } } diff --git a/pkg/sql/logictest/testdata/logic_test/edge b/pkg/sql/logictest/testdata/logic_test/edge index 965ab6bc4e72..fcfe3ca38445 100644 --- a/pkg/sql/logictest/testdata/logic_test/edge +++ b/pkg/sql/logictest/testdata/logic_test/edge @@ -195,7 +195,7 @@ WHERE ---- 131068 8589934588 -statement error sum exceeded int range +statement error integer out of range SELECT sum_int(t._int8) FROM t, t AS u WHERE t.key = 'max' query RRRRRRRRRR @@ -227,7 +227,7 @@ WHERE ---- -131072 -8589934592 -statement error sum exceeded int range +statement error integer out of range SELECT sum_int(t._int8) FROM t, t AS u WHERE t.key = 'min' query RRRRRRRRRR diff --git a/pkg/sql/logictest/testdata/logic_test/overflow b/pkg/sql/logictest/testdata/logic_test/overflow new file mode 100644 index 000000000000..74a200e0da47 --- /dev/null +++ b/pkg/sql/logictest/testdata/logic_test/overflow @@ -0,0 +1,21 @@ +# LogicTest: local-vec local-opt fakedist fakedist-opt fakedist-metadata + +# Test for overflow handling in sum aggregate. + +statement ok +CREATE TABLE large_numbers (a INT8) + +statement ok +INSERT INTO large_numbers VALUES (9223372036854775807),(1) + +statement error integer out of range +SELECT sum_int(a) FROM large_numbers + +statement ok +DELETE FROM large_numbers + +statement ok +INSERT INTO large_numbers VALUES (-9223372036854775808),(-1) + +statement error integer out of range +SELECT sum_int(a) FROM large_numbers diff --git a/pkg/sql/sem/builtins/aggregate_builtins.go b/pkg/sql/sem/builtins/aggregate_builtins.go index e4fe553957ec..3532d2727bcb 100644 --- a/pkg/sql/sem/builtins/aggregate_builtins.go +++ b/pkg/sql/sem/builtins/aggregate_builtins.go @@ -1025,9 +1025,7 @@ func (a *smallIntSumAggregate) Add(_ context.Context, datum tree.Datum, _ ...tre var ok bool a.sum, ok = arith.AddWithOverflow(a.sum, int64(tree.MustBeDInt(datum))) if !ok { - return pgerror.Newf(pgcode.NumericValueOutOfRange, - "sum exceeded int range", - ) + return tree.ErrIntOutOfRange } a.seenNonNull = true return nil diff --git a/pkg/sql/sem/tree/eval.go b/pkg/sql/sem/tree/eval.go index 4d58870efe33..5dc05824a840 100644 --- a/pkg/sql/sem/tree/eval.go +++ b/pkg/sql/sem/tree/eval.go @@ -48,7 +48,8 @@ import ( ) var ( - errIntOutOfRange = pgerror.New(pgcode.NumericValueOutOfRange, "integer out of range") + // ErrIntOutOfRange is reported when integer arithmetic overflows. + ErrIntOutOfRange = pgerror.New(pgcode.NumericValueOutOfRange, "integer out of range") errFloatOutOfRange = pgerror.New(pgcode.NumericValueOutOfRange, "float out of range") errDecOutOfRange = pgerror.New(pgcode.NumericValueOutOfRange, "decimal out of range") @@ -111,7 +112,7 @@ var UnaryOps = unaryOpFixups(map[UnaryOperator]unaryOpOverload{ Fn: func(_ *EvalContext, d Datum) (Datum, error) { i := MustBeDInt(d) if i == math.MinInt64 { - return nil, errIntOutOfRange + return nil, ErrIntOutOfRange } return NewDInt(-i), nil }, @@ -472,7 +473,7 @@ var BinOps = map[BinaryOperator]binOpOverload{ a, b := MustBeDInt(left), MustBeDInt(right) r, ok := arith.AddWithOverflow(int64(a), int64(b)) if !ok { - return nil, errIntOutOfRange + return nil, ErrIntOutOfRange } return NewDInt(DInt(r)), nil }, @@ -695,7 +696,7 @@ var BinOps = map[BinaryOperator]binOpOverload{ a, b := MustBeDInt(left), MustBeDInt(right) r, ok := arith.SubWithOverflow(int64(a), int64(b)) if !ok { - return nil, errIntOutOfRange + return nil, ErrIntOutOfRange } return NewDInt(DInt(r)), nil }, @@ -972,9 +973,9 @@ var BinOps = map[BinaryOperator]binOpOverload{ // ignore } else if a == math.MinInt64 || b == math.MinInt64 { // This test is required to detect math.MinInt64 * -1. - return nil, errIntOutOfRange + return nil, ErrIntOutOfRange } else if c/b != a { - return nil, errIntOutOfRange + return nil, ErrIntOutOfRange } return NewDInt(c), nil }, @@ -3090,7 +3091,7 @@ func PerformCast(ctx *EvalContext, d Datum, t *types.T) (Datum, error) { // is 9223372036854774784 (= float64(math.MaxInt64)-513), and both are // convertible to int without overflow. if math.IsNaN(f) || f <= float64(math.MinInt64) || f >= float64(math.MaxInt64) { - return nil, errIntOutOfRange + return nil, ErrIntOutOfRange } res = NewDInt(DInt(f)) case *DDecimal: @@ -3101,7 +3102,7 @@ func PerformCast(ctx *EvalContext, d Datum, t *types.T) (Datum, error) { } i, err := d.Int64() if err != nil { - return nil, errIntOutOfRange + return nil, ErrIntOutOfRange } res = NewDInt(DInt(i)) case *DString: @@ -3121,13 +3122,13 @@ func PerformCast(ctx *EvalContext, d Datum, t *types.T) (Datum, error) { case *DDate: // TODO(mjibson): This cast is unsupported by postgres. Should we remove ours? if !v.IsFinite() { - return nil, errIntOutOfRange + return nil, ErrIntOutOfRange } res = NewDInt(DInt(v.UnixEpochDays())) case *DInterval: iv, ok := v.AsInt64() if !ok { - return nil, errIntOutOfRange + return nil, ErrIntOutOfRange } res = NewDInt(DInt(iv)) case *DOid: @@ -5004,7 +5005,7 @@ func IntPow(x, y DInt) (*DInt, error) { } i, err := xd.Int64() if err != nil { - return nil, errIntOutOfRange + return nil, ErrIntOutOfRange } return NewDInt(DInt(i)), nil }