Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
98463: sqlliveness: migrate to rbr compatible index r=JeffSwenson a=JeffSwenson

Migrate sqlliveness to the regional by row compatible index. The version
gates are intended to follow the protocol discussed in the comment at
the top of upgrades/system_rbr_indexes.go.

As part of this change, the legacy session id encoding is treated as
belonging to the enum.One region. Treating the legacy encoding as
belonging to enum.One simplifies the migration as it is not necessary to
wait for old sessions to expire or clear old sessions from the jobs
table.

Part of #94843

Release note: none

98500: sql: disallow creating partial stats by default r=mgartner a=mgartner

This commit adds the `enable_create_stats_using_extremes` session
setting which allows users to run `CREATE STATISTICS .. USING EXTREMES`
when enabled. It is disabled by default.

This commit also presents a "feature not supported" error when a user
tries to create partial statistics with a `WHERE` clause. This feature
has not yet been implemented.

Epic: CRDB-19449

Fixes #95233 

Release note: None


98525: build: update version.txt with the next release r=lidorcarmel a=lidorcarmel

releasing v23.1.0-alpha.6, updating version.txt to alpha.7.

Epic: none

Release note: None

Co-authored-by: Jeff <[email protected]>
Co-authored-by: Marcus Gartner <[email protected]>
Co-authored-by: Lidor Carmel <[email protected]>
  • Loading branch information
4 people committed Mar 14, 2023
4 parents 60e7c78 + 6cd6962 + 240ad78 + d236b84 commit 74f993a
Show file tree
Hide file tree
Showing 30 changed files with 406 additions and 182 deletions.
2 changes: 1 addition & 1 deletion pkg/build/version.txt
Original file line number Diff line number Diff line change
@@ -1 +1 @@
v23.1.0-alpha.6
v23.1.0-alpha.7
4 changes: 4 additions & 0 deletions pkg/ccl/backupccl/testdata/backup-restore/metadata
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,10 @@ exec-sql
SET CLUSTER SETTING sql.stats.automatic_collection.enabled = false;
----

exec-sql
SET enable_create_stats_using_extremes = true;
----

