Skip to content

Commit

Permalink
kvserver: don't reset local keys in tscache on r1 lease move
Browse files Browse the repository at this point in the history
applyReadSummaryToTimestampCache had a bug in how it dealt with r1's
descriptor: it bumped the timestamp cache over the whole keyspace that
the descriptor declared ([\x00,\x02)). That's bad, because it includes
the \x01... keys. That's the "local" keyspace, which doesn't belong to
any range in particular (so, r1's descriptor is arguably lying). Bumping
the timestamp cache over the local keyspace had the consequence (maybe
among others) that the txn tombstone keys were included, leading to
transactions being aborted when they came to create their txn record
because the bumped timestamp cache was saying that there might have been
a tombstone in there.

Release note: A bug causing transactions to be spuriously aborted in
rare circumstances has been fixed.
  • Loading branch information
andreimatei committed Jun 10, 2021
1 parent 7a68e6e commit 2a2e749
Show file tree
Hide file tree
Showing 7 changed files with 148 additions and 20 deletions.
11 changes: 4 additions & 7 deletions pkg/keys/constants.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,9 +22,7 @@ import (
// These constants are single bytes for performance. They allow single-byte
// comparisons which are considerably faster than bytes.HasPrefix.
const (
LocalPrefixByte = '\x01'
localMaxByte = '\x02'
meta1PrefixByte = localMaxByte
meta1PrefixByte = roachpb.LocalMaxByte
meta2PrefixByte = '\x03'
metaMaxByte = '\x04'
systemPrefixByte = metaMaxByte
Expand All @@ -43,10 +41,9 @@ var (
MaxKey = roachpb.KeyMax

// LocalPrefix is the prefix for all local keys.
LocalPrefix = roachpb.Key{LocalPrefixByte}
// LocalMax is the end of the local key range. It is itself a global
// key.
LocalMax = roachpb.Key{localMaxByte}
LocalPrefix = roachpb.LocalPrefix
// LocalMax is the end of the local key range. It is itself a global key.
LocalMax = roachpb.LocalMax

// localSuffixLength specifies the length in bytes of all local
// key suffixes.
Expand Down
3 changes: 3 additions & 0 deletions pkg/kv/kvserver/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -265,6 +265,7 @@ go_test(
"replica_sst_snapshot_storage_test.go",
"replica_stats_test.go",
"replica_test.go",
"replica_tscache_test.go",
"replicate_queue_test.go",
"replicate_test.go",
"reset_quorum_test.go",
Expand Down Expand Up @@ -321,10 +322,12 @@ go_test(
"//pkg/kv/kvserver/protectedts/ptverifier",
"//pkg/kv/kvserver/raftentry",
"//pkg/kv/kvserver/rditer",
"//pkg/kv/kvserver/readsummary/rspb",
"//pkg/kv/kvserver/spanset",
"//pkg/kv/kvserver/split",
"//pkg/kv/kvserver/stateloader",
"//pkg/kv/kvserver/tenantrate",
"//pkg/kv/kvserver/tscache",
"//pkg/kv/kvserver/txnwait",
"//pkg/roachpb",
"//pkg/rpc",
Expand Down
12 changes: 3 additions & 9 deletions pkg/kv/kvserver/rditer/replica_data_iter.go
Original file line number Diff line number Diff line change
Expand Up @@ -189,16 +189,10 @@ func makeRangeLockTableKeyRanges(d *roachpb.RangeDescriptor) [2]KeyRange {

// MakeUserKeyRange returns the user key range.
func MakeUserKeyRange(d *roachpb.RangeDescriptor) KeyRange {
// The first range in the keyspace starts at KeyMin, which includes the
// node-local space. We need the original StartKey to find the range
// metadata, but the actual data starts at LocalMax.
dataStartKey := d.StartKey.AsRawKey()
if d.StartKey.Equal(roachpb.RKeyMin) {
dataStartKey = keys.LocalMax
}
userKeys := d.KeySpan()
return KeyRange{
Start: storage.MakeMVCCMetadataKey(dataStartKey),
End: storage.MakeMVCCMetadataKey(d.EndKey.AsRawKey()),
Start: storage.MakeMVCCMetadataKey(userKeys.Key.AsRawKey()),
End: storage.MakeMVCCMetadataKey(userKeys.EndKey.AsRawKey()),
}
}

Expand Down
11 changes: 7 additions & 4 deletions pkg/kv/kvserver/replica_tscache.go
Original file line number Diff line number Diff line change
Expand Up @@ -586,10 +586,12 @@ func collectReadSummaryFromTimestampCache(
keys.MakeRangeKeyPrefix(desc.StartKey),
keys.MakeRangeKeyPrefix(desc.EndKey),
)
userKeys := desc.KeySpan()
s.Global.LowWater, _ = tc.GetMax(
desc.StartKey.AsRawKey(),
desc.EndKey.AsRawKey(),
userKeys.Key.AsRawKey(),
userKeys.EndKey.AsRawKey(),
)

return s
}

Expand All @@ -606,9 +608,10 @@ func applyReadSummaryToTimestampCache(
s.Local.LowWater,
uuid.Nil, /* txnID */
)
userKeys := desc.KeySpan()
tc.Add(
desc.StartKey.AsRawKey(),
desc.EndKey.AsRawKey(),
userKeys.Key.AsRawKey(),
userKeys.EndKey.AsRawKey(),
s.Global.LowWater,
uuid.Nil, /* txnID */
)
Expand Down
100 changes: 100 additions & 0 deletions pkg/kv/kvserver/replica_tscache_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,100 @@
// Copyright 2018 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 kvserver

import (
"testing"
"time"

"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary/rspb"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/tscache"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/stretchr/testify/require"
)

// Test that, when applying the read summary for the range containing the
// beginning of the key space to the timestamp cache, the local keyspace is not
// generally bumped. The first range is special in that its descriptor declares
// that it includes the local keyspace (\x01...), except that key space is
// special and is not included in any range. applyReadToTimestampCache has
// special provisions for this.
func TestReadSummaryApplyForR1(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

baseTS := hlc.Timestamp{WallTime: 123}
manual := hlc.NewManualClock(baseTS.WallTime)
clock := hlc.NewClock(manual.UnixNano, time.Nanosecond)
tc := tscache.New(clock)

r1desc := roachpb.RangeDescriptor{
RangeID: 1,
StartKey: roachpb.RKeyMin,
EndKey: roachpb.RKeyMax,
}
ts1 := hlc.Timestamp{WallTime: 1000}
summary := rspb.ReadSummary{
Local: rspb.Segment{LowWater: ts1},
Global: rspb.Segment{LowWater: ts1},
}
applyReadSummaryToTimestampCache(tc, &r1desc, summary)
tc.GetMax(keys.LocalPrefix, nil)

// Make sure that updating the tscache did something, so the test is not
// fooling itself.
ts, _ := tc.GetMax(roachpb.Key("a"), nil)
require.Equal(t, ts1, ts)

// Check that the local keyspace was not affected.
ts, _ = tc.GetMax(keys.LocalPrefix, nil)
require.Equal(t, baseTS, ts)

// Check that the range-local keyspace for the range in question was affected.
ts, _ = tc.GetMax(keys.MakeRangeKeyPrefix(r1desc.StartKey), nil)
require.Equal(t, ts1, ts)
}

// This is the counter-part to TestReadSummaryApplyForR1, checking that the
// summary collection for first range has special logic avoiding the range-local
// keyspace.
func TestReadSummaryCollectForR1(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

baseTS := hlc.Timestamp{WallTime: 123}
manual := hlc.NewManualClock(baseTS.WallTime)
clock := hlc.NewClock(manual.UnixNano, time.Nanosecond)
tc := tscache.New(clock)

r1desc := roachpb.RangeDescriptor{
RangeID: 1,
StartKey: roachpb.RKeyMin,
EndKey: roachpb.RKey("a"),
}
r2desc := roachpb.RangeDescriptor{
RangeID: 1,
StartKey: roachpb.RKey("a"),
EndKey: roachpb.RKeyMax,
}
// Populate the timestamp cache for a range-local key for r2.
tc.Add(keys.MakeRangeKeyPrefix(r2desc.StartKey), nil, hlc.Timestamp{WallTime: 1000}, uuid.Nil)

// Assert that r1's summary was not influenced by the r2 range-local key we
// set above.
summary := collectReadSummaryFromTimestampCache(tc, &r1desc)
require.Equal(t, baseTS, summary.Global.LowWater)
require.Equal(t, baseTS, summary.Local.LowWater)
}
11 changes: 11 additions & 0 deletions pkg/roachpb/data.go
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,12 @@ import (
"go.etcd.io/etcd/raft/v3/raftpb"
)

const (
localPrefixByte = '\x01'
// LocalMaxByte is the end of the local key range.
LocalMaxByte = '\x02'
)

var (
// RKeyMin is a minimum key value which sorts before all other keys.
RKeyMin = RKey("")
Expand All @@ -55,6 +61,11 @@ var (
// KeyMax is a maximum key value which sorts after all other keys.
KeyMax = Key(RKeyMax)

// LocalPrefix is the prefix for all local keys.
LocalPrefix = Key{localPrefixByte}
// LocalMax is the end of the local key range. It is itself a global key.
LocalMax = Key{LocalMaxByte}

// PrettyPrintKey prints a key in human readable format. It's
// implemented in package git.com/cockroachdb/cockroach/keys to avoid
// package circle import.
Expand Down
20 changes: 20 additions & 0 deletions pkg/roachpb/metadata.go
Original file line number Diff line number Diff line change
Expand Up @@ -199,6 +199,26 @@ func (r *RangeDescriptor) RSpan() RSpan {
return RSpan{Key: r.StartKey, EndKey: r.EndKey}
}

// KeySpan returns the keys covered by this range. Local keys are not included.
//
// TODO(andrei): Consider if this logic should be lifted to
// RangeDescriptor.RSpan(). Or better yet, see if we can changes things such
// that the first range starts at LocalMax instead at starting at an empty key.
func (r *RangeDescriptor) KeySpan() RSpan {
start := r.StartKey
if r.StartKey.Equal(RKeyMin) {
// The first range in the keyspace is declared to start at KeyMin (the
// lowest possible key). That is a lie, however, since the local key space
// ([LocalMin,LocalMax)) doesn't belong to this range; it doesn't belong to
// any range in particular.
start = RKey(LocalMax)
}
return RSpan{
Key: start,
EndKey: r.EndKey,
}
}

// ContainsKey returns whether this RangeDescriptor contains the specified key.
func (r *RangeDescriptor) ContainsKey(key RKey) bool {
return r.RSpan().ContainsKey(key)
Expand Down

0 comments on commit 2a2e749

Please sign in to comment.