Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
125345: sql: dynamically determine histogram sample size r=Uzair5162 a=Uzair5162

We currently sample a default of 10k rows when collecting stats to construct histograms. We have seen cases of histograms for large tables missing frequent values as a result of poor samples, and users have frequently reported slow queries as a result of this.

This commit dynamically picks a sample size based on estimated table size $n$, using a sample size of $582n^{0.29}$ bounded between 10k and 300k. This formula was derived using empirical research on the performance of different sample sizes using the code in effc506. The formula approximates the following table:

| Table Size    | Sample Size |
| ------------- | ----------- |
| 10,000        | 10,000      |
| 100,000       | 15,000      |
| 1,000,000     | 30,000      |
| 10,000,000    | 60,000      |
| 100,000,000   | 100,000     |
| 1,000,000,000 | 300,000     |

These sample sizes empirically achieved the following coverage:

- 100k rows/15k samples: ~100% coverage of multiplicities down to 100x, ~80% down to 10x
- 1m rows/30k samples: ~100% coverage of multiplicities down to 1000x, ~95% down to 100x
- 10m rows/60k samples: ~100% coverage of multiplicities down to 10000x, ~95% down to 1000x, ~50% down to 100x
- 100m rows/100k samples: ~100% coverage of multiplicities down to 10000x, ~65% down to 1000x, ~10% down to 100x
- 1b rows/300k samples: ~100% coverage of multiplicities down to 100000x, ~95% down to 10000x, ~25% down to 1000x

Fixes: cockroachdb#123972

Fixes: cockroachdb#97701

Release note (sql change): Histograms are no longer constructed using a default sample size of 10k. Samples are now dynamically sized based on table size unless the sample count has been set in the table or cluster settings.

Co-authored-by: Uzair Ahmad <[email protected]>
  • Loading branch information
