Skip to content

Commit dae24c3

Browse files
craig[bot]Uzair5162annaw2193yuzefovichDrewKimball
committed
155242: sql/stats: avoid mutating input buckets in stripOuterBuckets r=Uzair5162 a=Uzair5162 This commit changes `stripOuterBuckets` to modify and return a copy of the given histogram buckets if it finds outer buckets to remove. Previously, we would mutate the caller's histograms with leading outer buckets in-place by zeroing the range counts on the first non-outer bucket. This effectively corrupts the first histogram bucket in stats passed in from the stats cache. Although this bug has existed since 90e311d (which zeroes the first buckets range counts), it would only impact full statistics that we tried merging with partial stats, as that was the only case in which `stripOuterBuckets` was called. The surface area of this bug increased after db9a344, which calls `stripOuterBuckets` on every full statistic, regardless of whether we end up merging it with partial stats or not. Fixes: #155184 Release note (bug fix): Previously, we could corrupt the first bucket of table statistic histograms in certain cases, causing underestimates for range counts near the lower end of the domain, which is now fixed. 155390: dev: refine behavior of `--cpus` further r=annaw2193 a=annaw2193 - `--local_cpu_resources` changed to `--local_resources=cpu=` - `--cpus` now additionaly implies `--local_test_jobs`, which should be set to an equivalent value. Also, using `--cpus` gives: Epic: None Fixes: #151139 Release Note: None 155402: sql: harden recent change about DO block recursion r=yuzefovich a=yuzefovich This commit reduces the max depth of recursion when evaluating routines with `tail-call-optimization-enabled=false` (recently added in 7b879ef) from 10k to 100. We've just seen a few cases where TestRandomSyntaxSQLSmith failed because DO block didn't respect context cancellation within 5s. I've manually tried it out a few times, and things worked, so my hypothesis is that extremely deep stacks (that are produced with the TCO disabled) is the root cause for slow cancellation, so let's just error out sooner. This commit also brings back the skip of DO blocks in TestComposeCompare (thinking there is that we should stabilize them in other tests first). Fixes: #155208. Fixes: #155210. Release note: None 155406: sql/hints: remove noisy logging r=DrewKimball a=DrewKimball This commit removes a noisy source of log messages by only logging for incremental updates with a non-empty list of events. This prevents spamming log messages every time the resolved timestamp for the rangefeed is incremented. Epic: None Release note: None Co-authored-by: Uzair Ahmad <uzair.ahmad@cockroachlabs.com> Co-authored-by: Anna Wang <anna.wang@cockroachlabs.com> Co-authored-by: Yahor Yuzefovich <yahor@cockroachlabs.com> Co-authored-by: Drew Kimball <drewk@cockroachlabs.com>
5 parents d0b9c08 + 86885b5 + 5cda02b + 07e5902 + e8c3b29 commit dae24c3

File tree

11 files changed

+184
-21
lines changed

11 files changed

+184
-21
lines changed

dev

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -8,7 +8,7 @@ fi
88
set -euo pipefail
99

1010
# Bump this counter to force rebuilding `dev` on all machines.
11-
DEV_VERSION=113
11+
DEV_VERSION=114
1212

1313
THIS_DIR=$(cd "$(dirname "$0")" && pwd)
1414
BINARY_DIR=$THIS_DIR/bin/dev-versions

pkg/cmd/dev/testdata/datadriven/bench

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -26,7 +26,7 @@ exec
2626
dev bench pkg/spanconfig/spanconfigkvsubscriber -f=BenchmarkSpanConfigDecoder --cpus=10 --ignore-cache=false -v --timeout=50s
2727
----
2828
echo $HOME/.cache
29-
bazel test --local_cpu_resources=10 --jobs=10 --test_timeout=50 pkg/spanconfig/spanconfigkvsubscriber:all --test_arg -test.run=- --test_arg -test.bench=BenchmarkSpanConfigDecoder --test_sharding_strategy=disabled --test_arg -test.cpu --test_arg 1 --test_arg -test.v --test_arg -test.benchmem --crdb_test_off --crdb_bench --test_env COCKROACH_TEST_FIXTURES_DIR=crdb-mock-test-fixtures/crdb-test-fixtures --sandbox_writable_path=crdb-mock-test-fixtures/crdb-test-fixtures --test_output streamed
29+
bazel test --local_resources=cpu=10 --jobs=10 --local_test_jobs=10 --test_timeout=50 pkg/spanconfig/spanconfigkvsubscriber:all --test_arg -test.run=- --test_arg -test.bench=BenchmarkSpanConfigDecoder --test_sharding_strategy=disabled --test_arg -test.cpu --test_arg 1 --test_arg -test.v --test_arg -test.benchmem --crdb_test_off --crdb_bench --test_env COCKROACH_TEST_FIXTURES_DIR=crdb-mock-test-fixtures/crdb-test-fixtures --sandbox_writable_path=crdb-mock-test-fixtures/crdb-test-fixtures --test_output streamed
3030

