Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
124573: release: automatically detect if version is latest r=celiala a=rail

Previously, we relied on manually set variable to distinguish latest versions. This has been error-prone.

This PR uses the releases DB to detect if the current version is latest.

Epic: none
Release note: None

124603: sql/stats: evict stats cache entry if user-defined types have changed r=DrewKimball,yuzefovich a=michae2

**sql: add defensive type check to sql.(\*optTableStat).init**

In statistics builder we assume that the datums decoded from histogram upper bounds are comparable with datums in spans and constraints. If the histogram column type were ever different from the table column type, however, this assumption would not hold.

This should never happen, because histograms are associated with a column ID, and ALTER TABLE should never re-use a column ID during ALTER TABLE ADD COLUMN or ALTER TABLE ALTER COLUMN TYPE. But just to be defensive, add a check to sql.(\*optTableStat).init that skips over the TableStatistic if the histogram column type isn't equivalent to the table column type.

Also add this same typecheck to ALTER TABLE INJECT STATISTICS to guard against injecting histograms that don't match the column type. As part of this fix I'm removing the testcase added in #46552 which deliberately injects statistics with a histogram of a different type.

Informs: #124181

Release note: None

---

**sql/stats: evict stats cache entry if user-defined types have changed**

When adding table statistics to the stats cache, we decode histogram upper bounds into datums. If the histogram column uses a user-defined type, we hydrate the type and use this to decode.

In statistics builder, these histogram upper bound datums are compared against datums in spans and constraints. The comparisons assume that the datums are of equivalent type, but if the user-defined type has changed sometime after loading the stats cache entry, this might not be true.

If the user-defined type has changed, we need to evict and re-load the stats cache entry so that we decode histogram datums with a freshly-hydrated type.

(We were already checking UDT versions when building the optTable in sql.(\*optCatalog).dataSourceForTable, but the newly-built optTable used the existing table statistics instead of refreshing those, too.)

Fixes: #124181

Release note (bug fix): Fix a bug where a change to a user-defined type could cause queries against tables using that type to fail with an error message like:

```
histogram.go:694: span must be fully contained in the bucket
```

The change to the user-defined type could come directly from an ALTER TYPE statement, or indirectly from an ALTER DATABASE ADD REGION or DROP REGION statement (which implicitly change the crdb_internal_region type).

This bug has existed since UDTs were introduced in v20.2.

---

**sem/tree: check oid and version in tree.(\*DEnum).CompareError**

Make sure when we're comparing two enum datums that they are, in fact, the same enum type.

Informs: #124181

Release note: None

124776: schemachanger_test: increase shard count r=rafiss a=rafiss

fixes #124714
Release note: None

Co-authored-by: Rail Aliiev <rail@iqchoice.com>
Co-authored-by: Michael Erickson <michae2@cockroachlabs.com>
Co-authored-by: Rafi Shamim <rafi@cockroachlabs.com>
  • Loading branch information
4 people committed May 29, 2024
4 parents 80d4f7b + 273b84f + b6aa169 + c6abf61 commit 76c744a
Show file tree
Hide file tree
Showing 23 changed files with 368 additions and 51 deletions.
6 changes: 6 additions & 0 deletions build/release/teamcity-publish-redhat-release.sh
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,12 @@ if [[ -z "$build_name" ]] ; then
echo "Unsupported version \"${NAME}\". Must be of the format \"vMAJOR.MINOR.PATCH\"."
exit 0
fi

PUBLISH_LATEST=
if is_latest "$build_name"; then
PUBLISH_LATEST=true
fi

# Hard coded release number used only by the RedHat images
rhel_release=1
rhel_project_id=5e61ea74fe2231a0c2860382
Expand Down
12 changes: 12 additions & 0 deletions build/release/teamcity-support.sh
Original file line number Diff line number Diff line change
Expand Up @@ -116,3 +116,15 @@ function is_release_or_master_build(){
# ^ calver prefix, e.g. 25.1
# We don't strictly match the suffix to allow different ones, e.g. "rc" or have none.
}

# Compare the passed version to the latest published version. Returns 0 if the
# passed version is the latest. Supports stable versions only.
function is_latest() {
version=$1
url="https://get.cockroachdb.com/api/is_latest?version=$version"
maybe_latest="$(curl -fsSL "$url" || echo "")"
if [[ $maybe_latest == "yes" ]]; then
return 0
fi
return 1
}
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,11 @@ if [[ $version == *"-"* ]]; then
exit 0
fi

PUBLISH_LATEST=
if is_latest "$version"; then
PUBLISH_LATEST=true
fi

# Hard coded release number used only by the RedHat images
rhel_release=1
rhel_project_id=5e61ea74fe2231a0c2860382
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,11 @@ if ! echo "${version}" | grep -E -o '^v(0|[1-9][0-9]*)\.(0|[1-9][0-9]*)\.(0|[1-9
exit 1
fi

