Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
109638: sql: use the correct locking strength for FOR SHARE clause r=michae2 a=arulajmani

Previously, FOR SHARE and FOR KEY SHARE would use non-locking KV scans.
Now that the lock table supports shared locks, we can use lock.Shared as
the locking strength for KV scans. This patch does that, and in doing
so, wires up SHARED locks end to end.

By default, we turn of this functionality for serializable transactions.
Instead, it's gated behind a session setting called
`enable_shared_locking_for_serializable`.

Informs #91545

Release note (sql change): SELECT FOR SHARE and SELECT FOR UPDATE
previously did not acquire any locks. Users issuing these statements
would expect them to acquire shared locks (multiple readers allowed,
no writers though). This patch switches over the behavior to acquire
such read locks.

For serializable transactions, we default to the old behaviour, unless
the `enable_shared_locking_for_serializable` session setting is set
to true. We'll probably switch this behavior in the future, but for
now, given shared locks are a preview feature, we gate things behind
a session setting.

111441: goschedstats: reduce underloadedRunnablePerProcThreshold r=aadityasondhi a=sumeerbhola

By reducing this threshold, we more often sample the stats at 1ms intervals, which is desirable for admission control slot adjustment.

Fixes #111125

Epic: none

Release note: None

111466: ui: update default timescale to 1h r=maryliag a=maryliag

On the Metrics page, the default value was 10min, which was too small. This commit updates it to 1h to match the most selected value on the SQL Activity.

Fixes #96479

Release note: None

111521: roachprod: fix createTenantCertBundle script r=andy-kimball a=herkolategan

The `createTenantCertBundle` function has a script in it that has erroneous "+" characters which results in an error when trying to start a secure cluster. This change removes those characters.

Epic: None
Release Note: None

111526: kv/spanset: permit writes to lock table by shared lock latching configuration r=nvanbenschoten a=nvanbenschoten

Fixes #111409.
Fixes #111492.

This commit updates addLockTableSpans to account for the way that shared locking requests declare their latches. Even though they declare a "read" latch, they do so at max timestamp, which gives them sufficient isolation to write to the lock table without having to declare a write latch and be serialized with other shared lock acquisitions.

Release note: None

Co-authored-by: Arul Ajmani <[email protected]>
Co-authored-by: sumeerbhola <[email protected]>
Co-authored-by: maryliag <[email protected]>
Co-authored-by: Herko Lategan <[email protected]>
Co-authored-by: Nathan VanBenschoten <[email protected]>
  • Loading branch information
6 people committed Sep 29, 2023
6 parents 1787e21 + b387ed0 + 32adda2 + 8e4ef1b + 6c6c34e + 2c17fb7 commit 41927e1
Show file tree
Hide file tree
Showing 35 changed files with 363 additions and 54 deletions.
7 changes: 7 additions & 0 deletions pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

