diff --git a/pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go b/pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go index 51aee2b6b7b8..67212396ded1 100644 --- a/pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go @@ -1732,10 +1732,10 @@ func TestPartialRollbackOnEndTransaction(t *testing.T) { if res.Intent != nil { t.Errorf("found intent, expected none: %+v", res.Intent) } - if res.Value == nil { + if !res.Value.Exists() { t.Errorf("no value found, expected one") } else { - s, err := res.Value.GetBytes() + s, err := res.Value.Value.GetBytes() if err != nil { t.Fatal(err) } diff --git a/pkg/kv/kvserver/batcheval/cmd_get.go b/pkg/kv/kvserver/batcheval/cmd_get.go index ff3b23b83ea3..4629684384a3 100644 --- a/pkg/kv/kvserver/batcheval/cmd_get.go +++ b/pkg/kv/kvserver/batcheval/cmd_get.go @@ -98,7 +98,7 @@ func Get( intents = append(intents, *getRes.Intent) } - reply.Value = getRes.Value + reply.Value = getRes.Value.ToPointer() if h.ReadConsistency == kvpb.READ_UNCOMMITTED { var intentVals []roachpb.KeyValue // NOTE: MVCCGet uses a Prefix iterator, so we want to use one in @@ -117,7 +117,7 @@ func Get( } } - shouldLockKey := getRes.Value != nil || args.LockNonExisting + shouldLockKey := getRes.Value.Exists() || args.LockNonExisting var res result.Result if args.KeyLockingStrength != lock.None && shouldLockKey { // ExpectExclusionSince is used by callers (namely, txnWriteBuffers) that diff --git a/pkg/kv/kvserver/batcheval/cmd_refresh.go b/pkg/kv/kvserver/batcheval/cmd_refresh.go index 41d5e92b9e53..14efb3be8a28 100644 --- a/pkg/kv/kvserver/batcheval/cmd_refresh.go +++ b/pkg/kv/kvserver/batcheval/cmd_refresh.go @@ -62,8 +62,8 @@ func Refresh( if err != nil { return result.Result{}, err - } else if res.Value != nil { - if ts := res.Value.Timestamp; refreshFrom.Less(ts) { + } else if res.Value.Exists() { + if ts := res.Value.Value.Timestamp; refreshFrom.Less(ts) { return result.Result{}, kvpb.NewRefreshFailedError(ctx, kvpb.RefreshFailedError_REASON_COMMITTED_VALUE, args.Key, ts) } diff --git a/pkg/kv/kvserver/batcheval/cmd_refresh_range_test.go b/pkg/kv/kvserver/batcheval/cmd_refresh_range_test.go index cb76c248a6f9..0a2b6f012c0c 100644 --- a/pkg/kv/kvserver/batcheval/cmd_refresh_range_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_refresh_range_test.go @@ -199,8 +199,8 @@ func TestRefreshRangeTimeBoundIterator(t *testing.T) { t.Fatal(err) } else if res.Intent != nil { t.Fatalf("got unexpected intent: %v", intent) - } else if !res.Value.EqualTagAndData(v) { - t.Fatalf("expected %v, got %v", v, res.Value) + } else if !res.Value.Value.EqualTagAndData(v) { + t.Fatalf("expected %v, got %v", v, res.Value.Value) } // Now the real test: a transaction at ts2 has been pushed to ts3 diff --git a/pkg/kv/kvserver/batcheval/cmd_resolve_intent_test.go b/pkg/kv/kvserver/batcheval/cmd_resolve_intent_test.go index 499ebcbe7354..212911dbf71e 100644 --- a/pkg/kv/kvserver/batcheval/cmd_resolve_intent_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_resolve_intent_test.go @@ -259,10 +259,10 @@ func TestResolveIntentAfterPartialRollback(t *testing.T) { if res.Intent != nil { t.Errorf("%s: found intent, expected none: %+v", k, res.Intent) } - if res.Value == nil { + if !res.Value.Exists() { t.Errorf("%s: no value found, expected one", k) } else { - s, err := res.Value.GetBytes() + s, err := res.Value.Value.GetBytes() if err != nil { t.Fatal(err) } @@ -366,8 +366,8 @@ func TestResolveIntentWithTargetBytes(t *testing.T) { valueRes, err := storage.MVCCGet(ctx, batch, testKeys[0], ts, storage.MVCCGetOptions{}) require.NoError(t, err) - require.Equal(t, values[0].RawBytes, valueRes.Value.RawBytes, - "the value %s in get result does not match the value %s in request", values[0].RawBytes, valueRes.Value.RawBytes) + require.Equal(t, values[0].RawBytes, valueRes.Value.Value.RawBytes, + "the value %s in get result does not match the value %s in request", values[0].RawBytes, valueRes.Value.Value.RawBytes) } } else { // Resolve an intent range for testKeys[0], testKeys[1], ..., @@ -435,8 +435,8 @@ func TestResolveIntentWithTargetBytes(t *testing.T) { valueRes, err := storage.MVCCGet(ctx, batch, testKeys[2], ts, storage.MVCCGetOptions{}) require.NoError(t, err) - require.Equal(t, values[2].RawBytes, valueRes.Value.RawBytes, - "the value %s in get result does not match the value %s in request", values[2].RawBytes, valueRes.Value.RawBytes) + require.Equal(t, values[2].RawBytes, valueRes.Value.Value.RawBytes, + "the value %s in get result does not match the value %s in request", values[2].RawBytes, valueRes.Value.Value.RawBytes) _, err = storage.MVCCGet(ctx, batch, testKeys[3], ts, storage.MVCCGetOptions{}) require.Error(t, err) } @@ -469,8 +469,8 @@ func TestResolveIntentWithTargetBytes(t *testing.T) { valueRes, err := storage.MVCCGet(ctx, batch, testKeys[nKeys-1], ts, storage.MVCCGetOptions{}) require.NoError(t, err) - require.Equal(t, values[nKeys-1].RawBytes, valueRes.Value.RawBytes, - "the value %s in get result does not match the value %s in request", values[nKeys-1].RawBytes, valueRes.Value.RawBytes) + require.Equal(t, values[nKeys-1].RawBytes, valueRes.Value.Value.RawBytes, + "the value %s in get result does not match the value %s in request", values[nKeys-1].RawBytes, valueRes.Value.Value.RawBytes) } } }) diff --git a/pkg/kv/kvserver/batcheval/cmd_subsume.go b/pkg/kv/kvserver/batcheval/cmd_subsume.go index 7effe28fa7fa..102f91abbb16 100644 --- a/pkg/kv/kvserver/batcheval/cmd_subsume.go +++ b/pkg/kv/kvserver/batcheval/cmd_subsume.go @@ -160,7 +160,7 @@ func Subsume( valRes, err := storage.MVCCGetAsTxn(ctx, readWriter, descKey, intentRes.Intent.Txn.WriteTimestamp, intentRes.Intent.Txn) if err != nil { return result.Result{}, errors.Wrap(err, "fetching local range descriptor as txn") - } else if valRes.Value != nil { + } else if valRes.Value.IsPresent() { return result.Result{}, errors.Errorf("non-deletion intent on local range descriptor") } diff --git a/pkg/kv/kvserver/batcheval/lock.go b/pkg/kv/kvserver/batcheval/lock.go index f0612add2fe2..5f5d4daeab34 100644 --- a/pkg/kv/kvserver/batcheval/lock.go +++ b/pkg/kv/kvserver/batcheval/lock.go @@ -79,11 +79,11 @@ func readProvisionalVal( if err != nil { return roachpb.KeyValue{}, err } - if valRes.Value == nil { + if !valRes.Value.IsPresent() { // Intent is a deletion. return roachpb.KeyValue{}, nil } - return roachpb.KeyValue{Key: intent.Key, Value: *valRes.Value}, nil + return roachpb.KeyValue{Key: intent.Key, Value: valRes.Value.Value}, nil } res, err := storage.MVCCScanAsTxn( ctx, reader, intent.Key, intent.Key.Next(), intent.Txn.WriteTimestamp, intent.Txn, diff --git a/pkg/kv/kvserver/client_replica_test.go b/pkg/kv/kvserver/client_replica_test.go index 6c56612686df..4957682b6ce6 100644 --- a/pkg/kv/kvserver/client_replica_test.go +++ b/pkg/kv/kvserver/client_replica_test.go @@ -216,7 +216,7 @@ func TestLeaseholdersRejectClockUpdateWithJump(t *testing.T) { valRes, err := storage.MVCCGet(context.Background(), store.StateEngine(), key, ts3, storage.MVCCGetOptions{}) require.NoError(t, err) - require.Equal(t, incArgs.Increment*numCmds, mustGetInt(valRes.Value)) + require.Equal(t, incArgs.Increment*numCmds, mustGetInt(valRes.Value.ToPointer())) } // TestTxnPutOutOfOrder tests a case where a put operation of an older diff --git a/pkg/kv/kvserver/loqrecovery/apply.go b/pkg/kv/kvserver/loqrecovery/apply.go index e23596ccf3d8..953ea57a0895 100644 --- a/pkg/kv/kvserver/loqrecovery/apply.go +++ b/pkg/kv/kvserver/loqrecovery/apply.go @@ -178,7 +178,7 @@ func applyReplicaUpdate( key := keys.RangeDescriptorKey(update.StartKey.AsRKey()) res, err := storage.MVCCGet( ctx, readWriter, key, clock.Now(), storage.MVCCGetOptions{Inconsistent: true}) - if res.Value == nil { + if !res.Value.Exists() { return PrepareReplicaReport{}, errors.Errorf( "failed to find a range descriptor for range %v", key) } @@ -186,7 +186,7 @@ func applyReplicaUpdate( return PrepareReplicaReport{}, err } var localDesc roachpb.RangeDescriptor - if err := res.Value.GetProto(&localDesc); err != nil { + if err := res.Value.Value.GetProto(&localDesc); err != nil { return PrepareReplicaReport{}, err } // Sanity check that this is indeed the right range. diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index 3a06c6757473..05397853bbba 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -2527,7 +2527,7 @@ func (r *Replica) maybeWatchForMergeLocked(ctx context.Context) (bool, error) { ctx, r.store.StateEngine(), descKey, intentRes.Intent.Txn.WriteTimestamp, intentRes.Intent.Txn) if err != nil { return false, err - } else if valRes.Value != nil { + } else if valRes.Value.IsPresent() { return false, nil } diff --git a/pkg/kv/kvserver/replica_rangefeed.go b/pkg/kv/kvserver/replica_rangefeed.go index ec738d23e0fd..969698b78c65 100644 --- a/pkg/kv/kvserver/replica_rangefeed.go +++ b/pkg/kv/kvserver/replica_rangefeed.go @@ -659,8 +659,8 @@ func populatePrevValsInLogicalOpLog( if err != nil { return errors.Wrapf(err, "consuming %T for key %v @ ts %v", op, key, ts) } - if prevValRes.Value != nil { - *prevValPtr = prevValRes.Value.RawBytes + if prevValRes.Value.Exists() { + *prevValPtr = prevValRes.Value.Value.RawBytes } else { *prevValPtr = nil } diff --git a/pkg/server/node_tombstone_storage.go b/pkg/server/node_tombstone_storage.go index 90d121733e92..37ddedfed2ef 100644 --- a/pkg/server/node_tombstone_storage.go +++ b/pkg/server/node_tombstone_storage.go @@ -58,12 +58,12 @@ func (s *nodeTombstoneStorage) IsDecommissioned( if err != nil { return time.Time{}, err } - if valRes.Value == nil { + if !valRes.Value.Exists() { // Not found. continue } var tsp hlc.Timestamp - if err := valRes.Value.GetProto(&tsp); err != nil { + if err := valRes.Value.Value.GetProto(&tsp); err != nil { return time.Time{}, err } // Found, offer to cache and return. diff --git a/pkg/storage/bench_test.go b/pkg/storage/bench_test.go index 5d7a3b5af2e0..483356a17c8f 100644 --- a/pkg/storage/bench_test.go +++ b/pkg/storage/bench_test.go @@ -1216,9 +1216,9 @@ func runMVCCGet(ctx context.Context, b *testing.B, opts mvccBenchData, useBatch ReadCategory: fs.BatchEvalReadCategory, }); err != nil { b.Fatalf("failed get: %+v", err) - } else if valRes.Value == nil { + } else if !valRes.Value.Exists() { b.Fatalf("failed get (key not found): %d@%d", keyIdx, walltime) - } else if valueBytes, err := valRes.Value.GetBytes(); err != nil { + } else if valueBytes, err := valRes.Value.Value.GetBytes(); err != nil { b.Fatal(err) } else if len(valueBytes) != opts.valueBytes { b.Fatalf("unexpected value size: %d", len(valueBytes)) diff --git a/pkg/storage/engine_test.go b/pkg/storage/engine_test.go index a7bbcc069703..3c76bc7f9610 100644 --- a/pkg/storage/engine_test.go +++ b/pkg/storage/engine_test.go @@ -203,8 +203,8 @@ func TestEngineBatchStaleCachedIterator(t *testing.T) { if valRes, err := MVCCGet(context.Background(), batch, key, hlc.Timestamp{}, MVCCGetOptions{}); err != nil { t.Fatal(err) - } else if valRes.Value != nil { - t.Fatalf("expected no value, got %+v", valRes.Value) + } else if valRes.Value.Exists() { + t.Fatalf("expected no value, got %+v", valRes.Value.Value) } } } diff --git a/pkg/storage/metamorphic/operations.go b/pkg/storage/metamorphic/operations.go index a07d02fab6c6..9db8592b0c82 100644 --- a/pkg/storage/metamorphic/operations.go +++ b/pkg/storage/metamorphic/operations.go @@ -190,7 +190,7 @@ func (m mvccGetOp) run(ctx context.Context) string { if err != nil { return fmt.Sprintf("error: %s", err) } - return fmt.Sprintf("val = %v, intent = %v", res.Value, res.Intent) + return fmt.Sprintf("val = %v, intent = %v", res.Value.ToPointer(), res.Intent) } type mvccPutOp struct { diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index 3029c1d5098f..67414d3ce4b2 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -396,26 +396,33 @@ func (r MVCCRangeKeyValue) Clone() MVCCRangeKeyValue { return r } -// optionalValue represents an optional MVCCValue. It is preferred -// over a *roachpb.Value or *MVCCValue to avoid the forced heap allocation. -type optionalValue struct { +// OptionalValue represents an optional roachpb.Value with its associated +// MVCCValueHeader. It is preferred over *roachpb.Value to avoid forced heap +// allocation. +type OptionalValue struct { MVCCValue exists bool } -func makeOptionalValue(v MVCCValue) optionalValue { - return optionalValue{MVCCValue: v, exists: true} +// MakeOptionalValue constructs an OptionalValue from an MVCCValue. +func MakeOptionalValue(v MVCCValue) OptionalValue { + return OptionalValue{MVCCValue: v, exists: true} } -func (v *optionalValue) IsPresent() bool { +// IsPresent returns true if the value exists and is not a tombstone (deletion). +func (v OptionalValue) IsPresent() bool { return v.exists && v.Value.IsPresent() } -func (v *optionalValue) IsTombstone() bool { - return v.exists && !v.Value.IsPresent() +// Exists returns true if the value exists (including tombstones). +func (v OptionalValue) Exists() bool { + return v.exists } -func (v *optionalValue) ToPointer() *roachpb.Value { +// ToPointer returns a pointer to the value for callers that need it. +// This forces an allocation but provides backward compatibility. +// Returns nil if the value does not exist. +func (v OptionalValue) ToPointer() *roachpb.Value { if !v.exists { return nil } @@ -424,21 +431,6 @@ func (v *optionalValue) ToPointer() *roachpb.Value { return &cpy } -func (v *optionalValue) isOriginTimestampWinner( - proposedTS hlc.Timestamp, inclusive bool, -) (bool, hlc.Timestamp) { - if !v.exists { - return true, hlc.Timestamp{} - } - - existTS := v.Value.Timestamp - if v.MVCCValueHeader.OriginTimestamp.IsSet() { - existTS = v.MVCCValueHeader.OriginTimestamp - } - - return existTS.Less(proposedTS) || (inclusive && existTS.Equal(proposedTS)), existTS -} - // isSysLocal returns whether the key is system-local. func isSysLocal(key roachpb.Key) bool { return key.Compare(keys.LocalMax) < 0 @@ -1103,13 +1095,13 @@ func MVCCGetProto( ) (bool, error) { // TODO(tschottdorf): Consider returning skipped intents to the caller. valueRes, mvccGetErr := MVCCGet(ctx, reader, key, timestamp, opts) - found := valueRes.Value != nil + found := valueRes.Value.Exists() // If we found a result, parse it regardless of the error returned by MVCCGet. if found && msg != nil { // If the unmarshal failed, return its result. Otherwise, pass // through the underlying error (which may be a LockConflictError // to be handled specially alongside the returned value). - if err := valueRes.Value.GetProto(msg); err != nil { + if err := valueRes.Value.Value.GetProto(msg); err != nil { return found, err } } @@ -1280,9 +1272,9 @@ type MVCCGetOptions struct { // MVCCGetResult bundles return values for the MVCCGet family of functions. type MVCCGetResult struct { // The most recent value for the specified key whose timestamp is less than - // or equal to the supplied timestamp. If no such value exists, nil is - // returned instead. - Value *roachpb.Value + // or equal to the supplied timestamp. Use Value.Exists() to check if a value + // was found, and Value.IsPresent() to check if it's not a tombstone. + Value OptionalValue // In inconsistent mode, the intent if an intent is encountered. In // consistent mode, an intent will generate a LockConflictError with the // intent embedded within and the intent parameter will be nil. @@ -1507,11 +1499,10 @@ func MVCCGetWithValueHeader( } defer iter.Close() value, intent, err := mvccGet(ctx, iter, key, timestamp, opts) - val := value.ToPointer() - if err == nil && val != nil { + if err == nil && value.exists { // NB: This calculation is different from Scan, since Scan responses include // the key/value pair while Get only includes the value. - numBytes := int64(len(val.RawBytes)) + numBytes := int64(len(value.Value.RawBytes)) if opts.TargetBytes > 0 && opts.AllowEmpty && numBytes > opts.TargetBytes { result.ResumeSpan = &roachpb.Span{Key: key} result.ResumeReason = kvpb.RESUME_BYTE_LIMIT @@ -1521,33 +1512,30 @@ func MVCCGetWithValueHeader( result.NumKeys = 1 result.NumBytes = numBytes } - result.Value = val + result.Value = value result.Intent = intent return result, value.MVCCValueHeader, err } -// mvccGet returns an optionalValue containing the MVCCValue for the -// given key (if it exists). -// -// The MVCCValueHeader is included in the returned MVCCValue. +// mvccGet returns an OptionalValue for the given key (if it exists). func mvccGet( ctx context.Context, iter MVCCIterator, key roachpb.Key, timestamp hlc.Timestamp, opts MVCCGetOptions, -) (value optionalValue, intent *roachpb.Intent, err error) { +) (value OptionalValue, intent *roachpb.Intent, err error) { if len(key) == 0 { - return optionalValue{}, nil, emptyKeyError() + return OptionalValue{}, nil, emptyKeyError() } if timestamp.WallTime < 0 { - return optionalValue{}, nil, errors.Errorf("cannot write to %q at timestamp %s", key, timestamp) + return OptionalValue{}, nil, errors.Errorf("cannot write to %q at timestamp %s", key, timestamp) } if util.RaceEnabled && !iter.IsPrefix() { - return optionalValue{}, nil, errors.AssertionFailedf("mvccGet called with non-prefix iterator") + return OptionalValue{}, nil, errors.AssertionFailedf("mvccGet called with non-prefix iterator") } if err := opts.validate(); err != nil { - return optionalValue{}, nil, err + return OptionalValue{}, nil, err } mvccScanner := pebbleMVCCScannerPool.Get().(*pebbleMVCCScanner) @@ -1590,39 +1578,39 @@ func mvccGet( } if mvccScanner.err != nil { - return optionalValue{}, nil, mvccScanner.err + return OptionalValue{}, nil, mvccScanner.err } intents, err := buildScanIntents(mvccScanner.intentsRepr()) if err != nil { - return optionalValue{}, nil, err + return OptionalValue{}, nil, err } if opts.errOnIntents() && len(intents) > 0 { lcErr := &kvpb.LockConflictError{Locks: roachpb.AsLocks(intents)} - return optionalValue{}, nil, lcErr + return OptionalValue{}, nil, lcErr } if len(intents) > 1 { - return optionalValue{}, nil, errors.Errorf("expected 0 or 1 intents, got %d", len(intents)) + return OptionalValue{}, nil, errors.Errorf("expected 0 or 1 intents, got %d", len(intents)) } else if len(intents) == 1 { intent = &intents[0] } if len(results.repr) == 0 { - return optionalValue{}, intent, nil + return OptionalValue{}, intent, nil } mvccKey, rawValue, _, err := MVCCScanDecodeKeyValue(results.repr) if err != nil { - return optionalValue{}, nil, err + return OptionalValue{}, nil, err } // NB: we may return MVCCValueHeader out of curUnsafeValue because that // type does not contain any pointers. A comment on MVCCValueHeader ensures // that this stays true. - value = makeOptionalValue(MVCCValue{Value: roachpb.Value{ - RawBytes: rawValue, - Timestamp: mvccKey.Timestamp, - }, MVCCValueHeader: mvccScanner.curUnsafeValue.MVCCValueHeader}) + value = MakeOptionalValue(MVCCValue{ + MVCCValueHeader: mvccScanner.curUnsafeValue.MVCCValueHeader, + Value: roachpb.Value{RawBytes: rawValue, Timestamp: mvccKey.Timestamp}, + }) return value, intent, nil } @@ -2052,7 +2040,7 @@ func mvccPutUsingIter( key roachpb.Key, timestamp hlc.Timestamp, value roachpb.Value, - valueFn func(optionalValue) (roachpb.Value, error), + valueFn func(OptionalValue) (roachpb.Value, error), opts MVCCWriteOptions, ) (roachpb.LockAcquisition, error) { buf := newPutBuffer() @@ -2104,10 +2092,10 @@ func replayTransactionalWrite( key roachpb.Key, value roachpb.Value, txn *roachpb.Transaction, - valueFn func(optionalValue) (roachpb.Value, error), + valueFn func(OptionalValue) (roachpb.Value, error), replayWriteTimestampProtection bool, ) error { - var writtenValue optionalValue + var writtenValue OptionalValue var err error if txn.Sequence == meta.Txn.Sequence { // This is a special case. This is when the intent hasn't made it @@ -2130,7 +2118,7 @@ func replayTransactionalWrite( if err != nil { return err } - writtenValue = makeOptionalValue(intentVal) + writtenValue = MakeOptionalValue(intentVal) } } if !writtenValue.exists { @@ -2152,7 +2140,7 @@ func replayTransactionalWrite( // If the valueFn is specified, we must apply it to the would-be value at the key. if valueFn != nil { - var exVal optionalValue + var exVal OptionalValue // If there's an intent history, use that. prevIntent, prevValueWritten := meta.GetPrevIntentSeq(txn.Sequence, txn.IgnoredSeqNums) @@ -2164,20 +2152,19 @@ func replayTransactionalWrite( if err != nil { return err } - exVal = makeOptionalValue(prevIntentVal) + exVal = MakeOptionalValue(prevIntentVal) } else { // If the previous value at the key wasn't written by this // transaction, or it was hidden by a rolled back seqnum, we look at // last committed value on the key. Since we want the last committed // value on the key, we read below our previous intents here. metaTimestamp := meta.Timestamp.ToTimestamp() - val, _, err := mvccGet(ctx, iter, key, metaTimestamp.Prev(), MVCCGetOptions{ + exVal, _, err = mvccGet(ctx, iter, key, metaTimestamp.Prev(), MVCCGetOptions{ Tombstones: true, }) if err != nil { return err } - exVal = val } value, err = valueFn(exVal) @@ -2267,7 +2254,7 @@ func mvccPutInternal( timestamp hlc.Timestamp, value roachpb.Value, buf *putBuffer, - valueFn func(optionalValue) (roachpb.Value, error), + valueFn func(OptionalValue) (roachpb.Value, error), opts MVCCWriteOptions, ) (bool, roachpb.LockAcquisition, error) { if len(key) == 0 { @@ -2362,9 +2349,10 @@ func mvccPutInternal( return false, roachpb.LockAcquisition{}, errors.Errorf("%q: inline writes not allowed within transactions", metaKey) } if valueFn != nil { - var inlineVal optionalValue + var inlineVal OptionalValue if ok { - inlineVal = makeOptionalValue(MVCCValue{Value: roachpb.Value{RawBytes: meta.RawBytes}}) + // Inline values don't have MVCCValueHeaders. + inlineVal = MakeOptionalValue(MVCCValue{Value: roachpb.Value{RawBytes: meta.RawBytes}}) } if value, err = valueFn(inlineVal); err != nil { return false, roachpb.LockAcquisition{}, err @@ -2480,7 +2468,7 @@ func mvccPutInternal( // committed values, and all past writes by this transaction have been // rolled back, either due to transaction retries or transaction savepoint // rollbacks.) - var exVal optionalValue + var exVal OptionalValue // Set when the current provisional value is not ignored due to a txn // restart or a savepoint rollback. Represents an encoded MVCCValue. var curProvValRaw []byte @@ -2508,7 +2496,7 @@ func mvccPutInternal( if err != nil { return false, roachpb.LockAcquisition{}, err } - exVal = makeOptionalValue(curIntentVal) + exVal = MakeOptionalValue(curIntentVal) } else { // Seqnum of last write was ignored. Try retrieving the value from the history. prevIntent, prevIntentOk := meta.GetPrevIntentSeq(opts.Txn.Sequence, opts.Txn.IgnoredSeqNums) @@ -2517,7 +2505,7 @@ func mvccPutInternal( if err != nil { return false, roachpb.LockAcquisition{}, err } - exVal = makeOptionalValue(prevIntentVal) + exVal = MakeOptionalValue(prevIntentVal) } } } @@ -2529,14 +2517,13 @@ func mvccPutInternal( // // Since we want the last committed value on the key, we must // read below our previous intents here. - optVal, _, err := mvccGet(ctx, iter, key, metaTimestamp.Prev(), MVCCGetOptions{ + exVal, _, err = mvccGet(ctx, iter, key, metaTimestamp.Prev(), MVCCGetOptions{ Tombstones: true, ReadCategory: opts.Category, }) if err != nil { return false, roachpb.LockAcquisition{}, err } - exVal = optVal } exReplaced = exVal.IsPresent() @@ -2671,12 +2658,12 @@ func mvccPutInternal( return false, roachpb.LockAcquisition{}, kvpb.NewExclusionViolationError(exclusionTimestamp, metaTimestamp, key) } else /* meta.Txn == nil && metaTimestamp.Less(readTimestamp) */ { // If a valueFn is specified, read the existing value using iter. - opts := MVCCGetOptions{ + getOpts := MVCCGetOptions{ Tombstones: true, ReadCategory: opts.Category, } if valueFn != nil { - exVal, _, err := mvccGet(ctx, iter, key, readTimestamp, opts) + exVal, _, err := mvccGet(ctx, iter, key, readTimestamp, getOpts) if err != nil { return false, roachpb.LockAcquisition{}, err } @@ -2691,7 +2678,7 @@ func mvccPutInternal( // There is no existing value for this key. Even if the new value is // nil write a deletion tombstone for the key. if valueFn != nil { - value, err = valueFn(optionalValue{exists: false}) + value, err = valueFn(OptionalValue{}) if err != nil { return false, roachpb.LockAcquisition{}, err } @@ -2862,7 +2849,7 @@ func MVCCIncrement( var int64Val int64 var newInt64Val int64 - valueFn := func(value optionalValue) (roachpb.Value, error) { + valueFn := func(value OptionalValue) (roachpb.Value, error) { if value.IsPresent() { var err error if int64Val, err = value.Value.GetInt(); err != nil { @@ -2996,11 +2983,29 @@ func MVCCBlindConditionalPut( // then a non-existent actual value is allowed even when // expected-value is non-empty. func maybeConditionFailedError( - expBytes []byte, actVal optionalValue, allowNoExisting bool, + expBytes []byte, actVal OptionalValue, allowNoExisting bool, ) *kvpb.ConditionFailedError { return mvcceval.MaybeConditionFailedError(expBytes, actVal.ToPointer(), actVal.IsPresent(), allowNoExisting) } +// isOriginTimestampWinner determines whether the proposed OriginTimestamp is +// newer than the existing value's OriginTimestamp. Returns true if the proposal +// wins, along with the existing timestamp for error reporting. +func isOriginTimestampWinner( + existVal OptionalValue, proposedTS hlc.Timestamp, inclusive bool, +) (bool, hlc.Timestamp) { + if !existVal.exists { + return true, hlc.Timestamp{} + } + + existTS := existVal.Value.Timestamp + if existVal.OriginTimestamp.IsSet() { + existTS = existVal.OriginTimestamp + } + + return existTS.Less(proposedTS) || (inclusive && existTS.Equal(proposedTS)), existTS +} + func mvccConditionalPutUsingIter( ctx context.Context, writer Writer, @@ -3022,17 +3027,17 @@ func mvccConditionalPutUsingIter( } } - var valueFn func(existVal optionalValue) (roachpb.Value, error) + var valueFn func(existVal OptionalValue) (roachpb.Value, error) if opts.OriginTimestamp.IsEmpty() { - valueFn = func(actualValue optionalValue) (roachpb.Value, error) { + valueFn = func(actualValue OptionalValue) (roachpb.Value, error) { if err := maybeConditionFailedError(expBytes, actualValue, bool(opts.AllowIfDoesNotExist)); err != nil { return roachpb.Value{}, err } return value, nil } } else { - valueFn = func(existVal optionalValue) (roachpb.Value, error) { - originTSWinner, existTS := existVal.isOriginTimestampWinner(opts.OriginTimestamp, false) + valueFn = func(existVal OptionalValue) (roachpb.Value, error) { + originTSWinner, existTS := isOriginTimestampWinner(existVal, opts.OriginTimestamp, false) if !originTSWinner { return roachpb.Value{}, &kvpb.ConditionFailedError{ OriginTimestampOlderThan: existTS, diff --git a/pkg/storage/mvcc_history_test.go b/pkg/storage/mvcc_history_test.go index 61ba9938991c..88ca79f9da85 100644 --- a/pkg/storage/mvcc_history_test.go +++ b/pkg/storage/mvcc_history_test.go @@ -1475,8 +1475,8 @@ func cmdGet(e *evalCtx) error { if res.Intent != nil { e.results.buf.Printf("get: %v -> intent {%s}\n", key, res.Intent.Txn) } - if res.Value != nil { - e.results.buf.Printf("get: %v -> %v @%v\n", key, res.Value.PrettyPrint(), res.Value.Timestamp) + if res.Value.Exists() { + e.results.buf.Printf("get: %v -> %v @%v\n", key, res.Value.Value.PrettyPrint(), res.Value.Value.Timestamp) } else { e.results.buf.Printf("get: %v -> \n", key) } diff --git a/pkg/storage/mvcc_test.go b/pkg/storage/mvcc_test.go index a694a9ccbf39..33332f6f8d76 100644 --- a/pkg/storage/mvcc_test.go +++ b/pkg/storage/mvcc_test.go @@ -291,7 +291,7 @@ func TestMVCCGetNotExist(t *testing.T) { if err != nil { t.Fatal(err) } - if valueRes.Value != nil { + if valueRes.Value.Exists() { t.Fatal("the value should be empty") } } @@ -324,7 +324,7 @@ func TestMVCCGetNoMoreOldVersion(t *testing.T) { if err != nil { t.Fatal(err) } - if valueRes.Value != nil { + if valueRes.Value.Exists() { t.Fatal("the value should be empty") } } @@ -344,7 +344,7 @@ func TestMVCCGetWithValueHeader(t *testing.T) { if err != nil { t.Fatal(err) } - if valueRes.Value == nil { + if !valueRes.Value.Exists() { t.Fatal("the value should not be empty") } require.Equal(t, hlc.ClockTimestamp{WallTime: 1}, vh.LocalTimestamp) @@ -359,7 +359,7 @@ func TestMVCCGetWithValueHeader(t *testing.T) { if err != nil { t.Fatal(err) } - if valueRes.Value != nil { + if valueRes.Value.Exists() { t.Fatal("the value should be empty") } require.Zero(t, vh.LocalTimestamp) @@ -368,8 +368,8 @@ func TestMVCCGetWithValueHeader(t *testing.T) { MVCCGetOptions{Tombstones: true}) if err != nil { t.Fatal(err) - } else if valueRes.Value == nil || len(valueRes.Value.RawBytes) != 0 { - t.Fatalf("the value should be non-nil with empty RawBytes; got %+v", valueRes.Value) + } else if !valueRes.Value.Exists() || len(valueRes.Value.Value.RawBytes) != 0 { + t.Fatalf("the value should be non-nil with empty RawBytes; got %+v", valueRes.Value.Value) } require.Equal(t, hlc.ClockTimestamp{WallTime: 2, Logical: 1}, vh.LocalTimestamp) @@ -381,7 +381,7 @@ func TestMVCCGetWithValueHeader(t *testing.T) { if err != nil { t.Fatal(err) } - if valueRes.Value == nil { + if !valueRes.Value.Exists() { t.Fatal("the value should not be empty") } require.Equal(t, hlc.ClockTimestamp{WallTime: 1}, vh.LocalTimestamp) @@ -465,7 +465,7 @@ func TestMVCCValueHeadersForRangefeeds(t *testing.T) { MVCCGetOptions{Tombstones: true}) require.NoError(t, err) require.NotNil(t, valueRes.Value) - require.Zero(t, len(valueRes.Value.RawBytes)) + require.Zero(t, len(valueRes.Value.Value.RawBytes)) require.Equal(t, omitInRangefeeds, vh.OmitInRangefeeds) require.Equal(t, originID, vh.OriginID) @@ -490,7 +490,7 @@ func TestMVCCValueHeadersForRangefeeds(t *testing.T) { MVCCGetOptions{Tombstones: true}) require.NoError(t, err) require.NotNil(t, valueRes.Value) - require.Zero(t, len(valueRes.Value.RawBytes)) + require.Zero(t, len(valueRes.Value.Value.RawBytes)) require.Equal(t, omitInRangefeeds, vh.OmitInRangefeeds) require.Equal(t, originID, vh.OriginID) }) @@ -539,8 +539,8 @@ func TestMVCCInlineWithTxn(t *testing.T) { if err != nil { t.Fatal(err) } - if !reflect.DeepEqual(value1, *valueRes.Value) { - t.Errorf("the inline value should be %v; got %v", value1, *valueRes.Value) + if !reflect.DeepEqual(value1, valueRes.Value.Value) { + t.Errorf("the inline value should be %v; got %v", value1, valueRes.Value.Value) } // Verify inline get with txn does still work (this will happen on a @@ -593,7 +593,7 @@ func TestMVCCGetAndDeleteInTxn(t *testing.T) { Txn: txn, }); err != nil { t.Fatal(err) - } else if valueRes.Value == nil { + } else if !valueRes.Value.Exists() { t.Fatal("the value should not be empty") } @@ -608,7 +608,7 @@ func TestMVCCGetAndDeleteInTxn(t *testing.T) { Txn: txn, }); err != nil { t.Fatal(err) - } else if valueRes.Value != nil { + } else if valueRes.Value.Exists() { t.Fatal("the value should be empty") } // Read the latest version with tombstone. @@ -617,16 +617,16 @@ func TestMVCCGetAndDeleteInTxn(t *testing.T) { Txn: txn, }); err != nil { t.Fatal(err) - } else if valueRes.Value == nil || len(valueRes.Value.RawBytes) != 0 { - t.Fatalf("the value should be non-nil with empty RawBytes; got %+v", valueRes.Value) + } else if !valueRes.Value.Exists() || len(valueRes.Value.Value.RawBytes) != 0 { + t.Fatalf("the value should be non-nil with empty RawBytes; got %+v", valueRes.Value.Value) } // Read the old version which shouldn't exist, as within a // transaction, we delete previous values. if valueRes, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 2}, MVCCGetOptions{}); err != nil { t.Fatal(err) - } else if valueRes.Value != nil { - t.Fatalf("expected value nil, got: %s", valueRes.Value) + } else if valueRes.Value.Exists() { + t.Fatalf("expected value nil, got: %s", valueRes.Value.Value) } } @@ -823,8 +823,8 @@ func TestMVCCGetInconsistent(t *testing.T) { t.Fatalf("expected %v, but got %v", testKey1, res.Intent) } } - if !bytes.Equal(res.Value.RawBytes, value1.RawBytes) { - t.Errorf("@%s expected %q; got %q", ts, value1.RawBytes, res.Value.RawBytes) + if !bytes.Equal(res.Value.Value.RawBytes, value1.RawBytes) { + t.Errorf("@%s expected %q; got %q", ts, value1.RawBytes, res.Value.Value.RawBytes) } } @@ -837,8 +837,8 @@ func TestMVCCGetInconsistent(t *testing.T) { if res.Intent == nil || !res.Intent.Key.Equal(testKey2) { t.Fatal(err) } - if res.Value != nil { - t.Errorf("expected empty val; got %+v", res.Value) + if res.Value.Exists() { + t.Errorf("expected empty val; got %+v", res.Value.Value) } } @@ -2642,9 +2642,9 @@ func TestMVCCResolveTxn(t *testing.T) { if err != nil { t.Fatal(err) } - if !bytes.Equal(value1.RawBytes, valueRes.Value.RawBytes) { + if !bytes.Equal(value1.RawBytes, valueRes.Value.Value.RawBytes) { t.Fatalf("the value %s in get result does not match the value %s in request", - value1.RawBytes, valueRes.Value.RawBytes) + value1.RawBytes, valueRes.Value.Value.RawBytes) } } @@ -2660,9 +2660,9 @@ func TestMVCCResolveTxn(t *testing.T) { if err != nil { t.Fatal(err) } - if !bytes.Equal(value1.RawBytes, valueRes.Value.RawBytes) { + if !bytes.Equal(value1.RawBytes, valueRes.Value.Value.RawBytes) { t.Fatalf("the value %s in get result does not match the value %s in request", - value1.RawBytes, valueRes.Value.RawBytes) + value1.RawBytes, valueRes.Value.Value.RawBytes) } } } @@ -2699,8 +2699,8 @@ func TestMVCCResolveNewerIntent(t *testing.T) { if err != nil { t.Fatal(err) } - if !bytes.Equal(value1.RawBytes, valueRes.Value.RawBytes) { - t.Fatalf("expected value1 bytes; got %q", valueRes.Value.RawBytes) + if !bytes.Equal(value1.RawBytes, valueRes.Value.Value.RawBytes) { + t.Fatalf("expected value1 bytes; got %q", valueRes.Value.Value.RawBytes) } } @@ -2900,7 +2900,7 @@ func TestMVCCConditionalPutOldTimestamp(t *testing.T) { ts := hlc.Timestamp{WallTime: 3} valueRes, err := MVCCGet(ctx, engine, testKey1, ts, MVCCGetOptions{}) require.NoError(t, err) - require.Equal(t, value2.RawBytes, valueRes.Value.RawBytes) + require.Equal(t, value2.RawBytes, valueRes.Value.Value.RawBytes) } } @@ -2931,7 +2931,7 @@ func TestMVCCMultiplePutOldTimestamp(t *testing.T) { // Verify no value was written. valueRes, err := MVCCGet(ctx, engine, testKey1, hlc.MaxTimestamp, MVCCGetOptions{Txn: txn}) require.NoError(t, err) - require.Equal(t, value1.RawBytes, valueRes.Value.RawBytes) + require.Equal(t, value1.RawBytes, valueRes.Value.Value.RawBytes) // Put again after advancing the txn's timestamp to the WriteTooOld error's // timestamp and verify no WriteTooOldError. @@ -2942,8 +2942,8 @@ func TestMVCCMultiplePutOldTimestamp(t *testing.T) { // Verify new value was actually written at (3, 1). valueRes, err = MVCCGet(ctx, engine, testKey1, hlc.MaxTimestamp, MVCCGetOptions{Txn: txn}) require.NoError(t, err) - require.Equal(t, expTS, valueRes.Value.Timestamp) - require.Equal(t, value3.RawBytes, valueRes.Value.RawBytes) + require.Equal(t, expTS, valueRes.Value.Value.Timestamp) + require.Equal(t, value3.RawBytes, valueRes.Value.Value.RawBytes) } func TestMVCCPutNegativeTimestampError(t *testing.T) { @@ -2993,7 +2993,7 @@ func TestMVCCPutOldOrigTimestampNewCommitTimestamp(t *testing.T) { // Verify no value was written. valueRes, err := MVCCGet(ctx, engine, testKey1, hlc.MaxTimestamp, MVCCGetOptions{Txn: txn}) require.NoError(t, err) - require.Equal(t, value1.RawBytes, valueRes.Value.RawBytes) + require.Equal(t, value1.RawBytes, valueRes.Value.Value.RawBytes) } func TestMVCCAbortTxn(t *testing.T) { @@ -3024,8 +3024,8 @@ func TestMVCCAbortTxn(t *testing.T) { ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, MVCCGetOptions{}, ); err != nil { t.Fatal(err) - } else if valueRes.Value != nil { - t.Fatalf("expected the value to be empty: %s", valueRes.Value) + } else if valueRes.Value.Exists() { + t.Fatalf("expected the value to be empty: %s", valueRes.Value.Value) } require.Empty(t, mvccGetRaw(t, engine, mvccKey(testKey1))) } @@ -3071,11 +3071,11 @@ func TestMVCCAbortTxnWithPreviousVersion(t *testing.T) { ctx, engine, testKey1, hlc.Timestamp{WallTime: 3}, MVCCGetOptions{}, ); err != nil { t.Fatal(err) - } else if expTS := (hlc.Timestamp{WallTime: 1}); valueRes.Value.Timestamp != expTS { - t.Fatalf("expected timestamp %+v == %+v", valueRes.Value.Timestamp, expTS) - } else if !bytes.Equal(value2.RawBytes, valueRes.Value.RawBytes) { + } else if expTS := (hlc.Timestamp{WallTime: 1}); valueRes.Value.Value.Timestamp != expTS { + t.Fatalf("expected timestamp %+v == %+v", valueRes.Value.Value.Timestamp, expTS) + } else if !bytes.Equal(value2.RawBytes, valueRes.Value.Value.RawBytes) { t.Fatalf("the value %q in get result does not match the value %q in request", - valueRes.Value.RawBytes, value2.RawBytes) + valueRes.Value.Value.RawBytes, value2.RawBytes) } } @@ -3131,8 +3131,8 @@ func TestMVCCWriteWithDiffTimestampsAndEpochs(t *testing.T) { // Verify no value was written. valueRes, err := MVCCGet(ctx, engine, testKey1, hlc.MaxTimestamp, MVCCGetOptions{}) require.NoError(t, err) - require.Equal(t, txne2Commit.WriteTimestamp, valueRes.Value.Timestamp) - require.Equal(t, value3.RawBytes, valueRes.Value.RawBytes) + require.Equal(t, txne2Commit.WriteTimestamp, valueRes.Value.Value.Timestamp) + require.Equal(t, value3.RawBytes, valueRes.Value.Value.RawBytes) // Attempt to read older timestamp; should fail. valueRes, err = MVCCGet(ctx, engine, testKey1, hlc.Timestamp{Logical: 0}, MVCCGetOptions{}) @@ -3141,8 +3141,8 @@ func TestMVCCWriteWithDiffTimestampsAndEpochs(t *testing.T) { // Read at correct timestamp. valueRes, err = MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, MVCCGetOptions{}) require.NoError(t, err) - require.Equal(t, txne2Commit.WriteTimestamp, valueRes.Value.Timestamp) - require.Equal(t, value3.RawBytes, valueRes.Value.RawBytes) + require.Equal(t, txne2Commit.WriteTimestamp, valueRes.Value.Value.Timestamp) + require.Equal(t, value3.RawBytes, valueRes.Value.Value.RawBytes) } // TestMVCCGetWithDiffEpochs writes a value first using epoch 1, then @@ -3191,8 +3191,8 @@ func TestMVCCGetWithDiffEpochs(t *testing.T) { } else if !errors.HasType(err, (*kvpb.LockConflictError)(nil)) { t.Errorf("test %d: expected lock conflict error; got %v", i, err) } - } else if err != nil || valueRes.Value == nil || !bytes.Equal(test.expValue.RawBytes, valueRes.Value.RawBytes) { - t.Errorf("test %d: expected value %q, err nil; got %+v, %v", i, test.expValue.RawBytes, valueRes.Value, err) + } else if err != nil || !valueRes.Value.Exists() || !bytes.Equal(test.expValue.RawBytes, valueRes.Value.Value.RawBytes) { + t.Errorf("test %d: expected value %q, err nil; got %+v, %v", i, test.expValue.RawBytes, valueRes.Value.Value, err) } }) } @@ -3253,8 +3253,8 @@ func TestMVCCGetWithDiffEpochsAndTimestamps(t *testing.T) { for i, test := range testCases { t.Run(strconv.Itoa(i), func(t *testing.T) { valueRes, err := MVCCGet(ctx, engine, testKey1, test.readTS, MVCCGetOptions{Txn: test.txn}) - if err != nil || valueRes.Value == nil || !bytes.Equal(test.expValue.RawBytes, valueRes.Value.RawBytes) { - t.Errorf("test %d: expected value %q, err nil; got %+v, %v", i, test.expValue.RawBytes, valueRes.Value, err) + if err != nil || !valueRes.Value.Exists() || !bytes.Equal(test.expValue.RawBytes, valueRes.Value.Value.RawBytes) { + t.Errorf("test %d: expected value %q, err nil; got %+v, %v", i, test.expValue.RawBytes, valueRes.Value.Value, err) } }) } @@ -3380,8 +3380,8 @@ func TestMVCCGetWithPushedTimestamp(t *testing.T) { valueRes, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{Logical: 1}, MVCCGetOptions{ Txn: txn1, }) - if err != nil || valueRes.Value == nil || !bytes.Equal(valueRes.Value.RawBytes, value1.RawBytes) { - t.Errorf("expected value %q, err nil; got %+v, %v", value1.RawBytes, valueRes.Value, err) + if err != nil || !valueRes.Value.Exists() || !bytes.Equal(valueRes.Value.Value.RawBytes, value1.RawBytes) { + t.Errorf("expected value %q, err nil; got %+v, %v", value1.RawBytes, valueRes.Value.Value, err) } } @@ -3412,8 +3412,8 @@ func TestMVCCResolveWithDiffEpochs(t *testing.T) { // Verify key1 is empty, as resolution with epoch 2 would have // aborted the epoch 1 intent. valueRes, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{Logical: 1}, MVCCGetOptions{}) - if valueRes.Value != nil || err != nil { - t.Errorf("expected value nil, err nil; got %+v, %v", valueRes.Value, err) + if valueRes.Value.Exists() || err != nil { + t.Errorf("expected value nil, err nil; got %+v, %v", valueRes.Value.Value, err) } // Key2 should be committed. @@ -3421,9 +3421,9 @@ func TestMVCCResolveWithDiffEpochs(t *testing.T) { if err != nil { t.Fatal(err) } - if !bytes.Equal(value2.RawBytes, valueRes.Value.RawBytes) { + if !bytes.Equal(value2.RawBytes, valueRes.Value.Value.RawBytes) { t.Fatalf("the value %s in get result does not match the value %s in request", - value2.RawBytes, valueRes.Value.RawBytes) + value2.RawBytes, valueRes.Value.Value.RawBytes) } } @@ -3445,9 +3445,9 @@ func TestMVCCResolveWithUpdatedTimestamp(t *testing.T) { if err != nil { t.Fatal(err) } - if !bytes.Equal(value1.RawBytes, valueRes.Value.RawBytes) { + if !bytes.Equal(value1.RawBytes, valueRes.Value.Value.RawBytes) { t.Fatalf("the value %s in get result does not match the value %s in request", - value1.RawBytes, valueRes.Value.RawBytes) + value1.RawBytes, valueRes.Value.Value.RawBytes) } // Resolve with a higher commit timestamp -- this should rewrite the @@ -3460,20 +3460,20 @@ func TestMVCCResolveWithUpdatedTimestamp(t *testing.T) { } valueRes, err = MVCCGet(ctx, engine, testKey1, hlc.Timestamp{Logical: 1}, MVCCGetOptions{}) - if valueRes.Value != nil || err != nil { - t.Fatalf("expected both value and err to be nil: %+v, %v", valueRes.Value, err) + if valueRes.Value.Exists() || err != nil { + t.Fatalf("expected both value and err to be nil: %+v, %v", valueRes.Value.Value, err) } valueRes, err = MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, MVCCGetOptions{}) if err != nil { t.Error(err) } - if expTS := (hlc.Timestamp{WallTime: 1}); valueRes.Value.Timestamp != expTS { - t.Fatalf("expected timestamp %+v == %+v", valueRes.Value.Timestamp, expTS) + if expTS := (hlc.Timestamp{WallTime: 1}); valueRes.Value.Value.Timestamp != expTS { + t.Fatalf("expected timestamp %+v == %+v", valueRes.Value.Value.Timestamp, expTS) } - if !bytes.Equal(value1.RawBytes, valueRes.Value.RawBytes) { + if !bytes.Equal(value1.RawBytes, valueRes.Value.Value.RawBytes) { t.Fatalf("the value %s in get result does not match the value %s in request", - value1.RawBytes, valueRes.Value.RawBytes) + value1.RawBytes, valueRes.Value.Value.RawBytes) } } @@ -3494,9 +3494,9 @@ func TestMVCCResolveWithPushedTimestamp(t *testing.T) { if err != nil { t.Fatal(err) } - if !bytes.Equal(value1.RawBytes, valueRes.Value.RawBytes) { + if !bytes.Equal(value1.RawBytes, valueRes.Value.Value.RawBytes) { t.Fatalf("the value %s in get result does not match the value %s in request", - value1.RawBytes, valueRes.Value.RawBytes) + value1.RawBytes, valueRes.Value.Value.RawBytes) } // Resolve with a higher commit timestamp, but with still-pending transaction. @@ -3509,8 +3509,8 @@ func TestMVCCResolveWithPushedTimestamp(t *testing.T) { } valueRes, err = MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, MVCCGetOptions{}) - if valueRes.Value != nil || err == nil { - t.Fatalf("expected both value nil and err to be a LockConflictError: %+v", valueRes.Value) + if valueRes.Value.Exists() || err == nil { + t.Fatalf("expected both value nil and err to be a LockConflictError: %+v", valueRes.Value.Value) } // Can still fetch the value using txn1. @@ -3520,12 +3520,12 @@ func TestMVCCResolveWithPushedTimestamp(t *testing.T) { if err != nil { t.Error(err) } - if expTS := (hlc.Timestamp{WallTime: 1}); valueRes.Value.Timestamp != expTS { - t.Fatalf("expected timestamp %+v == %+v", valueRes.Value.Timestamp, expTS) + if expTS := (hlc.Timestamp{WallTime: 1}); valueRes.Value.Value.Timestamp != expTS { + t.Fatalf("expected timestamp %+v == %+v", valueRes.Value.Value.Timestamp, expTS) } - if !bytes.Equal(value1.RawBytes, valueRes.Value.RawBytes) { + if !bytes.Equal(value1.RawBytes, valueRes.Value.Value.RawBytes) { t.Fatalf("the value %s in get result does not match the value %s in request", - value1.RawBytes, valueRes.Value.RawBytes) + value1.RawBytes, valueRes.Value.Value.RawBytes) } } @@ -3605,9 +3605,9 @@ func TestMVCCResolveTxnRange(t *testing.T) { if err != nil { t.Fatal(err) } - if !bytes.Equal(value1.RawBytes, valueRes.Value.RawBytes) { + if !bytes.Equal(value1.RawBytes, valueRes.Value.Value.RawBytes) { t.Fatalf("the value %s in get result does not match the value %s in request", - value1.RawBytes, valueRes.Value.RawBytes) + value1.RawBytes, valueRes.Value.Value.RawBytes) } } { @@ -3615,9 +3615,9 @@ func TestMVCCResolveTxnRange(t *testing.T) { if err != nil { t.Fatal(err) } - if !bytes.Equal(value2.RawBytes, valueRes.Value.RawBytes) { + if !bytes.Equal(value2.RawBytes, valueRes.Value.Value.RawBytes) { t.Fatalf("the value %s in get result does not match the value %s in request", - value2.RawBytes, valueRes.Value.RawBytes) + value2.RawBytes, valueRes.Value.Value.RawBytes) } } { @@ -3627,9 +3627,9 @@ func TestMVCCResolveTxnRange(t *testing.T) { if err != nil { t.Fatal(err) } - if !bytes.Equal(value3.RawBytes, valueRes.Value.RawBytes) { + if !bytes.Equal(value3.RawBytes, valueRes.Value.Value.RawBytes) { t.Fatalf("the value %s in get result does not match the value %s in request", - value3.RawBytes, valueRes.Value.RawBytes) + value3.RawBytes, valueRes.Value.Value.RawBytes) } } { @@ -3637,9 +3637,9 @@ func TestMVCCResolveTxnRange(t *testing.T) { if err != nil { t.Fatal(err) } - if !bytes.Equal(value4.RawBytes, valueRes.Value.RawBytes) { + if !bytes.Equal(value4.RawBytes, valueRes.Value.Value.RawBytes) { t.Fatalf("the value %s in get result does not match the value %s in request", - value1.RawBytes, valueRes.Value.RawBytes) + value1.RawBytes, valueRes.Value.Value.RawBytes) } } } @@ -6328,7 +6328,7 @@ func TestMVCCTimeSeriesPartialMerge(t *testing.T) { if valueRes, err := MVCCGet(ctx, engine, k, hlc.Timestamp{}, MVCCGetOptions{}); err != nil { t.Fatal(err) } else { - vals[i] = valueRes.Value + vals[i] = valueRes.Value.ToPointer() } } diff --git a/pkg/testutils/testcluster/testcluster.go b/pkg/testutils/testcluster/testcluster.go index ac41325e7122..6a877aec812a 100644 --- a/pkg/testutils/testcluster/testcluster.go +++ b/pkg/testutils/testcluster/testcluster.go @@ -1869,12 +1869,12 @@ func (tc *TestCluster) ReadIntFromStores(key roachpb.Key) []int64 { clock.Now(), storage.MVCCGetOptions{}) if err != nil { log.VEventf(context.Background(), 1, "store %d: error reading from key %s: %s", s.StoreID(), key, err) - } else if valRes.Value == nil { + } else if !valRes.Value.Exists() { log.VEventf(context.Background(), 1, "store %d: missing key %s", s.StoreID(), key) } else { - results[i], err = valRes.Value.GetInt() + results[i], err = valRes.Value.Value.GetInt() if err != nil { - log.Dev.Errorf(context.Background(), "store %d: error decoding %s from key %s: %+v", s.StoreID(), valRes.Value, key, err) + log.Dev.Errorf(context.Background(), "store %d: error decoding %s from key %s: %+v", s.StoreID(), valRes.Value.Value, key, err) } } return nil