Skip to content

Commit d82a204

Browse files
committed
sql/stats: handle range counts when skipping dropped enum hist buckets
In #136538 we changed DecodeHistogramBuckets to skip over histogram buckets for dropped enum values. This change assumed that histogram buckets for enum types would never contain range counts, but this is not true. For example, if the number of distinct enum values exceeds the number of histogram buckets there could be range counts. Fixes: #154461 Release note (bug fix): Fix a bug in which range counts in table statistics histograms were not handled correctly after a user-defined enum type was modified.
1 parent 3205dfc commit d82a204

File tree

12 files changed

+708
-69
lines changed

12 files changed

+708
-69
lines changed

pkg/sql/logictest/testdata/logic_test/enums

Lines changed: 129 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1987,3 +1987,132 @@ CREATE TABLE public.t (
19871987
rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(),
19881988
CONSTRAINT t_pkey PRIMARY KEY (rowid ASC)
19891989
) WITH (schema_locked = true);
1990+
1991+
# Testcase for issue 154461: drop an enum value when the histogram contains
1992+
# range counts.
1993+
subtest 154461
1994+
1995+
statement ok
1996+
USE test
1997+
1998+
statement ok
1999+
CREATE TYPE e154461 AS ENUM ('e', 'f', 'g')
2000+
2001+
statement ok
2002+
CREATE TABLE t154461 (a e154461, INDEX (a)) WITH (sql_stats_histogram_buckets_count = 2)
2003+
2004+
statement ok
2005+
INSERT INTO t154461 VALUES ('e'), ('e'), ('f'), ('g'), ('g')
2006+
2007+
statement ok
2008+
CREATE STATISTICS s FROM t154461
2009+
2010+
query T
2011+
SELECT * FROM t154461 WHERE a != 'g' ORDER BY a
2012+
----
2013+
e
2014+
e
2015+
f
2016+
2017+
let $hist_id_1
2018+
SELECT histogram_id FROM [SHOW STATISTICS FOR TABLE t154461]
2019+
WHERE statistics_name = 's' AND column_names = '{a}'
2020+
2021+
query TIRI colnames,nosort
2022+
SHOW HISTOGRAM $hist_id_1
2023+
----
2024+
upper_bound range_rows distinct_range_rows equal_rows
2025+
'e' 0 0 2
2026+
'g' 1 1 2
2027+
2028+
query T
2029+
SELECT jsonb_pretty(stat)
2030+
FROM (
2031+
SELECT json_array_elements(statistics) - 'created_at' - 'id' - 'avg_size' AS stat
2032+
FROM [SHOW STATISTICS USING JSON FOR TABLE t154461]
2033+
)
2034+
WHERE stat->>'columns' = '["a"]'
2035+
----
2036+
{
2037+
"columns": [
2038+
"a"
2039+
],
2040+
"distinct_count": 3,
2041+
"histo_buckets": [
2042+
{
2043+
"distinct_range": 0,
2044+
"num_eq": 2,
2045+
"num_range": 0,
2046+
"upper_bound": "e"
2047+
},
2048+
{
2049+
"distinct_range": 1,
2050+
"num_eq": 2,
2051+
"num_range": 1,
2052+
"upper_bound": "g"
2053+
}
2054+
],
2055+
"histo_col_type": "test.public.e154461",
2056+
"histo_version": 3,
2057+
"name": "s",
2058+
"null_count": 0,
2059+
"row_count": 5
2060+
}
2061+
2062+
# Now drop the enum value that is the first bucket in the 2-bucket histogram.
2063+
2064+
statement ok
2065+
DELETE FROM t154461 WHERE a = 'e'
2066+
2067+
statement ok
2068+
ALTER TYPE e154461 DROP VALUE 'e'
2069+
2070+
query T
2071+
SELECT * FROM t154461 WHERE a != 'g' ORDER BY a
2072+
----
2073+
f
2074+
2075+
let $hist_id_1
2076+
SELECT histogram_id FROM [SHOW STATISTICS FOR TABLE t154461]
2077+
WHERE statistics_name = 's' AND column_names = '{a}'
2078+
2079+
query TIRI colnames,nosort
2080+
SHOW HISTOGRAM $hist_id_1
2081+
----
2082+
upper_bound range_rows distinct_range_rows equal_rows
2083+
'f' 0 0 1
2084+
'g' 0 0 2
2085+
2086+
query T
2087+
SELECT jsonb_pretty(stat)
2088+
FROM (
2089+
SELECT json_array_elements(statistics) - 'created_at' - 'id' - 'avg_size' AS stat
2090+
FROM [SHOW STATISTICS USING JSON FOR TABLE t154461]
2091+
)
2092+
WHERE stat->>'columns' = '["a"]'
2093+
----
2094+
{
2095+
"columns": [
2096+
"a"
2097+
],
2098+
"distinct_count": 2,
2099+
"histo_buckets": [
2100+
{
2101+
"distinct_range": 0,
2102+
"num_eq": 1,
2103+
"num_range": 0,
2104+
"upper_bound": "f"
2105+
},
2106+
{
2107+
"distinct_range": 0,
2108+
"num_eq": 2,
2109+
"num_range": 0,
2110+
"upper_bound": "g"
2111+
}
2112+
],
2113+
"histo_col_type": "test.public.e154461",
2114+
"histo_version": 3,
2115+
"name": "s",
2116+
"null_count": 0,
2117+
"row_count": 5
2118+
}

