From 74d1fb9b7a1d34b1da73d5fdaba64789c0f10841 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Mon, 6 Jun 2022 14:30:39 -0400 Subject: [PATCH 1/3] keys: resolve subtle non-bug by exporting RaftLogKeyFromPrefix This commit resolves a subtle interaction that came tantalizingly close to a bug in `StateLoader.LoadLastIndex`. The method uses the `StateLoader`'s underlying `keys.RangeIDPrefixBuf` to generate two keys (`RaftLogPrefix` and `RaftLogKey`) that are in use as the same time. `RangeIDPrefixBuf` avoids heap allocations by sharing a single byte slice across all keys that it generates. This is why the type has the comment: > The generated keys are only valid until the next call to one of the > key generation methods. As would be expected, given this comment, the second use of the `RangeIDPrefixBuf` overwrote the buffer and invalidated the first key. However, it happened to get lucky and generate a new key with the same prefix as the old key. As a result, the contents of the first key did not change. To make this aliasing more explicit and avoid this becoming a real bug in the future, we introduce a new `RaftLogKeyFromPrefix` function that callers can use to generate raft log entry keys from a raft log prefix. --- pkg/keys/keys.go | 8 +++++++- pkg/kv/kvserver/stateloader/stateloader.go | 2 +- 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/pkg/keys/keys.go b/pkg/keys/keys.go index 87d1b4b91a96..5ab51a93c6e1 100644 --- a/pkg/keys/keys.go +++ b/pkg/keys/keys.go @@ -331,6 +331,12 @@ func RaftLogKey(rangeID roachpb.RangeID, logIndex uint64) roachpb.Key { return MakeRangeIDPrefixBuf(rangeID).RaftLogKey(logIndex) } +// RaftLogKeyFromPrefix returns a system-local key for a Raft log entry, using +// the provided Raft log prefix. +func RaftLogKeyFromPrefix(raftLogPrefix []byte, logIndex uint64) roachpb.Key { + return encoding.EncodeUint64Ascending(raftLogPrefix, logIndex) +} + // RaftReplicaIDKey returns a system-local key for a RaftReplicaID. func RaftReplicaIDKey(rangeID roachpb.RangeID) roachpb.Key { return MakeRangeIDPrefixBuf(rangeID).RaftReplicaIDKey() @@ -1029,7 +1035,7 @@ func (b RangeIDPrefixBuf) RaftLogPrefix() roachpb.Key { // RaftLogKey returns a system-local key for a Raft log entry. func (b RangeIDPrefixBuf) RaftLogKey(logIndex uint64) roachpb.Key { - return encoding.EncodeUint64Ascending(b.RaftLogPrefix(), logIndex) + return RaftLogKeyFromPrefix(b.RaftLogPrefix(), logIndex) } // RaftReplicaIDKey returns a system-local key for a RaftReplicaID. diff --git a/pkg/kv/kvserver/stateloader/stateloader.go b/pkg/kv/kvserver/stateloader/stateloader.go index ecabcd9fe8bd..6c9eb30c91e5 100644 --- a/pkg/kv/kvserver/stateloader/stateloader.go +++ b/pkg/kv/kvserver/stateloader/stateloader.go @@ -300,7 +300,7 @@ func (rsl StateLoader) LoadLastIndex(ctx context.Context, reader storage.Reader) defer iter.Close() var lastIndex uint64 - iter.SeekLT(storage.MakeMVCCMetadataKey(rsl.RaftLogKey(math.MaxUint64))) + iter.SeekLT(storage.MakeMVCCMetadataKey(keys.RaftLogKeyFromPrefix(prefix, math.MaxUint64))) if ok, _ := iter.Valid(); ok { key := iter.Key() var err error From 8012d6a9235eaae74fa9a4405c976ec5da90a137 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Mon, 6 Jun 2022 14:36:36 -0400 Subject: [PATCH 2/3] kv/kvserver: avoid redundant encoding work for repeat calls to RaftLogKey This commit uses the new `RaftLogKeyFromPrefix` function to avoid some redundant encoding work elsewhere due to repeat calls to `RaftLogKey`. Instead of repeatedly encoding the Raft log prefix, we encoded it once and repeatedly rewrite the suffix. --- pkg/kv/kvserver/replica_raft.go | 3 ++- pkg/kv/kvserver/replica_raftstorage.go | 5 +++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/pkg/kv/kvserver/replica_raft.go b/pkg/kv/kvserver/replica_raft.go index b7db2bd5f989..f80f91c4cdb1 100644 --- a/pkg/kv/kvserver/replica_raft.go +++ b/pkg/kv/kvserver/replica_raft.go @@ -2110,8 +2110,9 @@ func handleTruncatedStateBelowRaftPreApply( } else { // NB: RangeIDPrefixBufs have sufficient capacity (32 bytes) to // avoid allocating when constructing Raft log keys (16 bytes). + prefix := prefixBuf.RaftLogPrefix() for idx := currentTruncatedState.Index + 1; idx <= suggestedTruncatedState.Index; idx++ { - if err := readWriter.ClearUnversioned(prefixBuf.RaftLogKey(idx)); err != nil { + if err := readWriter.ClearUnversioned(keys.RaftLogKeyFromPrefix(prefix, idx)); err != nil { return false, errors.Wrapf(err, "unable to clear truncated Raft entries for %+v at index %d", suggestedTruncatedState, idx) } diff --git a/pkg/kv/kvserver/replica_raftstorage.go b/pkg/kv/kvserver/replica_raftstorage.go index 5da1475c1448..8408d9e2304f 100644 --- a/pkg/kv/kvserver/replica_raftstorage.go +++ b/pkg/kv/kvserver/replica_raftstorage.go @@ -649,11 +649,12 @@ func (r *Replica) append( if len(entries) == 0 { return prevLastIndex, prevLastTerm, prevRaftLogSize, nil } + prefix := r.raftMu.stateLoader.RaftLogPrefix() var diff enginepb.MVCCStats var value roachpb.Value for i := range entries { ent := &entries[i] - key := r.raftMu.stateLoader.RaftLogKey(ent.Index) + key := keys.RaftLogKeyFromPrefix(prefix, ent.Index) if err := value.SetProto(ent); err != nil { return 0, 0, 0, err @@ -689,7 +690,7 @@ func (r *Replica) append( for i := lastIndex + 1; i <= prevLastIndex; i++ { // Note that the caller is in charge of deleting any sideloaded payloads // (which they must only do *after* the batch has committed). - err := storage.MVCCDelete(ctx, eng, &diff, r.raftMu.stateLoader.RaftLogKey(i), + err := storage.MVCCDelete(ctx, eng, &diff, keys.RaftLogKeyFromPrefix(prefix, i), hlc.Timestamp{}, hlc.ClockTimestamp{}, nil) if err != nil { return 0, 0, 0, err From c5497ba4758c55cfb7be120c882a811f187d7513 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Mon, 6 Jun 2022 14:54:33 -0400 Subject: [PATCH 3/3] keys: move Raft log suffix parsing into pkg/keys This did not belong in stateloader.go. --- pkg/keys/keys.go | 9 ++++++++- pkg/kv/kvserver/stateloader/BUILD.bazel | 1 - pkg/kv/kvserver/stateloader/stateloader.go | 13 ++++++++----- 3 files changed, 16 insertions(+), 7 deletions(-) diff --git a/pkg/keys/keys.go b/pkg/keys/keys.go index 5ab51a93c6e1..1d39b9c0e700 100644 --- a/pkg/keys/keys.go +++ b/pkg/keys/keys.go @@ -337,6 +337,13 @@ func RaftLogKeyFromPrefix(raftLogPrefix []byte, logIndex uint64) roachpb.Key { return encoding.EncodeUint64Ascending(raftLogPrefix, logIndex) } +// DecodeRaftLogKeyFromSuffix parses the suffix of a system-local key for a Raft +// log entry and returns the entry's log index. +func DecodeRaftLogKeyFromSuffix(raftLogSuffix []byte) (uint64, error) { + _, logIndex, err := encoding.DecodeUint64Ascending(raftLogSuffix) + return logIndex, err +} + // RaftReplicaIDKey returns a system-local key for a RaftReplicaID. func RaftReplicaIDKey(rangeID roachpb.RangeID) roachpb.Key { return MakeRangeIDPrefixBuf(rangeID).RaftReplicaIDKey() @@ -453,7 +460,7 @@ func LockTableSingleKey(key roachpb.Key, buf []byte) (roachpb.Key, []byte) { } // DecodeLockTableSingleKey decodes the single-key lock table key to return the key -// that was locked.. +// that was locked. func DecodeLockTableSingleKey(key roachpb.Key) (lockedKey roachpb.Key, err error) { if !bytes.HasPrefix(key, LocalRangeLockTablePrefix) { return nil, errors.Errorf("key %q does not have %q prefix", diff --git a/pkg/kv/kvserver/stateloader/BUILD.bazel b/pkg/kv/kvserver/stateloader/BUILD.bazel index 56f558fb6afb..89250e5f4926 100644 --- a/pkg/kv/kvserver/stateloader/BUILD.bazel +++ b/pkg/kv/kvserver/stateloader/BUILD.bazel @@ -15,7 +15,6 @@ go_library( "//pkg/roachpb", "//pkg/storage", "//pkg/storage/enginepb", - "//pkg/util/encoding", "//pkg/util/hlc", "//pkg/util/log", "//pkg/util/protoutil", diff --git a/pkg/kv/kvserver/stateloader/stateloader.go b/pkg/kv/kvserver/stateloader/stateloader.go index 6c9eb30c91e5..575c3e7117b0 100644 --- a/pkg/kv/kvserver/stateloader/stateloader.go +++ b/pkg/kv/kvserver/stateloader/stateloader.go @@ -19,7 +19,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" - "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" @@ -45,7 +44,7 @@ type StateLoader struct { keys.RangeIDPrefixBuf } -// Make creates a a StateLoader. +// Make creates a StateLoader. func Make(rangeID roachpb.RangeID) StateLoader { rsl := StateLoader{ RangeIDPrefixBuf: keys.MakeRangeIDPrefixBuf(rangeID), @@ -302,11 +301,15 @@ func (rsl StateLoader) LoadLastIndex(ctx context.Context, reader storage.Reader) var lastIndex uint64 iter.SeekLT(storage.MakeMVCCMetadataKey(keys.RaftLogKeyFromPrefix(prefix, math.MaxUint64))) if ok, _ := iter.Valid(); ok { - key := iter.Key() + key := iter.UnsafeKey().Key + if len(key) < len(prefix) { + log.Fatalf(ctx, "unable to decode Raft log index key: len(%s) < len(%s)", key.String(), prefix.String()) + } + suffix := key[len(prefix):] var err error - _, lastIndex, err = encoding.DecodeUint64Ascending(key.Key[len(prefix):]) + lastIndex, err = keys.DecodeRaftLogKeyFromSuffix(suffix) if err != nil { - log.Fatalf(ctx, "unable to decode Raft log index key: %s", key) + log.Fatalf(ctx, "unable to decode Raft log index key: %s; %v", key.String(), err) } }