diff --git a/pkg/sql/sem/tree/datum.go b/pkg/sql/sem/tree/datum.go index be05c210f181..527a311427f6 100644 --- a/pkg/sql/sem/tree/datum.go +++ b/pkg/sql/sem/tree/datum.go @@ -924,7 +924,7 @@ func (d *DFloat) Prev(ctx CompareContext) (Datum, bool) { return nil, false } if f == math.Inf(-1) { - return dNaNFloat, true + return DNaNFloat, true } return NewDFloat(DFloat(math.Nextafter(f, math.Inf(-1)))), true } @@ -933,7 +933,7 @@ func (d *DFloat) Prev(ctx CompareContext) (Datum, bool) { func (d *DFloat) Next(ctx CompareContext) (Datum, bool) { f := float64(*d) if math.IsNaN(f) { - return dNegInfFloat, true + return DNegInfFloat, true } if f == math.Inf(+1) { return nil, false @@ -941,14 +941,20 @@ func (d *DFloat) Next(ctx CompareContext) (Datum, bool) { return NewDFloat(DFloat(math.Nextafter(f, math.Inf(+1)))), true } -var dZeroFloat = NewDFloat(0.0) -var dPosInfFloat = NewDFloat(DFloat(math.Inf(+1))) -var dNegInfFloat = NewDFloat(DFloat(math.Inf(-1))) -var dNaNFloat = NewDFloat(DFloat(math.NaN())) +var ( + // DZeroFloat is the DFloat for zero. + DZeroFloat = NewDFloat(0) + // DPosInfFloat is the DFloat for positive infinity. + DPosInfFloat = NewDFloat(DFloat(math.Inf(+1))) + // DNegInfFloat is the DFloat for negative infinity. + DNegInfFloat = NewDFloat(DFloat(math.Inf(-1))) + // DNaNFloat is the DFloat for NaN. + DNaNFloat = NewDFloat(DFloat(math.NaN())) +) // IsMax implements the Datum interface. func (d *DFloat) IsMax(ctx CompareContext) bool { - return *d == *dPosInfFloat + return *d == *DPosInfFloat } // IsMin implements the Datum interface. @@ -958,12 +964,12 @@ func (d *DFloat) IsMin(ctx CompareContext) bool { // Max implements the Datum interface. func (d *DFloat) Max(ctx CompareContext) (Datum, bool) { - return dPosInfFloat, true + return DPosInfFloat, true } // Min implements the Datum interface. func (d *DFloat) Min(ctx CompareContext) (Datum, bool) { - return dNaNFloat, true + return DNaNFloat, true } // AmbiguousFormat implements the Datum interface. @@ -2553,7 +2559,8 @@ func MustMakeDTimestamp(t time.Time, precision time.Duration) *DTimestamp { return ret } -var dZeroTimestamp = &DTimestamp{} +// DZeroTimestamp is the zero-valued DTimestamp. +var DZeroTimestamp = &DTimestamp{} // time.Time formats. const ( @@ -2868,7 +2875,8 @@ func ParseDTimestampTZ( return d, dependsOnContext, err } -var dZeroTimestampTZ = &DTimestampTZ{} +// DZeroTimestampTZ is the zero-valued DTimestampTZ. +var DZeroTimestampTZ = &DTimestampTZ{} // AsDTimestampTZ attempts to retrieve a DTimestampTZ from an Expr, returning a // DTimestampTZ and a flag signifying whether the assertion was successful. The @@ -5303,13 +5311,13 @@ func NewDefaultDatum(collationEnv *CollationEnvironment, t *types.T) (d Datum, e case types.IntFamily: return DZero, nil case types.FloatFamily: - return dZeroFloat, nil + return DZeroFloat, nil case types.DecimalFamily: return dZeroDecimal, nil case types.DateFamily: return dEpochDate, nil case types.TimestampFamily: - return dZeroTimestamp, nil + return DZeroTimestamp, nil case types.IntervalFamily: return dZeroInterval, nil case types.StringFamily: @@ -5317,7 +5325,7 @@ func NewDefaultDatum(collationEnv *CollationEnvironment, t *types.T) (d Datum, e case types.BytesFamily: return dEmptyBytes, nil case types.TimestampTZFamily: - return dZeroTimestampTZ, nil + return DZeroTimestampTZ, nil case types.CollatedStringFamily: return NewDCollatedString("", t.Locale(), collationEnv) case types.OidFamily: diff --git a/pkg/sql/stats/BUILD.bazel b/pkg/sql/stats/BUILD.bazel index 7d818e5ac120..fac5b36b18df 100644 --- a/pkg/sql/stats/BUILD.bazel +++ b/pkg/sql/stats/BUILD.bazel @@ -10,6 +10,7 @@ go_library( "histogram.go", "json.go", "new_stat.go", + "quantile.go", "row_sampling.go", "stats_cache.go", ], @@ -52,6 +53,7 @@ go_library( "//pkg/util/stop", "//pkg/util/syncutil", "//pkg/util/timeutil", + "//pkg/util/timeutil/pgdate", "//pkg/util/tracing", "@com_github_cockroachdb_errors//:errors", ], @@ -67,6 +69,7 @@ go_test( "delete_stats_test.go", "histogram_test.go", "main_test.go", + "quantile_test.go", "row_sampling_test.go", "stats_cache_test.go", ], @@ -117,6 +120,7 @@ go_test( "//pkg/util/randutil", "//pkg/util/retry", "//pkg/util/timeutil", + "//pkg/util/timeutil/pgdate", "@com_github_cockroachdb_errors//:errors", ], ) diff --git a/pkg/sql/stats/quantile.go b/pkg/sql/stats/quantile.go new file mode 100644 index 000000000000..93dd8a2b656c --- /dev/null +++ b/pkg/sql/stats/quantile.go @@ -0,0 +1,173 @@ +// Copyright 2022 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 stats + +import ( + "math" + "time" + + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/cockroach/pkg/util/timeutil/pgdate" + "github.com/cockroachdb/errors" +) + +// CanMakeQuantile returns true if a quantile function can be created for a +// histogram of the given type. +// TODO(michae2): Add support for DECIMAL, TIME, TIMETZ, and INTERVAL. +func CanMakeQuantile(colType *types.T) bool { + if colType.UserDefined() { + return false + } + switch colType.Family() { + case types.IntFamily, + types.FloatFamily, + types.DateFamily, + types.TimestampFamily, + types.TimestampTZFamily: + return true + default: + return false + } +} + +// ToQuantileValue converts from a datum to a float suitable for use in a quantile +// function. It differs from eval.PerformCast in a few ways: +// 1. It supports conversions that are not legal casts (e.g. DATE to FLOAT). +// 2. It errors on NaN and infinite values because they will break our model. +// FromQuantileValue is the inverse of this function, and together they should +// support round-trip conversions. +// TODO(michae2): Add support for DECIMAL, TIME, TIMETZ, and INTERVAL. +func ToQuantileValue(d tree.Datum) (float64, error) { + switch v := d.(type) { + case *tree.DInt: + return float64(*v), nil + case *tree.DFloat: + if math.IsNaN(float64(*v)) || math.IsInf(float64(*v), 0) { + return 0, tree.ErrFloatOutOfRange + } + return float64(*v), nil + case *tree.DDate: + if !v.IsFinite() { + return 0, tree.ErrFloatOutOfRange + } + // We use PG epoch instead of Unix epoch to simplify clamping when + // converting back. + return float64(v.PGEpochDays()), nil + case *tree.DTimestamp: + if v.Equal(pgdate.TimeInfinity) || v.Equal(pgdate.TimeNegativeInfinity) { + return 0, tree.ErrFloatOutOfRange + } + return float64(v.Unix()) + float64(v.Nanosecond())*1e-9, nil + case *tree.DTimestampTZ: + // TIMESTAMPTZ doesn't store a timezone, so this is the same as TIMESTAMP. + if v.Equal(pgdate.TimeInfinity) || v.Equal(pgdate.TimeNegativeInfinity) { + return 0, tree.ErrFloatOutOfRange + } + return float64(v.Unix()) + float64(v.Nanosecond())*1e-9, nil + default: + return 0, errors.Errorf("cannot make quantile value from %v", d) + } +} + +var ( + // quantileMinTimestamp is an alternative minimum finite DTimestamp value to + // avoid the problems around TimeNegativeInfinity, see #41564. + quantileMinTimestamp = tree.MinSupportedTime.Add(time.Second) + quantileMinTimestampSec = float64(quantileMinTimestamp.Unix()) + // quantileMaxTimestamp is an alternative maximum finite DTimestamp value to + // avoid the problems around TimeInfinity, see #41564. + quantileMaxTimestamp = tree.MaxSupportedTime.Add(-1 * time.Second).Truncate(time.Second) + quantileMaxTimestampSec = float64(quantileMaxTimestamp.Unix()) +) + +// FromQuantileValue converts from a quantile value back to a datum suitable for +// use in a histogram. It is the inverse of ToQuantileValue. It differs from +// eval.PerformCast in a few ways: +// 1. It supports conversions that are not legal casts (e.g. FLOAT to DATE). +// 2. It errors on NaN and infinite values because they indicate a problem with +// the regression model rather than valid values. +// 3. On overflow or underflow it clamps to maximum or minimum finite values +// rather than failing the conversion (and thus the entire histogram). +// TODO(michae2): Add support for DECIMAL, TIME, TIMETZ, and INTERVAL. +func FromQuantileValue(colType *types.T, val float64) (tree.Datum, error) { + if math.IsNaN(val) || math.IsInf(val, 0) { + return nil, tree.ErrFloatOutOfRange + } + switch colType.Family() { + case types.IntFamily: + i := math.Round(val) + // Clamp instead of truncating. + switch colType.Width() { + case 16: + if i < math.MinInt16 { + i = math.MinInt16 + } else if i > math.MaxInt16 { + i = math.MaxInt16 + } + case 32: + if i < math.MinInt32 { + i = math.MinInt32 + } else if i > math.MaxInt32 { + i = math.MaxInt32 + } + default: + if i < math.MinInt64 { + i = math.MinInt64 + } else if i >= math.MaxInt64 { + // float64 cannot represent 2^63 - 1 exactly, so cast directly to DInt. + return tree.NewDInt(tree.DInt(math.MaxInt64)), nil + } + } + return tree.NewDInt(tree.DInt(i)), nil + case types.FloatFamily: + switch colType.Width() { + case 32: + if val <= -math.MaxFloat32 { + val = -math.MaxFloat32 + } else if val >= math.MaxFloat32 { + val = math.MaxFloat32 + } else { + val = float64(float32(val)) + } + } + return tree.NewDFloat(tree.DFloat(val)), nil + case types.DateFamily: + days := math.Round(val) + // First clamp to int32. + if days < math.MinInt32 { + days = math.MinInt32 + } else if days > math.MaxInt32 { + days = math.MaxInt32 + } + // Then clamp to pgdate.Date. + return tree.NewDDate(pgdate.MakeDateFromPGEpochClampFinite(int32(days))), nil + case types.TimestampFamily, types.TimestampTZFamily: + sec, frac := math.Modf(val) + var t time.Time + // Clamp to (our alternative finite) DTimestamp bounds. + if sec <= quantileMinTimestampSec { + t = quantileMinTimestamp + } else if sec >= quantileMaxTimestampSec { + t = quantileMaxTimestamp + } else { + t = timeutil.Unix(int64(sec), int64(frac*1e9)) + } + roundTo := tree.TimeFamilyPrecisionToRoundDuration(colType.Precision()) + if colType.Family() == types.TimestampFamily { + return tree.MakeDTimestamp(t, roundTo) + } + return tree.MakeDTimestampTZ(t, roundTo) + default: + return nil, errors.Errorf("cannot convert quantile value to type %s", colType.Name()) + } +} diff --git a/pkg/sql/stats/quantile_test.go b/pkg/sql/stats/quantile_test.go new file mode 100644 index 000000000000..ea4027fe7c70 --- /dev/null +++ b/pkg/sql/stats/quantile_test.go @@ -0,0 +1,573 @@ +// Copyright 2022 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 stats + +import ( + "math" + "strconv" + "testing" + + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/timeutil/pgdate" +) + +// TODO(michae2): Test that a random histogram can round-trip to quantile +// and back. + +// Test conversions from datum to quantile value and back. +func TestQuantileValueRoundTrip(t *testing.T) { + testCases := []struct { + typ *types.T + dat tree.Datum + val float64 + err bool + }{ + // Integer cases. + { + typ: types.Int, + dat: tree.NewDInt(tree.DInt(0)), + val: 0, + }, + { + typ: types.Int, + dat: tree.NewDInt(tree.DInt(42)), + val: 42, + }, + { + typ: types.Int, + dat: tree.NewDInt(tree.DInt(math.MinInt32 - 1)), + val: math.MinInt32 - 1, + }, + { + typ: types.Int, + dat: tree.NewDInt(tree.DInt(math.MaxInt32 + 1)), + val: math.MaxInt32 + 1, + }, + { + typ: types.Int, + dat: tree.NewDInt(tree.DInt(math.MinInt64)), + val: math.MinInt64, + }, + { + typ: types.Int, + dat: tree.NewDInt(tree.DInt(math.MaxInt64)), + val: math.MaxInt64, + }, + { + typ: types.Int4, + dat: tree.NewDInt(tree.DInt(math.MinInt16 - 1)), + val: math.MinInt16 - 1, + }, + { + typ: types.Int4, + dat: tree.NewDInt(tree.DInt(math.MaxInt16 + 1)), + val: math.MaxInt16 + 1, + }, + { + typ: types.Int4, + dat: tree.NewDInt(tree.DInt(math.MinInt32)), + val: math.MinInt32, + }, + { + typ: types.Int4, + dat: tree.NewDInt(tree.DInt(math.MaxInt32)), + val: math.MaxInt32, + }, + { + typ: types.Int2, + dat: tree.NewDInt(tree.DInt(math.MinInt16)), + val: math.MinInt16, + }, + { + typ: types.Int2, + dat: tree.NewDInt(tree.DInt(math.MaxInt16)), + val: math.MaxInt16, + }, + // Float cases. + { + typ: types.Float, + dat: tree.DZeroFloat, + val: 0, + }, + { + typ: types.Float, + dat: tree.NewDFloat(tree.DFloat(-math.MaxFloat32 - 1)), + val: -math.MaxFloat32 - 1, + }, + { + typ: types.Float, + dat: tree.NewDFloat(tree.DFloat(math.MaxFloat32 + 1)), + val: math.MaxFloat32 + 1, + }, + { + typ: types.Float, + dat: tree.NewDFloat(tree.DFloat(-math.MaxFloat64)), + val: -math.MaxFloat64, + }, + { + typ: types.Float, + dat: tree.NewDFloat(tree.DFloat(math.MaxFloat64)), + val: math.MaxFloat64, + }, + { + typ: types.Float, + dat: tree.NewDFloat(tree.DFloat(math.Pi)), + val: math.Pi, + }, + { + typ: types.Float, + dat: tree.NewDFloat(tree.DFloat(math.SmallestNonzeroFloat64)), + val: math.SmallestNonzeroFloat64, + }, + { + typ: types.Float, + dat: tree.DNaNFloat, + err: true, + }, + { + typ: types.Float, + dat: tree.DNegInfFloat, + err: true, + }, + { + typ: types.Float, + dat: tree.DPosInfFloat, + err: true, + }, + { + typ: types.Float4, + dat: tree.NewDFloat(tree.DFloat(-math.MaxFloat32)), + val: -math.MaxFloat32, + }, + { + typ: types.Float4, + dat: tree.NewDFloat(tree.DFloat(math.MaxFloat32)), + val: math.MaxFloat32, + }, + { + typ: types.Float4, + dat: tree.NewDFloat(tree.DFloat(float32(math.Pi))), + val: float64(float32(math.Pi)), + }, + { + typ: types.Float4, + dat: tree.NewDFloat(tree.DFloat(math.SmallestNonzeroFloat32)), + val: math.SmallestNonzeroFloat32, + }, + { + typ: types.Float4, + dat: tree.DNaNFloat, + err: true, + }, + { + typ: types.Float4, + dat: tree.DNegInfFloat, + err: true, + }, + { + typ: types.Float4, + dat: tree.DPosInfFloat, + err: true, + }, + // Date cases. + { + typ: types.Date, + dat: tree.NewDDate(pgdate.MakeDateFromPGEpochClampFinite(0)), + val: 0, + }, + { + typ: types.Date, + dat: tree.NewDDate(pgdate.LowDate), + val: float64(pgdate.LowDate.PGEpochDays()), + }, + { + typ: types.Date, + dat: tree.NewDDate(pgdate.HighDate), + val: float64(pgdate.HighDate.PGEpochDays()), + }, + { + typ: types.Date, + dat: tree.NewDDate(pgdate.PosInfDate), + err: true, + }, + { + typ: types.Date, + dat: tree.NewDDate(pgdate.NegInfDate), + err: true, + }, + // Timestamp cases. + { + typ: types.Timestamp, + dat: tree.DZeroTimestamp, + val: float64(tree.DZeroTimestamp.Unix()), + }, + { + typ: types.Timestamp, + dat: &tree.DTimestamp{Time: quantileMinTimestamp}, + val: quantileMinTimestampSec, + }, + { + typ: types.Timestamp, + dat: &tree.DTimestamp{Time: quantileMaxTimestamp}, + val: quantileMaxTimestampSec, + }, + { + typ: types.Timestamp, + dat: &tree.DTimestamp{Time: pgdate.TimeNegativeInfinity}, + err: true, + }, + { + typ: types.Timestamp, + dat: &tree.DTimestamp{Time: pgdate.TimeInfinity}, + err: true, + }, + { + typ: types.TimestampTZ, + dat: tree.DZeroTimestampTZ, + val: float64(tree.DZeroTimestampTZ.Unix()), + }, + { + typ: types.TimestampTZ, + dat: &tree.DTimestampTZ{Time: quantileMinTimestamp}, + val: quantileMinTimestampSec, + }, + { + typ: types.TimestampTZ, + dat: &tree.DTimestampTZ{Time: quantileMaxTimestamp}, + val: quantileMaxTimestampSec, + }, + { + typ: types.TimestampTZ, + dat: &tree.DTimestampTZ{Time: pgdate.TimeNegativeInfinity}, + err: true, + }, + { + typ: types.TimestampTZ, + dat: &tree.DTimestampTZ{Time: pgdate.TimeInfinity}, + err: true, + }, + } + evalCtx := eval.NewTestingEvalContext(cluster.MakeTestingClusterSettings()) + for i, tc := range testCases { + t.Run(strconv.Itoa(i), func(t *testing.T) { + val, err := ToQuantileValue(tc.dat) + if err != nil { + if !tc.err { + t.Errorf("test case %d (%v) unexpected ToQuantileValue err: %v", i, tc.typ.Name(), err) + } + return + } + if tc.err { + t.Errorf("test case %d (%v) expected ToQuantileValue err", i, tc.typ.Name()) + return + } + if val != tc.val { + t.Errorf("test case %d (%v) incorrect val %v expected %v", i, tc.typ.Name(), val, tc.val) + return + } + // Check that we can make the round trip. + res, err := FromQuantileValue(tc.typ, val) + if err != nil { + t.Errorf("test case %d (%v) unexpected FromQuantileValue err: %v", i, tc.typ.Name(), err) + return + } + cmp, err := res.CompareError(evalCtx, tc.dat) + if err != nil { + t.Errorf("test case %d (%v) unexpected CompareError err: %v", i, tc.typ.Name(), err) + return + } + if cmp != 0 { + t.Errorf("test case %d (%v) incorrect datum %v expected %v", i, tc.typ.Name(), res, tc.dat) + } + }) + } +} + +// Test conversions from quantile value to datum and back. +// TestQuantileValueRoundTrip covers similar ground, so here we focus on cases +// that overflow or underflow and have to clamp. +func TestQuantileValueRoundTripOverflow(t *testing.T) { + testCases := []struct { + typ *types.T + val float64 + dat tree.Datum + err bool + res float64 + }{ + // Integer cases. + { + typ: types.Int, + val: math.MinInt64 - 1, + dat: tree.NewDInt(tree.DInt(math.MinInt64)), + res: math.MinInt64, + }, + { + typ: types.Int, + val: math.MaxInt64 + 1, + dat: tree.NewDInt(tree.DInt(math.MaxInt64)), + res: math.MaxInt64, + }, + { + typ: types.Int, + val: -math.MaxFloat64, + dat: tree.NewDInt(tree.DInt(math.MinInt64)), + res: math.MinInt64, + }, + { + typ: types.Int, + val: math.MaxFloat64, + dat: tree.NewDInt(tree.DInt(math.MaxInt64)), + res: math.MaxInt64, + }, + { + typ: types.Int4, + val: math.MinInt32 - 1, + dat: tree.NewDInt(tree.DInt(math.MinInt32)), + res: math.MinInt32, + }, + { + typ: types.Int4, + val: math.MaxInt32 + 1, + dat: tree.NewDInt(tree.DInt(math.MaxInt32)), + res: math.MaxInt32, + }, + { + typ: types.Int4, + val: -math.MaxFloat64, + dat: tree.NewDInt(tree.DInt(math.MinInt32)), + res: math.MinInt32, + }, + { + typ: types.Int4, + val: math.MaxFloat64, + dat: tree.NewDInt(tree.DInt(math.MaxInt32)), + res: math.MaxInt32, + }, + { + typ: types.Int2, + val: math.MinInt16 - 1, + dat: tree.NewDInt(tree.DInt(math.MinInt16)), + res: math.MinInt16, + }, + { + typ: types.Int2, + val: math.MaxInt16 + 1, + dat: tree.NewDInt(tree.DInt(math.MaxInt16)), + res: math.MaxInt16, + }, + { + typ: types.Int2, + val: -math.MaxFloat64, + dat: tree.NewDInt(tree.DInt(math.MinInt16)), + res: math.MinInt16, + }, + { + typ: types.Int2, + val: math.MaxFloat64, + dat: tree.NewDInt(tree.DInt(math.MaxInt16)), + res: math.MaxInt16, + }, + // Float cases. + { + typ: types.Float, + val: -math.MaxFloat64, + dat: tree.NewDFloat(tree.DFloat(-math.MaxFloat64)), + res: -math.MaxFloat64, + }, + { + typ: types.Float, + val: math.MaxFloat64, + dat: tree.NewDFloat(tree.DFloat(math.MaxFloat64)), + res: math.MaxFloat64, + }, + { + typ: types.Float, + val: -math.SmallestNonzeroFloat64, + dat: tree.NewDFloat(tree.DFloat(-math.SmallestNonzeroFloat64)), + res: -math.SmallestNonzeroFloat64, + }, + { + typ: types.Float, + val: math.SmallestNonzeroFloat64, + dat: tree.NewDFloat(tree.DFloat(math.SmallestNonzeroFloat64)), + res: math.SmallestNonzeroFloat64, + }, + { + typ: types.Float, + val: math.NaN(), + err: true, + }, + { + typ: types.Float, + val: math.Inf(-1), + err: true, + }, + { + typ: types.Float, + val: math.Inf(+1), + err: true, + }, + { + typ: types.Float4, + val: -math.MaxFloat32 - 1, + dat: tree.NewDFloat(tree.DFloat(-math.MaxFloat32)), + res: -math.MaxFloat32, + }, + { + typ: types.Float4, + val: math.MaxFloat32 + 1, + dat: tree.NewDFloat(tree.DFloat(math.MaxFloat32)), + res: math.MaxFloat32, + }, + { + typ: types.Float4, + val: -math.MaxFloat64, + dat: tree.NewDFloat(tree.DFloat(-math.MaxFloat32)), + res: -math.MaxFloat32, + }, + { + typ: types.Float4, + val: math.MaxFloat64, + dat: tree.NewDFloat(tree.DFloat(math.MaxFloat32)), + res: math.MaxFloat32, + }, + { + typ: types.Float4, + val: math.Pi, + dat: tree.NewDFloat(tree.DFloat(float32(math.Pi))), + res: float64(float32(math.Pi)), + }, + { + typ: types.Float4, + val: -math.SmallestNonzeroFloat64, + dat: tree.DZeroFloat, + res: 0, + }, + { + typ: types.Float4, + val: math.SmallestNonzeroFloat64, + dat: tree.DZeroFloat, + res: 0, + }, + // Date cases. + { + typ: types.Date, + val: float64(pgdate.LowDate.PGEpochDays()) - 1, + dat: tree.NewDDate(pgdate.LowDate), + res: float64(pgdate.LowDate.PGEpochDays()), + }, + { + typ: types.Date, + val: float64(pgdate.HighDate.PGEpochDays()) + 1, + dat: tree.NewDDate(pgdate.HighDate), + res: float64(pgdate.HighDate.PGEpochDays()), + }, + { + typ: types.Date, + val: -math.MaxFloat64, + dat: tree.NewDDate(pgdate.LowDate), + res: float64(pgdate.LowDate.PGEpochDays()), + }, + { + typ: types.Date, + val: math.MaxFloat64, + dat: tree.NewDDate(pgdate.HighDate), + res: float64(pgdate.HighDate.PGEpochDays()), + }, + // Timestamp cases. + { + typ: types.Timestamp, + val: float64(pgdate.TimeNegativeInfinity.Unix()), + dat: &tree.DTimestamp{Time: quantileMinTimestamp}, + res: quantileMinTimestampSec, + }, + { + typ: types.Timestamp, + val: float64(pgdate.TimeInfinity.Unix()), + dat: &tree.DTimestamp{Time: quantileMaxTimestamp}, + res: quantileMaxTimestampSec, + }, + { + typ: types.Timestamp, + val: -math.MaxFloat64, + dat: &tree.DTimestamp{Time: quantileMinTimestamp}, + res: quantileMinTimestampSec, + }, + { + typ: types.Timestamp, + val: math.MaxFloat64, + dat: &tree.DTimestamp{Time: quantileMaxTimestamp}, + res: quantileMaxTimestampSec, + }, + { + typ: types.TimestampTZ, + val: float64(pgdate.TimeNegativeInfinity.Unix()), + dat: &tree.DTimestampTZ{Time: quantileMinTimestamp}, + res: quantileMinTimestampSec, + }, + { + typ: types.TimestampTZ, + val: float64(pgdate.TimeInfinity.Unix()), + dat: &tree.DTimestampTZ{Time: quantileMaxTimestamp}, + res: quantileMaxTimestampSec, + }, + { + typ: types.TimestampTZ, + val: -math.MaxFloat64, + dat: &tree.DTimestampTZ{Time: quantileMinTimestamp}, + res: quantileMinTimestampSec, + }, + { + typ: types.TimestampTZ, + val: math.MaxFloat64, + dat: &tree.DTimestampTZ{Time: quantileMaxTimestamp}, + res: quantileMaxTimestampSec, + }, + } + evalCtx := eval.NewTestingEvalContext(cluster.MakeTestingClusterSettings()) + for i, tc := range testCases { + t.Run(strconv.Itoa(i), func(t *testing.T) { + d, err := FromQuantileValue(tc.typ, tc.val) + if err != nil { + if !tc.err { + t.Errorf("test case %d (%v) unexpected FromQuantileValue err: %v", i, tc.typ.Name(), err) + } + return + } + if tc.err { + t.Errorf("test case %d (%v) expected FromQuantileValue err", i, tc.typ.Name()) + return + } + cmp, err := d.CompareError(evalCtx, tc.dat) + if err != nil { + t.Errorf("test case %d (%v) unexpected CompareError err: %v", i, tc.typ.Name(), err) + return + } + if cmp != 0 { + t.Errorf("test case %d (%v) incorrect datum %v expected %v", i, tc.typ.Name(), d, tc.dat) + return + } + // Check that we can make the round trip with the clamped value. + res, err := ToQuantileValue(d) + if err != nil { + t.Errorf("test case %d (%v) unexpected ToQuantileValue err: %v", i, tc.typ.Name(), err) + return + } + if res != tc.res { + t.Errorf("test case %d (%v) incorrect val %v expected %v", i, tc.typ.Name(), res, tc.res) + return + } + }) + } +} diff --git a/pkg/util/timeutil/pgdate/pgdate.go b/pkg/util/timeutil/pgdate/pgdate.go index 9260c1d91696..34f92318a2e1 100644 --- a/pkg/util/timeutil/pgdate/pgdate.go +++ b/pkg/util/timeutil/pgdate/pgdate.go @@ -141,6 +141,17 @@ func MakeDateFromPGEpoch(days int32) (Date, error) { return Date{days: days}, nil } +// MakeDateFromPGEpochClampFinite creates a Date from the number of days since +// 2000-01-01, clamping to LowDate or HighDate if outside those bounds. +func MakeDateFromPGEpochClampFinite(days int32) Date { + if days < lowDays { + return LowDate + } else if days > highDays { + return HighDate + } + return Date{days: days} +} + // ToTime returns d as a time.Time. Non finite dates return an error. func (d Date) ToTime() (time.Time, error) { if d.days == math.MinInt32 || d.days == math.MaxInt32 {