' tenant-rw
diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html
index 00ece0ccad2b..f83716d38945 100644
--- a/docs/generated/settings/settings.html
+++ b/docs/generated/settings/settings.html
@@ -260,6 +260,6 @@
trace.span_registry.enabled
| boolean | true | if set, ongoing traces can be seen at https://<ui>/#/debug/tracez | Serverless/Dedicated/Self-Hosted |
trace.zipkin.collector
| string |
| the address of a Zipkin instance to receive traces, as <host>:<port>. If no port is specified, 9411 will be used. | Serverless/Dedicated/Self-Hosted |
ui.display_timezone
| enumeration | etc/utc | the timezone used to format timestamps in the ui [etc/utc = 0, america/new_york = 1] | Dedicated/Self-Hosted |
-version
| version | 1000023.1-16 | set the active cluster version in the format '<major>.<minor>' | Serverless/Dedicated/Self-Hosted |
+version
| version | 1000023.1-18 | set the active cluster version in the format '<major>.<minor>' | Serverless/Dedicated/Self-Hosted |
diff --git a/pkg/cli/clisqlclient/statement_diag.go b/pkg/cli/clisqlclient/statement_diag.go
index 3e45233e2cea..db20e1a08ccf 100644
--- a/pkg/cli/clisqlclient/statement_diag.go
+++ b/pkg/cli/clisqlclient/statement_diag.go
@@ -102,8 +102,13 @@ func stmtDiagListBundlesInternal(ctx context.Context, conn Conn) ([]StmtDiagBund
type StmtDiagActivationRequest struct {
ID int64
// Statement is the SQL statement fingerprint.
- Statement string
- RequestedAt time.Time
+ Statement string
+ // If empty then any plan will do.
+ PlanGist string
+ // If true and PlanGist is not empty, then any plan not matching the gist
+ // will do.
+ AntiPlanGist bool
+ RequestedAt time.Time
// Zero value indicates that there is no sampling probability set on the
// request.
SamplingProbability float64
@@ -127,17 +132,16 @@ func StmtDiagListOutstandingRequests(
return result, nil
}
-// TODO(irfansharif): Remove this in 23.1.
-func isAtLeast22dot2ClusterVersion(ctx context.Context, conn Conn) (bool, error) {
- // Check whether the upgrade to add the sampling_probability column to the
- // statement_diagnostics_requests system table has already been run.
+func isAtLeast23dot2ClusterVersion(ctx context.Context, conn Conn) (bool, error) {
+ // Check whether the upgrade to add the plan_gist and anti_plan_gist columns
+ // to the statement_diagnostics_requests system table has already been run.
row, err := conn.QueryRow(ctx, `
SELECT
count(*)
FROM
[SHOW COLUMNS FROM system.statement_diagnostics_requests]
WHERE
- column_name = 'sampling_probability';`)
+ column_name = 'plan_gist';`)
if err != nil {
return false, err
}
@@ -152,12 +156,12 @@ func stmtDiagListOutstandingRequestsInternal(
ctx context.Context, conn Conn,
) ([]StmtDiagActivationRequest, error) {
var extraColumns string
- atLeast22dot2, err := isAtLeast22dot2ClusterVersion(ctx, conn)
+ atLeast23dot2, err := isAtLeast23dot2ClusterVersion(ctx, conn)
if err != nil {
return nil, err
}
- if atLeast22dot2 {
- extraColumns = ", sampling_probability"
+ if atLeast23dot2 {
+ extraColumns = ", plan_gist, anti_plan_gist"
}
// Converting an INTERVAL to a number of milliseconds within that interval
@@ -169,7 +173,7 @@ func stmtDiagListOutstandingRequestsInternal(
EXTRACT(millisecond FROM min_execution_latency)::INT8 -
EXTRACT(second FROM min_execution_latency)::INT8 * 1000`
rows, err := conn.Query(ctx,
- fmt.Sprintf("SELECT id, statement_fingerprint, requested_at, "+getMilliseconds+`, expires_at%s
+ fmt.Sprintf("SELECT id, statement_fingerprint, requested_at, "+getMilliseconds+`, expires_at, sampling_probability%s
FROM system.statement_diagnostics_requests
WHERE NOT completed
ORDER BY requested_at DESC`, extraColumns),
@@ -178,7 +182,7 @@ func stmtDiagListOutstandingRequestsInternal(
return nil, err
}
var result []StmtDiagActivationRequest
- vals := make([]driver.Value, 6)
+ vals := make([]driver.Value, 8)
for {
if err := rows.Next(vals); err == io.EOF {
break
@@ -188,6 +192,8 @@ func stmtDiagListOutstandingRequestsInternal(
var minExecutionLatency time.Duration
var expiresAt time.Time
var samplingProbability float64
+ var planGist string
+ var antiPlanGist bool
if ms, ok := vals[3].(int64); ok {
minExecutionLatency = time.Millisecond * time.Duration(ms)
@@ -195,14 +201,22 @@ func stmtDiagListOutstandingRequestsInternal(
if e, ok := vals[4].(time.Time); ok {
expiresAt = e
}
- if atLeast22dot2 {
- if sp, ok := vals[5].(float64); ok {
- samplingProbability = sp
+ if sp, ok := vals[5].(float64); ok {
+ samplingProbability = sp
+ }
+ if atLeast23dot2 {
+ if gist, ok := vals[6].(string); ok {
+ planGist = gist
+ }
+ if antiGist, ok := vals[7].(bool); ok {
+ antiPlanGist = antiGist
}
}
info := StmtDiagActivationRequest{
ID: vals[0].(int64),
Statement: vals[1].(string),
+ PlanGist: planGist,
+ AntiPlanGist: antiPlanGist,
RequestedAt: vals[2].(time.Time),
SamplingProbability: samplingProbability,
MinExecutionLatency: minExecutionLatency,
diff --git a/pkg/cli/statement_diag.go b/pkg/cli/statement_diag.go
index 36a8273dd96c..8e10a1e12ac2 100644
--- a/pkg/cli/statement_diag.go
+++ b/pkg/cli/statement_diag.go
@@ -85,7 +85,7 @@ func runStmtDiagList(cmd *cobra.Command, args []string) (resErr error) {
} else {
fmt.Printf("Outstanding activation requests:\n")
w := tabwriter.NewWriter(&buf, 4, 0, 2, ' ', 0)
- fmt.Fprint(w, " ID\tActivation time\tStatement\tSampling probability\tMin execution latency\tExpires at\n")
+ fmt.Fprint(w, " ID\tActivation time\tStatement\tPlan gist\tAnti plan gist\tSampling probability\tMin execution latency\tExpires at\n")
for _, r := range reqs {
minExecLatency := "N/A"
if r.MinExecutionLatency != 0 {
@@ -102,8 +102,8 @@ func runStmtDiagList(cmd *cobra.Command, args []string) (resErr error) {
samplingProbability = fmt.Sprintf("%0.4f", r.SamplingProbability)
}
fmt.Fprintf(
- w, " %d\t%s\t%s\t%s\t%s\t%s\n",
- r.ID, r.RequestedAt.UTC().Format(timeFmt), r.Statement, samplingProbability, minExecLatency, expiresAt,
+ w, " %d\t%s\t%s\t%s\t%t\t%s\t%s\t%s\n",
+ r.ID, r.RequestedAt.UTC().Format(timeFmt), r.Statement, r.PlanGist, r.AntiPlanGist, samplingProbability, minExecLatency, expiresAt,
)
}
_ = w.Flush()
diff --git a/pkg/cli/statement_diag_test.go b/pkg/cli/statement_diag_test.go
index 3ed7c3b846b8..1b891cb85d01 100644
--- a/pkg/cli/statement_diag_test.go
+++ b/pkg/cli/statement_diag_test.go
@@ -31,13 +31,15 @@ func Example_statement_diag() {
(20, 'SELECT _ FROM _ WHERE _ > _', 'SELECT a FROM t WHERE b > 1', '2010-01-02 03:04:06', ARRAY[1001,1002,1003]),
(30, 'SELECT _ FROM _ WHERE _ > _', 'SELECT a FROM t WHERE b > 1', '2010-01-02 03:04:07', ARRAY[1001])`,
- `INSERT INTO system.statement_diagnostics_requests(id, completed, statement_fingerprint, statement_diagnostics_id, requested_at, sampling_probability, min_execution_latency, expires_at)
- VALUES (1, TRUE, 'SELECT _ FROM _', 10, '2010-01-02 03:04:00', NULL, NULL, NULL),
- (2, TRUE, 'SELECT _ FROM _ WHERE _ > _', 20, '2010-01-02 03:04:02', 0.5, NULL, NULL),
- (3, TRUE, 'SELECT _ FROM _ WHERE _ > _', 30, '2010-01-02 03:04:05', 1.0, NULL, NULL),
- (4, FALSE, 'SELECT _ + _', NULL, '2010-01-02 03:04:10', 0.8, '1d 2h 3m 4s 5ms 6us', NULL),
- (5, FALSE, 'SELECT _ - _', NULL, '2010-01-02 03:04:11', 1.0, NULL, '2030-01-02 03:04:12'),
- (6, FALSE, 'SELECT _ / _', NULL, '2010-01-02 03:04:12', NULL, '0s', NULL)`,
+ `INSERT INTO system.statement_diagnostics_requests(
+ id, completed, statement_fingerprint, plan_gist, anti_plan_gist, statement_diagnostics_id,
+ requested_at, sampling_probability, min_execution_latency, expires_at)
+ VALUES (1, TRUE, 'SELECT _ FROM _', '', NULL, 10, '2010-01-02 03:04:00', NULL, NULL, NULL),
+ (2, TRUE, 'SELECT _ FROM _ WHERE _ > _', '', NULL, 20, '2010-01-02 03:04:02', 0.5, NULL, NULL),
+ (3, TRUE, 'SELECT _ FROM _ WHERE _ > _', '', NULL, 30, '2010-01-02 03:04:05', 1.0, NULL, NULL),
+ (4, FALSE, 'SELECT _ + _', '', NULL, NULL, '2010-01-02 03:04:10', 0.8, '1d 2h 3m 4s 5ms 6us', NULL),
+ (5, FALSE, 'SELECT _ - _', 'foo', false, NULL, '2010-01-02 03:04:11', 1.0, NULL, '2030-01-02 03:04:12'),
+ (6, FALSE, 'SELECT _ / _', 'bar', true, NULL, '2010-01-02 03:04:12', NULL, '0s', NULL)`,
}
for _, cmd := range commands {
@@ -94,10 +96,10 @@ func Example_statement_diag() {
// 10 2010-01-02 03:04:05 UTC SELECT _ FROM _
//
// Outstanding activation requests:
- // ID Activation time Statement Sampling probability Min execution latency Expires at
- // 6 2010-01-02 03:04:12 UTC SELECT _ / _ 1.0000 N/A never
- // 5 2010-01-02 03:04:11 UTC SELECT _ - _ 1.0000 N/A 2030-01-02 03:04:12 +0000 UTC
- // 4 2010-01-02 03:04:10 UTC SELECT _ + _ 0.8000 26h3m4.005s never
+ // ID Activation time Statement Plan gist Anti plan gist Sampling probability Min execution latency Expires at
+ // 6 2010-01-02 03:04:12 UTC SELECT _ / _ bar true 1.0000 N/A never
+ // 5 2010-01-02 03:04:11 UTC SELECT _ - _ foo false 1.0000 N/A 2030-01-02 03:04:12 +0000 UTC
+ // 4 2010-01-02 03:04:10 UTC SELECT _ + _ false 0.8000 26h3m4.005s never
// statement-diag download 13
// ERROR: failed to download statement diagnostics bundle 13 to 'stmt-bundle-13.zip': no statement diagnostics bundle with ID 13
// statement-diag download 20 tempfile.zip
@@ -120,18 +122,18 @@ func Example_statement_diag() {
// 20 2010-01-02 03:04:06 UTC SELECT _ FROM _ WHERE _ > _
//
// Outstanding activation requests:
- // ID Activation time Statement Sampling probability Min execution latency Expires at
- // 6 2010-01-02 03:04:12 UTC SELECT _ / _ 1.0000 N/A never
- // 5 2010-01-02 03:04:11 UTC SELECT _ - _ 1.0000 N/A 2030-01-02 03:04:12 +0000 UTC
- // 4 2010-01-02 03:04:10 UTC SELECT _ + _ 0.8000 26h3m4.005s never
+ // ID Activation time Statement Plan gist Anti plan gist Sampling probability Min execution latency Expires at
+ // 6 2010-01-02 03:04:12 UTC SELECT _ / _ bar true 1.0000 N/A never
+ // 5 2010-01-02 03:04:11 UTC SELECT _ - _ foo false 1.0000 N/A 2030-01-02 03:04:12 +0000 UTC
+ // 4 2010-01-02 03:04:10 UTC SELECT _ + _ false 0.8000 26h3m4.005s never
// statement-diag delete --all
// statement-diag list
// No statement diagnostics bundles available.
// Outstanding activation requests:
- // ID Activation time Statement Sampling probability Min execution latency Expires at
- // 6 2010-01-02 03:04:12 UTC SELECT _ / _ 1.0000 N/A never
- // 5 2010-01-02 03:04:11 UTC SELECT _ - _ 1.0000 N/A 2030-01-02 03:04:12 +0000 UTC
- // 4 2010-01-02 03:04:10 UTC SELECT _ + _ 0.8000 26h3m4.005s never
+ // ID Activation time Statement Plan gist Anti plan gist Sampling probability Min execution latency Expires at
+ // 6 2010-01-02 03:04:12 UTC SELECT _ / _ bar true 1.0000 N/A never
+ // 5 2010-01-02 03:04:11 UTC SELECT _ - _ foo false 1.0000 N/A 2030-01-02 03:04:12 +0000 UTC
+ // 4 2010-01-02 03:04:10 UTC SELECT _ + _ false 0.8000 26h3m4.005s never
// statement-diag cancel xx
// ERROR: invalid ID
// statement-diag cancel 5 6
@@ -142,9 +144,9 @@ func Example_statement_diag() {
// statement-diag list
// No statement diagnostics bundles available.
// Outstanding activation requests:
- // ID Activation time Statement Sampling probability Min execution latency Expires at
- // 6 2010-01-02 03:04:12 UTC SELECT _ / _ 1.0000 N/A never
- // 5 2010-01-02 03:04:11 UTC SELECT _ - _ 1.0000 N/A 2030-01-02 03:04:12 +0000 UTC
+ // ID Activation time Statement Plan gist Anti plan gist Sampling probability Min execution latency Expires at
+ // 6 2010-01-02 03:04:12 UTC SELECT _ / _ bar true 1.0000 N/A never
+ // 5 2010-01-02 03:04:11 UTC SELECT _ - _ foo false 1.0000 N/A 2030-01-02 03:04:12 +0000 UTC
// statement-diag cancel 123
// ERROR: no outstanding activation request with ID 123
// statement-diag cancel --all
diff --git a/pkg/cli/zip_table_registry.go b/pkg/cli/zip_table_registry.go
index 513f2e246b92..52634b30363e 100644
--- a/pkg/cli/zip_table_registry.go
+++ b/pkg/cli/zip_table_registry.go
@@ -1232,6 +1232,8 @@ var zipSystemTables = DebugZipTableRegistry{
"min_execution_latency",
"expires_at",
"sampling_probability",
+ "plan_gist",
+ "anti_plan_gist",
},
},
"system.table_statistics": {
diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go
index 26d4c035463f..f20d5af1a55c 100644
--- a/pkg/clusterversion/cockroach_versions.go
+++ b/pkg/clusterversion/cockroach_versions.go
@@ -201,10 +201,6 @@ const (
TODODelete_V22_2RemoveGrantPrivilege
// TODODelete_V22_2MVCCRangeTombstones enables the use of MVCC range tombstones.
TODODelete_V22_2MVCCRangeTombstones
- // TODODelete_V22_2SampledStmtDiagReqs enables installing statement diagnostic requests that
- // probabilistically collects stmt bundles, controlled by the user provided
- // sampling rate.
- TODODelete_V22_2SampledStmtDiagReqs
// TODODelete_V22_2SystemPrivilegesTable adds system.privileges table.
TODODelete_V22_2SystemPrivilegesTable
// TODODelete_V22_2EnablePredicateProjectionChangefeed indicates that changefeeds support
@@ -557,6 +553,10 @@ const (
// FormatVirtualSSTables, allowing use of virtual sstables in Pebble.
V23_2_PebbleFormatVirtualSSTables
+ // V23_2_StmtDiagForPlanGist enables statement diagnostic feature to collect
+ // the bundle for particular plan gist.
+ V23_2_StmtDiagForPlanGist
+
// *************************************************
// Step (1) Add new versions here.
// Do not add new versions to a patch release.
@@ -650,10 +650,6 @@ var rawVersionsSingleton = keyedVersions{
Key: TODODelete_V22_2MVCCRangeTombstones,
Version: roachpb.Version{Major: 22, Minor: 1, Internal: 16},
},
- {
- Key: TODODelete_V22_2SampledStmtDiagReqs,
- Version: roachpb.Version{Major: 22, Minor: 1, Internal: 20},
- },
{
Key: TODODelete_V22_2SystemPrivilegesTable,
Version: roachpb.Version{Major: 22, Minor: 1, Internal: 24},
@@ -970,6 +966,10 @@ var rawVersionsSingleton = keyedVersions{
Key: V23_2_PebbleFormatVirtualSSTables,
Version: roachpb.Version{Major: 23, Minor: 1, Internal: 16},
},
+ {
+ Key: V23_2_StmtDiagForPlanGist,
+ Version: roachpb.Version{Major: 23, Minor: 1, Internal: 18},
+ },
// *************************************************
// Step (2): Add new versions here.
diff --git a/pkg/server/serverpb/status.proto b/pkg/server/serverpb/status.proto
index d56a9c0088f8..cbab6e4b0ad1 100644
--- a/pkg/server/serverpb/status.proto
+++ b/pkg/server/serverpb/status.proto
@@ -1800,6 +1800,17 @@ message CreateStatementDiagnosticsReportRequest {
// likelihood? Or provide a hint for how long T is for the currently chosen
// sampling probability.
double sampling_probability = 4;
+ // PlanGist, when set, indicates a particular plan that we want collect
+ // diagnostics for. This can be useful when a single fingerprint can result in
+ // multiple plans.
+ //
+ // There is a caveat to using this filtering: since the plan gist for a
+ // running query is only available after the optimizer has done its part, the
+ // trace will only include things after the optimizer is done.
+ string plan_gist = 5;
+ // AntiPlanGist, when set, indicates that any plan not matching PlanGist will
+ // do.
+ bool anti_plan_gist = 6;
}
message CreateStatementDiagnosticsReportResponse {
diff --git a/pkg/server/statement_diagnostics_requests.go b/pkg/server/statement_diagnostics_requests.go
index 084695988812..8e7654c68bac 100644
--- a/pkg/server/statement_diagnostics_requests.go
+++ b/pkg/server/statement_diagnostics_requests.go
@@ -25,8 +25,13 @@ import (
)
type stmtDiagnosticsRequest struct {
- ID int
- StatementFingerprint string
+ ID int
+ StatementFingerprint string
+ // Empty plan gist indicates that any plan will do.
+ PlanGist string
+ // If true and PlanGist is not empty, then any plan not matching the gist
+ // will do.
+ AntiPlanGist bool
Completed bool
StatementDiagnosticsID int
RequestedAt time.Time
@@ -86,6 +91,8 @@ func (s *statusServer) CreateStatementDiagnosticsReport(
err := s.stmtDiagnosticsRequester.InsertRequest(
ctx,
req.StatementFingerprint,
+ req.PlanGist,
+ req.AntiPlanGist,
req.SamplingProbability,
req.MinExecutionLatency,
req.ExpiresAfter,
@@ -137,11 +144,11 @@ func (s *statusServer) StatementDiagnosticsRequests(
var err error
- // TODO(irfansharif): Remove this version gating in 23.1.
var extraColumns string
- if s.st.Version.IsActive(ctx, clusterversion.TODODelete_V22_2SampledStmtDiagReqs) {
+ if s.st.Version.IsActive(ctx, clusterversion.V23_2_StmtDiagForPlanGist) {
extraColumns = `,
- sampling_probability`
+ plan_gist,
+ anti_plan_gist`
}
// TODO(davidh): Add pagination to this request.
it, err := s.internalExecutor.QueryIteratorEx(ctx, "stmt-diag-get-all", nil, /* txn */
@@ -153,7 +160,8 @@ func (s *statusServer) StatementDiagnosticsRequests(
statement_diagnostics_id,
requested_at,
min_execution_latency,
- expires_at%s
+ expires_at,
+ sampling_probability%s
FROM
system.statement_diagnostics_requests`, extraColumns))
if err != nil {
@@ -179,12 +187,9 @@ func (s *statusServer) StatementDiagnosticsRequests(
if requestedAt, ok := row[4].(*tree.DTimestampTZ); ok {
req.RequestedAt = requestedAt.Time
}
- if extraColumns != "" {
- if samplingProbability, ok := row[7].(*tree.DFloat); ok {
- req.SamplingProbability = float64(*samplingProbability)
- }
+ if samplingProbability, ok := row[7].(*tree.DFloat); ok {
+ req.SamplingProbability = float64(*samplingProbability)
}
-
if minExecutionLatency, ok := row[5].(*tree.DInterval); ok {
req.MinExecutionLatency = time.Duration(minExecutionLatency.Duration.Nanos())
}
@@ -195,6 +200,14 @@ func (s *statusServer) StatementDiagnosticsRequests(
continue
}
}
+ if extraColumns != "" {
+ if planGist, ok := row[8].(*tree.DString); ok {
+ req.PlanGist = string(*planGist)
+ }
+ if antiGist, ok := row[9].(*tree.DBool); ok {
+ req.AntiPlanGist = bool(*antiGist)
+ }
+ }
requests = append(requests, req)
}
diff --git a/pkg/server/status.go b/pkg/server/status.go
index 91b1d30903db..d0820d5dd4ea 100644
--- a/pkg/server/status.go
+++ b/pkg/server/status.go
@@ -504,9 +504,16 @@ type StmtDiagnosticsRequester interface {
// tracing a query with the given fingerprint. Once this returns, calling
// stmtdiagnostics.ShouldCollectDiagnostics() on the current node will
// return true depending on the parameters below.
+ // - planGist, when set, indicates a particular plan that we want collect
+ // diagnostics for. This can be useful when a single fingerprint can
+ // result in multiple plans.
+ // - There is a caveat to using this filtering: since the plan gist for a
+ // running query is only available after the optimizer has done its part,
+ // the trace will only include things after the optimizer is done.
+ // - if antiPlanGist is true, then any plan not matching the gist will do.
// - samplingProbability controls how likely we are to try and collect a
- // diagnostics report for a given execution. The semantics with
- // minExecutionLatency are as follows:
+ // diagnostics report for a given execution. The semantics with
+ // minExecutionLatency are as follows:
// - If samplingProbability is zero, we're always sampling. This is for
// compatibility with pre-22.2 versions where this parameter was not
// available.
@@ -524,6 +531,8 @@ type StmtDiagnosticsRequester interface {
InsertRequest(
ctx context.Context,
stmtFingerprint string,
+ planGist string,
+ antiPlanGist bool,
samplingProbability float64,
minExecutionLatency time.Duration,
expiresAfter time.Duration,
diff --git a/pkg/sql/catalog/bootstrap/testdata/testdata b/pkg/sql/catalog/bootstrap/testdata/testdata
index 4583e77fe891..da565e75a201 100644
--- a/pkg/sql/catalog/bootstrap/testdata/testdata
+++ b/pkg/sql/catalog/bootstrap/testdata/testdata
@@ -1,4 +1,4 @@
-system hash=82145f88506a8e8649d2ada00aa0e05e72b5fc38db4273499684409b93a3cb11
+system hash=5900445bee6b095554932db9d115f0e265710880878c288cabfa6f9f554bf6a6
----
[{"key":"04646573632d696467656e","value":"01c801"}
,{"key":"8b"}
@@ -28,7 +28,7 @@ system hash=82145f88506a8e8649d2ada00aa0e05e72b5fc38db4273499684409b93a3cb11
,{"key":"8b89a88a89","value":"030a80060a1470726f7465637465645f74735f7265636f7264731820200128013a0042280a02696410011a0d080e10001800300050861760002000300068007000780080010088010098010042280a02747310021a0d080310001800300050a40d600020003000680070007800800100880100980100422e0a096d6574615f7479706510031a0c0807100018003000501960002000300068007000780080010088010098010042290a046d65746110041a0c08081000180030005011600020013000680070007800800100880100980100422e0a096e756d5f7370616e7310051a0c08011040180030005014600020003000680070007800800100880100980100422a0a057370616e7310061a0c0808100018003000501160002000300068007000780080010088010098010042340a08766572696669656410071a0c08001000180030005010600020002a0566616c73653000680070007800800100880100980100422b0a0674617267657410081a0c08081000180030005011600020013000680070007800800100880100980100480952aa010a077072696d61727910011801220269642a0274732a096d6574615f747970652a046d6574612a096e756d5f7370616e732a057370616e732a0876657269666965642a06746172676574300140004a10080010001a00200028003000380040005a0070027003700470057006700770087a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651802800101880103980100b2015a0a077072696d61727910001a0269641a0274731a096d6574615f747970651a046d6574611a096e756d5f7370616e731a057370616e731a0876657269666965641a06746172676574200120022003200420052006200720082800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300"}
,{"key":"8b89a98a89","value":"030af8040a0c726f6c655f6f7074696f6e731821200128013a00422d0a08757365726e616d6510011a0c08071000180030005019600020003000680070007800800100880100980100422b0a066f7074696f6e10021a0c08071000180030005019600020003000680070007800800100880100980100422a0a0576616c756510031a0c08071000180030005019600020013000680070007800800100880100980100422c0a07757365725f696410041a0c080c100018003000501a60002000300068007000780080010088010098010048055289010a077072696d617279100118012208757365726e616d6522066f7074696f6e2a0576616c75652a07757365725f696430013002400040004a10080010001a00200028003000380040005a00700370047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a760a1175736572735f757365725f69645f696478100218002207757365725f696430043801380240004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100b201370a077072696d61727910001a08757365726e616d651a066f7074696f6e1a0576616c75651a07757365725f696420012002200320042800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300"}
,{"key":"8b89aa8a89","value":"030ace030a1773746174656d656e745f62756e646c655f6368756e6b731822200128013a0042370a02696410011a0c08011040180030005014600020002a0e756e697175655f726f7769642829300068007000780080010088010098010042300a0b6465736372697074696f6e10021a0c0807100018003000501960002001300068007000780080010088010098010042290a046461746110031a0c080810001800300050116000200030006800700078008001008801009801004804527a0a077072696d61727910011801220269642a0b6465736372697074696f6e2a0464617461300140004a10080010001a00200028003000380040005a00700270037a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100b2012a0a077072696d61727910001a0269641a0b6465736372697074696f6e1a04646174612001200220032800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300"}
-,{"key":"8b89ab8a89","value":"030abd0a0a1e73746174656d656e745f646961676e6f73746963735f72657175657374731823200128013a0042370a02696410011a0c08011040180030005014600020002a0e756e697175655f726f7769642829300068007000780080010088010098010042350a09636f6d706c6574656410021a0c08001000180030005010600020002a0566616c73653000680070007800800100880100980100423a0a1573746174656d656e745f66696e6765727072696e7410031a0c08071000180030005019600020003000680070007800800100880100980100423d0a1873746174656d656e745f646961676e6f73746963735f696410041a0c0801104018003000501460002001300068007000780080010088010098010042320a0c7265717565737465645f617410051a0d080910001800300050a00960002000300068007000780080010088010098010042410a156d696e5f657865637574696f6e5f6c6174656e637910061a13080610001800300050a20960006a04080010002001300068007000780080010088010098010042300a0a657870697265735f617410071a0d080910001800300050a009600020013000680070007800800100880100980100423a0a1473616d706c696e675f70726f626162696c69747910081a0d080210401800300050bd05600020013000680070007800800100880100980100480952f4010a077072696d61727910011801220269642a09636f6d706c657465642a1573746174656d656e745f66696e6765727072696e742a1873746174656d656e745f646961676e6f73746963735f69642a0c7265717565737465645f61742a156d696e5f657865637574696f6e5f6c6174656e63792a0a657870697265735f61742a1473616d706c696e675f70726f626162696c697479300140004a10080010001a00200028003000380040005a0070027003700470057006700770087a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005ad0010a0d636f6d706c657465645f696478100218002209636f6d706c65746564220269642a1573746174656d656e745f66696e6765727072696e742a156d696e5f657865637574696f6e5f6c6174656e63792a0a657870697265735f61742a1473616d706c696e675f70726f626162696c69747930023001400040004a10080010001a00200028003000380040005a0070037006700770087a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100a201620a3a73616d706c696e675f70726f626162696c697479204245545745454e20302e303a3a3a464c4f41543820414e4420312e303a3a3a464c4f415438121a636865636b5f73616d706c696e675f70726f626162696c69747918002808300038004002b201a4010a077072696d61727910001a0269641a09636f6d706c657465641a1573746174656d656e745f66696e6765727072696e741a1873746174656d656e745f646961676e6f73746963735f69641a0c7265717565737465645f61741a156d696e5f657865637574696f6e5f6c6174656e63791a0a657870697265735f61741a1473616d706c696e675f70726f626162696c697479200120022003200420052006200720082800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300"}
+,{"key":"8b89ab8a89","value":"030a820c0a1e73746174656d656e745f646961676e6f73746963735f72657175657374731823200128013a0042370a02696410011a0c08011040180030005014600020002a0e756e697175655f726f7769642829300068007000780080010088010098010042350a09636f6d706c6574656410021a0c08001000180030005010600020002a0566616c73653000680070007800800100880100980100423a0a1573746174656d656e745f66696e6765727072696e7410031a0c08071000180030005019600020003000680070007800800100880100980100423d0a1873746174656d656e745f646961676e6f73746963735f696410041a0c0801104018003000501460002001300068007000780080010088010098010042320a0c7265717565737465645f617410051a0d080910001800300050a00960002000300068007000780080010088010098010042410a156d696e5f657865637574696f6e5f6c6174656e637910061a13080610001800300050a20960006a04080010002001300068007000780080010088010098010042300a0a657870697265735f617410071a0d080910001800300050a009600020013000680070007800800100880100980100423a0a1473616d706c696e675f70726f626162696c69747910081a0d080210401800300050bd05600020013000680070007800800100880100980100422e0a09706c616e5f6769737410091a0c0807100018003000501960002001300068007000780080010088010098010042330a0e616e74695f706c616e5f67697374100a1a0c08001000180030005010600020013000680070007800800100880100980100480b5293020a077072696d61727910011801220269642a09636f6d706c657465642a1573746174656d656e745f66696e6765727072696e742a1873746174656d656e745f646961676e6f73746963735f69642a0c7265717565737465645f61742a156d696e5f657865637574696f6e5f6c6174656e63792a0a657870697265735f61742a1473616d706c696e675f70726f626162696c6974792a09706c616e5f676973742a0e616e74695f706c616e5f67697374300140004a10080010001a00200028003000380040005a0070027003700470057006700770087009700a7a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005af2010a10636f6d706c657465645f6964785f7632100218002209636f6d706c65746564220269642a1573746174656d656e745f66696e6765727072696e742a156d696e5f657865637574696f6e5f6c6174656e63792a0a657870697265735f61742a1473616d706c696e675f70726f626162696c6974792a09706c616e5f676973742a0e616e74695f706c616e5f6769737430023001400040004a10080010001a00200028003000380040005a0070037006700770087009700a7a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100a201620a3a73616d706c696e675f70726f626162696c697479204245545745454e20302e303a3a3a464c4f41543820414e4420312e303a3a3a464c4f415438121a636865636b5f73616d706c696e675f70726f626162696c69747918002808300038004002b201c3010a077072696d61727910001a0269641a09636f6d706c657465641a1573746174656d656e745f66696e6765727072696e741a1873746174656d656e745f646961676e6f73746963735f69641a0c7265717565737465645f61741a156d696e5f657865637574696f6e5f6c6174656e63791a0a657870697265735f61741a1473616d706c696e675f70726f626162696c6974791a09706c616e5f676973741a0e616e74695f706c616e5f67697374200120022003200420052006200720082009200a2800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300"}
,{"key":"8b89ac8a89","value":"030ab2060a1573746174656d656e745f646961676e6f73746963731824200128013a0042370a02696410011a0c08011040180030005014600020002a0e756e697175655f726f77696428293000680070007800800100880100980100423a0a1573746174656d656e745f66696e6765727072696e7410021a0c08071000180030005019600020003000680070007800800100880100980100422e0a0973746174656d656e7410031a0c0807100018003000501960002000300068007000780080010088010098010042320a0c636f6c6c65637465645f617410041a0d080910001800300050a009600020003000680070007800800100880100980100422b0a05747261636510051a0d081210001800300050da1d60002001300068007000780080010088010098010042430a0d62756e646c655f6368756e6b7310061a1d080f104018003000380150f8075a0c080110401800300050146000600020013000680070007800800100880100980100422a0a056572726f7210071a0c08071000180030005019600020013000680070007800800100880100980100480852bc010a077072696d61727910011801220269642a1573746174656d656e745f66696e6765727072696e742a0973746174656d656e742a0c636f6c6c65637465645f61742a0574726163652a0d62756e646c655f6368756e6b732a056572726f72300140004a10080010001a00200028003000380040005a007002700370047005700670077a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100b2016c0a077072696d61727910001a0269641a1573746174656d656e745f66696e6765727072696e741a0973746174656d656e741a0c636f6c6c65637465645f61741a0574726163651a0d62756e646c655f6368756e6b731a056572726f7220012002200320042005200620072800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300"}
,{"key":"8b89ad8a89","value":"030ac7090a0e7363686564756c65645f6a6f62731825200128013a0042400a0b7363686564756c655f696410011a0c08011040180030005014600020002a0e756e697175655f726f7769642829300068007000780080010088010098010042320a0d7363686564756c655f6e616d6510021a0c0807100018003000501960002000300068007000780080010088010098010042420a076372656174656410031a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a3000680070007800800100880100980100422a0a056f776e657210041a0c08071000180030005019600020003000680070007800800100880100980100422e0a086e6578745f72756e10051a0d080910001800300050a00960002001300068007000780080010088010098010042330a0e7363686564756c655f737461746510061a0c0808100018003000501160002001300068007000780080010088010098010042320a0d7363686564756c655f6578707210071a0c0807100018003000501960002001300068007000780080010088010098010042350a107363686564756c655f64657461696c7310081a0c0808100018003000501160002001300068007000780080010088010098010042320a0d6578656375746f725f7479706510091a0c0807100018003000501960002000300068007000780080010088010098010042330a0e657865637574696f6e5f61726773100a1a0c08081000180030005011600020003000680070007800800100880100980100480b52f7010a077072696d61727910011801220b7363686564756c655f69642a0d7363686564756c655f6e616d652a07637265617465642a056f776e65722a086e6578745f72756e2a0e7363686564756c655f73746174652a0d7363686564756c655f657870722a107363686564756c655f64657461696c732a0d6578656375746f725f747970652a0e657865637574696f6e5f61726773300140004a10080010001a00200028003000380040005a0070027003700470057006700770087009700a7a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a700a0c6e6578745f72756e5f6964781002180022086e6578745f72756e3005380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100b201380a05736368656410001a0b7363686564756c655f69641a086e6578745f72756e1a0e7363686564756c655f73746174652001200520062800b201780a056f7468657210011a0d7363686564756c655f6e616d651a07637265617465641a056f776e65721a0d7363686564756c655f657870721a107363686564756c655f64657461696c731a0d6578656375746f725f747970651a0e657865637574696f6e5f61726773200220032004200720082009200a2800b80102c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300"}
,{"key":"8b89af8a89","value":"030ae0030a0b73716c6c6976656e6573731827200128013a00422f0a0a73657373696f6e5f696410011a0c0808100018003000501160002000300068007000780080010088010098010042300a0a65787069726174696f6e10021a0d080310001800300050a40d60002000300068007000780080010088010098010042300a0b637264625f726567696f6e10031a0c080810001800300050116000200030006800700078008001008801009801004804528a010a077072696d61727910021801220b637264625f726567696f6e220a73657373696f6e5f69642a0a65787069726174696f6e30033001400040004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100b201380a077072696d61727910001a0b637264625f726567696f6e1a0a73657373696f6e5f69641a0a65787069726174696f6e2003200120022802b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300"}
@@ -179,7 +179,7 @@ system hash=82145f88506a8e8649d2ada00aa0e05e72b5fc38db4273499684409b93a3cb11
,{"key":"c6898888","value":"0102"}
]
-tenant hash=ab457a08f79b2c2c0d9a2e09fc2b929f220709ec390bbfdf2573c2590c4b43ca
+tenant hash=0c6b496911eb1ed106f6890d197a8d255b4ee60f5e95ef7988cf8a46929b528d
----
[{"key":""}
,{"key":"8b89898a89","value":"0312390a0673797374656d10011a250a0d0a0561646d696e1080101880100a0c0a04726f6f7410801018801012046e6f646518022200280140004a00"}
@@ -207,7 +207,7 @@ tenant hash=ab457a08f79b2c2c0d9a2e09fc2b929f220709ec390bbfdf2573c2590c4b43ca
,{"key":"8b89a88a89","value":"030a80060a1470726f7465637465645f74735f7265636f7264731820200128013a0042280a02696410011a0d080e10001800300050861760002000300068007000780080010088010098010042280a02747310021a0d080310001800300050a40d600020003000680070007800800100880100980100422e0a096d6574615f7479706510031a0c0807100018003000501960002000300068007000780080010088010098010042290a046d65746110041a0c08081000180030005011600020013000680070007800800100880100980100422e0a096e756d5f7370616e7310051a0c08011040180030005014600020003000680070007800800100880100980100422a0a057370616e7310061a0c0808100018003000501160002000300068007000780080010088010098010042340a08766572696669656410071a0c08001000180030005010600020002a0566616c73653000680070007800800100880100980100422b0a0674617267657410081a0c08081000180030005011600020013000680070007800800100880100980100480952aa010a077072696d61727910011801220269642a0274732a096d6574615f747970652a046d6574612a096e756d5f7370616e732a057370616e732a0876657269666965642a06746172676574300140004a10080010001a00200028003000380040005a0070027003700470057006700770087a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651802800101880103980100b2015a0a077072696d61727910001a0269641a0274731a096d6574615f747970651a046d6574611a096e756d5f7370616e731a057370616e731a0876657269666965641a06746172676574200120022003200420052006200720082800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300"}
,{"key":"8b89a98a89","value":"030af8040a0c726f6c655f6f7074696f6e731821200128013a00422d0a08757365726e616d6510011a0c08071000180030005019600020003000680070007800800100880100980100422b0a066f7074696f6e10021a0c08071000180030005019600020003000680070007800800100880100980100422a0a0576616c756510031a0c08071000180030005019600020013000680070007800800100880100980100422c0a07757365725f696410041a0c080c100018003000501a60002000300068007000780080010088010098010048055289010a077072696d617279100118012208757365726e616d6522066f7074696f6e2a0576616c75652a07757365725f696430013002400040004a10080010001a00200028003000380040005a00700370047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a760a1175736572735f757365725f69645f696478100218002207757365725f696430043801380240004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100b201370a077072696d61727910001a08757365726e616d651a066f7074696f6e1a0576616c75651a07757365725f696420012002200320042800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300"}
,{"key":"8b89aa8a89","value":"030ace030a1773746174656d656e745f62756e646c655f6368756e6b731822200128013a0042370a02696410011a0c08011040180030005014600020002a0e756e697175655f726f7769642829300068007000780080010088010098010042300a0b6465736372697074696f6e10021a0c0807100018003000501960002001300068007000780080010088010098010042290a046461746110031a0c080810001800300050116000200030006800700078008001008801009801004804527a0a077072696d61727910011801220269642a0b6465736372697074696f6e2a0464617461300140004a10080010001a00200028003000380040005a00700270037a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100b2012a0a077072696d61727910001a0269641a0b6465736372697074696f6e1a04646174612001200220032800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300"}
-,{"key":"8b89ab8a89","value":"030abd0a0a1e73746174656d656e745f646961676e6f73746963735f72657175657374731823200128013a0042370a02696410011a0c08011040180030005014600020002a0e756e697175655f726f7769642829300068007000780080010088010098010042350a09636f6d706c6574656410021a0c08001000180030005010600020002a0566616c73653000680070007800800100880100980100423a0a1573746174656d656e745f66696e6765727072696e7410031a0c08071000180030005019600020003000680070007800800100880100980100423d0a1873746174656d656e745f646961676e6f73746963735f696410041a0c0801104018003000501460002001300068007000780080010088010098010042320a0c7265717565737465645f617410051a0d080910001800300050a00960002000300068007000780080010088010098010042410a156d696e5f657865637574696f6e5f6c6174656e637910061a13080610001800300050a20960006a04080010002001300068007000780080010088010098010042300a0a657870697265735f617410071a0d080910001800300050a009600020013000680070007800800100880100980100423a0a1473616d706c696e675f70726f626162696c69747910081a0d080210401800300050bd05600020013000680070007800800100880100980100480952f4010a077072696d61727910011801220269642a09636f6d706c657465642a1573746174656d656e745f66696e6765727072696e742a1873746174656d656e745f646961676e6f73746963735f69642a0c7265717565737465645f61742a156d696e5f657865637574696f6e5f6c6174656e63792a0a657870697265735f61742a1473616d706c696e675f70726f626162696c697479300140004a10080010001a00200028003000380040005a0070027003700470057006700770087a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005ad0010a0d636f6d706c657465645f696478100218002209636f6d706c65746564220269642a1573746174656d656e745f66696e6765727072696e742a156d696e5f657865637574696f6e5f6c6174656e63792a0a657870697265735f61742a1473616d706c696e675f70726f626162696c69747930023001400040004a10080010001a00200028003000380040005a0070037006700770087a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100a201620a3a73616d706c696e675f70726f626162696c697479204245545745454e20302e303a3a3a464c4f41543820414e4420312e303a3a3a464c4f415438121a636865636b5f73616d706c696e675f70726f626162696c69747918002808300038004002b201a4010a077072696d61727910001a0269641a09636f6d706c657465641a1573746174656d656e745f66696e6765727072696e741a1873746174656d656e745f646961676e6f73746963735f69641a0c7265717565737465645f61741a156d696e5f657865637574696f6e5f6c6174656e63791a0a657870697265735f61741a1473616d706c696e675f70726f626162696c697479200120022003200420052006200720082800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300"}
+,{"key":"8b89ab8a89","value":"030a820c0a1e73746174656d656e745f646961676e6f73746963735f72657175657374731823200128013a0042370a02696410011a0c08011040180030005014600020002a0e756e697175655f726f7769642829300068007000780080010088010098010042350a09636f6d706c6574656410021a0c08001000180030005010600020002a0566616c73653000680070007800800100880100980100423a0a1573746174656d656e745f66696e6765727072696e7410031a0c08071000180030005019600020003000680070007800800100880100980100423d0a1873746174656d656e745f646961676e6f73746963735f696410041a0c0801104018003000501460002001300068007000780080010088010098010042320a0c7265717565737465645f617410051a0d080910001800300050a00960002000300068007000780080010088010098010042410a156d696e5f657865637574696f6e5f6c6174656e637910061a13080610001800300050a20960006a04080010002001300068007000780080010088010098010042300a0a657870697265735f617410071a0d080910001800300050a009600020013000680070007800800100880100980100423a0a1473616d706c696e675f70726f626162696c69747910081a0d080210401800300050bd05600020013000680070007800800100880100980100422e0a09706c616e5f6769737410091a0c0807100018003000501960002001300068007000780080010088010098010042330a0e616e74695f706c616e5f67697374100a1a0c08001000180030005010600020013000680070007800800100880100980100480b5293020a077072696d61727910011801220269642a09636f6d706c657465642a1573746174656d656e745f66696e6765727072696e742a1873746174656d656e745f646961676e6f73746963735f69642a0c7265717565737465645f61742a156d696e5f657865637574696f6e5f6c6174656e63792a0a657870697265735f61742a1473616d706c696e675f70726f626162696c6974792a09706c616e5f676973742a0e616e74695f706c616e5f67697374300140004a10080010001a00200028003000380040005a0070027003700470057006700770087009700a7a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005af2010a10636f6d706c657465645f6964785f7632100218002209636f6d706c65746564220269642a1573746174656d656e745f66696e6765727072696e742a156d696e5f657865637574696f6e5f6c6174656e63792a0a657870697265735f61742a1473616d706c696e675f70726f626162696c6974792a09706c616e5f676973742a0e616e74695f706c616e5f6769737430023001400040004a10080010001a00200028003000380040005a0070037006700770087009700a7a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100a201620a3a73616d706c696e675f70726f626162696c697479204245545745454e20302e303a3a3a464c4f41543820414e4420312e303a3a3a464c4f415438121a636865636b5f73616d706c696e675f70726f626162696c69747918002808300038004002b201c3010a077072696d61727910001a0269641a09636f6d706c657465641a1573746174656d656e745f66696e6765727072696e741a1873746174656d656e745f646961676e6f73746963735f69641a0c7265717565737465645f61741a156d696e5f657865637574696f6e5f6c6174656e63791a0a657870697265735f61741a1473616d706c696e675f70726f626162696c6974791a09706c616e5f676973741a0e616e74695f706c616e5f67697374200120022003200420052006200720082009200a2800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300"}
,{"key":"8b89ac8a89","value":"030ab2060a1573746174656d656e745f646961676e6f73746963731824200128013a0042370a02696410011a0c08011040180030005014600020002a0e756e697175655f726f77696428293000680070007800800100880100980100423a0a1573746174656d656e745f66696e6765727072696e7410021a0c08071000180030005019600020003000680070007800800100880100980100422e0a0973746174656d656e7410031a0c0807100018003000501960002000300068007000780080010088010098010042320a0c636f6c6c65637465645f617410041a0d080910001800300050a009600020003000680070007800800100880100980100422b0a05747261636510051a0d081210001800300050da1d60002001300068007000780080010088010098010042430a0d62756e646c655f6368756e6b7310061a1d080f104018003000380150f8075a0c080110401800300050146000600020013000680070007800800100880100980100422a0a056572726f7210071a0c08071000180030005019600020013000680070007800800100880100980100480852bc010a077072696d61727910011801220269642a1573746174656d656e745f66696e6765727072696e742a0973746174656d656e742a0c636f6c6c65637465645f61742a0574726163652a0d62756e646c655f6368756e6b732a056572726f72300140004a10080010001a00200028003000380040005a007002700370047005700670077a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100b2016c0a077072696d61727910001a0269641a1573746174656d656e745f66696e6765727072696e741a0973746174656d656e741a0c636f6c6c65637465645f61741a0574726163651a0d62756e646c655f6368756e6b731a056572726f7220012002200320042005200620072800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300"}
,{"key":"8b89ad8a89","value":"030ac7090a0e7363686564756c65645f6a6f62731825200128013a0042400a0b7363686564756c655f696410011a0c08011040180030005014600020002a0e756e697175655f726f7769642829300068007000780080010088010098010042320a0d7363686564756c655f6e616d6510021a0c0807100018003000501960002000300068007000780080010088010098010042420a076372656174656410031a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a3000680070007800800100880100980100422a0a056f776e657210041a0c08071000180030005019600020003000680070007800800100880100980100422e0a086e6578745f72756e10051a0d080910001800300050a00960002001300068007000780080010088010098010042330a0e7363686564756c655f737461746510061a0c0808100018003000501160002001300068007000780080010088010098010042320a0d7363686564756c655f6578707210071a0c0807100018003000501960002001300068007000780080010088010098010042350a107363686564756c655f64657461696c7310081a0c0808100018003000501160002001300068007000780080010088010098010042320a0d6578656375746f725f7479706510091a0c0807100018003000501960002000300068007000780080010088010098010042330a0e657865637574696f6e5f61726773100a1a0c08081000180030005011600020003000680070007800800100880100980100480b52f7010a077072696d61727910011801220b7363686564756c655f69642a0d7363686564756c655f6e616d652a07637265617465642a056f776e65722a086e6578745f72756e2a0e7363686564756c655f73746174652a0d7363686564756c655f657870722a107363686564756c655f64657461696c732a0d6578656375746f725f747970652a0e657865637574696f6e5f61726773300140004a10080010001a00200028003000380040005a0070027003700470057006700770087009700a7a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a700a0c6e6578745f72756e5f6964781002180022086e6578745f72756e3005380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100b201380a05736368656410001a0b7363686564756c655f69641a086e6578745f72756e1a0e7363686564756c655f73746174652001200520062800b201780a056f7468657210011a0d7363686564756c655f6e616d651a07637265617465641a056f776e65721a0d7363686564756c655f657870721a107363686564756c655f64657461696c731a0d6578656375746f725f747970651a0e657865637574696f6e5f61726773200220032004200720082009200a2800b80102c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300"}
,{"key":"8b89af8a89","value":"030ae0030a0b73716c6c6976656e6573731827200128013a00422f0a0a73657373696f6e5f696410011a0c0808100018003000501160002000300068007000780080010088010098010042300a0a65787069726174696f6e10021a0d080310001800300050a40d60002000300068007000780080010088010098010042300a0b637264625f726567696f6e10031a0c080810001800300050116000200030006800700078008001008801009801004804528a010a077072696d61727910021801220b637264625f726567696f6e220a73657373696f6e5f69642a0a65787069726174696f6e30033001400040004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100b201380a077072696d61727910001a0b637264625f726567696f6e1a0a73657373696f6e5f69641a0a65787069726174696f6e2003200120022802b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300"}
diff --git a/pkg/sql/catalog/systemschema/system.go b/pkg/sql/catalog/systemschema/system.go
index 3a543a89ac03..ef853ac7074c 100644
--- a/pkg/sql/catalog/systemschema/system.go
+++ b/pkg/sql/catalog/systemschema/system.go
@@ -444,10 +444,12 @@ CREATE TABLE system.statement_diagnostics_requests(
min_execution_latency INTERVAL NULL,
expires_at TIMESTAMPTZ NULL,
sampling_probability FLOAT NULL,
+ plan_gist STRING NULL,
+ anti_plan_gist BOOL NULL,
CONSTRAINT "primary" PRIMARY KEY (id),
CONSTRAINT check_sampling_probability CHECK (sampling_probability BETWEEN 0.0 AND 1.0),
- INDEX completed_idx (completed, id) STORING (statement_fingerprint, min_execution_latency, expires_at, sampling_probability),
- FAMILY "primary" (id, completed, statement_fingerprint, statement_diagnostics_id, requested_at, min_execution_latency, expires_at, sampling_probability)
+ INDEX completed_idx_v2 (completed, id) STORING (statement_fingerprint, min_execution_latency, expires_at, sampling_probability, plan_gist, anti_plan_gist),
+ FAMILY "primary" (id, completed, statement_fingerprint, statement_diagnostics_id, requested_at, min_execution_latency, expires_at, sampling_probability, plan_gist, anti_plan_gist)
);`
StatementDiagnosticsTableSchema = `
@@ -2410,25 +2412,27 @@ var (
{Name: "min_execution_latency", ID: 6, Type: types.Interval, Nullable: true},
{Name: "expires_at", ID: 7, Type: types.TimestampTZ, Nullable: true},
{Name: "sampling_probability", ID: 8, Type: types.Float, Nullable: true},
+ {Name: "plan_gist", ID: 9, Type: types.String, Nullable: true},
+ {Name: "anti_plan_gist", ID: 10, Type: types.Bool, Nullable: true},
},
[]descpb.ColumnFamilyDescriptor{
{
Name: "primary",
- ColumnNames: []string{"id", "completed", "statement_fingerprint", "statement_diagnostics_id", "requested_at", "min_execution_latency", "expires_at", "sampling_probability"},
- ColumnIDs: []descpb.ColumnID{1, 2, 3, 4, 5, 6, 7, 8},
+ ColumnNames: []string{"id", "completed", "statement_fingerprint", "statement_diagnostics_id", "requested_at", "min_execution_latency", "expires_at", "sampling_probability", "plan_gist", "anti_plan_gist"},
+ ColumnIDs: []descpb.ColumnID{1, 2, 3, 4, 5, 6, 7, 8, 9, 10},
},
},
pk("id"),
// Index for the polling query.
descpb.IndexDescriptor{
- Name: "completed_idx",
+ Name: "completed_idx_v2",
ID: 2,
Unique: false,
KeyColumnNames: []string{"completed", "id"},
- StoreColumnNames: []string{"statement_fingerprint", "min_execution_latency", "expires_at", "sampling_probability"},
+ StoreColumnNames: []string{"statement_fingerprint", "min_execution_latency", "expires_at", "sampling_probability", "plan_gist", "anti_plan_gist"},
KeyColumnIDs: []descpb.ColumnID{2, 1},
KeyColumnDirections: []catenumpb.IndexColumn_Direction{catenumpb.IndexColumn_ASC, catenumpb.IndexColumn_ASC},
- StoreColumnIDs: []descpb.ColumnID{3, 6, 7, 8},
+ StoreColumnIDs: []descpb.ColumnID{3, 6, 7, 8, 9, 10},
Version: descpb.StrictIndexColumnIDGuaranteesVersion,
},
),
diff --git a/pkg/sql/catalog/systemschema_test/testdata/bootstrap_system b/pkg/sql/catalog/systemschema_test/testdata/bootstrap_system
index c11289dc8170..fb5f4af0f90a 100644
--- a/pkg/sql/catalog/systemschema_test/testdata/bootstrap_system
+++ b/pkg/sql/catalog/systemschema_test/testdata/bootstrap_system
@@ -267,8 +267,10 @@ CREATE TABLE public.statement_diagnostics_requests (
min_execution_latency INTERVAL NULL,
expires_at TIMESTAMPTZ NULL,
sampling_probability FLOAT8 NULL,
+ plan_gist STRING NULL,
+ anti_plan_gist BOOL NULL,
CONSTRAINT "primary" PRIMARY KEY (id ASC),
- INDEX completed_idx (completed ASC, id ASC) STORING (statement_fingerprint, min_execution_latency, expires_at, sampling_probability),
+ INDEX completed_idx_v2 (completed ASC, id ASC) STORING (statement_fingerprint, min_execution_latency, expires_at, sampling_probability, plan_gist, anti_plan_gist),
CONSTRAINT check_sampling_probability CHECK (sampling_probability BETWEEN 0.0:::FLOAT8 AND 1.0:::FLOAT8)
);
CREATE TABLE public.statement_diagnostics (
@@ -590,7 +592,7 @@ schema_telemetry
{"table":{"name":"statement_activity","id":60,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"aggregated_ts","id":1,"type":{"family":"TimestampTZFamily","oid":1184}},{"name":"fingerprint_id","id":2,"type":{"family":"BytesFamily","oid":17}},{"name":"transaction_fingerprint_id","id":3,"type":{"family":"BytesFamily","oid":17}},{"name":"plan_hash","id":4,"type":{"family":"BytesFamily","oid":17}},{"name":"app_name","id":5,"type":{"family":"StringFamily","oid":25}},{"name":"agg_interval","id":6,"type":{"family":"IntervalFamily","oid":1186,"intervalDurationField":{}}},{"name":"metadata","id":7,"type":{"family":"JsonFamily","oid":3802}},{"name":"statistics","id":8,"type":{"family":"JsonFamily","oid":3802}},{"name":"plan","id":9,"type":{"family":"JsonFamily","oid":3802}},{"name":"index_recommendations","id":10,"type":{"family":"ArrayFamily","arrayElemType":"StringFamily","oid":1009,"arrayContents":{"family":"StringFamily","oid":25}},"defaultExpr":"ARRAY[]:::STRING[]"},{"name":"execution_count","id":11,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"execution_total_seconds","id":12,"type":{"family":"FloatFamily","width":64,"oid":701}},{"name":"execution_total_cluster_seconds","id":13,"type":{"family":"FloatFamily","width":64,"oid":701}},{"name":"contention_time_avg_seconds","id":14,"type":{"family":"FloatFamily","width":64,"oid":701}},{"name":"cpu_sql_avg_nanos","id":15,"type":{"family":"FloatFamily","width":64,"oid":701}},{"name":"service_latency_avg_seconds","id":16,"type":{"family":"FloatFamily","width":64,"oid":701}},{"name":"service_latency_p99_seconds","id":17,"type":{"family":"FloatFamily","width":64,"oid":701}}],"nextColumnId":18,"families":[{"name":"primary","columnNames":["aggregated_ts","fingerprint_id","transaction_fingerprint_id","plan_hash","app_name","agg_interval","metadata","statistics","plan","index_recommendations","execution_count","execution_total_seconds","execution_total_cluster_seconds","contention_time_avg_seconds","cpu_sql_avg_nanos","service_latency_avg_seconds","service_latency_p99_seconds"],"columnIds":[1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17]}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["aggregated_ts","fingerprint_id","transaction_fingerprint_id","plan_hash","app_name"],"keyColumnDirections":["ASC","ASC","ASC","ASC","ASC"],"storeColumnNames":["agg_interval","metadata","statistics","plan","index_recommendations","execution_count","execution_total_seconds","execution_total_cluster_seconds","contention_time_avg_seconds","cpu_sql_avg_nanos","service_latency_avg_seconds","service_latency_p99_seconds"],"keyColumnIds":[1,2,3,4,5],"storeColumnIds":[6,7,8,9,10,11,12,13,14,15,16,17],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":1},"indexes":[{"name":"fingerprint_id_idx","id":2,"version":3,"keyColumnNames":["fingerprint_id","transaction_fingerprint_id"],"keyColumnDirections":["ASC","ASC"],"keyColumnIds":[2,3],"keySuffixColumnIds":[1,4,5],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}},{"name":"execution_count_idx","id":3,"version":3,"keyColumnNames":["aggregated_ts","execution_count"],"keyColumnDirections":["ASC","DESC"],"keyColumnIds":[1,11],"keySuffixColumnIds":[2,3,4,5],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}},{"name":"execution_total_seconds_idx","id":4,"version":3,"keyColumnNames":["aggregated_ts","execution_total_seconds"],"keyColumnDirections":["ASC","DESC"],"keyColumnIds":[1,12],"keySuffixColumnIds":[2,3,4,5],"compositeColumnIds":[12],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}},{"name":"contention_time_avg_seconds_idx","id":5,"version":3,"keyColumnNames":["aggregated_ts","contention_time_avg_seconds"],"keyColumnDirections":["ASC","DESC"],"keyColumnIds":[1,14],"keySuffixColumnIds":[2,3,4,5],"compositeColumnIds":[14],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}},{"name":"cpu_sql_avg_nanos_idx","id":6,"version":3,"keyColumnNames":["aggregated_ts","cpu_sql_avg_nanos"],"keyColumnDirections":["ASC","DESC"],"keyColumnIds":[1,15],"keySuffixColumnIds":[2,3,4,5],"compositeColumnIds":[15],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}},{"name":"service_latency_avg_seconds_idx","id":7,"version":3,"keyColumnNames":["aggregated_ts","service_latency_avg_seconds"],"keyColumnDirections":["ASC","DESC"],"keyColumnIds":[1,16],"keySuffixColumnIds":[2,3,4,5],"compositeColumnIds":[16],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}},{"name":"service_latency_p99_seconds_idx","id":8,"version":3,"keyColumnNames":["aggregated_ts","service_latency_p99_seconds"],"keyColumnDirections":["ASC","DESC"],"keyColumnIds":[1,17],"keySuffixColumnIds":[2,3,4,5],"compositeColumnIds":[17],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}}],"nextIndexId":9,"privileges":{"users":[{"userProto":"admin","privileges":"32","withGrantOption":"32"},{"userProto":"root","privileges":"32","withGrantOption":"32"}],"ownerProto":"node","version":2},"nextMutationId":1,"formatVersion":3,"replacementOf":{"time":{}},"createAsOfTime":{},"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":{},"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":{},"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":{},"nextConstraintId":3}}
+{"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},{"name":"plan_gist","id":9,"type":{"family":"StringFamily","oid":25},"nullable":true},{"name":"anti_plan_gist","id":10,"type":{"oid":16},"nullable":true}],"nextColumnId":11,"families":[{"name":"primary","columnNames":["id","completed","statement_fingerprint","statement_diagnostics_id","requested_at","min_execution_latency","expires_at","sampling_probability","plan_gist","anti_plan_gist"],"columnIds":[1,2,3,4,5,6,7,8,9,10]}],"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","plan_gist","anti_plan_gist"],"keyColumnIds":[1],"storeColumnIds":[2,3,4,5,6,7,8,9,10],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":1},"indexes":[{"name":"completed_idx_v2","id":2,"version":3,"keyColumnNames":["completed","id"],"keyColumnDirections":["ASC","ASC"],"storeColumnNames":["statement_fingerprint","min_execution_latency","expires_at","sampling_probability","plan_gist","anti_plan_gist"],"keyColumnIds":[2,1],"storeColumnIds":[3,6,7,8,9,10],"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":{},"nextConstraintId":3}}
{"table":{"name":"statement_statistics","id":42,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"aggregated_ts","id":1,"type":{"family":"TimestampTZFamily","oid":1184}},{"name":"fingerprint_id","id":2,"type":{"family":"BytesFamily","oid":17}},{"name":"transaction_fingerprint_id","id":3,"type":{"family":"BytesFamily","oid":17}},{"name":"plan_hash","id":4,"type":{"family":"BytesFamily","oid":17}},{"name":"app_name","id":5,"type":{"family":"StringFamily","oid":25}},{"name":"node_id","id":6,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"agg_interval","id":7,"type":{"family":"IntervalFamily","oid":1186,"intervalDurationField":{}}},{"name":"metadata","id":8,"type":{"family":"JsonFamily","oid":3802}},{"name":"statistics","id":9,"type":{"family":"JsonFamily","oid":3802}},{"name":"plan","id":10,"type":{"family":"JsonFamily","oid":3802}},{"name":"crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_plan_hash_transaction_fingerprint_id_shard_8","id":11,"type":{"family":"IntFamily","width":32,"oid":23},"hidden":true,"computeExpr":"mod(fnv32(crdb_internal.datums_to_bytes(aggregated_ts, app_name, fingerprint_id, node_id, plan_hash, transaction_fingerprint_id)), _:::INT8)"},{"name":"index_recommendations","id":12,"type":{"family":"ArrayFamily","arrayElemType":"StringFamily","oid":1009,"arrayContents":{"family":"StringFamily","oid":25}},"defaultExpr":"ARRAY[]:::STRING[]"},{"name":"indexes_usage","id":13,"type":{"family":"JsonFamily","oid":3802},"nullable":true,"computeExpr":"(statistics-\u003e'_':::STRING)-\u003e'_':::STRING","virtual":true},{"name":"execution_count","id":14,"type":{"family":"IntFamily","width":64,"oid":20},"nullable":true,"computeExpr":"((statistics-\u003e'_':::STRING)-\u003e'_':::STRING)::INT8"},{"name":"service_latency","id":15,"type":{"family":"FloatFamily","width":64,"oid":701},"nullable":true,"computeExpr":"(((statistics-\u003e'_':::STRING)-\u003e'_':::STRING)-\u003e'_':::STRING)::FLOAT8"},{"name":"cpu_sql_nanos","id":16,"type":{"family":"FloatFamily","width":64,"oid":701},"nullable":true,"computeExpr":"(((statistics-\u003e'_':::STRING)-\u003e'_':::STRING)-\u003e'_':::STRING)::FLOAT8"},{"name":"contention_time","id":17,"type":{"family":"FloatFamily","width":64,"oid":701},"nullable":true,"computeExpr":"(((statistics-\u003e'_':::STRING)-\u003e'_':::STRING)-\u003e'_':::STRING)::FLOAT8"},{"name":"total_estimated_execution_time","id":18,"type":{"family":"FloatFamily","width":64,"oid":701},"nullable":true,"computeExpr":"((statistics-\u003e'_':::STRING)-\u003e\u003e'_':::STRING)::FLOAT8 * (((statistics-\u003e'_':::STRING)-\u003e'_':::STRING)-\u003e\u003e'_':::STRING)::FLOAT8"},{"name":"p99_latency","id":19,"type":{"family":"FloatFamily","width":64,"oid":701},"nullable":true,"computeExpr":"(((statistics-\u003e'_':::STRING)-\u003e'_':::STRING)-\u003e'_':::STRING)::FLOAT8"}],"nextColumnId":20,"families":[{"name":"primary","columnNames":["crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_plan_hash_transaction_fingerprint_id_shard_8","aggregated_ts","fingerprint_id","transaction_fingerprint_id","plan_hash","app_name","node_id","agg_interval","metadata","statistics","plan","index_recommendations","execution_count","service_latency","cpu_sql_nanos","contention_time","total_estimated_execution_time","p99_latency"],"columnIds":[11,1,2,3,4,5,6,7,8,9,10,12,14,15,16,17,18,19]}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_plan_hash_transaction_fingerprint_id_shard_8","aggregated_ts","fingerprint_id","transaction_fingerprint_id","plan_hash","app_name","node_id"],"keyColumnDirections":["ASC","ASC","ASC","ASC","ASC","ASC","ASC"],"storeColumnNames":["agg_interval","metadata","statistics","plan","index_recommendations","execution_count","service_latency","cpu_sql_nanos","contention_time","total_estimated_execution_time","p99_latency"],"keyColumnIds":[11,1,2,3,4,5,6],"storeColumnIds":[7,8,9,10,12,14,15,16,17,18,19],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{"isSharded":true,"name":"crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_plan_hash_transaction_fingerprint_id_shard_8","shardBuckets":8,"columnNames":["aggregated_ts","app_name","fingerprint_id","node_id","plan_hash","transaction_fingerprint_id"]},"geoConfig":{},"constraintId":1},"indexes":[{"name":"fingerprint_stats_idx","id":2,"version":3,"keyColumnNames":["fingerprint_id","transaction_fingerprint_id"],"keyColumnDirections":["ASC","ASC"],"keyColumnIds":[2,3],"keySuffixColumnIds":[11,1,4,5,6],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}},{"name":"indexes_usage_idx","id":3,"version":3,"keyColumnNames":["indexes_usage"],"keyColumnDirections":["ASC"],"invertedColumnKinds":["DEFAULT"],"keyColumnIds":[13],"keySuffixColumnIds":[11,1,2,3,4,5,6],"foreignKey":{},"interleave":{},"partitioning":{},"type":"INVERTED","sharded":{},"geoConfig":{}},{"name":"execution_count_idx","id":4,"version":3,"keyColumnNames":["aggregated_ts","app_name","execution_count"],"keyColumnDirections":["ASC","ASC","DESC"],"keyColumnIds":[1,5,14],"keySuffixColumnIds":[11,2,3,4,6],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{},"predicate":"app_name NOT LIKE '_':::STRING"},{"name":"service_latency_idx","id":5,"version":3,"keyColumnNames":["aggregated_ts","app_name","service_latency"],"keyColumnDirections":["ASC","ASC","DESC"],"keyColumnIds":[1,5,15],"keySuffixColumnIds":[11,2,3,4,6],"compositeColumnIds":[15],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{},"predicate":"app_name NOT LIKE '_':::STRING"},{"name":"cpu_sql_nanos_idx","id":6,"version":3,"keyColumnNames":["aggregated_ts","app_name","cpu_sql_nanos"],"keyColumnDirections":["ASC","ASC","DESC"],"keyColumnIds":[1,5,16],"keySuffixColumnIds":[11,2,3,4,6],"compositeColumnIds":[16],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{},"predicate":"app_name NOT LIKE '_':::STRING"},{"name":"contention_time_idx","id":7,"version":3,"keyColumnNames":["aggregated_ts","app_name","contention_time"],"keyColumnDirections":["ASC","ASC","DESC"],"keyColumnIds":[1,5,17],"keySuffixColumnIds":[11,2,3,4,6],"compositeColumnIds":[17],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{},"predicate":"app_name NOT LIKE '_':::STRING"},{"name":"total_estimated_execution_time_idx","id":8,"version":3,"keyColumnNames":["aggregated_ts","app_name","total_estimated_execution_time"],"keyColumnDirections":["ASC","ASC","DESC"],"keyColumnIds":[1,5,18],"keySuffixColumnIds":[11,2,3,4,6],"compositeColumnIds":[18],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{},"predicate":"app_name NOT LIKE '_':::STRING"},{"name":"p99_latency_idx","id":9,"version":3,"keyColumnNames":["aggregated_ts","app_name","p99_latency"],"keyColumnDirections":["ASC","ASC","DESC"],"keyColumnIds":[1,5,19],"keySuffixColumnIds":[11,2,3,4,6],"compositeColumnIds":[19],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{},"predicate":"app_name NOT LIKE '_':::STRING"}],"nextIndexId":10,"privileges":{"users":[{"userProto":"admin","privileges":"32","withGrantOption":"32"},{"userProto":"root","privileges":"32","withGrantOption":"32"}],"ownerProto":"node","version":2},"nextMutationId":1,"formatVersion":3,"checks":[{"expr":"crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_plan_hash_transaction_fingerprint_id_shard_8 IN (_:::INT8, _:::INT8, _:::INT8, _:::INT8, _:::INT8, _:::INT8, _:::INT8, _:::INT8)","name":"check_crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_plan_hash_transaction_fingerprint_id_shard_8","columnIds":[11],"fromHashShardedColumn":true,"constraintId":2}],"replacementOf":{"time":{}},"createAsOfTime":{},"nextConstraintId":3}}
{"table":{"name":"table_statistics","id":20,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"tableID","id":1,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"statisticID","id":2,"type":{"family":"IntFamily","width":64,"oid":20},"defaultExpr":"unique_rowid()"},{"name":"name","id":3,"type":{"family":"StringFamily","oid":25},"nullable":true},{"name":"columnIDs","id":4,"type":{"family":"ArrayFamily","width":64,"arrayElemType":"IntFamily","oid":1016,"arrayContents":{"family":"IntFamily","width":64,"oid":20}}},{"name":"createdAt","id":5,"type":{"family":"TimestampFamily","oid":1114},"defaultExpr":"now():::TIMESTAMP"},{"name":"rowCount","id":6,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"distinctCount","id":7,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"nullCount","id":8,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"histogram","id":9,"type":{"family":"BytesFamily","oid":17},"nullable":true},{"name":"avgSize","id":10,"type":{"family":"IntFamily","width":64,"oid":20},"defaultExpr":"_:::INT8"},{"name":"partialPredicate","id":11,"type":{"family":"StringFamily","oid":25},"nullable":true},{"name":"fullStatisticID","id":12,"type":{"family":"IntFamily","width":64,"oid":20},"nullable":true}],"nextColumnId":13,"families":[{"name":"fam_0_tableID_statisticID_name_columnIDs_createdAt_rowCount_distinctCount_nullCount_histogram","columnNames":["tableID","statisticID","name","columnIDs","createdAt","rowCount","distinctCount","nullCount","histogram","avgSize","partialPredicate","fullStatisticID"],"columnIds":[1,2,3,4,5,6,7,8,9,10,11,12]}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["tableID","statisticID"],"keyColumnDirections":["ASC","ASC"],"storeColumnNames":["name","columnIDs","createdAt","rowCount","distinctCount","nullCount","histogram","avgSize","partialPredicate","fullStatisticID"],"keyColumnIds":[1,2],"storeColumnIds":[3,4,5,6,7,8,9,10,11,12],"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":{},"nextConstraintId":2}}
{"table":{"name":"task_payloads","id":58,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"id","id":1,"type":{"family":"StringFamily","oid":25}},{"name":"created","id":2,"type":{"family":"TimestampTZFamily","oid":1184},"defaultExpr":"now():::TIMESTAMPTZ"},{"name":"owner","id":3,"type":{"family":"StringFamily","oid":25}},{"name":"owner_id","id":4,"type":{"family":"OidFamily","oid":26}},{"name":"min_version","id":5,"type":{"family":"StringFamily","oid":25}},{"name":"description","id":6,"type":{"family":"StringFamily","oid":25},"nullable":true},{"name":"type","id":7,"type":{"family":"StringFamily","oid":25}},{"name":"value","id":8,"type":{"family":"BytesFamily","oid":17}}],"nextColumnId":9,"families":[{"name":"primary","columnNames":["id","created","owner","owner_id","min_version","description","type","value"],"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":["created","owner","owner_id","min_version","description","type","value"],"keyColumnIds":[1],"storeColumnIds":[2,3,4,5,6,7,8],"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":{},"nextConstraintId":2}}
diff --git a/pkg/sql/catalog/systemschema_test/testdata/bootstrap_tenant b/pkg/sql/catalog/systemschema_test/testdata/bootstrap_tenant
index be1dafb6f0e8..ec420beb4c25 100644
--- a/pkg/sql/catalog/systemschema_test/testdata/bootstrap_tenant
+++ b/pkg/sql/catalog/systemschema_test/testdata/bootstrap_tenant
@@ -256,8 +256,10 @@ CREATE TABLE public.statement_diagnostics_requests (
min_execution_latency INTERVAL NULL,
expires_at TIMESTAMPTZ NULL,
sampling_probability FLOAT8 NULL,
+ plan_gist STRING NULL,
+ anti_plan_gist BOOL NULL,
CONSTRAINT "primary" PRIMARY KEY (id ASC),
- INDEX completed_idx (completed ASC, id ASC) STORING (statement_fingerprint, min_execution_latency, expires_at, sampling_probability),
+ INDEX completed_idx_v2 (completed ASC, id ASC) STORING (statement_fingerprint, min_execution_latency, expires_at, sampling_probability, plan_gist, anti_plan_gist),
CONSTRAINT check_sampling_probability CHECK (sampling_probability BETWEEN 0.0:::FLOAT8 AND 1.0:::FLOAT8)
);
CREATE TABLE public.statement_diagnostics (
@@ -531,7 +533,7 @@ schema_telemetry
{"table":{"name":"statement_activity","id":58,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"aggregated_ts","id":1,"type":{"family":"TimestampTZFamily","oid":1184}},{"name":"fingerprint_id","id":2,"type":{"family":"BytesFamily","oid":17}},{"name":"transaction_fingerprint_id","id":3,"type":{"family":"BytesFamily","oid":17}},{"name":"plan_hash","id":4,"type":{"family":"BytesFamily","oid":17}},{"name":"app_name","id":5,"type":{"family":"StringFamily","oid":25}},{"name":"agg_interval","id":6,"type":{"family":"IntervalFamily","oid":1186,"intervalDurationField":{}}},{"name":"metadata","id":7,"type":{"family":"JsonFamily","oid":3802}},{"name":"statistics","id":8,"type":{"family":"JsonFamily","oid":3802}},{"name":"plan","id":9,"type":{"family":"JsonFamily","oid":3802}},{"name":"index_recommendations","id":10,"type":{"family":"ArrayFamily","arrayElemType":"StringFamily","oid":1009,"arrayContents":{"family":"StringFamily","oid":25}},"defaultExpr":"ARRAY[]:::STRING[]"},{"name":"execution_count","id":11,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"execution_total_seconds","id":12,"type":{"family":"FloatFamily","width":64,"oid":701}},{"name":"execution_total_cluster_seconds","id":13,"type":{"family":"FloatFamily","width":64,"oid":701}},{"name":"contention_time_avg_seconds","id":14,"type":{"family":"FloatFamily","width":64,"oid":701}},{"name":"cpu_sql_avg_nanos","id":15,"type":{"family":"FloatFamily","width":64,"oid":701}},{"name":"service_latency_avg_seconds","id":16,"type":{"family":"FloatFamily","width":64,"oid":701}},{"name":"service_latency_p99_seconds","id":17,"type":{"family":"FloatFamily","width":64,"oid":701}}],"nextColumnId":18,"families":[{"name":"primary","columnNames":["aggregated_ts","fingerprint_id","transaction_fingerprint_id","plan_hash","app_name","agg_interval","metadata","statistics","plan","index_recommendations","execution_count","execution_total_seconds","execution_total_cluster_seconds","contention_time_avg_seconds","cpu_sql_avg_nanos","service_latency_avg_seconds","service_latency_p99_seconds"],"columnIds":[1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17]}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["aggregated_ts","fingerprint_id","transaction_fingerprint_id","plan_hash","app_name"],"keyColumnDirections":["ASC","ASC","ASC","ASC","ASC"],"storeColumnNames":["agg_interval","metadata","statistics","plan","index_recommendations","execution_count","execution_total_seconds","execution_total_cluster_seconds","contention_time_avg_seconds","cpu_sql_avg_nanos","service_latency_avg_seconds","service_latency_p99_seconds"],"keyColumnIds":[1,2,3,4,5],"storeColumnIds":[6,7,8,9,10,11,12,13,14,15,16,17],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":1},"indexes":[{"name":"fingerprint_id_idx","id":2,"version":3,"keyColumnNames":["fingerprint_id","transaction_fingerprint_id"],"keyColumnDirections":["ASC","ASC"],"keyColumnIds":[2,3],"keySuffixColumnIds":[1,4,5],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}},{"name":"execution_count_idx","id":3,"version":3,"keyColumnNames":["aggregated_ts","execution_count"],"keyColumnDirections":["ASC","DESC"],"keyColumnIds":[1,11],"keySuffixColumnIds":[2,3,4,5],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}},{"name":"execution_total_seconds_idx","id":4,"version":3,"keyColumnNames":["aggregated_ts","execution_total_seconds"],"keyColumnDirections":["ASC","DESC"],"keyColumnIds":[1,12],"keySuffixColumnIds":[2,3,4,5],"compositeColumnIds":[12],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}},{"name":"contention_time_avg_seconds_idx","id":5,"version":3,"keyColumnNames":["aggregated_ts","contention_time_avg_seconds"],"keyColumnDirections":["ASC","DESC"],"keyColumnIds":[1,14],"keySuffixColumnIds":[2,3,4,5],"compositeColumnIds":[14],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}},{"name":"cpu_sql_avg_nanos_idx","id":6,"version":3,"keyColumnNames":["aggregated_ts","cpu_sql_avg_nanos"],"keyColumnDirections":["ASC","DESC"],"keyColumnIds":[1,15],"keySuffixColumnIds":[2,3,4,5],"compositeColumnIds":[15],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}},{"name":"service_latency_avg_seconds_idx","id":7,"version":3,"keyColumnNames":["aggregated_ts","service_latency_avg_seconds"],"keyColumnDirections":["ASC","DESC"],"keyColumnIds":[1,16],"keySuffixColumnIds":[2,3,4,5],"compositeColumnIds":[16],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}},{"name":"service_latency_p99_seconds_idx","id":8,"version":3,"keyColumnNames":["aggregated_ts","service_latency_p99_seconds"],"keyColumnDirections":["ASC","DESC"],"keyColumnIds":[1,17],"keySuffixColumnIds":[2,3,4,5],"compositeColumnIds":[17],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}}],"nextIndexId":9,"privileges":{"users":[{"userProto":"admin","privileges":"32","withGrantOption":"32"},{"userProto":"root","privileges":"32","withGrantOption":"32"}],"ownerProto":"node","version":2},"nextMutationId":1,"formatVersion":3,"replacementOf":{"time":{}},"createAsOfTime":{},"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":{},"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":{},"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":{},"nextConstraintId":3}}
+{"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},{"name":"plan_gist","id":9,"type":{"family":"StringFamily","oid":25},"nullable":true},{"name":"anti_plan_gist","id":10,"type":{"oid":16},"nullable":true}],"nextColumnId":11,"families":[{"name":"primary","columnNames":["id","completed","statement_fingerprint","statement_diagnostics_id","requested_at","min_execution_latency","expires_at","sampling_probability","plan_gist","anti_plan_gist"],"columnIds":[1,2,3,4,5,6,7,8,9,10]}],"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","plan_gist","anti_plan_gist"],"keyColumnIds":[1],"storeColumnIds":[2,3,4,5,6,7,8,9,10],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":1},"indexes":[{"name":"completed_idx_v2","id":2,"version":3,"keyColumnNames":["completed","id"],"keyColumnDirections":["ASC","ASC"],"storeColumnNames":["statement_fingerprint","min_execution_latency","expires_at","sampling_probability","plan_gist","anti_plan_gist"],"keyColumnIds":[2,1],"storeColumnIds":[3,6,7,8,9,10],"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":{},"nextConstraintId":3}}
{"table":{"name":"statement_statistics","id":42,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"aggregated_ts","id":1,"type":{"family":"TimestampTZFamily","oid":1184}},{"name":"fingerprint_id","id":2,"type":{"family":"BytesFamily","oid":17}},{"name":"transaction_fingerprint_id","id":3,"type":{"family":"BytesFamily","oid":17}},{"name":"plan_hash","id":4,"type":{"family":"BytesFamily","oid":17}},{"name":"app_name","id":5,"type":{"family":"StringFamily","oid":25}},{"name":"node_id","id":6,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"agg_interval","id":7,"type":{"family":"IntervalFamily","oid":1186,"intervalDurationField":{}}},{"name":"metadata","id":8,"type":{"family":"JsonFamily","oid":3802}},{"name":"statistics","id":9,"type":{"family":"JsonFamily","oid":3802}},{"name":"plan","id":10,"type":{"family":"JsonFamily","oid":3802}},{"name":"crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_plan_hash_transaction_fingerprint_id_shard_8","id":11,"type":{"family":"IntFamily","width":32,"oid":23},"hidden":true,"computeExpr":"mod(fnv32(crdb_internal.datums_to_bytes(aggregated_ts, app_name, fingerprint_id, node_id, plan_hash, transaction_fingerprint_id)), _:::INT8)"},{"name":"index_recommendations","id":12,"type":{"family":"ArrayFamily","arrayElemType":"StringFamily","oid":1009,"arrayContents":{"family":"StringFamily","oid":25}},"defaultExpr":"ARRAY[]:::STRING[]"},{"name":"indexes_usage","id":13,"type":{"family":"JsonFamily","oid":3802},"nullable":true,"computeExpr":"(statistics-\u003e'_':::STRING)-\u003e'_':::STRING","virtual":true},{"name":"execution_count","id":14,"type":{"family":"IntFamily","width":64,"oid":20},"nullable":true,"computeExpr":"((statistics-\u003e'_':::STRING)-\u003e'_':::STRING)::INT8"},{"name":"service_latency","id":15,"type":{"family":"FloatFamily","width":64,"oid":701},"nullable":true,"computeExpr":"(((statistics-\u003e'_':::STRING)-\u003e'_':::STRING)-\u003e'_':::STRING)::FLOAT8"},{"name":"cpu_sql_nanos","id":16,"type":{"family":"FloatFamily","width":64,"oid":701},"nullable":true,"computeExpr":"(((statistics-\u003e'_':::STRING)-\u003e'_':::STRING)-\u003e'_':::STRING)::FLOAT8"},{"name":"contention_time","id":17,"type":{"family":"FloatFamily","width":64,"oid":701},"nullable":true,"computeExpr":"(((statistics-\u003e'_':::STRING)-\u003e'_':::STRING)-\u003e'_':::STRING)::FLOAT8"},{"name":"total_estimated_execution_time","id":18,"type":{"family":"FloatFamily","width":64,"oid":701},"nullable":true,"computeExpr":"((statistics-\u003e'_':::STRING)-\u003e\u003e'_':::STRING)::FLOAT8 * (((statistics-\u003e'_':::STRING)-\u003e'_':::STRING)-\u003e\u003e'_':::STRING)::FLOAT8"},{"name":"p99_latency","id":19,"type":{"family":"FloatFamily","width":64,"oid":701},"nullable":true,"computeExpr":"(((statistics-\u003e'_':::STRING)-\u003e'_':::STRING)-\u003e'_':::STRING)::FLOAT8"}],"nextColumnId":20,"families":[{"name":"primary","columnNames":["crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_plan_hash_transaction_fingerprint_id_shard_8","aggregated_ts","fingerprint_id","transaction_fingerprint_id","plan_hash","app_name","node_id","agg_interval","metadata","statistics","plan","index_recommendations","execution_count","service_latency","cpu_sql_nanos","contention_time","total_estimated_execution_time","p99_latency"],"columnIds":[11,1,2,3,4,5,6,7,8,9,10,12,14,15,16,17,18,19]}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_plan_hash_transaction_fingerprint_id_shard_8","aggregated_ts","fingerprint_id","transaction_fingerprint_id","plan_hash","app_name","node_id"],"keyColumnDirections":["ASC","ASC","ASC","ASC","ASC","ASC","ASC"],"storeColumnNames":["agg_interval","metadata","statistics","plan","index_recommendations","execution_count","service_latency","cpu_sql_nanos","contention_time","total_estimated_execution_time","p99_latency"],"keyColumnIds":[11,1,2,3,4,5,6],"storeColumnIds":[7,8,9,10,12,14,15,16,17,18,19],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{"isSharded":true,"name":"crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_plan_hash_transaction_fingerprint_id_shard_8","shardBuckets":8,"columnNames":["aggregated_ts","app_name","fingerprint_id","node_id","plan_hash","transaction_fingerprint_id"]},"geoConfig":{},"constraintId":1},"indexes":[{"name":"fingerprint_stats_idx","id":2,"version":3,"keyColumnNames":["fingerprint_id","transaction_fingerprint_id"],"keyColumnDirections":["ASC","ASC"],"keyColumnIds":[2,3],"keySuffixColumnIds":[11,1,4,5,6],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}},{"name":"indexes_usage_idx","id":3,"version":3,"keyColumnNames":["indexes_usage"],"keyColumnDirections":["ASC"],"invertedColumnKinds":["DEFAULT"],"keyColumnIds":[13],"keySuffixColumnIds":[11,1,2,3,4,5,6],"foreignKey":{},"interleave":{},"partitioning":{},"type":"INVERTED","sharded":{},"geoConfig":{}},{"name":"execution_count_idx","id":4,"version":3,"keyColumnNames":["aggregated_ts","app_name","execution_count"],"keyColumnDirections":["ASC","ASC","DESC"],"keyColumnIds":[1,5,14],"keySuffixColumnIds":[11,2,3,4,6],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{},"predicate":"app_name NOT LIKE '_':::STRING"},{"name":"service_latency_idx","id":5,"version":3,"keyColumnNames":["aggregated_ts","app_name","service_latency"],"keyColumnDirections":["ASC","ASC","DESC"],"keyColumnIds":[1,5,15],"keySuffixColumnIds":[11,2,3,4,6],"compositeColumnIds":[15],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{},"predicate":"app_name NOT LIKE '_':::STRING"},{"name":"cpu_sql_nanos_idx","id":6,"version":3,"keyColumnNames":["aggregated_ts","app_name","cpu_sql_nanos"],"keyColumnDirections":["ASC","ASC","DESC"],"keyColumnIds":[1,5,16],"keySuffixColumnIds":[11,2,3,4,6],"compositeColumnIds":[16],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{},"predicate":"app_name NOT LIKE '_':::STRING"},{"name":"contention_time_idx","id":7,"version":3,"keyColumnNames":["aggregated_ts","app_name","contention_time"],"keyColumnDirections":["ASC","ASC","DESC"],"keyColumnIds":[1,5,17],"keySuffixColumnIds":[11,2,3,4,6],"compositeColumnIds":[17],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{},"predicate":"app_name NOT LIKE '_':::STRING"},{"name":"total_estimated_execution_time_idx","id":8,"version":3,"keyColumnNames":["aggregated_ts","app_name","total_estimated_execution_time"],"keyColumnDirections":["ASC","ASC","DESC"],"keyColumnIds":[1,5,18],"keySuffixColumnIds":[11,2,3,4,6],"compositeColumnIds":[18],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{},"predicate":"app_name NOT LIKE '_':::STRING"},{"name":"p99_latency_idx","id":9,"version":3,"keyColumnNames":["aggregated_ts","app_name","p99_latency"],"keyColumnDirections":["ASC","ASC","DESC"],"keyColumnIds":[1,5,19],"keySuffixColumnIds":[11,2,3,4,6],"compositeColumnIds":[19],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{},"predicate":"app_name NOT LIKE '_':::STRING"}],"nextIndexId":10,"privileges":{"users":[{"userProto":"admin","privileges":"32","withGrantOption":"32"},{"userProto":"root","privileges":"32","withGrantOption":"32"}],"ownerProto":"node","version":2},"nextMutationId":1,"formatVersion":3,"checks":[{"expr":"crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_plan_hash_transaction_fingerprint_id_shard_8 IN (_:::INT8, _:::INT8, _:::INT8, _:::INT8, _:::INT8, _:::INT8, _:::INT8, _:::INT8)","name":"check_crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_plan_hash_transaction_fingerprint_id_shard_8","columnIds":[11],"fromHashShardedColumn":true,"constraintId":2}],"replacementOf":{"time":{}},"createAsOfTime":{},"nextConstraintId":3}}
{"table":{"name":"table_statistics","id":20,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"tableID","id":1,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"statisticID","id":2,"type":{"family":"IntFamily","width":64,"oid":20},"defaultExpr":"unique_rowid()"},{"name":"name","id":3,"type":{"family":"StringFamily","oid":25},"nullable":true},{"name":"columnIDs","id":4,"type":{"family":"ArrayFamily","width":64,"arrayElemType":"IntFamily","oid":1016,"arrayContents":{"family":"IntFamily","width":64,"oid":20}}},{"name":"createdAt","id":5,"type":{"family":"TimestampFamily","oid":1114},"defaultExpr":"now():::TIMESTAMP"},{"name":"rowCount","id":6,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"distinctCount","id":7,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"nullCount","id":8,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"histogram","id":9,"type":{"family":"BytesFamily","oid":17},"nullable":true},{"name":"avgSize","id":10,"type":{"family":"IntFamily","width":64,"oid":20},"defaultExpr":"_:::INT8"},{"name":"partialPredicate","id":11,"type":{"family":"StringFamily","oid":25},"nullable":true},{"name":"fullStatisticID","id":12,"type":{"family":"IntFamily","width":64,"oid":20},"nullable":true}],"nextColumnId":13,"families":[{"name":"fam_0_tableID_statisticID_name_columnIDs_createdAt_rowCount_distinctCount_nullCount_histogram","columnNames":["tableID","statisticID","name","columnIDs","createdAt","rowCount","distinctCount","nullCount","histogram","avgSize","partialPredicate","fullStatisticID"],"columnIds":[1,2,3,4,5,6,7,8,9,10,11,12]}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["tableID","statisticID"],"keyColumnDirections":["ASC","ASC"],"storeColumnNames":["name","columnIDs","createdAt","rowCount","distinctCount","nullCount","histogram","avgSize","partialPredicate","fullStatisticID"],"keyColumnIds":[1,2],"storeColumnIds":[3,4,5,6,7,8,9,10,11,12],"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":{},"nextConstraintId":2}}
{"table":{"name":"transaction_activity","id":59,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"aggregated_ts","id":1,"type":{"family":"TimestampTZFamily","oid":1184}},{"name":"fingerprint_id","id":2,"type":{"family":"BytesFamily","oid":17}},{"name":"app_name","id":3,"type":{"family":"StringFamily","oid":25}},{"name":"agg_interval","id":4,"type":{"family":"IntervalFamily","oid":1186,"intervalDurationField":{}}},{"name":"metadata","id":5,"type":{"family":"JsonFamily","oid":3802}},{"name":"statistics","id":6,"type":{"family":"JsonFamily","oid":3802}},{"name":"query","id":7,"type":{"family":"StringFamily","oid":25}},{"name":"execution_count","id":8,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"execution_total_seconds","id":9,"type":{"family":"FloatFamily","width":64,"oid":701}},{"name":"execution_total_cluster_seconds","id":10,"type":{"family":"FloatFamily","width":64,"oid":701}},{"name":"contention_time_avg_seconds","id":11,"type":{"family":"FloatFamily","width":64,"oid":701}},{"name":"cpu_sql_avg_nanos","id":12,"type":{"family":"FloatFamily","width":64,"oid":701}},{"name":"service_latency_avg_seconds","id":13,"type":{"family":"FloatFamily","width":64,"oid":701}},{"name":"service_latency_p99_seconds","id":14,"type":{"family":"FloatFamily","width":64,"oid":701}}],"nextColumnId":15,"families":[{"name":"primary","columnNames":["aggregated_ts","fingerprint_id","app_name","agg_interval","metadata","statistics","query","execution_count","execution_total_seconds","execution_total_cluster_seconds","contention_time_avg_seconds","cpu_sql_avg_nanos","service_latency_avg_seconds","service_latency_p99_seconds"],"columnIds":[1,2,3,4,5,6,7,8,9,10,11,12,13,14]}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["aggregated_ts","fingerprint_id","app_name"],"keyColumnDirections":["ASC","ASC","ASC"],"storeColumnNames":["agg_interval","metadata","statistics","query","execution_count","execution_total_seconds","execution_total_cluster_seconds","contention_time_avg_seconds","cpu_sql_avg_nanos","service_latency_avg_seconds","service_latency_p99_seconds"],"keyColumnIds":[1,2,3],"storeColumnIds":[4,5,6,7,8,9,10,11,12,13,14],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":1},"indexes":[{"name":"fingerprint_id_idx","id":2,"version":3,"keyColumnNames":["fingerprint_id"],"keyColumnDirections":["ASC"],"keyColumnIds":[2],"keySuffixColumnIds":[1,3],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}},{"name":"execution_count_idx","id":3,"version":3,"keyColumnNames":["aggregated_ts","execution_count"],"keyColumnDirections":["ASC","DESC"],"keyColumnIds":[1,8],"keySuffixColumnIds":[2,3],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}},{"name":"execution_total_seconds_idx","id":4,"version":3,"keyColumnNames":["aggregated_ts","execution_total_seconds"],"keyColumnDirections":["ASC","DESC"],"keyColumnIds":[1,9],"keySuffixColumnIds":[2,3],"compositeColumnIds":[9],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}},{"name":"contention_time_avg_seconds_idx","id":5,"version":3,"keyColumnNames":["aggregated_ts","contention_time_avg_seconds"],"keyColumnDirections":["ASC","DESC"],"keyColumnIds":[1,11],"keySuffixColumnIds":[2,3],"compositeColumnIds":[11],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}},{"name":"cpu_sql_avg_nanos_idx","id":6,"version":3,"keyColumnNames":["aggregated_ts","cpu_sql_avg_nanos"],"keyColumnDirections":["ASC","DESC"],"keyColumnIds":[1,12],"keySuffixColumnIds":[2,3],"compositeColumnIds":[12],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}},{"name":"service_latency_avg_seconds_idx","id":7,"version":3,"keyColumnNames":["aggregated_ts","service_latency_avg_seconds"],"keyColumnDirections":["ASC","DESC"],"keyColumnIds":[1,13],"keySuffixColumnIds":[2,3],"compositeColumnIds":[13],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}},{"name":"service_latency_p99_seconds_idx","id":8,"version":3,"keyColumnNames":["aggregated_ts","service_latency_p99_seconds"],"keyColumnDirections":["ASC","DESC"],"keyColumnIds":[1,14],"keySuffixColumnIds":[2,3],"compositeColumnIds":[14],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}}],"nextIndexId":9,"privileges":{"users":[{"userProto":"admin","privileges":"32","withGrantOption":"32"},{"userProto":"root","privileges":"32","withGrantOption":"32"}],"ownerProto":"node","version":2},"nextMutationId":1,"formatVersion":3,"replacementOf":{"time":{}},"createAsOfTime":{},"nextConstraintId":2}}
@@ -550,7 +552,7 @@ schema_telemetry snapshot_id=7cd8a9ae-f35c-4cd2-970a-757174600874 max_records=10
{"table":{"name":"span_count","id":50,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"singleton","id":1,"type":{"oid":16},"defaultExpr":"true"},{"name":"span_count","id":2,"type":{"family":"IntFamily","width":64,"oid":20}}],"nextColumnId":3,"families":[{"name":"primary","columnNames":["singleton","span_count"],"columnIds":[1,2],"defaultColumnId":2}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["singleton"],"keyColumnDirections":["ASC"],"storeColumnNames":["span_count"],"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,"checks":[{"expr":"singleton","name":"single_row","columnIds":[1],"constraintId":2}],"replacementOf":{"time":{}},"createAsOfTime":{},"nextConstraintId":3}}
{"table":{"name":"span_stats_buckets","id":55,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"id","id":1,"type":{"family":"UuidFamily","oid":2950},"defaultExpr":"gen_random_uuid()"},{"name":"sample_id","id":2,"type":{"family":"UuidFamily","oid":2950}},{"name":"start_key_id","id":3,"type":{"family":"UuidFamily","oid":2950}},{"name":"end_key_id","id":4,"type":{"family":"UuidFamily","oid":2950}},{"name":"requests","id":5,"type":{"family":"IntFamily","width":64,"oid":20}}],"nextColumnId":6,"families":[{"name":"primary","columnNames":["id","sample_id","start_key_id","end_key_id","requests"],"columnIds":[1,2,3,4,5]}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["id"],"keyColumnDirections":["ASC"],"storeColumnNames":["sample_id","start_key_id","end_key_id","requests"],"keyColumnIds":[1],"storeColumnIds":[2,3,4,5],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":1},"indexes":[{"name":"buckets_sample_id_idx","id":2,"version":3,"keyColumnNames":["sample_id"],"keyColumnDirections":["ASC"],"keyColumnIds":[2],"keySuffixColumnIds":[1],"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,"replacementOf":{"time":{}},"createAsOfTime":{},"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":{},"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":{},"nextConstraintId":3}}
+{"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},{"name":"plan_gist","id":9,"type":{"family":"StringFamily","oid":25},"nullable":true},{"name":"anti_plan_gist","id":10,"type":{"oid":16},"nullable":true}],"nextColumnId":11,"families":[{"name":"primary","columnNames":["id","completed","statement_fingerprint","statement_diagnostics_id","requested_at","min_execution_latency","expires_at","sampling_probability","plan_gist","anti_plan_gist"],"columnIds":[1,2,3,4,5,6,7,8,9,10]}],"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","plan_gist","anti_plan_gist"],"keyColumnIds":[1],"storeColumnIds":[2,3,4,5,6,7,8,9,10],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":1},"indexes":[{"name":"completed_idx_v2","id":2,"version":3,"keyColumnNames":["completed","id"],"keyColumnDirections":["ASC","ASC"],"storeColumnNames":["statement_fingerprint","min_execution_latency","expires_at","sampling_probability","plan_gist","anti_plan_gist"],"keyColumnIds":[2,1],"storeColumnIds":[3,6,7,8,9,10],"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":{},"nextConstraintId":3}}
{"table":{"name":"table_statistics","id":20,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"tableID","id":1,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"statisticID","id":2,"type":{"family":"IntFamily","width":64,"oid":20},"defaultExpr":"unique_rowid()"},{"name":"name","id":3,"type":{"family":"StringFamily","oid":25},"nullable":true},{"name":"columnIDs","id":4,"type":{"family":"ArrayFamily","width":64,"arrayElemType":"IntFamily","oid":1016,"arrayContents":{"family":"IntFamily","width":64,"oid":20}}},{"name":"createdAt","id":5,"type":{"family":"TimestampFamily","oid":1114},"defaultExpr":"now():::TIMESTAMP"},{"name":"rowCount","id":6,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"distinctCount","id":7,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"nullCount","id":8,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"histogram","id":9,"type":{"family":"BytesFamily","oid":17},"nullable":true},{"name":"avgSize","id":10,"type":{"family":"IntFamily","width":64,"oid":20},"defaultExpr":"_:::INT8"},{"name":"partialPredicate","id":11,"type":{"family":"StringFamily","oid":25},"nullable":true},{"name":"fullStatisticID","id":12,"type":{"family":"IntFamily","width":64,"oid":20},"nullable":true}],"nextColumnId":13,"families":[{"name":"fam_0_tableID_statisticID_name_columnIDs_createdAt_rowCount_distinctCount_nullCount_histogram","columnNames":["tableID","statisticID","name","columnIDs","createdAt","rowCount","distinctCount","nullCount","histogram","avgSize","partialPredicate","fullStatisticID"],"columnIds":[1,2,3,4,5,6,7,8,9,10,11,12]}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["tableID","statisticID"],"keyColumnDirections":["ASC","ASC"],"storeColumnNames":["name","columnIDs","createdAt","rowCount","distinctCount","nullCount","histogram","avgSize","partialPredicate","fullStatisticID"],"keyColumnIds":[1,2],"storeColumnIds":[3,4,5,6,7,8,9,10,11,12],"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":{},"nextConstraintId":2}}
{"table":{"name":"users","id":4,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"username","id":1,"type":{"family":"StringFamily","oid":25}},{"name":"hashedPassword","id":2,"type":{"family":"BytesFamily","oid":17},"nullable":true},{"name":"isRole","id":3,"type":{"oid":16},"defaultExpr":"false"},{"name":"user_id","id":4,"type":{"family":"OidFamily","oid":26}}],"nextColumnId":5,"families":[{"name":"primary","columnNames":["username","user_id"],"columnIds":[1,4],"defaultColumnId":4},{"name":"fam_2_hashedPassword","id":2,"columnNames":["hashedPassword"],"columnIds":[2],"defaultColumnId":2},{"name":"fam_3_isRole","id":3,"columnNames":["isRole"],"columnIds":[3],"defaultColumnId":3}],"nextFamilyId":4,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["username"],"keyColumnDirections":["ASC"],"storeColumnNames":["hashedPassword","isRole","user_id"],"keyColumnIds":[1],"storeColumnIds":[2,3,4],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":2},"indexes":[{"name":"users_user_id_idx","id":2,"unique":true,"version":3,"keyColumnNames":["user_id"],"keyColumnDirections":["ASC"],"keyColumnIds":[4],"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":{},"nextConstraintId":3}}
{"table":{"name":"zones","id":5,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"id","id":1,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"config","id":2,"type":{"family":"BytesFamily","oid":17},"nullable":true}],"nextColumnId":3,"families":[{"name":"primary","columnNames":["id"],"columnIds":[1]},{"name":"fam_2_config","id":2,"columnNames":["config"],"columnIds":[2],"defaultColumnId":2}],"nextFamilyId":3,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["id"],"keyColumnDirections":["ASC"],"storeColumnNames":["config"],"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":{},"nextConstraintId":2}}
@@ -563,7 +565,7 @@ schema_telemetry snapshot_id=7cd8a9ae-f35c-4cd2-970a-757174600874 max_records=10
{"table":{"name":"span_count","id":50,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"singleton","id":1,"type":{"oid":16},"defaultExpr":"true"},{"name":"span_count","id":2,"type":{"family":"IntFamily","width":64,"oid":20}}],"nextColumnId":3,"families":[{"name":"primary","columnNames":["singleton","span_count"],"columnIds":[1,2],"defaultColumnId":2}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["singleton"],"keyColumnDirections":["ASC"],"storeColumnNames":["span_count"],"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,"checks":[{"expr":"singleton","name":"single_row","columnIds":[1],"constraintId":2}],"replacementOf":{"time":{}},"createAsOfTime":{},"nextConstraintId":3}}
{"table":{"name":"span_stats_buckets","id":55,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"id","id":1,"type":{"family":"UuidFamily","oid":2950},"defaultExpr":"gen_random_uuid()"},{"name":"sample_id","id":2,"type":{"family":"UuidFamily","oid":2950}},{"name":"start_key_id","id":3,"type":{"family":"UuidFamily","oid":2950}},{"name":"end_key_id","id":4,"type":{"family":"UuidFamily","oid":2950}},{"name":"requests","id":5,"type":{"family":"IntFamily","width":64,"oid":20}}],"nextColumnId":6,"families":[{"name":"primary","columnNames":["id","sample_id","start_key_id","end_key_id","requests"],"columnIds":[1,2,3,4,5]}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["id"],"keyColumnDirections":["ASC"],"storeColumnNames":["sample_id","start_key_id","end_key_id","requests"],"keyColumnIds":[1],"storeColumnIds":[2,3,4,5],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":1},"indexes":[{"name":"buckets_sample_id_idx","id":2,"version":3,"keyColumnNames":["sample_id"],"keyColumnDirections":["ASC"],"keyColumnIds":[2],"keySuffixColumnIds":[1],"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,"replacementOf":{"time":{}},"createAsOfTime":{},"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":{},"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":{},"nextConstraintId":3}}
+{"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},{"name":"plan_gist","id":9,"type":{"family":"StringFamily","oid":25},"nullable":true},{"name":"anti_plan_gist","id":10,"type":{"oid":16},"nullable":true}],"nextColumnId":11,"families":[{"name":"primary","columnNames":["id","completed","statement_fingerprint","statement_diagnostics_id","requested_at","min_execution_latency","expires_at","sampling_probability","plan_gist","anti_plan_gist"],"columnIds":[1,2,3,4,5,6,7,8,9,10]}],"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","plan_gist","anti_plan_gist"],"keyColumnIds":[1],"storeColumnIds":[2,3,4,5,6,7,8,9,10],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":1},"indexes":[{"name":"completed_idx_v2","id":2,"version":3,"keyColumnNames":["completed","id"],"keyColumnDirections":["ASC","ASC"],"storeColumnNames":["statement_fingerprint","min_execution_latency","expires_at","sampling_probability","plan_gist","anti_plan_gist"],"keyColumnIds":[2,1],"storeColumnIds":[3,6,7,8,9,10],"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":{},"nextConstraintId":3}}
{"table":{"name":"table_statistics","id":20,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"tableID","id":1,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"statisticID","id":2,"type":{"family":"IntFamily","width":64,"oid":20},"defaultExpr":"unique_rowid()"},{"name":"name","id":3,"type":{"family":"StringFamily","oid":25},"nullable":true},{"name":"columnIDs","id":4,"type":{"family":"ArrayFamily","width":64,"arrayElemType":"IntFamily","oid":1016,"arrayContents":{"family":"IntFamily","width":64,"oid":20}}},{"name":"createdAt","id":5,"type":{"family":"TimestampFamily","oid":1114},"defaultExpr":"now():::TIMESTAMP"},{"name":"rowCount","id":6,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"distinctCount","id":7,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"nullCount","id":8,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"histogram","id":9,"type":{"family":"BytesFamily","oid":17},"nullable":true},{"name":"avgSize","id":10,"type":{"family":"IntFamily","width":64,"oid":20},"defaultExpr":"_:::INT8"},{"name":"partialPredicate","id":11,"type":{"family":"StringFamily","oid":25},"nullable":true},{"name":"fullStatisticID","id":12,"type":{"family":"IntFamily","width":64,"oid":20},"nullable":true}],"nextColumnId":13,"families":[{"name":"fam_0_tableID_statisticID_name_columnIDs_createdAt_rowCount_distinctCount_nullCount_histogram","columnNames":["tableID","statisticID","name","columnIDs","createdAt","rowCount","distinctCount","nullCount","histogram","avgSize","partialPredicate","fullStatisticID"],"columnIds":[1,2,3,4,5,6,7,8,9,10,11,12]}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["tableID","statisticID"],"keyColumnDirections":["ASC","ASC"],"storeColumnNames":["name","columnIDs","createdAt","rowCount","distinctCount","nullCount","histogram","avgSize","partialPredicate","fullStatisticID"],"keyColumnIds":[1,2],"storeColumnIds":[3,4,5,6,7,8,9,10,11,12],"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":{},"nextConstraintId":2}}
{"table":{"name":"users","id":4,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"username","id":1,"type":{"family":"StringFamily","oid":25}},{"name":"hashedPassword","id":2,"type":{"family":"BytesFamily","oid":17},"nullable":true},{"name":"isRole","id":3,"type":{"oid":16},"defaultExpr":"false"},{"name":"user_id","id":4,"type":{"family":"OidFamily","oid":26}}],"nextColumnId":5,"families":[{"name":"primary","columnNames":["username","user_id"],"columnIds":[1,4],"defaultColumnId":4},{"name":"fam_2_hashedPassword","id":2,"columnNames":["hashedPassword"],"columnIds":[2],"defaultColumnId":2},{"name":"fam_3_isRole","id":3,"columnNames":["isRole"],"columnIds":[3],"defaultColumnId":3}],"nextFamilyId":4,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["username"],"keyColumnDirections":["ASC"],"storeColumnNames":["hashedPassword","isRole","user_id"],"keyColumnIds":[1],"storeColumnIds":[2,3,4],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":2},"indexes":[{"name":"users_user_id_idx","id":2,"unique":true,"version":3,"keyColumnNames":["user_id"],"keyColumnDirections":["ASC"],"keyColumnIds":[4],"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":{},"nextConstraintId":3}}
{"table":{"name":"zones","id":5,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"id","id":1,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"config","id":2,"type":{"family":"BytesFamily","oid":17},"nullable":true}],"nextColumnId":3,"families":[{"name":"primary","columnNames":["id"],"columnIds":[1]},{"name":"fam_2_config","id":2,"columnNames":["config"],"columnIds":[2],"defaultColumnId":2}],"nextFamilyId":3,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["id"],"keyColumnDirections":["ASC"],"storeColumnNames":["config"],"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":{},"nextConstraintId":2}}
diff --git a/pkg/sql/conn_executor_exec.go b/pkg/sql/conn_executor_exec.go
index cf63d3b6a786..97afdc251c4c 100644
--- a/pkg/sql/conn_executor_exec.go
+++ b/pkg/sql/conn_executor_exec.go
@@ -623,11 +623,10 @@ func (ex *connExecutor) execStmtInOpenState(
ast = stmt.Statement.AST
}
- var needFinish bool
- // For pausable portal, the instrumentation helper needs to be set up only when
- // the portal is executed for the first time.
+ // For pausable portal, the instrumentation helper needs to be set up only
+ // when the portal is executed for the first time.
if !isPausablePortal() || portal.pauseInfo.execStmtInOpenState.ihWrapper == nil {
- ctx, needFinish = ih.Setup(
+ ctx = ih.Setup(
ctx, ex.server.cfg, ex.statsCollector, p, ex.stmtDiagnosticsRecorder,
stmt.StmtNoConstants, os.ImplicitTxn.Get(), ex.state.priority,
ex.extraTxnState.shouldCollectTxnExecutionStats,
@@ -654,35 +653,37 @@ func (ex *connExecutor) execStmtInOpenState(
p.instrumentation = portal.pauseInfo.execStmtInOpenState.ihWrapper.ih
}
}
- if needFinish {
- sql := stmt.SQL
- defer func() {
- processCleanupFunc("finish instrumentation helper", func() {
- // We need this weird thing because we need to make sure we're closing
- // the correct instrumentation helper for the paused portal.
- ihToFinish := ih
- curRes := res
- if isPausablePortal() {
- ihToFinish = &portal.pauseInfo.execStmtInOpenState.ihWrapper.ih
- curRes = portal.pauseInfo.curRes
- retErr = portal.pauseInfo.execStmtInOpenState.retErr
- retPayload = portal.pauseInfo.execStmtInOpenState.retPayload
- }
- retErr = ihToFinish.Finish(
- ex.server.cfg,
- ex.statsCollector,
- &ex.extraTxnState.accumulatedStats,
- ihToFinish.collectExecStats,
- p,
- ast,
- sql,
- curRes,
- retPayload,
- retErr,
- )
- })
- }()
- }
+
+ // Note that here we always unconditionally defer a function that takes care
+ // of finishing the instrumentation helper. This is needed since in order to
+ // support plan-gist-matching of the statement diagnostics we might not know
+ // right now whether Finish needs to happen.
+ defer processCleanupFunc("finish instrumentation helper", func() {
+ // We need this weird thing because we need to make sure we're
+ // closing the correct instrumentation helper for the paused portal.
+ ihToFinish := ih
+ curRes := res
+ if isPausablePortal() {
+ ihToFinish = &portal.pauseInfo.execStmtInOpenState.ihWrapper.ih
+ curRes = portal.pauseInfo.curRes
+ retErr = portal.pauseInfo.execStmtInOpenState.retErr
+ retPayload = portal.pauseInfo.execStmtInOpenState.retPayload
+ }
+ if ihToFinish.needFinish {
+ retErr = ihToFinish.Finish(
+ ex.server.cfg,
+ ex.statsCollector,
+ &ex.extraTxnState.accumulatedStats,
+ ihToFinish.collectExecStats,
+ p,
+ ast,
+ stmt.SQL,
+ curRes,
+ retPayload,
+ retErr,
+ )
+ }
+ })
if ex.sessionData().TransactionTimeout > 0 && !ex.implicitTxn() && ex.executorType != executorTypeInternal {
timerDuration :=
@@ -1489,7 +1490,17 @@ func (ex *connExecutor) dispatchToExecutionEngine(
}
// Include gist in error reports.
- ctx = withPlanGist(ctx, planner.instrumentation.planGist.String())
+ planGist := planner.instrumentation.planGist.String()
+ ctx = withPlanGist(ctx, planGist)
+ if ppInfo := getPausablePortalInfo(); ppInfo == nil || !ppInfo.dispatchToExecutionEngine.cleanup.isComplete {
+ // If we're not using pausable portals, or it's the first execution of
+ // the pausable portal, and we're not collecting a bundle yet, check
+ // whether we should get a bundle for this particular plan gist.
+ if ih := &planner.instrumentation; !ih.collectBundle && ih.outputMode == unmodifiedOutput {
+ ctx = ih.setupWithPlanGist(ctx, ex.server.cfg, stmt.StmtNoConstants, planGist, &planner.curPlan)
+ }
+ }
+
if planner.extendedEvalCtx.TxnImplicit {
planner.curPlan.flags.Set(planFlagImplicitTxn)
}
diff --git a/pkg/sql/create_stats.go b/pkg/sql/create_stats.go
index dd67725ab8ed..77c0f8721578 100644
--- a/pkg/sql/create_stats.go
+++ b/pkg/sql/create_stats.go
@@ -233,12 +233,6 @@ func (n *createStatsNode) makeJobRecord(ctx context.Context) (*jobs.Record, erro
)
}
- if tableDesc.GetID() == keys.JobsTableID {
- return nil, pgerror.New(
- pgcode.WrongObjectType, "cannot create statistics on system.jobs",
- )
- }
-
if tableDesc.GetID() == keys.ScheduledJobsTableID {
return nil, pgerror.New(
pgcode.WrongObjectType, "cannot create statistics on system.scheduled_jobs",
diff --git a/pkg/sql/instrumentation.go b/pkg/sql/instrumentation.go
index 1930dee0555c..b830381f3b0b 100644
--- a/pkg/sql/instrumentation.go
+++ b/pkg/sql/instrumentation.go
@@ -99,6 +99,12 @@ type instrumentationHelper struct {
// statement; it triggers saving of extra information like the plan string.
collectBundle bool
+ // planGistMatchingBundle is set when the bundle collection was enabled for
+ // a request with plan-gist matching enabled. In particular, such a bundle
+ // will be somewhat incomplete (it'll miss the plan string as well as the
+ // trace will miss all the events that happened in the optimizer).
+ planGistMatchingBundle bool
+
// collectExecStats is set when we are collecting execution statistics for a
// statement.
collectExecStats bool
@@ -117,21 +123,30 @@ type instrumentationHelper struct {
stmtDiagnosticsRecorder *stmtdiagnostics.Registry
withStatementTrace func(trace tracingpb.Recording, stmt string)
- // sp is always populated by the instrumentationHelper Setup method, except in
- // the scenario where we do not need tracing information. This scenario occurs
- // with the confluence of:
+ // sp is populated by the instrumentationHelper, except in the scenario
+ // where we do not need tracing information. This scenario occurs with the
+ // confluence of:
// - not collecting a bundle (collectBundle is false)
// - withStatementTrace is nil (only populated by testing knobs)
// - outputMode is unmodifiedOutput (i.e. outputMode not specified)
// - not collecting execution statistics (collectExecStats is false)
// TODO(yuzefovich): refactor statement span creation #85820
sp *tracing.Span
-
- // shouldFinishSpan determines whether sp needs to be finished in
- // instrumentationHelper.Finish.
+ // parentSp, if set, is the parent span of sp, created by the
+ // instrumentationHelper for plan-gist based bundle collection. It is set
+ // if both Setup and setupWithPlanGist methods create their own spans, but
+ // Setup one is non-verbose (which is insufficient for the bundle
+ // collection). It is stored only to be finished in Finish and should
+ // **not** be accessed otherwise.
+ parentSp *tracing.Span
+
+ // shouldFinishSpan determines whether sp and parentSp (if set) need to be
+ // finished in instrumentationHelper.Finish.
shouldFinishSpan bool
- origCtx context.Context
- evalCtx *eval.Context
+ // needFinish determines whether Finish must be called.
+ needFinish bool
+ origCtx context.Context
+ evalCtx *eval.Context
queryLevelStatsWithErr *execstats.QueryLevelStatsWithErr
@@ -249,9 +264,19 @@ func (ih *instrumentationHelper) SetOutputMode(outputMode outputMode, explainFla
ih.explainFlags = explainFlags
}
+func (ih *instrumentationHelper) finalizeSetup(ctx context.Context) {
+ if ih.ShouldBuildExplainPlan() {
+ // Populate traceMetadata at the end once we have all properties of the
+ // helper setup.
+ ih.traceMetadata = make(execNodeTraceMetadata)
+ }
+ // Make sure that the builtins use the correct context.
+ ih.evalCtx.SetDeprecatedContext(ctx)
+}
+
// Setup potentially enables verbose tracing for the statement, depending on
// output mode or statement diagnostic activation requests. Finish() must be
-// called after the statement finishes execution (unless needFinish=false, in
+// called after the statement finishes execution (unless ih.needFinish=false, in
// which case Finish() is a no-op).
func (ih *instrumentationHelper) Setup(
ctx context.Context,
@@ -263,7 +288,7 @@ func (ih *instrumentationHelper) Setup(
implicitTxn bool,
txnPriority roachpb.UserPriority,
collectTxnExecStats bool,
-) (newCtx context.Context, needFinish bool) {
+) (newCtx context.Context) {
ih.fingerprint = fingerprint
ih.implicitTxn = implicitTxn
ih.txnPriority = txnPriority
@@ -287,7 +312,7 @@ func (ih *instrumentationHelper) Setup(
default:
ih.collectBundle, ih.diagRequestID, ih.diagRequest =
- stmtDiagnosticsRecorder.ShouldCollectDiagnostics(ctx, fingerprint)
+ stmtDiagnosticsRecorder.ShouldCollectDiagnostics(ctx, fingerprint, "" /* planGist */)
}
ih.stmtDiagnosticsRecorder = stmtDiagnosticsRecorder
@@ -296,15 +321,7 @@ func (ih *instrumentationHelper) Setup(
var previouslySampled bool
previouslySampled, ih.savePlanForStats = statsCollector.ShouldSample(fingerprint, implicitTxn, p.SessionData().Database)
- defer func() {
- if ih.ShouldBuildExplainPlan() {
- // Populate traceMetadata at the end once we have all properties of
- // the helper setup.
- ih.traceMetadata = make(execNodeTraceMetadata)
- }
- // Make sure that the builtins use the correct context.
- ih.evalCtx.SetDeprecatedContext(newCtx)
- }()
+ defer func() { ih.finalizeSetup(newCtx) }()
if sp := tracing.SpanFromContext(ctx); sp != nil {
if sp.IsVerbose() {
@@ -317,7 +334,8 @@ func (ih *instrumentationHelper) Setup(
// span in order to fetch the trace from it, but the span won't be
// finished.
ih.sp = sp
- return ctx, true /* needFinish */
+ ih.needFinish = true
+ return ctx
}
} else {
if buildutil.CrdbTestBuild {
@@ -344,9 +362,10 @@ func (ih *instrumentationHelper) Setup(
newCtx, ih.sp = tracing.EnsureChildSpan(ctx, cfg.AmbientCtx.Tracer, "traced statement",
tracing.WithRecording(tracingpb.RecordingStructured))
ih.shouldFinishSpan = true
- return newCtx, true
+ ih.needFinish = true
+ return newCtx
}
- return ctx, false
+ return ctx
}
ih.collectExecStats = true
@@ -362,7 +381,53 @@ func (ih *instrumentationHelper) Setup(
}
newCtx, ih.sp = tracing.EnsureChildSpan(ctx, cfg.AmbientCtx.Tracer, "traced statement", tracing.WithRecording(recType))
ih.shouldFinishSpan = true
- return newCtx, true
+ ih.needFinish = true
+ return newCtx
+}
+
+// setupWithPlanGist checks whether the bundle should be collected for the
+// provided fingerprint and plan gist. It assumes that the bundle is not
+// currently being collected.
+func (ih *instrumentationHelper) setupWithPlanGist(
+ ctx context.Context, cfg *ExecutorConfig, fingerprint, planGist string, plan *planTop,
+) context.Context {
+ ih.collectBundle, ih.diagRequestID, ih.diagRequest =
+ ih.stmtDiagnosticsRecorder.ShouldCollectDiagnostics(ctx, fingerprint, planGist)
+ if ih.collectBundle {
+ ih.needFinish = true
+ ih.collectExecStats = true
+ ih.planGistMatchingBundle = true
+ if ih.sp == nil || !ih.sp.IsVerbose() {
+ // We will create a verbose span
+ // - if we don't have a span yet, or
+ // - we do have a span, but it's not verbose.
+ //
+ // ih.sp can be non-nil and non-verbose when it was created in Setup
+ // because the stmt got sampled (i.e. ih.collectExecStats was true).
+ // (Note that it couldn't have been EXPLAIN ANALYZE code path in
+ // Setup because it uses a different output mode.) In any case,
+ // we're responsible for finishing this span, so we reassign it to
+ // ih.parentSp to keep track of.
+ //
+ // Note that we don't need to explicitly use ih.sp when creating a
+ // child span because it's implicitly stored in ctx.
+ if ih.sp != nil {
+ ih.parentSp = ih.sp
+ }
+ ctx, ih.sp = tracing.EnsureChildSpan(
+ ctx, cfg.AmbientCtx.Tracer, "plan-gist bundle",
+ tracing.WithRecording(tracingpb.RecordingVerbose),
+ )
+ ih.shouldFinishSpan = true
+ ih.finalizeSetup(ctx)
+ log.VEventf(ctx, 1, "plan-gist matching bundle collection began after the optimizer finished its part")
+ }
+ } else {
+ // We won't need the memo and the catalog, so free it up.
+ plan.mem = nil
+ plan.catalog = nil
+ }
+ return ctx
}
func (ih *instrumentationHelper) Finish(
@@ -388,7 +453,15 @@ func (ih *instrumentationHelper) Finish(
if ih.shouldFinishSpan {
trace = ih.sp.FinishAndGetConfiguredRecording()
+ if ih.parentSp != nil {
+ defer ih.parentSp.Finish()
+ }
} else {
+ if buildutil.CrdbTestBuild {
+ if ih.parentSp != nil {
+ panic(errors.AssertionFailedf("parentSp is non-nil but shouldFinishSpan is false"))
+ }
+ }
trace = ih.sp.GetConfiguredRecording()
}
@@ -443,10 +516,16 @@ func (ih *instrumentationHelper) Finish(
bundleCtx, cancel = context.WithTimeout(context.Background(), 10*time.Second) // nolint:context
defer cancel()
}
+ planString := ob.BuildString()
+ if ih.planGistMatchingBundle {
+ // We don't have the plan string available since the stmt bundle
+ // collection was enabled _after_ the optimizer was done.
+ planString = "-- plan elided due to gist matching"
+ }
bundle = buildStatementBundle(
- bundleCtx, ih.explainFlags, cfg.DB, ie.(*InternalExecutor), stmtRawSQL, &p.curPlan,
- ob.BuildString(), trace, placeholders, res.Err(), payloadErr, retErr,
- &p.extendedEvalCtx.Settings.SV,
+ bundleCtx, ih.explainFlags, cfg.DB, ie.(*InternalExecutor),
+ stmtRawSQL, &p.curPlan, planString, trace, placeholders, res.Err(),
+ payloadErr, retErr, &p.extendedEvalCtx.Settings.SV,
)
// Include all non-critical errors as warnings. Note that these
// error strings might contain PII, but the warnings are only shown
@@ -528,11 +607,6 @@ func (ih *instrumentationHelper) ShouldCollectExecStats() bool {
return ih.collectExecStats
}
-// ShouldSaveMemo returns true if we should save the memo and catalog in planTop.
-func (ih *instrumentationHelper) ShouldSaveMemo() bool {
- return ih.collectBundle
-}
-
// RecordExplainPlan records the explain.Plan for this query.
func (ih *instrumentationHelper) RecordExplainPlan(explainPlan *explain.Plan) {
ih.explainPlan = explainPlan
diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog b/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog
index 3b7d5f928d67..d1fd1c9a5561 100644
--- a/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog
+++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog
@@ -131,7 +131,7 @@ SELECT id, strip_volatile(descriptor) FROM crdb_internal.kv_catalog_descriptor O
32 {"table": {"columns": [{"id": 1, "name": "id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 2, "name": "ts", "type": {"family": "DecimalFamily", "oid": 1700}}, {"id": 3, "name": "meta_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "meta", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 5, "name": "num_spans", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "spans", "type": {"family": "BytesFamily", "oid": 17}}, {"defaultExpr": "false", "id": 7, "name": "verified", "type": {"oid": 16}}, {"id": 8, "name": "target", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 32, "name": "protected_ts_records", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "parentId": 1, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["id"], "name": "primary", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6, 7, 8], "storeColumnNames": ["ts", "meta_type", "meta", "num_spans", "spans", "verified", "target"], "unique": true, "version": 4}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "admin", "withGrantOption": "32"}, {"privileges": "32", "userProto": "root", "withGrantOption": "32"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 29, "version": "1"}}
33 {"table": {"columns": [{"id": 1, "name": "username", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "option", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "value", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "user_id", "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 33, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [4], "keyColumnNames": ["user_id"], "keySuffixColumnIds": [1, 2], "name": "users_user_id_idx", "partitioning": {}, "sharded": {}, "version": 3}], "name": "role_options", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "parentId": 1, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC", "ASC"], "keyColumnIds": [1, 2], "keyColumnNames": ["username", "option"], "name": "primary", "partitioning": {}, "sharded": {}, "storeColumnIds": [3, 4], "storeColumnNames": ["value", "user_id"], "unique": true, "version": 4}, "privileges": {"ownerProto": "node", "users": [{"privileges": "480", "userProto": "admin", "withGrantOption": "480"}, {"privileges": "480", "userProto": "root", "withGrantOption": "480"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 29, "version": "2"}}
34 {"table": {"columns": [{"defaultExpr": "unique_rowid()", "id": 1, "name": "id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "description", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "data", "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 34, "name": "statement_bundle_chunks", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "parentId": 1, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["id"], "name": "primary", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3], "storeColumnNames": ["description", "data"], "unique": true, "version": 4}, "privileges": {"ownerProto": "node", "users": [{"privileges": "480", "userProto": "admin", "withGrantOption": "480"}, {"privileges": "480", "userProto": "root", "withGrantOption": "480"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 29, "version": "1"}}
-35 {"table": {"checks": [{"columnIds": [8], "constraintId": 2, "expr": "sampling_probability BETWEEN 0.0:::FLOAT8 AND 1.0:::FLOAT8", "name": "check_sampling_probability"}], "columns": [{"defaultExpr": "unique_rowid()", "id": 1, "name": "id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"defaultExpr": "false", "id": 2, "name": "completed", "type": {"oid": 16}}, {"id": 3, "name": "statement_fingerprint", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "statement_diagnostics_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "requested_at", "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 6, "name": "min_execution_latency", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 7, "name": "expires_at", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 8, "name": "sampling_probability", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}], "formatVersion": 3, "id": 35, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC", "ASC"], "keyColumnIds": [2, 1], "keyColumnNames": ["completed", "id"], "name": "completed_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [3, 6, 7, 8], "storeColumnNames": ["statement_fingerprint", "min_execution_latency", "expires_at", "sampling_probability"], "version": 3}], "name": "statement_diagnostics_requests", "nextColumnId": 9, "nextConstraintId": 3, "nextIndexId": 3, "nextMutationId": 1, "parentId": 1, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["id"], "name": "primary", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6, 7, 8], "storeColumnNames": ["completed", "statement_fingerprint", "statement_diagnostics_id", "requested_at", "min_execution_latency", "expires_at", "sampling_probability"], "unique": true, "version": 4}, "privileges": {"ownerProto": "node", "users": [{"privileges": "480", "userProto": "admin", "withGrantOption": "480"}, {"privileges": "480", "userProto": "root", "withGrantOption": "480"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 29, "version": "1"}}
+35 {"table": {"checks": [{"columnIds": [8], "constraintId": 2, "expr": "sampling_probability BETWEEN 0.0:::FLOAT8 AND 1.0:::FLOAT8", "name": "check_sampling_probability"}], "columns": [{"defaultExpr": "unique_rowid()", "id": 1, "name": "id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"defaultExpr": "false", "id": 2, "name": "completed", "type": {"oid": 16}}, {"id": 3, "name": "statement_fingerprint", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "statement_diagnostics_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "requested_at", "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 6, "name": "min_execution_latency", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 7, "name": "expires_at", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 8, "name": "sampling_probability", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 9, "name": "plan_gist", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "anti_plan_gist", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 35, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC", "ASC"], "keyColumnIds": [2, 1], "keyColumnNames": ["completed", "id"], "name": "completed_idx_v2", "partitioning": {}, "sharded": {}, "storeColumnIds": [3, 6, 7, 8, 9, 10], "storeColumnNames": ["statement_fingerprint", "min_execution_latency", "expires_at", "sampling_probability", "plan_gist", "anti_plan_gist"], "version": 3}], "name": "statement_diagnostics_requests", "nextColumnId": 11, "nextConstraintId": 3, "nextIndexId": 3, "nextMutationId": 1, "parentId": 1, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["id"], "name": "primary", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6, 7, 8, 9, 10], "storeColumnNames": ["completed", "statement_fingerprint", "statement_diagnostics_id", "requested_at", "min_execution_latency", "expires_at", "sampling_probability", "plan_gist", "anti_plan_gist"], "unique": true, "version": 4}, "privileges": {"ownerProto": "node", "users": [{"privileges": "480", "userProto": "admin", "withGrantOption": "480"}, {"privileges": "480", "userProto": "root", "withGrantOption": "480"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 29, "version": "1"}}
36 {"table": {"columns": [{"defaultExpr": "unique_rowid()", "id": 1, "name": "id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "statement_fingerprint", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "statement", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "collected_at", "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 5, "name": "trace", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 6, "name": "bundle_chunks", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 20, "width": 64}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1016, "width": 64}}, {"id": 7, "name": "error", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 36, "name": "statement_diagnostics", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "parentId": 1, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["id"], "name": "primary", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6, 7], "storeColumnNames": ["statement_fingerprint", "statement", "collected_at", "trace", "bundle_chunks", "error"], "unique": true, "version": 4}, "privileges": {"ownerProto": "node", "users": [{"privileges": "480", "userProto": "admin", "withGrantOption": "480"}, {"privileges": "480", "userProto": "root", "withGrantOption": "480"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 29, "version": "1"}}
37 {"table": {"columns": [{"defaultExpr": "unique_rowid()", "id": 1, "name": "schedule_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "schedule_name", "type": {"family": "StringFamily", "oid": 25}}, {"defaultExpr": "now():::TIMESTAMPTZ", "id": 3, "name": "created", "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 4, "name": "owner", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "next_run", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 6, "name": "schedule_state", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 7, "name": "schedule_expr", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "schedule_details", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 9, "name": "executor_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "execution_args", "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 37, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [5], "keyColumnNames": ["next_run"], "keySuffixColumnIds": [1], "name": "next_run_idx", "partitioning": {}, "sharded": {}, "version": 3}], "name": "scheduled_jobs", "nextColumnId": 11, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "parentId": 1, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["schedule_id"], "name": "primary", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6, 7, 8, 9, 10], "storeColumnNames": ["schedule_name", "created", "owner", "next_run", "schedule_state", "schedule_expr", "schedule_details", "executor_type", "execution_args"], "unique": true, "version": 4}, "privileges": {"ownerProto": "node", "users": [{"privileges": "480", "userProto": "admin", "withGrantOption": "480"}, {"privileges": "480", "userProto": "root", "withGrantOption": "480"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 29, "version": "1"}}
39 {"table": {"columns": [{"id": 1, "name": "session_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 2, "name": "expiration", "type": {"family": "DecimalFamily", "oid": 1700}}, {"id": 3, "name": "crdb_region", "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 39, "name": "sqlliveness", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "parentId": 1, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC", "ASC"], "keyColumnIds": [3, 1], "keyColumnNames": ["crdb_region", "session_id"], "name": "primary", "partitioning": {}, "sharded": {}, "storeColumnIds": [2], "storeColumnNames": ["expiration"], "unique": true, "version": 4}, "privileges": {"ownerProto": "node", "users": [{"privileges": "480", "userProto": "admin", "withGrantOption": "480"}, {"privileges": "480", "userProto": "root", "withGrantOption": "480"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 29, "version": "1"}}
diff --git a/pkg/sql/logictest/testdata/logic_test/distsql_stats b/pkg/sql/logictest/testdata/logic_test/distsql_stats
index 88cc2f09fff8..02d9db026792 100644
--- a/pkg/sql/logictest/testdata/logic_test/distsql_stats
+++ b/pkg/sql/logictest/testdata/logic_test/distsql_stats
@@ -1657,8 +1657,8 @@ ANALYZE system.lease
statement error pq: cannot create statistics on system.table_statistics
ANALYZE system.table_statistics
-# Collecting stats on system.jobs is disallowed.
-statement error pq: cannot create statistics on system.jobs
+# Collecting stats on system.jobs is allowed.
+statement ok
ANALYZE system.jobs
# Collecting stats on system.scheduled_jobs is disallowed.
diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema
index 69a4185aafb2..d4f88249b2c6 100644
--- a/pkg/sql/logictest/testdata/logic_test/information_schema
+++ b/pkg/sql/logictest/testdata/logic_test/information_schema
@@ -2534,10 +2534,12 @@ system public statement_diagnostics id
system public statement_diagnostics statement 3
system public statement_diagnostics statement_fingerprint 2
system public statement_diagnostics trace 5
+system public statement_diagnostics_requests anti_plan_gist 10
system public statement_diagnostics_requests completed 2
system public statement_diagnostics_requests expires_at 7
system public statement_diagnostics_requests id 1
system public statement_diagnostics_requests min_execution_latency 6
+system public statement_diagnostics_requests plan_gist 9
system public statement_diagnostics_requests requested_at 5
system public statement_diagnostics_requests sampling_probability 8
system public statement_diagnostics_requests statement_diagnostics_id 4
diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog
index 081c87aaf28c..1288e276a390 100644
--- a/pkg/sql/logictest/testdata/logic_test/pg_catalog
+++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog
@@ -1112,112 +1112,112 @@ SELECT *
FROM pg_catalog.pg_index
ORDER BY indexrelid
----
-indexrelid indrelid indnatts indisunique indnullsnotdistinct indisprimary indisexclusion indimmediate indisclustered indisvalid indcheckxmin indisready indislive indisreplident indkey indcollation indclass indoption indexprs indpred indnkeyatts
-144368028 32 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1
-190763692 48 1 false false true false false false true false false true false 1 0 0 2 NULL NULL 1
-404104296 39 2 true false true false true false true false false true false 3 1 0 0 0 0 2 2 NULL NULL 2
-450499960 55 1 false false false false false false true false false true false 2 0 0 2 NULL NULL 1
-450499963 55 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1
-496895627 7 1 false false true false false false true false false true false 1 0 0 2 NULL NULL 1
-543291288 23 1 false false false false false false true false false true false 1 3403232968 0 2 NULL NULL 1
-543291289 23 1 false false false false false false true false false true false 2 3403232968 0 2 NULL NULL 1
-543291291 23 2 true false true false true false true false false true false 1 2 3403232968 3403232968 0 0 2 2 NULL NULL 2
-543291292 23 2 true false false false true false true false false true false 4 5 0 0 0 0 2 2 NULL NULL 2
-543291294 23 1 false false false false false false true false false true false 4 0 0 2 NULL NULL 1
-543291295 23 1 false false false false false false true false false true false 5 0 0 2 NULL NULL 1
-663840560 42 3 false false false false false false true false false true false 1 5 17 0 3403232968 0 0 0 0 2 2 1 NULL app_name NOT LIKE '$ internal%'::STRING 3
-663840561 42 3 false false false false false false true false false true false 1 5 16 0 3403232968 0 0 0 0 2 2 1 NULL app_name NOT LIKE '$ internal%'::STRING 3
-663840562 42 3 false false false false false false true false false true false 1 5 15 0 3403232968 0 0 0 0 2 2 1 NULL app_name NOT LIKE '$ internal%'::STRING 3
-663840563 42 3 false false false false false false true false false true false 1 5 14 0 3403232968 0 0 0 0 2 2 1 NULL app_name NOT LIKE '$ internal%'::STRING 3
-663840564 42 1 false false false false false false true false false true false 13 0 0 2 NULL NULL 1
-663840565 42 2 false false false false false false true false false true false 2 3 0 0 0 0 2 2 NULL NULL 2
-663840566 42 6 true false true false true false true false false true false 1 2 3 4 5 6 0 0 0 0 3403232968 0 0 0 0 0 0 0 2 2 2 2 2 2 NULL NULL 6
-663840574 42 3 false false false false false false true false false true false 1 5 19 0 3403232968 0 0 0 0 2 2 1 NULL app_name NOT LIKE '$ internal%'::STRING 3
-663840575 42 3 false false false false false false true false false true false 1 5 18 0 3403232968 0 0 0 0 2 2 1 NULL app_name NOT LIKE '$ internal%'::STRING 3
-710236230 58 1 true false true false true false true false false true false 1 3403232968 0 2 NULL NULL 1
-803027558 26 3 true false true false true false true false false true false 1 2 3 0 0 3403232968 0 0 0 2 2 2 NULL NULL 3
-923576837 41 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1
-969972501 57 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1
-1062763829 25 4 true false true false true false true false false true false 1 2 3 4 0 0 3403232968 3403232968 0 0 0 0 2 2 2 2 NULL NULL 4
-1183313104 44 2 true false true false true false true false false true false 1 2 0 3403232968 0 0 2 2 NULL NULL 2
-1183313107 44 3 true false false false true false true false false true false 1 4 3 0 0 0 0 2 2 NULL NULL 2
-1229708768 60 5 true false true false true false true false false true false 1 2 3 4 5 0 0 0 0 3403232968 0 0 0 0 0 2 2 2 2 2 NULL NULL 5
-1229708770 60 2 false false false false false false true false false true false 1 11 0 0 0 0 2 1 NULL NULL 2
-1229708771 60 2 false false false false false false true false false true false 2 3 0 0 0 0 2 2 NULL NULL 2
-1229708772 60 2 false false false false false false true false false true false 1 14 0 0 0 0 2 1 NULL NULL 2
-1229708773 60 2 false false false false false false true false false true false 1 12 0 0 0 0 2 1 NULL NULL 2
-1229708774 60 2 false false false false false false true false false true false 1 16 0 0 0 0 2 1 NULL NULL 2
-1229708775 60 2 false false false false false false true false false true false 1 15 0 0 0 0 2 1 NULL NULL 2
-1229708777 60 2 false false false false false false true false false true false 1 17 0 0 0 0 2 1 NULL NULL 2
-1276104432 12 2 true false true false true false true false false true false 1 6 0 0 0 0 2 2 NULL NULL 2
-1322500096 28 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1
-1489445036 35 6 false false false false false false true false false true false 2 1 3 6 7 8 0 0 0 0 2 2 NULL NULL 2
-1489445039 35 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1
-1535840700 51 4 true false false false true false true false false true false 2 5 3 4 3403232968 0 0 0 2 2 NULL NULL 2
-1535840701 51 4 true false false false true false true false false true false 2 1 3 4 3403232968 3403232968 0 0 2 2 NULL NULL 2
-1535840703 51 2 true false true false true false true false false true false 1 2 3403232968 3403232968 0 0 2 2 NULL NULL 2
-1582236367 3 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1
-1628632026 19 1 false false false false false false true false false true false 6 0 0 2 NULL NULL 1
-1628632027 19 1 false false false false false false true false false true false 7 0 0 2 NULL NULL 1
-1628632028 19 1 false false false false false false true false false true false 5 0 0 2 NULL NULL 1
-1628632029 19 1 false false false false false false true false false true false 4 0 0 2 NULL NULL 1
-1628632031 19 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1
-1795576969 54 1 true false false false true false true false false true false 2 0 0 2 NULL NULL 1
-1795576970 54 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1
-1841972634 6 1 true false true false true false true false false true false 1 3403232968 0 2 NULL NULL 1
-2008917577 37 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1
-2008917578 37 1 false false false false false false true false false true false 5 0 0 2 NULL NULL 1
-2055313241 53 3 true false true false true false true false false true false 1 2 3 0 3403232968 0 0 0 0 2 2 1 NULL NULL 3
-2101708905 5 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1
-2148104569 21 2 true false true false true false true false false true false 1 2 3403232968 3403232968 0 0 2 2 NULL NULL 2
-2268653844 40 4 true false true false true false true false false true false 1 2 3 4 0 0 0 0 0 0 0 0 2 2 2 2 NULL NULL 4
-2315049508 56 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1
-2315049511 56 1 true false false false true false true false false true false 2 0 0 2 NULL NULL 1
-2361445172 8 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1
-2361445174 8 1 false false false false false false true false false true false 6 0 0 2 NULL NULL 1
-2361445175 8 1 true false false false true false true false false true false 4 3403232968 0 2 NULL NULL 1
-2407840836 24 3 true false true false true false true false false true false 1 2 3 0 0 0 0 0 0 2 2 2 NULL NULL 3
-2528390115 47 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1
-2621181440 15 2 false false false false false false true false false true false 2 3 3403232968 0 0 0 2 2 NULL NULL 2
-2621181441 15 3 false false false false false false true false false true false 6 7 2 3403232968 0 0 0 2 2 NULL NULL 2
-2621181443 15 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1
-2621181446 15 6 false false false false false false true false false true false 8 2 3 11 10 9 0 3403232968 0 0 0 0 2 2 2 NULL status IN ('running'::STRING, 'reverting'::STRING, 'pending'::STRING, 'pause-requested'::STRING, 'cancel-requested'::STRING) 3
-2621181447 15 1 false false false false false false true false false true false 12 3403232968 0 2 NULL NULL 1
-2667577107 31 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1
-2834522046 34 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1
-2880917710 50 2 true false true false true false true false false true false 1 2 0 3403232968 0 0 2 2 NULL NULL 2
-3094258317 33 2 true false true false true false true false false true false 1 2 3403232968 3403232968 0 0 2 2 NULL NULL 2
-3094258318 33 1 false false false false false false true false false true false 4 0 0 2 NULL NULL 1
-3353994584 36 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1
-3400390248 52 1 true false true false true false true false false true false 1 3403232968 0 2 NULL NULL 1
-3446785912 4 1 true false true false true false true false false true false 1 3403232968 0 2 NULL NULL 1
-3446785915 4 1 true false false false true false true false false true false 4 0 0 2 NULL NULL 1
-3493181576 20 2 true false true false true false true false false true false 1 2 0 0 0 0 2 2 NULL NULL 2
-3613730848 43 3 false false false false false false true false false true false 1 3 12 0 3403232968 0 0 0 0 2 2 1 NULL app_name NOT LIKE '$ internal%'::STRING 3
-3613730849 43 3 false false false false false false true false false true false 1 3 13 0 3403232968 0 0 0 0 2 2 1 NULL app_name NOT LIKE '$ internal%'::STRING 3
-3613730850 43 3 false false false false false false true false false true false 1 3 10 0 3403232968 0 0 0 0 2 2 1 NULL app_name NOT LIKE '$ internal%'::STRING 3
-3613730851 43 3 false false false false false false true false false true false 1 3 11 0 3403232968 0 0 0 0 2 2 1 NULL app_name NOT LIKE '$ internal%'::STRING 3
-3613730852 43 1 false false false false false false true false false true false 2 0 0 2 NULL NULL 1
-3613730853 43 3 false false false false false false true false false true false 1 3 9 0 3403232968 0 0 0 0 2 2 1 NULL app_name NOT LIKE '$ internal%'::STRING 3
-3613730855 43 4 true false true false true false true false false true false 1 2 3 4 0 0 3403232968 0 0 0 0 0 2 2 2 2 NULL NULL 4
-3613730862 43 3 false false false false false false true false false true false 1 3 14 0 3403232968 0 0 0 0 2 2 1 NULL app_name NOT LIKE '$ internal%'::STRING 3
-3660126519 59 3 true false true false true false true false false true false 1 2 3 0 3403232968 0 0 0 0 2 2 2 NULL NULL 3
-3706522180 11 5 true false true false true false true false false true false 5 1 2 4 3 0 0 0 0 0 0 0 0 0 0 2 2 2 2 2 NULL NULL 5
-3752917847 27 2 true false true false true false true false false true false 1 2 0 0 0 0 2 2 NULL NULL 2
-3873467121 46 2 true false true false true false true false false true false 6 1 0 0 0 0 2 2 NULL NULL 2
-3919862786 62 1 false false true false false false true false false true false 1 0 0 2 NULL NULL 1
-3966258450 14 1 true false true false true false true false false true false 1 3403232968 0 2 NULL NULL 1
-4012654114 30 3 true false true false true false true false false true false 1 2 3 0 0 3403232968 0 0 0 2 2 2 NULL NULL 3
-4133203393 45 2 true false true false true false true false false true false 1 2 0 0 0 0 2 2 NULL NULL 2
-4179599057 61 3 true false true false true false true false false true false 1 2 3 0 0 3403232968 0 0 0 2 2 2 NULL NULL 3
-4179599058 61 1 false false false false false false true false false true false 2 0 0 2 NULL NULL 1
-4179599059 61 2 false false false false false false true false false true false 1 8 0 0 0 0 2 1 NULL NULL 2
-4179599060 61 2 false false false false false false true false false true false 1 9 0 0 0 0 2 1 NULL NULL 2
-4179599061 61 2 false false false false false false true false false true false 1 11 0 0 0 0 2 1 NULL NULL 2
-4179599062 61 2 false false false false false false true false false true false 1 12 0 0 0 0 2 1 NULL NULL 2
-4179599063 61 2 false false false false false false true false false true false 1 13 0 0 0 0 2 1 NULL NULL 2
-4179599064 61 2 false false false false false false true false false true false 1 14 0 0 0 0 2 1 NULL NULL 2
-4225994721 13 2 true false true false true false true false false true false 1 7 0 0 0 0 2 2 NULL NULL 2
+indexrelid indrelid indnatts indisunique indnullsnotdistinct indisprimary indisexclusion indimmediate indisclustered indisvalid indcheckxmin indisready indislive indisreplident indkey indcollation indclass indoption indexprs indpred indnkeyatts
+144368028 32 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1
+190763692 48 1 false false true false false false true false false true false 1 0 0 2 NULL NULL 1
+404104296 39 2 true false true false true false true false false true false 3 1 0 0 0 0 2 2 NULL NULL 2
+450499960 55 1 false false false false false false true false false true false 2 0 0 2 NULL NULL 1
+450499963 55 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1
+496895627 7 1 false false true false false false true false false true false 1 0 0 2 NULL NULL 1
+543291288 23 1 false false false false false false true false false true false 1 3403232968 0 2 NULL NULL 1
+543291289 23 1 false false false false false false true false false true false 2 3403232968 0 2 NULL NULL 1
+543291291 23 2 true false true false true false true false false true false 1 2 3403232968 3403232968 0 0 2 2 NULL NULL 2
+543291292 23 2 true false false false true false true false false true false 4 5 0 0 0 0 2 2 NULL NULL 2
+543291294 23 1 false false false false false false true false false true false 4 0 0 2 NULL NULL 1
+543291295 23 1 false false false false false false true false false true false 5 0 0 2 NULL NULL 1
+663840560 42 3 false false false false false false true false false true false 1 5 17 0 3403232968 0 0 0 0 2 2 1 NULL app_name NOT LIKE '$ internal%'::STRING 3
+663840561 42 3 false false false false false false true false false true false 1 5 16 0 3403232968 0 0 0 0 2 2 1 NULL app_name NOT LIKE '$ internal%'::STRING 3
+663840562 42 3 false false false false false false true false false true false 1 5 15 0 3403232968 0 0 0 0 2 2 1 NULL app_name NOT LIKE '$ internal%'::STRING 3
+663840563 42 3 false false false false false false true false false true false 1 5 14 0 3403232968 0 0 0 0 2 2 1 NULL app_name NOT LIKE '$ internal%'::STRING 3
+663840564 42 1 false false false false false false true false false true false 13 0 0 2 NULL NULL 1
+663840565 42 2 false false false false false false true false false true false 2 3 0 0 0 0 2 2 NULL NULL 2
+663840566 42 6 true false true false true false true false false true false 1 2 3 4 5 6 0 0 0 0 3403232968 0 0 0 0 0 0 0 2 2 2 2 2 2 NULL NULL 6
+663840574 42 3 false false false false false false true false false true false 1 5 19 0 3403232968 0 0 0 0 2 2 1 NULL app_name NOT LIKE '$ internal%'::STRING 3
+663840575 42 3 false false false false false false true false false true false 1 5 18 0 3403232968 0 0 0 0 2 2 1 NULL app_name NOT LIKE '$ internal%'::STRING 3
+710236230 58 1 true false true false true false true false false true false 1 3403232968 0 2 NULL NULL 1
+803027558 26 3 true false true false true false true false false true false 1 2 3 0 0 3403232968 0 0 0 2 2 2 NULL NULL 3
+923576837 41 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1
+969972501 57 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1
+1062763829 25 4 true false true false true false true false false true false 1 2 3 4 0 0 3403232968 3403232968 0 0 0 0 2 2 2 2 NULL NULL 4
+1183313104 44 2 true false true false true false true false false true false 1 2 0 3403232968 0 0 2 2 NULL NULL 2
+1183313107 44 3 true false false false true false true false false true false 1 4 3 0 0 0 0 2 2 NULL NULL 2
+1229708768 60 5 true false true false true false true false false true false 1 2 3 4 5 0 0 0 0 3403232968 0 0 0 0 0 2 2 2 2 2 NULL NULL 5
+1229708770 60 2 false false false false false false true false false true false 1 11 0 0 0 0 2 1 NULL NULL 2
+1229708771 60 2 false false false false false false true false false true false 2 3 0 0 0 0 2 2 NULL NULL 2
+1229708772 60 2 false false false false false false true false false true false 1 14 0 0 0 0 2 1 NULL NULL 2
+1229708773 60 2 false false false false false false true false false true false 1 12 0 0 0 0 2 1 NULL NULL 2
+1229708774 60 2 false false false false false false true false false true false 1 16 0 0 0 0 2 1 NULL NULL 2
+1229708775 60 2 false false false false false false true false false true false 1 15 0 0 0 0 2 1 NULL NULL 2
+1229708777 60 2 false false false false false false true false false true false 1 17 0 0 0 0 2 1 NULL NULL 2
+1276104432 12 2 true false true false true false true false false true false 1 6 0 0 0 0 2 2 NULL NULL 2
+1322500096 28 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1
+1489445036 35 8 false false false false false false true false false true false 2 1 3 6 7 8 9 10 0 0 0 0 2 2 NULL NULL 2
+1489445039 35 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1
+1535840700 51 4 true false false false true false true false false true false 2 5 3 4 3403232968 0 0 0 2 2 NULL NULL 2
+1535840701 51 4 true false false false true false true false false true false 2 1 3 4 3403232968 3403232968 0 0 2 2 NULL NULL 2
+1535840703 51 2 true false true false true false true false false true false 1 2 3403232968 3403232968 0 0 2 2 NULL NULL 2
+1582236367 3 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1
+1628632026 19 1 false false false false false false true false false true false 6 0 0 2 NULL NULL 1
+1628632027 19 1 false false false false false false true false false true false 7 0 0 2 NULL NULL 1
+1628632028 19 1 false false false false false false true false false true false 5 0 0 2 NULL NULL 1
+1628632029 19 1 false false false false false false true false false true false 4 0 0 2 NULL NULL 1
+1628632031 19 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1
+1795576969 54 1 true false false false true false true false false true false 2 0 0 2 NULL NULL 1
+1795576970 54 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1
+1841972634 6 1 true false true false true false true false false true false 1 3403232968 0 2 NULL NULL 1
+2008917577 37 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1
+2008917578 37 1 false false false false false false true false false true false 5 0 0 2 NULL NULL 1
+2055313241 53 3 true false true false true false true false false true false 1 2 3 0 3403232968 0 0 0 0 2 2 1 NULL NULL 3
+2101708905 5 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1
+2148104569 21 2 true false true false true false true false false true false 1 2 3403232968 3403232968 0 0 2 2 NULL NULL 2
+2268653844 40 4 true false true false true false true false false true false 1 2 3 4 0 0 0 0 0 0 0 0 2 2 2 2 NULL NULL 4
+2315049508 56 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1
+2315049511 56 1 true false false false true false true false false true false 2 0 0 2 NULL NULL 1
+2361445172 8 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1
+2361445174 8 1 false false false false false false true false false true false 6 0 0 2 NULL NULL 1
+2361445175 8 1 true false false false true false true false false true false 4 3403232968 0 2 NULL NULL 1
+2407840836 24 3 true false true false true false true false false true false 1 2 3 0 0 0 0 0 0 2 2 2 NULL NULL 3
+2528390115 47 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1
+2621181440 15 2 false false false false false false true false false true false 2 3 3403232968 0 0 0 2 2 NULL NULL 2
+2621181441 15 3 false false false false false false true false false true false 6 7 2 3403232968 0 0 0 2 2 NULL NULL 2
+2621181443 15 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1
+2621181446 15 6 false false false false false false true false false true false 8 2 3 11 10 9 0 3403232968 0 0 0 0 2 2 2 NULL status IN ('running'::STRING, 'reverting'::STRING, 'pending'::STRING, 'pause-requested'::STRING, 'cancel-requested'::STRING) 3
+2621181447 15 1 false false false false false false true false false true false 12 3403232968 0 2 NULL NULL 1
+2667577107 31 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1
+2834522046 34 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1
+2880917710 50 2 true false true false true false true false false true false 1 2 0 3403232968 0 0 2 2 NULL NULL 2
+3094258317 33 2 true false true false true false true false false true false 1 2 3403232968 3403232968 0 0 2 2 NULL NULL 2
+3094258318 33 1 false false false false false false true false false true false 4 0 0 2 NULL NULL 1
+3353994584 36 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1
+3400390248 52 1 true false true false true false true false false true false 1 3403232968 0 2 NULL NULL 1
+3446785912 4 1 true false true false true false true false false true false 1 3403232968 0 2 NULL NULL 1
+3446785915 4 1 true false false false true false true false false true false 4 0 0 2 NULL NULL 1
+3493181576 20 2 true false true false true false true false false true false 1 2 0 0 0 0 2 2 NULL NULL 2
+3613730848 43 3 false false false false false false true false false true false 1 3 12 0 3403232968 0 0 0 0 2 2 1 NULL app_name NOT LIKE '$ internal%'::STRING 3
+3613730849 43 3 false false false false false false true false false true false 1 3 13 0 3403232968 0 0 0 0 2 2 1 NULL app_name NOT LIKE '$ internal%'::STRING 3
+3613730850 43 3 false false false false false false true false false true false 1 3 10 0 3403232968 0 0 0 0 2 2 1 NULL app_name NOT LIKE '$ internal%'::STRING 3
+3613730851 43 3 false false false false false false true false false true false 1 3 11 0 3403232968 0 0 0 0 2 2 1 NULL app_name NOT LIKE '$ internal%'::STRING 3
+3613730852 43 1 false false false false false false true false false true false 2 0 0 2 NULL NULL 1
+3613730853 43 3 false false false false false false true false false true false 1 3 9 0 3403232968 0 0 0 0 2 2 1 NULL app_name NOT LIKE '$ internal%'::STRING 3
+3613730855 43 4 true false true false true false true false false true false 1 2 3 4 0 0 3403232968 0 0 0 0 0 2 2 2 2 NULL NULL 4
+3613730862 43 3 false false false false false false true false false true false 1 3 14 0 3403232968 0 0 0 0 2 2 1 NULL app_name NOT LIKE '$ internal%'::STRING 3
+3660126519 59 3 true false true false true false true false false true false 1 2 3 0 3403232968 0 0 0 0 2 2 2 NULL NULL 3
+3706522180 11 5 true false true false true false true false false true false 5 1 2 4 3 0 0 0 0 0 0 0 0 0 0 2 2 2 2 2 NULL NULL 5
+3752917847 27 2 true false true false true false true false false true false 1 2 0 0 0 0 2 2 NULL NULL 2
+3873467121 46 2 true false true false true false true false false true false 6 1 0 0 0 0 2 2 NULL NULL 2
+3919862786 62 1 false false true false false false true false false true false 1 0 0 2 NULL NULL 1
+3966258450 14 1 true false true false true false true false false true false 1 3403232968 0 2 NULL NULL 1
+4012654114 30 3 true false true false true false true false false true false 1 2 3 0 0 3403232968 0 0 0 2 2 2 NULL NULL 3
+4133203393 45 2 true false true false true false true false false true false 1 2 0 0 0 0 2 2 NULL NULL 2
+4179599057 61 3 true false true false true false true false false true false 1 2 3 0 0 3403232968 0 0 0 2 2 2 NULL NULL 3
+4179599058 61 1 false false false false false false true false false true false 2 0 0 2 NULL NULL 1
+4179599059 61 2 false false false false false false true false false true false 1 8 0 0 0 0 2 1 NULL NULL 2
+4179599060 61 2 false false false false false false true false false true false 1 9 0 0 0 0 2 1 NULL NULL 2
+4179599061 61 2 false false false false false false true false false true false 1 11 0 0 0 0 2 1 NULL NULL 2
+4179599062 61 2 false false false false false false true false false true false 1 12 0 0 0 0 2 1 NULL NULL 2
+4179599063 61 2 false false false false false false true false false true false 1 13 0 0 0 0 2 1 NULL NULL 2
+4179599064 61 2 false false false false false false true false false true false 1 14 0 0 0 0 2 1 NULL NULL 2
+4225994721 13 2 true false true false true false true false false true false 1 7 0 0 0 0 2 2 NULL NULL 2
# From #26504
skipif config 3node-tenant-default-configs
diff --git a/pkg/sql/plan.go b/pkg/sql/plan.go
index d5a3f156ba80..2a8bf5e19360 100644
--- a/pkg/sql/plan.go
+++ b/pkg/sql/plan.go
@@ -304,7 +304,8 @@ type planTop struct {
planComponents
// mem/catalog retains the memo and catalog that were used to create the
- // plan. Only set if needed by instrumentation (see ShouldSaveMemo).
+ // plan. Set unconditionally but used only by instrumentation (in order to
+ // build the stmt bundle).
mem *memo.Memo
catalog optPlanningCatalog
diff --git a/pkg/sql/plan_opt.go b/pkg/sql/plan_opt.go
index d5bf0808886d..e62796b5c92e 100644
--- a/pkg/sql/plan_opt.go
+++ b/pkg/sql/plan_opt.go
@@ -713,10 +713,8 @@ func (opc *optPlanningCtx) runExecBuilder(
if bld.ContainsNonDefaultKeyLocking {
planTop.flags.Set(planFlagContainsNonDefaultLocking)
}
- if planTop.instrumentation.ShouldSaveMemo() {
- planTop.mem = mem
- planTop.catalog = opc.catalog
- }
+ planTop.mem = mem
+ planTop.catalog = opc.catalog
return nil
}
diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go
index 9cdc32adb71e..667d0f125016 100644
--- a/pkg/sql/sem/builtins/builtins.go
+++ b/pkg/sql/sem/builtins/builtins.go
@@ -7744,65 +7744,9 @@ specified store on the node it's run from. One of 'mvccGC', 'merge', 'split',
Category: builtinconstants.CategorySystemInfo,
DistsqlBlocklist: true, // applicable only on the gateway
},
- tree.Overload{
- Types: tree.ParamTypes{
- {Name: "stmtFingerprint", Typ: types.String},
- {Name: "samplingProbability", Typ: types.Float},
- {Name: "minExecutionLatency", Typ: types.Interval},
- {Name: "expiresAfter", Typ: types.Interval},
- },
- ReturnType: tree.FixedReturnType(types.Bool),
- Fn: func(ctx context.Context, evalCtx *eval.Context, args tree.Datums) (tree.Datum, error) {
- hasViewActivity, err := evalCtx.SessionAccessor.HasRoleOption(
- ctx, roleoption.VIEWACTIVITY)
- if err != nil {
- return nil, err
- }
-
- if !hasViewActivity {
- return nil, errors.New("requesting statement bundle requires " +
- "VIEWACTIVITY or ADMIN role option")
- }
-
- isAdmin, err := evalCtx.SessionAccessor.HasAdminRole(ctx)
- if err != nil {
- return nil, err
- }
-
- hasViewActivityRedacted, err := evalCtx.SessionAccessor.HasRoleOption(
- ctx, roleoption.VIEWACTIVITYREDACTED)
- if err != nil {
- return nil, err
- }
-
- if !isAdmin && hasViewActivityRedacted {
- return nil, errors.New("VIEWACTIVITYREDACTED role option cannot request " +
- "statement bundle")
- }
-
- stmtFingerprint := string(tree.MustBeDString(args[0]))
- samplingProbability := float64(tree.MustBeDFloat(args[1]))
- minExecutionLatency := time.Duration(tree.MustBeDInterval(args[2]).Nanos())
- expiresAfter := time.Duration(tree.MustBeDInterval(args[3]).Nanos())
-
- if err := evalCtx.StmtDiagnosticsRequestInserter(
- ctx,
- stmtFingerprint,
- samplingProbability,
- minExecutionLatency,
- expiresAfter,
- ); err != nil {
- return nil, err
- }
-
- return tree.DBoolTrue, nil
- },
- Volatility: volatility.Volatile,
- Info: `Used to request statement bundle for a given statement fingerprint
-that has execution latency greater than the 'minExecutionLatency'. If the
-'expiresAfter' argument is empty, then the statement bundle request never
-expires until the statement bundle is collected`,
- },
+ makeRequestStatementBundleBuiltinOverload(false /* withPlanGist */, false /* withAntiPlanGist */),
+ makeRequestStatementBundleBuiltinOverload(true /* withPlanGist */, false /* withAntiPlanGist */),
+ makeRequestStatementBundleBuiltinOverload(true /* withPlanGist */, true /* withAntiPlanGist */),
),
"crdb_internal.set_compaction_concurrency": makeBuiltin(
@@ -10937,3 +10881,110 @@ func spanToDatum(span roachpb.Span) (tree.Datum, error) {
}
return result, nil
}
+
+func makeRequestStatementBundleBuiltinOverload(
+ withPlanGist bool, withAntiPlanGist bool,
+) tree.Overload {
+ typs := tree.ParamTypes{{Name: "stmtFingerprint", Typ: types.String}}
+ lastTyps := tree.ParamTypes{
+ {Name: "samplingProbability", Typ: types.Float},
+ {Name: "minExecutionLatency", Typ: types.Interval},
+ {Name: "expiresAfter", Typ: types.Interval},
+ }
+ info := `Used to request statement bundle for a given statement fingerprint
+that has execution latency greater than the 'minExecutionLatency'. If the
+'expiresAfter' argument is empty, then the statement bundle request never
+expires until the statement bundle is collected`
+ if withPlanGist {
+ typs = append(typs, tree.ParamType{Name: "planGist", Typ: types.String})
+ info += `. If 'planGist' argument is
+not empty, then only the execution of the statement with the matching plan
+will be used`
+ if withAntiPlanGist {
+ typs = append(typs, tree.ParamType{Name: "antiPlanGist", Typ: types.Bool})
+ info += `. If 'antiPlanGist' argument is
+true, then any plan other then the specified gist will be used`
+ }
+ }
+ typs = append(typs, lastTyps...)
+ return tree.Overload{
+ Types: typs,
+ ReturnType: tree.FixedReturnType(types.Bool),
+ Fn: func(ctx context.Context, evalCtx *eval.Context, args tree.Datums) (tree.Datum, error) {
+ hasViewActivity, err := evalCtx.SessionAccessor.HasRoleOption(
+ ctx, roleoption.VIEWACTIVITY)
+ if err != nil {
+ return nil, err
+ }
+
+ if !hasViewActivity {
+ return nil, errors.New("requesting statement bundle requires " +
+ "VIEWACTIVITY or ADMIN role option")
+ }
+
+ isAdmin, err := evalCtx.SessionAccessor.HasAdminRole(ctx)
+ if err != nil {
+ return nil, err
+ }
+
+ hasViewActivityRedacted, err := evalCtx.SessionAccessor.HasRoleOption(
+ ctx, roleoption.VIEWACTIVITYREDACTED)
+ if err != nil {
+ return nil, err
+ }
+
+ if !isAdmin && hasViewActivityRedacted {
+ return nil, errors.New("VIEWACTIVITYREDACTED role option cannot request " +
+ "statement bundle")
+ }
+
+ if args[0] == tree.DNull {
+ return nil, errors.New("stmtFingerprint must be non-NULL")
+ }
+
+ stmtFingerprint := string(tree.MustBeDString(args[0]))
+ var planGist string
+ var antiPlanGist bool
+ spIdx, melIdx, eaIdx := 1, 2, 3
+ if withPlanGist {
+ if args[1] != tree.DNull {
+ planGist = string(tree.MustBeDString(args[1]))
+ }
+ spIdx, melIdx, eaIdx = 2, 3, 4
+ if withAntiPlanGist {
+ if args[2] != tree.DNull {
+ antiPlanGist = bool(tree.MustBeDBool(args[2]))
+ }
+ spIdx, melIdx, eaIdx = 3, 4, 5
+ }
+ }
+ var samplingProbability float64
+ if args[spIdx] != tree.DNull {
+ samplingProbability = float64(tree.MustBeDFloat(args[spIdx]))
+ }
+ var minExecutionLatency, expiresAfter time.Duration
+ if args[melIdx] != tree.DNull {
+ minExecutionLatency = time.Duration(tree.MustBeDInterval(args[melIdx]).Nanos())
+ }
+ if args[eaIdx] != tree.DNull {
+ expiresAfter = time.Duration(tree.MustBeDInterval(args[eaIdx]).Nanos())
+ }
+
+ if err = evalCtx.StmtDiagnosticsRequestInserter(
+ ctx,
+ stmtFingerprint,
+ planGist,
+ antiPlanGist,
+ samplingProbability,
+ minExecutionLatency,
+ expiresAfter,
+ ); err != nil {
+ return nil, err
+ }
+
+ return tree.DBoolTrue, nil
+ },
+ Volatility: volatility.Volatile,
+ Info: info,
+ }
+}
diff --git a/pkg/sql/sem/builtins/fixed_oids.go b/pkg/sql/sem/builtins/fixed_oids.go
index c7f2580f7ac0..26486974f306 100644
--- a/pkg/sql/sem/builtins/fixed_oids.go
+++ b/pkg/sql/sem/builtins/fixed_oids.go
@@ -2437,6 +2437,8 @@ var builtinOidsArray = []string{
2464: `workload_index_recs(budget: string) -> string`,
2465: `workload_index_recs(timestamptz: timestamptz, budget: string) -> string`,
2466: `crdb_internal.setup_span_configs_stream(tenant_name: string) -> bytes`,
+ 2467: `crdb_internal.request_statement_bundle(stmtFingerprint: string, planGist: string, samplingProbability: float, minExecutionLatency: interval, expiresAfter: interval) -> bool`,
+ 2468: `crdb_internal.request_statement_bundle(stmtFingerprint: string, planGist: string, antiPlanGist: bool, samplingProbability: float, minExecutionLatency: interval, expiresAfter: interval) -> bool`,
}
var builtinOidsBySignature map[string]oid.Oid
diff --git a/pkg/sql/sem/eval/deps.go b/pkg/sql/sem/eval/deps.go
index 8deef15f3245..d16ca07aaff6 100644
--- a/pkg/sql/sem/eval/deps.go
+++ b/pkg/sql/sem/eval/deps.go
@@ -680,6 +680,8 @@ type IndexUsageStatsController interface {
type StmtDiagnosticsRequestInsertFunc func(
ctx context.Context,
stmtFingerprint string,
+ planGist string,
+ antiPlanGist bool,
samplingProbability float64,
minExecutionLatency time.Duration,
expiresAfter time.Duration,
diff --git a/pkg/sql/stmtdiagnostics/BUILD.bazel b/pkg/sql/stmtdiagnostics/BUILD.bazel
index bfbd1698b0f0..0fecab86bca6 100644
--- a/pkg/sql/stmtdiagnostics/BUILD.bazel
+++ b/pkg/sql/stmtdiagnostics/BUILD.bazel
@@ -49,6 +49,7 @@ go_test(
"//pkg/sql/sqlerrors",
"//pkg/testutils",
"//pkg/testutils/serverutils",
+ "//pkg/testutils/sqlutils",
"//pkg/testutils/testcluster",
"//pkg/util/leaktest",
"//pkg/util/log",
diff --git a/pkg/sql/stmtdiagnostics/statement_diagnostics.go b/pkg/sql/stmtdiagnostics/statement_diagnostics.go
index b2cfe427a5d7..1c5683bf1800 100644
--- a/pkg/sql/stmtdiagnostics/statement_diagnostics.go
+++ b/pkg/sql/stmtdiagnostics/statement_diagnostics.go
@@ -114,6 +114,8 @@ type Registry struct {
// information.
type Request struct {
fingerprint string
+ planGist string
+ antiPlanGist bool
samplingProbability float64
minExecutionLatency time.Duration
expiresAt time.Time
@@ -229,6 +231,8 @@ func (r *Registry) addRequestInternalLocked(
ctx context.Context,
id RequestID,
queryFingerprint string,
+ planGist string,
+ antiPlanGist bool,
samplingProbability float64,
minExecutionLatency time.Duration,
expiresAt time.Time,
@@ -242,6 +246,8 @@ func (r *Registry) addRequestInternalLocked(
}
r.mu.requestFingerprints[id] = Request{
fingerprint: queryFingerprint,
+ planGist: planGist,
+ antiPlanGist: antiPlanGist,
samplingProbability: samplingProbability,
minExecutionLatency: minExecutionLatency,
expiresAt: expiresAt,
@@ -273,30 +279,31 @@ func (r *Registry) cancelRequest(requestID RequestID) {
delete(r.mu.unconditionalOngoing, requestID)
}
-// InsertRequest is part of the StmtDiagnosticsRequester interface.
+// InsertRequest is part of the server.StmtDiagnosticsRequester interface.
func (r *Registry) InsertRequest(
ctx context.Context,
stmtFingerprint string,
+ planGist string,
+ antiPlanGist bool,
samplingProbability float64,
minExecutionLatency time.Duration,
expiresAfter time.Duration,
) error {
- _, err := r.insertRequestInternal(ctx, stmtFingerprint, samplingProbability, minExecutionLatency, expiresAfter)
+ _, err := r.insertRequestInternal(ctx, stmtFingerprint, planGist, antiPlanGist, samplingProbability, minExecutionLatency, expiresAfter)
return err
}
func (r *Registry) insertRequestInternal(
ctx context.Context,
stmtFingerprint string,
+ planGist string,
+ antiPlanGist bool,
samplingProbability float64,
minExecutionLatency time.Duration,
expiresAfter time.Duration,
) (RequestID, error) {
- isSamplingProbabilitySupported := r.st.Version.IsActive(ctx, clusterversion.TODODelete_V22_2SampledStmtDiagReqs)
- if !isSamplingProbabilitySupported && samplingProbability != 0 {
- return 0, errors.New(
- "sampling probability only supported after 22.2 version migrations have completed",
- )
+ if planGist != "" && !r.st.Version.IsActive(ctx, clusterversion.V23_2_StmtDiagForPlanGist) {
+ return 0, errors.Newf("plan gists only supported after 23.2 version migrations have completed")
}
if samplingProbability != 0 {
if samplingProbability < 0 || samplingProbability > 1 {
@@ -340,9 +347,14 @@ func (r *Registry) insertRequestInternal(
now := timeutil.Now()
insertColumns := "statement_fingerprint, requested_at"
- qargs := make([]interface{}, 2, 5)
+ qargs := make([]interface{}, 2, 7)
qargs[0] = stmtFingerprint // statement_fingerprint
qargs[1] = now // requested_at
+ if planGist != "" {
+ insertColumns += ", plan_gist, anti_plan_gist"
+ qargs = append(qargs, planGist) // plan_gist
+ qargs = append(qargs, antiPlanGist) // anti_plan_gist
+ }
if samplingProbability != 0 {
insertColumns += ", sampling_probability"
qargs = append(qargs, samplingProbability) // sampling_probability
@@ -387,7 +399,7 @@ func (r *Registry) insertRequestInternal(
r.mu.Lock()
defer r.mu.Unlock()
r.mu.epoch++
- r.addRequestInternalLocked(ctx, reqID, stmtFingerprint, samplingProbability, minExecutionLatency, expiresAt)
+ r.addRequestInternalLocked(ctx, reqID, stmtFingerprint, planGist, antiPlanGist, samplingProbability, minExecutionLatency, expiresAt)
}()
return reqID, nil
@@ -454,7 +466,7 @@ func (r *Registry) MaybeRemoveRequest(requestID RequestID, req Request, execLate
//
// If shouldCollect is true, MaybeRemoveRequest needs to be called.
func (r *Registry) ShouldCollectDiagnostics(
- ctx context.Context, fingerprint string,
+ ctx context.Context, fingerprint string, planGist string,
) (shouldCollect bool, reqID RequestID, req Request) {
r.mu.Lock()
defer r.mu.Unlock()
@@ -470,9 +482,23 @@ func (r *Registry) ShouldCollectDiagnostics(
delete(r.mu.requestFingerprints, id)
return false, 0, req
}
- reqID = id
- req = f
- break
+ // We found non-expired request that matches the fingerprint.
+ if f.planGist == "" {
+ // The request didn't specify the plan gist, so this execution
+ // will do.
+ reqID = id
+ req = f
+ break
+ }
+ if (f.planGist == planGist && !f.antiPlanGist) ||
+ (planGist != "" && f.planGist != planGist && f.antiPlanGist) {
+ // The execution's plan gist matches the one from the request,
+ // or the execution's plan gist doesn't match the one from the
+ // request and "anti-match" is requested.
+ reqID = id
+ req = f
+ break
+ }
}
}
@@ -641,7 +667,7 @@ func (r *Registry) InsertStatementDiagnostics(
// updates r.mu.requests accordingly.
func (r *Registry) pollRequests(ctx context.Context) error {
var rows []tree.Datums
- isSamplingProbabilitySupported := r.st.Version.IsActive(ctx, clusterversion.TODODelete_V22_2SampledStmtDiagReqs)
+ isPlanGistSupported := r.st.Version.IsActive(ctx, clusterversion.V23_2_StmtDiagForPlanGist)
// Loop until we run the query without straddling an epoch increment.
for {
@@ -650,12 +676,12 @@ func (r *Registry) pollRequests(ctx context.Context) error {
r.mu.Unlock()
var extraColumns string
- if isSamplingProbabilitySupported {
- extraColumns = ", sampling_probability"
+ if isPlanGistSupported {
+ extraColumns = ", plan_gist, anti_plan_gist"
}
it, err := r.db.Executor().QueryIteratorEx(ctx, "stmt-diag-poll", nil, /* txn */
sessiondata.RootUserSessionDataOverride,
- fmt.Sprintf(`SELECT id, statement_fingerprint, min_execution_latency, expires_at%s
+ fmt.Sprintf(`SELECT id, statement_fingerprint, min_execution_latency, expires_at, sampling_probability%s
FROM system.statement_diagnostics_requests
WHERE completed = false AND (expires_at IS NULL OR expires_at > now())`, extraColumns),
)
@@ -691,6 +717,8 @@ func (r *Registry) pollRequests(ctx context.Context) error {
var minExecutionLatency time.Duration
var expiresAt time.Time
var samplingProbability float64
+ var planGist string
+ var antiPlanGist bool
if minExecLatency, ok := row[2].(*tree.DInterval); ok {
minExecutionLatency = time.Duration(minExecLatency.Nanos())
@@ -698,18 +726,24 @@ func (r *Registry) pollRequests(ctx context.Context) error {
if e, ok := row[3].(*tree.DTimestampTZ); ok {
expiresAt = e.Time
}
- if isSamplingProbabilitySupported {
- if prob, ok := row[4].(*tree.DFloat); ok {
- samplingProbability = float64(*prob)
- if samplingProbability < 0 || samplingProbability > 1 {
- log.Warningf(ctx, "malformed sampling probability for request %d: %f (expected in range [0, 1]), resetting to 1.0",
- id, samplingProbability)
- samplingProbability = 1.0
- }
+ if prob, ok := row[4].(*tree.DFloat); ok {
+ samplingProbability = float64(*prob)
+ if samplingProbability < 0 || samplingProbability > 1 {
+ log.Warningf(ctx, "malformed sampling probability for request %d: %f (expected in range [0, 1]), resetting to 1.0",
+ id, samplingProbability)
+ samplingProbability = 1.0
+ }
+ }
+ if isPlanGistSupported {
+ if gist, ok := row[5].(*tree.DString); ok {
+ planGist = string(*gist)
+ }
+ if antiGist, ok := row[6].(*tree.DBool); ok {
+ antiPlanGist = bool(*antiGist)
}
}
ids.Add(int(id))
- r.addRequestInternalLocked(ctx, id, stmtFingerprint, samplingProbability, minExecutionLatency, expiresAt)
+ r.addRequestInternalLocked(ctx, id, stmtFingerprint, planGist, antiPlanGist, samplingProbability, minExecutionLatency, expiresAt)
}
// Remove all other requests.
diff --git a/pkg/sql/stmtdiagnostics/statement_diagnostics_helpers_test.go b/pkg/sql/stmtdiagnostics/statement_diagnostics_helpers_test.go
index 541b3414c94b..7c5e6717bbf0 100644
--- a/pkg/sql/stmtdiagnostics/statement_diagnostics_helpers_test.go
+++ b/pkg/sql/stmtdiagnostics/statement_diagnostics_helpers_test.go
@@ -27,11 +27,13 @@ func (r *Registry) TestingFindRequest(requestID int64) bool {
func (r *Registry) InsertRequestInternal(
ctx context.Context,
fprint string,
+ planGist string,
+ antiPlanGist bool,
samplingProbability float64,
minExecutionLatency time.Duration,
expiresAfter time.Duration,
) (int64, error) {
- id, err := r.insertRequestInternal(ctx, fprint, samplingProbability, minExecutionLatency, expiresAfter)
+ id, err := r.insertRequestInternal(ctx, fprint, planGist, antiPlanGist, samplingProbability, minExecutionLatency, expiresAfter)
return int64(id), err
}
diff --git a/pkg/sql/stmtdiagnostics/statement_diagnostics_test.go b/pkg/sql/stmtdiagnostics/statement_diagnostics_test.go
index 4acd4b1bfe88..69266fae27be 100644
--- a/pkg/sql/stmtdiagnostics/statement_diagnostics_test.go
+++ b/pkg/sql/stmtdiagnostics/statement_diagnostics_test.go
@@ -31,6 +31,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/stmtdiagnostics"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
+ "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
@@ -47,19 +48,18 @@ func TestDiagnosticsRequest(t *testing.T) {
ctx := context.Background()
defer s.Stopper().Stop(ctx)
registry := s.ExecutorConfig().(sql.ExecutorConfig).StmtDiagnosticsRecorder
- _, err := db.Exec("CREATE TABLE test (x int PRIMARY KEY)")
- require.NoError(t, err)
+ runner := sqlutils.MakeSQLRunner(db)
+ runner.Exec(t, "CREATE TABLE test (x int PRIMARY KEY)")
// Disable polling interval since we're inserting requests directly into the
// registry manually and want precise control of updating the registry.
- _, err = db.Exec("SET CLUSTER SETTING sql.stmt_diagnostics.poll_interval = '0';")
- require.NoError(t, err)
+ runner.Exec(t, "SET CLUSTER SETTING sql.stmt_diagnostics.poll_interval = '0';")
var collectUntilExpirationEnabled bool
isCompleted := func(reqID int64) (completed bool, diagnosticsID gosql.NullInt64) {
completedQuery := "SELECT completed, statement_diagnostics_id FROM system.statement_diagnostics_requests WHERE ID = $1"
- reqRow := db.QueryRow(completedQuery, reqID)
- require.NoError(t, reqRow.Scan(&completed, &diagnosticsID))
+ reqRow := runner.QueryRow(t, completedQuery, reqID)
+ reqRow.Scan(&completed, &diagnosticsID)
if completed && !collectUntilExpirationEnabled {
// Ensure that if the request was completed and the continuous
// collection is not enabled, the local registry no longer has the
@@ -83,23 +83,25 @@ func TestDiagnosticsRequest(t *testing.T) {
}
setCollectUntilExpiration := func(v bool) {
collectUntilExpirationEnabled = v
- _, err := db.Exec(
- fmt.Sprintf("SET CLUSTER SETTING sql.stmt_diagnostics.collect_continuously.enabled = %t", v))
- require.NoError(t, err)
+ runner.Exec(t, fmt.Sprintf("SET CLUSTER SETTING sql.stmt_diagnostics.collect_continuously.enabled = %t", v))
}
- var minExecutionLatency, expiresAfter time.Duration
- var samplingProbability float64
+ var anyPlan string
+ var noAntiMatch bool
+ var sampleAll float64
+ var noLatencyThreshold, noExpiration time.Duration
// Ask to trace a particular query.
t.Run("basic", func(t *testing.T) {
- reqID, err := registry.InsertRequestInternal(ctx, "INSERT INTO test VALUES (_)", samplingProbability, minExecutionLatency, expiresAfter)
+ reqID, err := registry.InsertRequestInternal(
+ ctx, "INSERT INTO test VALUES (_)", anyPlan, noAntiMatch,
+ sampleAll, noLatencyThreshold, noExpiration,
+ )
require.NoError(t, err)
checkNotCompleted(reqID)
// Run the query.
- _, err = db.Exec("INSERT INTO test VALUES (1)")
- require.NoError(t, err)
+ runner.Exec(t, "INSERT INTO test VALUES (1)")
// Check that the row from statement_diagnostics_request was marked as
// completed.
@@ -108,55 +110,62 @@ func TestDiagnosticsRequest(t *testing.T) {
// Verify that we can handle multiple requests at the same time.
t.Run("multiple", func(t *testing.T) {
- id1, err := registry.InsertRequestInternal(ctx, "INSERT INTO test VALUES (_)", samplingProbability, minExecutionLatency, expiresAfter)
+ id1, err := registry.InsertRequestInternal(
+ ctx, "INSERT INTO test VALUES (_)", anyPlan, noAntiMatch,
+ sampleAll, noLatencyThreshold, noExpiration,
+ )
require.NoError(t, err)
- id2, err := registry.InsertRequestInternal(ctx, "SELECT x FROM test", samplingProbability, minExecutionLatency, expiresAfter)
+ id2, err := registry.InsertRequestInternal(
+ ctx, "SELECT x FROM test", anyPlan, noAntiMatch,
+ sampleAll, noLatencyThreshold, noExpiration,
+ )
require.NoError(t, err)
- id3, err := registry.InsertRequestInternal(ctx, "SELECT x FROM test WHERE x > _", samplingProbability, minExecutionLatency, expiresAfter)
+ id3, err := registry.InsertRequestInternal(
+ ctx, "SELECT x FROM test WHERE x > _", anyPlan, noAntiMatch,
+ sampleAll, noLatencyThreshold, noExpiration,
+ )
require.NoError(t, err)
// Run the queries in a different order.
- _, err = db.Exec("SELECT x FROM test")
- require.NoError(t, err)
+ runner.Exec(t, "SELECT x FROM test")
checkCompleted(id2)
- _, err = db.Exec("SELECT x FROM test WHERE x > 1")
- require.NoError(t, err)
+ runner.Exec(t, "SELECT x FROM test WHERE x > 1")
checkCompleted(id3)
- _, err = db.Exec("INSERT INTO test VALUES (2)")
- require.NoError(t, err)
+ runner.Exec(t, "INSERT INTO test VALUES (2)")
checkCompleted(id1)
})
// Verify that EXECUTE triggers diagnostics collection (#66048).
t.Run("execute", func(t *testing.T) {
- id, err := registry.InsertRequestInternal(ctx, "SELECT x + $1 FROM test", samplingProbability, minExecutionLatency, expiresAfter)
- require.NoError(t, err)
- _, err = db.Exec("PREPARE stmt AS SELECT x + $1 FROM test")
- require.NoError(t, err)
- _, err = db.Exec("EXECUTE stmt(1)")
+ id, err := registry.InsertRequestInternal(
+ ctx, "SELECT x + $1 FROM test", anyPlan, noAntiMatch,
+ sampleAll, noLatencyThreshold, noExpiration,
+ )
require.NoError(t, err)
+ runner.Exec(t, "PREPARE stmt AS SELECT x + $1 FROM test")
+ runner.Exec(t, "EXECUTE stmt(1)")
checkCompleted(id)
})
// Verify that if the traced query times out, the bundle is still saved.
t.Run("timeout", func(t *testing.T) {
- reqID, err := registry.InsertRequestInternal(ctx, "SELECT pg_sleep(_)", samplingProbability, minExecutionLatency, expiresAfter)
+ reqID, err := registry.InsertRequestInternal(
+ ctx, "SELECT pg_sleep(_)", anyPlan, noAntiMatch,
+ sampleAll, noLatencyThreshold, noExpiration,
+ )
require.NoError(t, err)
checkNotCompleted(reqID)
// Set the statement timeout (as well as clean it up in a defer).
- _, err = db.Exec("SET statement_timeout = '100ms';")
- require.NoError(t, err)
+ runner.Exec(t, "SET statement_timeout = '100ms';")
defer func() {
- _, err = db.Exec("RESET statement_timeout;")
- require.NoError(t, err)
+ runner.Exec(t, "RESET statement_timeout;")
}()
// Run the query that times out.
_, err = db.Exec("SELECT pg_sleep(999999)")
- require.Error(t, err)
require.True(t, strings.Contains(err.Error(), sqlerrors.QueryTimeoutError.Error()))
checkCompleted(reqID)
})
@@ -165,35 +174,36 @@ func TestDiagnosticsRequest(t *testing.T) {
// condition is satisfied.
t.Run("conditional", func(t *testing.T) {
minExecutionLatency := 100 * time.Millisecond
- reqID, err := registry.InsertRequestInternal(ctx, "SELECT pg_sleep(_)", samplingProbability, minExecutionLatency, expiresAfter)
+ reqID, err := registry.InsertRequestInternal(
+ ctx, "SELECT pg_sleep(_)", anyPlan, noAntiMatch,
+ sampleAll, minExecutionLatency, noExpiration,
+ )
require.NoError(t, err)
checkNotCompleted(reqID)
// Run the fast query.
- _, err = db.Exec("SELECT pg_sleep(0)")
- require.NoError(t, err)
+ runner.Exec(t, "SELECT pg_sleep(0)")
checkNotCompleted(reqID)
// Run the slow query.
- _, err = db.Exec("SELECT pg_sleep(0.2)")
- require.NoError(t, err)
+ runner.Exec(t, "SELECT pg_sleep(0.2)")
checkCompleted(reqID)
})
// Verify that if a conditional request expired, the bundle for it is not
// created even if the condition is satisfied.
t.Run("conditional expired", func(t *testing.T) {
- minExecutionLatency := 100 * time.Millisecond
+ minExecutionLatency, expiresAfter := 100*time.Millisecond, time.Nanosecond
reqID, err := registry.InsertRequestInternal(
- ctx, "SELECT pg_sleep(_)", samplingProbability, minExecutionLatency, time.Nanosecond,
+ ctx, "SELECT pg_sleep(_)", anyPlan, noAntiMatch,
+ sampleAll, minExecutionLatency, expiresAfter,
)
require.NoError(t, err)
checkNotCompleted(reqID)
// Sleep for a bit and then run the slow query.
time.Sleep(100 * time.Millisecond)
- _, err = db.Exec("SELECT pg_sleep(0.2)")
- require.NoError(t, err)
+ runner.Exec(t, "SELECT pg_sleep(0.2)")
// The request must have expired by now.
checkNotCompleted(reqID)
})
@@ -203,7 +213,10 @@ func TestDiagnosticsRequest(t *testing.T) {
// conditional, then the bundle is collected.
t.Run("conditional with concurrency", func(t *testing.T) {
minExecutionLatency := 100 * time.Millisecond
- reqID, err := registry.InsertRequestInternal(ctx, "SELECT pg_sleep($1)", samplingProbability, minExecutionLatency, expiresAfter)
+ reqID, err := registry.InsertRequestInternal(
+ ctx, "SELECT pg_sleep($1)", anyPlan, noAntiMatch,
+ sampleAll, minExecutionLatency, noExpiration,
+ )
require.NoError(t, err)
checkNotCompleted(reqID)
@@ -219,8 +232,7 @@ func TestDiagnosticsRequest(t *testing.T) {
if i == numGoroutines-1 {
sleepDuration = "0.2"
}
- _, err := db.Exec("SELECT pg_sleep($1)", sleepDuration)
- require.NoError(t, err)
+ runner.Exec(t, "SELECT pg_sleep($1)", sleepDuration)
}(i)
}
@@ -249,7 +261,8 @@ func TestDiagnosticsRequest(t *testing.T) {
for _, expiresAfter := range []time.Duration{0, time.Second} {
t.Run(fmt.Sprintf("expiresAfter=%s", expiresAfter), func(t *testing.T) {
reqID, err := registry.InsertRequestInternal(
- ctx, fprint, samplingProbability, minExecutionLatency, expiresAfter,
+ ctx, fprint, anyPlan, noAntiMatch,
+ sampleAll, minExecutionLatency, expiresAfter,
)
require.NoError(t, err)
checkNotCompleted(reqID)
@@ -260,8 +273,7 @@ func TestDiagnosticsRequest(t *testing.T) {
// Run the query that is slow enough to satisfy the
// conditional request.
- _, err = db.Exec("SELECT pg_sleep(0.2)")
- require.NoError(t, err)
+ runner.Exec(t, "SELECT pg_sleep(0.2)")
checkNotCompleted(reqID)
})
}
@@ -283,7 +295,8 @@ func TestDiagnosticsRequest(t *testing.T) {
minExecutionLatency = 100 * time.Millisecond
}
reqID, err := registry.InsertRequestInternal(
- ctx, fprint, samplingProbability, minExecutionLatency, expiresAfter,
+ ctx, fprint, anyPlan, noAntiMatch,
+ sampleAll, minExecutionLatency, noExpiration,
)
require.NoError(t, err)
checkNotCompleted(reqID)
@@ -304,8 +317,7 @@ func TestDiagnosticsRequest(t *testing.T) {
// Now run the query that is slow enough to satisfy the
// conditional request.
close(waitCh)
- _, err = db.Exec("SELECT pg_sleep(0.2)")
- require.NoError(t, err)
+ runner.Exec(t, "SELECT pg_sleep(0.2)")
wg.Wait()
@@ -323,14 +335,15 @@ func TestDiagnosticsRequest(t *testing.T) {
// Ask to trace a statement probabilistically.
t.Run("probabilistic sample", func(t *testing.T) {
samplingProbability, minExecutionLatency := 0.9999, time.Microsecond
- reqID, err := registry.InsertRequestInternal(ctx, "SELECT pg_sleep(_)",
- samplingProbability, minExecutionLatency, expiresAfter)
+ reqID, err := registry.InsertRequestInternal(
+ ctx, "SELECT pg_sleep(_)", anyPlan, noAntiMatch,
+ samplingProbability, minExecutionLatency, noExpiration,
+ )
require.NoError(t, err)
checkNotCompleted(reqID)
testutils.SucceedsSoon(t, func() error {
- _, err = db.Exec("SELECT pg_sleep(0.01)") // run the query
- require.NoError(t, err)
+ runner.Exec(t, "SELECT pg_sleep(0.01)") // run the query
completed, _ := isCompleted(reqID)
if completed {
return nil
@@ -341,8 +354,10 @@ func TestDiagnosticsRequest(t *testing.T) {
t.Run("sampling without latency threshold disallowed", func(t *testing.T) {
samplingProbability, expiresAfter := 0.5, time.Second
- _, err := registry.InsertRequestInternal(ctx, "SELECT pg_sleep(_)",
- samplingProbability, 0 /* minExecutionLatency */, expiresAfter)
+ _, err := registry.InsertRequestInternal(
+ ctx, "SELECT pg_sleep(_)", anyPlan, noAntiMatch,
+ samplingProbability, noLatencyThreshold, expiresAfter,
+ )
testutils.IsError(err, "empty min exec latency")
})
@@ -353,8 +368,10 @@ func TestDiagnosticsRequest(t *testing.T) {
// the expiration point +1h from now (we don't mark continuous captures
// as completed until they've expired).
samplingProbability, minExecutionLatency, expiresAfter := 0.0, time.Microsecond, time.Hour
- reqID, err := registry.InsertRequestInternal(ctx, "SELECT pg_sleep(_)",
- samplingProbability, minExecutionLatency, expiresAfter)
+ reqID, err := registry.InsertRequestInternal(
+ ctx, "SELECT pg_sleep(_)", anyPlan, noAntiMatch,
+ samplingProbability, minExecutionLatency, expiresAfter,
+ )
require.NoError(t, err)
checkNotCompleted(reqID)
@@ -362,8 +379,7 @@ func TestDiagnosticsRequest(t *testing.T) {
defer setCollectUntilExpiration(false)
testutils.SucceedsSoon(t, func() error {
- _, err := db.Exec("SELECT pg_sleep(0.01)") // run the query
- require.NoError(t, err)
+ runner.Exec(t, "SELECT pg_sleep(0.01)") // run the query
completed, _ := isCompleted(reqID)
if completed {
return nil
@@ -376,9 +392,11 @@ func TestDiagnosticsRequest(t *testing.T) {
// We don't mark continuous captures as completed until they've expired,
// so we require an explicit expiration set. See previous test case for
// some commentary.
- samplingProbability, minExecutionLatency, expiresAfter := 0.999, time.Microsecond, 0*time.Hour
- reqID, err := registry.InsertRequestInternal(ctx, "SELECT pg_sleep(_)",
- samplingProbability, minExecutionLatency, expiresAfter)
+ samplingProbability, minExecutionLatency := 0.999, time.Microsecond
+ reqID, err := registry.InsertRequestInternal(
+ ctx, "SELECT pg_sleep(_)", anyPlan, noAntiMatch,
+ samplingProbability, minExecutionLatency, noExpiration,
+ )
require.NoError(t, err)
checkNotCompleted(reqID)
@@ -386,8 +404,7 @@ func TestDiagnosticsRequest(t *testing.T) {
defer setCollectUntilExpiration(false)
testutils.SucceedsSoon(t, func() error {
- _, err := db.Exec("SELECT pg_sleep(0.01)") // run the query
- require.NoError(t, err)
+ runner.Exec(t, "SELECT pg_sleep(0.01)") // run the query
completed, _ := isCompleted(reqID)
if completed {
return nil
@@ -398,8 +415,10 @@ func TestDiagnosticsRequest(t *testing.T) {
t.Run("continuous capture", func(t *testing.T) {
samplingProbability, minExecutionLatency, expiresAfter := 0.9999, time.Microsecond, time.Hour
- reqID, err := registry.InsertRequestInternal(ctx, "SELECT pg_sleep(_)",
- samplingProbability, minExecutionLatency, expiresAfter)
+ reqID, err := registry.InsertRequestInternal(
+ ctx, "SELECT pg_sleep(_)", anyPlan, noAntiMatch,
+ samplingProbability, minExecutionLatency, expiresAfter,
+ )
require.NoError(t, err)
checkNotCompleted(reqID)
@@ -408,8 +427,7 @@ func TestDiagnosticsRequest(t *testing.T) {
var firstDiagnosticID int64
testutils.SucceedsSoon(t, func() error {
- _, err := db.Exec("SELECT pg_sleep(0.01)") // run the query
- require.NoError(t, err)
+ runner.Exec(t, "SELECT pg_sleep(0.01)") // run the query
completed, diagnosticID := isCompleted(reqID)
if !diagnosticID.Valid {
return errors.New("expected to capture diagnostic bundle")
@@ -430,7 +448,8 @@ func TestDiagnosticsRequest(t *testing.T) {
t.Run("continuous capture until expiration", func(t *testing.T) {
samplingProbability, minExecutionLatency, expiresAfter := 0.9999, time.Microsecond, 100*time.Millisecond
reqID, err := registry.InsertRequestInternal(
- ctx, "SELECT pg_sleep(_)", samplingProbability, minExecutionLatency, expiresAfter,
+ ctx, "SELECT pg_sleep(_)", anyPlan, noAntiMatch,
+ samplingProbability, minExecutionLatency, expiresAfter,
)
require.NoError(t, err)
checkNotCompleted(reqID)
@@ -445,13 +464,74 @@ func TestDiagnosticsRequest(t *testing.T) {
// registry yet (because we disabled the polling interval). When we run
// the query that matches the fingerprint, the expired request is
// removed, and the bundle is not collected.
- _, err = db.Exec("SELECT pg_sleep(0.01)") // run the query
- require.NoError(t, err)
+ runner.Exec(t, "SELECT pg_sleep(0.01)") // run the query
checkNotCompleted(reqID)
// Sanity check that the request is no longer in the registry.
require.False(t, registry.TestingFindRequest(reqID))
})
+
+ t.Run("plan-gist matching", func(t *testing.T) {
+ // Set up two tables such that the same query fingerprint would get
+ // different plans based on the placeholder values.
+ runner.Exec(t, "CREATE TABLE small (k PRIMARY KEY) AS VALUES (1), (2);")
+ runner.Exec(t, "ANALYZE small;")
+ runner.Exec(t, "CREATE TABLE large (v INT, INDEX (v));")
+ runner.Exec(t, "INSERT INTO large VALUES (1);")
+ runner.Exec(t, "INSERT INTO large SELECT 2 FROM generate_series(1, 10000);")
+ runner.Exec(t, "ANALYZE large;")
+
+ // query1 results in scan + lookup join whereas query2 does two scans +
+ // merge join.
+ const (
+ fprint = `SELECT v FROM small INNER JOIN large ON (k = v) AND (k = _)`
+ query1 = "SELECT v FROM small INNER JOIN large ON k = v AND k = 0;"
+ query2 = "SELECT v FROM small INNER JOIN large ON k = v AND k = 1;"
+ )
+ getGist := func(query string) string {
+ row := runner.QueryRow(t, "EXPLAIN (GIST) "+query)
+ var gist string
+ row.Scan(&gist)
+ return gist
+ }
+ for _, antiMatch := range []bool{false, true} {
+ t.Run(fmt.Sprintf("anti-match=%t", antiMatch), func(t *testing.T) {
+ for _, tc := range []struct {
+ target, other string
+ }{
+ {target: query1, other: query2},
+ {target: query2, other: query1},
+ } {
+ targetGist, otherGist := getGist(tc.target), getGist(tc.other)
+ // Sanity check that two queries have different plans.
+ require.NotEqual(t, targetGist, otherGist)
+ target, other := tc.target, tc.other
+ if antiMatch {
+ // Flip the queries when testing the anti-match.
+ target, other = other, target
+ }
+
+ reqID, err := registry.InsertRequestInternal(
+ ctx, fprint, targetGist, antiMatch, sampleAll, noLatencyThreshold, noExpiration,
+ )
+ require.NoError(t, err)
+ checkNotCompleted(reqID)
+
+ // Run other query several times and ensure that the bundle
+ // wasn't collected because the plan gist didn't match.
+ for i := 0; i < 3; i++ {
+ runner.Exec(t, other)
+ }
+ checkNotCompleted(reqID)
+
+ // Now run our target query and verify that the bundle is
+ // now collected.
+ runner.Exec(t, target)
+ checkCompleted(reqID)
+ }
+ })
+ }
+ })
}
// Test that a different node can service a diagnostics request.
@@ -470,12 +550,17 @@ func TestDiagnosticsRequestDifferentNode(t *testing.T) {
_, err = db0.Exec("SET CLUSTER SETTING sql.stmt_diagnostics.poll_interval = '1ms'")
require.NoError(t, err)
- var minExecutionLatency, expiresAfter time.Duration
- var samplingProbability float64
+ var anyPlan string
+ var noAntiMatch bool
+ var sampleAll float64
+ var noLatencyThreshold, noExpiration time.Duration
// Ask to trace a particular query using node 0.
registry := tc.Server(0).ExecutorConfig().(sql.ExecutorConfig).StmtDiagnosticsRecorder
- reqID, err := registry.InsertRequestInternal(ctx, "INSERT INTO test VALUES (_)", samplingProbability, minExecutionLatency, expiresAfter)
+ reqID, err := registry.InsertRequestInternal(
+ ctx, "INSERT INTO test VALUES (_)", anyPlan, noAntiMatch,
+ sampleAll, noLatencyThreshold, noExpiration,
+ )
require.NoError(t, err)
reqRow := db0.QueryRow(
`SELECT completed, statement_diagnostics_id FROM system.statement_diagnostics_requests
@@ -510,11 +595,20 @@ func TestDiagnosticsRequestDifferentNode(t *testing.T) {
runUntilTraced("INSERT INTO test VALUES (1)", reqID)
// Verify that we can handle multiple requests at the same time.
- id1, err := registry.InsertRequestInternal(ctx, "INSERT INTO test VALUES (_)", samplingProbability, minExecutionLatency, expiresAfter)
+ id1, err := registry.InsertRequestInternal(
+ ctx, "INSERT INTO test VALUES (_)", anyPlan, noAntiMatch,
+ sampleAll, noLatencyThreshold, noExpiration,
+ )
require.NoError(t, err)
- id2, err := registry.InsertRequestInternal(ctx, "SELECT x FROM test", samplingProbability, minExecutionLatency, expiresAfter)
+ id2, err := registry.InsertRequestInternal(
+ ctx, "SELECT x FROM test", anyPlan, noAntiMatch,
+ sampleAll, noLatencyThreshold, noExpiration,
+ )
require.NoError(t, err)
- id3, err := registry.InsertRequestInternal(ctx, "SELECT x FROM test WHERE x > _", samplingProbability, minExecutionLatency, expiresAfter)
+ id3, err := registry.InsertRequestInternal(
+ ctx, "SELECT x FROM test WHERE x > _", anyPlan, noAntiMatch,
+ sampleAll, noLatencyThreshold, noExpiration,
+ )
require.NoError(t, err)
// Run the queries in a different order.
diff --git a/pkg/upgrade/upgrades/BUILD.bazel b/pkg/upgrade/upgrades/BUILD.bazel
index 7a9763355944..3acbb2b1614f 100644
--- a/pkg/upgrade/upgrades/BUILD.bazel
+++ b/pkg/upgrade/upgrades/BUILD.bazel
@@ -21,8 +21,8 @@ go_library(
"first_upgrade.go",
"key_visualizer_migration.go",
"permanent_upgrades.go",
+ "plan_gist_stmt_diagnostics_requests.go",
"role_members_ids_migration.go",
- "sampled_stmt_diagnostics_requests.go",
"schema_changes.go",
"schemachanger_elements.go",
"sql_stats_ttl.go",
@@ -114,8 +114,8 @@ go_test(
"json_forward_indexes_test.go",
"key_visualizer_migration_test.go",
"main_test.go",
+ "plan_gist_stmt_diagnostics_requests_test.go",
"role_members_ids_migration_test.go",
- "sampled_stmt_diagnostics_requests_test.go",
"schema_changes_external_test.go",
"schema_changes_helpers_test.go",
"schemachanger_elements_test.go",
diff --git a/pkg/upgrade/upgrades/plan_gist_stmt_diagnostics_requests.go b/pkg/upgrade/upgrades/plan_gist_stmt_diagnostics_requests.go
new file mode 100644
index 000000000000..62983969884b
--- /dev/null
+++ b/pkg/upgrade/upgrades/plan_gist_stmt_diagnostics_requests.go
@@ -0,0 +1,80 @@
+// Copyright 2023 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 upgrades
+
+import (
+ "context"
+
+ "github.com/cockroachdb/cockroach/pkg/clusterversion"
+ "github.com/cockroachdb/cockroach/pkg/keys"
+ "github.com/cockroachdb/cockroach/pkg/sql/catalog"
+ "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema"
+ "github.com/cockroachdb/cockroach/pkg/upgrade"
+)
+
+// Target schema changes in the system.statement_diagnostics_requests table,
+// adding two columns and updating the secondary index to store those columns.
+const (
+ addPlanGistColToStmtDiagReqs = `
+ALTER TABLE system.statement_diagnostics_requests
+ ADD COLUMN plan_gist STRING NULL FAMILY "primary"`
+
+ addAntiPlanGistColToStmtDiagReqs = `
+ALTER TABLE system.statement_diagnostics_requests
+ ADD COLUMN anti_plan_gist BOOL NULL FAMILY "primary"`
+
+ createCompletedIdxV2 = `
+CREATE INDEX completed_idx_v2 ON system.statement_diagnostics_requests (completed, ID)
+ STORING (statement_fingerprint, min_execution_latency, expires_at, sampling_probability, plan_gist, anti_plan_gist)`
+
+ dropCompletedIdx = `DROP INDEX system.statement_diagnostics_requests@completed_idx`
+)
+
+// stmtDiagForPlanGistMigration changes the schema of the
+// system.statement_diagnostics_requests table to support plan gist matching.
+func stmtDiagForPlanGistMigration(
+ ctx context.Context, cs clusterversion.ClusterVersion, d upgrade.TenantDeps,
+) error {
+ for _, op := range []operation{
+ {
+ name: "add-stmt-diag-reqs-plan-gist-column",
+ schemaList: []string{"plan_gist"},
+ query: addPlanGistColToStmtDiagReqs,
+ schemaExistsFn: hasColumn,
+ },
+ {
+ name: "add-stmt-diag-reqs-anti-plan-gist-column",
+ schemaList: []string{"anti_plan_gist"},
+ query: addAntiPlanGistColToStmtDiagReqs,
+ schemaExistsFn: hasColumn,
+ },
+ {
+ name: "create-stmt-diag-reqs-v2-index",
+ schemaList: []string{"completed_idx_v2"},
+ query: createCompletedIdxV2,
+ schemaExistsFn: hasIndex,
+ },
+ {
+ name: "drop-stmt-diag-reqs-old-index",
+ schemaList: []string{"completed_idx"},
+ query: dropCompletedIdx,
+ schemaExistsFn: func(catalog.TableDescriptor, catalog.TableDescriptor, string) (bool, error) {
+ return false, nil
+ },
+ },
+ } {
+ if err := migrateTable(ctx, cs, d, op, keys.StatementDiagnosticsRequestsTableID,
+ systemschema.StatementDiagnosticsRequestsTable); err != nil {
+ return err
+ }
+ }
+ return nil
+}
diff --git a/pkg/upgrade/upgrades/sampled_stmt_diagnostics_requests_test.go b/pkg/upgrade/upgrades/plan_gist_stmt_diagnostics_requests_test.go
similarity index 78%
rename from pkg/upgrade/upgrades/sampled_stmt_diagnostics_requests_test.go
rename to pkg/upgrade/upgrades/plan_gist_stmt_diagnostics_requests_test.go
index d4438e55cd77..37acb22e6d1c 100644
--- a/pkg/upgrade/upgrades/sampled_stmt_diagnostics_requests_test.go
+++ b/pkg/upgrade/upgrades/plan_gist_stmt_diagnostics_requests_test.go
@@ -1,4 +1,4 @@
-// Copyright 2022 The Cockroach Authors.
+// Copyright 2023 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
@@ -26,25 +26,22 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
"github.com/cockroachdb/cockroach/pkg/sql/types"
- "github.com/cockroachdb/cockroach/pkg/testutils/skip"
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
"github.com/cockroachdb/cockroach/pkg/upgrade/upgrades"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
)
-func TestSampledStmtDiagReqsMigration(t *testing.T) {
+func TestStmtDiagForPlanGistMigration(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
- skip.WithIssue(t, 95530, "bump minBinary to 22.2. Skip 22.2 mixed-version tests for future cleanup")
-
clusterArgs := base.TestClusterArgs{
ServerArgs: base.TestServerArgs{
Knobs: base.TestingKnobs{
Server: &server.TestingKnobs{
DisableAutomaticVersionUpgrade: make(chan struct{}),
- BinaryVersionOverride: clusterversion.ByKey(clusterversion.TODODelete_V22_2SampledStmtDiagReqs - 1),
+ BinaryVersionOverride: clusterversion.ByKey(clusterversion.V23_2_StmtDiagForPlanGist - 1),
},
},
},
@@ -60,19 +57,21 @@ func TestSampledStmtDiagReqsMigration(t *testing.T) {
var (
validationStmts = []string{
- `SELECT sampling_probability FROM system.statement_diagnostics_requests LIMIT 0`,
- `SELECT sampling_probability FROM system.statement_diagnostics_requests@completed_idx LIMIT 0`,
+ `SELECT plan_gist, anti_plan_gist FROM system.statement_diagnostics_requests LIMIT 0`,
+ `SELECT plan_gist, anti_plan_gist FROM system.statement_diagnostics_requests@completed_idx_v2 LIMIT 0`,
}
validationSchemas = []upgrades.Schema{
- {Name: "sampling_probability", ValidationFn: upgrades.HasColumn},
+ {Name: "plan_gist", ValidationFn: upgrades.HasColumn},
+ {Name: "anti_plan_gist", ValidationFn: upgrades.HasColumn},
{Name: "primary", ValidationFn: upgrades.HasColumnFamily},
- {Name: "completed_idx_v2", ValidationFn: upgrades.DoesNotHaveIndex},
+ {Name: "completed_idx", ValidationFn: upgrades.DoesNotHaveIndex},
+ {Name: "completed_idx_v2", ValidationFn: upgrades.HasIndex},
}
)
// Inject the old copy of the descriptor.
upgrades.InjectLegacyTable(ctx, t, s, systemschema.StatementDiagnosticsRequestsTable,
- getV2StmtDiagReqsDescriptor)
+ getOldStmtDiagReqsDescriptor)
validateSchemaExists := func(expectExists bool) {
upgrades.ValidateSchemaExists(
ctx,
@@ -93,7 +92,7 @@ func TestSampledStmtDiagReqsMigration(t *testing.T) {
upgrades.Upgrade(
t,
sqlDB,
- clusterversion.TODODelete_V22_2SampledStmtDiagReqs,
+ clusterversion.V23_2_StmtDiagForPlanGist,
nil, /* done */
false, /* expectError */
)
@@ -101,10 +100,11 @@ func TestSampledStmtDiagReqsMigration(t *testing.T) {
validateSchemaExists(true)
}
-// getV2StmtDiagReqsDescriptor returns the system.statement_diagnostics_requests
-// table descriptor that was being used before adding the sampling_probability
-// column to the current version.
-func getV2StmtDiagReqsDescriptor() *descpb.TableDescriptor {
+// getOldStmtDiagReqsDescriptor returns the
+// system.statement_diagnostics_requests table descriptor that was being used
+// before adding the plan_gist and anti_plan_gist columns to the current
+// version.
+func getOldStmtDiagReqsDescriptor() *descpb.TableDescriptor {
uniqueRowIDString := "unique_rowid()"
falseBoolString := "false"
@@ -122,13 +122,14 @@ func getV2StmtDiagReqsDescriptor() *descpb.TableDescriptor {
{Name: "requested_at", ID: 5, Type: types.TimestampTZ, Nullable: false},
{Name: "min_execution_latency", ID: 6, Type: types.Interval, Nullable: true},
{Name: "expires_at", ID: 7, Type: types.TimestampTZ, Nullable: true},
+ {Name: "sampling_probability", ID: 8, Type: types.Float, Nullable: true},
},
- NextColumnID: 8,
+ NextColumnID: 9,
Families: []descpb.ColumnFamilyDescriptor{
{
Name: "primary",
- ColumnNames: []string{"id", "completed", "statement_fingerprint", "statement_diagnostics_id", "requested_at", "min_execution_latency", "expires_at"},
- ColumnIDs: []descpb.ColumnID{1, 2, 3, 4, 5, 6, 7},
+ ColumnNames: []string{"id", "completed", "statement_fingerprint", "statement_diagnostics_id", "requested_at", "min_execution_latency", "expires_at", "sampling_probability"},
+ ColumnIDs: []descpb.ColumnID{1, 2, 3, 4, 5, 6, 7, 8},
},
},
NextFamilyID: 1,
@@ -142,18 +143,18 @@ func getV2StmtDiagReqsDescriptor() *descpb.TableDescriptor {
},
Indexes: []descpb.IndexDescriptor{
{
- Name: "completed_idx_v2",
+ Name: "completed_idx",
ID: 2,
Unique: false,
KeyColumnNames: []string{"completed", "id"},
- StoreColumnNames: []string{"statement_fingerprint", "min_execution_latency", "expires_at"},
+ StoreColumnNames: []string{"statement_fingerprint", "min_execution_latency", "expires_at", "sampling_probability"},
KeyColumnIDs: []descpb.ColumnID{2, 1},
KeyColumnDirections: []catenumpb.IndexColumn_Direction{catenumpb.IndexColumn_ASC, catenumpb.IndexColumn_ASC},
- StoreColumnIDs: []descpb.ColumnID{3, 6, 7},
+ StoreColumnIDs: []descpb.ColumnID{3, 6, 7, 8},
Version: descpb.StrictIndexColumnIDGuaranteesVersion,
},
},
- NextIndexID: 3,
+ NextIndexID: 4,
Privileges: catpb.NewCustomSuperuserPrivilegeDescriptor(privilege.ReadWriteData, username.NodeUserName()),
NextMutationID: 1,
FormatVersion: 3,
diff --git a/pkg/upgrade/upgrades/sampled_stmt_diagnostics_requests.go b/pkg/upgrade/upgrades/sampled_stmt_diagnostics_requests.go
deleted file mode 100644
index 03c632df806f..000000000000
--- a/pkg/upgrade/upgrades/sampled_stmt_diagnostics_requests.go
+++ /dev/null
@@ -1,92 +0,0 @@
-// Copyright 2022 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 upgrades
-
-import (
- "context"
-
- "github.com/cockroachdb/cockroach/pkg/clusterversion"
- "github.com/cockroachdb/cockroach/pkg/keys"
- "github.com/cockroachdb/cockroach/pkg/sql/catalog"
- "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema"
- "github.com/cockroachdb/cockroach/pkg/upgrade"
-)
-
-// Target schema changes in the system.statement_diagnostics_requests table,
-// adding a column and updating the secondary index to store those columns.
-const (
- addSamplingProbColToStmtDiagReqs = `
-ALTER TABLE system.statement_diagnostics_requests
- ADD COLUMN sampling_probability FLOAT NULL FAMILY "primary"`
-
- dropCompletedIdxV1 = `DROP INDEX IF EXISTS system.statement_diagnostics_requests@completed_idx`
-
- createCompletedIdxV3 = `
-CREATE INDEX completed_idx ON system.statement_diagnostics_requests (completed, ID)
- STORING (statement_fingerprint, min_execution_latency, expires_at, sampling_probability)`
-
- dropCompletedIdxV2 = `DROP INDEX IF EXISTS system.statement_diagnostics_requests@completed_idx_v2`
-)
-
-// sampledStmtDiagReqsMigration changes the schema of the
-// system.statement_diagnostics_requests table to support probabilistic bundle
-// collection.
-func sampledStmtDiagReqsMigration(
- ctx context.Context, cs clusterversion.ClusterVersion, d upgrade.TenantDeps,
-) error {
- for _, op := range []operation{
- {
- name: "add-stmt-diag-reqs-sampling-probability-column",
- schemaList: []string{"sampling_probability"},
- query: addSamplingProbColToStmtDiagReqs,
- schemaExistsFn: hasColumn,
- },
- {
- name: "drop-stmt-diag-reqs-v1-index",
- schemaList: []string{"completed_idx"},
- query: dropCompletedIdxV1,
- schemaExistsFn: func(existing, _ catalog.TableDescriptor, _ string) (bool, error) {
- // We want to determine whether the old index from 21.2 exists.
- // That index has one stored column. The index we introduce below has
- // four.
- idx := catalog.FindIndexByName(existing, "completed_idx")
- // If the index does not exist, we're good to proceed.
- if idx == nil {
- return true, nil
- }
- // Say that the schema does exist if the column count does not
- // correspond to the old 21.2 count. If we return true, then
- // the drop index command will not happen.
- return idx.NumSecondaryStoredColumns() != 1, nil
- },
- },
- {
- name: "create-stmt-diag-reqs-v3-index",
- schemaList: []string{"completed_idx"},
- query: createCompletedIdxV3,
- schemaExistsFn: hasIndex,
- },
- {
- name: "drop-stmt-diag-reqs-v2-index",
- schemaList: []string{"completed_idx_v2"},
- query: dropCompletedIdxV2,
- schemaExistsFn: func(catalog.TableDescriptor, catalog.TableDescriptor, string) (bool, error) {
- return false, nil
- },
- },
- } {
- if err := migrateTable(ctx, cs, d, op, keys.StatementDiagnosticsRequestsTableID,
- systemschema.StatementDiagnosticsRequestsTable); err != nil {
- return err
- }
- }
- return nil
-}
diff --git a/pkg/upgrade/upgrades/upgrades.go b/pkg/upgrade/upgrades/upgrades.go
index dc6f51b4ea7f..3dc47e68cfc8 100644
--- a/pkg/upgrade/upgrades/upgrades.go
+++ b/pkg/upgrade/upgrades/upgrades.go
@@ -92,12 +92,6 @@ var upgrades = []upgradebase.Upgrade{
createDefaultDbs,
"create default databases", // v22_2StartupMigrationName
),
- upgrade.NewTenantUpgrade(
- "update system.statement_diagnostics_requests to support sampling probabilities",
- toCV(clusterversion.TODODelete_V22_2SampledStmtDiagReqs),
- upgrade.NoPrecondition,
- sampledStmtDiagReqsMigration,
- ),
upgrade.NewTenantUpgrade(
"add the system.external_connections table",
toCV(clusterversion.TODODelete_V22_2SystemExternalConnectionsTable),
@@ -317,6 +311,12 @@ var upgrades = []upgradebase.Upgrade{
upgrade.NoPrecondition,
NoTenantUpgradeFunc,
),
+ upgrade.NewTenantUpgrade(
+ "update system.statement_diagnostics_requests to support plan gist matching",
+ toCV(clusterversion.V23_2_StmtDiagForPlanGist),
+ upgrade.NoPrecondition,
+ stmtDiagForPlanGistMigration,
+ ),
}
var (