Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
118497: sql: add telemetry/dashboard for configured and used isolation level r=rafiss a=rafiss

This telemetry will allow us to see when non-SERIALIZABLE isolation levels are used, and also when they are requested but automatically upgraded to SERIALIZABLE due to settings not being enabled yet.

This does not track instances of SERIALIZABLE transactions, since that is the default, so the noise would overwhelm the other telemetry data.

The telemetry is under the following feature counters:
- sql.txn.isolation.executed_at.level
- sql.txn.isolation.upgraded_from.level

This also adds a chart to the DB Console to help discover if you are affected by changes to isolation level support.
<img width="990" alt="image" src="https://github.com/cockroachdb/cockroach/assets/1320573/66884b64-b5c7-4d91-aded-f3b1ebfdefc7">


fixes cockroachdb#115647

Release note: None

118832: go.mod: bump Pebble to e49380ba5068, remove WaitForFileOnly() calls r=jbowens a=itsbilal


### go.mod: bump Pebble to e49380ba5068

Changes:

 * [`e49380ba`](cockroachdb/pebble@e49380ba) ingest: narrow transition of EFOS to file-only, delay excise panic
 * [`82baca4b`](cockroachdb/pebble@82baca4b) metamorphic: prevent write-after-close through history
 * [`b3570e83`](cockroachdb/pebble@b3570e83) metamorphic: fix passed test.run argument
 * [`11c37fc2`](cockroachdb/pebble@11c37fc2) wal: add facilities to list WAL files across Dirs
 * [`9da4d304`](cockroachdb/pebble@9da4d304) metamorphic: factor out key generation code
 * [`087fd92a`](cockroachdb/pebble@087fd92a) db: remove ExternalIterForwardOnly
 * [`7644abbe`](cockroachdb/pebble@7644abbe) db: use wal package
 * [`34c9b9df`](cockroachdb/pebble@34c9b9df) wal: standalone implementation (single log file per WAL)
 * [`24e7146a`](cockroachdb/pebble@24e7146a) internal/manifest: remove unused keyTypeAnnotator
 * [`b2d3930d`](cockroachdb/pebble@b2d3930d) internal/rangekey: remove coalesce's error return value
 * [`904a6c99`](cockroachdb/pebble@904a6c99) metamorphic: minor fixes to the reducer
 * [`e53e09a9`](cockroachdb/pebble@e53e09a9) metamorphic: consolidate key manager per-object structures
 * [`1d7852b3`](cockroachdb/pebble@1d7852b3) db: have excises wait for EFOS below them, reducing EFOS waits

Release note: none.
Epic: none.

----

### *: remove calls to WaitForFileOnly()

Previously, we were calling WaitForFileOnly() on
EventuallyFileOnlySnapshots to avoid the need to handle
pebble.ErrSnapshotExcised errors where an excise conflicts
with an open EventuallyFileOnlySnapshot. This is now
unnecessary as EventuallyFileOnlySnapshots are no longer
expected to return that error. This change removes
those waits as well as any associated infrastructure to speed
up that wait for tests. This should have the effect of speeding
up foreground consistency checks.

Epic: none

Release note: None

Co-authored-by: Rafi Shamim <[email protected]>
Co-authored-by: Bilal Akhtar <[email protected]>
  • Loading branch information
