diff --git a/pkg/sql/rowexec/sample_aggregator.go b/pkg/sql/rowexec/sample_aggregator.go index 46d09424d877..cba01aacd3dc 100644 --- a/pkg/sql/rowexec/sample_aggregator.go +++ b/pkg/sql/rowexec/sample_aggregator.go @@ -616,6 +616,8 @@ func (s *sampleAggregator) generateHistogram( prevCapacity, sr.Cap(), ) } + // TODO(michae2): Instead of using the flowCtx's evalCtx, investigate + // whether this can use a nil *eval.Context. h, _, err := stats.EquiDepthHistogram(evalCtx, colType, values, numRows, distinctCount, maxBuckets) return h, err } diff --git a/pkg/sql/sem/eval/context.go b/pkg/sql/sem/eval/context.go index 666328012ec0..8d92cf6973c7 100644 --- a/pkg/sql/sem/eval/context.go +++ b/pkg/sql/sem/eval/context.go @@ -566,6 +566,9 @@ func (ec *Context) SetStmtTimestamp(ts time.Time) { // GetLocation returns the session timezone. func (ec *Context) GetLocation() *time.Location { + if ec == nil { + return time.UTC + } return ec.SessionData().GetLocation() } diff --git a/pkg/sql/show_stats.go b/pkg/sql/show_stats.go index d683b246b6b7..86e847d2e3fe 100644 --- a/pkg/sql/show_stats.go +++ b/pkg/sql/show_stats.go @@ -179,7 +179,7 @@ func (p *planner) ShowTableStats(ctx context.Context, n *tree.ShowTableStats) (p observed[i], observed[j] = observed[j], observed[i] } - forecasts := stats.ForecastTableStatistics(ctx, p.EvalContext(), observed) + forecasts := stats.ForecastTableStatistics(ctx, observed) // Iterate in reverse order to match the ORDER BY "columnIDs". for i := len(forecasts) - 1; i >= 0; i-- { diff --git a/pkg/sql/stats/forecast.go b/pkg/sql/stats/forecast.go index e1ed47595bb0..08670e78ab23 100644 --- a/pkg/sql/stats/forecast.go +++ b/pkg/sql/stats/forecast.go @@ -17,7 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" - "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" @@ -57,12 +57,7 @@ const maxForecastDistance = time.Hour * 24 * 7 // // ForecastTableStatistics is deterministic: given the same observations it will // return the same forecasts. -// -// TODO(michae2): Use nil *eval.Context or custom tree.CompareContext instead of -// taking an evalCtx. -func ForecastTableStatistics( - ctx context.Context, evalCtx tree.CompareContext, observed []*TableStatistic, -) []*TableStatistic { +func ForecastTableStatistics(ctx context.Context, observed []*TableStatistic) []*TableStatistic { // Early sanity check. We'll check this again in forecastColumnStatistics. if len(observed) < minObservationsForForecast { return nil @@ -102,9 +97,7 @@ func ForecastTableStatistics( forecasts := make([]*TableStatistic, 0, len(forecastCols)) for _, colKey := range forecastCols { - forecast, err := forecastColumnStatistics( - ctx, evalCtx, observedByCols[colKey], at, minGoodnessOfFit, - ) + forecast, err := forecastColumnStatistics(ctx, observedByCols[colKey], at, minGoodnessOfFit) if err != nil { log.VEventf( ctx, 2, "could not forecast statistics for table %v columns %s: %v", @@ -135,11 +128,7 @@ func ForecastTableStatistics( // forecastColumnStatistics is deterministic: given the same observations and // forecast time, it will return the same forecast. func forecastColumnStatistics( - ctx context.Context, - evalCtx tree.CompareContext, - observed []*TableStatistic, - at time.Time, - minRequiredFit float64, + ctx context.Context, observed []*TableStatistic, at time.Time, minRequiredFit float64, ) (forecast *TableStatistic, err error) { if len(observed) < minObservationsForForecast { return nil, errors.New("not enough observations to forecast statistics") @@ -263,9 +252,7 @@ func forecastColumnStatistics( // histogram. NOTE: If any of the observed histograms were for inverted // indexes this will produce an incorrect histogram. if observed[0].HistogramData != nil { - hist, err := predictHistogram( - ctx, evalCtx, observed, forecastAt, minRequiredFit, nonNullRowCount, - ) + hist, err := predictHistogram(ctx, observed, forecastAt, minRequiredFit, nonNullRowCount) if err != nil { // If we did not successfully predict a histogram then copy the latest // histogram so we can adjust it. @@ -276,8 +263,10 @@ func forecastColumnStatistics( hist.buckets = append([]cat.HistogramBucket{}, observed[0].nonNullHistogram().buckets...) } - // Now adjust for consistency. - hist.adjustCounts(evalCtx, nonNullRowCount, nonNullDistinctCount) + // Now adjust for consistency. We don't use any session data for operations + // on upper bounds, so a nil *eval.Context works as our tree.CompareContext. + var compareCtx *eval.Context + hist.adjustCounts(compareCtx, nonNullRowCount, nonNullDistinctCount) // Finally, convert back to HistogramData. histData, err := hist.toHistogramData(observed[0].HistogramData.ColumnType) @@ -294,7 +283,6 @@ func forecastColumnStatistics( // predictHistogram tries to predict the histogram at forecast time. func predictHistogram( ctx context.Context, - evalCtx tree.CompareContext, observed []*TableStatistic, forecastAt float64, minRequiredFit float64, @@ -359,5 +347,5 @@ func predictHistogram( } // Finally, convert the predicted quantile function back to a histogram. - return yₙ.toHistogram(evalCtx, colType, nonNullRowCount) + return yₙ.toHistogram(colType, nonNullRowCount) } diff --git a/pkg/sql/stats/forecast_test.go b/pkg/sql/stats/forecast_test.go index b4a4ad43dc5a..65d384711964 100644 --- a/pkg/sql/stats/forecast_test.go +++ b/pkg/sql/stats/forecast_test.go @@ -18,10 +18,8 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" - "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/catid" - "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/timeutil" ) @@ -581,7 +579,6 @@ func TestForecastColumnStatistics(t *testing.T) { }, } ctx := context.Background() - evalCtx := eval.NewTestingEvalContext(cluster.MakeTestingClusterSettings()) for i, tc := range testCases { t.Run(strconv.Itoa(i), func(t *testing.T) { @@ -593,7 +590,7 @@ func TestForecastColumnStatistics(t *testing.T) { expected := tc.forecast.toTableStatistic(jobspb.ForecastStatsName, i) at := testStatTime(tc.at) - forecast, err := forecastColumnStatistics(ctx, evalCtx, observed, at, 1) + forecast, err := forecastColumnStatistics(ctx, observed, at, 1) if err != nil { if !tc.err { t.Errorf("test case %d unexpected forecastColumnStatistics err: %v", i, err) diff --git a/pkg/sql/stats/quantile.go b/pkg/sql/stats/quantile.go index e48130f36109..2e69d6cd154c 100644 --- a/pkg/sql/stats/quantile.go +++ b/pkg/sql/stats/quantile.go @@ -16,6 +16,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" + "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" @@ -237,9 +238,7 @@ func makeQuantile(hist histogram, rowCount float64) (quantile, error) { // toHistogram converts a quantile into a histogram, using the provided type and // row count. It returns an error if the conversion fails. The quantile must be // well-formed before calling toHistogram. -func (q quantile) toHistogram( - compareCtx tree.CompareContext, colType *types.T, rowCount float64, -) (histogram, error) { +func (q quantile) toHistogram(colType *types.T, rowCount float64) (histogram, error) { if len(q) < 2 || q[0].p != 0 || q[len(q)-1].p != 1 { return histogram{}, errors.AssertionFailedf("invalid quantile: %v", q) } @@ -249,6 +248,10 @@ func (q quantile) toHistogram( return histogram{buckets: make([]cat.HistogramBucket, 0)}, nil } + // We don't use any session data for conversions or operations on upper + // bounds, so a nil *eval.Context works as our tree.CompareContext. + var compareCtx *eval.Context + hist := histogram{buckets: make([]cat.HistogramBucket, 0, len(q)-1)} var i quantileIndex diff --git a/pkg/sql/stats/quantile_test.go b/pkg/sql/stats/quantile_test.go index ea2818a07ae2..48381cd1898f 100644 --- a/pkg/sql/stats/quantile_test.go +++ b/pkg/sql/stats/quantile_test.go @@ -20,7 +20,6 @@ import ( "strconv" "testing" - "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -41,19 +40,18 @@ func TestRandomQuantileRoundTrip(t *testing.T) { types.Float4, } colTypes = append(colTypes, types.Scalar...) - evalCtx := eval.NewTestingEvalContext(cluster.MakeTestingClusterSettings()) rng, seed := randutil.NewTestRand() for _, colType := range colTypes { if canMakeQuantile(histVersion, colType) { for i := 0; i < 5; i++ { t.Run(fmt.Sprintf("%v/%v", colType.Name(), i), func(t *testing.T) { - hist, rowCount := randHist(evalCtx, colType, rng) + hist, rowCount := randHist(colType, rng) qfun, err := makeQuantile(hist, rowCount) if err != nil { t.Errorf("seed: %v unexpected makeQuantile error: %v", seed, err) return } - hist2, err := qfun.toHistogram(evalCtx, colType, rowCount) + hist2, err := qfun.toHistogram(colType, rowCount) if err != nil { t.Errorf("seed: %v unexpected quantile.toHistogram error: %v", seed, err) return @@ -70,12 +68,10 @@ func TestRandomQuantileRoundTrip(t *testing.T) { // randHist makes a random histogram of the specified type, with [1, 200] // buckets. Not all types are supported. Every bucket will have NumEq > 0 but // could have NumRange == 0. -func randHist( - compareCtx tree.CompareContext, colType *types.T, rng *rand.Rand, -) (histogram, float64) { +func randHist(colType *types.T, rng *rand.Rand) (histogram, float64) { numBuckets := rng.Intn(200) + 1 buckets := make([]cat.HistogramBucket, numBuckets) - bounds := randBounds(compareCtx, colType, rng, numBuckets) + bounds := randBounds(colType, rng, numBuckets) buckets[0].NumEq = float64(rng.Intn(100) + 1) buckets[0].UpperBound = bounds[0] rowCount := buckets[0].NumEq @@ -98,6 +94,7 @@ func randHist( rowCount += rows } // Set DistinctRange in all buckets. + var compareCtx *eval.Context for i := 1; i < len(buckets); i++ { lowerBound := getNextLowerBound(compareCtx, buckets[i-1].UpperBound) buckets[i].DistinctRange = estimatedDistinctValuesInRange( @@ -111,9 +108,7 @@ func randHist( // type. Not all types are supported. This differs from randgen.RandDatum in // that it generates no "interesting" Datums, and differs from // randgen.RandDatumSimple in that it generates distinct Datums without repeats. -func randBounds( - compareCtx tree.CompareContext, colType *types.T, rng *rand.Rand, num int, -) tree.Datums { +func randBounds(colType *types.T, rng *rand.Rand, num int) tree.Datums { datums := make(tree.Datums, num) // randInts creates an ordered slice of num distinct random ints in the closed @@ -566,10 +561,9 @@ func TestQuantileToHistogram(t *testing.T) { err: true, }, } - evalCtx := eval.NewTestingEvalContext(cluster.MakeTestingClusterSettings()) for i, tc := range testCases { t.Run(strconv.Itoa(i), func(t *testing.T) { - hist, err := tc.qfun.toHistogram(evalCtx, types.Float, tc.rows) + hist, err := tc.qfun.toHistogram(types.Float, tc.rows) if err != nil { if !tc.err { t.Errorf("test case %d unexpected quantile.toHistogram err: %v", i, err) @@ -843,7 +837,7 @@ func TestQuantileValueRoundTrip(t *testing.T) { err: true, }, } - evalCtx := eval.NewTestingEvalContext(cluster.MakeTestingClusterSettings()) + var compareCtx *eval.Context for i, tc := range testCases { t.Run(strconv.Itoa(i), func(t *testing.T) { val, err := toQuantileValue(tc.dat) @@ -867,7 +861,7 @@ func TestQuantileValueRoundTrip(t *testing.T) { t.Errorf("test case %d (%v) unexpected fromQuantileValue err: %v", i, tc.typ.Name(), err) return } - cmp, err := res.CompareError(evalCtx, tc.dat) + cmp, err := res.CompareError(compareCtx, tc.dat) if err != nil { t.Errorf("test case %d (%v) unexpected CompareError err: %v", i, tc.typ.Name(), err) return @@ -1120,7 +1114,7 @@ func TestQuantileValueRoundTripOverflow(t *testing.T) { res: quantileMaxTimestampSec, }, } - evalCtx := eval.NewTestingEvalContext(cluster.MakeTestingClusterSettings()) + var compareCtx *eval.Context for i, tc := range testCases { t.Run(strconv.Itoa(i), func(t *testing.T) { d, err := fromQuantileValue(tc.typ, tc.val) @@ -1134,7 +1128,7 @@ func TestQuantileValueRoundTripOverflow(t *testing.T) { t.Errorf("test case %d (%v) expected fromQuantileValue err", i, tc.typ.Name()) return } - cmp, err := d.CompareError(evalCtx, tc.dat) + cmp, err := d.CompareError(compareCtx, tc.dat) if err != nil { t.Errorf("test case %d (%v) unexpected CompareError err: %v", i, tc.typ.Name(), err) return