craig[bot] and Uzair5162 committed Jun 12, 2024
2 parents f5e65c5 + 2c036cf commit d65a04b
Show file tree
Hide file tree
Showing 6 changed files with 207 additions and 26 deletions.
2 changes: 1 addition & 1 deletion docs/generated/settings/settings-for-tenants.txt
Original file line number Diff line number Diff line change
Expand Up @@ -307,7 +307,7 @@ sql.stats.forecasts.min_goodness_of_fit float 0.95 the minimum R² (goodness of
sql.stats.forecasts.min_observations integer 3 the mimimum number of observed statistics required to produce a statistics forecast application
sql.stats.histogram_buckets.count integer 200 maximum number of histogram buckets to build during table statistics collection application
sql.stats.histogram_collection.enabled boolean true histogram collection mode application
sql.stats.histogram_samples.count integer 10000 number of rows sampled for histogram construction during table statistics collection application
sql.stats.histogram_samples.count integer 0 number of rows sampled for histogram construction during table statistics collection. Not setting this or setting a value of 0 means that a reasonable sample size will be automatically picked based on the table size. application
sql.stats.multi_column_collection.enabled boolean true multi-column statistics collection mode application
sql.stats.non_default_columns.min_retention_period duration 24h0m0s minimum retention period for table statistics collected on non-default columns application
sql.stats.persisted_rows.max integer 1000000 maximum number of rows of statement and transaction statistics that will be persisted in the system tables before compaction begins application
Expand Down
2 changes: 1 addition & 1 deletion docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -258,7 +258,7 @@
<tr><td><div id="setting-sql-stats-forecasts-min-observations" class="anchored"><code>sql.stats.forecasts.min_observations</code></div></td><td>integer</td><td><code>3</code></td><td>the mimimum number of observed statistics required to produce a statistics forecast</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-sql-stats-histogram-buckets-count" class="anchored"><code>sql.stats.histogram_buckets.count</code></div></td><td>integer</td><td><code>200</code></td><td>maximum number of histogram buckets to build during table statistics collection</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-sql-stats-histogram-collection-enabled" class="anchored"><code>sql.stats.histogram_collection.enabled</code></div></td><td>boolean</td><td><code>true</code></td><td>histogram collection mode</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-sql-stats-histogram-samples-count" class="anchored"><code>sql.stats.histogram_samples.count</code></div></td><td>integer</td><td><code>10000</code></td><td>number of rows sampled for histogram construction during table statistics collection</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-sql-stats-histogram-samples-count" class="anchored"><code>sql.stats.histogram_samples.count</code></div></td><td>integer</td><td><code>0</code></td><td>number of rows sampled for histogram construction during table statistics collection. Not setting this or setting a value of 0 means that a reasonable sample size will be automatically picked based on the table size.</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-sql-stats-multi-column-collection-enabled" class="anchored"><code>sql.stats.multi_column_collection.enabled</code></div></td><td>boolean</td><td><code>true</code></td><td>multi-column statistics collection mode</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-sql-stats-non-default-columns-min-retention-period" class="anchored"><code>sql.stats.non_default_columns.min_retention_period</code></div></td><td>duration</td><td><code>24h0m0s</code></td><td>minimum retention period for table statistics collected on non-default columns</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-sql-stats-persisted-rows-max" class="anchored"><code>sql.stats.persisted_rows.max</code></div></td><td>integer</td><td><code>1000000</code></td><td>maximum number of rows of statement and transaction statistics that will be persisted in the system tables before compaction begins</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -651,6 +651,7 @@ go_test(
"distsql_plan_bulk_test.go",
"distsql_plan_changefeed_test.go",
"distsql_plan_set_op_test.go",
"distsql_plan_stats_test.go",
"distsql_running_test.go",
"drop_function_test.go",
"drop_helpers_test.go",
Expand Down
89 changes: 68 additions & 21 deletions pkg/sql/distsql_plan_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -49,12 +49,15 @@ type requestedStat struct {

// histogramSamples is the number of sample rows to be collected for histogram
// construction. For larger tables, it may be beneficial to increase this number
// to get a more accurate distribution.
// to get a more accurate distribution. The default value is 0, which means that
// we will automatically pick a reasonable default based on the table size.
var histogramSamples = settings.RegisterIntSetting(
settings.ApplicationLevel,
"sql.stats.histogram_samples.count",
"number of rows sampled for histogram construction during table statistics collection",
10000,
"number of rows sampled for histogram construction during table statistics collection. "+
"Not setting this or setting a value of 0 means that a reasonable sample size will be "+
"automatically picked based on the table size.",
0,
settings.NonNegativeIntWithMaximum(math.MaxUint32),
settings.WithPublic)

Expand All @@ -71,6 +74,46 @@ var maxTimestampAge = settings.RegisterDurationSetting(
5*time.Minute,
)

// computeNumberSamples dynamically determines the number of samples to collect
// based on the estimated number of rows in the table. The formula 582n^0.29 is
// based on empirical data collected by running the sampler with different
// sample sizes on a variety of table sizes and observing the proportion of
// heavy hitters (most frequent elements) represented in the sample. It was
// derived by fitting a best-fit curve to the table below. The number of samples
// returned is bounded between 10,000 and 300,000.
// +---------------+-------------+
// | Table Size | Sample Size |
// +---------------+-------------+
// | 10,000 | 10,000 |
// | 100,000 | 15,000 |
// | 1,000,000 | 30,000 |
// | 10,000,000 | 60,000 |
// | 100,000,000 | 100,000 |
// | 1,000,000,000 | 300,000 |
// +---------------+-------------+
//
// The sample sizes above empirically achieved the following coverage:
// - 100k rows/15k samples: ~100% coverage of multiplicities down to 100x,
// ~80% down to 10x
// - 1m rows/30k samples: ~100% coverage of multiplicities down to 1000x, ~95%
// down to 100x
// - 10m rows/60k samples: ~100% coverage of multiplicities down to 10000x,
// ~95% down to 1000x, ~50% down to 100x
// - 100m rows/100k samples: ~100% coverage of multiplicities down to 10000x,
// ~65% down to 1000x, ~10% down to 100x
// - 1b rows/300k samples: ~100% coverage of multiplicities down to 100000x,
// ~95% down to 10000x, ~25% down to 1000x
func computeNumberSamples(numRows uint64) uint32 {
numSamples := math.Max(
math.Min(
582.0*math.Pow(float64(numRows), 0.29),
300000.0,
),
10000.0,
)
return uint32(numSamples)
}

func (dsp *DistSQLPlanner) createAndAttachSamplers(
ctx context.Context,
p *PhysicalPlan,
Expand All @@ -82,6 +125,21 @@ func (dsp *DistSQLPlanner) createAndAttachSamplers(
reqStats []requestedStat,
sketchSpec, invSketchSpec []execinfrapb.SketchSpec,
) *PhysicalPlan {
// Estimate the expected number of rows based on existing stats in the cache.
var rowsExpected uint64
if len(tableStats) > 0 {
overhead := stats.AutomaticStatisticsFractionStaleRows.Get(&dsp.st.SV)
if autoStatsFractionStaleRowsForTable, ok := desc.AutoStatsFractionStaleRows(); ok {
overhead = autoStatsFractionStaleRowsForTable
}
// Convert to a signed integer first to make the linter happy.
rowsExpected = uint64(int64(
// The total expected number of rows is the same number that was measured
// most recently, plus some overhead for possible insertions.
float64(tableStats[0].RowCount) * (1 + overhead),
))
}

// Set up the samplers.
sampler := &execinfrapb.SamplerSpec{
Sketches: sketchSpec,
Expand All @@ -92,11 +150,15 @@ func (dsp *DistSQLPlanner) createAndAttachSamplers(
// since we only support one reqStat at a time.
for _, s := range reqStats {
if s.histogram {
if count, ok := desc.HistogramSamplesCount(); ok {
sampler.SampleSize = count
var histogramSamplesCount uint32
if tableSampleCount, ok := desc.HistogramSamplesCount(); ok {
histogramSamplesCount = tableSampleCount
} else if clusterSampleCount := histogramSamples.Get(&dsp.st.SV); clusterSampleCount != histogramSamples.Default() {
histogramSamplesCount = uint32(clusterSampleCount)
} else {
sampler.SampleSize = uint32(histogramSamples.Get(&dsp.st.SV))
histogramSamplesCount = computeNumberSamples(rowsExpected)
}
sampler.SampleSize = histogramSamplesCount
// This could be anything >= 2 to produce a histogram, but the max number
// of buckets is probably also a reasonable minimum number of samples. (If
// there are fewer rows than this in the table, there will be fewer
Expand Down Expand Up @@ -134,21 +196,6 @@ func (dsp *DistSQLPlanner) createAndAttachSamplers(
execinfrapb.Ordering{},
)

// Estimate the expected number of rows based on existing stats in the cache.
var rowsExpected uint64
if len(tableStats) > 0 {
overhead := stats.AutomaticStatisticsFractionStaleRows.Get(&dsp.st.SV)
if autoStatsFractionStaleRowsForTable, ok := desc.AutoStatsFractionStaleRows(); ok {
overhead = autoStatsFractionStaleRowsForTable
}
// Convert to a signed integer first to make the linter happy.
rowsExpected = uint64(int64(
// The total expected number of rows is the same number that was measured
// most recently, plus some overhead for possible insertions.
float64(tableStats[0].RowCount) * (1 + overhead),
))
}

// Set up the final SampleAggregator stage.
agg := &execinfrapb.SampleAggregatorSpec{
Sketches: sketchSpec,
Expand Down
49 changes: 49 additions & 0 deletions pkg/sql/distsql_plan_stats_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,49 @@
// Copyright 2024 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 sql

import (
"math"
"testing"

"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
)

func TestComputeNumberSamples(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

testData := []struct {
numRows int
expectedNumSamples int
}{
{0, 10000},
{100, 10000},
{10000, 10000},
{100000, 16402},
{1000000, 31983},
{10000000, 62362},
{100000000, 121597},
{1000000000, 237095},
{10000000000, 300000},
{math.MaxInt, 300000},
}

checkComputeNumberSamples := func(computedNumSamples, expectedNumSamples int) {
if computedNumSamples != expectedNumSamples {
t.Fatalf("expected %d samples, got %d", expectedNumSamples, computedNumSamples)
}
}
for _, td := range testData {
checkComputeNumberSamples(int(computeNumberSamples(uint64(td.numRows))), td.expectedNumSamples)
}
}
90 changes: 87 additions & 3 deletions pkg/sql/logictest/testdata/logic_test/distsql_stats
Original file line number Diff line number Diff line change
Expand Up @@ -161,18 +161,102 @@ upper_bound range_rows distinct_range_rows equal_rows
statement ok
ALTER TABLE data RESET (sql_stats_histogram_buckets_count)

# Verify that we dynamically determine the number of samples if
# unspecified by table and cluster settings.

statement ok
RESET CLUSTER SETTING sql.stats.histogram_samples.count

# We can verify the number of samples collected based on the number of
# buckets produced.
statement ok
SET CLUSTER SETTING sql.stats.histogram_buckets.count = 20000

statement ok
SET CLUSTER SETTING sql.stats.histogram_samples.count = 20000

statement ok
CREATE TABLE big (i INT PRIMARY KEY);
INSERT INTO big SELECT generate_series(1, 20000)

statement ok
CREATE STATISTICS s_dynamic FROM big

let $hist_id_dynamic
SELECT histogram_id FROM [SHOW STATISTICS FOR TABLE big] WHERE statistics_name = 's_dynamic';

# We expect 10000 samples because there are no previous stats collections
# with table size estimates.
# Perform integer division by 10 because there may be 2 extra buckets added
# on either end of the histogram to account for the 20000 distinct values.
query I
SELECT (count(*) // 10) * 10 FROM [SHOW HISTOGRAM $hist_id_dynamic]
----
10000

statement ok
CREATE STATISTICS s_dynamic FROM big

let $hist_id_dynamic
SELECT histogram_id FROM [SHOW STATISTICS FOR TABLE big] WHERE statistics_name = 's_dynamic';

# Now we can dynamically determine the number of samples because we have
# a table size estimate from the previous stats collection.
# Perform integer division by 10 because there may be 2 extra buckets added
# on either end of the histogram to account for the 20000 distinct values.
query I
SELECT (count(*) // 10) * 10 FROM [SHOW HISTOGRAM $hist_id_dynamic]
----
10840

# Verify that the row count estimate from injected stats is used to determine
# the number of samples collected.
statement ok
ALTER TABLE big INJECT STATISTICS '[
{
"columns": [
"i"
],
"created_at": "1988-08-05 00:00:00",
"name": "injected_stats",
"row_count": 100000
}
]'

statement ok
CREATE STATISTICS s_injected FROM big

let $hist_id_injected
SELECT histogram_id FROM [SHOW STATISTICS FOR TABLE big] WHERE statistics_name = 's_injected';

query I
SELECT (count(*) // 10) * 10 FROM [SHOW HISTOGRAM $hist_id_injected]
----
17290

statement ok
ALTER TABLE big INJECT STATISTICS '[
{
"columns": [
"i"
],
"created_at": "2024-06-10 00:00:00",
"name": "injected_stats",
"row_count": 1000000000
}
]'

statement ok
CREATE STATISTICS s_injected FROM big

let $hist_id_injected
SELECT histogram_id FROM [SHOW STATISTICS FOR TABLE big] WHERE statistics_name = 's_injected';

query I
SELECT (count(*) // 10) * 10 FROM [SHOW HISTOGRAM $hist_id_injected]
----
20000

statement ok
SET CLUSTER SETTING sql.stats.histogram_samples.count = 20000

statement ok
CREATE STATISTICS s20000 FROM big

Expand Down

0 comments on commit d65a04b

Please sign in to comment.