Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
73496: storage: add issue for sequence number errors on replaying `DelRange` r=AlexTalks a=AlexTalks

Enhance the error message on sequence number errors when replaying a
transactional batch with a link to the possible cause, #71236, stemming
from an issue where a `DelRange` operation finds new keys to delete upon
replay.  This also changes the error from a generic error to an
`AssertionFailed` error.

Release note: None

73578: storage: remove leftover logic related to interleaved intents r=sumeerbhola a=nvanbenschoten

This commit is a follow-up to #72536.

It addresses a few of the remaining items left over from removing the bulk of the interleaved intent logic. Specifically, it removes:
- the `PrecedingIntentState` type
- the `PrecedingIntentState` parameter in `Writer.ClearIntent`
- the `Writer.OverrideTxnDidNotUpdateMetaToFalse` method
- the `txnDidNotUpdateMetaHelper` type

The commit does not include any behavioral changes.

73591: ui: show per-node series for "Read Amplification" and "SSTables" graphs r=dhartunian a=nvanbenschoten

This commit addresses a longstanding usability issue with the Storage dashboard. Previously, the dashboard would show the average read amplification and the average sstable count across the cluster. When looking at these metrics, we are
specifically interested in the outliers, so this made little sense. As a result, a few of our runbooks (e.g. [RocksDB inverted LSM](https://cockroachlabs.atlassian.net/wiki/spaces/TS/pages/1157890147/RocksDB+inverted+LSM)) require operators to grab custom graphs with the "Per Node" option.

This commit fixes this by splitting these graphs out to show per-node series.

_Example:_

<img width="1132" alt="Screen Shot 2021-12-07 at 10 24 14 PM" src="https://user-images.githubusercontent.com/5438456/145142909-0babdd04-54a6-46d3-9d4e-002a2d375811.png">


Co-authored-by: Alex Sarkesian <[email protected]>
Co-authored-by: Nathan VanBenschoten <[email protected]>
  • Loading branch information
3 people committed Dec 10, 2021
4 parents b145aba + f609772 + e05a787 + fda96e4 commit 0b25e92
Show file tree
Hide file tree
Showing 18 changed files with 137 additions and 317 deletions.
9 changes: 2 additions & 7 deletions pkg/kv/kvserver/replica_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -6116,7 +6116,6 @@ func TestRangeStatsComputation(t *testing.T) {
stopper := stop.NewStopper()
defer stopper.Stop(context.Background())
tc.Start(t, stopper)
ctx := context.Background()

baseStats := tc.repl.GetMVCCStats()

Expand Down Expand Up @@ -6162,20 +6161,16 @@ func TestRangeStatsComputation(t *testing.T) {
}
expMS = baseStats
expMS.Add(enginepb.MVCCStats{
LiveBytes: 101,
LiveBytes: 103,
KeyBytes: 28,
ValBytes: 73,
ValBytes: 75,
IntentBytes: 23,
LiveCount: 2,
KeyCount: 2,
ValCount: 2,
IntentCount: 1,
SeparatedIntentCount: 1,
})
if !tc.engine.OverrideTxnDidNotUpdateMetaToFalse(ctx) {
expMS.LiveBytes += 2
expMS.ValBytes += 2
}
if err := verifyRangeStats(tc.engine, tc.repl.RangeID, expMS); err != nil {
t.Fatal(err)
}
Expand Down
8 changes: 2 additions & 6 deletions pkg/kv/kvserver/spanset/batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -534,12 +534,12 @@ func (s spanSetWriter) ClearUnversioned(key roachpb.Key) error {
}

func (s spanSetWriter) ClearIntent(
key roachpb.Key, state storage.PrecedingIntentState, txnDidNotUpdateMeta bool, txnUUID uuid.UUID,
key roachpb.Key, txnDidNotUpdateMeta bool, txnUUID uuid.UUID,
) error {
if err := s.checkAllowed(key); err != nil {
return err
}
return s.w.ClearIntent(key, state, txnDidNotUpdateMeta, txnUUID)
return s.w.ClearIntent(key, txnDidNotUpdateMeta, txnUUID)
}

func (s spanSetWriter) ClearEngineKey(key storage.EngineKey) error {
Expand Down Expand Up @@ -655,10 +655,6 @@ func (s spanSetWriter) LogLogicalOp(
s.w.LogLogicalOp(op, details)
}

func (s spanSetWriter) OverrideTxnDidNotUpdateMetaToFalse(ctx context.Context) bool {
return s.w.OverrideTxnDidNotUpdateMetaToFalse(ctx)
}

// ReadWriter is used outside of the spanset package internally, in ccl.
type ReadWriter struct {
spanSetReader
Expand Down
55 changes: 1 addition & 54 deletions pkg/storage/engine.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,6 @@ package storage
import (
"bytes"
"context"
"fmt"
"io"
"time"

Expand Down Expand Up @@ -489,30 +488,6 @@ type Reader interface {
PinEngineStateForIterators() error
}

// PrecedingIntentState is information needed when writing or clearing an
// intent for a transaction. It specifies the state of the intent that was
// there before this write (for the specified transaction).
type PrecedingIntentState int

const (
// ExistingIntentSeparated specifies that there is an existing intent and
// that it is separated (in the lock table key space).
ExistingIntentSeparated PrecedingIntentState = 1
// NoExistingIntent specifies that there isn't an existing intent.
NoExistingIntent PrecedingIntentState = 2
)

func (is PrecedingIntentState) String() string {
switch is {
case ExistingIntentSeparated:
return "ExistingIntentSeparated"
case NoExistingIntent:
return "NoExistingIntent"
default:
return fmt.Sprintf("PrecedingIntentState(%d)", is)
}
}

// Writer is the write interface to an engine's data.
type Writer interface {
// ApplyBatchRepr atomically applies a set of batched updates. Created by
Expand Down Expand Up @@ -546,7 +521,6 @@ type Writer interface {
// txnDidNotUpdateMeta allows for performance optimization when set to true,
// and has semantics defined in MVCCMetadata.TxnDidNotUpdateMeta (it can
// be conservatively set to false).
// REQUIRES: state is ExistingIntentSeparated.
//
// It is safe to modify the contents of the arguments after it returns.
//
Expand All @@ -555,9 +529,7 @@ type Writer interface {
// that does a <single-clear, put> pair. If there isn't a performance
// decrease, we can stop tracking txnDidNotUpdateMeta and still optimize
// ClearIntent by always doing single-clear.
ClearIntent(
key roachpb.Key, state PrecedingIntentState, txnDidNotUpdateMeta bool, txnUUID uuid.UUID,
) error
ClearIntent(key roachpb.Key, txnDidNotUpdateMeta bool, txnUUID uuid.UUID) error
// ClearEngineKey removes the item from the db with the given EngineKey.
// Note that clear actually removes entries from the storage engine. This is
// a general-purpose and low-level method that should be used sparingly,
Expand Down Expand Up @@ -682,31 +654,6 @@ type Writer interface {
//
// It is safe to modify the contents of the arguments after it returns.
SingleClearEngineKey(key EngineKey) error

// OverrideTxnDidNotUpdateMetaToFalse is a temporary method that will be removed
// for 22.1.
//
// See #69891 for details on the bug related to usage of SingleDelete in
// separated intent resolution. The following is needed for correctly
// migrating from 21.1 to 21.2.
//
// We have fixed the intent resolution code path in 21.2-beta to use
// SingleDelete more conservatively. The 21.2-GA will also likely include
// Pebble changes to make the old buggy usage of SingleDelete correct.
// However, there is a problem if someone upgrades from 21.1 to
// 21.2-beta/21.2-GA:
// 21.1 nodes will not write separated intents while they are the
// leaseholder for a range. However they can become the leaseholder for a
// range after a separated intent was written (in a mixed version cluster).
// Hence they can resolve separated intents. The logic in 21.1 for using
// SingleDelete when resolving intents is similarly buggy, and the Pebble
// code included in 21.1 will not make this buggy usage correct. The
// solution is for 21.2 nodes to never set txnDidNotUpdateMeta=true when
// writing separated intents, until the cluster version is at the version
// when the buggy code was fixed in 21.2. So 21.1 code will never use
// SingleDelete when resolving these separated intents (since the only
// separated intents being written are by 21.2 nodes).
OverrideTxnDidNotUpdateMetaToFalse(ctx context.Context) bool
}

// ReadWriter is the read/write interface to an engine's data.
Expand Down
30 changes: 10 additions & 20 deletions pkg/storage/intent_reader_writer.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/cockroachdb/errors"
)

// This file defines wrappers for Reader and Writer, and functions to do the
Expand All @@ -40,27 +39,18 @@ func wrapIntentWriter(ctx context.Context, w Writer) intentDemuxWriter {
// scratch-space to avoid allocations -- its contents will be overwritten and
// not appended to, and a possibly different buf returned.
func (idw intentDemuxWriter) ClearIntent(
key roachpb.Key,
state PrecedingIntentState,
txnDidNotUpdateMeta bool,
txnUUID uuid.UUID,
buf []byte,
key roachpb.Key, txnDidNotUpdateMeta bool, txnUUID uuid.UUID, buf []byte,
) (_ []byte, _ error) {
switch state {
case ExistingIntentSeparated:
var engineKey EngineKey
engineKey, buf = LockTableKey{
Key: key,
Strength: lock.Exclusive,
TxnUUID: txnUUID[:],
}.ToEngineKey(buf)
if txnDidNotUpdateMeta {
return buf, idw.w.SingleClearEngineKey(engineKey)
}
return buf, idw.w.ClearEngineKey(engineKey)
default:
return buf, errors.AssertionFailedf("ClearIntent: invalid preceding state %d", state)
var engineKey EngineKey
engineKey, buf = LockTableKey{
Key: key,
Strength: lock.Exclusive,
TxnUUID: txnUUID[:],
}.ToEngineKey(buf)
if txnDidNotUpdateMeta {
return buf, idw.w.SingleClearEngineKey(engineKey)
}
return buf, idw.w.ClearEngineKey(engineKey)
}

// PutIntent has the same behavior as Writer.PutIntent. buf is used as
Expand Down
15 changes: 1 addition & 14 deletions pkg/storage/intent_reader_writer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,18 +28,6 @@ import (
"github.com/cockroachdb/datadriven"
)

func readPrecedingIntentState(t *testing.T, d *datadriven.TestData) PrecedingIntentState {
var str string
d.ScanArgs(t, "preceding", &str)
switch str {
case "separated":
return ExistingIntentSeparated
case "none":
return NoExistingIntent
}
panic("unknown state")
}

func readTxnDidNotUpdateMeta(t *testing.T, d *datadriven.TestData) bool {
var txnDidNotUpdateMeta bool
d.ScanArgs(t, "txn-did-not-update-meta", &txnDidNotUpdateMeta)
Expand Down Expand Up @@ -246,9 +234,8 @@ func TestIntentDemuxWriter(t *testing.T) {
var txn int
d.ScanArgs(t, "txn", &txn)
txnUUID := uuid.FromUint128(uint128.FromInts(0, uint64(txn)))
state := readPrecedingIntentState(t, d)
txnDidNotUpdateMeta := readTxnDidNotUpdateMeta(t, d)
scratch, err = w.ClearIntent(key, state, txnDidNotUpdateMeta, txnUUID, scratch)
scratch, err = w.ClearIntent(key, txnDidNotUpdateMeta, txnUUID, scratch)
if err != nil {
return err.Error()
}
Expand Down
Loading

0 comments on commit 0b25e92

Please sign in to comment.