Skip to content

Commit

Permalink
storage: omit unnecessary seek in MVCCDelete
Browse files Browse the repository at this point in the history
Recently, `MVCCDelete` was extended to return a boolean indicating
whether an existing, live key was deleted. This relied on passing a
`valueFn` callback to `mvccPutInternal()` which inspected the existing
value. Unfortunately, this incurs an additional, expensive seek. In the
common case when writing above the latest version, `mvccPutInternal`
has already read the existing value when synthesizing the meta record,
making this additional seek unnecessary.

This patch takes a conservative approach, by modifying `mvccPutInternal`
to return a boolean indicating whether an existing value was replaced by
the put. This is motivated by backport concerns. It does, however, incur
an additional read for all write-write conflicts, not just `MVCCDelete`,
but this cost is expected to be negligible considering the cost of a
transaction refresh.

A better solution might be to remove the need for an additional seek
with `valueFn` in the common case, which would also benefit e.g.
`MVCCConditionalPut` and `MVCCIncrement`, and avoid the read for
non-delete writes. This is left for the future.

```
name                     old time/op  new time/op  delta
MVCCPutDelete_Pebble-24  20.0µs ± 7%  15.5µs ± 3%  -22.43%  (p=0.008 n=5+5)

name                            old time/op  new time/op  delta
KV/Delete/Native/rows=1-24       153µs ± 6%   146µs ± 6%   -4.41%  (p=0.009 n=10+10)
KV/Delete/Native/rows=10-24      278µs ± 3%   227µs ± 4%  -18.22%  (p=0.000 n=10+10)
KV/Delete/Native/rows=100-24    1.28ms ± 3%  0.86ms ± 4%  -33.01%  (p=0.000 n=10+10)
KV/Delete/Native/rows=1000-24   10.8ms ± 4%   6.0ms ± 6%  -44.98%  (p=0.000 n=10+10)
KV/Delete/Native/rows=10000-24   110ms ± 4%    62ms ± 5%  -43.88%  (p=0.000 n=10+10)
KV/Delete/SQL/rows=1-24          598µs ± 2%   588µs ± 2%   -1.62%  (p=0.009 n=10+10)
KV/Delete/SQL/rows=10-24         753µs ± 3%   701µs ± 2%   -6.90%  (p=0.000 n=10+10)
KV/Delete/SQL/rows=100-24       2.34ms ± 2%  1.77ms ± 1%  -24.42%  (p=0.000 n=10+9)
KV/Delete/SQL/rows=1000-24      26.0ms ± 4%  18.5ms ± 1%  -29.08%  (p=0.000 n=10+9)
KV/Delete/SQL/rows=10000-24      248ms ± 8%   185ms ± 4%  -25.40%  (p=0.000 n=10+10)
```

Release note: None
  • Loading branch information
erikgrinaker committed Oct 25, 2022
1 parent 45d609a commit fbe8852
Showing 1 changed file with 68 additions and 44 deletions.
112 changes: 68 additions & 44 deletions pkg/storage/mvcc.go
Original file line number Diff line number Diff line change
Expand Up @@ -1367,13 +1367,12 @@ func MVCCDelete(
})
defer iter.Close()

// TODO(yuzefovich): can we avoid the actual put if foundKey is false?
valueFn := func(value optionalValue) (roachpb.Value, error) {
foundKey = value.IsPresent()
return noValue, nil
}
err = mvccPutUsingIter(ctx, rw, iter, ms, key, timestamp, localTimestamp, noValue, txn, valueFn)
return foundKey, err
buf := newPutBuffer()
defer buf.release()

// TODO(yuzefovich): can we avoid the put if the key does not exist?
return mvccPutInternal(
ctx, rw, iter, ms, key, timestamp, localTimestamp, noValue, txn, buf, nil)
}

