Skip to content

Commit b5e9cfd

Browse files
craig[bot]Xiang-Gustevendannaknz
committed
106738: logic: skip_on_retry works when errors are expected r=Xiang-Gu a=Xiang-Gu Previously, we have `skip_on_retry` directive for logic test which, when set, it skips the rest of test if a statement fails with TransactionRetryError. However, it won't skip if the statement is expected to fail with certain error message. This PR ensures that whenever we have a TransactionRetryError and `skip_on_retry` is set, we always skip the rest of the test, even if the stmt is expected to fail. fixes cockroachdb#104464 Release note: None 106759: streamingccl: unskip TestStreamDeleteRange r=msbutler a=stevendanna This test had previously timed out. The timeout we saw was the result of a couple of issues. When waiting for all delete ranges, our loop exit condition was very strict. We would only stop looking for rows if the number of delete ranges was exactly 3. If, however, we got 4 delete ranges, with 2 coming in a single batch, we would never hit this condition. How would that happen though? One possibility are rangefeed duplicates. Another, and what appears to have been happening in this test, is that the representation of the range deletes observed by the rangefeed consumer is slightly different depending on whether the range delete is delivered as part of a catch-up scan or as part of the rangefeeds steady state. I believe this is because the range deletes overlap but are issued at different time points. When we get them as part of the steady state, we get a trimmed version of the original event. When we get them as part of the catch-up scan, we get them broke up at the point of overlap. Fixes cockroachdb#93568 Epic: none Release note: None 106814: testutils: add helper to target transactions for retries r=lidorcarmel a=stevendanna This helper makes it a little quicker to write a test that tests whether a particular transaction is retry safe. Informs cockroachdb#106417 Epic: none Release note: none 106822: spanconfigccl: remove uses of `TODOTestTenantDisabled` r=stevendanna a=knz Informs cockroachdb#76378 . Epic: CRDB-18499 There's a mix of tests that control their tenants directly, and tests that should really work with virtualization enabled but don't. Followup issues: cockroachdb#106821 and cockroachdb#106818. Release note: None 106832: server: bark loudly if the test tenant cannot be created r=herkolategan a=knz Informs cockroachdb#76378 Informs cockroachdb#103772. Epic: CRDB-18499 For context, the automatic test tenant machinery is currently dependent on a CCL enterprise license check. (This is fundamentally not necessary - see cockroachdb#103772 - but sadly this is the way it is for now) Prior to this patch, if the user or a test selected the creation of a test tenant, but the test code forgot to import the required CCL go package, the framework would announce that "a test tenant was created" but it was actually silently failing to do so. This led to confusing investigations where a test tenant was expected, a test was appearing to succeed, but with a release build the same condition would fail. This commit enhances the situation by ensuring we have clear logging output when the test tenant cannot be created due to the missing CCL import. Release note: None Co-authored-by: Xiang Gu <[email protected]> Co-authored-by: Steven Danna <[email protected]> Co-authored-by: Raphael 'kena' Poss <[email protected]>
6 parents b30d31a + 00d78c7 + ab581e5 + a3ecaaa + 68ca3ec + e2eb31e commit b5e9cfd

File tree

17 files changed

+181
-103
lines changed

17 files changed

+181
-103
lines changed

pkg/ccl/logictestccl/testdata/logic_test/schema_change_in_txn

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1,3 +1,7 @@
1+
# Skip the rest of the test if a retry occurs. They can happen and are fine
2+
# but there's no way to encapsulate that in logictests.
3+
skip_on_retry
4+
15
# Backing up and restoring a descriptor will increment the version of the
26
# descriptor before restoring it so we cannot achieve the expected behaviour in
37
# this test.

