diff --git a/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp_test.go b/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp_test.go index 215082997776..f7e75cfec449 100644 --- a/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp_test.go @@ -228,7 +228,7 @@ func TestQueryResolvedTimestampErrors(t *testing.T) { lockTableKey := storage.LockTableKey{ Key: roachpb.Key("a"), - Strength: lock.Exclusive, + Strength: lock.Intent, TxnUUID: txnUUID, } engineKey, buf := lockTableKey.ToEngineKey(nil) diff --git a/pkg/kv/kvserver/rditer/replica_data_iter_test.go b/pkg/kv/kvserver/rditer/replica_data_iter_test.go index 152fe709c47a..1574664186bf 100644 --- a/pkg/kv/kvserver/rditer/replica_data_iter_test.go +++ b/pkg/kv/kvserver/rditer/replica_data_iter_test.go @@ -110,11 +110,11 @@ func createRangeData( locks := []storage.LockTableKey{ { Key: keys.RangeDescriptorKey(desc.StartKey), // mark [1] above as intent - Strength: lock.Exclusive, + Strength: lock.Intent, TxnUUID: testTxnID, }, { Key: desc.StartKey.AsRawKey(), // mark [2] above as intent - Strength: lock.Exclusive, + Strength: lock.Intent, TxnUUID: testTxnID, }, } diff --git a/pkg/roachprod/prometheus/prometheus.go b/pkg/roachprod/prometheus/prometheus.go index 22a59af72a24..73ad3ce60e79 100644 --- a/pkg/roachprod/prometheus/prometheus.go +++ b/pkg/roachprod/prometheus/prometheus.go @@ -341,12 +341,13 @@ sudo systemd-run --unit prometheus --same-dir \ l.Stderr, cfg.PrometheusNode, "install grafana", fmt.Sprintf(` sudo apt-get install -qqy apt-transport-https && -sudo apt-get install -qqy software-properties-common wget && +sudo apt-get install -qqy software-properties-common && sudo apt-get install -y adduser libfontconfig1 && -wget https://dl.grafana.com/enterprise/release/grafana-enterprise_9.2.3_%s.deb -O grafana-enterprise_9.2.3_%s.deb && -sudo dpkg -i grafana-enterprise_9.2.3_%s.deb && +echo "Downloading https://dl.grafana.com/enterprise/release/grafana-enterprise_9.2.3_%[1]s.deb" && +curl https://dl.grafana.com/enterprise/release/grafana-enterprise_9.2.3_%[1]s.deb -sS -o grafana-enterprise_9.2.3_%[1]s.deb && +sudo dpkg -i grafana-enterprise_9.2.3_%[1]s.deb && sudo mkdir -p /var/lib/grafana/dashboards`, - binArch, binArch, binArch)); err != nil { + binArch)); err != nil { return nil, err } diff --git a/pkg/storage/engine_key.go b/pkg/storage/engine_key.go index c0dc3de8503f..a0adebba8da5 100644 --- a/pkg/storage/engine_key.go +++ b/pkg/storage/engine_key.go @@ -173,9 +173,9 @@ func (k EngineKey) ToLockTableKey() (LockTableKey, error) { key := LockTableKey{Key: lockedKey} switch len(k.Version) { case engineKeyVersionLockTableLen: - key.Strength = lock.Strength(k.Version[0]) - if key.Strength < lock.None || key.Strength > lock.Exclusive { - return LockTableKey{}, errors.Errorf("unknown strength %d", key.Strength) + key.Strength, err = getReplicatedLockStrengthForByte(k.Version[0]) + if err != nil { + return LockTableKey{}, err } key.TxnUUID = *(*uuid.UUID)(k.Version[1:]) default: @@ -255,11 +255,73 @@ type LockTableKey struct { TxnUUID uuid.UUID } +// replicatedLockStrengthToByte is a mapping between lock.Strength and the +// strength byte persisted in a lock table key's encoding. See +// LockTableKey.ToEngineKey(). +var replicatedLockStrengthToByte = [...]byte{ + lock.Intent: 3, +} + +// byteToReplicatedLockStrength is a mapping between the strength byte persisted +// in a lock table key's encoding and the lock.Strength of the lock it +// corresponds to. Also see EngineKey.ToLockTableKey(). +var byteToReplicatedLockStrength = func() (arr []lock.Strength) { + maxByte := byte(0) + for _, b := range replicatedLockStrengthToByte { + if b > maxByte { + maxByte = b + } + } + arr = make([]lock.Strength, maxByte+1) + for str, b := range replicatedLockStrengthToByte { + if b != 0 { + arr[b] = lock.Strength(str) + } + } + return arr +}() + +// getByteForReplicatedLockStrength returns a strength byte, suitable for use in +// a lock's key encoding, given its lock strength. +func getByteForReplicatedLockStrength(str lock.Strength) byte { + if str < 0 || int(str) >= len(replicatedLockStrengthToByte) { + panic(errors.AssertionFailedf("unknown lock strength %s", str)) + } + b := replicatedLockStrengthToByte[str] + if b == 0 { + panic(errors.AssertionFailedf("unexpected empty byte")) + } + return b +} + +// getReplicatedLockStrengthForByte returns a replicated lock's strength given +// the strength byte from its key encoding. +func getReplicatedLockStrengthForByte(b byte) (lock.Strength, error) { + if int(b) >= len(byteToReplicatedLockStrength) { // byte cannot be < 0 + return lock.None, errors.AssertionFailedf("unsupported byte %d", b) + } + str := byteToReplicatedLockStrength[b] + if str == 0 { + return lock.None, errors.AssertionFailedf("unknown lock strength %s", str) + } + return str, nil +} + +// mustGetReplicatedLockStrengthForByte is like mustGetReplicatedLockStrength +// except it panics if there is an error. +func mustGetReplicatedLockStrengthForByte(b byte) lock.Strength { + str, err := getReplicatedLockStrengthForByte(b) + if err != nil { + panic(err) + } + return str +} + // ToEngineKey converts a lock table key to an EngineKey. buf is used as // scratch-space to avoid allocations -- its contents will be overwritten and // not appended to. func (lk LockTableKey) ToEngineKey(buf []byte) (EngineKey, []byte) { - if lk.Strength != lock.Exclusive { + if lk.Strength != lock.Intent { panic("unsupported lock strength") } // The first term in estimatedLen is for LockTableSingleKey. @@ -277,7 +339,7 @@ func (lk LockTableKey) ToEngineKey(buf []byte) (EngineKey, []byte) { // estimatedLen was an underestimate. k.Version = make([]byte, engineKeyVersionLockTableLen) } - k.Version[0] = byte(lk.Strength) + k.Version[0] = getByteForReplicatedLockStrength(lk.Strength) copy(k.Version[1:], lk.TxnUUID[:]) return k, buf } diff --git a/pkg/storage/engine_key_test.go b/pkg/storage/engine_key_test.go index e2abe9b79751..8dc9ce068581 100644 --- a/pkg/storage/engine_key_test.go +++ b/pkg/storage/engine_key_test.go @@ -36,12 +36,12 @@ func TestLockTableKeyEncodeDecode(t *testing.T) { testCases := []struct { key LockTableKey }{ - {key: LockTableKey{Key: roachpb.Key("foo"), Strength: lock.Exclusive, TxnUUID: uuid1}}, - {key: LockTableKey{Key: roachpb.Key("a"), Strength: lock.Exclusive, TxnUUID: uuid2}}, + {key: LockTableKey{Key: roachpb.Key("foo"), Strength: lock.Intent, TxnUUID: uuid1}}, + {key: LockTableKey{Key: roachpb.Key("a"), Strength: lock.Intent, TxnUUID: uuid2}}, // Causes a doubly-local range local key. {key: LockTableKey{ Key: keys.RangeDescriptorKey(roachpb.RKey("baz")), - Strength: lock.Exclusive, + Strength: lock.Intent, TxnUUID: uuid1}}, } buf := make([]byte, 100) @@ -83,6 +83,48 @@ func TestLockTableKeyEncodeDecode(t *testing.T) { } } +// TestLockTableKeyMixedVersionV23_123_2 ensures a lock table key written by a +// <= v23.1 node can be decoded by a 23.2 node and a lock table key written by +// a 23.2 node cna be decoded by a 23.1 node. +func TestLockTableKeyMixedVersionV23_1V23_2(t *testing.T) { + defer leaktest.AfterTest(t)() + + uuid := uuid.MakeV4() + t.Run("decode_v23_1", func(t *testing.T) { + key := LockTableKey{ + Key: roachpb.Key("foo"), + Strength: lock.Intent, // strength corresponding to an intent written by a 23.2 node + TxnUUID: uuid, + } + + eKey, _ := key.ToEngineKey(nil) + require.True(t, eKey.IsLockTableKey()) + eKeyDecoded, ok := DecodeEngineKey(eKey.Encode()) + require.True(t, ok) + require.True(t, eKeyDecoded.IsLockTableKey()) + keyDecoded, err := eKeyDecoded.ToLockTableKey() + require.NoError(t, err) + // v23.1 nodes expect intents to have 3 as the strength byte. + require.Equal(t, byte(3), getByteForReplicatedLockStrength(keyDecoded.Strength)) + }) + + t.Run("encode_v23_1", func(t *testing.T) { + key := LockTableKey{ + Key: roachpb.Key("foo"), + Strength: mustGetReplicatedLockStrengthForByte(3), // strength byte used by v23.1 nodes + TxnUUID: uuid, + } + eKey, _ := key.ToEngineKey(nil) + require.True(t, eKey.IsLockTableKey()) + eKeyDecoded, ok := DecodeEngineKey(eKey.Encode()) + require.True(t, ok) + require.True(t, eKeyDecoded.IsLockTableKey()) + keyDecoded, err := eKeyDecoded.ToLockTableKey() + require.NoError(t, err) + require.Equal(t, lock.Intent, keyDecoded.Strength) + }) +} + func TestMVCCAndEngineKeyEncodeDecode(t *testing.T) { defer leaktest.AfterTest(t)() testCases := []struct { @@ -176,14 +218,14 @@ func TestEngineKeyValidate(t *testing.T) { { key: LockTableKey{ Key: roachpb.Key("foo"), - Strength: lock.Exclusive, + Strength: lock.Intent, TxnUUID: uuid1, }, }, { key: LockTableKey{ Key: keys.RangeDescriptorKey(roachpb.RKey("bar")), - Strength: lock.Exclusive, + Strength: lock.Intent, TxnUUID: uuid1, }, }, @@ -268,7 +310,7 @@ func randomMVCCKey(r *rand.Rand) MVCCKey { func randomLockTableKey(r *rand.Rand) LockTableKey { k := LockTableKey{ Key: randutil.RandBytes(r, randutil.RandIntInRange(r, 1, 12)), - Strength: lock.Exclusive, + Strength: lock.Intent, } var txnID uuid.UUID txnID.DeterministicV4(r.Uint64(), math.MaxUint64) diff --git a/pkg/storage/intent_interleaving_iter.go b/pkg/storage/intent_interleaving_iter.go index 66e4b1a882c2..89bee4974f60 100644 --- a/pkg/storage/intent_interleaving_iter.go +++ b/pkg/storage/intent_interleaving_iter.go @@ -609,7 +609,7 @@ func (i *intentInterleavingIter) SeekIntentGE(key roachpb.Key, txnUUID uuid.UUID var engineKey EngineKey engineKey, i.intentKeyBuf = LockTableKey{ Key: key, - Strength: lock.Exclusive, + Strength: lock.Intent, TxnUUID: txnUUID, }.ToEngineKey(i.intentKeyBuf) var limitKey roachpb.Key diff --git a/pkg/storage/intent_interleaving_iter_test.go b/pkg/storage/intent_interleaving_iter_test.go index 0f22e2fd141f..f6a9f1142a4a 100644 --- a/pkg/storage/intent_interleaving_iter_test.go +++ b/pkg/storage/intent_interleaving_iter_test.go @@ -294,7 +294,7 @@ func TestIntentInterleavingIter(t *testing.T) { return err.Error() } } else { - ltKey := LockTableKey{Key: key, Strength: lock.Exclusive, TxnUUID: txnUUID} + ltKey := LockTableKey{Key: key, Strength: lock.Intent, TxnUUID: txnUUID} eKey, _ := ltKey.ToEngineKey(nil) if err := batch.PutEngineKey(eKey, val); err != nil { return err.Error() @@ -561,7 +561,7 @@ func generateRandomData( } val, err := protoutil.Marshal(&meta) require.NoError(t, err) - ltKey := LockTableKey{Key: key, Strength: lock.Exclusive, TxnUUID: txnUUID} + ltKey := LockTableKey{Key: key, Strength: lock.Intent, TxnUUID: txnUUID} lkv = append(lkv, lockKeyValue{ key: ltKey, val: val, liveIntent: hasIntent && i == 0}) mvcckv = append(mvcckv, MVCCKeyValue{ @@ -823,7 +823,7 @@ func writeBenchData( require.NoError(b, err) if separated { eKey, _ := - LockTableKey{Key: key, Strength: lock.Exclusive, TxnUUID: txnUUID}.ToEngineKey(nil) + LockTableKey{Key: key, Strength: lock.Intent, TxnUUID: txnUUID}.ToEngineKey(nil) require.NoError(b, batch.PutEngineKey(eKey, val)) } else { require.NoError(b, batch.PutUnversioned(key, val)) diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index d9800ed37f2c..dd5012bf53f8 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -1476,7 +1476,7 @@ func (b *putBuffer) putIntentMeta( return 0, 0, errors.AssertionFailedf( "meta.Timestamp != meta.Txn.WriteTimestamp: %s != %s", meta.Timestamp, meta.Txn.WriteTimestamp) } - lockTableKey := b.lockTableKey(key.Key, lock.Exclusive, meta.Txn.ID) + lockTableKey := b.lockTableKey(key.Key, lock.Intent, meta.Txn.ID) if alreadyExists { // Absence represents false. meta.TxnDidNotUpdateMeta = nil @@ -1505,7 +1505,7 @@ func (b *putBuffer) putIntentMeta( func (b *putBuffer) clearIntentMeta( writer Writer, key MVCCKey, txnDidNotUpdateMeta bool, txnUUID uuid.UUID, opts ClearOptions, ) (keyBytes, valBytes int64, err error) { - lockTableKey := b.lockTableKey(key.Key, lock.Exclusive, txnUUID) + lockTableKey := b.lockTableKey(key.Key, lock.Intent, txnUUID) if txnDidNotUpdateMeta { err = writer.SingleClearEngineKey(lockTableKey) } else { diff --git a/pkg/storage/pebble_mvcc_scanner_test.go b/pkg/storage/pebble_mvcc_scanner_test.go index 9f254ffeb365..ade147e50b47 100644 --- a/pkg/storage/pebble_mvcc_scanner_test.go +++ b/pkg/storage/pebble_mvcc_scanner_test.go @@ -73,7 +73,7 @@ func TestMVCCScanWithManyVersionsAndSeparatedIntents(t *testing.T) { for _, k := range keys { lockTableKey, _ := LockTableKey{ Key: k, - Strength: lock.Exclusive, + Strength: lock.Intent, TxnUUID: uuid, }.ToEngineKey(nil) err = eng.PutEngineKey(lockTableKey, metaBytes) diff --git a/pkg/storage/pebble_test.go b/pkg/storage/pebble_test.go index 0d63d5f54835..25fb4ff0b49e 100644 --- a/pkg/storage/pebble_test.go +++ b/pkg/storage/pebble_test.go @@ -597,7 +597,7 @@ func TestPebbleMVCCTimeIntervalCollector(t *testing.T) { // Nothing added. finishAndCheck(0, 0) uuid := uuid.Must(uuid.FromString("6ba7b810-9dad-11d1-80b4-00c04fd430c8")) - ek, _ := LockTableKey{aKey, lock.Exclusive, uuid}.ToEngineKey(nil) + ek, _ := LockTableKey{aKey, lock.Intent, uuid}.ToEngineKey(nil) require.NoError(t, collector.Add(pebble.InternalKey{UserKey: ek.Encode()}, []byte("foo"))) // The added key was not an MVCCKey. finishAndCheck(0, 0) @@ -1237,7 +1237,7 @@ func TestShortAttributeExtractor(t *testing.T) { var txnUUID [uuid.Size]byte lockKey, _ := LockTableKey{ - Key: roachpb.Key("a"), Strength: lock.Exclusive, TxnUUID: txnUUID}.ToEngineKey(nil) + Key: roachpb.Key("a"), Strength: lock.Intent, TxnUUID: txnUUID}.ToEngineKey(nil) v := MVCCValue{} tombstoneVal, err := EncodeMVCCValue(v) require.NoError(t, err)