PUBLISH_LATEST=
if is_latest "$version"; then
PUBLISH_LATEST=true
fi

release_branch=$(echo "${version}" | grep -E -o '^v[0-9]+\.[0-9]+')

if [[ -z "${DRY_RUN}" ]] ; then
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,11 @@ if [[ -z "$build_name" ]] ; then
exit 1
fi

PUBLISH_LATEST=
if is_latest "$build_name"; then
PUBLISH_LATEST=true
fi

release_branch=$(echo ${build_name} | grep -E -o '^v[0-9]+\.[0-9]+')

if [[ -z "${DRY_RUN}" ]] ; then
Expand Down
127 changes: 127 additions & 0 deletions pkg/ccl/logictestccl/testdata/logic_test/multi_region_stats
Original file line number Diff line number Diff line change
@@ -0,0 +1,127 @@
# LogicTest: multiregion-9node-3region-3azs multiregion-9node-3region-3azs-tenant multiregion-9node-3region-3azs-no-los

query TTTTT colnames,rowsort
SHOW REGIONS
----
region zones database_names primary_region_of secondary_region_of
ap-southeast-2 {ap-az1,ap-az2,ap-az3} {} {} {}
ca-central-1 {ca-az1,ca-az2,ca-az3} {} {} {}
us-east-1 {us-az1,us-az2,us-az3} {} {} {}

query TT colnames,rowsort
SHOW REGIONS FROM CLUSTER
----
region zones
ap-southeast-2 {ap-az1,ap-az2,ap-az3}
ca-central-1 {ca-az1,ca-az2,ca-az3}
us-east-1 {us-az1,us-az2,us-az3}

# Regression test for #124181: check that we re-load table statistics after
# running ALTER DATABASE ADD REGION.

statement ok
CREATE DATABASE db124181 PRIMARY REGION "ap-southeast-2" REGIONS "us-east-1" SURVIVE ZONE FAILURE

statement ok
USE db124181

query TTTT
SHOW ENUMS
----
public crdb_internal_region {ap-southeast-2,us-east-1} root

statement ok
CREATE TABLE t124181 (
region crdb_internal_region NOT NULL,
id UUID NOT NULL DEFAULT gen_random_uuid(),
a INT NOT NULL,
PRIMARY KEY (id),
UNIQUE INDEX (a)
) LOCALITY REGIONAL BY ROW AS region

statement ok
INSERT INTO t124181 (region, a) VALUES ('ap-southeast-2', 0), ('us-east-1', 1)

statement ok
ANALYZE t124181

let $hist_id_1
SELECT histogram_id FROM [SHOW STATISTICS FOR TABLE t124181] WHERE column_names = ARRAY['region']

query TIRI colnames,nosort
SHOW HISTOGRAM $hist_id_1
----
upper_bound range_rows distinct_range_rows equal_rows
'ap-southeast-2' 0 0 1
'us-east-1' 0 0 1

query T
SELECT jsonb_pretty(stat->'histo_buckets')
FROM (
SELECT jsonb_array_elements(statistics) AS stat
FROM [SHOW STATISTICS USING JSON FOR TABLE t124181]
)
WHERE stat->>'columns' = '["region"]'
----
[
{
"distinct_range": 0,
"num_eq": 1,
"num_range": 0,
"upper_bound": "ap-southeast-2"
},
{
"distinct_range": 0,
"num_eq": 1,
"num_range": 0,
"upper_bound": "us-east-1"
}
]

# Implicitly add a value to the crdb_internal_region enum.
statement ok
ALTER DATABASE db124181 ADD REGION "ca-central-1"

query TTTT
SHOW ENUMS
----
public crdb_internal_region {ap-southeast-2,ca-central-1,us-east-1} root

# Make sure we can still SHOW STATISTICS and SHOW HISTOGRAM.
let $hist_id_2
SELECT histogram_id FROM [SHOW STATISTICS FOR TABLE t124181] WHERE column_names = ARRAY['region']

query TIRI colnames,nosort
SHOW HISTOGRAM $hist_id_2
----
upper_bound range_rows distinct_range_rows equal_rows
'ap-southeast-2' 0 0 1
'us-east-1' 0 0 1

# Make sure we can still SHOW STATISTICS USING JSON.
query T
SELECT jsonb_pretty(stat->'histo_buckets')
FROM (
SELECT jsonb_array_elements(statistics) AS stat
FROM [SHOW STATISTICS USING JSON FOR TABLE t124181]
)
WHERE stat->>'columns' = '["region"]'
----
[
{
"distinct_range": 0,
"num_eq": 1,
"num_range": 0,
"upper_bound": "ap-southeast-2"
},
{
"distinct_range": 0,
"num_eq": 1,
"num_range": 0,
"upper_bound": "us-east-1"
}
]