3131
exec
3232
dev bench pkg/bench -f='BenchmarkTracing/1node/scan/trace=off' --test-args '-test.memprofile=mem.out -test.cpuprofile=cpu.out'

pkg/cmd/dev/testdata/datadriven/dev-build

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -13,7 +13,7 @@ cp sandbox/pkg/cmd/cockroach-short/cockroach-short_/cockroach-short crdb-checkou
1313
exec
1414
dev build cockroach-short --cpus=12
1515
----
16-
bazel build --local_cpu_resources=12 --jobs=12 //pkg/cmd/cockroach-short:cockroach-short --build_event_binary_file=/tmp/path
16+
bazel build --local_resources=cpu=12 --jobs=12 --local_test_jobs=12 //pkg/cmd/cockroach-short:cockroach-short --build_event_binary_file=/tmp/path
1717
bazel info workspace --color=no
1818
mkdir crdb-checkout/bin
1919
bazel info bazel-bin --color=no

pkg/cmd/dev/testdata/datadriven/testlogic

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -59,7 +59,7 @@ bazel info workspace --color=no
5959
bazel info workspace --color=no
6060
bazel run pkg/cmd/generate-logictest -- -out-dir=crdb-checkout
6161
bazel run //pkg/gen:schemachanger
62-
bazel test //pkg/sql/logictest/tests/... --test_env=GOTRACEBACK=all --local_cpu_resources=8 --jobs=8 --test_arg -show-sql --test_timeout=60 --test_env=COCKROACH_STRESS=true --notest_keep_going --runs_per_test=500 --test_filter auto_span_config_reconciliation/ --test_sharding_strategy=disabled --test_output errors
62+
bazel test //pkg/sql/logictest/tests/... --test_env=GOTRACEBACK=all --local_resources=cpu=8 --jobs=8 --local_test_jobs=8 --test_arg -show-sql --test_timeout=60 --test_env=COCKROACH_STRESS=true --notest_keep_going --runs_per_test=500 --test_filter auto_span_config_reconciliation/ --test_sharding_strategy=disabled --test_output errors
6363

6464
exec
6565
dev testlogic base --files=auto_span_config_reconciliation --stress

