Skip to content

Commit

Permalink
sql/stats: conversion of datums to and from quantile function values
Browse files Browse the repository at this point in the history
To predict histograms in statistics forecasts, we will use linear
regression over quantile functions. (Quantile functions are another
representation of histogram data, in a form more amenable to statistical
manipulation.)

The conversion of histograms to quantile functions will require
conversion of histogram bounds (datums) to quantile values (float64s).
And likewise, the inverse conversion from quantile functions back to
histograms will require the inverse conversion of float64 quantile
values back to datums. These conversions are a little different from our
usual SQL conversions in `eval.PerformCast`, so we add them to a new
quantile file in the `sql/stats` module.

This code was originally part of #77070 but has been pulled out to
simplify that PR. A few changes have been made:
- `histogramValue` has been renamed to `FromQuantileValue`.
- Support for `DECIMAL`, `TIME`, `TIMETZ`, and `INTERVAL` has been
  dropped. Clamping these types in `FromQuantileValue` was too complex
  for the first iteration of statistics forecasting. We expect the
  overwhelming majority of ascending keys to use `INT` or `TIMESTAMP`
  types.
- Bugs in `FLOAT4`, `TIMESTAMP` and `TIMESTAMPTZ` conversions have been
  fixed.
- We're now clamping timestamps to slightly tighter bounds to avoid the
  problems with infinite timestamps (see #41564).

Assists: #79872

Release note: None
  • Loading branch information
michae2 committed Jul 6, 2022
1 parent 9c5472e commit 73db7db
Show file tree
Hide file tree
Showing 5 changed files with 791 additions and 14 deletions.
36 changes: 22 additions & 14 deletions pkg/sql/sem/tree/datum.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
Expand All @@ -933,22 +933,28 @@ 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
}
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.
Expand All @@ -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.
Expand Down Expand Up @@ -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 (
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -5301,21 +5309,21 @@ 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:
return dEmptyString, nil
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:
Expand Down
4 changes: 4 additions & 0 deletions pkg/sql/stats/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@ go_library(
"histogram.go",
"json.go",
"new_stat.go",
"quantile.go",
"row_sampling.go",
"stats_cache.go",
],
Expand Down Expand Up @@ -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",
],
Expand All @@ -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",
],
Expand Down Expand Up @@ -117,6 +120,7 @@ go_test(
"//pkg/util/randutil",
"//pkg/util/retry",
"//pkg/util/timeutil",
"//pkg/util/timeutil/pgdate",
"@com_github_cockroachdb_errors//:errors",
],
)
Expand Down
184 changes: 184 additions & 0 deletions pkg/sql/stats/quantile.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,184 @@
// 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:
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 {
return tree.NewDInt(tree.DInt(math.MinInt16)), nil
}
if i >= math.MaxInt16 {
return tree.NewDInt(tree.DInt(math.MaxInt16)), nil
}
case 32:
if i <= math.MinInt32 {
return tree.NewDInt(tree.DInt(math.MinInt32)), nil
}
if i >= math.MaxInt32 {
return tree.NewDInt(tree.DInt(math.MaxInt32)), nil
}
default:
if i <= math.MinInt64 {
return tree.NewDInt(tree.DInt(math.MinInt64)), nil
}
if i >= math.MaxInt64 {
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:
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())
return tree.MakeDTimestamp(t, roundTo)
case 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())
return tree.MakeDTimestampTZ(t, roundTo)
default:
return nil, errors.Errorf("cannot convert quantile value to type %s", colType.Name())
}
}
Loading

0 comments on commit 73db7db

Please sign in to comment.