# Make sure we can still use the histogram in statistics_builder.
statement ok
INSERT INTO t124181 (region, a) VALUES ('ca-central-1', 2)
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,7 @@ go_test(
"//pkg/ccl/logictestccl:testdata", # keep
],
exec_properties = {"test.Pool": "large"},
shard_count = 20,
shard_count = 21,
tags = ["cpu:4"],
deps = [
"//pkg/base",
Expand Down

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,7 @@ go_test(
"//pkg/ccl/logictestccl:testdata", # keep
],
exec_properties = {"test.Pool": "large"},
shard_count = 16,
shard_count = 17,
tags = ["cpu:4"],
deps = [
"//pkg/base",
Expand Down

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,7 @@ go_test(
"//pkg/ccl/logictestccl:testdata", # keep
],
exec_properties = {"test.Pool": "large"},
shard_count = 28,
shard_count = 29,
tags = ["cpu:4"],
deps = [
"//pkg/base",
Expand Down

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

19 changes: 18 additions & 1 deletion pkg/sql/alter_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -1469,7 +1469,8 @@ func injectTableStats(
}
}

// First, delete all statistics for the table.
// First, delete all statistics for the table. (We use the current transaction
// so that this will rollback on any error.)
if _ /* rows */, err := params.p.InternalSQLTxn().Exec(
params.ctx,
"delete-stats",
Expand All @@ -1487,6 +1488,22 @@ StatsLoop:
if err != nil {
return err
}

// Check that the type matches.
// TODO(49698): When we support multi-column histograms this check will need
// adjustment.
if len(s.Columns) == 1 {
col := catalog.FindColumnByName(desc, s.Columns[0])
// Ignore dropped columns (they are handled below).
if col != nil {
if err := h.TypeCheck(
col.GetType(), desc.GetName(), s.Columns[0], s.CreatedAt,
); err != nil {
return pgerror.WithCandidateCode(err, pgcode.DatatypeMismatch)
}
}
}

// histogram will be passed to the INSERT statement; we want it to be a
// nil interface{} if we don't generate a histogram.
var histogram interface{}
Expand Down
8 changes: 4 additions & 4 deletions pkg/sql/distsql_plan_changefeed.go
Original file line number Diff line number Diff line change
Expand Up @@ -381,7 +381,7 @@ func (c *cdcOptCatalog) ResolveDataSource(
return nil, cat.DataSourceName{}, err
}

ds, err := c.newCDCDataSource(desc, c.targetFamilyID)
ds, err := c.newCDCDataSource(ctx, desc, c.targetFamilyID)
if err != nil {
return nil, cat.DataSourceName{}, err
}
Expand All @@ -399,7 +399,7 @@ func (c *cdcOptCatalog) ResolveDataSourceByID(
return nil, false, err
}

ds, err := c.newCDCDataSource(desc, c.targetFamilyID)
ds, err := c.newCDCDataSource(ctx, desc, c.targetFamilyID)
if err != nil {
return nil, false, err
}
Expand All @@ -423,13 +423,13 @@ func (c *cdcOptCatalog) ResolveFunction(

// newCDCDataSource builds an optTable for the target cdc table and family.
func (c *cdcOptCatalog) newCDCDataSource(
original catalog.TableDescriptor, familyID catid.FamilyID,
ctx context.Context, original catalog.TableDescriptor, familyID catid.FamilyID,
) (cat.DataSource, error) {
d, err := newFamilyTableDescriptor(original, familyID, c.extraColumns)
if err != nil {
return nil, err
}
return newOptTable(d, c.codec(), nil /* stats */, emptyZoneConfig)
return newOptTable(ctx, d, c.codec(), nil /* stats */, emptyZoneConfig)
}

// familyTableDescriptor wraps underlying catalog.TableDescriptor,
Expand Down
18 changes: 8 additions & 10 deletions pkg/sql/logictest/testdata/logic_test/prepare
Original file line number Diff line number Diff line change
Expand Up @@ -1238,11 +1238,15 @@ EXECUTE rcc('t')

user root

# Regression test for #46217. Histogram type doesn't match column type.
statement ok
CREATE TABLE ts (d DATE PRIMARY KEY, x INT);

user root

# Regression test for #46217. Test that we cannot inject a histogram type that
# doesn't match the column type.
statement ok
CREATE TABLE ts (d DATE PRIMARY KEY, x INT)

statement error pq: histogram for table ts column d created_at 2020-03-24 15:34:22\.863634\+00:00 does not match column type DATE: TIMESTAMP
ALTER TABLE ts INJECT STATISTICS '[
{
"columns": ["d"],
Expand All @@ -1267,13 +1271,7 @@ ALTER TABLE ts INJECT STATISTICS '[
"null_count": 0,
"row_count": 100000
}
]';

statement ok
PREPARE q AS DELETE FROM ts WHERE ts.d <= $1

statement ok
EXECUTE q ('2020-03-25')
]'

# Test that if we replace a view the cached plan is invalidated.
statement ok
Expand Down
Loading

0 comments on commit 76c744a

Please sign in to comment.