Skip to content

Commit

Permalink
*: fix tests when DisallowSeparatedIntents=false
Browse files Browse the repository at this point in the history
- storage: fix all storage tests
- storage: prevent intentIter from stepping out of the lock table.
  It was throwing an error on finding /Local/Store/clusterVersion
  because callers were iterating forward without an upper bound.
- storage: fix engine iter bounds bug in pebbleBatch
- kv: fix TestRangeStatsComputation to account for TxnDidNotUpdateMeta
- keys: pretty print lock table keys
- rditer: fix lock table ranges to be non-overlapping
- kvserver: fix TestStoreRangeMergeRaftSnapshot to handle separated
  locks. The SST counts are different when we have separated locks.

We are leaving DisallowSeparatedIntents=true for now. This
is just preparatory work before we can test with both settings.

Release note: None
  • Loading branch information
sumeerbhola committed Dec 10, 2020
1 parent f298478 commit ce4b23d
Show file tree
Hide file tree
Showing 27 changed files with 633 additions and 75 deletions.
7 changes: 7 additions & 0 deletions pkg/keys/constants.go
Original file line number Diff line number Diff line change
Expand Up @@ -161,6 +161,13 @@ var (
// other strengths, or range locks.
LocalRangeLockTablePrefix = roachpb.Key(makeKey(localPrefix, roachpb.RKey("l")))
LockTableSingleKeyInfix = []byte("k")
// LockTableSingleKeyStart is the inclusive start key of the key range
// containing single key locks.
LockTableSingleKeyStart = roachpb.Key(makeKey(LocalRangeLockTablePrefix, LockTableSingleKeyInfix))
// LockTableSingleKeyEnd is the exclusive end key of the key range
// containing single key locks.
LockTableSingleKeyEnd = roachpb.Key(
makeKey(LocalRangeLockTablePrefix, roachpb.Key(LockTableSingleKeyInfix).PrefixEnd()))

// 5. Store local keys
//
Expand Down
24 changes: 24 additions & 0 deletions pkg/keys/printer.go
Original file line number Diff line number Diff line change
Expand Up @@ -77,6 +77,8 @@ var (
ppFunc: localRangeIDKeyPrint, PSFunc: localRangeIDKeyParse},
{Name: "/Range", prefix: LocalRangePrefix, ppFunc: localRangeKeyPrint,
PSFunc: parseUnsupported},
{Name: "/Lock", prefix: LocalRangeLockTablePrefix, ppFunc: localRangeLockTablePrint,
PSFunc: parseUnsupported},
}},
{Name: "/Meta1", start: Meta1Prefix, end: Meta1KeyMax, Entries: []DictEntry{
{Name: "", prefix: Meta1Prefix, ppFunc: print,
Expand Down Expand Up @@ -523,6 +525,27 @@ func localRangeKeyPrint(valDirs []encoding.Direction, key roachpb.Key) string {
return buf.String()
}

// lockTablePrintLockedKey is initialized to prettyPrintInternal in init() to break an
// initialization loop.
var lockTablePrintLockedKey func(valDirs []encoding.Direction, key roachpb.Key, quoteRawKeys bool) string

func localRangeLockTablePrint(valDirs []encoding.Direction, key roachpb.Key) string {
var buf bytes.Buffer
if !bytes.HasPrefix(key, LockTableSingleKeyInfix) {
fmt.Fprintf(&buf, "/\"%x\"", key)
return buf.String()
}
buf.WriteString("/Intent")
key = key[len(LockTableSingleKeyInfix):]
b, lockedKey, err := encoding.DecodeBytesAscending(key, nil)
if err != nil || len(b) != 0 {
fmt.Fprintf(&buf, "/\"%x\"", key)
return buf.String()
}
buf.WriteString(lockTablePrintLockedKey(valDirs, lockedKey, true))
return buf.String()
}

// ErrUglifyUnsupported is returned when UglyPrint doesn't know how to process a
// key.
type ErrUglifyUnsupported struct {
Expand Down Expand Up @@ -675,6 +698,7 @@ func PrettyPrint(valDirs []encoding.Direction, key roachpb.Key) string {
func init() {
roachpb.PrettyPrintKey = PrettyPrint
roachpb.PrettyPrintRange = PrettyPrintRange
lockTablePrintLockedKey = prettyPrintInternal
}

// PrettyPrintRange pretty prints a compact representation of a key range. The
Expand Down
9 changes: 9 additions & 0 deletions pkg/keys/printer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,11 @@ import (
"github.com/cockroachdb/errors"
)

func lockTableKey(key roachpb.Key) roachpb.Key {
k, _ := keys.LockTableSingleKey(key, nil)
return k
}

func TestPrettyPrint(t *testing.T) {
tenSysCodec := keys.SystemSQLCodec
ten5Codec := keys.MakeSQLCodec(roachpb.MakeTenantID(5))
Expand Down Expand Up @@ -81,12 +86,16 @@ func TestPrettyPrint(t *testing.T) {
{keys.RangeDescriptorKey(roachpb.RKey(tenSysCodec.TablePrefix(42))), `/Local/Range/Table/42/RangeDescriptor`, revertSupportUnknown},
{keys.TransactionKey(tenSysCodec.TablePrefix(42), txnID), fmt.Sprintf(`/Local/Range/Table/42/Transaction/%q`, txnID), revertSupportUnknown},
{keys.QueueLastProcessedKey(roachpb.RKey(tenSysCodec.TablePrefix(42)), "foo"), `/Local/Range/Table/42/QueueLastProcessed/"foo"`, revertSupportUnknown},
{lockTableKey(keys.RangeDescriptorKey(roachpb.RKey(tenSysCodec.TablePrefix(42)))), `/Local/Lock/Intent/Local/Range/Table/42/RangeDescriptor`, revertSupportUnknown},
{lockTableKey(tenSysCodec.TablePrefix(111)), "/Local/Lock/Intent/Table/111", revertSupportUnknown},

{keys.MakeRangeKeyPrefix(roachpb.RKey(ten5Codec.TenantPrefix())), `/Local/Range/Tenant/5`, revertSupportUnknown},
{keys.MakeRangeKeyPrefix(roachpb.RKey(ten5Codec.TablePrefix(42))), `/Local/Range/Tenant/5/Table/42`, revertSupportUnknown},
{keys.RangeDescriptorKey(roachpb.RKey(ten5Codec.TablePrefix(42))), `/Local/Range/Tenant/5/Table/42/RangeDescriptor`, revertSupportUnknown},
{keys.TransactionKey(ten5Codec.TablePrefix(42), txnID), fmt.Sprintf(`/Local/Range/Tenant/5/Table/42/Transaction/%q`, txnID), revertSupportUnknown},
{keys.QueueLastProcessedKey(roachpb.RKey(ten5Codec.TablePrefix(42)), "foo"), `/Local/Range/Tenant/5/Table/42/QueueLastProcessed/"foo"`, revertSupportUnknown},
{lockTableKey(keys.RangeDescriptorKey(roachpb.RKey(ten5Codec.TablePrefix(42)))), `/Local/Lock/Intent/Local/Range/Tenant/5/Table/42/RangeDescriptor`, revertSupportUnknown},
{lockTableKey(ten5Codec.TablePrefix(111)), "/Local/Lock/Intent/Tenant/5/Table/111", revertSupportUnknown},

{keys.LocalMax, `/Meta1/""`, revertSupportUnknown}, // LocalMax == Meta1Prefix

Expand Down
80 changes: 51 additions & 29 deletions pkg/kv/kvserver/client_merge_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -202,7 +202,7 @@ func TestStoreRangeMergeMetadataCleanup(t *testing.T) {
}
delete(postKeys, tombstoneKey)

// Keep only the subsumed range's local keys.
// Keep only the subsumed range's local range-ID keys.
localRangeKeyPrefix := string(keys.MakeRangeIDPrefix(rhsDesc.RangeID))
for k := range postKeys {
if !strings.HasPrefix(k, localRangeKeyPrefix) {
Expand Down Expand Up @@ -3085,39 +3085,56 @@ func TestStoreRangeMergeRaftSnapshot(t *testing.T) {
// TODO(sumeer): fix this test (and others in this file) when
// DisallowSeparatedIntents=false

// The seven SSTs we are expecting to ingest are in the following order:
// 1. Replicated range-id local keys of the range in the snapshot.
// 2. Range-local keys of the range in the snapshot.
// 3. User keys of the range in the snapshot.
// 4. Unreplicated range-id local keys of the range in the snapshot.
// 5. SST to clear range-id local keys of the subsumed replica with
// RangeID 3.
// 6. SST to clear range-id local keys of the subsumed replica with
// RangeID 4.
// 7. SST to clear the user keys of the subsumed replicas.
// The seven to nine SSTs we are expecting to ingest are in the following order:
// - Replicated range-id local keys of the range in the snapshot.
// - Range-local keys of the range in the snapshot.
// - Optionally, two SSTs for the lock table keys of the range in the
// snapshot
// - User keys of the range in the snapshot.
// - Unreplicated range-id local keys of the range in the snapshot.
// - SST to clear range-id local keys of the subsumed replica with
// RangeID 3.
// - SST to clear range-id local keys of the subsumed replica with
// RangeID 4.
// - SST to clear the user keys of the subsumed replicas.
//
// NOTE: There are no range-local keys in [d, /Max) in the store we're
// sending a snapshot to, so we aren't expecting an SST to clear those
// keys.
if len(sstNames) != 7 {
return errors.Errorf("expected to ingest 7 SSTs, got %d SSTs", len(sstNames))
}

// Only try to predict SSTs 3 and 5-7. SSTs 1, 2 and 4 are excluded in
// the test since the state of the Raft log can be non-deterministic
// with extra entries being appended to the sender's log after the
// snapshot has already been sent.
// NOTE: There are no range-local keys or lock table keys, in [d, /Max) in
// the store we're sending a snapshot to, so we aren't expecting SSTs to
// clear those keys.
expectedSSTCount := 7
indexAdjustment := 0
if !storage.DisallowSeparatedIntents {
expectedSSTCount += 2
indexAdjustment = 2
}
if len(sstNames) != expectedSSTCount {
return errors.Errorf("expected to ingest %d SSTs, got %d SSTs",
expectedSSTCount, len(sstNames))
}

// Only try to predict SSTs for:
// - The user keys in the snapshot
// - Clearing rhe range-id local keys of the subsumed replicas.
// - Clearing the user keys of the subsumed replicas.
// The snapshot SSTs that are excluded from this checking are the
// replicated range-id, range-local keys, lock table keys in the snapshot,
// and the unreplicated range-id local keys in the snapshot. The latter is
// excluded since the state of the Raft log can be non-deterministic with
// extra entries being appended to the sender's log after the snapshot has
// already been sent.
var sstNamesSubset []string
sstNamesSubset = append(sstNamesSubset, sstNames[2])
sstNamesSubset = append(sstNamesSubset, sstNames[4:]...)
// The SST with the user keys in the snapshot.
sstNamesSubset = append(sstNamesSubset, sstNames[2+indexAdjustment])
// Remaining ones from the predict list above.
sstNamesSubset = append(sstNamesSubset, sstNames[4+indexAdjustment:]...)

// Construct the expected SSTs and ensure that they are byte-by-byte
// equal. This verification ensures that the SSTs have the same
// tombstones and range deletion tombstones.
var expectedSSTs [][]byte

// Construct SST #1 through #3 as numbered above, but only ultimately
// keep the 3rd one.
// Construct SSTs for the the first 4 bullets as numbered above, but only
// ultimately keep the last one.
keyRanges := rditer.MakeReplicatedKeyRanges(inSnap.State.Desc)
it := rditer.NewReplicaEngineDataIterator(inSnap.State.Desc, sendingEng, true /* replicatedOnly */)
defer it.Close()
Expand Down Expand Up @@ -3150,9 +3167,14 @@ func TestStoreRangeMergeRaftSnapshot(t *testing.T) {
}
}
}
expectedSSTs = expectedSSTs[2:]
if len(expectedSSTs) != 3+indexAdjustment {
return errors.Errorf("len of expectedSSTs should expected to be %d, but got %d",
3+indexAdjustment, len(expectedSSTs))
}
// Keep the last one which contains the user keys.
expectedSSTs = expectedSSTs[len(expectedSSTs)-1:]

// Construct SSTs #5 and #6: range-id local keys of subsumed replicas
// Construct SSTs for the range-id local keys of the subsumed replicas.
// with RangeIDs 3 and 4.
for _, rangeID := range []roachpb.RangeID{roachpb.RangeID(3), roachpb.RangeID(4)} {
sstFile := &storage.MemFile{}
Expand All @@ -3174,7 +3196,7 @@ func TestStoreRangeMergeRaftSnapshot(t *testing.T) {
expectedSSTs = append(expectedSSTs, sstFile.Data())
}

// Construct SST #7: user key range of subsumed replicas.
// Construct an SST for the user key range of the subsumed replicas.
sstFile := &storage.MemFile{}
sst := storage.MakeIngestionSSTWriter(sstFile)
defer sst.Close()
Expand Down
10 changes: 9 additions & 1 deletion pkg/kv/kvserver/rditer/replica_data_iter.go
Original file line number Diff line number Diff line change
Expand Up @@ -173,7 +173,15 @@ func makeRangeLockTableKeyRanges(d *roachpb.RangeDescriptor) [2]KeyRange {
// is a range local key that can have a replicated lock acquired on it.
startRangeLocal, _ := keys.LockTableSingleKey(keys.MakeRangeKeyPrefix(d.StartKey), nil)
endRangeLocal, _ := keys.LockTableSingleKey(keys.MakeRangeKeyPrefix(d.EndKey), nil)
startGlobal, _ := keys.LockTableSingleKey(roachpb.Key(d.StartKey), nil)
// The first range in the global keyspace can start earlier than LocalMax,
// at RKeyMin, but the actual data starts at LocalMax. We need to make this
// adjustment here to prevent [startRangeLocal, endRangeLocal) and
// [startGlobal, endGlobal) from overlapping.
globalStartKey := d.StartKey.AsRawKey()
if d.StartKey.Equal(roachpb.RKeyMin) {
globalStartKey = keys.LocalMax
}
startGlobal, _ := keys.LockTableSingleKey(globalStartKey, nil)
endGlobal, _ := keys.LockTableSingleKey(roachpb.Key(d.EndKey), nil)
return [2]KeyRange{
{
Expand Down
23 changes: 23 additions & 0 deletions pkg/kv/kvserver/rditer/replica_data_iter_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -317,3 +317,26 @@ func TestReplicaDataIterator(t *testing.T) {
})
}
}

func checkOrdering(t *testing.T, ranges []KeyRange) {
for i := 1; i < len(ranges); i++ {
if ranges[i].Start.Less(ranges[i-1].End) {
t.Fatalf("ranges need to be ordered and non-overlapping, but %s > %s",
ranges[i-1].End, ranges[i].Start)
}
}
}

func TestReplicaKeyRanges(t *testing.T) {
defer leaktest.AfterTest(t)()

desc := roachpb.RangeDescriptor{
RangeID: 1,
StartKey: roachpb.RKeyMin,
EndKey: roachpb.RKeyMax,
}
checkOrdering(t, MakeAllKeyRanges(&desc))
checkOrdering(t, MakeReplicatedKeyRanges(&desc))
checkOrdering(t, MakeReplicatedKeyRangesExceptLockTable(&desc))
checkOrdering(t, MakeReplicatedKeyRangesExceptRangeID(&desc))
}
23 changes: 13 additions & 10 deletions pkg/kv/kvserver/replica_command.go
Original file line number Diff line number Diff line change
Expand Up @@ -1935,12 +1935,14 @@ func recordRangeEventsInLog(
// the snapshot was a success.
//
// `receiveSnapshot` takes the key-value pairs sent and incrementally creates
// three SSTs from them for direct ingestion: one for the replicated range-ID
// local keys, one for the range local keys, and one for the user keys. The
// reason it creates three separate SSTs is to prevent overlaps with the
// memtable and existing SSTs in RocksDB. Each of the SSTs also has a range
// deletion tombstone to delete the existing data in the range.
// three to five SSTs from them for direct ingestion: one for the replicated
// range-ID local keys, one for the range local keys, optionally two for the
// lock table keys, and one for the user keys. The reason it creates these as
// separate SSTs is to prevent overlaps with the memtable and existing SSTs in
// RocksDB. Each of the SSTs also has a range deletion tombstone to delete the
// existing data in the range.
//

// Applying the snapshot: After the recipient has received the message
// indicating it has all the data, it hands it all to
// `(Store).processRaftSnapshotRequest` to be applied. First, this re-checks
Expand All @@ -1953,11 +1955,12 @@ func recordRangeEventsInLog(
// process, several other SSTs may be created for direct ingestion. An SST for
// the unreplicated range-ID local keys is created for the Raft entries, hard
// state, and truncated state. An SST is created for deleting each subsumed
// replica's range-ID local keys and at most two SSTs are created for deleting
// the user keys and range local keys of all subsumed replicas. All in all, a
// maximum of 6 + SR SSTs will be created for direct ingestion where SR is the
// number of subsumed replicas. In the case where there are no subsumed
// replicas, 4 SSTs will be created.
// replica's range-ID local keys and at most four SSTs are created for
// deleting the user keys, range local keys, and lock table keys (up to 2
// ssts) of all subsumed replicas. All in all, a maximum of 6 + 4*SR SSTs will
// be created for direct ingestion where SR is the number of subsumed
// replicas. In the case where there are no subsumed replicas, 4 to 6 SSTs
// will be created.
//
// [1]: The largest class of rejections here is if the store contains a replica
// that overlaps the snapshot but has a different id (we maintain an invariant
Expand Down
8 changes: 4 additions & 4 deletions pkg/kv/kvserver/replica_raftstorage.go
Original file line number Diff line number Diff line change
Expand Up @@ -1095,10 +1095,10 @@ func (r *Replica) clearSubsumedReplicaDiskData(
}
}

// We might have to create SSTs for the range local keys and user keys
// depending on if the subsumed replicas are not fully contained by the
// replica in our snapshot. The following is an example to this case
// happening.
// We might have to create SSTs for the range local keys, lock table keys,
// and user keys depending on if the subsumed replicas are not fully
// contained by the replica in our snapshot. The following is an example to
// this case happening.
//
// a b c d
// |---1---|-------2-------| S1
Expand Down
5 changes: 5 additions & 0 deletions pkg/kv/kvserver/replica_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -6242,6 +6242,11 @@ func TestRangeStatsComputation(t *testing.T) {
ValCount: 2,
IntentCount: 1,
})
if !storage.DisallowSeparatedIntents {
// Account for TxnDidNotUpdateMeta
expMS.LiveBytes += 2
expMS.ValBytes += 2
}
if err := verifyRangeStats(tc.engine, tc.repl.RangeID, expMS); err != nil {
t.Fatal(err)
}
Expand Down
6 changes: 5 additions & 1 deletion pkg/storage/engine.go
Original file line number Diff line number Diff line change
Expand Up @@ -237,7 +237,11 @@ type IterOptions struct {
// [start, end] time range. If you must guarantee that you never see a key
// outside of the time bounds, perform your own filtering.
//
// These fields are only relevant for MVCCIterators.
// These fields are only relevant for MVCCIterators. Additionally, an
// MVCCIterator with timestamp hints will not see separated intents, and may
// not see some interleaved intents. Currently, the only way to correctly
// use such an iterator is to use it in concert with an iterator without
// timestamp hints, as done by MVCCIncrementalIterator.
MinTimestampHint, MaxTimestampHint hlc.Timestamp
}

Expand Down
18 changes: 16 additions & 2 deletions pkg/storage/intent_interleaving_iter.go
Original file line number Diff line number Diff line change
Expand Up @@ -103,9 +103,17 @@ func newIntentInterleavingIterator(reader Reader, opts IterOptions) MVCCIterator
var intentKeyBuf []byte
if opts.LowerBound != nil {
intentOpts.LowerBound, intentKeyBuf = keys.LockTableSingleKey(opts.LowerBound, nil)
} else {
// Sometimes callers iterate backwards without having a lower bound.
// Make sure we don't step outside the lock table key space.
intentOpts.LowerBound = keys.LockTableSingleKeyStart
}
if opts.UpperBound != nil {
intentOpts.UpperBound, _ = keys.LockTableSingleKey(opts.UpperBound, nil)
} else {
// Sometimes callers iterate forwards without having an upper bound.
// Make sure we don't step outside the lock table key space.
intentOpts.UpperBound = keys.LockTableSingleKeyEnd
}
// Note that we can reuse intentKeyBuf after NewEngineIterator returns.
intentIter := reader.NewEngineIterator(intentOpts)
Expand Down Expand Up @@ -682,8 +690,14 @@ func (i *intentInterleavingIter) CheckForKeyCollisions(
func (i *intentInterleavingIter) SetUpperBound(key roachpb.Key) {
i.iter.SetUpperBound(key)
var intentUpperBound roachpb.Key
intentUpperBound, i.intentKeyBuf = keys.LockTableSingleKey(key, i.intentKeyBuf)
// Note that we can reuse intentKeyBuf after SetUpperBound returns.
if key != nil {
intentUpperBound, i.intentKeyBuf = keys.LockTableSingleKey(key, i.intentKeyBuf)
// Note that we can reuse intentKeyBuf after SetUpperBound returns.
} else {
// Sometimes callers iterate forwards without having an upper bound.
// Make sure we don't step outside the lock table key space.
intentUpperBound = keys.LockTableSingleKeyEnd
}
i.intentIter.SetUpperBound(intentUpperBound)
i.hasUpperBound = key != nil
}
Expand Down
8 changes: 6 additions & 2 deletions pkg/storage/intent_reader_writer.go
Original file line number Diff line number Diff line change
Expand Up @@ -219,8 +219,12 @@ func (imr intentInterleavingReader) MVCCGetProto(
func (imr intentInterleavingReader) NewMVCCIterator(
iterKind MVCCIterKind, opts IterOptions,
) MVCCIterator {
if iterKind != MVCCKeyAndIntentsIterKind {
panic("intentInterleavingReader should only be used with MVCCKeyAndIntentsIterKing")
if (!opts.MinTimestampHint.IsEmpty() || !opts.MaxTimestampHint.IsEmpty()) &&
iterKind == MVCCKeyAndIntentsIterKind {
panic("cannot ask for interleaved intents when specifying timestamp hints")
}
if iterKind == MVCCKeyIterKind {
return imr.wrappableReader.NewMVCCIterator(MVCCKeyIterKind, opts)
}
return newIntentInterleavingIterator(imr.wrappableReader, opts)
}
6 changes: 6 additions & 0 deletions pkg/storage/mvcc_history_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -92,6 +92,12 @@ func TestMVCCHistories(t *testing.T) {
span := roachpb.Span{Key: key, EndKey: key.PrefixEnd()}

datadriven.Walk(t, "testdata/mvcc_histories", func(t *testing.T, path string) {
if strings.Contains(path, "_disallow_separated") && !DisallowSeparatedIntents {
return
}
if strings.Contains(path, "_allow_separated") && DisallowSeparatedIntents {
return
}
// We start from a clean slate in every test file.
engine := engineImpl.create()
defer engine.Close()
Expand Down
Loading

0 comments on commit ce4b23d

Please sign in to comment.