Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
95710: sql: support `*` in udf bodies r=rharding6373 a=rharding6373

This change allows `*` usage in UDF bodies. We rewrite UDF ASTs in place
to expand `*`s into the columns they reference.

Informs: #90080

Epic: CRDB-19496
Release note (sql change): Allow `*` expressions in UDFs.

96029: pkg/util/metric: option to use legacy hdrhistogram model, increase bucket fidelity r=tbg,aadityasondhi a=dhartunian

This patch reeintroduces the old HdrHistogram model to optionally be
enabled in favor of the new Prometheus model, gated behind
an environment variable called `COCKROACH_ENABLE_HDR_HISTOGRAMS`,
allowing users a means to "fall back" to the old model in the
event that the new model does not adequately serve their needs
(think of this as an "insurance policy" to protect against
this from happening again with no real mitigation - ideally,
this environment variable should never have to be used).

It also updates the pre-defined bucket boundaries used by the Prometheus
backed histograms with more buckets. This aims to improve precision,
especially for latency histograms, when calculating quantiles (the low precision
being the core cause of the issue at hand).

Note: some histograms were introduced *after* the new
Prometheus histograms were added to CockroachDB. In this
case, we use the `ForceUsePrometheus` option in the
`HistogramOptions` struct to ignore the value of the env
var, since there never was a time where these specific
histograms used the HdrHistogram model.

Release note (ops change): Histogram metrics can now optionally
use the legacy HdrHistogram model by setting the environment var
`COCKROACH_ENABLE_HDR_HISTOGRAMS=true` on CockroachDB nodes.
**Note that this is not recommended** unless users are having
difficulties with the newer Prometheus-backed histogram model.
Enabling can cause performance issues with timeseries databases
like Prometheus, as processing and storing the increased number
of buckets is taxing on both CPU and storage. Note that the
HdrHistogram model is slated for full deprecation in upcoming
releases.

Fixes: #95833

96269: sql/physicalplan: allow passing replica oracle to planning r=dt a=dt

This PR pulls out a subset of the changes in #93810, namely allowing a replica-choice oracle to be passed on a per-plan basis, but without changing which oracle is actually used in any existing planning, and only introducing the API for future users. 

This refactors span resolver and the physical planning methods to allow passing a replica-choice oracle per planning instead of using a fixed oracle (either bin-packing or closest depending on server type). The existing API that uses the default oracle is kept as is for all existing callers, while a new WithOracle variant is added for future callers who wish to pass their own oracle.

An new oracle that prefers followers over leaseholders is also added here, however it is not used at this time.

Release note: none.
Epic: none.

96349: logictest: skip another flaky test in ranges r=mgartner a=mgartner

This commit skips a flaky test that I missed in #96271.

Informs #96136

Epic: None

Release note: None

96433: storage: add string repr for FullReplicaID r=pavelkalinnikov a=tbg

Helpful for subsequent unit testing involving this type.

NB: this type is in the wrong place. Now's not the time
to move it.

Epic: none
Release note: None


96434: kvserver,kvstorage: move read/write methods for cluster versions r=pavelkalinnikov a=tbg

Same rationale as #95432 - they belong in `kvstorage` and I need
them there now as I'm working on a datadriven test.

Touches #93310.

Epic: CRDB-220
Release note: None


96445: backupccl: add 'aws-weekly' tag to restore/tpce/32tb test r=healthy-pod,renatolabs a=msbutler

Epic: none

Release note: None

Co-authored-by: rharding6373 <[email protected]>
Co-authored-by: David Hartunian <[email protected]>
Co-authored-by: Alex Barganier <[email protected]>
Co-authored-by: Rui Hu <[email protected]>
Co-authored-by: Marcus Gartner <[email protected]>
Co-authored-by: Tobias Grieger <[email protected]>
Co-authored-by: Michael Butler <[email protected]>
  • Loading branch information
