From e484435cbea5b828bdb9a026031851800e40d79f Mon Sep 17 00:00:00 2001 From: Arul Ajmani Date: Wed, 26 Apr 2023 20:17:50 -0400 Subject: [PATCH] lockspanset: introduce LockSpanSets Today we use `spanset.SpanSet` to declare both lock and latch spans. However, `SpanSets` do not generalize to more lock strengths. This patch introduces `LockSpanSets` in preperation for shared locks. We don't make use of them in the `concurrency` package yet; that will happen in an upcoming commit. The main motivations for the new `LockSpanSets` structure are: - A desire to store/declare lock spans using lock strengths, not span access. - There isn't a need to store MVCC spans -- lock strengths do not have associated timestamps. - There is no need to store global/local lock keys separately, like we do for latches. Furthermore, we only port over methods on `SpanSet` that lock spans made use of in this patch. Informs: #102008 Release note: None --- .github/CODEOWNERS | 1 + pkg/BUILD.bazel | 4 + pkg/kv/kvserver/concurrency/lock/locking.go | 8 ++ pkg/kv/kvserver/lockspanset/BUILD.bazel | 29 ++++ pkg/kv/kvserver/lockspanset/lockspanset.go | 106 ++++++++++++++ .../kvserver/lockspanset/lockspanset_test.go | 129 ++++++++++++++++++ 6 files changed, 277 insertions(+) create mode 100644 pkg/kv/kvserver/lockspanset/BUILD.bazel create mode 100644 pkg/kv/kvserver/lockspanset/lockspanset.go create mode 100644 pkg/kv/kvserver/lockspanset/lockspanset_test.go diff --git a/.github/CODEOWNERS b/.github/CODEOWNERS index e1590709ef1b..5b23a978d9b5 100644 --- a/.github/CODEOWNERS +++ b/.github/CODEOWNERS @@ -304,6 +304,7 @@ /pkg/kv/kvserver/kvstorage/ @cockroachdb/repl-prs /pkg/kv/kvserver/liveness/ @cockroachdb/kv-prs /pkg/kv/kvserver/load/ @cockroachdb/kv-prs +/pkg/kv/kvserver/lockspanset/ @cockroachdb/kv-prs /pkg/kv/kvserver/logstore/ @cockroachdb/repl-prs /pkg/kv/kvserver/loqrecovery/ @cockroachdb/repl-prs /pkg/kv/kvserver/multiqueue/ @cockroachdb/kv-prs diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index 5d9f2d872021..0f9484a519db 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -231,6 +231,7 @@ ALL_TESTS = [ "//pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker:kvflowtokentracker_test", "//pkg/kv/kvserver/kvstorage:kvstorage_test", "//pkg/kv/kvserver/liveness:liveness_test", + "//pkg/kv/kvserver/lockspanset:lockspanset_test", "//pkg/kv/kvserver/logstore:logstore_test", "//pkg/kv/kvserver/loqrecovery/loqrecoverypb:loqrecoverypb_test", "//pkg/kv/kvserver/loqrecovery:loqrecovery_test", @@ -1312,6 +1313,8 @@ GO_TARGETS = [ "//pkg/kv/kvserver/liveness:liveness", "//pkg/kv/kvserver/liveness:liveness_test", "//pkg/kv/kvserver/load:load", + "//pkg/kv/kvserver/lockspanset:lockspanset", + "//pkg/kv/kvserver/lockspanset:lockspanset_test", "//pkg/kv/kvserver/logstore:logstore", "//pkg/kv/kvserver/logstore:logstore_test", "//pkg/kv/kvserver/loqrecovery/loqrecoverypb:loqrecoverypb", @@ -2769,6 +2772,7 @@ GET_X_DATA_TARGETS = [ "//pkg/kv/kvserver/liveness:get_x_data", "//pkg/kv/kvserver/liveness/livenesspb:get_x_data", "//pkg/kv/kvserver/load:get_x_data", + "//pkg/kv/kvserver/lockspanset:get_x_data", "//pkg/kv/kvserver/logstore:get_x_data", "//pkg/kv/kvserver/loqrecovery:get_x_data", "//pkg/kv/kvserver/loqrecovery/loqrecoverypb:get_x_data", diff --git a/pkg/kv/kvserver/concurrency/lock/locking.go b/pkg/kv/kvserver/concurrency/lock/locking.go index 111393ebad96..0747620ccfa8 100644 --- a/pkg/kv/kvserver/concurrency/lock/locking.go +++ b/pkg/kv/kvserver/concurrency/lock/locking.go @@ -50,10 +50,18 @@ var ExclusiveLocksBlockNonLockingReads = settings.RegisterBoolSetting( true, ) +// MaxStrength is the maximum value in the Strength enum. +const MaxStrength = Intent + // MaxDurability is the maximum value in the Durability enum. const MaxDurability = Unreplicated func init() { + for v := range Strength_name { + if st := Strength(v); st > MaxStrength { + panic(fmt.Sprintf("Strength (%s) with value larger than MaxDurability", st)) + } + } for v := range Durability_name { if d := Durability(v); d > MaxDurability { panic(fmt.Sprintf("Durability (%s) with value larger than MaxDurability", d)) diff --git a/pkg/kv/kvserver/lockspanset/BUILD.bazel b/pkg/kv/kvserver/lockspanset/BUILD.bazel new file mode 100644 index 000000000000..65aa3c825c65 --- /dev/null +++ b/pkg/kv/kvserver/lockspanset/BUILD.bazel @@ -0,0 +1,29 @@ +load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data") +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "lockspanset", + srcs = ["lockspanset.go"], + importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/lockspanset", + visibility = ["//visibility:public"], + deps = [ + "//pkg/kv/kvserver/concurrency/lock", + "//pkg/roachpb", + ], +) + +go_test( + name = "lockspanset_test", + srcs = ["lockspanset_test.go"], + args = ["-test.timeout=295s"], + embed = [":lockspanset"], + deps = [ + "//pkg/kv/kvserver/concurrency/lock", + "//pkg/roachpb", + "//pkg/util/leaktest", + "//pkg/util/log", + "@com_github_stretchr_testify//require", + ], +) + +get_x_data(name = "get_x_data") diff --git a/pkg/kv/kvserver/lockspanset/lockspanset.go b/pkg/kv/kvserver/lockspanset/lockspanset.go new file mode 100644 index 000000000000..4fe1b2e9fa2d --- /dev/null +++ b/pkg/kv/kvserver/lockspanset/lockspanset.go @@ -0,0 +1,106 @@ +// 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 lockspanset + +import ( + "fmt" + "strings" + "sync" + + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" + "github.com/cockroachdb/cockroach/pkg/roachpb" +) + +const NumLockStrength = lock.MaxStrength + 1 + +type LockSpanSet struct { + spans [NumLockStrength][]roachpb.Span +} + +var lockSpanSetPool = sync.Pool{ + New: func() interface{} { return new(LockSpanSet) }, +} + +// New creates a new empty LockSpanSet. +func New() *LockSpanSet { + return lockSpanSetPool.Get().(*LockSpanSet) +} + +// GetSpans returns a slice of spans with the given strength. +func (l *LockSpanSet) GetSpans(str lock.Strength) []roachpb.Span { + return l.spans[str] +} + +// Add adds the supplied span to the LockSpanSet to be accessed with the given +// lock strength. +func (l *LockSpanSet) Add(str lock.Strength, span roachpb.Span) { + l.spans[str] = append(l.spans[str], span) +} + +// SortAndDeDup sorts the spans in the LockSpanSet and removes any duplicates. +func (l *LockSpanSet) SortAndDeDup() { + for st := range l.spans { + l.spans[st], _ /* distinct */ = roachpb.MergeSpans(&l.spans[st]) + } +} + +// Release releases the LockSpanSet and its underlying slices. The receiver +// should not be used after being released. +func (l *LockSpanSet) Release() { + for st := range l.spans { + // Recycle slice if capacity below threshold. + const maxRecycleCap = 8 + var recycle []roachpb.Span + if sl := l.spans[st]; cap(sl) <= maxRecycleCap { + for i := range sl { + sl[i] = roachpb.Span{} + } + recycle = sl[:0] + } + l.spans[st] = recycle + } + lockSpanSetPool.Put(l) +} + +// Empty returns whether the set contains any spans across all lock strengths. +func (l *LockSpanSet) Empty() bool { + return l.Len() == 0 +} + +// String prints a string representation of the LockSpanSet. +func (l *LockSpanSet) String() string { + var buf strings.Builder + for st := lock.Strength(0); st < NumLockStrength; st++ { + for _, span := range l.GetSpans(st) { + fmt.Fprintf(&buf, "%s: %s\n", + st, span) + } + } + return buf.String() +} + +// Len returns the total number of spans tracked across all strengths. +func (l *LockSpanSet) Len() int { + var count int + for st := lock.Strength(0); st < NumLockStrength; st++ { + count += len(l.GetSpans(st)) + } + return count +} + +// Copy copies the LockSpanSet. +func (l *LockSpanSet) Copy() *LockSpanSet { + n := New() + for st := range l.spans { + n.spans[st] = append(n.spans[st], l.spans[st]...) + } + return n +} diff --git a/pkg/kv/kvserver/lockspanset/lockspanset_test.go b/pkg/kv/kvserver/lockspanset/lockspanset_test.go new file mode 100644 index 000000000000..1f2ccef6595b --- /dev/null +++ b/pkg/kv/kvserver/lockspanset/lockspanset_test.go @@ -0,0 +1,129 @@ +// 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 lockspanset + +import ( + "testing" + + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/stretchr/testify/require" +) + +// Test that spans are properly classified according to the lock strength they +// are added with. +func TestGetSpansStrength(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + var lss LockSpanSet + spA := roachpb.Span{Key: roachpb.Key("a")} + spBC := roachpb.Span{Key: roachpb.Key("b"), EndKey: roachpb.Key("c")} + spD := roachpb.Span{Key: roachpb.Key("d")} + spE := roachpb.Span{Key: roachpb.Key("e")} + spF := roachpb.Span{Key: roachpb.Key("f")} + spGH := roachpb.Span{Key: roachpb.Key("g"), EndKey: roachpb.Key("h")} + + lss.Add(lock.None, spA) + lss.Add(lock.Shared, spBC) + lss.Add(lock.Update, spD) + + lss.Add(lock.Exclusive, spE) + lss.Add(lock.Exclusive, spGH) + + lss.Add(lock.Intent, spF) + + spans := lss.GetSpans(lock.None) + require.Equal(t, []roachpb.Span{spA}, spans) + + spans = lss.GetSpans(lock.Shared) + require.Equal(t, []roachpb.Span{spBC}, spans) + + spans = lss.GetSpans(lock.Update) + require.Equal(t, []roachpb.Span{spD}, spans) + + spans = lss.GetSpans(lock.Exclusive) + require.Equal(t, []roachpb.Span{spE, spGH}, spans) + + spans = lss.GetSpans(lock.Intent) + require.Equal(t, []roachpb.Span{spF}, spans) +} + +// TestLockSpanSetSortAndDeDup ensures that spans in a lock span set are sorted +// and de-duplicated correctly. Spans should be sorted and de-duplicated within +// a particular lock strength but not amongst different lock strengths. +func TestLockSpanSetSortAndDeDup(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + makeSpan := func(start, end string) roachpb.Span { + var endKey roachpb.Key + if end != "" { + endKey = roachpb.Key(end) + } + return roachpb.Span{Key: roachpb.Key(start), EndKey: endKey} + } + + spA := makeSpan("a", "") + spB := makeSpan("b", "") + spCF := makeSpan("c", "f") + spEH := makeSpan("e", "h") + spGJ := makeSpan("g", "j") + spIL := makeSpan("i", "l") + spXZ := makeSpan("x", "z") + + var lss LockSpanSet + lss.Add(lock.None, spA) + lss.Add(lock.None, spA) // duplicate + lss.Add(lock.None, spA) // duplicate + lss.Add(lock.None, spA) // duplicate + lss.Add(lock.None, spCF) // overlapping + lss.Add(lock.None, spEH) // overlapping + lss.Add(lock.None, spGJ) // overlapping + lss.Add(lock.None, spB) // out of order + + // Shared. + lss.Add(lock.Shared, spXZ) // out of order + lss.Add(lock.Shared, spA) // should not be considered a duplicate + lss.Add(lock.Shared, spIL) // should not overlap + + lss.SortAndDeDup() + + spans := lss.GetSpans(lock.None) + require.Len(t, spans, 3) + require.Equal(t, spans, []roachpb.Span{spA, spB, makeSpan("c", "j")}) + + spans = lss.GetSpans(lock.Shared) + require.Equal(t, spans, []roachpb.Span{spA, spIL, spXZ}) + + require.Len(t, lss.GetSpans(lock.Update), 0) + require.Len(t, lss.GetSpans(lock.Exclusive), 0) + require.Len(t, lss.GetSpans(lock.Intent), 0) +} + +// TestLockSpanSetCopy tests copying of lock span sets. +func TestLockSpanSetCopy(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + lss := New() + lss.Add(lock.None, roachpb.Span{Key: roachpb.Key("abc")}) + lss.Add(lock.Update, roachpb.Span{Key: roachpb.Key("b"), EndKey: roachpb.Key("c")}) + + c := lss.Copy() + require.Equal(t, lss, c) + + // modifying element in lss should not modify copy + lss.Add(lock.None, roachpb.Span{Key: roachpb.Key("a"), EndKey: roachpb.Key("b")}) + require.NotEqual(t, lss, c) +}