Skip to content

Commit 888ba33

Browse files
craig[bot]andy-kimballDarrylWongarulajmani
committed
155820: sql: add index_rows_read metric r=yuzefovich a=andy-kimball Add new `sql.statements.index_rows_read.count` metric that counts the number of index rows read by SQL statements. This is the same value that's collected by SQL stats for each statement and transaction, except in aggregated metric form. Epic: none Release note (sql change): Added sql.statements.index_rows_read.count metric that counts the number of index rows read by SQL statements. 155981: roachtest: recover panics in post test assertions r=golgeek a=DarrylWong We previously were recovering panics in the main test goroutine, but not post tests assertions, which could cause an entire CI run to crash. This patch adds a similar recover to post tests assertions. Fixes: none Epic: none Release note: none 156103: kv: add debugging obs for TestFlowControlSendQueueRangeFeed r=wenyihu6 a=arulajmani I haven't been able to repro this. The test has failed twice in the last couple of days, so hopefully we'll be able to see why the rangefeed isn't ending up on n3 the next time this fails. Closes #156064 Epic: none Release note: None 156106: kv: deflake TestCheckConsistencyInconsistent r=pav-kv a=arulajmani This patch skips invariant checks when iterating through range descriptors from a checkpoint to prevent spurious assertion failures. These assertion failures did not hold because checkpoints are not complete stores, so anything beyond the checkpoints span may not be considered a consistent snapshot. Closes #155680 Epic: none Release note: None Co-authored-by: Andrew Kimball <[email protected]> Co-authored-by: DarrylWong <[email protected]> Co-authored-by: Arul Ajmani <[email protected]>
5 parents 3a397eb + f29684e + 08dc6ce + 1effb29 + 7c7a170 commit 888ba33

File tree

12 files changed

+139
-4
lines changed

12 files changed

+139
-4
lines changed

docs/generated/metrics/metrics.yaml

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -8747,6 +8747,22 @@ layers:
87478747
unit: COUNT
87488748
aggregation: AVG
87498749
derivative: NON_NEGATIVE_DERIVATIVE
8750+
- name: sql.statements.rows_read.count
8751+
exported_name: sql_statements_rows_read_count
8752+
description: Number of rows read by SQL statements from primary and secondary indexes
8753+
y_axis_label: SQL Statements
8754+
type: COUNTER
8755+
unit: COUNT
8756+
aggregation: AVG
8757+
derivative: NON_NEGATIVE_DERIVATIVE
8758+
- name: sql.statements.rows_read.count.internal
8759+
exported_name: sql_statements_rows_read_count_internal
8760+
description: Number of rows read by SQL statements from primary and secondary indexes (internal queries)
8761+
y_axis_label: SQL Internal Statements
8762+
type: COUNTER
8763+
unit: COUNT
8764+
aggregation: AVG
8765+
derivative: NON_NEGATIVE_DERIVATIVE
87508766
- name: sql.stats.activity.update.latency
87518767
exported_name: sql_stats_activity_update_latency
87528768
description: The latency of updates made by the SQL activity updater job. Includes failed update attempts