1 change: 1 addition & 0 deletions pkg/kv/kvserver/spanset/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@ go_test(
embed = [":spanset"],
deps = [
"//pkg/keys",
"//pkg/kv/kvserver/concurrency/lock",
"//pkg/roachpb",
"//pkg/storage",
"//pkg/testutils",
Expand Down
13 changes: 13 additions & 0 deletions pkg/kv/kvserver/spanset/batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -891,6 +891,19 @@ func addLockTableSpans(spans *SpanSet) *SpanSet {
if span.EndKey != nil {
ltEndKey, _ = keys.LockTableSingleKey(span.EndKey, nil)
}
if sa == SpanReadOnly && span.Timestamp == hlc.MaxTimestamp {
// Shared lock acquisition uses a read-only latch access with
// the maximum timestamp. This gives it sufficient isolation to
// write to the lock table without having to declare a write
// latch and be serialized with other shared lock acquisitions.
// For details, see DefaultDeclareIsolatedKeys.
//
// For the sake of this function, we consider this to be strong
// enough to declare write access to the lock table. This could
// be made cleaner if latch spans operated on locking strengths
// instead of read/write access.
sa = SpanReadWrite
}
withLocks.AddNonMVCC(sa, roachpb.Span{Key: ltKey, EndKey: ltEndKey})
})
return withLocks
Expand Down
31 changes: 25 additions & 6 deletions pkg/kv/kvserver/spanset/batch_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ import (
"testing"

"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/storage"
Expand Down Expand Up @@ -46,13 +47,31 @@ func TestReadWriterDeclareLockTable(t *testing.T) {
},
}
for fnName, fn := range fns {
for _, sa := range []spanset.SpanAccess{spanset.SpanReadOnly, spanset.SpanReadWrite} {
for _, str := range []lock.Strength{lock.None, lock.Shared, lock.Exclusive, lock.Intent} {
for _, mvcc := range []bool{false, true} {
t.Run(fmt.Sprintf("%s,access=%s,mvcc=%t", fnName, sa, mvcc), func(t *testing.T) {
if !mvcc && (str == lock.Shared || str == lock.Exclusive) {
// Invalid combination.
continue
}
t.Run(fmt.Sprintf("%s,strength=%s,mvcc=%t", fnName, str, mvcc), func(t *testing.T) {
span := roachpb.Span{Key: startKey, EndKey: endKey}
var sa spanset.SpanAccess
var latchTs hlc.Timestamp
switch str {
case lock.None:
sa, latchTs = spanset.SpanReadOnly, ts
case lock.Shared:
sa, latchTs = spanset.SpanReadOnly, hlc.MaxTimestamp
case lock.Exclusive:
sa, latchTs = spanset.SpanReadWrite, ts
case lock.Intent:
sa, latchTs = spanset.SpanReadWrite, ts
default:
t.Fatal("unexpected")
}
ss := spanset.New()
if mvcc {
ss.AddMVCC(sa, span, ts)
ss.AddMVCC(sa, span, latchTs)
} else {
ss.AddNonMVCC(sa, span)
}
Expand All @@ -64,10 +83,10 @@ func TestReadWriterDeclareLockTable(t *testing.T) {
require.Error(t, rw.MVCCIterate(ltEndKey, ltEndKey.Next(), storage.MVCCKeyIterKind, storage.IterKeyTypePointsOnly, nil))

err := rw.PutUnversioned(ltStartKey, []byte("value"))
if sa == spanset.SpanReadWrite {
require.NoError(t, err)
} else {
if str == lock.None {
require.Error(t, err)
} else {
require.NoError(t, err)
}
require.Error(t, rw.PutUnversioned(ltEndKey, []byte("value")))
})
Expand Down
6 changes: 3 additions & 3 deletions pkg/roachprod/install/cluster_synced.go
Original file line number Diff line number Diff line change
Expand Up @@ -1548,9 +1548,9 @@ if [[ $VERSION = v22 ]]; then
fi
%[1]s cert create-node %[2]s $SHARED_ARGS
%[1]s cert create-tenant-client %[3]d %[2]s $SHARED_ARGS
+%[1]s cert create-client root $TENANT_SCOPE_OPT $SHARED_ARGS
+%[1]s cert create-client testuser $TENANT_SCOPE_OPT $SHARED_ARGS
+tar cvf %[4]s $CERT_DIR
%[1]s cert create-client root $TENANT_SCOPE_OPT $SHARED_ARGS
%[1]s cert create-client testuser $TENANT_SCOPE_OPT $SHARED_ARGS
tar cvf %[4]s $CERT_DIR
`,
cockroachNodeBinary(c, node),
strings.Join(nodeNames, " "),
Expand Down
4 changes: 4 additions & 0 deletions pkg/sql/exec_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -3641,6 +3641,10 @@ func (m *sessionDataMutator) SetDurableLockingForSerializable(val bool) {
m.data.DurableLockingForSerializable = val
}

func (m *sessionDataMutator) SetSharedLockingForSerializable(val bool) {
m.data.SharedLockingForSerializable = val
}

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

// quantizeCounts ensures that the Count field in the
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/logictest/testdata/logic_test/information_schema
Original file line number Diff line number Diff line change
Expand Up @@ -5346,6 +5346,7 @@ enable_insert_fast_path on
enable_multiple_modifications_of_table off
enable_multiregion_placement_policy off
enable_seqscan on
enable_shared_locking_for_serializable off
enable_super_regions off
enable_zigzag_join off
enforce_home_region off
Expand Down
3 changes: 3 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/pg_catalog
Original file line number Diff line number Diff line change
Expand Up @@ -2788,6 +2788,7 @@ enable_insert_fast_path on N
enable_multiple_modifications_of_table off NULL NULL NULL string
enable_multiregion_placement_policy off NULL NULL NULL string
enable_seqscan on NULL NULL NULL string
enable_shared_locking_for_serializable off NULL NULL NULL string
enable_super_regions off NULL NULL NULL string
enable_zigzag_join off NULL NULL NULL string
enforce_home_region off NULL NULL NULL string
Expand Down Expand Up @@ -2949,6 +2950,7 @@ enable_insert_fast_path on N
enable_multiple_modifications_of_table off NULL user NULL off off
enable_multiregion_placement_policy off NULL user NULL off off
enable_seqscan on NULL user NULL on on
enable_shared_locking_for_serializable off NULL user NULL off off
enable_super_regions off NULL user NULL off off
enable_zigzag_join off NULL user NULL off off
enforce_home_region off NULL user NULL off off
Expand Down Expand Up @@ -3107,6 +3109,7 @@ enable_insert_fast_path NULL NULL NULL
enable_multiple_modifications_of_table NULL NULL NULL NULL NULL
enable_multiregion_placement_policy NULL NULL NULL NULL NULL
enable_seqscan NULL NULL NULL NULL NULL
enable_shared_locking_for_serializable NULL NULL NULL NULL NULL
enable_super_regions NULL NULL NULL NULL NULL
enable_zigzag_join NULL NULL NULL NULL NULL
enforce_home_region NULL NULL NULL NULL NULL
Expand Down
119 changes: 119 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/select_for_share
Original file line number Diff line number Diff line change
@@ -0,0 +1,119 @@
# LogicTest: !local-mixed-22.2-23.1

statement ok
CREATE TABLE t(a INT PRIMARY KEY);
INSERT INTO t VALUES(1);
GRANT ALL ON t TO testuser;
CREATE USER testuser2 WITH VIEWACTIVITY;
GRANT SYSTEM MODIFYCLUSTERSETTING TO testuser;
GRANT ALL ON t TO testuser2;

user testuser

statement ok
SET enable_shared_locking_for_serializable = true;

statement ok
BEGIN

query I
SELECT * FROM t WHERE a = 1 FOR SHARE;
----
1

# Start another transaction to show multiple transactions can acquire SHARED
# locks at the same time.

user root

statement ok
SET enable_shared_locking_for_serializable = true;

statement ok
BEGIN

query I
SELECT * FROM t WHERE a = 1 FOR SHARE;
----
1

user testuser2

statement async writeReq count 1
UPDATE t SET a = 2 WHERE a = 1

# TODO(arul): Until https://github.com/cockroachdb/cockroach/issues/107766 is
# addressed, we'll incorrectly report shared locks as having "Exclusive" lock
# strength; We'll also only report a single holder (the other row in there is
# the waiting UPDATE request, not the second shared lock holder). However,
# having this query in here is useful to make sure there are locks and waiters
# on our key, meaning setting the cluster setting above actually did something;
# otherwise, had we used non-locking reads, we'd have failed here.
query TTTTTTTBB colnames,retry,rowsort
SELECT database_name, schema_name, table_name, lock_key_pretty, lock_strength, durability, isolation_level, granted, contended FROM crdb_internal.cluster_locks
----
database_name schema_name table_name lock_key_pretty lock_strength durability isolation_level granted contended
test public t /Table/106/1/1/0 Exclusive Unreplicated SERIALIZABLE true true
test public t /Table/106/1/1/0 Exclusive Unreplicated SERIALIZABLE false true

# Commit the first transaction and rollback the second.

user testuser

statement ok
COMMIT

user root

statement ok
ROLLBACK

user testuser2

# Now that both the transactions that issued shared lock reads have been
# finalized, the write should be able to proceed.

awaitstatement writeReq

query I
SELECT * FROM t;
----
2

# ------------------------------------------------------------------------------
# Tests to ensure the enable_shared_locking_for_serializable session variable
# works as expected.
# -----------------------------------------------------------------------------

user testuser

statement ok
SET enable_shared_locking_for_serializable = false

statement ok
BEGIN ISOLATION LEVEL SERIALIZABLE

query I
SELECT * FROM t WHERE a = 2 FOR SHARE
----
2

user testuser2

query TTTTTTTBB colnames,retry,rowsort
SELECT database_name, schema_name, table_name, lock_key_pretty, lock_strength, durability, isolation_level, granted, contended FROM crdb_internal.cluster_locks
----
database_name schema_name table_name lock_key_pretty lock_strength durability isolation_level granted contended

user testuser

statement ok
COMMIT

# TODO(arul): Add a test to show that the session setting doesn't apply to read
# committed transactions. We currently can't issue SELECT FOR SHARE statements
# in read committed transactions because durable locking hasn't been fully
# hooked up.



11 changes: 11 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/select_for_update
Original file line number Diff line number Diff line change
Expand Up @@ -293,6 +293,7 @@ ROLLBACK
statement ok
BEGIN READ ONLY

skipif config local-mixed-22.2-23.1
statement error cannot execute FOR SHARE in a read-only transaction
SELECT * FROM t FOR SHARE

Expand All @@ -302,6 +303,7 @@ ROLLBACK
statement ok
BEGIN READ ONLY

skipif config local-mixed-22.2-23.1
statement error cannot execute FOR KEY SHARE in a read-only transaction
SELECT * FROM t FOR KEY SHARE

Expand Down Expand Up @@ -357,24 +359,33 @@ BEGIN; UPDATE t SET v = 2 WHERE k = 1

user testuser

statement ok
SET enable_shared_locking_for_serializable = true

skipif config local-mixed-22.2-23.1
query error pgcode 55P03 could not obtain lock on row \(k\)=\(1\) in t@t_pkey
SELECT v, v2 FROM t JOIN t2 USING (k) FOR SHARE FOR SHARE OF t NOWAIT

skipif config local-mixed-22.2-23.1
query error pgcode 55P03 could not obtain lock on row \(k\)=\(1\) in t@t_pkey
SELECT v, v2 FROM t JOIN t2 USING (k) FOR SHARE OF t2 FOR SHARE OF t NOWAIT

skipif config local-mixed-22.2-23.1
query error pgcode 55P03 could not obtain lock on row \(k\)=\(1\) in t@t_pkey
SELECT v, v2 FROM t JOIN t2 USING (k) FOR SHARE NOWAIT FOR SHARE OF t

skipif config local-mixed-22.2-23.1
query error pgcode 55P03 could not obtain lock on row \(k\)=\(1\) in t@t_pkey
SELECT v, v2 FROM t JOIN t2 USING (k) FOR SHARE NOWAIT FOR SHARE OF t2

statement ok
SET statement_timeout = '10ms'

skipif config local-mixed-22.2-23.1
query error pgcode 57014 query execution canceled due to statement timeout
SELECT v, v2 FROM t JOIN t2 USING (k) FOR SHARE FOR SHARE OF t2 NOWAIT

skipif config local-mixed-22.2-23.1
query error pgcode 57014 query execution canceled due to statement timeout
SELECT v, v2 FROM t JOIN t2 USING (k) FOR SHARE OF t FOR SHARE OF t2 NOWAIT

Expand Down
1 change: 1 addition & 0 deletions pkg/sql/logictest/testdata/logic_test/show_source
Original file line number Diff line number Diff line change
Expand Up @@ -70,6 +70,7 @@ enable_insert_fast_path on
enable_multiple_modifications_of_table off
enable_multiregion_placement_policy off
enable_seqscan on
enable_shared_locking_for_serializable off
enable_super_regions off
enable_zigzag_join off
enforce_home_region off
Expand Down
7 changes: 7 additions & 0 deletions pkg/sql/logictest/tests/fakedist-disk/generated_test.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

7 changes: 7 additions & 0 deletions pkg/sql/logictest/tests/fakedist-vec-off/generated_test.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

7 changes: 7 additions & 0 deletions pkg/sql/logictest/tests/fakedist/generated_test.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

Loading

0 comments on commit 41927e1

Please sign in to comment.