var noValue = roachpb.Value{}
Expand All @@ -1395,11 +1394,12 @@ func mvccPutUsingIter(
valueFn func(optionalValue) (roachpb.Value, error),
) error {
buf := newPutBuffer()
defer buf.release()

err := mvccPutInternal(ctx, writer, iter, ms, key, timestamp, localTimestamp, value, txn, buf, valueFn)

// Using defer would be more convenient, but it is measurably slower.
buf.release()
// Most callers don't care about the returned exReplaced value. The ones that
// do can call mvccPutInternal directly.
_, err := mvccPutInternal(
ctx, writer, iter, ms, key, timestamp, localTimestamp, value, txn, buf, valueFn)
return err
}

Expand Down Expand Up @@ -1570,6 +1570,17 @@ func replayTransactionalWrite(
// and vice versa. valueFn can delete by returning nil. Returning
// []byte{} will write an empty value, not delete.
//
// The returned boolean indicates whether the put replaced an existing live key
// (including one written previously by the same transaction). This is evaluated
// at the transaction's read timestamp, and may not be valid when
// `WriteTooOldError` is returned having written at a higher timestamp.
// TODO(erikgrinaker): This return value exists solely because valueFn incurs an
// additional seek via maybeGetValue(). In most cases we have already read the
// value via other means, e.g. mvccGetMetadata(). We should restructure the code
// such that valueFn omits unnecessary reads in the common case and then use it
// rather than the returned boolean where needed. See also:
// https://github.com/cockroachdb/cockroach/issues/90609
//
// The given iter must surface range keys to correctly account for
// MVCC range tombstones in MVCC stats.
//
Expand Down Expand Up @@ -1610,40 +1621,40 @@ func mvccPutInternal(
txn *roachpb.Transaction,
buf *putBuffer,
valueFn func(optionalValue) (roachpb.Value, error),
) error {
) (bool, error) {
if len(key) == 0 {
return emptyKeyError()
return false, emptyKeyError()
}
if timestamp.WallTime < 0 {
return errors.Errorf("cannot write to %q at timestamp %s", key, timestamp)
return false, errors.Errorf("cannot write to %q at timestamp %s", key, timestamp)
}
if !value.Timestamp.IsEmpty() {
return errors.Errorf("cannot have timestamp set in value")
return false, errors.Errorf("cannot have timestamp set in value")
}

metaKey := MakeMVCCMetadataKey(key)
ok, origMetaKeySize, origMetaValSize, origRealKeyChanged, err :=
mvccGetMetadata(iter, metaKey, &buf.meta)
if err != nil {
return err
return false, err
}

// Verify we're not mixing inline and non-inline values.
putIsInline := timestamp.IsEmpty()
if ok && putIsInline != buf.meta.IsInline() {
return errors.Errorf("%q: put is inline=%t, but existing value is inline=%t",
return false, errors.Errorf("%q: put is inline=%t, but existing value is inline=%t",
metaKey, putIsInline, buf.meta.IsInline())
}
// Handle inline put. No IntentHistory is required for inline writes as they
// aren't allowed within transactions. MVCC range tombstones cannot exist
// across them either.
if putIsInline {
if txn != nil {
return errors.Errorf("%q: inline writes not allowed within transactions", metaKey)
return false, errors.Errorf("%q: inline writes not allowed within transactions", metaKey)
}
var metaKeySize, metaValSize int64
if value, err = maybeGetValue(ctx, iter, key, value, ok, timestamp, valueFn); err != nil {
return err
return false, err
}
if !value.IsPresent() {
metaKeySize, metaValSize, err = 0, 0, writer.ClearUnversioned(metaKey.Key)
Expand All @@ -1654,7 +1665,7 @@ func mvccPutInternal(
if ms != nil {
updateStatsForInline(ms, key, origMetaKeySize, origMetaValSize, metaKeySize, metaValSize)
}
return err
return ok && !buf.meta.Deleted, err
}

// Determine the read and write timestamps for the write. For a
Expand All @@ -1667,7 +1678,7 @@ func mvccPutInternal(
if txn != nil {
readTimestamp = txn.ReadTimestamp
if readTimestamp != timestamp {
return errors.AssertionFailedf(
return false, errors.AssertionFailedf(
"mvccPutInternal: txn's read timestamp %s does not match timestamp %s",
readTimestamp, timestamp)
}
Expand All @@ -1691,10 +1702,12 @@ func mvccPutInternal(
var maybeTooOldErr error
var prevIsValue bool
var prevValSize int64
var exReplaced bool
if ok {
// There is existing metadata for this key; ensure our write is permitted.
meta = &buf.meta
metaTimestamp := meta.Timestamp.ToTimestamp()
exReplaced = !meta.Deleted

// Handle intents. MVCC range tombstones should not require any special
// handling, since they cannot be transactional.
Expand All @@ -1703,17 +1716,17 @@ func mvccPutInternal(
if txn == nil || meta.Txn.ID != txn.ID {
// The current Put operation does not come from the same
// transaction.
return &roachpb.WriteIntentError{Intents: []roachpb.Intent{
return false, &roachpb.WriteIntentError{Intents: []roachpb.Intent{
roachpb.MakeIntent(meta.Txn, key),
}}
} else if txn.Epoch < meta.Txn.Epoch {
return errors.Errorf("put with epoch %d came after put with epoch %d in txn %s",
return false, errors.Errorf("put with epoch %d came after put with epoch %d in txn %s",
txn.Epoch, meta.Txn.Epoch, txn.ID)
} else if txn.Epoch == meta.Txn.Epoch && txn.Sequence <= meta.Txn.Sequence {
// The transaction has executed at this sequence before. This is merely a
// replay of the transactional write. Assert that all is in order and return
// early.
return replayTransactionalWrite(ctx, iter, meta, key, readTimestamp, value, txn, valueFn)
return false, replayTransactionalWrite(ctx, iter, meta, key, readTimestamp, value, txn, valueFn)
}

// We're overwriting the intent that was present at this key, before we do
Expand Down Expand Up @@ -1749,20 +1762,20 @@ func mvccPutInternal(
iter.SeekGE(oldVersionKey)
var hasPoint bool
if valid, err := iter.Valid(); err != nil {
return err
return false, err
} else if valid {
hasPoint, _ = iter.HasPointAndRange()
}
if !hasPoint || !iter.UnsafeKey().Equal(oldVersionKey) {
return errors.Errorf("existing intent value missing: %s", oldVersionKey)
return false, errors.Errorf("existing intent value missing: %s", oldVersionKey)
}

// NOTE: we use Value instead of UnsafeValue so that we can move the
// iterator below without invalidating this byte slice.
curProvValRaw = iter.Value()
curIntentVal, err := DecodeMVCCValue(curProvValRaw)
if err != nil {
return err
return false, err
}
exVal = makeOptionalValue(curIntentVal.Value)
} else {
Expand All @@ -1771,7 +1784,7 @@ func mvccPutInternal(
if prevIntentOk {
prevIntentVal, err := DecodeMVCCValue(prevIntent.Value)
if err != nil {
return err
return false, err
}
exVal = makeOptionalValue(prevIntentVal.Value)
}
Expand All @@ -1790,17 +1803,19 @@ func mvccPutInternal(
Tombstones: true,
})
if err != nil {
return err
return false, err
}
}

exReplaced = exVal.IsPresent()

// Make sure we process valueFn before clearing any earlier
// version. For example, a conditional put within same
// transaction should read previous write.
if valueFn != nil {
value, err = valueFn(exVal)
if err != nil {
return err
return false, err
}
}

Expand All @@ -1822,21 +1837,21 @@ func mvccPutInternal(
iter.SeekGE(prevKey)
valid, err := iter.Valid()
if err != nil {
return err
return false, err
} else if valid {
// If we land on a bare range key, step onto the next key. This may
// be a point key at the same key position, or a different key.
if hasPoint, hasRange := iter.HasPointAndRange(); hasRange && !hasPoint {
iter.Next()
if valid, err = iter.Valid(); err != nil {
return err
return false, err
}
}
}
if valid && iter.UnsafeKey().Key.Equal(prevKey.Key) {
prevUnsafeKey := iter.UnsafeKey()
if !prevUnsafeKey.IsValue() {
return errors.Errorf("expected an MVCC value key: %s", prevUnsafeKey)
return false, errors.Errorf("expected an MVCC value key: %s", prevUnsafeKey)
}

// We must now be on a point key, but it may be covered by an
Expand All @@ -1845,7 +1860,7 @@ func mvccPutInternal(
if !hasRange || iter.RangeKeys().Versions[0].Timestamp.Less(prevUnsafeKey.Timestamp) {
prevValLen, prevValIsTombstone, err := iter.MVCCValueLenAndIsTombstone()
if err != nil {
return err
return false, err
}
if !prevValIsTombstone {
prevIsValue = !prevValIsTombstone
Expand All @@ -1857,7 +1872,7 @@ func mvccPutInternal(
}

if err := writer.ClearMVCC(oldVersionKey); err != nil {
return err
return false, err
}
} else if writeTimestamp.Less(metaTimestamp) {
// This case occurs when we're writing a key twice within a
Expand Down Expand Up @@ -1928,12 +1943,21 @@ func mvccPutInternal(
if txn == nil {
readTimestamp = writeTimestamp
}
if value, err = maybeGetValue(ctx, iter, key, value, ok, readTimestamp, valueFn); err != nil {
return err
// Inject a function to inspect the existing value at readTimestamp and
// populate exReplaced.
exReplacedFn := func(exVal optionalValue) (roachpb.Value, error) {
exReplaced = exVal.IsPresent()
if valueFn != nil {
return valueFn(exVal)
}
return value, nil // pass through original value
}
if value, err = maybeGetValue(ctx, iter, key, value, ok, readTimestamp, exReplacedFn); err != nil {
return false, err
}
} else {
if value, err = maybeGetValue(ctx, iter, key, value, ok, readTimestamp, valueFn); err != nil {
return err
return false, err
}
}
} else {
Expand All @@ -1942,7 +1966,7 @@ func mvccPutInternal(
if valueFn != nil {
value, err = valueFn(optionalValue{exists: false})
if err != nil {
return err
return false, err
}
}
}
Expand Down Expand Up @@ -1994,7 +2018,7 @@ func mvccPutInternal(
metaKeySize, metaValSize, err = buf.putIntentMeta(
ctx, writer, metaKey, newMeta, alreadyExists)
if err != nil {
return err
return false, err
}
} else {
// Per-key stats count the full-key once and MVCCVersionTimestampSize for
Expand All @@ -2012,7 +2036,7 @@ func mvccPutInternal(
// RocksDB's skiplist memtable implementation includes a fast-path for
// sequential insertion patterns.
if err := writer.PutMVCC(versionKey, versionValue); err != nil {
return err
return false, err
}

// Update MVCC stats.
Expand Down Expand Up @@ -2047,7 +2071,7 @@ func mvccPutInternal(
}
writer.LogLogicalOp(logicalOp, logicalOpDetails)

return maybeTooOldErr
return exReplaced, maybeTooOldErr
}

// MVCCIncrement fetches the value for key, and assuming the value is
Expand Down Expand Up @@ -2839,7 +2863,7 @@ func MVCCDeleteRange(

var keys []roachpb.Key
for i, kv := range res.KVs {
if err := mvccPutInternal(
if _, err := mvccPutInternal(
ctx, rw, iter, ms, kv.Key, timestamp, localTimestamp, noValue, txn, buf, nil,
); err != nil {
return nil, nil, 0, err
Expand Down Expand Up @@ -3021,7 +3045,7 @@ func MVCCPredicateDeleteRange(
} else {
// Use Point tombstones
for i := int64(0); i < runSize; i++ {
if err := mvccPutInternal(ctx, rw, pointTombstoneIter, ms, buf[i], endTime, localTimestamp, noValue,
if _, err := mvccPutInternal(ctx, rw, pointTombstoneIter, ms, buf[i], endTime, localTimestamp, noValue,
nil, pointTombstoneBuf, nil); err != nil {
return err
}
Expand Down

0 comments on commit fbe8852

Please sign in to comment.