3 people committed Feb 6, 2024
3 parents 8c16d7d + a0a0f13 + 3f0b37a commit b106b81
Show file tree
Hide file tree
Showing 23 changed files with 356 additions and 104 deletions.
6 changes: 3 additions & 3 deletions DEPS.bzl
Original file line number Diff line number Diff line change
Expand Up @@ -1613,10 +1613,10 @@ def go_deps():
patches = [
"@com_github_cockroachdb_cockroach//build/patches:com_github_cockroachdb_pebble.patch",
],
sha256 = "34984c356903add93953068151896854806ac05de4dcec4555605eadf15a01fd",
strip_prefix = "github.com/cockroachdb/[email protected]20240131165956-fea6cbbb629b",
sha256 = "06cbd97d50b1ca2cdd3795b467fb6e3b36ed7bc3c7590cca1004f3337153def2",
strip_prefix = "github.com/cockroachdb/[email protected]20240206033832-e49380ba5068",
urls = [
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/pebble/com_github_cockroachdb_pebble-v0.0.0-20240131165956-fea6cbbb629b.zip",
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/pebble/com_github_cockroachdb_pebble-v0.0.0-20240206033832-e49380ba5068.zip",
],
)
go_repository(
Expand Down
2 changes: 1 addition & 1 deletion build/bazelutil/distdir_files.bzl
Original file line number Diff line number Diff line change
Expand Up @@ -322,7 +322,7 @@ DISTDIR_FILES = {
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/gostdlib/com_github_cockroachdb_gostdlib-v1.19.0.zip": "c4d516bcfe8c07b6fc09b8a9a07a95065b36c2855627cb3514e40c98f872b69e",
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/logtags/com_github_cockroachdb_logtags-v0.0.0-20230118201751-21c54148d20b.zip": "ca7776f47e5fecb4c495490a679036bfc29d95bd7625290cfdb9abb0baf97476",
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/metamorphic/com_github_cockroachdb_metamorphic-v0.0.0-20231108215700-4ba948b56895.zip": "28c8cf42192951b69378cf537be5a9a43f2aeb35542908cc4fe5f689505853ea",
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/pebble/com_github_cockroachdb_pebble-v0.0.0-20240131165956-fea6cbbb629b.zip": "34984c356903add93953068151896854806ac05de4dcec4555605eadf15a01fd",
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/pebble/com_github_cockroachdb_pebble-v0.0.0-20240206033832-e49380ba5068.zip": "06cbd97d50b1ca2cdd3795b467fb6e3b36ed7bc3c7590cca1004f3337153def2",
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/redact/com_github_cockroachdb_redact-v1.1.5.zip": "11b30528eb0dafc8bc1a5ba39d81277c257cbe6946a7564402f588357c164560",
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/returncheck/com_github_cockroachdb_returncheck-v0.0.0-20200612231554-92cdbca611dd.zip": "ce92ba4352deec995b1f2eecf16eba7f5d51f5aa245a1c362dfe24c83d31f82b",
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/stress/com_github_cockroachdb_stress-v0.0.0-20220803192808-1806698b1b7b.zip": "3fda531795c600daf25532a4f98be2a1335cd1e5e182c72789bca79f5f69fcc1",
Expand Down
2 changes: 2 additions & 0 deletions docs/generated/metrics/metrics.html
Original file line number Diff line number Diff line change
Expand Up @@ -1481,6 +1481,8 @@
<tr><td>APPLICATION</td><td>sql.txn.rollback.count.internal</td><td>Number of SQL transaction ROLLBACK statements successfully executed (internal queries)</td><td>SQL Internal Statements</td><td>COUNTER</td><td>COUNT</td><td>AVG</td><td>NON_NEGATIVE_DERIVATIVE</td></tr>
<tr><td>APPLICATION</td><td>sql.txn.rollback.started.count</td><td>Number of SQL transaction ROLLBACK statements started</td><td>SQL Statements</td><td>COUNTER</td><td>COUNT</td><td>AVG</td><td>NON_NEGATIVE_DERIVATIVE</td></tr>
<tr><td>APPLICATION</td><td>sql.txn.rollback.started.count.internal</td><td>Number of SQL transaction ROLLBACK statements started (internal queries)</td><td>SQL Internal Statements</td><td>COUNTER</td><td>COUNT</td><td>AVG</td><td>NON_NEGATIVE_DERIVATIVE</td></tr>
<tr><td>APPLICATION</td><td>sql.txn.upgraded_iso_level.count</td><td>Number of times a weak isolation level was automatically upgraded to a stronger one</td><td>SQL Statements</td><td>COUNTER</td><td>COUNT</td><td>AVG</td><td>NON_NEGATIVE_DERIVATIVE</td></tr>
<tr><td>APPLICATION</td><td>sql.txn.upgraded_iso_level.count.internal</td><td>Number of times a weak isolation level was automatically upgraded to a stronger one (internal queries)</td><td>SQL Internal Statements</td><td>COUNTER</td><td>COUNT</td><td>AVG</td><td>NON_NEGATIVE_DERIVATIVE</td></tr>
<tr><td>APPLICATION</td><td>sql.txns.open</td><td>Number of currently open user SQL transactions</td><td>Open SQL Transactions</td><td>GAUGE</td><td>COUNT</td><td>AVG</td><td>NONE</td></tr>
<tr><td>APPLICATION</td><td>sql.txns.open.internal</td><td>Number of currently open user SQL transactions (internal queries)</td><td>SQL Internal Statements</td><td>GAUGE</td><td>COUNT</td><td>AVG</td><td>NONE</td></tr>
<tr><td>APPLICATION</td><td>sql.update.count</td><td>Number of SQL UPDATE statements successfully executed</td><td>SQL Statements</td><td>COUNTER</td><td>COUNT</td><td>AVG</td><td>NON_NEGATIVE_DERIVATIVE</td></tr>
Expand Down
1 change: 0 additions & 1 deletion docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -67,7 +67,6 @@
<tr><td><div id="setting-kv-closed-timestamp-lead-for-global-reads-override" class="anchored"><code>kv.closed_timestamp.lead_for_global_reads_override</code></div></td><td>duration</td><td><code>0s</code></td><td>if nonzero, overrides the lead time that global_read ranges use to publish closed timestamps</td><td>Serverless/Dedicated/Self-Hosted (read-only)</td></tr>
<tr><td><div id="setting-kv-closed-timestamp-side-transport-interval" class="anchored"><code>kv.closed_timestamp.side_transport_interval</code></div></td><td>duration</td><td><code>200ms</code></td><td>the interval at which the closed timestamp side-transport attempts to advance each range&#39;s closed timestamp; set to 0 to disable the side-transport</td><td>Serverless/Dedicated/Self-Hosted (read-only)</td></tr>
<tr><td><div id="setting-kv-closed-timestamp-target-duration" class="anchored"><code>kv.closed_timestamp.target_duration</code></div></td><td>duration</td><td><code>3s</code></td><td>if nonzero, attempt to provide closed timestamp notifications for timestamps trailing cluster time by approximately this duration</td><td>Serverless/Dedicated/Self-Hosted (read-only)</td></tr>
<tr><td><div id="setting-kv-consistency-queue-testing-fast-efos-acquisition-enabled" class="anchored"><code>kv.consistency_queue.testing_fast_efos_acquisition.enabled</code></div></td><td>boolean</td><td><code>false</code></td><td>set to true to speed up EventuallyFileOnlySnapshot acquisition/transition for tests at the expense of excessive flushes</td><td>Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-kv-lease-transfer-read-summary-global-budget" class="anchored"><code>kv.lease_transfer_read_summary.global_budget</code></div></td><td>byte size</td><td><code>0 B</code></td><td>controls the maximum number of bytes that will be used to summarize the global segment of the timestamp cache during lease transfers and range merges. A smaller budget will result in loss of precision.</td><td>Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-kv-lease-transfer-read-summary-local-budget" class="anchored"><code>kv.lease_transfer_read_summary.local_budget</code></div></td><td>byte size</td><td><code>4.0 MiB</code></td><td>controls the maximum number of bytes that will be used to summarize the local segment of the timestamp cache during lease transfers and range merges. A smaller budget will result in loss of precision.</td><td>Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-kv-log-range-and-node-events-enabled" class="anchored"><code>kv.log_range_and_node_events.enabled</code></div></td><td>boolean</td><td><code>true</code></td><td>set to true to transactionally log range events (e.g., split, merge, add/remove voter/non-voter) into system.rangelogand node join and restart events into system.eventolog</td><td>Dedicated/Self-Hosted</td></tr>
Expand Down
2 changes: 1 addition & 1 deletion go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -123,7 +123,7 @@ require (
github.com/cockroachdb/go-test-teamcity v0.0.0-20191211140407-cff980ad0a55
github.com/cockroachdb/gostdlib v1.19.0
github.com/cockroachdb/logtags v0.0.0-20230118201751-21c54148d20b
github.com/cockroachdb/pebble v0.0.0-20240131165956-fea6cbbb629b
github.com/cockroachdb/pebble v0.0.0-20240206033832-e49380ba5068
github.com/cockroachdb/redact v1.1.5
github.com/cockroachdb/returncheck v0.0.0-20200612231554-92cdbca611dd
github.com/cockroachdb/stress v0.0.0-20220803192808-1806698b1b7b
Expand Down
4 changes: 2 additions & 2 deletions go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -490,8 +490,8 @@ github.com/cockroachdb/logtags v0.0.0-20230118201751-21c54148d20b h1:r6VH0faHjZe
github.com/cockroachdb/logtags v0.0.0-20230118201751-21c54148d20b/go.mod h1:Vz9DsVWQQhf3vs21MhPMZpMGSht7O/2vFW2xusFUVOs=
github.com/cockroachdb/metamorphic v0.0.0-20231108215700-4ba948b56895 h1:XANOgPYtvELQ/h4IrmPAohXqe2pWA8Bwhejr3VQoZsA=
github.com/cockroachdb/metamorphic v0.0.0-20231108215700-4ba948b56895/go.mod h1:aPd7gM9ov9M8v32Yy5NJrDyOcD8z642dqs+F0CeNXfA=
github.com/cockroachdb/pebble v0.0.0-20240131165956-fea6cbbb629b h1:bzfrm3/AOpuDul6G2n5Urpwhsj9ChcYa5l9soTfJYQA=
github.com/cockroachdb/pebble v0.0.0-20240131165956-fea6cbbb629b/go.mod h1:BHuaMa/lK7fUe75BlsteiiTu8ptIG+qSAuDtGMArP18=
github.com/cockroachdb/pebble v0.0.0-20240206033832-e49380ba5068 h1:+XngF3r6yKBbJdqwwe/tG8xemlm5wcwE+/OD+bVKHks=
github.com/cockroachdb/pebble v0.0.0-20240206033832-e49380ba5068/go.mod h1:BHuaMa/lK7fUe75BlsteiiTu8ptIG+qSAuDtGMArP18=
github.com/cockroachdb/redact v1.1.3/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg=
github.com/cockroachdb/redact v1.1.5 h1:u1PMllDkdFfPWaNGMyLD1+so+aq3uUItthCFqzwPJ30=
github.com/cockroachdb/redact v1.1.5/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg=
Expand Down
8 changes: 0 additions & 8 deletions pkg/cmd/roachtest/roachtestutil/validation_check.go
Original file line number Diff line number Diff line change
Expand Up @@ -46,14 +46,6 @@ func CheckReplicaDivergenceOnDB(ctx context.Context, l *logger.Logger, db *gosql
return err
}

// EFOS can slow down consistency checks, since we need to take a snapshot for
// every range sequentially. Disable them.
_, err = db.ExecContext(ctx,
"SET CLUSTER SETTING kv.consistency_queue.testing_fast_efos_acquisition.enabled = true")
if err != nil {
return err
}

// NB: we set a statement_timeout since context cancellation won't work here.
// We've seen the consistency checks hang indefinitely in some cases.
// https://github.com/cockroachdb/cockroach/pull/34520
Expand Down
8 changes: 0 additions & 8 deletions pkg/kv/kvserver/consistency_queue_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -439,14 +439,6 @@ func TestCheckConsistencyInconsistent(t *testing.T) {
func TestConsistencyQueueRecomputeStats(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
// This test relies on repeated sequential storage.EventuallyFileOnlySnapshot
// acquisitions. Reduce the max wait time for each acquisition to speed up
// this test.
origEFOSWait := storage.MaxEFOSWait
storage.MaxEFOSWait = 30 * time.Millisecond
defer func() {
storage.MaxEFOSWait = origEFOSWait
}()
testutils.RunTrueAndFalse(t, "hadEstimates", testConsistencyQueueRecomputeStatsImpl)
}

Expand Down
2 changes: 0 additions & 2 deletions pkg/kv/kvserver/gc/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,6 @@ go_library(
"//pkg/keys",
"//pkg/kv/kvpb",
"//pkg/kv/kvserver/abortspan",
"//pkg/kv/kvserver/benignerror",
"//pkg/kv/kvserver/concurrency/lock",
"//pkg/kv/kvserver/rditer",
"//pkg/roachpb",
Expand All @@ -28,7 +27,6 @@ go_library(
"//pkg/util/timeutil",
"//pkg/util/uuid",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_pebble//:pebble",
],
)

Expand Down
11 changes: 0 additions & 11 deletions pkg/kv/kvserver/gc/gc.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/abortspan"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/benignerror"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/rditer"
"github.com/cockroachdb/cockroach/pkg/roachpb"
Expand All @@ -42,7 +41,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/pebble"
)

const (
Expand Down Expand Up @@ -362,24 +360,15 @@ func Run(
intentCleanupBatchTimeout: options.IntentCleanupBatchTimeout,
}, cleanupIntentsFn, &info)
if err != nil {
if errors.Is(err, pebble.ErrSnapshotExcised) {
err = benignerror.NewStoreBenign(err)
}
return Info{}, err
}
fastPath, err := processReplicatedKeyRange(ctx, desc, snap, newThreshold,
populateBatcherOptions(options), gcer, &info)
if err != nil {
if errors.Is(err, pebble.ErrSnapshotExcised) {
err = benignerror.NewStoreBenign(err)
}
return Info{}, err
}
err = processReplicatedRangeTombstones(ctx, desc, snap, fastPath, now, newThreshold, gcer, &info)
if err != nil {
if errors.Is(err, pebble.ErrSnapshotExcised) {
err = benignerror.NewStoreBenign(err)
}
return Info{}, err
}

Expand Down
32 changes: 0 additions & 32 deletions pkg/kv/kvserver/replica_consistency.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/stateloader"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/rpc"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
Expand Down Expand Up @@ -58,15 +57,6 @@ type replicaChecksum struct {
result chan CollectChecksumResponse
}

// TestingFastEFOSAcquisition speeds up EFOS WaitForFileOnly() to speed up
// node-wide replica consistency check calls in roachtests.
var TestingFastEFOSAcquisition = settings.RegisterBoolSetting(
settings.SystemOnly,
"kv.consistency_queue.testing_fast_efos_acquisition.enabled",
"set to true to speed up EventuallyFileOnlySnapshot acquisition/transition for tests at the expense of excessive flushes",
false, /* defaultValue */
settings.WithPublic)

// CheckConsistency runs a consistency check on the range. It first applies a
// ComputeChecksum through Raft and then issues CollectChecksum commands to the
// other replicas. These are inspected and a CheckConsistencyResponse is assembled.
Expand Down Expand Up @@ -501,28 +491,6 @@ func CalcReplicaDigest(
var uuidBuf [uuid.Size]byte
hasher := sha512.New()

if efos, ok := snap.(storage.EventuallyFileOnlyReader); ok {
// We start off by waiting for this snapshot to become a file-only snapshot.
// This is preferable as it reduces the amount of in-memory tables
// (memtables) pinned for the iterations below, and reduces errors later on
// in checksum computation. A wait here is safe as we're running
// asynchronously and not blocking other computation requests. Other checksum
// computation requests can run concurrently and start computation while
// we're waiting for this EFOS to transition to a file-only snapshot, however
// both requests are likely sharing the same `limiter` so if too many
// requests run concurrently, some of them could time out due to a
// combination of this wait and the limiter-induced wait.
efosWait := storage.MaxEFOSWait
if settings != nil && TestingFastEFOSAcquisition.Get(&settings.SV) {
if efosWait > 10*time.Millisecond {
efosWait = 10 * time.Millisecond
}
}
if err := efos.WaitForFileOnly(ctx, efosWait); err != nil {
return nil, err
}
}

// Request quota from the limiter in chunks of at least targetBatchSize, to
// amortize the overhead of the limiter when reading many small KVs.
var batchSize int64
Expand Down
10 changes: 0 additions & 10 deletions pkg/kv/kvserver/reset_quorum_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/server"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
Expand Down Expand Up @@ -60,15 +59,6 @@ func TestResetQuorum(t *testing.T) {
skip.UnderRace(t, "takes >1m under race")
skip.UnderShort(t)

// This test relies on repeated sequential storage.EventuallyFileOnlySnapshot
// acquisitions. Reduce the max wait time for each acquisition to speed up
// this test.
origEFOSWait := storage.MaxEFOSWait
storage.MaxEFOSWait = 30 * time.Millisecond
defer func() {
storage.MaxEFOSWait = origEFOSWait
}()

ctx := context.Background()
livenessDuration := 3000 * time.Millisecond

Expand Down
21 changes: 21 additions & 0 deletions pkg/sql/conn_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -66,6 +66,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sqlstats/insights"
"github.com/cockroachdb/cockroach/pkg/sql/sqlstats/persistedsqlstats"
"github.com/cockroachdb/cockroach/pkg/sql/sqlstats/sslocal"
"github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry"
"github.com/cockroachdb/cockroach/pkg/sql/stmtdiagnostics"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/buildutil"
Expand Down Expand Up @@ -1102,6 +1103,9 @@ func (s *Server) newConnExecutor(
ex.dataMutatorIterator.onTempSchemaCreation = func() {
ex.hasCreatedTemporarySchema = true
}
ex.dataMutatorIterator.upgradedIsolationLevel = func() {
ex.metrics.ExecutedStatementCounters.TxnUpgradedCount.Inc(1)
}

ex.applicationName.Store(ex.sessionData().ApplicationName)
ex.applicationStats = applicationStats
Expand Down Expand Up @@ -3448,13 +3452,15 @@ func (ex *connExecutor) txnIsolationLevelToKV(
if level == tree.UnspecifiedIsolation {
level = tree.IsolationLevel(ex.sessionData().DefaultTxnIsolationLevel)
}
upgraded := false
allowLevelCustomization := ex.server.cfg.Settings.Version.IsActive(ctx, clusterversion.V23_2)
ret := isolation.Serializable
if allowLevelCustomization {
switch level {
case tree.ReadUncommittedIsolation:
// READ UNCOMMITTED is mapped to READ COMMITTED. PostgreSQL also does
// this: https://www.postgresql.org/docs/current/transaction-iso.html.
upgraded = true
fallthrough
case tree.ReadCommittedIsolation:
// READ COMMITTED is only allowed if the cluster setting is enabled.
Expand All @@ -3463,10 +3469,12 @@ func (ex *connExecutor) txnIsolationLevelToKV(
if allowReadCommitted {
ret = isolation.ReadCommitted
} else {
upgraded = true
ret = isolation.Serializable
}
case tree.RepeatableReadIsolation:
// REPEATABLE READ is mapped to SNAPSHOT.
upgraded = true
fallthrough
case tree.SnapshotIsolation:
// SNAPSHOT is only allowed if the cluster setting is enabled. Otherwise
Expand All @@ -3475,6 +3483,7 @@ func (ex *connExecutor) txnIsolationLevelToKV(
if allowSnapshot {
ret = isolation.Snapshot
} else {
upgraded = true
ret = isolation.Serializable
}
case tree.SerializableIsolation:
Expand All @@ -3483,6 +3492,13 @@ func (ex *connExecutor) txnIsolationLevelToKV(
log.Fatalf(context.Background(), "unknown isolation level: %s", level)
}
}
if upgraded {
ex.metrics.ExecutedStatementCounters.TxnUpgradedCount.Inc(1)
telemetry.Inc(sqltelemetry.IsolationLevelUpgradedCounter(ctx, level))
}
if ret != isolation.Serializable {
telemetry.Inc(sqltelemetry.IsolationLevelCounter(ctx, ret))
}
return ret
}

Expand Down Expand Up @@ -4280,6 +4296,7 @@ type StatementCounters struct {
TxnBeginCount telemetry.CounterWithMetric
TxnCommitCount telemetry.CounterWithMetric
TxnRollbackCount telemetry.CounterWithMetric
TxnUpgradedCount *metric.Counter

// Savepoint operations. SavepointCount is for real SQL savepoints;
// the RestartSavepoint variants are for the
Expand Down Expand Up @@ -4316,6 +4333,8 @@ func makeStartedStatementCounters(internal bool) StatementCounters {
getMetricMeta(MetaTxnCommitStarted, internal)),
TxnRollbackCount: telemetry.NewCounterWithMetric(
getMetricMeta(MetaTxnRollbackStarted, internal)),
TxnUpgradedCount: metric.NewCounter(
getMetricMeta(MetaTxnUpgradedFromWeakIsolation, internal)),
RestartSavepointCount: telemetry.NewCounterWithMetric(
getMetricMeta(MetaRestartSavepointStarted, internal)),
ReleaseRestartSavepointCount: telemetry.NewCounterWithMetric(
Expand Down Expand Up @@ -4357,6 +4376,8 @@ func makeExecutedStatementCounters(internal bool) StatementCounters {
getMetricMeta(MetaTxnCommitExecuted, internal)),
TxnRollbackCount: telemetry.NewCounterWithMetric(
getMetricMeta(MetaTxnRollbackExecuted, internal)),
TxnUpgradedCount: metric.NewCounter(
getMetricMeta(MetaTxnUpgradedFromWeakIsolation, internal)),
RestartSavepointCount: telemetry.NewCounterWithMetric(
getMetricMeta(MetaRestartSavepointExecuted, internal)),
ReleaseRestartSavepointCount: telemetry.NewCounterWithMetric(
Expand Down
Loading

0 comments on commit b106b81

Please sign in to comment.