diff --git a/docs/generated/sql/functions.md b/docs/generated/sql/functions.md
index 13d7eb6fde2c..529fcd5e722b 100644
--- a/docs/generated/sql/functions.md
+++ b/docs/generated/sql/functions.md
@@ -3116,6 +3116,7 @@ Case mode values range between 0 - 1, representing lower casing and upper casing
a CockroachDB HLC in decimal form.
Note that uses of this function disable server-side optimizations and
may increase either contention or retry errors, or both.
+Returns an error if run in a transaction with an isolation level weaker than SERIALIZABLE.
Volatile |
crdb_internal.active_version() → jsonb | Returns the current active cluster version.
| Volatile |
diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go
index ea0d782e16b7..6c23a4f68079 100644
--- a/pkg/sql/conn_executor.go
+++ b/pkg/sql/conn_executor.go
@@ -3433,21 +3433,6 @@ func (ex *connExecutor) txnIsolationLevelToKV(
return ret
}
-func kvTxnIsolationLevelToTree(level isolation.Level) tree.IsolationLevel {
- var ret tree.IsolationLevel
- switch level {
- case isolation.Serializable:
- ret = tree.SerializableIsolation
- case isolation.ReadCommitted:
- ret = tree.ReadCommittedIsolation
- case isolation.Snapshot:
- ret = tree.SnapshotIsolation
- default:
- log.Fatalf(context.Background(), "unknown isolation level: %s", level)
- }
- return ret
-}
-
func txnPriorityToProto(mode tree.UserPriority) roachpb.UserPriority {
var pri roachpb.UserPriority
switch mode {
@@ -4003,7 +3988,7 @@ func (ex *connExecutor) serialize() serverpb.Session {
Priority: ex.state.priority.String(),
QualityOfService: sessiondatapb.ToQoSLevelString(txn.AdmissionHeader().Priority),
LastAutoRetryReason: autoRetryReasonStr,
- IsolationLevel: kvTxnIsolationLevelToTree(ex.state.isolationLevel).String(),
+ IsolationLevel: tree.IsolationLevelFromKVTxnIsolationLevel(ex.state.isolationLevel).String(),
}
}
diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go
index 84086ba8d8c8..c64d9490feb4 100644
--- a/pkg/sql/crdb_internal.go
+++ b/pkg/sql/crdb_internal.go
@@ -7705,7 +7705,7 @@ func genClusterLocksGenerator(
tree.MakeDBool(tree.DBool(granted)), /* granted */
tree.MakeDBool(len(curLock.Waiters) > 0), /* contended */
durationDatum, /* duration */
- tree.NewDString(kvTxnIsolationLevelToTree(curLock.LockHolder.IsoLevel).String()), /* isolation_level */
+ tree.NewDString(tree.IsolationLevelFromKVTxnIsolationLevel(curLock.LockHolder.IsoLevel).String()), /* isolation_level */
}, nil
}, nil, nil
diff --git a/pkg/sql/logictest/testdata/logic_test/txn b/pkg/sql/logictest/testdata/logic_test/txn
index 20c710de791d..c8f62b02e4e4 100644
--- a/pkg/sql/logictest/testdata/logic_test/txn
+++ b/pkg/sql/logictest/testdata/logic_test/txn
@@ -1523,3 +1523,20 @@ regular
statement error pq: SET CLUSTER SETTING cannot be used inside a multi-statement transaction
SET CLUSTER SETTING sql.defaults.use_declarative_schema_changer = 'on';
SET CLUSTER SETTING sql.defaults.use_declarative_schema_changer = 'off';
+
+statement ok
+BEGIN TRANSACTION ISOLATION LEVEL SERIALIZABLE;
+SELECT cluster_logical_timestamp();
+
+statement ok
+ROLLBACK
+
+statement ok
+BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
+
+skipif config local-mixed-22.2-23.1
+statement error pq: cluster_logical_timestamp\(\): unsupported in READ COMMITTED isolation
+SELECT cluster_logical_timestamp();
+
+statement ok
+ROLLBACK
diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go
index fea2f4782d8b..d1c16025a3ea 100644
--- a/pkg/sql/sem/builtins/builtins.go
+++ b/pkg/sql/sem/builtins/builtins.go
@@ -2779,7 +2779,9 @@ nearest replica.`, builtinconstants.DefaultFollowerReadDuration),
a CockroachDB HLC in decimal form.
Note that uses of this function disable server-side optimizations and
-may increase either contention or retry errors, or both.`,
+may increase either contention or retry errors, or both.
+
+Returns an error if run in a transaction with an isolation level weaker than SERIALIZABLE.`,
Volatility: volatility.Volatile,
},
),
diff --git a/pkg/sql/sem/eval/context.go b/pkg/sql/sem/eval/context.go
index 698dfc36bb9b..6b03eb222cd2 100644
--- a/pkg/sql/sem/eval/context.go
+++ b/pkg/sql/sem/eval/context.go
@@ -540,6 +540,15 @@ func (ec *Context) GetClusterTimestamp() (*tree.DDecimal, error) {
if ec.Txn == nil {
return nil, ErrNilTxnInClusterContext
}
+
+ // CommitTimestamp panics for isolation levels that can operate across
+ // multiple timestamps. Prevent this with a gate at the SQL level and return
+ // a pgerror until we decide how this will officially behave. See #103245.
+ if ec.TxnIsoLevel.ToleratesWriteSkew() {
+ treeIso := tree.IsolationLevelFromKVTxnIsolationLevel(ec.TxnIsoLevel)
+ return nil, pgerror.Newf(pgcode.FeatureNotSupported, "unsupported in %s isolation", treeIso.String())
+ }
+
ts := ec.Txn.CommitTimestamp()
if ts.IsEmpty() {
return nil, errors.AssertionFailedf("zero cluster timestamp in txn")
diff --git a/pkg/sql/sem/tree/BUILD.bazel b/pkg/sql/sem/tree/BUILD.bazel
index 7d5c19330ffd..499e1f143501 100644
--- a/pkg/sql/sem/tree/BUILD.bazel
+++ b/pkg/sql/sem/tree/BUILD.bazel
@@ -125,6 +125,7 @@ go_library(
"//pkg/col/typeconv", # keep
"//pkg/geo",
"//pkg/geo/geopb",
+ "//pkg/kv/kvserver/concurrency/isolation",
"//pkg/sql/lex",
"//pkg/sql/lexbase",
"//pkg/sql/pgrepl/lsn",
@@ -204,6 +205,7 @@ go_test(
"pretty_test.go",
"table_name_test.go",
"time_test.go",
+ "txn_test.go",
"type_check_internal_test.go",
"type_check_test.go",
"type_name_test.go",
@@ -218,6 +220,7 @@ go_test(
"//pkg/col/coldata",
"//pkg/col/coldataext",
"//pkg/internal/rsg",
+ "//pkg/kv/kvserver/concurrency/isolation",
"//pkg/security/securityassets",
"//pkg/security/securitytest",
"//pkg/settings/cluster",
diff --git a/pkg/sql/sem/tree/txn.go b/pkg/sql/sem/tree/txn.go
index c32568b497e3..c825baa83e03 100644
--- a/pkg/sql/sem/tree/txn.go
+++ b/pkg/sql/sem/tree/txn.go
@@ -14,6 +14,7 @@ import (
"fmt"
"strings"
+ "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/isolation"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/redact"
@@ -63,6 +64,24 @@ func (i IsolationLevel) String() string {
return isolationLevelNames[i]
}
+// IsolationLevelFromKVTxnIsolationLevel converts a kv level isolation.Level to
+// its SQL semantic equivalent.
+func IsolationLevelFromKVTxnIsolationLevel(level isolation.Level) IsolationLevel {
+ var ret IsolationLevel
+ switch level {
+ case isolation.Serializable:
+ ret = SerializableIsolation
+ case isolation.ReadCommitted:
+ ret = ReadCommittedIsolation
+ case isolation.Snapshot:
+ ret = SnapshotIsolation
+ default:
+ panic("What to do here? Log is a banned import")
+ // log.Fatalf(context.Background(), "unknown isolation level: %s", level)
+ }
+ return ret
+}
+
// UserPriority holds the user priority for a transaction.
type UserPriority int
diff --git a/pkg/sql/sem/tree/txn_test.go b/pkg/sql/sem/tree/txn_test.go
new file mode 100644
index 000000000000..2c72e10a2c34
--- /dev/null
+++ b/pkg/sql/sem/tree/txn_test.go
@@ -0,0 +1,46 @@
+// Copyright 2015 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 tree_test
+
+import (
+ "testing"
+
+ "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/isolation"
+ "github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
+ "github.com/cockroachdb/cockroach/pkg/util/leaktest"
+ "github.com/stretchr/testify/require"
+)
+
+func TestIsolationLevelFromKVTxnIsolationLevel(t *testing.T) {
+ defer leaktest.AfterTest(t)()
+
+ testCases := []struct {
+ In isolation.Level
+ Out tree.IsolationLevel
+ }{
+ {
+ In: isolation.Serializable,
+ Out: tree.SerializableIsolation,
+ },
+ {
+ In: isolation.ReadCommitted,
+ Out: tree.ReadCommittedIsolation,
+ },
+ {
+ In: isolation.Snapshot,
+ Out: tree.SnapshotIsolation,
+ },
+ }
+
+ for _, tc := range testCases {
+ require.Equal(t, tc.Out, tree.IsolationLevelFromKVTxnIsolationLevel(tc.In))
+ }
+}
diff --git a/pkg/sql/vars.go b/pkg/sql/vars.go
index fd00afe1c7a5..476f88b48efc 100644
--- a/pkg/sql/vars.go
+++ b/pkg/sql/vars.go
@@ -1491,7 +1491,7 @@ var varGen = map[string]sessionVar{
// See https://github.com/postgres/postgres/blob/REL_10_STABLE/src/backend/utils/misc/guc.c#L3401-L3409
`transaction_isolation`: {
Get: func(evalCtx *extendedEvalContext, _ *kv.Txn) (string, error) {
- level := kvTxnIsolationLevelToTree(evalCtx.Txn.IsoLevel())
+ level := tree.IsolationLevelFromKVTxnIsolationLevel(evalCtx.Txn.IsoLevel())
return strings.ToLower(level.String()), nil
},
RuntimeSet: func(ctx context.Context, evalCtx *extendedEvalContext, local bool, s string) error {