8 people committed Feb 3, 2023
8 parents 18fda97 + ab4af3e + 4b32a98 + c91ee21 + 91abb91 + 1398eb8 + 877fd63 + 9e2332c commit dd97d0c
Show file tree
Hide file tree
Showing 94 changed files with 2,176 additions and 701 deletions.
2 changes: 2 additions & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -204,6 +204,7 @@ ALL_TESTS = [
"//pkg/kv/kvserver/gc:gc_test",
"//pkg/kv/kvserver/idalloc:idalloc_test",
"//pkg/kv/kvserver/intentresolver:intentresolver_test",
"//pkg/kv/kvserver/kvstorage:kvstorage_test",
"//pkg/kv/kvserver/liveness:liveness_test",
"//pkg/kv/kvserver/logstore:logstore_test",
"//pkg/kv/kvserver/loqrecovery/loqrecoverypb:loqrecoverypb_test",
Expand Down Expand Up @@ -1220,6 +1221,7 @@ GO_TARGETS = [
"//pkg/kv/kvserver/kvserverbase:kvserverbase",
"//pkg/kv/kvserver/kvserverpb:kvserverpb",
"//pkg/kv/kvserver/kvstorage:kvstorage",
"//pkg/kv/kvserver/kvstorage:kvstorage_test",
"//pkg/kv/kvserver/liveness/livenesspb:livenesspb",
"//pkg/kv/kvserver/liveness:liveness",
"//pkg/kv/kvserver/liveness:liveness_test",
Expand Down
96 changes: 76 additions & 20 deletions pkg/ccl/changefeedccl/metrics.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,14 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
)

const (
changefeedCheckpointHistMaxLatency = 30 * time.Second
changefeedBatchHistMaxLatency = 30 * time.Second
changefeedFlushHistMaxLatency = 1 * time.Minute
admitLatencyMaxValue = 1 * time.Minute
commitLatencyMaxValue = 10 * time.Minute
)

// max length for the scope name.
const maxSLIScopeNameLen = 128

Expand Down Expand Up @@ -488,16 +496,46 @@ func newAggregateMetrics(histogramWindow time.Duration) *AggMetrics {
ErrorRetries: b.Counter(metaChangefeedErrorRetries),
EmittedMessages: b.Counter(metaChangefeedEmittedMessages),
FilteredMessages: b.Counter(metaChangefeedFilteredMessages),
MessageSize: b.Histogram(metaMessageSize, histogramWindow, metric.DataSize16MBBuckets),
MessageSize: b.Histogram(metric.HistogramOptions{
Metadata: metaMessageSize,
Duration: histogramWindow,
MaxVal: 10 << 20, /* 10MB max message size */
SigFigs: 1,
Buckets: metric.DataSize16MBBuckets,
}),
EmittedBytes: b.Counter(metaChangefeedEmittedBytes),
FlushedBytes: b.Counter(metaChangefeedFlushedBytes),
Flushes: b.Counter(metaChangefeedFlushes),
SizeBasedFlushes: b.Counter(metaSizeBasedFlushes),

BatchHistNanos: b.Histogram(metaChangefeedBatchHistNanos, histogramWindow, metric.BatchProcessLatencyBuckets),
FlushHistNanos: b.Histogram(metaChangefeedFlushHistNanos, histogramWindow, metric.BatchProcessLatencyBuckets),
CommitLatency: b.Histogram(metaCommitLatency, histogramWindow, metric.BatchProcessLatencyBuckets),
AdmitLatency: b.Histogram(metaAdmitLatency, histogramWindow, metric.BatchProcessLatencyBuckets),
BatchHistNanos: b.Histogram(metric.HistogramOptions{
Metadata: metaChangefeedBatchHistNanos,
Duration: histogramWindow,
MaxVal: changefeedBatchHistMaxLatency.Nanoseconds(),
SigFigs: 1,
Buckets: metric.BatchProcessLatencyBuckets,
}),
FlushHistNanos: b.Histogram(metric.HistogramOptions{
Metadata: metaChangefeedFlushHistNanos,
Duration: histogramWindow,
MaxVal: changefeedFlushHistMaxLatency.Nanoseconds(),
SigFigs: 2,
Buckets: metric.BatchProcessLatencyBuckets,
}),
CommitLatency: b.Histogram(metric.HistogramOptions{
Metadata: metaCommitLatency,
Duration: histogramWindow,
MaxVal: commitLatencyMaxValue.Nanoseconds(),
SigFigs: 1,
Buckets: metric.BatchProcessLatencyBuckets,
}),
AdmitLatency: b.Histogram(metric.HistogramOptions{
Metadata: metaAdmitLatency,
Duration: histogramWindow,
MaxVal: admitLatencyMaxValue.Nanoseconds(),
SigFigs: 1,
Buckets: metric.BatchProcessLatencyBuckets,
}),
BackfillCount: b.Gauge(metaChangefeedBackfillCount),
BackfillPendingRanges: b.Gauge(metaChangefeedBackfillPendingRanges),
RunningCount: b.Gauge(metaChangefeedRunning),
Expand Down Expand Up @@ -572,12 +610,12 @@ type Metrics struct {
Failures *metric.Counter
ResolvedMessages *metric.Counter
QueueTimeNanos *metric.Counter
CheckpointHistNanos *metric.Histogram
CheckpointHistNanos metric.IHistogram
FrontierUpdates *metric.Counter
ThrottleMetrics cdcutils.Metrics
ReplanCount *metric.Counter
ParallelConsumerFlushNanos *metric.Histogram
ParallelConsumerConsumeNanos *metric.Histogram
ParallelConsumerFlushNanos metric.IHistogram
ParallelConsumerConsumeNanos metric.IHistogram
ParallelConsumerInFlightEvents *metric.Gauge

mu struct {
Expand All @@ -599,18 +637,36 @@ func (m *Metrics) getSLIMetrics(scope string) (*sliMetrics, error) {
// MakeMetrics makes the metrics for changefeed monitoring.
func MakeMetrics(histogramWindow time.Duration) metric.Struct {
m := &Metrics{
AggMetrics: newAggregateMetrics(histogramWindow),
KVFeedMetrics: kvevent.MakeMetrics(histogramWindow),
SchemaFeedMetrics: schemafeed.MakeMetrics(histogramWindow),
ResolvedMessages: metric.NewCounter(metaChangefeedForwardedResolvedMessages),
Failures: metric.NewCounter(metaChangefeedFailures),
QueueTimeNanos: metric.NewCounter(metaEventQueueTime),
CheckpointHistNanos: metric.NewHistogram(metaChangefeedCheckpointHistNanos, histogramWindow, metric.IOLatencyBuckets),
FrontierUpdates: metric.NewCounter(metaChangefeedFrontierUpdates),
ThrottleMetrics: cdcutils.MakeMetrics(histogramWindow),
ReplanCount: metric.NewCounter(metaChangefeedReplanCount),
ParallelConsumerFlushNanos: metric.NewHistogram(metaChangefeedEventConsumerFlushNanos, histogramWindow, metric.IOLatencyBuckets),
ParallelConsumerConsumeNanos: metric.NewHistogram(metaChangefeedEventConsumerConsumeNanos, histogramWindow, metric.IOLatencyBuckets),
AggMetrics: newAggregateMetrics(histogramWindow),
KVFeedMetrics: kvevent.MakeMetrics(histogramWindow),
SchemaFeedMetrics: schemafeed.MakeMetrics(histogramWindow),
ResolvedMessages: metric.NewCounter(metaChangefeedForwardedResolvedMessages),
Failures: metric.NewCounter(metaChangefeedFailures),
QueueTimeNanos: metric.NewCounter(metaEventQueueTime),
CheckpointHistNanos: metric.NewHistogram(metric.HistogramOptions{
Metadata: metaChangefeedCheckpointHistNanos,
Duration: histogramWindow,
MaxVal: changefeedCheckpointHistMaxLatency.Nanoseconds(),
SigFigs: 2,
Buckets: metric.IOLatencyBuckets,
}),
FrontierUpdates: metric.NewCounter(metaChangefeedFrontierUpdates),
ThrottleMetrics: cdcutils.MakeMetrics(histogramWindow),
ReplanCount: metric.NewCounter(metaChangefeedReplanCount),
// Below two metrics were never implemented using the hdr histogram. Set ForceUsePrometheus
// to true.
ParallelConsumerFlushNanos: metric.NewHistogram(metric.HistogramOptions{
Metadata: metaChangefeedEventConsumerFlushNanos,
Duration: histogramWindow,
Buckets: metric.IOLatencyBuckets,
Mode: metric.HistogramModePrometheus,
}),
ParallelConsumerConsumeNanos: metric.NewHistogram(metric.HistogramOptions{
Metadata: metaChangefeedEventConsumerConsumeNanos,
Duration: histogramWindow,
Buckets: metric.IOLatencyBuckets,
Mode: metric.HistogramModePrometheus,
}),
ParallelConsumerInFlightEvents: metric.NewGauge(metaChangefeedEventConsumerInFlightEvents),
}

Expand Down
7 changes: 7 additions & 0 deletions pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

2 changes: 1 addition & 1 deletion pkg/ccl/sqlproxyccl/connector.go
Original file line number Diff line number Diff line change
Expand Up @@ -77,7 +77,7 @@ type connector struct {

// DialTenantLatency tracks how long it takes to retrieve the address for
// a tenant and set up a tcp connection to the address.
DialTenantLatency *metric.Histogram
DialTenantLatency metric.IHistogram

// DialTenantRetries counts how often dialing a tenant is retried.
DialTenantRetries *metric.Counter
Expand Down
27 changes: 18 additions & 9 deletions pkg/ccl/sqlproxyccl/connector_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -380,9 +380,12 @@ func TestConnector_dialTenantCluster(t *testing.T) {

c := &connector{
TenantID: roachpb.MustMakeTenantID(42),
DialTenantLatency: metric.NewHistogram(
metaDialTenantLatency, time.Millisecond, metric.NetworkLatencyBuckets,
),
DialTenantLatency: metric.NewHistogram(metric.HistogramOptions{
Mode: metric.HistogramModePrometheus,
Metadata: metaDialTenantLatency,
Duration: time.Millisecond,
Buckets: metric.NetworkLatencyBuckets,
}),
DialTenantRetries: metric.NewCounter(metaDialTenantRetries),
}
dc := &testTenantDirectoryCache{}
Expand Down Expand Up @@ -460,9 +463,12 @@ func TestConnector_dialTenantCluster(t *testing.T) {
defer cancel()

c := &connector{
DialTenantLatency: metric.NewHistogram(
metaDialTenantLatency, time.Millisecond, metric.NetworkLatencyBuckets,
),
DialTenantLatency: metric.NewHistogram(metric.HistogramOptions{
Mode: metric.HistogramModePreferHdrLatency,
Metadata: metaDialTenantLatency,
Duration: time.Millisecond,
Buckets: metric.NetworkLatencyBuckets,
}),
DialTenantRetries: metric.NewCounter(metaDialTenantRetries),
}
c.testingKnobs.lookupAddr = func(ctx context.Context) (string, error) {
Expand Down Expand Up @@ -491,9 +497,12 @@ func TestConnector_dialTenantCluster(t *testing.T) {
var reportFailureFnCount int
c := &connector{
TenantID: roachpb.MustMakeTenantID(42),
DialTenantLatency: metric.NewHistogram(
metaDialTenantLatency, time.Millisecond, metric.NetworkLatencyBuckets,
),
DialTenantLatency: metric.NewHistogram(metric.HistogramOptions{
Mode: metric.HistogramModePreferHdrLatency,
Metadata: metaDialTenantLatency,
Duration: time.Millisecond,
Buckets: metric.NetworkLatencyBuckets,
}),
DialTenantRetries: metric.NewCounter(metaDialTenantRetries),
}
c.DirectoryCache = &testTenantDirectoryCache{
Expand Down
61 changes: 38 additions & 23 deletions pkg/ccl/sqlproxyccl/metrics.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,19 +23,19 @@ type metrics struct {
RoutingErrCount *metric.Counter
RefusedConnCount *metric.Counter
SuccessfulConnCount *metric.Counter
ConnectionLatency *metric.Histogram
ConnectionLatency metric.IHistogram
AuthFailedCount *metric.Counter
ExpiredClientConnCount *metric.Counter

DialTenantLatency *metric.Histogram
DialTenantLatency metric.IHistogram
DialTenantRetries *metric.Counter

ConnMigrationSuccessCount *metric.Counter
ConnMigrationErrorFatalCount *metric.Counter
ConnMigrationErrorRecoverableCount *metric.Counter
ConnMigrationAttemptedCount *metric.Counter
ConnMigrationAttemptedLatency *metric.Histogram
ConnMigrationTransferResponseMessageSize *metric.Histogram
ConnMigrationAttemptedLatency metric.IHistogram
ConnMigrationTransferResponseMessageSize metric.IHistogram

QueryCancelReceivedPGWire *metric.Counter
QueryCancelReceivedHTTP *metric.Counter
Expand All @@ -49,6 +49,16 @@ func (metrics) MetricStruct() {}

var _ metric.Struct = metrics{}

const (
// maxExpectedTransferResponseMessageSize corresponds to maximum expected
// response message size for the SHOW TRANSFER STATE query. We choose 16MB
// here to match the defaultMaxReadBufferSize used for ingesting SQL
// statements in the SQL server (see pkg/sql/pgwire/pgwirebase/encoding.go).
//
// This will be used to tune sql.session_transfer.max_session_size.
maxExpectedTransferResponseMessageSize = 1 << 24 // 16MB
)

var (
metaCurConnCount = metric.Metadata{
Name: "proxy.sql.conns",
Expand Down Expand Up @@ -213,35 +223,40 @@ func makeProxyMetrics() metrics {
RoutingErrCount: metric.NewCounter(metaRoutingErrCount),
RefusedConnCount: metric.NewCounter(metaRefusedConnCount),
SuccessfulConnCount: metric.NewCounter(metaSuccessfulConnCount),
ConnectionLatency: metric.NewHistogram(
metaConnMigrationAttemptedCount,
base.DefaultHistogramWindowInterval(),
metric.NetworkLatencyBuckets,
),
ConnectionLatency: metric.NewHistogram(metric.HistogramOptions{
Mode: metric.HistogramModePreferHdrLatency,
Metadata: metaConnMigrationAttemptedCount,
Duration: base.DefaultHistogramWindowInterval(),
Buckets: metric.NetworkLatencyBuckets,
}),
AuthFailedCount: metric.NewCounter(metaAuthFailedCount),
ExpiredClientConnCount: metric.NewCounter(metaExpiredClientConnCount),
// Connector metrics.
DialTenantLatency: metric.NewHistogram(
metaDialTenantLatency,
base.DefaultHistogramWindowInterval(),
metric.NetworkLatencyBuckets,
DialTenantLatency: metric.NewHistogram(metric.HistogramOptions{
Mode: metric.HistogramModePreferHdrLatency,
Metadata: metaDialTenantLatency,
Duration: base.DefaultHistogramWindowInterval(),
Buckets: metric.NetworkLatencyBuckets},
),
DialTenantRetries: metric.NewCounter(metaDialTenantRetries),
// Connection migration metrics.
ConnMigrationSuccessCount: metric.NewCounter(metaConnMigrationSuccessCount),
ConnMigrationErrorFatalCount: metric.NewCounter(metaConnMigrationErrorFatalCount),
ConnMigrationErrorRecoverableCount: metric.NewCounter(metaConnMigrationErrorRecoverableCount),
ConnMigrationAttemptedCount: metric.NewCounter(metaConnMigrationAttemptedCount),
ConnMigrationAttemptedLatency: metric.NewHistogram(
metaConnMigrationAttemptedLatency,
base.DefaultHistogramWindowInterval(),
metric.NetworkLatencyBuckets,
),
ConnMigrationTransferResponseMessageSize: metric.NewHistogram(
metaConnMigrationTransferResponseMessageSize,
base.DefaultHistogramWindowInterval(),
metric.DataSize16MBBuckets,
),
ConnMigrationAttemptedLatency: metric.NewHistogram(metric.HistogramOptions{
Mode: metric.HistogramModePreferHdrLatency,
Metadata: metaConnMigrationAttemptedLatency,
Duration: base.DefaultHistogramWindowInterval(),
Buckets: metric.NetworkLatencyBuckets,
}),
ConnMigrationTransferResponseMessageSize: metric.NewHistogram(metric.HistogramOptions{
Metadata: metaConnMigrationTransferResponseMessageSize,
Duration: base.DefaultHistogramWindowInterval(),
Buckets: metric.DataSize16MBBuckets,
MaxVal: maxExpectedTransferResponseMessageSize,
SigFigs: 1,
}),
QueryCancelReceivedPGWire: metric.NewCounter(metaQueryCancelReceivedPGWire),
QueryCancelReceivedHTTP: metric.NewCounter(metaQueryCancelReceivedHTTP),
QueryCancelIgnored: metric.NewCounter(metaQueryCancelIgnored),
Expand Down
42 changes: 33 additions & 9 deletions pkg/ccl/streamingccl/streamingest/metrics.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,12 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/metric"
)

const (
streamingFlushHistMaxLatency = 1 * time.Minute
streamingAdmitLatencyMaxValue = 3 * time.Minute
streamingCommitLatencyMaxValue = 10 * time.Minute
)

var (
metaReplicationEventsIngested = metric.Metadata{
Name: "replication.events_ingested",
Expand Down Expand Up @@ -120,9 +126,9 @@ type Metrics struct {
Flushes *metric.Counter
JobProgressUpdates *metric.Counter
ResolvedEvents *metric.Counter
FlushHistNanos *metric.Histogram
CommitLatency *metric.Histogram
AdmitLatency *metric.Histogram
FlushHistNanos metric.IHistogram
CommitLatency metric.IHistogram
AdmitLatency metric.IHistogram
RunningCount *metric.Gauge
EarliestDataCheckpointSpan *metric.Gauge
LatestDataCheckpointSpan *metric.Gauge
Expand All @@ -143,12 +149,30 @@ func MakeMetrics(histogramWindow time.Duration) metric.Struct {
Flushes: metric.NewCounter(metaReplicationFlushes),
ResolvedEvents: metric.NewCounter(metaReplicationResolvedEventsIngested),
JobProgressUpdates: metric.NewCounter(metaJobProgressUpdates),
FlushHistNanos: metric.NewHistogram(metaReplicationFlushHistNanos,
histogramWindow, metric.BatchProcessLatencyBuckets),
CommitLatency: metric.NewHistogram(metaReplicationCommitLatency,
histogramWindow, metric.BatchProcessLatencyBuckets),
AdmitLatency: metric.NewHistogram(metaReplicationAdmitLatency,
histogramWindow, metric.BatchProcessLatencyBuckets),
FlushHistNanos: metric.NewHistogram(metric.HistogramOptions{
Metadata: metaReplicationFlushHistNanos,
Duration: histogramWindow,
Buckets: metric.BatchProcessLatencyBuckets,
MaxVal: streamingFlushHistMaxLatency.Nanoseconds(),
SigFigs: 1,
Mode: metric.HistogramModePreferHdrLatency,
}),
CommitLatency: metric.NewHistogram(metric.HistogramOptions{
Metadata: metaReplicationCommitLatency,
Duration: histogramWindow,
Buckets: metric.BatchProcessLatencyBuckets,
MaxVal: streamingCommitLatencyMaxValue.Nanoseconds(),
SigFigs: 1,
Mode: metric.HistogramModePreferHdrLatency,
}),
AdmitLatency: metric.NewHistogram(metric.HistogramOptions{
Metadata: metaReplicationAdmitLatency,
Duration: histogramWindow,
Buckets: metric.BatchProcessLatencyBuckets,
MaxVal: streamingAdmitLatencyMaxValue.Nanoseconds(),
SigFigs: 1,
Mode: metric.HistogramModePreferHdrLatency,
}),
RunningCount: metric.NewGauge(metaStreamsRunning),
EarliestDataCheckpointSpan: metric.NewGauge(metaEarliestDataCheckpointSpan),
LatestDataCheckpointSpan: metric.NewGauge(metaLatestDataCheckpointSpan),
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/restore.go
Original file line number Diff line number Diff line change
Expand Up @@ -696,7 +696,7 @@ func registerRestore(r registry.Registry) {
aost: "'2023-01-12 03:00:00'",
workload: tpceRestore{customers: 2000000}}),
timeout: 24 * time.Hour,
tags: []string{"weekly"},
tags: []string{"weekly", "aws-weekly"},
},
// TODO(msbutler): add the following tests once roachperf/grafana is hooked up and old tests are
// removed:
Expand Down
Loading

0 comments on commit dd97d0c

Please sign in to comment.