pkg/sql/opt/exec/execbuilder/testdata/forecast

Lines changed: 0 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -396,23 +396,11 @@ WHERE stat->>'name' = '__forecast__'
396396
"created_at": "1988-08-08 00:00:00",
397397
"distinct_count": 1,
398398
"histo_buckets": [
399-
{
400-
"distinct_range": 0,
401-
"num_eq": 0,
402-
"num_range": 0,
403-
"upper_bound": "-1"
404-
},
405399
{
406400
"distinct_range": 0,
407401
"num_eq": 1,
408402
"num_range": 0,
409403
"upper_bound": "0"
410-
},
411-
{
412-
"distinct_range": 0,
413-
"num_eq": 0,
414-
"num_range": 0,
415-
"upper_bound": "1"
416404
}
417405
],
418406
"histo_col_type": "INT8",

pkg/sql/opt/exec/execbuilder/testdata/partial_stats

Lines changed: 0 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -375,12 +375,6 @@ WHERE stat->>'name' = '__forecast__';
375375
"num_range": 0,
376376
"upper_bound": "9"
377377
},
378-
{
379-
"distinct_range": 0,
380-
"num_eq": 0,
381-
"num_range": 0,
382-
"upper_bound": "10"
383-
},
384378
{
385379
"distinct_range": 0,
386380
"num_eq": 2,
@@ -411,12 +405,6 @@ WHERE stat->>'name' = '__forecast__';
411405
"num_range": 0,
412406
"upper_bound": "15"
413407
},
414-
{
415-
"distinct_range": 0,
416-
"num_eq": 0,
417-
"num_range": 0,
418-
"upper_bound": "16"
419-
},
420408
{
421409
"distinct_range": 0,
422410
"num_eq": 1,

pkg/sql/randgen/mutator.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -226,7 +226,7 @@ func statisticsMutator(
226226
colType := tree.MustBeStaticallyKnownType(col.Type)
227227
h := randHistogram(rng, colType)
228228
statIdx := colNameToStatIdx[col.Name]
229-
if err := allStats[statIdx].SetHistogram(&h); err != nil {
229+
if err := allStats[statIdx].SetHistogram(context.Background(), &h); err != nil {
230230
panic(err)
231231
}
232232
}

pkg/sql/show_histogram.go

Lines changed: 8 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -90,20 +90,17 @@ func (p *planner) ShowHistogram(ctx context.Context, n *tree.ShowHistogram) (pla
9090
if err := typedesc.EnsureTypeIsHydrated(ctx, histogram.ColumnType, &resolver); err != nil {
9191
return nil, err
9292
}
93-
var a tree.DatumAlloc
94-
for _, b := range histogram.Buckets {
95-
var upperBound string
96-
datum, err := stats.DecodeUpperBound(histogram.Version, histogram.ColumnType, &a, b.UpperBound)
97-
if err != nil {
98-
upperBound = fmt.Sprintf("<error: %v>", err)
99-
} else {
100-
upperBound = datum.String()
101-
}
93+
decodedBuckets, _, err := histogram.DecodeBuckets(ctx)
94+
if err != nil {
95+
return nil, err
96+
}
97+
for _, b := range decodedBuckets {
98+
upperBound := b.UpperBound.String()
10299
row := tree.Datums{
103100
tree.NewDString(upperBound),
104-
tree.NewDInt(tree.DInt(b.NumRange)),
101+
tree.NewDInt(tree.DInt(int64(b.NumRange))),
105102
tree.NewDFloat(tree.DFloat(b.DistinctRange)),
106-
tree.NewDInt(tree.DInt(b.NumEq)),
103+
tree.NewDInt(tree.DInt(int64(b.NumEq))),
107104
}
108105
if _, err := v.rows.AddRow(ctx, row); err != nil {
109106
v.Close(ctx)

pkg/sql/show_stats.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -182,7 +182,7 @@ func (p *planner) ShowTableStats(ctx context.Context, n *tree.ShowTableStats) (p
182182
}
183183
obs := &stats.TableStatistic{TableStatisticProto: *stat}
184184
if obs.HistogramData != nil && !obs.HistogramData.ColumnType.UserDefined() {
185-
if err := stats.DecodeHistogramBuckets(obs); err != nil {
185+
if err := stats.DecodeHistogramBuckets(ctx, obs); err != nil {
186186
return nil, err
187187
}
188188
}

pkg/sql/stats/histogram.go

Lines changed: 31 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -140,6 +140,37 @@ func DecodeUpperBound(
140140
return datum, err
141141
}
142142

143+
// enumValueExistsBetweenEncodedUpperBounds finds whether any values exist in
144+
// the enum type with physical representation greater than lowerBound and less
145+
// than upperBound, exclusive. The encoded bounds could represent old values
146+
// that were dropped from the enum type, or current values that exist in the
147+
// enum type.
148+
//
149+
// If there are any values in the enum type with physical representation between
150+
// the bounds, nil is returned. Otherwise an error is returned. If the bounds
151+
// cannot be decoded as enum physical representations an error is returned.
152+
func enumValueExistsBetweenEncodedUpperBounds(
153+
version HistogramVersion, enumType *types.T, lowerBound, upperBound []byte,
154+
) error {
155+
// If the encoded bounds represent old values that were dropped from the enum
156+
// type, we won't be able to decode them as enums. Instead, decode them as
157+
// bytes, which should give the physical representation. (This relies on enum
158+
// encoding being simply the encoding of the physical representation bytes.)
159+
var a tree.DatumAlloc
160+
lowerPhys, err := DecodeUpperBound(version, types.Bytes, &a, lowerBound)
161+
if err != nil {
162+
return err
163+
}
164+
upperPhys, err := DecodeUpperBound(version, types.Bytes, &a, upperBound)
165+
if err != nil {
166+
return err
167+
}
168+
_, err = enumType.EnumGetFirstIdxOfPhysicalBetween(
169+
[]byte(*lowerPhys.(*tree.DBytes)), []byte(*upperPhys.(*tree.DBytes)),
170+
)
171+
return err
172+
}
173+
143174
// GetDefaultHistogramBuckets gets the default number of histogram buckets to
144175
// create for the given table.
145176
func GetDefaultHistogramBuckets(sv *settings.Values, desc catalog.TableDescriptor) uint32 {

pkg/sql/stats/histogram_test.go

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1265,6 +1265,7 @@ func TestUpperBoundsRoundTrip(t *testing.T) {
12651265
h.buckets = make([]cat.HistogramBucket, numBuckets)
12661266
for i := 0; i < numBuckets; i++ {
12671267
h.buckets[i].UpperBound = upperBounds[i]
1268+
h.buckets[i].NumEq = 1
12681269
}
12691270
hd, err := h.toHistogramData(context.Background(), typ, st)
12701271
if err != nil {
@@ -1274,7 +1275,7 @@ func TestUpperBoundsRoundTrip(t *testing.T) {
12741275
// original ones.
12751276
var stat TableStatistic
12761277
stat.HistogramData = &hd
1277-
if err = DecodeHistogramBuckets(&stat); err != nil {
1278+
if err = DecodeHistogramBuckets(context.Background(), &stat); err != nil {
12781279
t.Fatal(err)
12791280
}
12801281
evalCtx := eval.MakeTestingEvalContext(st)

pkg/sql/stats/json.go

Lines changed: 13 additions & 19 deletions
Original file line numberDiff line numberDiff line change
@@ -8,6 +8,7 @@ package stats
88
import (
99
"context"
1010
"fmt"
11+
"math"
1112

1213
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
1314
"github.com/cockroachdb/cockroach/pkg/sql/parser"
@@ -55,7 +56,7 @@ type JSONHistoBucket struct {
5556
}
5657

5758
// SetHistogram fills in the HistogramColumnType and HistogramBuckets fields.
58-
func (js *JSONStatistic) SetHistogram(h *HistogramData) error {
59+
func (js *JSONStatistic) SetHistogram(ctx context.Context, h *HistogramData) error {
5960
typ := h.ColumnType
6061
if typ == nil {
6162
return fmt.Errorf("histogram type is unset")
@@ -66,26 +67,19 @@ func (js *JSONStatistic) SetHistogram(h *HistogramData) error {
6667
js.HistogramColumnType = typ.SQLStringFullyQualified()
6768
js.HistogramBuckets = make([]JSONHistoBucket, 0, len(h.Buckets))
6869
js.HistogramVersion = h.Version
69-
var a tree.DatumAlloc
70-
for i := range h.Buckets {
71-
b := &h.Buckets[i]
72-
if b.UpperBound == nil {
73-
return fmt.Errorf("histogram bucket upper bound is unset")
74-
}
75-
datum, err := DecodeUpperBound(h.Version, typ, &a, b.UpperBound)
76-
if err != nil {
77-
if h.ColumnType.Family() == types.EnumFamily && errors.Is(err, types.EnumValueNotFound) {
78-
// Skip over buckets for enum values that were dropped.
79-
continue
80-
}
81-
return err
82-
}
8370

71+
decodedBuckets, distinctAdjustment, err := h.DecodeBuckets(ctx)
72+
if err != nil {
73+
return err
74+
}
75+
js.DistinctCount = uint64(math.Max(0, float64(js.DistinctCount)+distinctAdjustment))
76+
for i := range decodedBuckets {
77+
b := &decodedBuckets[i]
8478
js.HistogramBuckets = append(js.HistogramBuckets, JSONHistoBucket{
85-
NumEq: b.NumEq,
86-
NumRange: b.NumRange,
79+
NumEq: int64(b.NumEq),
80+
NumRange: int64(b.NumRange),
8781
DistinctRange: b.DistinctRange,
88-
UpperBound: tree.AsStringWithFlags(datum, tree.FmtExport|tree.FmtAlwaysQualifyUserDefinedTypeNames),
82+
UpperBound: tree.AsStringWithFlags(b.UpperBound, tree.FmtExport|tree.FmtAlwaysQualifyUserDefinedTypeNames),
8983
})
9084
}
9185
return nil
@@ -123,7 +117,7 @@ func (js *JSONStatistic) DecodeAndSetHistogram(
123117
}
124118
h.ColumnType = typ
125119
}
126-
return js.SetHistogram(h)
120+
return js.SetHistogram(ctx, h)
127121
}
128122

129123
// GetHistogram converts the json histogram into HistogramData.

0 commit comments

Comments
 (0)