exec-sql
CREATE DATABASE db1;
USE db1;
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/multitenantccl/tenantcostclient/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,7 @@ go_test(
"//pkg/server",
"//pkg/settings/cluster",
"//pkg/sql",
"//pkg/sql/catalog/systemschema",
"//pkg/sql/distsql",
"//pkg/sql/execinfra",
"//pkg/sql/sqlliveness",
Expand Down
4 changes: 3 additions & 1 deletion pkg/ccl/multitenantccl/tenantcostclient/tenant_side_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/server"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema"
"github.com/cockroachdb/cockroach/pkg/sql/distsql"
"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
"github.com/cockroachdb/cockroach/pkg/sql/sqlliveness"
Expand Down Expand Up @@ -963,7 +964,8 @@ func TestSQLLivenessExemption(t *testing.T) {
_ = r

codec := keys.MakeSQLCodec(tenantID)
key := codec.IndexPrefix(keys.SqllivenessID, 1)
indexID := uint32(systemschema.SqllivenessTable().GetPrimaryIndexID())
key := codec.IndexPrefix(keys.SqllivenessID, indexID)

// livenessValue returns the KV value for the one row in the
// system.sqlliveness table. The value contains the session expiration time
Expand Down
2 changes: 1 addition & 1 deletion pkg/server/server_sql.go
Original file line number Diff line number Diff line change
Expand Up @@ -554,7 +554,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) {
}
cfg.sqlLivenessProvider = slprovider.New(
cfg.AmbientCtx,
cfg.stopper, cfg.clock, cfg.db, codec, cfg.Settings, sqllivenessKnobs, sessionEventsConsumer,
cfg.stopper, cfg.clock, cfg.db, codec, cfg.Settings, settingsWatcher, sqllivenessKnobs, sessionEventsConsumer,
)

cfg.sqlInstanceStorage = instancestorage.NewStorage(
Expand Down
5 changes: 5 additions & 0 deletions pkg/server/settingswatcher/version_guard.go
Original file line number Diff line number Diff line change
Expand Up @@ -87,3 +87,8 @@ func (s *SettingsWatcher) MakeVersionGuard(
func (v *VersionGuard) IsActive(version clusterversion.Key) bool {
return v.activeVersion.IsActive(version)
}

// TestMakeVersionGuard initializes a version guard at specific version.
func TestMakeVersionGuard(activeVersion clusterversion.ClusterVersion) VersionGuard {
return VersionGuard{activeVersion: activeVersion}
}
8 changes: 4 additions & 4 deletions pkg/sql/catalog/bootstrap/testdata/testdata

Large diffs are not rendered by default.

54 changes: 12 additions & 42 deletions pkg/sql/catalog/systemschema/system.go
Original file line number Diff line number Diff line change
Expand Up @@ -483,14 +483,6 @@ CREATE TABLE system.scheduled_jobs (
)`

SqllivenessTableSchema = `
CREATE TABLE system.sqlliveness (
session_id BYTES NOT NULL,
expiration DECIMAL NOT NULL,
CONSTRAINT "primary" PRIMARY KEY (session_id),
FAMILY fam0_session_id_expiration (session_id, expiration)
)`

MrSqllivenessTableSchema = `
CREATE TABLE system.sqlliveness (
session_id BYTES NOT NULL,
expiration DECIMAL NOT NULL,
Expand Down Expand Up @@ -2306,36 +2298,6 @@ var (
// TODO(jeffswenson): remove the function wrapper around the
// SqllivenessTable descriptor. See TestSupportMultiRegion for context.
SqllivenessTable = func() SystemTable {
if TestSupportMultiRegion() {
return makeSystemTable(
MrSqllivenessTableSchema,
systemTable(
catconstants.SqllivenessTableName,
keys.SqllivenessID,
[]descpb.ColumnDescriptor{
{Name: "session_id", ID: 1, Type: types.Bytes, Nullable: false},
{Name: "expiration", ID: 2, Type: types.Decimal, Nullable: false},
{Name: "crdb_region", ID: 3, Type: types.Bytes, Nullable: false},
},
[]descpb.ColumnFamilyDescriptor{
{
Name: "primary",
ID: 0,
ColumnNames: []string{"session_id", "expiration", "crdb_region"},
ColumnIDs: []descpb.ColumnID{1, 2, 3},
DefaultColumnID: 2,
},
},
descpb.IndexDescriptor{
Name: "primary",
ID: 2,
Unique: true,
KeyColumnNames: []string{"crdb_region", "session_id"},
KeyColumnDirections: []catenumpb.IndexColumn_Direction{catenumpb.IndexColumn_ASC, catenumpb.IndexColumn_ASC},
KeyColumnIDs: []descpb.ColumnID{3, 1},
},
))
}
return makeSystemTable(
SqllivenessTableSchema,
systemTable(
Expand All @@ -2344,17 +2306,25 @@ var (
[]descpb.ColumnDescriptor{
{Name: "session_id", ID: 1, Type: types.Bytes, Nullable: false},
{Name: "expiration", ID: 2, Type: types.Decimal, Nullable: false},
{Name: "crdb_region", ID: 3, Type: types.Bytes, Nullable: false},
},
[]descpb.ColumnFamilyDescriptor{
{
Name: "fam0_session_id_expiration",
Name: "primary",
ID: 0,
ColumnNames: []string{"session_id", "expiration"},
ColumnIDs: []descpb.ColumnID{1, 2},
ColumnNames: []string{"crdb_region", "session_id", "expiration"},
ColumnIDs: []descpb.ColumnID{3, 1, 2},
DefaultColumnID: 2,
},
},
pk("session_id"),
descpb.IndexDescriptor{
Name: "primary",
ID: 2,
Unique: true,
KeyColumnNames: []string{"crdb_region", "session_id"},
KeyColumnDirections: []catenumpb.IndexColumn_Direction{catenumpb.IndexColumn_ASC, catenumpb.IndexColumn_ASC},
KeyColumnIDs: []descpb.ColumnID{3, 1},
},
))
}

Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/catalog/systemschema_test/testdata/bootstrap
Original file line number Diff line number Diff line change
Expand Up @@ -299,8 +299,8 @@ CREATE TABLE public.scheduled_jobs (
CREATE TABLE public.sqlliveness (
session_id BYTES NOT NULL,
expiration DECIMAL NOT NULL,
CONSTRAINT "primary" PRIMARY KEY (session_id ASC),
FAMILY fam0_session_id_expiration (session_id, expiration)
crdb_region BYTES NOT NULL,
CONSTRAINT "primary" PRIMARY KEY (crdb_region ASC, session_id ASC)
);
CREATE TABLE public.migrations (
major INT8 NOT NULL,
Expand Down Expand Up @@ -486,7 +486,7 @@ schema_telemetry
{"table":{"name":"span_stats_tenant_boundaries","id":57,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"tenant_id","id":1,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"boundaries","id":2,"type":{"family":"BytesFamily","oid":17}}],"nextColumnId":3,"families":[{"name":"primary","columnNames":["tenant_id","boundaries"],"columnIds":[1,2],"defaultColumnId":2}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["tenant_id"],"keyColumnDirections":["ASC"],"storeColumnNames":["boundaries"],"keyColumnIds":[1],"storeColumnIds":[2],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":1},"nextIndexId":2,"privileges":{"users":[{"userProto":"admin","privileges":"480","withGrantOption":"480"},{"userProto":"root","privileges":"480","withGrantOption":"480"}],"ownerProto":"node","version":2},"nextMutationId":1,"formatVersion":3,"replacementOf":{"time":{}},"createAsOfTime":{"wallTime":"0"},"nextConstraintId":2}}
{"table":{"name":"span_stats_unique_keys","id":54,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"id","id":1,"type":{"family":"UuidFamily","oid":2950},"defaultExpr":"gen_random_uuid()"},{"name":"key_bytes","id":2,"type":{"family":"BytesFamily","oid":17},"nullable":true}],"nextColumnId":3,"families":[{"name":"primary","columnNames":["id","key_bytes"],"columnIds":[1,2],"defaultColumnId":2}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["id"],"keyColumnDirections":["ASC"],"storeColumnNames":["key_bytes"],"keyColumnIds":[1],"storeColumnIds":[2],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":2},"indexes":[{"name":"unique_keys_key_bytes_idx","id":2,"unique":true,"version":3,"keyColumnNames":["key_bytes"],"keyColumnDirections":["ASC"],"keyColumnIds":[2],"keySuffixColumnIds":[1],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{},"constraintId":1}],"nextIndexId":3,"privileges":{"users":[{"userProto":"admin","privileges":"480","withGrantOption":"480"},{"userProto":"root","privileges":"480","withGrantOption":"480"}],"ownerProto":"node","version":2},"nextMutationId":1,"formatVersion":3,"replacementOf":{"time":{}},"createAsOfTime":{"wallTime":"0"},"nextConstraintId":3}}
{"table":{"name":"sql_instances","id":46,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"id","id":1,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"addr","id":2,"type":{"family":"StringFamily","oid":25},"nullable":true},{"name":"session_id","id":3,"type":{"family":"BytesFamily","oid":17},"nullable":true},{"name":"locality","id":4,"type":{"family":"JsonFamily","oid":3802},"nullable":true},{"name":"sql_addr","id":5,"type":{"family":"StringFamily","oid":25},"nullable":true}],"nextColumnId":6,"families":[{"name":"primary","columnNames":["id","addr","session_id","locality","sql_addr"],"columnIds":[1,2,3,4,5]}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["id"],"keyColumnDirections":["ASC"],"storeColumnNames":["addr","session_id","locality","sql_addr"],"keyColumnIds":[1],"storeColumnIds":[2,3,4,5],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":1},"nextIndexId":2,"privileges":{"users":[{"userProto":"admin","privileges":"480","withGrantOption":"480"},{"userProto":"root","privileges":"480","withGrantOption":"480"}],"ownerProto":"node","version":2},"nextMutationId":1,"formatVersion":3,"replacementOf":{"time":{}},"createAsOfTime":{"wallTime":"0"},"nextConstraintId":2}}
{"table":{"name":"sqlliveness","id":39,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"session_id","id":1,"type":{"family":"BytesFamily","oid":17}},{"name":"expiration","id":2,"type":{"family":"DecimalFamily","oid":1700}}],"nextColumnId":3,"families":[{"name":"fam0_session_id_expiration","columnNames":["session_id","expiration"],"columnIds":[1,2],"defaultColumnId":2}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["session_id"],"keyColumnDirections":["ASC"],"storeColumnNames":["expiration"],"keyColumnIds":[1],"storeColumnIds":[2],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":1},"nextIndexId":2,"privileges":{"users":[{"userProto":"admin","privileges":"480","withGrantOption":"480"},{"userProto":"root","privileges":"480","withGrantOption":"480"}],"ownerProto":"node","version":2},"nextMutationId":1,"formatVersion":3,"replacementOf":{"time":{}},"createAsOfTime":{"wallTime":"0"},"nextConstraintId":2}}
{"table":{"name":"sqlliveness","id":39,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"session_id","id":1,"type":{"family":"BytesFamily","oid":17}},{"name":"expiration","id":2,"type":{"family":"DecimalFamily","oid":1700}},{"name":"crdb_region","id":3,"type":{"family":"BytesFamily","oid":17}}],"nextColumnId":4,"families":[{"name":"primary","columnNames":["crdb_region","session_id","expiration"],"columnIds":[3,1,2],"defaultColumnId":2}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":2,"unique":true,"version":4,"keyColumnNames":["crdb_region","session_id"],"keyColumnDirections":["ASC","ASC"],"storeColumnNames":["expiration"],"keyColumnIds":[3,1],"storeColumnIds":[2],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":1},"nextIndexId":3,"privileges":{"users":[{"userProto":"admin","privileges":"480","withGrantOption":"480"},{"userProto":"root","privileges":"480","withGrantOption":"480"}],"ownerProto":"node","version":2},"nextMutationId":1,"formatVersion":3,"replacementOf":{"time":{}},"createAsOfTime":{"wallTime":"0"},"nextConstraintId":2}}
{"table":{"name":"statement_bundle_chunks","id":34,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"id","id":1,"type":{"family":"IntFamily","width":64,"oid":20},"defaultExpr":"unique_rowid()"},{"name":"description","id":2,"type":{"family":"StringFamily","oid":25},"nullable":true},{"name":"data","id":3,"type":{"family":"BytesFamily","oid":17}}],"nextColumnId":4,"families":[{"name":"primary","columnNames":["id","description","data"],"columnIds":[1,2,3]}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["id"],"keyColumnDirections":["ASC"],"storeColumnNames":["description","data"],"keyColumnIds":[1],"storeColumnIds":[2,3],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":1},"nextIndexId":2,"privileges":{"users":[{"userProto":"admin","privileges":"480","withGrantOption":"480"},{"userProto":"root","privileges":"480","withGrantOption":"480"}],"ownerProto":"node","version":2},"nextMutationId":1,"formatVersion":3,"replacementOf":{"time":{}},"createAsOfTime":{"wallTime":"0"},"nextConstraintId":2}}
{"table":{"name":"statement_diagnostics","id":36,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"id","id":1,"type":{"family":"IntFamily","width":64,"oid":20},"defaultExpr":"unique_rowid()"},{"name":"statement_fingerprint","id":2,"type":{"family":"StringFamily","oid":25}},{"name":"statement","id":3,"type":{"family":"StringFamily","oid":25}},{"name":"collected_at","id":4,"type":{"family":"TimestampTZFamily","oid":1184}},{"name":"trace","id":5,"type":{"family":"JsonFamily","oid":3802},"nullable":true},{"name":"bundle_chunks","id":6,"type":{"family":"ArrayFamily","width":64,"arrayElemType":"IntFamily","oid":1016,"arrayContents":{"family":"IntFamily","width":64,"oid":20}},"nullable":true},{"name":"error","id":7,"type":{"family":"StringFamily","oid":25},"nullable":true}],"nextColumnId":8,"families":[{"name":"primary","columnNames":["id","statement_fingerprint","statement","collected_at","trace","bundle_chunks","error"],"columnIds":[1,2,3,4,5,6,7]}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["id"],"keyColumnDirections":["ASC"],"storeColumnNames":["statement_fingerprint","statement","collected_at","trace","bundle_chunks","error"],"keyColumnIds":[1],"storeColumnIds":[2,3,4,5,6,7],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":1},"nextIndexId":2,"privileges":{"users":[{"userProto":"admin","privileges":"480","withGrantOption":"480"},{"userProto":"root","privileges":"480","withGrantOption":"480"}],"ownerProto":"node","version":2},"nextMutationId":1,"formatVersion":3,"replacementOf":{"time":{}},"createAsOfTime":{"wallTime":"0"},"nextConstraintId":2}}
{"table":{"name":"statement_diagnostics_requests","id":35,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"id","id":1,"type":{"family":"IntFamily","width":64,"oid":20},"defaultExpr":"unique_rowid()"},{"name":"completed","id":2,"type":{"oid":16},"defaultExpr":"false"},{"name":"statement_fingerprint","id":3,"type":{"family":"StringFamily","oid":25}},{"name":"statement_diagnostics_id","id":4,"type":{"family":"IntFamily","width":64,"oid":20},"nullable":true},{"name":"requested_at","id":5,"type":{"family":"TimestampTZFamily","oid":1184}},{"name":"min_execution_latency","id":6,"type":{"family":"IntervalFamily","oid":1186,"intervalDurationField":{}},"nullable":true},{"name":"expires_at","id":7,"type":{"family":"TimestampTZFamily","oid":1184},"nullable":true},{"name":"sampling_probability","id":8,"type":{"family":"FloatFamily","width":64,"oid":701},"nullable":true}],"nextColumnId":9,"families":[{"name":"primary","columnNames":["id","completed","statement_fingerprint","statement_diagnostics_id","requested_at","min_execution_latency","expires_at","sampling_probability"],"columnIds":[1,2,3,4,5,6,7,8]}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["id"],"keyColumnDirections":["ASC"],"storeColumnNames":["completed","statement_fingerprint","statement_diagnostics_id","requested_at","min_execution_latency","expires_at","sampling_probability"],"keyColumnIds":[1],"storeColumnIds":[2,3,4,5,6,7,8],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":1},"indexes":[{"name":"completed_idx","id":2,"version":3,"keyColumnNames":["completed","id"],"keyColumnDirections":["ASC","ASC"],"storeColumnNames":["statement_fingerprint","min_execution_latency","expires_at","sampling_probability"],"keyColumnIds":[2,1],"storeColumnIds":[3,6,7,8],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}}],"nextIndexId":3,"privileges":{"users":[{"userProto":"admin","privileges":"480","withGrantOption":"480"},{"userProto":"root","privileges":"480","withGrantOption":"480"}],"ownerProto":"node","version":2},"nextMutationId":1,"formatVersion":3,"checks":[{"expr":"sampling_probability BETWEEN _:::FLOAT8 AND _:::FLOAT8","name":"check_sampling_probability","columnIds":[8],"constraintId":2}],"replacementOf":{"time":{}},"createAsOfTime":{"wallTime":"0"},"nextConstraintId":3}}
Expand Down
12 changes: 12 additions & 0 deletions pkg/sql/create_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -250,6 +250,18 @@ func (n *createStatsNode) makeJobRecord(ctx context.Context) (*jobs.Record, erro
)
}

if n.Options.UsingExtremes && !n.p.SessionData().EnableCreateStatsUsingExtremes {
return nil, pgerror.New(pgcode.FeatureNotSupported,
"creating partial statistics at extremes is not yet supported",
)
}

if n.Options.Where != nil {
return nil, pgerror.New(pgcode.FeatureNotSupported,
"creating partial statistics with a WHERE clause is not yet supported",
)
}

if err := n.p.CheckPrivilege(ctx, tableDesc, privilege.SELECT); err != nil {
return nil, err
}
Expand Down
4 changes: 4 additions & 0 deletions pkg/sql/exec_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -3470,6 +3470,10 @@ func (m *sessionDataMutator) SetOptimizerAlwaysUseHistograms(val bool) {
m.data.OptimizerAlwaysUseHistograms = val
}

func (m *sessionDataMutator) SetEnableCreateStatsUsingExtremes(val bool) {
m.data.EnableCreateStatsUsingExtremes = val
}

// Utility functions related to scrubbing sensitive information on SQL Stats.

// quantizeCounts ensures that the Count field in the
Expand Down
Loading

0 comments on commit 74f993a

Please sign in to comment.