Skip to content

Commit

Permalink
MVCCResolveWriteIntent{,Range} optimizations.
Browse files Browse the repository at this point in the history
Reduce memory allocations. Reuse an iterator between
MVCCResolveWriteIntentRange and the calls to MVCCResolveWriteIntent
instead of creating an iterator on every call to MVCCResolveWriteIntent.
  • Loading branch information
petermattis committed Feb 17, 2016
1 parent 7e5e523 commit 4444618
Show file tree
Hide file tree
Showing 2 changed files with 48 additions and 23 deletions.
70 changes: 48 additions & 22 deletions storage/engine/mvcc.go
Original file line number Diff line number Diff line change
Expand Up @@ -779,6 +779,7 @@ func mvccGetInternal(iter Iterator, metaKey MVCCKey,
type putBuffer struct {
meta MVCCMetadata
newMeta MVCCMetadata
newTxn roachpb.TxnMeta
}

var putBufferPool = sync.Pool{
Expand Down Expand Up @@ -1351,11 +1352,22 @@ func MVCCResolveWriteIntent(engine Engine, ms *MVCCStats, intent roachpb.Intent)
return util.Errorf("can't resolve range intent as point intent")
}

buf := putBufferPool.Get().(*putBuffer)

iter := engine.NewIterator(intent.Key)
defer iter.Close()

err := mvccResolveWriteIntent(engine, iter, ms, intent, buf)

// Using defer would be more convenient, but it is measurably slower.
putBufferPool.Put(buf)
return err
}

func mvccResolveWriteIntent(engine Engine, iter Iterator, ms *MVCCStats,
intent roachpb.Intent, buf *putBuffer) error {
metaKey := MakeMVCCMetadataKey(intent.Key)
meta := &MVCCMetadata{}
meta := &buf.meta
ok, origMetaKeySize, origMetaValSize, err := mvccGetMetadata(iter, metaKey, meta)
if err != nil {
return err
Expand Down Expand Up @@ -1408,16 +1420,17 @@ func MVCCResolveWriteIntent(engine Engine, ms *MVCCStats, intent roachpb.Intent)
// version value (remove old and create new with proper
// timestamp-encoded key) if timestamp changed.
if commit || pushed {
newMeta := *meta
buf.newMeta = *meta
// Set the timestamp for upcoming write (or at least the stats update).
newMeta.Timestamp = intent.Txn.Timestamp
buf.newMeta.Timestamp = intent.Txn.Timestamp

var metaKeySize, metaValSize int64
var err error
if pushed {
// Keep intent if we're pushing timestamp.
newMeta.Txn = &intent.Txn
metaKeySize, metaValSize, err = PutProto(engine, metaKey, &newMeta)
buf.newTxn = intent.Txn
buf.newMeta.Txn = &buf.newTxn
metaKeySize, metaValSize, err = PutProto(engine, metaKey, &buf.newMeta)
} else {
metaKeySize = int64(metaKey.EncodedSize())
err = engine.Clear(metaKey)
Expand All @@ -1428,7 +1441,8 @@ func MVCCResolveWriteIntent(engine Engine, ms *MVCCStats, intent roachpb.Intent)

// Update stat counters related to resolving the intent.
if ms != nil {
ms.Add(updateStatsOnResolve(intent.Key, origMetaKeySize, origMetaValSize, metaKeySize, metaValSize, *meta, newMeta, commit))
ms.Add(updateStatsOnResolve(intent.Key, origMetaKeySize, origMetaValSize,
metaKeySize, metaValSize, *meta, buf.newMeta, commit))
}

// If timestamp of value changed, need to rewrite versioned value.
Expand Down Expand Up @@ -1499,7 +1513,7 @@ func MVCCResolveWriteIntent(engine Engine, ms *MVCCStats, intent roachpb.Intent)
// Get the bytes for the next version so we have size for stat counts.
valueSize := int64(len(iter.unsafeValue()))
// Update the keyMetadata with the next version.
newMeta := &MVCCMetadata{
buf.newMeta = MVCCMetadata{
Deleted: valueSize == 0,
KeyBytes: mvccVersionTimestampSize,
ValBytes: valueSize,
Expand All @@ -1513,7 +1527,7 @@ func MVCCResolveWriteIntent(engine Engine, ms *MVCCStats, intent roachpb.Intent)
// Update stat counters with older version.
if ms != nil {
ms.Add(updateStatsOnAbort(intent.Key, origMetaKeySize, origMetaValSize,
metaKeySize, metaValSize, meta, newMeta, iterKey.Timestamp.WallTime,
metaKeySize, metaValSize, meta, &buf.newMeta, iterKey.Timestamp.WallTime,
intent.Txn.Timestamp.WallTime))
}

Expand All @@ -1529,35 +1543,47 @@ func MVCCResolveWriteIntentRange(engine Engine, ms *MVCCStats, intent roachpb.In
encEndKey := MakeMVCCMetadataKey(intent.EndKey)
nextKey := encKey

buf := putBufferPool.Get().(*putBuffer)
defer putBufferPool.Put(buf)

iter := engine.NewIterator(nil)
defer iter.Close()

var keyBuf []byte
num := int64(0)
intent.EndKey = nil

for {
kvs, err := Scan(engine, nextKey, encEndKey, 1)
if err != nil {
return num, err
}
// No more keys exists in the given range.
if len(kvs) == 0 {
iter.Seek(nextKey)
if !iter.Valid() || !iter.unsafeKey().Less(encEndKey) {
// No more keys exists in the given range.
break
}

key0 := kvs[0].Key
if !key0.IsValue() {
intent.Key = key0.Key
err = MVCCResolveWriteIntent(engine, ms, intent)
// Manually copy the underlying bytes of the unsafe key. This construction
// reuses keyBuf across iterations.
key := iter.unsafeKey()
keyBuf = append(keyBuf[:0], key.Key...)
key.Key = keyBuf

var err error
if !key.IsValue() {
intent.Key = key.Key
err = mvccResolveWriteIntent(engine, iter, ms, intent, buf)
}
if err != nil {
log.Warningf("failed to resolve intent for key %q: %v", key0.Key, err)
log.Warningf("failed to resolve intent for key %q: %v", key.Key, err)
} else {
num++
if max != 0 && max == num {
break
}
}

// In order to efficiently skip the possibly long list of
// old versions for this key, we make a non-version MVCC key.
nextKey = MakeMVCCMetadataKey(key0.Key.Next())
// nextKey is already a metadata key. We append a 0 byte which is the same
// as roachpb.Key.Next but without creating a new copy.
keyBuf = append(keyBuf, 0)
nextKey.Key = keyBuf
}

return num, nil
Expand Down
1 change: 0 additions & 1 deletion storage/engine/mvcc_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2377,7 +2377,6 @@ func TestMVCCStatsBasic(t *testing.T) {
// Resolve the deletion by aborting it.
txn.Status = roachpb.ABORTED
txn.Timestamp.Forward(ts2)
fmt.Printf("after delete: %+v\n", expMS2)
if err := MVCCResolveWriteIntent(engine, ms, roachpb.Intent{Span: roachpb.Span{Key: key}, Status: txn.Status, Txn: txn.TxnMeta}); err != nil {
t.Fatal(err)
}
Expand Down

0 comments on commit 4444618

Please sign in to comment.