pkg/ccl/spanconfigccl/spanconfigkvaccessorccl/kvaccessor_test.go

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -38,8 +38,7 @@ func TestCommitTSIntervals(t *testing.T) {
3838

3939
var i interceptor
4040
ts, _, _ := serverutils.StartServer(t, base.TestServerArgs{
41-
// Manually starts a tenant below. No need to start one here.
42-
DefaultTestTenant: base.TODOTestTenantDisabled,
41+
DefaultTestTenant: base.TestControlsTenantsExplicitly,
4342
Knobs: base.TestingKnobs{
4443
Server: &server.TestingKnobs{
4544
WallClock: manual,

pkg/ccl/spanconfigccl/spanconfigreconcilerccl/datadriven_test.go

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -99,9 +99,7 @@ func TestDataDriven(t *testing.T) {
9999
}
100100
tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{
101101
ServerArgs: base.TestServerArgs{
102-
// Test fails when run under the default test tenant. More
103-
// investigation is required.
104-
DefaultTestTenant: base.TODOTestTenantDisabled,
102+
DefaultTestTenant: base.TestControlsTenantsExplicitly,
105103
Knobs: base.TestingKnobs{
106104
JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), // speeds up test
107105
SpanConfig: scKnobs,

pkg/ccl/spanconfigccl/spanconfigsplitterccl/datadriven_test.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -67,8 +67,8 @@ func TestDataDriven(t *testing.T) {
6767
datadriven.Walk(t, datapathutils.TestDataPath(t), func(t *testing.T, path string) {
6868
tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{
6969
ServerArgs: base.TestServerArgs{
70-
// Fails with nil pointer dereference. Tracked with #76378.
71-
DefaultTestTenant: base.TODOTestTenantDisabled,
70+
// Fails with nil pointer dereference. Tracked with #76378 and #106818.
71+
DefaultTestTenant: base.TestDoesNotWorkWithSecondaryTenantsButWeDontKnowWhyYet(106818),
7272
Knobs: base.TestingKnobs{
7373
SpanConfig: scKnobs,
7474
},

pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/datadriven_test.go

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -109,9 +109,8 @@ func TestDataDriven(t *testing.T) {
109109
datadriven.Walk(t, datapathutils.TestDataPath(t), func(t *testing.T, path string) {
110110
tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{
111111
ServerArgs: base.TestServerArgs{
112-
// Test fails when run within a tenant. More investigation
113-
// is required. Tracked with #76378.
114-
DefaultTestTenant: base.TODOTestTenantDisabled,
112+
// Fails with nil pointer dereference. Tracked with #76378 and #106818.
113+
DefaultTestTenant: base.TestDoesNotWorkWithSecondaryTenantsButWeDontKnowWhyYet(106818),
115114
Knobs: base.TestingKnobs{
116115
GCJob: gcTestingKnobs,
117116
SpanConfig: scKnobs,

pkg/ccl/spanconfigccl/spanconfigsqlwatcherccl/sqlwatcher_test.go

Lines changed: 10 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -60,9 +60,8 @@ func TestSQLWatcherReactsToUpdates(t *testing.T) {
6060
defer dirCleanupFn()
6161
tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{
6262
ServerArgs: base.TestServerArgs{
63-
ExternalIODir: dir,
64-
// Test already runs from a tenant.
65-
DefaultTestTenant: base.TODOTestTenantDisabled,
63+
ExternalIODir: dir,
64+
DefaultTestTenant: base.TestControlsTenantsExplicitly,
6665
Knobs: base.TestingKnobs{
6766
SpanConfig: &spanconfig.TestingKnobs{
6867
ManagerDisableJobCreation: true, // disable the automatic job creation.
@@ -288,8 +287,7 @@ func TestSQLWatcherMultiple(t *testing.T) {
288287

289288
tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{
290289
ServerArgs: base.TestServerArgs{
291-
// Test already runs from a tenant.
292-
DefaultTestTenant: base.TODOTestTenantDisabled,
290+
DefaultTestTenant: base.TestDoesNotWorkWithSecondaryTenantsButWeDontKnowWhyYet(106821),
293291
Knobs: base.TestingKnobs{
294292
SpanConfig: &spanconfig.TestingKnobs{
295293
ManagerDisableJobCreation: true, // disable the automatic job creation.
@@ -307,7 +305,7 @@ func TestSQLWatcherMultiple(t *testing.T) {
307305

308306
noopCheckpointDuration := 100 * time.Millisecond
309307
sqlWatcher := spanconfigsqlwatcher.New(
310-
keys.SystemSQLCodec,
308+
ts.Codec(),
311309
ts.ClusterSettings(),
312310
ts.RangeFeedFactory().(*rangefeed.Factory),
313311
1<<20, /* 1 MB, bufferMemLimit */
@@ -420,8 +418,7 @@ func TestSQLWatcherOnEventError(t *testing.T) {
420418

421419
tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{
422420
ServerArgs: base.TestServerArgs{
423-
// Test already runs from a tenant.
424-
DefaultTestTenant: base.TODOTestTenantDisabled,
421+
DefaultTestTenant: base.TestDoesNotWorkWithSecondaryTenantsButWeDontKnowWhyYet(106821),
425422
Knobs: base.TestingKnobs{
426423
SpanConfig: &spanconfig.TestingKnobs{
427424
ManagerDisableJobCreation: true, // disable the automatic job creation.
@@ -438,7 +435,7 @@ func TestSQLWatcherOnEventError(t *testing.T) {
438435
tdb.Exec(t, `SET CLUSTER SETTING kv.closed_timestamp.target_duration = '100ms'`)
439436

440437
sqlWatcher := spanconfigsqlwatcher.New(
441-
keys.SystemSQLCodec,
438+
ts.Codec(),
442439
ts.ClusterSettings(),
443440
ts.RangeFeedFactory().(*rangefeed.Factory),
444441
1<<20, /* 1 MB, bufferMemLimit */
@@ -471,8 +468,7 @@ func TestSQLWatcherHandlerError(t *testing.T) {
471468

472469
tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{
473470
ServerArgs: base.TestServerArgs{
474-
// Test already runs from a tenant.
475-
DefaultTestTenant: base.TODOTestTenantDisabled,
471+
DefaultTestTenant: base.TestDoesNotWorkWithSecondaryTenantsButWeDontKnowWhyYet(106821),
476472
Knobs: base.TestingKnobs{
477473
SpanConfig: &spanconfig.TestingKnobs{
478474
ManagerDisableJobCreation: true, // disable the automatic job creation.
@@ -490,7 +486,7 @@ func TestSQLWatcherHandlerError(t *testing.T) {
490486

491487
noopCheckpointDuration := 100 * time.Millisecond
492488
sqlWatcher := spanconfigsqlwatcher.New(
493-
keys.SystemSQLCodec,
489+
ts.Codec(),
494490
ts.ClusterSettings(),
495491
ts.RangeFeedFactory().(*rangefeed.Factory),
496492
1<<20, /* 1 MB, bufferMemLimit */
@@ -549,8 +545,7 @@ func TestWatcherReceivesNoopCheckpoints(t *testing.T) {
549545

550546
tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{
551547
ServerArgs: base.TestServerArgs{
552-
// Test already runs from a tenant.
553-
DefaultTestTenant: base.TODOTestTenantDisabled,
548+
DefaultTestTenant: base.TestDoesNotWorkWithSecondaryTenantsButWeDontKnowWhyYet(106821),
554549
Knobs: base.TestingKnobs{
555550
SpanConfig: &spanconfig.TestingKnobs{
556551
ManagerDisableJobCreation: true, // disable the automatic job creation.
@@ -568,7 +563,7 @@ func TestWatcherReceivesNoopCheckpoints(t *testing.T) {
568563

569564
noopCheckpointDuration := 25 * time.Millisecond
570565
sqlWatcher := spanconfigsqlwatcher.New(
571-
keys.SystemSQLCodec,
566+
ts.Codec(),
572567
ts.ClusterSettings(),
573568
ts.RangeFeedFactory().(*rangefeed.Factory),
574569
1<<20, /* 1 MB, bufferMemLimit */

pkg/ccl/streamingccl/streamproducer/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -111,6 +111,7 @@ go_test(
111111
"//pkg/util/log",
112112
"//pkg/util/protoutil",
113113
"//pkg/util/randutil",
114+
"//pkg/util/span",
114115
"//pkg/util/syncutil",
115116
"//pkg/util/timeutil",
116117
"//pkg/util/uuid",

pkg/ccl/streamingccl/streamproducer/replication_stream_test.go

Lines changed: 78 additions & 44 deletions
Original file line numberDiff line numberDiff line change
@@ -13,7 +13,6 @@ import (
1313
"fmt"
1414
"net/http"
1515
"net/http/httptest"
16-
"sort"
1716
"strings"
1817
"testing"
1918
"time"
@@ -44,6 +43,7 @@ import (
4443
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
4544
"github.com/cockroachdb/cockroach/pkg/util/log"
4645
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
46+
"github.com/cockroachdb/cockroach/pkg/util/span"
4747
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
4848
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
4949
"github.com/jackc/pgx/v4"
@@ -297,6 +297,16 @@ func TestReplicationStreamInitialization(t *testing.T) {
297297
})
298298
}
299299

300+
func spansForTables(db *kv.DB, codec keys.SQLCodec, tables ...string) []roachpb.Span {
301+
spans := make([]roachpb.Span, 0, len(tables))
302+
for _, table := range tables {
303+
desc := desctestutils.TestingGetPublicTableDescriptor(
304+
db, codec, "d", table)
305+
spans = append(spans, desc.PrimaryIndexSpan(codec))
306+
}
307+
return spans
308+
}
309+
300310
func encodeSpec(
301311
t *testing.T,
302312
h *replicationtestutils.ReplicationHelper,
@@ -305,13 +315,16 @@ func encodeSpec(
305315
previousReplicatedTime hlc.Timestamp,
306316
tables ...string,
307317
) []byte {
308-
var spans []roachpb.Span
309-
for _, table := range tables {
310-
desc := desctestutils.TestingGetPublicTableDescriptor(
311-
h.SysServer.DB(), srcTenant.Codec, "d", table)
312-
spans = append(spans, desc.PrimaryIndexSpan(srcTenant.Codec))
313-
}
318+
spans := spansForTables(h.SysServer.DB(), srcTenant.Codec, tables...)
319+
return encodeSpecForSpans(t, initialScanTime, previousReplicatedTime, spans)
320+
}
314321

322+
func encodeSpecForSpans(
323+
t *testing.T,
324+
initialScanTime hlc.Timestamp,
325+
previousReplicatedTime hlc.Timestamp,
326+
spans []roachpb.Span,
327+
) []byte {
315328
spec := &streampb.StreamPartitionSpec{
316329
InitialScanTimestamp: initialScanTime,
317330
PreviousReplicatedTimestamp: previousReplicatedTime,
@@ -632,23 +645,10 @@ func TestCompleteStreamReplication(t *testing.T) {
632645
}
633646
}
634647

635-
func sortDelRanges(receivedDelRanges []kvpb.RangeFeedDeleteRange) {
636-
sort.Slice(receivedDelRanges, func(i, j int) bool {
637-
if !receivedDelRanges[i].Timestamp.Equal(receivedDelRanges[j].Timestamp) {
638-
return receivedDelRanges[i].Timestamp.Compare(receivedDelRanges[j].Timestamp) < 0
639-
}
640-
if !receivedDelRanges[i].Span.Key.Equal(receivedDelRanges[j].Span.Key) {
641-
return receivedDelRanges[i].Span.Key.Compare(receivedDelRanges[j].Span.Key) < 0
642-
}
643-
return receivedDelRanges[i].Span.EndKey.Compare(receivedDelRanges[j].Span.EndKey) < 0
644-
})
645-
}
646-
647648
func TestStreamDeleteRange(t *testing.T) {
648649
defer leaktest.AfterTest(t)()
649650
defer log.Scope(t).Close(t)
650651

651-
skip.WithIssue(t, 93568)
652652
skip.UnderStressRace(t, "disabled under stress and race")
653653

654654
h, cleanup := replicationtestutils.NewReplicationHelper(t, base.TestServerArgs{
@@ -676,15 +676,46 @@ USE d;
676676
replicationProducerSpec := h.StartReplicationStream(t, testTenantName)
677677
streamID := replicationProducerSpec.StreamID
678678
initialScanTimestamp := replicationProducerSpec.ReplicationStartTime
679+
streamResumeTimestamp := h.SysServer.Clock().Now()
679680

680681
const streamPartitionQuery = `SELECT * FROM crdb_internal.stream_partition($1, $2)`
681682
// Only subscribe to table t1 and t2, not t3.
683+
// We start the stream at a resume timestamp to avoid any initial scan.
684+
spans := spansForTables(h.SysServer.DB(), srcTenant.Codec, "t1", "t2")
685+
spec := encodeSpecForSpans(t, initialScanTimestamp, streamResumeTimestamp, spans)
686+
682687
source, feed := startReplication(ctx, t, h, makePartitionStreamDecoder,
683-
streamPartitionQuery, streamID, encodeSpec(t, h, srcTenant, initialScanTimestamp,
684-
hlc.Timestamp{}, "t1", "t2"))
688+
streamPartitionQuery, streamID, spec)
685689
defer feed.Close(ctx)
690+
codec := source.mu.codec.(*partitionStreamDecoder)
691+
692+
// We wait for the frontier to advance because we want to
693+
// ensure that we encounter the range deletes during the
694+
// rangefeed's steady state rather than the catchup scan.
695+
//
696+
// The representation of the range deletes we send is slightly
697+
// different if we encounter them during the catchup scan.
698+
//
699+
// NB: It is _still_ possible that we encounter the range
700+
// deletes during a catchup scan if we hit a rangefeed restart
701+
// during the test.
702+
f, err := span.MakeFrontier(spans...)
703+
require.NoError(t, err)
704+
for f.Frontier().IsEmpty() {
705+
t.Logf("waiting for frontier to advance to a non-zero timestamp")
706+
source.mu.Lock()
707+
source.mu.rows.Next()
708+
source.mu.codec.decode()
709+
if codec.e.Checkpoint != nil {
710+
for _, rs := range codec.e.Checkpoint.ResolvedSpans {
711+
_, err := f.Forward(rs.Span, rs.Timestamp)
712+
require.NoError(t, err)
713+
}
714+
}
715+
source.mu.Unlock()
716+
}
717+
t.Logf("frontier advanced to a %s", f.Frontier())
686718

687-
// TODO(casper): Replace with DROP TABLE once drop table uses the MVCC-compatible DelRange
688719
t1Span, t2Span, t3Span := h.TableSpan(srcTenant.Codec, "t1"),
689720
h.TableSpan(srcTenant.Codec, "t2"), h.TableSpan(srcTenant.Codec, "t3")
690721
// Range deleted is outside the subscribed spans
@@ -694,30 +725,31 @@ USE d;
694725
// Range is t1e - t2sn, emitting t2s - t2sn.
695726
require.NoError(t, h.SysServer.DB().DelRangeUsingTombstone(ctx, t1Span.EndKey, t2Span.Key.Next()))
696727

697-
// Expected DelRange spans after sorting.
698-
expectedDelRangeSpan1 := roachpb.Span{Key: t1Span.Key, EndKey: t1Span.EndKey}
699-
expectedDelRangeSpan2 := roachpb.Span{Key: t2Span.Key, EndKey: t2Span.EndKey}
700-
expectedDelRangeSpan3 := roachpb.Span{Key: t2Span.Key, EndKey: t2Span.Key.Next()}
728+
// Expected DelRange events. We store these and the received
729+
// del ranges in maps to account for possible duplicate
730+
// delivery.
731+
expectedDelRanges := make(map[string]struct{})
732+
expectedDelRanges[roachpb.Span{Key: t1Span.Key, EndKey: t1Span.EndKey}.String()] = struct{}{}
733+
expectedDelRanges[roachpb.Span{Key: t2Span.Key, EndKey: t2Span.EndKey}.String()] = struct{}{}
734+
expectedDelRanges[roachpb.Span{Key: t2Span.Key, EndKey: t2Span.Key.Next()}.String()] = struct{}{}
701735

702-
codec := source.mu.codec.(*partitionStreamDecoder)
703-
receivedDelRanges := make([]kvpb.RangeFeedDeleteRange, 0, 3)
736+
receivedDelRanges := make(map[string]struct{})
704737
for {
705738
source.mu.Lock()
706739
require.True(t, source.mu.rows.Next())
707740
source.mu.codec.decode()
708741
if codec.e.Batch != nil {
709-
receivedDelRanges = append(receivedDelRanges, codec.e.Batch.DelRanges...)
742+
for _, dr := range codec.e.Batch.DelRanges {
743+
receivedDelRanges[dr.Span.String()] = struct{}{}
744+
}
710745
}
711746
source.mu.Unlock()
712-
if len(receivedDelRanges) == 3 {
747+
if len(receivedDelRanges) >= 3 {
713748
break
714749
}
715750
}
716751

717-
sortDelRanges(receivedDelRanges)
718-
require.Equal(t, expectedDelRangeSpan1, receivedDelRanges[0].Span)
719-
require.Equal(t, expectedDelRangeSpan2, receivedDelRanges[1].Span)
720-
require.Equal(t, expectedDelRangeSpan3, receivedDelRanges[2].Span)
752+
require.Equal(t, expectedDelRanges, receivedDelRanges)
721753

722754
// Adding a SSTable that contains DeleteRange
723755
batchHLCTime := h.SysServer.Clock().Now()
@@ -734,17 +766,19 @@ USE d;
734766
// Delete range for t3s - t3e, emitting nothing.
735767
storageutils.RangeKV(string(t3Span.Key), string(t3Span.EndKey), ts, ""),
736768
})
737-
expectedDelRange1 := kvpb.RangeFeedDeleteRange{Span: t1Span, Timestamp: batchHLCTime}
738-
expectedDelRange2 := kvpb.RangeFeedDeleteRange{Span: t2Span, Timestamp: batchHLCTime}
739769
require.Equal(t, t1Span.Key, start)
740770
require.Equal(t, t3Span.EndKey, end)
741771

772+
expectedDelRanges = make(map[string]struct{})
773+
expectedDelRanges[t1Span.String()] = struct{}{}
774+
expectedDelRanges[t2Span.String()] = struct{}{}
775+
742776
// Using same batch ts so that this SST can be emitted through rangefeed.
743-
_, _, _, err := h.SysServer.DB().AddSSTableAtBatchTimestamp(ctx, start, end, data, false,
777+
_, _, _, err = h.SysServer.DB().AddSSTableAtBatchTimestamp(ctx, start, end, data, false,
744778
false, hlc.Timestamp{}, nil, false, batchHLCTime)
745779
require.NoError(t, err)
746780

747-
receivedDelRanges = receivedDelRanges[:0]
781+
receivedDelRanges = make(map[string]struct{})
748782
receivedKVs := make([]roachpb.KeyValue, 0)
749783
for {
750784
source.mu.Lock()
@@ -753,18 +787,18 @@ USE d;
753787
if codec.e.Batch != nil {
754788
require.Empty(t, codec.e.Batch.Ssts)
755789
receivedKVs = append(receivedKVs, codec.e.Batch.KeyValues...)
756-
receivedDelRanges = append(receivedDelRanges, codec.e.Batch.DelRanges...)
790+
for _, dr := range codec.e.Batch.DelRanges {
791+
receivedDelRanges[dr.Span.String()] = struct{}{}
792+
}
757793
}
758794
source.mu.Unlock()
759795

760-
if len(receivedDelRanges) == 2 && len(receivedKVs) == 1 {
796+
if len(receivedDelRanges) >= 2 && len(receivedKVs) >= 1 {
761797
break
762798
}
763799
}
764800

765-
sortDelRanges(receivedDelRanges)
766801
require.Equal(t, t2Span.Key, receivedKVs[0].Key)
767802
require.Equal(t, batchHLCTime, receivedKVs[0].Value.Timestamp)
768-
require.Equal(t, expectedDelRange1, receivedDelRanges[0])
769-
require.Equal(t, expectedDelRange2, receivedDelRanges[1])
803+
require.Equal(t, expectedDelRanges, receivedDelRanges)
770804
}

pkg/jobs/BUILD.bazel

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -114,7 +114,6 @@ go_test(
114114
"//pkg/keys",
115115
"//pkg/keyvisualizer",
116116
"//pkg/kv",
117-
"//pkg/kv/kvpb",
118117
"//pkg/kv/kvserver",
119118
"//pkg/kv/kvserver/protectedts/ptpb",
120119
"//pkg/roachpb",

0 commit comments

Comments
 (0)