Skip to content

Commit

Permalink
Merge #107090
Browse files Browse the repository at this point in the history
107090: sql: error in cluster_logical_timestamp when not SERIALIZABLE r=chrisseto a=chrisseto

Previously, running `cluster_logical_timestamp` within an isolation level that tolerated write skew would result in an unhandled panic.

This commit adds a gate at the SQL level that will instead return a FeatureNotSupported pgerror. This behavior may be permanent or may be removed once we determine how CommitTimestamp should function at these isolation levels.

Epic:  CRDB-26546
Fixes: #103245
Release note (sql change): cluster_logical_timestamp now returns an error when called at isolation levels lower than SERIALIZABLE.

Co-authored-by: Chris Seto <[email protected]>
  • Loading branch information
craig[bot] and chrisseto committed Jul 26, 2023
2 parents 9db3a0e + 55ed31f commit c10ca05
Show file tree
Hide file tree
Showing 10 changed files with 101 additions and 19 deletions.
1 change: 1 addition & 0 deletions docs/generated/sql/functions.md
Original file line number Diff line number Diff line change
Expand Up @@ -3116,6 +3116,7 @@ Case mode values range between 0 - 1, representing lower casing and upper casing
a CockroachDB HLC in decimal form.</p>
<p>Note that uses of this function disable server-side optimizations and
may increase either contention or retry errors, or both.</p>
<p>Returns an error if run in a transaction with an isolation level weaker than SERIALIZABLE.</p>
</span></td><td>Volatile</td></tr>
<tr><td><a name="crdb_internal.active_version"></a><code>crdb_internal.active_version() &rarr; jsonb</code></td><td><span class="funcdesc"><p>Returns the current active cluster version.</p>
</span></td><td>Volatile</td></tr>
Expand Down
17 changes: 1 addition & 16 deletions pkg/sql/conn_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down Expand Up @@ -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(),
}
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/crdb_internal.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
17 changes: 17 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/txn
Original file line number Diff line number Diff line change
Expand Up @@ -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
4 changes: 3 additions & 1 deletion pkg/sql/sem/builtins/builtins.go
Original file line number Diff line number Diff line change
Expand Up @@ -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,
},
),
Expand Down
9 changes: 9 additions & 0 deletions pkg/sql/sem/eval/context.go
Original file line number Diff line number Diff line change
Expand Up @@ -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")
Expand Down
3 changes: 3 additions & 0 deletions pkg/sql/sem/tree/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down Expand Up @@ -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",
Expand All @@ -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",
Expand Down
19 changes: 19 additions & 0 deletions pkg/sql/sem/tree/txn.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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

Expand Down
46 changes: 46 additions & 0 deletions pkg/sql/sem/tree/txn_test.go
Original file line number Diff line number Diff line change
@@ -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))
}
}
2 changes: 1 addition & 1 deletion pkg/sql/vars.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down

0 comments on commit c10ca05

Please sign in to comment.