pkg/cmd/roachtest/test_runner.go

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1703,6 +1703,15 @@ func (r *testRunner) postTestAssertions(
17031703
_ = r.stopper.RunAsyncTask(ctx, "test-post-assertions", func(ctx context.Context) {
17041704
defer close(postAssertCh)
17051705

1706+
defer func() {
1707+
// Unlike the main test goroutine, we _do_ want to log t.Fatal* calls here
1708+
// to make it clear that the post-test assertions failed. Otherwise, the fatal
1709+
// will be recorded as a normal test failure.
1710+
if r := recover(); r != nil {
1711+
postAssertionErr(fmt.Errorf("post-test assertion panicked: %v", r))
1712+
}
1713+
}()
1714+
17061715
// We collect all the admin health endpoints in parallel,
17071716
// and select the first one that succeeds to run the validation queries
17081717
statuses, err := c.HealthStatus(ctx, t.L(), c.CRDBNodes())

pkg/kv/kvclient/kvcoord/dist_sender_rangefeed.go

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -623,6 +623,7 @@ func newTransportForRange(
623623
return nil, err
624624
}
625625
replicas.OptimizeReplicaOrder(ctx, ds.st, ds.nodeIDGetter(), ds.healthFunc, ds.latencyFunc, ds.locality)
626+
log.VEventf(ctx, 2, "replica order for rangefeed transport is: %s", replicas)
626627
opts := SendOptions{class: defRangefeedConnClass}
627628
return ds.transportFactory(opts, replicas), nil
628629
}

pkg/kv/kvserver/consistency_queue_test.go

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -418,13 +418,14 @@ func TestCheckConsistencyInconsistent(t *testing.T) {
418418

419419
// Find the problematic range in the storage.
420420
var desc *roachpb.RangeDescriptor
421-
require.NoError(t, kvstorage.IterateRangeDescriptorsFromDisk(context.Background(), cpEng,
421+
require.NoError(t, kvstorage.IterateRangeDescriptorsFromCheckpoint(context.Background(), cpEng,
422422
func(rd roachpb.RangeDescriptor) error {
423423
if rd.RangeID == resp.Result[0].RangeID {
424424
desc = &rd
425425
}
426426
return nil
427-
}))
427+
},
428+
))
428429
require.NotNil(t, desc)
429430

430431
// Compute a checksum over the content of the problematic range.

pkg/kv/kvserver/flow_control_integration_test.go

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -3220,6 +3220,7 @@ func TestFlowControlSendQueueRangeSplitMerge(t *testing.T) {
32203220
func TestFlowControlSendQueueRangeFeed(t *testing.T) {
32213221
defer leaktest.AfterTest(t)()
32223222
defer log.Scope(t).Close(t)
3223+
testutils.SetVModule(t, "dist_sender_mux_rangefeed=2,dist_sender_rangefeed=2")
32233224

32243225
// rangeFeed will create a rangefeed suitable for testing. It will start a
32253226
// rangefeed and return a function that can be used to stop it.

pkg/kv/kvserver/kvstorage/init.go

Lines changed: 27 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -245,6 +245,28 @@ func ReadStoreIdent(ctx context.Context, eng storage.Engine) (roachpb.StoreIdent
245245
// that error.
246246
func IterateRangeDescriptorsFromDisk(
247247
ctx context.Context, reader storage.Reader, fn func(desc roachpb.RangeDescriptor) error,
248+
) error {
249+
return iterateRangeDescriptorsFromDiskHelper(ctx, reader, fn, buildutil.CrdbTestBuild)
250+
}
251+
252+
// IterateRangeDescriptorsFromCheckpoint is like IterateRangeDescriptorsFromDisk
253+
// except it is intended to be used when reading from checkpoints. Checkpoints
254+
// do not correspond to full stores, and as such, certain invariant checks do
255+
// not hold for them.
256+
func IterateRangeDescriptorsFromCheckpoint(
257+
ctx context.Context, reader storage.Reader, fn func(desc roachpb.RangeDescriptor) error,
258+
) error {
259+
return iterateRangeDescriptorsFromDiskHelper(ctx, reader, fn, false /* performInvariantChecks */)
260+
}
261+
262+
// iterateRangeDescriptorsFromDisk performs the actual heavy lifting of reading
263+
// range descriptors from the supplied storage.Reader. The caller may specify
264+
// whether or not invariant checks should be performed.
265+
func iterateRangeDescriptorsFromDiskHelper(
266+
ctx context.Context,
267+
reader storage.Reader,
268+
fn func(desc roachpb.RangeDescriptor) error,
269+
performInvariantChecks bool,
248270
) error {
249271
log.KvExec.Info(ctx, "beginning range descriptor iteration")
250272

@@ -296,8 +318,11 @@ func IterateRangeDescriptorsFromDisk(
296318
iter.SeekGE(storage.MVCCKey{Key: keys.RangeDescriptorKey(keys.MustAddr(startKey))})
297319
} else {
298320
// This case shouldn't happen in practice: we have a key that isn't
299-
// associated with any range descriptor.
300-
if buildutil.CrdbTestBuild {
321+
// associated with any range descriptor. However, when reading from
322+
// checkpoints (incomplete stores), this can happen because checkpoints
323+
// only guarantee to include specific key spans and may have arbitrary
324+
// keys outside those spans.
325+
if performInvariantChecks {
301326
return errors.AssertionFailedf("range local key %s outside of a known range", key.Key)
302327
}
303328
iter.NextKey()

pkg/roachprod/agents/opentelemetry/cockroachdb_metrics.go

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2169,6 +2169,8 @@ var cockroachdbMetrics = map[string]string{
21692169
"sql_statements_active_internal": "sql.statements.active.internal",
21702170
"sql_statements_auto_retry_count": "sql.statements.auto_retry.count",
21712171
"sql_statements_auto_retry_count_internal": "sql.statements.auto_retry.count.internal",
2172+
"sql_statements_rows_read_count": "sql.statements.rows_read.count",
2173+
"sql_statements_rows_read_count_internal": "sql.statements.rows_read.count.internal",
21722174
"sql_stats_activity_update_latency": "sql.stats.activity.update.latency",
21732175
"sql_stats_activity_update_latency_bucket": "sql.stats.activity.update.latency.bucket",
21742176
"sql_stats_activity_update_latency_count": "sql.stats.activity.update.latency.count",

pkg/sql/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -972,6 +972,7 @@ go_test(
972972
"//pkg/util/log/logconfig",
973973
"//pkg/util/log/logpb",
974974
"//pkg/util/log/logtestutils",
975+
"//pkg/util/metamorphic",
975976
"//pkg/util/metric",
976977
"//pkg/util/mon",
977978
"//pkg/util/protoutil",

pkg/sql/conn_executor.go

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -625,6 +625,7 @@ func makeMetrics(internal bool, sv *settings.Values) Metrics {
625625
FullTableOrIndexScanRejectedCount: metric.NewCounter(getMetricMeta(MetaFullTableOrIndexScanRejected, internal)),
626626
TxnRetryCount: metric.NewCounter(getMetricMeta(MetaTxnRetry, internal)),
627627
StatementRetryCount: metric.NewCounter(getMetricMeta(MetaStatementRetry, internal)),
628+
StatementRowsRead: metric.NewCounter(getMetricMeta(MetaStatementRowsRead, internal)),
628629
},
629630
StartedStatementCounters: makeStartedStatementCounters(internal),
630631
ExecutedStatementCounters: makeExecutedStatementCounters(internal),

pkg/sql/exec_util.go

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1514,6 +1514,12 @@ var (
15141514
Measurement: "SQL Statements",
15151515
Unit: metric.Unit_COUNT,
15161516
}
1517+
MetaStatementRowsRead = metric.Metadata{
1518+
Name: "sql.statements.rows_read.count",
1519+
Help: "Number of rows read by SQL statements from primary and secondary indexes",
1520+
Measurement: "SQL Statements",
1521+
Unit: metric.Unit_COUNT,
1522+
}
15171523
)
15181524

15191525
func getMetricMeta(meta metric.Metadata, internal bool) metric.Metadata {

0 commit comments

Comments
 (0)