pkg/cmd/dev/util.go

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -264,8 +264,9 @@ func (d *dev) getMergeBaseHash(ctx context.Context) (string, error) {
264264

265265
func addCommonBazelArguments(args *[]string) {
266266
if numCPUs != 0 {
267-
*args = append(*args, fmt.Sprintf("--local_cpu_resources=%d", numCPUs))
267+
*args = append(*args, fmt.Sprintf("--local_resources=cpu=%d", numCPUs))
268268
*args = append(*args, fmt.Sprintf("--jobs=%d", numCPUs))
269+
*args = append(*args, fmt.Sprintf("--local_test_jobs=%d", numCPUs))
269270
}
270271
if pgoEnabled {
271272
*args = append(*args, "--config=pgo")

pkg/compose/compare/compare/compare_test.go

Lines changed: 7 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -79,8 +79,13 @@ func TestCompare(t *testing.T) {
7979
}
8080
configs := map[string]testConfig{
8181
"mutators": {
82-
setup: sqlsmith.Setups[sqlsmith.RandTableSetupName],
83-
opts: []sqlsmith.SmitherOption{sqlsmith.CompareMode()},
82+
setup: sqlsmith.Setups[sqlsmith.RandTableSetupName],
83+
opts: []sqlsmith.SmitherOption{
84+
sqlsmith.CompareMode(),
85+
// TODO(yuzefovich): perhaps allow DO blocks again after they
86+
// have been stabilized in other tests.
87+
sqlsmith.DisableDoBlocks(),
88+
},
8489
ignoreSQLErrors: true,
8590
conns: []testConn{
8691
{

pkg/sql/hints/hint_cache.go

Lines changed: 2 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -210,7 +210,8 @@ func (c *StatementHintsCache) onUpdate(
210210
if update.Type == rangefeedcache.CompleteUpdate {
211211
log.Dev.Info(ctx, "statement_hints rangefeed completed initial scan")
212212
c.handleInitialScan(update)
213-
} else {
213+
} else if len(update.Events) > 0 {
214+
// Ignore empty updates that only bump the resolved timestamp.
214215
log.Dev.Info(ctx, "statement_hints rangefeed applying incremental update")
215216
c.handleIncrementalUpdate(ctx, update)
216217
}
@@ -248,10 +249,6 @@ func (c *StatementHintsCache) handleInitialScan(update rangefeedcache.Update[*bu
248249
func (c *StatementHintsCache) handleIncrementalUpdate(
249250
ctx context.Context, update rangefeedcache.Update[*bufferEvent],
250251
) {
251-
if len(update.Events) == 0 {
252-
// Avoid synchronization when we're just bumping the resolved timestamp.
253-
return
254-
}
255252
defer c.generation.Add(1)
256253
c.mu.Lock()
257254
defer c.mu.Unlock()

pkg/sql/logictest/testdata/logic_test/distsql_stats

Lines changed: 61 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -3890,3 +3890,64 @@ SELECT jsonb_pretty(statistics->0->'histo_buckets') FROM
38903890
"upper_bound": "c"
38913891
}
38923892
]
3893+
3894+
# Ensure that stripOuterBuckets doesn't overwrite statistics (see #155184).
3895+
3896+
statement ok
3897+
CREATE TABLE t155184 (
3898+
a INT PRIMARY KEY
3899+
) WITH (sql_stats_automatic_collection_enabled = false, sql_stats_histogram_samples_count = 2)
3900+
3901+
# These stats were created with the following statements:
3902+
#
3903+
# INSERT INTO t155184 SELECT generate_series(1,10)
3904+
# ANALYZE t155184
3905+
3906+
statement ok
3907+
ALTER TABLE t155184 INJECT STATISTICS '[
3908+
{
3909+
"avg_size": 1,
3910+
"columns": [
3911+
"a"
3912+
],
3913+
"created_at": "2025-10-10 13:41:08.711908",
3914+
"distinct_count": 10,
3915+
"histo_buckets": [
3916+
{
3917+
"distinct_range": 0,
3918+
"num_eq": 0,
3919+
"num_range": 0,
3920+
"upper_bound": "-9223372036854775808"
3921+
},
3922+
{
3923+
"distinct_range": 3.5,
3924+
"num_eq": 1,
3925+
"num_range": 4,
3926+
"upper_bound": "8"
3927+
},
3928+
{
3929+
"distinct_range": 1,
3930+
"num_eq": 1,
3931+
"num_range": 1,
3932+
"upper_bound": "10"
3933+
},
3934+
{
3935+
"distinct_range": 3.5,
3936+
"num_eq": 0,
3937+
"num_range": 4,
3938+
"upper_bound": "9223372036854775807"
3939+
}
3940+
],
3941+
"histo_col_type": "INT8",
3942+
"histo_version": 3,
3943+
"id": 1114221014922133505,
3944+
"null_count": 0,
3945+
"row_count": 10
3946+
}
3947+
]'
3948+
3949+
# All we care about is the row counts, so don't error if other parts of the explain output change
3950+
query T
3951+
SELECT info FROM [EXPLAIN SELECT * FROM t155184 WHERE a < 8] WHERE info LIKE '%estimated row count:%'
3952+
----
3953+
estimated row count: 4 (36% of the table; stats collected <hidden> ago)

pkg/sql/routine.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -163,7 +163,7 @@ func (p *planner) EvalRoutineExpr(
163163
if routineDepthValue := ctx.Value(routineDepthKey{}); routineDepthValue != nil {
164164
routineDepth = routineDepthValue.(int)
165165
}
166-
const maxDepth = 10000
166+
const maxDepth = 100
167167
if routineDepth > maxDepth {
168168
return nil, pgerror.Newf(pgcode.ProgramLimitExceeded,
169169
"routine reached recursion depth limit: %d (probably infinite loop)", maxDepth)

pkg/sql/stats/merge.go

Lines changed: 20 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -86,8 +86,9 @@ func MergedStatistics(
8686
return mergedStats
8787
}
8888

89-
// stripOuterBuckets removes the outer buckets from a histogram without a
90-
// leading NULL bucket.
89+
// stripOuterBuckets returns a copy of the histogram buckets with any outer
90+
// buckets that may have been added by addOuterBuckets removed. histogram must
91+
// not have a leading NULL bucket.
9192
func stripOuterBuckets(
9293
ctx context.Context, evalCtx *eval.Context, histogram []cat.HistogramBucket,
9394
) []cat.HistogramBucket {
@@ -96,17 +97,28 @@ func stripOuterBuckets(
9697
}
9798
startIdx := 0
9899
endIdx := len(histogram)
99-
if histogram[0].UpperBound.IsMin(ctx, evalCtx) && histogram[0].NumEq == 0 {
100+
hasLowerOuter := histogram[0].UpperBound.IsMin(ctx, evalCtx) && histogram[0].NumEq == 0
101+
if hasLowerOuter {
100102
startIdx = 1
101-
// Set the first range counts to zero to counteract range counts added by
102-
// addOuterBuckets.
103-
histogram[startIdx].NumRange = 0
104-
histogram[startIdx].DistinctRange = 0
105103
}
106104
if histogram[len(histogram)-1].UpperBound.IsMax(ctx, evalCtx) && histogram[len(histogram)-1].NumEq == 0 {
107105
endIdx = len(histogram) - 1
108106
}
109-
return histogram[startIdx:endIdx]
107+
if startIdx == 0 && endIdx == len(histogram) {
108+
return histogram
109+
}
110+
if startIdx >= endIdx {
111+
return nil
112+
}
113+
114+
out := append([]cat.HistogramBucket(nil), histogram[startIdx:endIdx]...)
115+
if hasLowerOuter {
116+
// Set the first range counts to zero to counteract range counts added by
117+
// addOuterBuckets.
118+
out[0].NumRange = 0
119+
out[0].DistinctRange = 0
120+
}
121+
return out
110122
}
111123

112124
// mergePartialStatistic merges a full statistic with a more recent partial

0 commit comments

Comments
 (0)