From 3fbb4ae312a7bee5fcfa8c637e67c19d505353ad Mon Sep 17 00:00:00 2001 From: Steven Danna Date: Thu, 17 Oct 2024 10:19:04 +0100 Subject: [PATCH] rangefeed: remove LegacyProcessor The ScheduledProcessor has been in default use since 24.1, substantially out performs the LegacyProcessor, and is the likely place that new improvements will be implemented. Here, we remove the LegacyProcessor, retiring the associated cluster setting. This is similar to #114410, but reconstructed from scratch. Epic: none Release note (ops change): The setting kv.rangefeed.scheduler.enabled has been retired. The rangefeed scheduler is now unconditionally enabled. Co-authored-by: Erik Grinaker # Conflicts: # pkg/kv/kvserver/rangefeed/BUILD.bazel # pkg/kv/kvserver/rangefeed/bench_test.go # pkg/kv/kvserver/rangefeed/processor.go # pkg/kv/kvserver/rangefeed/processor_helpers_test.go # pkg/kv/kvserver/rangefeed/processor_test.go # pkg/kv/kvserver/replica_rangefeed.go --- .../rangefeed/rangefeed_external_test.go | 2380 ++++++++--------- pkg/kv/kvserver/BUILD.bazel | 1 - pkg/kv/kvserver/rangefeed/BUILD.bazel | 1 - pkg/kv/kvserver/rangefeed/processor.go | 669 +---- .../rangefeed/processor_helpers_test.go | 54 +- pkg/kv/kvserver/rangefeed/processor_test.go | 1011 ++++--- pkg/kv/kvserver/rangefeed/task_test.go | 10 +- pkg/kv/kvserver/rangefeed/testutil.go | 15 - pkg/kv/kvserver/replica_rangefeed.go | 47 +- pkg/kv/kvserver/replica_rangefeed_test.go | 57 - 10 files changed, 1782 insertions(+), 2463 deletions(-) delete mode 100644 pkg/kv/kvserver/rangefeed/testutil.go diff --git a/pkg/kv/kvclient/rangefeed/rangefeed_external_test.go b/pkg/kv/kvclient/rangefeed/rangefeed_external_test.go index 99cc636b4c37..d5423709eb17 100644 --- a/pkg/kv/kvclient/rangefeed/rangefeed_external_test.go +++ b/pkg/kv/kvclient/rangefeed/rangefeed_external_test.go @@ -7,7 +7,6 @@ package rangefeed_test import ( "context" - "fmt" "runtime/pprof" "sync" "sync/atomic" @@ -56,122 +55,99 @@ var ( type kvs = storageutils.KVs -type feedProcessorType struct { - useScheduler bool -} - -func (t feedProcessorType) String() string { - return fmt.Sprintf("mux/scheduler=%t", t.useScheduler) -} - -var procTypes = []feedProcessorType{ - { - useScheduler: false, - }, - { - useScheduler: true, - }, -} - // TestRangeFeedIntegration is a basic integration test demonstrating all of // the pieces working together. func TestRangeFeedIntegration(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - testutils.RunValues(t, "feed_type", procTypes, func(t *testing.T, s feedProcessorType) { - ctx := context.Background() - settings := cluster.MakeTestingClusterSettings() - // We must enable desired scheduler settings before we start cluster, - // otherwise we will trigger processor restarts later and this test can't - // handle duplicated events. - kvserver.RangeFeedUseScheduler.Override(ctx, &settings.SV, s.useScheduler) - srv, _, db := serverutils.StartServer(t, base.TestServerArgs{ - Settings: settings, - }) - defer srv.Stopper().Stop(ctx) - ts := srv.ApplicationLayer() + ctx := context.Background() + settings := cluster.MakeTestingClusterSettings() + srv, _, db := serverutils.StartServer(t, base.TestServerArgs{ + Settings: settings, + }) + defer srv.Stopper().Stop(ctx) + ts := srv.ApplicationLayer() - scratchKey := append(ts.Codec().TenantPrefix(), keys.ScratchRangeMin...) - _, _, err := srv.SplitRange(scratchKey) + scratchKey := append(ts.Codec().TenantPrefix(), keys.ScratchRangeMin...) + _, _, err := srv.SplitRange(scratchKey) + require.NoError(t, err) + scratchKey = scratchKey[:len(scratchKey):len(scratchKey)] + mkKey := func(k string) roachpb.Key { + return encoding.EncodeStringAscending(scratchKey, k) + } + // Split the range a bunch of times. + const splits = 10 + for i := 0; i < splits; i++ { + _, _, err := srv.SplitRange(mkKey(string([]byte{'a' + byte(i)}))) require.NoError(t, err) - scratchKey = scratchKey[:len(scratchKey):len(scratchKey)] - mkKey := func(k string) roachpb.Key { - return encoding.EncodeStringAscending(scratchKey, k) - } - // Split the range a bunch of times. - const splits = 10 - for i := 0; i < splits; i++ { - _, _, err := srv.SplitRange(mkKey(string([]byte{'a' + byte(i)}))) - require.NoError(t, err) - } + } - require.NoError(t, db.Put(ctx, mkKey("a"), 1)) - require.NoError(t, db.Put(ctx, mkKey("b"), 2)) - afterB := db.Clock().Now() - require.NoError(t, db.Put(ctx, mkKey("c"), 3)) + require.NoError(t, db.Put(ctx, mkKey("a"), 1)) + require.NoError(t, db.Put(ctx, mkKey("b"), 2)) + afterB := db.Clock().Now() + require.NoError(t, db.Put(ctx, mkKey("c"), 3)) - sp := roachpb.Span{ - Key: scratchKey, - EndKey: scratchKey.PrefixEnd(), - } + sp := roachpb.Span{ + Key: scratchKey, + EndKey: scratchKey.PrefixEnd(), + } - // Enable rangefeeds, otherwise the thing will retry until they are enabled. - for _, l := range []serverutils.ApplicationLayerInterface{ts, srv.SystemLayer()} { - // Whether rangefeeds are enabled is a property of both the - // storage layer and the application layer. - kvserver.RangefeedEnabled.Override(ctx, &l.ClusterSettings().SV, true) - } + // Enable rangefeeds, otherwise the thing will retry until they are enabled. + for _, l := range []serverutils.ApplicationLayerInterface{ts, srv.SystemLayer()} { + // Whether rangefeeds are enabled is a property of both the + // storage layer and the application layer. + kvserver.RangefeedEnabled.Override(ctx, &l.ClusterSettings().SV, true) + } - f, err := rangefeed.NewFactory(ts.AppStopper(), db, ts.ClusterSettings(), nil) + f, err := rangefeed.NewFactory(ts.AppStopper(), db, ts.ClusterSettings(), nil) + require.NoError(t, err) + rows := make(chan *kvpb.RangeFeedValue) + initialScanDone := make(chan struct{}) + r, err := f.RangeFeed(ctx, "test", []roachpb.Span{sp}, afterB, + func(ctx context.Context, value *kvpb.RangeFeedValue) { + select { + case rows <- value: + case <-ctx.Done(): + } + }, + rangefeed.WithDiff(true), + rangefeed.WithInitialScan(func(ctx context.Context) { + close(initialScanDone) + }), + ) + require.NoError(t, err) + defer r.Close() + { + v1 := <-rows + require.Equal(t, mkKey("a"), v1.Key) + // Ensure the initial scan contract is fulfilled when WithDiff is specified. + require.Equal(t, v1.Value.RawBytes, v1.PrevValue.RawBytes) + require.Equal(t, v1.Value.Timestamp, afterB) + require.True(t, v1.PrevValue.Timestamp.IsEmpty()) + } + { + v2 := <-rows + require.Equal(t, mkKey("b"), v2.Key) + } + <-initialScanDone + { + v3 := <-rows + require.Equal(t, mkKey("c"), v3.Key) + } + + // Write a new value for "a" and make sure it is seen. + require.NoError(t, db.Put(ctx, mkKey("a"), 4)) + { + v4 := <-rows + require.Equal(t, mkKey("a"), v4.Key) + prev, err := v4.PrevValue.GetInt() require.NoError(t, err) - rows := make(chan *kvpb.RangeFeedValue) - initialScanDone := make(chan struct{}) - r, err := f.RangeFeed(ctx, "test", []roachpb.Span{sp}, afterB, - func(ctx context.Context, value *kvpb.RangeFeedValue) { - select { - case rows <- value: - case <-ctx.Done(): - } - }, - rangefeed.WithDiff(true), - rangefeed.WithInitialScan(func(ctx context.Context) { - close(initialScanDone) - }), - ) + require.Equal(t, int64(1), prev) + updated, err := v4.Value.GetInt() require.NoError(t, err) - defer r.Close() - { - v1 := <-rows - require.Equal(t, mkKey("a"), v1.Key) - // Ensure the initial scan contract is fulfilled when WithDiff is specified. - require.Equal(t, v1.Value.RawBytes, v1.PrevValue.RawBytes) - require.Equal(t, v1.Value.Timestamp, afterB) - require.True(t, v1.PrevValue.Timestamp.IsEmpty()) - } - { - v2 := <-rows - require.Equal(t, mkKey("b"), v2.Key) - } - <-initialScanDone - { - v3 := <-rows - require.Equal(t, mkKey("c"), v3.Key) - } - - // Write a new value for "a" and make sure it is seen. - require.NoError(t, db.Put(ctx, mkKey("a"), 4)) - { - v4 := <-rows - require.Equal(t, mkKey("a"), v4.Key) - prev, err := v4.PrevValue.GetInt() - require.NoError(t, err) - require.Equal(t, int64(1), prev) - updated, err := v4.Value.GetInt() - require.NoError(t, err) - require.Equal(t, int64(4), updated) - } - }) + require.Equal(t, int64(4), updated) + } } // TestWithOnFrontierAdvance sets up a rangefeed on a span that has more than @@ -181,170 +157,33 @@ func TestWithOnFrontierAdvance(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - testutils.RunValues(t, "feed_type", procTypes, func(t *testing.T, s feedProcessorType) { - ctx := context.Background() - settings := cluster.MakeTestingClusterSettings() - // We must enable desired scheduler settings before we start cluster, - // otherwise we will trigger processor restarts later and this test can't - // handle duplicated events. - kvserver.RangeFeedUseScheduler.Override(ctx, &settings.SV, s.useScheduler) - tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ - ReplicationMode: base.ReplicationManual, - ServerArgs: base.TestServerArgs{Settings: settings}, - }) - defer tc.Stopper().Stop(ctx) - - srv := tc.Server(0) - ts := srv.ApplicationLayer() - db := ts.DB() - scratchKey := append(ts.Codec().TenantPrefix(), keys.ScratchRangeMin...) - _, _, err := srv.SplitRange(scratchKey) - require.NoError(t, err) - scratchKey = scratchKey[:len(scratchKey):len(scratchKey)] - mkKey := func(k string) roachpb.Key { - return encoding.EncodeStringAscending(scratchKey, k) - } - - sp := roachpb.Span{ - Key: scratchKey, - EndKey: scratchKey.PrefixEnd(), - } - testutils.RunTrueAndFalse(t, "quantized", func(t *testing.T, q bool) { - var quant time.Duration - if q { - quant = time.Millisecond - } - for _, l := range []serverutils.ApplicationLayerInterface{ts, srv.SystemLayer()} { - // Enable rangefeeds, otherwise the thing will retry until they are enabled. - kvserver.RangefeedEnabled.Override(ctx, &l.ClusterSettings().SV, true) - // Lower the closed timestamp target duration to speed up the test. - closedts.TargetDuration.Override(ctx, &l.ClusterSettings().SV, 100*time.Millisecond) - } - - // Split the range into two so we know the frontier has more than one span to - // track for certain. We later write to both these ranges. - _, _, err = srv.SplitRange(mkKey("b")) - require.NoError(t, err) - - f, err := rangefeed.NewFactory(ts.AppStopper(), db, ts.ClusterSettings(), nil) - require.NoError(t, err) - - // mu protects secondWriteTS. - var mu syncutil.Mutex - secondWriteFinished := false - frontierAdvancedAfterSecondWrite := false - - // Track the checkpoint TS for spans belonging to both the ranges we split - // above. This can then be used to compute the minimum timestamp for both - // these spans. We use the key we write to for the ranges below as keys for - // this map. - spanCheckpointTimestamps := make(map[string]hlc.Timestamp) - forwardCheckpointForKey := func(key string, checkpoint *kvpb.RangeFeedCheckpoint) { - ts := hlc.MinTimestamp - if prevTS, found := spanCheckpointTimestamps[key]; found { - ts = prevTS - } - ts.Forward(checkpoint.ResolvedTS) - spanCheckpointTimestamps[key] = ts - } - rows := make(chan *kvpb.RangeFeedValue) - r, err := f.RangeFeed(ctx, "test", []roachpb.Span{sp}, db.Clock().Now(), - func(ctx context.Context, value *kvpb.RangeFeedValue) { - select { - case rows <- value: - case <-ctx.Done(): - } - }, - rangefeed.WithOnCheckpoint(func(ctx context.Context, checkpoint *kvpb.RangeFeedCheckpoint) { - if checkpoint.Span.ContainsKey(mkKey("a")) { - forwardCheckpointForKey("a", checkpoint) - } - if checkpoint.Span.ContainsKey(mkKey("c")) { - forwardCheckpointForKey("c", checkpoint) - } - }), - rangefeed.WithOnFrontierAdvance(func(ctx context.Context, frontierTS hlc.Timestamp) { - minTS := hlc.MaxTimestamp - for _, ts := range spanCheckpointTimestamps { - minTS.Backward(ts) - } - if q { - minTS.Backward(hlc.Timestamp{WallTime: minTS.WallTime - (minTS.WallTime % int64(quant))}) - } - assert.Truef( - t, - frontierTS.Equal(minTS), - "expected frontier timestamp to be equal to minimum timestamp across spans %s, found %s", - minTS, - frontierTS, - ) - mu.Lock() - defer mu.Unlock() - if secondWriteFinished { - frontierAdvancedAfterSecondWrite = true - } - }), - rangefeed.WithFrontierQuantized(quant), - ) - require.NoError(t, err) - defer r.Close() - - // Write to a key on both the ranges. - require.NoError(t, db.Put(ctx, mkKey("a"), 1)) - - v := <-rows - require.Equal(t, mkKey("a"), v.Key) - - require.NoError(t, db.Put(ctx, mkKey("c"), 1)) - mu.Lock() - secondWriteFinished = true - mu.Unlock() - - v = <-rows - require.Equal(t, mkKey("c"), v.Key) - - testutils.SucceedsSoon(t, func() error { - mu.Lock() - defer mu.Unlock() - if frontierAdvancedAfterSecondWrite { - return nil - } - return errors.New("expected frontier to advance after second write") - }) - }) + ctx := context.Background() + settings := cluster.MakeTestingClusterSettings() + tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{Settings: settings}, }) -} - -// TestWithOnCheckpoint verifies that we correctly emit rangefeed checkpoint -// events. -func TestWithOnCheckpoint(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - testutils.RunValues(t, "feed_type", procTypes, func(t *testing.T, s feedProcessorType) { - ctx := context.Background() - settings := cluster.MakeTestingClusterSettings() - // We must enable desired scheduler settings before we start cluster, - // otherwise we will trigger processor restarts later and this test can't - // handle duplicated events. - kvserver.RangeFeedUseScheduler.Override(ctx, &settings.SV, s.useScheduler) - srv, _, db := serverutils.StartServer(t, base.TestServerArgs{ - Settings: settings, - }) - defer srv.Stopper().Stop(ctx) - ts := srv.ApplicationLayer() + defer tc.Stopper().Stop(ctx) - scratchKey := append(ts.Codec().TenantPrefix(), keys.ScratchRangeMin...) - _, _, err := srv.SplitRange(scratchKey) - require.NoError(t, err) - scratchKey = scratchKey[:len(scratchKey):len(scratchKey)] - mkKey := func(k string) roachpb.Key { - return encoding.EncodeStringAscending(scratchKey, k) - } + srv := tc.Server(0) + ts := srv.ApplicationLayer() + db := ts.DB() + scratchKey := append(ts.Codec().TenantPrefix(), keys.ScratchRangeMin...) + _, _, err := srv.SplitRange(scratchKey) + require.NoError(t, err) + scratchKey = scratchKey[:len(scratchKey):len(scratchKey)] + mkKey := func(k string) roachpb.Key { + return encoding.EncodeStringAscending(scratchKey, k) + } - sp := roachpb.Span{ - Key: scratchKey, - EndKey: scratchKey.PrefixEnd(), + sp := roachpb.Span{ + Key: scratchKey, + EndKey: scratchKey.PrefixEnd(), + } + testutils.RunTrueAndFalse(t, "quantized", func(t *testing.T, q bool) { + var quant time.Duration + if q { + quant = time.Millisecond } for _, l := range []serverutils.ApplicationLayerInterface{ts, srv.SystemLayer()} { // Enable rangefeeds, otherwise the thing will retry until they are enabled. @@ -353,42 +192,32 @@ func TestWithOnCheckpoint(t *testing.T) { closedts.TargetDuration.Override(ctx, &l.ClusterSettings().SV, 100*time.Millisecond) } - f, err := rangefeed.NewFactory(ts.AppStopper(), db, ts.ClusterSettings(), nil) + // Split the range into two so we know the frontier has more than one span to + // track for certain. We later write to both these ranges. + _, _, err = srv.SplitRange(mkKey("b")) require.NoError(t, err) - var mu syncutil.RWMutex - var afterWriteTS hlc.Timestamp - checkpoints := make(chan *kvpb.RangeFeedCheckpoint) - - // We need to start a goroutine that reads of the checkpoints channel, so to - // not block the rangefeed itself. - var wg sync.WaitGroup - wg.Add(1) - go func() { - defer wg.Done() - // We should expect a checkpoint event covering the key we just wrote, at a - // timestamp greater than when we wrote it. - testutils.SucceedsSoon(t, func() error { - for { - select { - case c := <-checkpoints: - mu.RLock() - writeTSUnset := afterWriteTS.IsEmpty() - mu.RUnlock() - if writeTSUnset { - return errors.New("write to key hasn't gone through yet") - } - - if afterWriteTS.LessEq(c.ResolvedTS) && c.Span.ContainsKey(mkKey("a")) { - return nil - } - default: - return errors.New("no valid checkpoints found") - } - } - }) - }() + f, err := rangefeed.NewFactory(ts.AppStopper(), db, ts.ClusterSettings(), nil) + require.NoError(t, err) + // mu protects secondWriteTS. + var mu syncutil.Mutex + secondWriteFinished := false + frontierAdvancedAfterSecondWrite := false + + // Track the checkpoint TS for spans belonging to both the ranges we split + // above. This can then be used to compute the minimum timestamp for both + // these spans. We use the key we write to for the ranges below as keys for + // this map. + spanCheckpointTimestamps := make(map[string]hlc.Timestamp) + forwardCheckpointForKey := func(key string, checkpoint *kvpb.RangeFeedCheckpoint) { + ts := hlc.MinTimestamp + if prevTS, found := spanCheckpointTimestamps[key]; found { + ts = prevTS + } + ts.Forward(checkpoint.ResolvedTS) + spanCheckpointTimestamps[key] = ts + } rows := make(chan *kvpb.RangeFeedValue) r, err := f.RangeFeed(ctx, "test", []roachpb.Span{sp}, db.Clock().Now(), func(ctx context.Context, value *kvpb.RangeFeedValue) { @@ -398,148 +227,277 @@ func TestWithOnCheckpoint(t *testing.T) { } }, rangefeed.WithOnCheckpoint(func(ctx context.Context, checkpoint *kvpb.RangeFeedCheckpoint) { - select { - case checkpoints <- checkpoint: - case <-ctx.Done(): + if checkpoint.Span.ContainsKey(mkKey("a")) { + forwardCheckpointForKey("a", checkpoint) + } + if checkpoint.Span.ContainsKey(mkKey("c")) { + forwardCheckpointForKey("c", checkpoint) } }), + rangefeed.WithOnFrontierAdvance(func(ctx context.Context, frontierTS hlc.Timestamp) { + minTS := hlc.MaxTimestamp + for _, ts := range spanCheckpointTimestamps { + minTS.Backward(ts) + } + if q { + minTS.Backward(hlc.Timestamp{WallTime: minTS.WallTime - (minTS.WallTime % int64(quant))}) + } + assert.Truef( + t, + frontierTS.Equal(minTS), + "expected frontier timestamp to be equal to minimum timestamp across spans %s, found %s", + minTS, + frontierTS, + ) + mu.Lock() + defer mu.Unlock() + if secondWriteFinished { + frontierAdvancedAfterSecondWrite = true + } + }), + rangefeed.WithFrontierQuantized(quant), ) require.NoError(t, err) defer r.Close() + // Write to a key on both the ranges. require.NoError(t, db.Put(ctx, mkKey("a"), 1)) + + v := <-rows + require.Equal(t, mkKey("a"), v.Key) + + require.NoError(t, db.Put(ctx, mkKey("c"), 1)) mu.Lock() - afterWriteTS = db.Clock().Now() + secondWriteFinished = true mu.Unlock() - { - v := <-rows - require.Equal(t, mkKey("a"), v.Key) - } - wg.Wait() + v = <-rows + require.Equal(t, mkKey("c"), v.Key) + + testutils.SucceedsSoon(t, func() error { + mu.Lock() + defer mu.Unlock() + if frontierAdvancedAfterSecondWrite { + return nil + } + return errors.New("expected frontier to advance after second write") + }) }) } -// TestRangefeedValueTimestamps tests that the rangefeed values (and previous -// values) have the kind of timestamps we expect when writing, overwriting, and -// deleting keys. -func TestRangefeedValueTimestamps(t *testing.T) { +// TestWithOnCheckpoint verifies that we correctly emit rangefeed checkpoint +// events. +func TestWithOnCheckpoint(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - testutils.RunValues(t, "feed_type", procTypes, func(t *testing.T, s feedProcessorType) { - ctx := context.Background() - settings := cluster.MakeTestingClusterSettings() - // We must enable desired scheduler settings before we start cluster, - // otherwise we will trigger processor restarts later and this test can't - // handle duplicated events. - kvserver.RangeFeedUseScheduler.Override(ctx, &settings.SV, s.useScheduler) - srv, _, db := serverutils.StartServer(t, base.TestServerArgs{ - Settings: settings, - }) - defer srv.Stopper().Stop(ctx) - ts := srv.ApplicationLayer() + ctx := context.Background() + settings := cluster.MakeTestingClusterSettings() + srv, _, db := serverutils.StartServer(t, base.TestServerArgs{ + Settings: settings, + }) + defer srv.Stopper().Stop(ctx) + ts := srv.ApplicationLayer() - scratchKey := append(ts.Codec().TenantPrefix(), keys.ScratchRangeMin...) - _, _, err := srv.SplitRange(scratchKey) - require.NoError(t, err) - scratchKey = scratchKey[:len(scratchKey):len(scratchKey)] - mkKey := func(k string) roachpb.Key { - return encoding.EncodeStringAscending(scratchKey, k) - } + scratchKey := append(ts.Codec().TenantPrefix(), keys.ScratchRangeMin...) + _, _, err := srv.SplitRange(scratchKey) + require.NoError(t, err) + scratchKey = scratchKey[:len(scratchKey):len(scratchKey)] + mkKey := func(k string) roachpb.Key { + return encoding.EncodeStringAscending(scratchKey, k) + } - sp := roachpb.Span{ - Key: scratchKey, - EndKey: scratchKey.PrefixEnd(), - } - for _, l := range []serverutils.ApplicationLayerInterface{ts, srv.SystemLayer()} { - // Enable rangefeeds, otherwise the thing will retry until they are enabled. - kvserver.RangefeedEnabled.Override(ctx, &l.ClusterSettings().SV, true) - // Lower the closed timestamp target duration to speed up the test. - closedts.TargetDuration.Override(ctx, &l.ClusterSettings().SV, 100*time.Millisecond) - } + sp := roachpb.Span{ + Key: scratchKey, + EndKey: scratchKey.PrefixEnd(), + } + for _, l := range []serverutils.ApplicationLayerInterface{ts, srv.SystemLayer()} { + // Enable rangefeeds, otherwise the thing will retry until they are enabled. + kvserver.RangefeedEnabled.Override(ctx, &l.ClusterSettings().SV, true) + // Lower the closed timestamp target duration to speed up the test. + closedts.TargetDuration.Override(ctx, &l.ClusterSettings().SV, 100*time.Millisecond) + } - f, err := rangefeed.NewFactory(ts.AppStopper(), db, ts.ClusterSettings(), nil) - require.NoError(t, err) + f, err := rangefeed.NewFactory(ts.AppStopper(), db, ts.ClusterSettings(), nil) + require.NoError(t, err) - rows := make(chan *kvpb.RangeFeedValue) - r, err := f.RangeFeed(ctx, "test", []roachpb.Span{sp}, db.Clock().Now(), - func(ctx context.Context, value *kvpb.RangeFeedValue) { + var mu syncutil.RWMutex + var afterWriteTS hlc.Timestamp + checkpoints := make(chan *kvpb.RangeFeedCheckpoint) + + // We need to start a goroutine that reads of the checkpoints channel, so to + // not block the rangefeed itself. + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + // We should expect a checkpoint event covering the key we just wrote, at a + // timestamp greater than when we wrote it. + testutils.SucceedsSoon(t, func() error { + for { select { - case rows <- value: - case <-ctx.Done(): + case c := <-checkpoints: + mu.RLock() + writeTSUnset := afterWriteTS.IsEmpty() + mu.RUnlock() + if writeTSUnset { + return errors.New("write to key hasn't gone through yet") + } + + if afterWriteTS.LessEq(c.ResolvedTS) && c.Span.ContainsKey(mkKey("a")) { + return nil + } + default: + return errors.New("no valid checkpoints found") } - }, - rangefeed.WithDiff(true), - ) - require.NoError(t, err) - defer r.Close() + } + }) + }() - mustGetInt := func(value roachpb.Value) int { - val, err := value.GetInt() - require.NoError(t, err) - return int(val) - } + rows := make(chan *kvpb.RangeFeedValue) + r, err := f.RangeFeed(ctx, "test", []roachpb.Span{sp}, db.Clock().Now(), + func(ctx context.Context, value *kvpb.RangeFeedValue) { + select { + case rows <- value: + case <-ctx.Done(): + } + }, + rangefeed.WithOnCheckpoint(func(ctx context.Context, checkpoint *kvpb.RangeFeedCheckpoint) { + select { + case checkpoints <- checkpoint: + case <-ctx.Done(): + } + }), + ) + require.NoError(t, err) + defer r.Close() - { - beforeWriteTS := db.Clock().Now() - require.NoError(t, db.Put(ctx, mkKey("a"), 1)) - afterWriteTS := db.Clock().Now() + require.NoError(t, db.Put(ctx, mkKey("a"), 1)) + mu.Lock() + afterWriteTS = db.Clock().Now() + mu.Unlock() + { + v := <-rows + require.Equal(t, mkKey("a"), v.Key) + } - v := <-rows - require.Equal(t, mustGetInt(v.Value), 1) - require.True(t, beforeWriteTS.Less(v.Value.Timestamp)) - require.True(t, v.Value.Timestamp.Less(afterWriteTS)) + wg.Wait() +} - require.False(t, v.PrevValue.IsPresent()) - } +// TestRangefeedValueTimestamps tests that the rangefeed values (and previous +// values) have the kind of timestamps we expect when writing, overwriting, and +// deleting keys. +func TestRangefeedValueTimestamps(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) - { - beforeOverwriteTS := db.Clock().Now() - require.NoError(t, db.Put(ctx, mkKey("a"), 2)) - afterOverwriteTS := db.Clock().Now() + ctx := context.Background() + settings := cluster.MakeTestingClusterSettings() + srv, _, db := serverutils.StartServer(t, base.TestServerArgs{ + Settings: settings, + }) + defer srv.Stopper().Stop(ctx) + ts := srv.ApplicationLayer() - v := <-rows - require.Equal(t, mustGetInt(v.Value), 2) - require.True(t, beforeOverwriteTS.Less(v.Value.Timestamp)) - require.True(t, v.Value.Timestamp.Less(afterOverwriteTS)) + scratchKey := append(ts.Codec().TenantPrefix(), keys.ScratchRangeMin...) + _, _, err := srv.SplitRange(scratchKey) + require.NoError(t, err) + scratchKey = scratchKey[:len(scratchKey):len(scratchKey)] + mkKey := func(k string) roachpb.Key { + return encoding.EncodeStringAscending(scratchKey, k) + } - require.True(t, v.PrevValue.IsPresent()) - require.Equal(t, mustGetInt(v.PrevValue), 1) - require.True(t, v.PrevValue.Timestamp.IsEmpty()) - } + sp := roachpb.Span{ + Key: scratchKey, + EndKey: scratchKey.PrefixEnd(), + } + for _, l := range []serverutils.ApplicationLayerInterface{ts, srv.SystemLayer()} { + // Enable rangefeeds, otherwise the thing will retry until they are enabled. + kvserver.RangefeedEnabled.Override(ctx, &l.ClusterSettings().SV, true) + // Lower the closed timestamp target duration to speed up the test. + closedts.TargetDuration.Override(ctx, &l.ClusterSettings().SV, 100*time.Millisecond) + } - { - beforeDelTS := db.Clock().Now() - _, err = db.Del(ctx, mkKey("a")) - require.NoError(t, err) - afterDelTS := db.Clock().Now() + f, err := rangefeed.NewFactory(ts.AppStopper(), db, ts.ClusterSettings(), nil) + require.NoError(t, err) - v := <-rows - require.False(t, v.Value.IsPresent()) - require.True(t, beforeDelTS.Less(v.Value.Timestamp)) - require.True(t, v.Value.Timestamp.Less(afterDelTS)) + rows := make(chan *kvpb.RangeFeedValue) + r, err := f.RangeFeed(ctx, "test", []roachpb.Span{sp}, db.Clock().Now(), + func(ctx context.Context, value *kvpb.RangeFeedValue) { + select { + case rows <- value: + case <-ctx.Done(): + } + }, + rangefeed.WithDiff(true), + ) + require.NoError(t, err) + defer r.Close() - require.True(t, v.PrevValue.IsPresent()) - require.Equal(t, mustGetInt(v.PrevValue), 2) - require.True(t, v.PrevValue.Timestamp.IsEmpty()) - } + mustGetInt := func(value roachpb.Value) int { + val, err := value.GetInt() + require.NoError(t, err) + return int(val) + } - { - beforeDelTS := db.Clock().Now() - _, err = db.Del(ctx, mkKey("a")) - require.NoError(t, err) - afterDelTS := db.Clock().Now() + { + beforeWriteTS := db.Clock().Now() + require.NoError(t, db.Put(ctx, mkKey("a"), 1)) + afterWriteTS := db.Clock().Now() - v := <-rows - require.False(t, v.Value.IsPresent()) - require.True(t, beforeDelTS.Less(v.Value.Timestamp)) - require.True(t, v.Value.Timestamp.Less(afterDelTS)) + v := <-rows + require.Equal(t, mustGetInt(v.Value), 1) + require.True(t, beforeWriteTS.Less(v.Value.Timestamp)) + require.True(t, v.Value.Timestamp.Less(afterWriteTS)) - require.False(t, v.PrevValue.IsPresent()) - require.True(t, v.PrevValue.Timestamp.IsEmpty()) - } - }) + require.False(t, v.PrevValue.IsPresent()) + } + + { + beforeOverwriteTS := db.Clock().Now() + require.NoError(t, db.Put(ctx, mkKey("a"), 2)) + afterOverwriteTS := db.Clock().Now() + + v := <-rows + require.Equal(t, mustGetInt(v.Value), 2) + require.True(t, beforeOverwriteTS.Less(v.Value.Timestamp)) + require.True(t, v.Value.Timestamp.Less(afterOverwriteTS)) + + require.True(t, v.PrevValue.IsPresent()) + require.Equal(t, mustGetInt(v.PrevValue), 1) + require.True(t, v.PrevValue.Timestamp.IsEmpty()) + } + + { + beforeDelTS := db.Clock().Now() + _, err = db.Del(ctx, mkKey("a")) + require.NoError(t, err) + afterDelTS := db.Clock().Now() + + v := <-rows + require.False(t, v.Value.IsPresent()) + require.True(t, beforeDelTS.Less(v.Value.Timestamp)) + require.True(t, v.Value.Timestamp.Less(afterDelTS)) + + require.True(t, v.PrevValue.IsPresent()) + require.Equal(t, mustGetInt(v.PrevValue), 2) + require.True(t, v.PrevValue.Timestamp.IsEmpty()) + } + + { + beforeDelTS := db.Clock().Now() + _, err = db.Del(ctx, mkKey("a")) + require.NoError(t, err) + afterDelTS := db.Clock().Now() + + v := <-rows + require.False(t, v.Value.IsPresent()) + require.True(t, beforeDelTS.Less(v.Value.Timestamp)) + require.True(t, v.Value.Timestamp.Less(afterDelTS)) + + require.False(t, v.PrevValue.IsPresent()) + require.True(t, v.PrevValue.Timestamp.IsEmpty()) + } } // TestWithOnSSTable tests that the rangefeed emits SST ingestions correctly. @@ -547,99 +505,93 @@ func TestWithOnSSTable(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - testutils.RunValues(t, "feed_type", procTypes, func(t *testing.T, s feedProcessorType) { - ctx := context.Background() - settings := cluster.MakeTestingClusterSettings() - // We must enable desired scheduler settings before we start cluster, - // otherwise we will trigger processor restarts later and this test can't - // handle duplicated events. - kvserver.RangeFeedUseScheduler.Override(ctx, &settings.SV, s.useScheduler) - srv, _, db := serverutils.StartServer(t, base.TestServerArgs{ - Settings: settings, - DefaultTestTenant: base.TestIsForStuffThatShouldWorkWithSecondaryTenantsButDoesntYet(109473), - }) - defer srv.Stopper().Stop(ctx) - tsrv := srv.ApplicationLayer() + ctx := context.Background() + settings := cluster.MakeTestingClusterSettings() + srv, _, db := serverutils.StartServer(t, base.TestServerArgs{ + Settings: settings, + DefaultTestTenant: base.TestIsForStuffThatShouldWorkWithSecondaryTenantsButDoesntYet(109473), + }) + defer srv.Stopper().Stop(ctx) + tsrv := srv.ApplicationLayer() - _, _, err := srv.SplitRange(roachpb.Key("a")) - require.NoError(t, err) + _, _, err := srv.SplitRange(roachpb.Key("a")) + require.NoError(t, err) - for _, l := range []serverutils.ApplicationLayerInterface{tsrv, srv.SystemLayer()} { - // Enable rangefeeds, otherwise the thing will retry until they are enabled. - kvserver.RangefeedEnabled.Override(ctx, &l.ClusterSettings().SV, true) - } + for _, l := range []serverutils.ApplicationLayerInterface{tsrv, srv.SystemLayer()} { + // Enable rangefeeds, otherwise the thing will retry until they are enabled. + kvserver.RangefeedEnabled.Override(ctx, &l.ClusterSettings().SV, true) + } - f, err := rangefeed.NewFactory(tsrv.AppStopper(), db, tsrv.ClusterSettings(), nil) - require.NoError(t, err) + f, err := rangefeed.NewFactory(tsrv.AppStopper(), db, tsrv.ClusterSettings(), nil) + require.NoError(t, err) - // We start the rangefeed over a narrower span than the AddSSTable (c-e vs - // a-f), to ensure the entire SST is emitted even if the registration is - // narrower. - var once sync.Once - checkpointC := make(chan struct{}) - sstC := make(chan kvcoord.RangeFeedMessage) - spans := []roachpb.Span{{Key: roachpb.Key("c"), EndKey: roachpb.Key("e")}} - r, err := f.RangeFeed(ctx, "test", spans, db.Clock().Now(), - func(ctx context.Context, value *kvpb.RangeFeedValue) {}, - rangefeed.WithOnCheckpoint(func(ctx context.Context, checkpoint *kvpb.RangeFeedCheckpoint) { - once.Do(func() { - close(checkpointC) - }) - }), - rangefeed.WithOnSSTable(func( - ctx context.Context, sst *kvpb.RangeFeedSSTable, registeredSpan roachpb.Span, - ) { - select { - case sstC <- kvcoord.RangeFeedMessage{ - RangeFeedEvent: &kvpb.RangeFeedEvent{ - SST: sst, - }, - RegisteredSpan: registeredSpan, - }: - case <-ctx.Done(): - } - }), - ) - require.NoError(t, err) - defer r.Close() + // We start the rangefeed over a narrower span than the AddSSTable (c-e vs + // a-f), to ensure the entire SST is emitted even if the registration is + // narrower. + var once sync.Once + checkpointC := make(chan struct{}) + sstC := make(chan kvcoord.RangeFeedMessage) + spans := []roachpb.Span{{Key: roachpb.Key("c"), EndKey: roachpb.Key("e")}} + r, err := f.RangeFeed(ctx, "test", spans, db.Clock().Now(), + func(ctx context.Context, value *kvpb.RangeFeedValue) {}, + rangefeed.WithOnCheckpoint(func(ctx context.Context, checkpoint *kvpb.RangeFeedCheckpoint) { + once.Do(func() { + close(checkpointC) + }) + }), + rangefeed.WithOnSSTable(func( + ctx context.Context, sst *kvpb.RangeFeedSSTable, registeredSpan roachpb.Span, + ) { + select { + case sstC <- kvcoord.RangeFeedMessage{ + RangeFeedEvent: &kvpb.RangeFeedEvent{ + SST: sst, + }, + RegisteredSpan: registeredSpan, + }: + case <-ctx.Done(): + } + }), + ) + require.NoError(t, err) + defer r.Close() - // Wait for initial checkpoint. - select { - case <-checkpointC: - case <-time.After(3 * time.Second): - require.Fail(t, "timed out waiting for checkpoint") - } + // Wait for initial checkpoint. + select { + case <-checkpointC: + case <-time.After(3 * time.Second): + require.Fail(t, "timed out waiting for checkpoint") + } - // Ingest an SST. - now := db.Clock().Now() - now.Logical = 0 - ts := int(now.WallTime) - sstKVs := kvs{ - pointKV("a", ts, "1"), - pointKV("b", ts, "2"), - pointKV("c", ts, "3"), - rangeKV("d", "e", ts, ""), - } - sst, sstStart, sstEnd := storageutils.MakeSST(t, tsrv.ClusterSettings(), sstKVs) - _, _, _, pErr := db.AddSSTableAtBatchTimestamp(ctx, sstStart, sstEnd, sst, - false /* disallowConflicts */, false /* disallowShadowing */, hlc.Timestamp{}, - nil, /* stats */ - false /* ingestAsWrites */, now) - require.Nil(t, pErr) - - // Wait for the SST event and check its contents. - var sstMessage kvcoord.RangeFeedMessage - select { - case sstMessage = <-sstC: - case <-time.After(3 * time.Second): - require.Fail(t, "timed out waiting for SST event") - } + // Ingest an SST. + now := db.Clock().Now() + now.Logical = 0 + ts := int(now.WallTime) + sstKVs := kvs{ + pointKV("a", ts, "1"), + pointKV("b", ts, "2"), + pointKV("c", ts, "3"), + rangeKV("d", "e", ts, ""), + } + sst, sstStart, sstEnd := storageutils.MakeSST(t, tsrv.ClusterSettings(), sstKVs) + _, _, _, pErr := db.AddSSTableAtBatchTimestamp(ctx, sstStart, sstEnd, sst, + false /* disallowConflicts */, false /* disallowShadowing */, hlc.Timestamp{}, + nil, /* stats */ + false /* ingestAsWrites */, now) + require.Nil(t, pErr) + + // Wait for the SST event and check its contents. + var sstMessage kvcoord.RangeFeedMessage + select { + case sstMessage = <-sstC: + case <-time.After(3 * time.Second): + require.Fail(t, "timed out waiting for SST event") + } - require.Equal(t, roachpb.Span{Key: sstStart, EndKey: sstEnd}, sstMessage.SST.Span) - require.Equal(t, now, sstMessage.SST.WriteTS) - require.Equal(t, sstKVs, storageutils.ScanSST(t, sstMessage.SST.Data)) - require.Equal(t, spans[0], sstMessage.RegisteredSpan) - }) + require.Equal(t, roachpb.Span{Key: sstStart, EndKey: sstEnd}, sstMessage.SST.Span) + require.Equal(t, now, sstMessage.SST.WriteTS) + require.Equal(t, sstKVs, storageutils.ScanSST(t, sstMessage.SST.Data)) + require.Equal(t, spans[0], sstMessage.RegisteredSpan) } // TestWithOnSSTableCatchesUpIfNotSet tests that the rangefeed runs a catchup @@ -650,110 +602,104 @@ func TestWithOnSSTableCatchesUpIfNotSet(t *testing.T) { storage.DisableMetamorphicSimpleValueEncoding(t) - testutils.RunValues(t, "feed_type", procTypes, func(t *testing.T, s feedProcessorType) { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - settings := cluster.MakeTestingClusterSettings() - // We must enable desired scheduler settings before we start cluster, - // otherwise we will trigger processor restarts later and this test can't - // handle duplicated events. - kvserver.RangeFeedUseScheduler.Override(ctx, &settings.SV, s.useScheduler) - tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ - ServerArgs: base.TestServerArgs{ - DefaultTestTenant: base.TestIsForStuffThatShouldWorkWithSecondaryTenantsButDoesntYet(109473), - Knobs: base.TestingKnobs{ - Store: &kvserver.StoreTestingKnobs{ - SmallEngineBlocks: smallEngineBlocks, - }, + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + settings := cluster.MakeTestingClusterSettings() + tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + DefaultTestTenant: base.TestIsForStuffThatShouldWorkWithSecondaryTenantsButDoesntYet(109473), + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + SmallEngineBlocks: smallEngineBlocks, }, - Settings: settings, }, - }) - defer tc.Stopper().Stop(ctx) - tsrv := tc.Server(0) - srv := tsrv.ApplicationLayer() - db := srv.DB() - - _, _, err := tc.SplitRange(roachpb.Key("a")) - require.NoError(t, err) - require.NoError(t, tc.WaitForFullReplication()) - - for _, l := range []serverutils.ApplicationLayerInterface{srv, tsrv.SystemLayer()} { - // Enable rangefeeds, otherwise the thing will retry until they are enabled. - kvserver.RangefeedEnabled.Override(ctx, &l.ClusterSettings().SV, true) - } + Settings: settings, + }, + }) + defer tc.Stopper().Stop(ctx) + tsrv := tc.Server(0) + srv := tsrv.ApplicationLayer() + db := srv.DB() - f, err := rangefeed.NewFactory(srv.AppStopper(), db, srv.ClusterSettings(), nil) - require.NoError(t, err) + _, _, err := tc.SplitRange(roachpb.Key("a")) + require.NoError(t, err) + require.NoError(t, tc.WaitForFullReplication()) - // We start the rangefeed over a narrower span than the AddSSTable (c-e vs - // a-f), to ensure only the restricted span is emitted by the catchup scan. - var once sync.Once - checkpointC := make(chan struct{}) - rowC := make(chan *kvpb.RangeFeedValue) - spans := []roachpb.Span{{Key: roachpb.Key("c"), EndKey: roachpb.Key("e")}} - r, err := f.RangeFeed(ctx, "test", spans, db.Clock().Now(), - func(ctx context.Context, value *kvpb.RangeFeedValue) { - select { - case rowC <- value: - case <-ctx.Done(): - } - }, - rangefeed.WithOnCheckpoint(func(ctx context.Context, checkpoint *kvpb.RangeFeedCheckpoint) { - once.Do(func() { - close(checkpointC) - }) - }), - ) - require.NoError(t, err) - defer r.Close() + for _, l := range []serverutils.ApplicationLayerInterface{srv, tsrv.SystemLayer()} { + // Enable rangefeeds, otherwise the thing will retry until they are enabled. + kvserver.RangefeedEnabled.Override(ctx, &l.ClusterSettings().SV, true) + } - // Wait for initial checkpoint. - select { - case <-checkpointC: - case <-time.After(3 * time.Second): - require.Fail(t, "timed out waiting for checkpoint") - } + f, err := rangefeed.NewFactory(srv.AppStopper(), db, srv.ClusterSettings(), nil) + require.NoError(t, err) - // Ingest an SST. - now := db.Clock().Now() - now.Logical = 0 - ts := int(now.WallTime) - sstKVs := kvs{ - pointKV("a", ts, "1"), - pointKV("b", ts, "2"), - pointKV("c", ts, "3"), - pointKV("d", ts, "4"), - pointKV("e", ts, "5"), - } - expectKVs := kvs{pointKV("c", ts, "3"), pointKV("d", ts, "4")} - sst, sstStart, sstEnd := storageutils.MakeSST(t, srv.ClusterSettings(), sstKVs) - _, _, _, pErr := db.AddSSTableAtBatchTimestamp(ctx, sstStart, sstEnd, sst, - false /* disallowConflicts */, false /* disallowShadowing */, hlc.Timestamp{}, - nil, /* stats */ - false /* ingestAsWrites */, now) - require.Nil(t, pErr) - - // Assert that we receive the KV pairs within the rangefeed span. - timer := time.NewTimer(3 * time.Second) - var seenKVs kvs - for len(seenKVs) < len(expectKVs) { + // We start the rangefeed over a narrower span than the AddSSTable (c-e vs + // a-f), to ensure only the restricted span is emitted by the catchup scan. + var once sync.Once + checkpointC := make(chan struct{}) + rowC := make(chan *kvpb.RangeFeedValue) + spans := []roachpb.Span{{Key: roachpb.Key("c"), EndKey: roachpb.Key("e")}} + r, err := f.RangeFeed(ctx, "test", spans, db.Clock().Now(), + func(ctx context.Context, value *kvpb.RangeFeedValue) { select { - case row := <-rowC: - seenKVs = append(seenKVs, storage.MVCCKeyValue{ - Key: storage.MVCCKey{ - Key: row.Key, - Timestamp: row.Value.Timestamp, - }, - Value: row.Value.RawBytes, - }) - case <-timer.C: - require.Fail(t, "timed out waiting for catchup scan", "saw entries: %v", seenKVs) + case rowC <- value: + case <-ctx.Done(): } + }, + rangefeed.WithOnCheckpoint(func(ctx context.Context, checkpoint *kvpb.RangeFeedCheckpoint) { + once.Do(func() { + close(checkpointC) + }) + }), + ) + require.NoError(t, err) + defer r.Close() + + // Wait for initial checkpoint. + select { + case <-checkpointC: + case <-time.After(3 * time.Second): + require.Fail(t, "timed out waiting for checkpoint") + } + + // Ingest an SST. + now := db.Clock().Now() + now.Logical = 0 + ts := int(now.WallTime) + sstKVs := kvs{ + pointKV("a", ts, "1"), + pointKV("b", ts, "2"), + pointKV("c", ts, "3"), + pointKV("d", ts, "4"), + pointKV("e", ts, "5"), + } + expectKVs := kvs{pointKV("c", ts, "3"), pointKV("d", ts, "4")} + sst, sstStart, sstEnd := storageutils.MakeSST(t, srv.ClusterSettings(), sstKVs) + _, _, _, pErr := db.AddSSTableAtBatchTimestamp(ctx, sstStart, sstEnd, sst, + false /* disallowConflicts */, false /* disallowShadowing */, hlc.Timestamp{}, + nil, /* stats */ + false /* ingestAsWrites */, now) + require.Nil(t, pErr) + + // Assert that we receive the KV pairs within the rangefeed span. + timer := time.NewTimer(3 * time.Second) + var seenKVs kvs + for len(seenKVs) < len(expectKVs) { + select { + case row := <-rowC: + seenKVs = append(seenKVs, storage.MVCCKeyValue{ + Key: storage.MVCCKey{ + Key: row.Key, + Timestamp: row.Value.Timestamp, + }, + Value: row.Value.RawBytes, + }) + case <-timer.C: + require.Fail(t, "timed out waiting for catchup scan", "saw entries: %v", seenKVs) } - require.Equal(t, expectKVs, seenKVs) - }) + } + require.Equal(t, expectKVs, seenKVs) } // TestWithOnDeleteRange tests that the rangefeed emits MVCC range tombstones. @@ -764,185 +710,179 @@ func TestWithOnDeleteRange(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - testutils.RunValues(t, "feed_type", procTypes, func(t *testing.T, s feedProcessorType) { - ctx := context.Background() - settings := cluster.MakeTestingClusterSettings() - // We must enable desired scheduler settings before we start cluster, - // otherwise we will trigger processor restarts later and this test can't - // handle duplicated events. - kvserver.RangeFeedUseScheduler.Override(ctx, &settings.SV, s.useScheduler) - tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ - ServerArgs: base.TestServerArgs{ - Settings: settings, - Knobs: base.TestingKnobs{ - Store: &kvserver.StoreTestingKnobs{ - SmallEngineBlocks: smallEngineBlocks, - }, + ctx := context.Background() + settings := cluster.MakeTestingClusterSettings() + tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + Settings: settings, + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + SmallEngineBlocks: smallEngineBlocks, }, }, - }) - defer tc.Stopper().Stop(ctx) - tsrv := tc.Server(0) - srv := tsrv.ApplicationLayer() - db := srv.DB() + }, + }) + defer tc.Stopper().Stop(ctx) + tsrv := tc.Server(0) + srv := tsrv.ApplicationLayer() + db := srv.DB() - _, _, err := tc.SplitRange(roachpb.Key("a")) - require.NoError(t, err) - require.NoError(t, tc.WaitForFullReplication()) + _, _, err := tc.SplitRange(roachpb.Key("a")) + require.NoError(t, err) + require.NoError(t, tc.WaitForFullReplication()) - for _, l := range []serverutils.ApplicationLayerInterface{srv, tsrv.SystemLayer()} { - // Enable rangefeeds, otherwise the thing will retry until they are enabled. - kvserver.RangefeedEnabled.Override(ctx, &l.ClusterSettings().SV, true) - } + for _, l := range []serverutils.ApplicationLayerInterface{srv, tsrv.SystemLayer()} { + // Enable rangefeeds, otherwise the thing will retry until they are enabled. + kvserver.RangefeedEnabled.Override(ctx, &l.ClusterSettings().SV, true) + } + + f, err := rangefeed.NewFactory(srv.AppStopper(), db, srv.ClusterSettings(), nil) + require.NoError(t, err) + + mkKey := func(s string) string { + return string(append(srv.Codec().TenantPrefix(), roachpb.Key(s)...)) + } + // We lay down a few MVCC range tombstones and points. The first range + // tombstone should not be visible, because initial scans do not emit + // tombstones, nor should the points covered by it. The second range tombstone + // should be visible, because catchup scans do emit tombstones. The range + // tombstone should be ordered after the initial point, but before the foo + // catchup point, and the previous values should respect the range tombstones. + require.NoError(t, db.Put(ctx, mkKey("covered"), "covered")) + require.NoError(t, db.Put(ctx, mkKey("foo"), "covered")) + require.NoError(t, db.DelRangeUsingTombstone(ctx, mkKey("a"), mkKey("z"))) + require.NoError(t, db.Put(ctx, mkKey("foo"), "initial")) + rangeFeedTS := db.Clock().Now() + require.NoError(t, db.Put(ctx, mkKey("covered"), "catchup")) + require.NoError(t, db.DelRangeUsingTombstone(ctx, mkKey("a"), mkKey("z"))) + require.NoError(t, db.Put(ctx, mkKey("foo"), "catchup")) + + // We start the rangefeed over a narrower span than the DeleteRanges (c-g), + // to ensure the DeleteRange event is truncated to the registration span. + var checkpointOnce sync.Once + checkpointC := make(chan struct{}) + deleteRangeC := make(chan *kvpb.RangeFeedDeleteRange) + rowC := make(chan *kvpb.RangeFeedValue) + + spans := []roachpb.Span{{ + Key: append(srv.Codec().TenantPrefix(), roachpb.Key("c")...), + EndKey: append(srv.Codec().TenantPrefix(), roachpb.Key("g")...), + }} + r, err := f.RangeFeed(ctx, "test", spans, rangeFeedTS, + func(ctx context.Context, e *kvpb.RangeFeedValue) { + select { + case rowC <- e: + case <-ctx.Done(): + } + }, + rangefeed.WithDiff(true), + rangefeed.WithInitialScan(nil), + rangefeed.WithOnCheckpoint(func(ctx context.Context, checkpoint *kvpb.RangeFeedCheckpoint) { + checkpointOnce.Do(func() { + close(checkpointC) + }) + }), + rangefeed.WithOnDeleteRange(func(ctx context.Context, e *kvpb.RangeFeedDeleteRange) { + select { + case deleteRangeC <- e: + case <-ctx.Done(): + } + }), + ) + require.NoError(t, err) + defer r.Close() - f, err := rangefeed.NewFactory(srv.AppStopper(), db, srv.ClusterSettings(), nil) + // Wait for initial scan. We should see the foo=initial point, but not the + // range tombstone nor the covered points. + select { + case e := <-rowC: + require.Equal(t, roachpb.Key(mkKey("foo")), e.Key) + value, err := e.Value.GetBytes() + require.NoError(t, err) + require.Equal(t, "initial", string(value)) + prevValue, err := e.PrevValue.GetBytes() require.NoError(t, err) + require.Equal(t, "initial", string(prevValue)) // initial scans supply current as prev + case <-time.After(3 * time.Second): + require.Fail(t, "timed out waiting for initial scan event") + } - mkKey := func(s string) string { - return string(append(srv.Codec().TenantPrefix(), roachpb.Key(s)...)) - } - // We lay down a few MVCC range tombstones and points. The first range - // tombstone should not be visible, because initial scans do not emit - // tombstones, nor should the points covered by it. The second range tombstone - // should be visible, because catchup scans do emit tombstones. The range - // tombstone should be ordered after the initial point, but before the foo - // catchup point, and the previous values should respect the range tombstones. - require.NoError(t, db.Put(ctx, mkKey("covered"), "covered")) - require.NoError(t, db.Put(ctx, mkKey("foo"), "covered")) - require.NoError(t, db.DelRangeUsingTombstone(ctx, mkKey("a"), mkKey("z"))) - require.NoError(t, db.Put(ctx, mkKey("foo"), "initial")) - rangeFeedTS := db.Clock().Now() - require.NoError(t, db.Put(ctx, mkKey("covered"), "catchup")) - require.NoError(t, db.DelRangeUsingTombstone(ctx, mkKey("a"), mkKey("z"))) - require.NoError(t, db.Put(ctx, mkKey("foo"), "catchup")) - - // We start the rangefeed over a narrower span than the DeleteRanges (c-g), - // to ensure the DeleteRange event is truncated to the registration span. - var checkpointOnce sync.Once - checkpointC := make(chan struct{}) - deleteRangeC := make(chan *kvpb.RangeFeedDeleteRange) - rowC := make(chan *kvpb.RangeFeedValue) - - spans := []roachpb.Span{{ + // Wait for catchup scan. We should see the second range tombstone, truncated + // to the rangefeed bounds (c-g), and it should be ordered before the points + // covered=catchup and foo=catchup. both points should have a tombstone as the + // previous value. + select { + case e := <-deleteRangeC: + require.Equal(t, roachpb.Span{ Key: append(srv.Codec().TenantPrefix(), roachpb.Key("c")...), EndKey: append(srv.Codec().TenantPrefix(), roachpb.Key("g")...), - }} - r, err := f.RangeFeed(ctx, "test", spans, rangeFeedTS, - func(ctx context.Context, e *kvpb.RangeFeedValue) { - select { - case rowC <- e: - case <-ctx.Done(): - } - }, - rangefeed.WithDiff(true), - rangefeed.WithInitialScan(nil), - rangefeed.WithOnCheckpoint(func(ctx context.Context, checkpoint *kvpb.RangeFeedCheckpoint) { - checkpointOnce.Do(func() { - close(checkpointC) - }) - }), - rangefeed.WithOnDeleteRange(func(ctx context.Context, e *kvpb.RangeFeedDeleteRange) { - select { - case deleteRangeC <- e: - case <-ctx.Done(): - } - }), - ) - require.NoError(t, err) - defer r.Close() - - // Wait for initial scan. We should see the foo=initial point, but not the - // range tombstone nor the covered points. - select { - case e := <-rowC: - require.Equal(t, roachpb.Key(mkKey("foo")), e.Key) - value, err := e.Value.GetBytes() - require.NoError(t, err) - require.Equal(t, "initial", string(value)) - prevValue, err := e.PrevValue.GetBytes() - require.NoError(t, err) - require.Equal(t, "initial", string(prevValue)) // initial scans supply current as prev - case <-time.After(3 * time.Second): - require.Fail(t, "timed out waiting for initial scan event") - } - - // Wait for catchup scan. We should see the second range tombstone, truncated - // to the rangefeed bounds (c-g), and it should be ordered before the points - // covered=catchup and foo=catchup. both points should have a tombstone as the - // previous value. - select { - case e := <-deleteRangeC: - require.Equal(t, roachpb.Span{ - Key: append(srv.Codec().TenantPrefix(), roachpb.Key("c")...), - EndKey: append(srv.Codec().TenantPrefix(), roachpb.Key("g")...), - }, e.Span) - require.NotEmpty(t, e.Timestamp) - case <-time.After(3 * time.Second): - require.Fail(t, "timed out waiting for DeleteRange event") - } + }, e.Span) + require.NotEmpty(t, e.Timestamp) + case <-time.After(3 * time.Second): + require.Fail(t, "timed out waiting for DeleteRange event") + } - select { - case e := <-rowC: - require.Equal(t, roachpb.Key(mkKey("covered")), e.Key) - value, err := e.Value.GetBytes() - require.NoError(t, err) - require.Equal(t, "catchup", string(value)) - prevValue, err := storage.DecodeMVCCValue(e.PrevValue.RawBytes) - require.NoError(t, err) - require.True(t, prevValue.IsTombstone()) - case <-time.After(3 * time.Second): - require.Fail(t, "timed out waiting for foo=catchup event") - } + select { + case e := <-rowC: + require.Equal(t, roachpb.Key(mkKey("covered")), e.Key) + value, err := e.Value.GetBytes() + require.NoError(t, err) + require.Equal(t, "catchup", string(value)) + prevValue, err := storage.DecodeMVCCValue(e.PrevValue.RawBytes) + require.NoError(t, err) + require.True(t, prevValue.IsTombstone()) + case <-time.After(3 * time.Second): + require.Fail(t, "timed out waiting for foo=catchup event") + } - select { - case e := <-rowC: - require.Equal(t, roachpb.Key(mkKey("foo")), e.Key) - value, err := e.Value.GetBytes() - require.NoError(t, err) - require.Equal(t, "catchup", string(value)) - prevValue, err := storage.DecodeMVCCValue(e.PrevValue.RawBytes) - require.NoError(t, err) - require.True(t, prevValue.IsTombstone()) - case <-time.After(3 * time.Second): - require.Fail(t, "timed out waiting for foo=catchup event") - } + select { + case e := <-rowC: + require.Equal(t, roachpb.Key(mkKey("foo")), e.Key) + value, err := e.Value.GetBytes() + require.NoError(t, err) + require.Equal(t, "catchup", string(value)) + prevValue, err := storage.DecodeMVCCValue(e.PrevValue.RawBytes) + require.NoError(t, err) + require.True(t, prevValue.IsTombstone()) + case <-time.After(3 * time.Second): + require.Fail(t, "timed out waiting for foo=catchup event") + } - // Wait for checkpoint after catchup scan. - select { - case <-checkpointC: - case <-time.After(3 * time.Second): - require.Fail(t, "timed out waiting for checkpoint") - } + // Wait for checkpoint after catchup scan. + select { + case <-checkpointC: + case <-time.After(3 * time.Second): + require.Fail(t, "timed out waiting for checkpoint") + } - // Send another DeleteRange, and wait for the rangefeed event. This should - // be truncated to the rangefeed bounds (c-g). - require.NoError(t, db.DelRangeUsingTombstone(ctx, mkKey("a"), mkKey("z"))) - select { - case e := <-deleteRangeC: - require.Equal(t, roachpb.Span{ - Key: append(srv.Codec().TenantPrefix(), roachpb.Key("c")...), - EndKey: append(srv.Codec().TenantPrefix(), roachpb.Key("g")...), - }, e.Span) - require.NotEmpty(t, e.Timestamp) - case <-time.After(3 * time.Second): - require.Fail(t, "timed out waiting for DeleteRange event") - } + // Send another DeleteRange, and wait for the rangefeed event. This should + // be truncated to the rangefeed bounds (c-g). + require.NoError(t, db.DelRangeUsingTombstone(ctx, mkKey("a"), mkKey("z"))) + select { + case e := <-deleteRangeC: + require.Equal(t, roachpb.Span{ + Key: append(srv.Codec().TenantPrefix(), roachpb.Key("c")...), + EndKey: append(srv.Codec().TenantPrefix(), roachpb.Key("g")...), + }, e.Span) + require.NotEmpty(t, e.Timestamp) + case <-time.After(3 * time.Second): + require.Fail(t, "timed out waiting for DeleteRange event") + } - // A final point write should be emitted with a tombstone as the previous value. - require.NoError(t, db.Put(ctx, mkKey("foo"), "final")) - select { - case e := <-rowC: - require.Equal(t, roachpb.Key(mkKey("foo")), e.Key) - value, err := e.Value.GetBytes() - require.NoError(t, err) - require.Equal(t, "final", string(value)) - prevValue, err := storage.DecodeMVCCValue(e.PrevValue.RawBytes) - require.NoError(t, err) - require.True(t, prevValue.IsTombstone()) - case <-time.After(3 * time.Second): - require.Fail(t, "timed out waiting for foo=final event") - } - }) + // A final point write should be emitted with a tombstone as the previous value. + require.NoError(t, db.Put(ctx, mkKey("foo"), "final")) + select { + case e := <-rowC: + require.Equal(t, roachpb.Key(mkKey("foo")), e.Key) + value, err := e.Value.GetBytes() + require.NoError(t, err) + require.Equal(t, "final", string(value)) + prevValue, err := storage.DecodeMVCCValue(e.PrevValue.RawBytes) + require.NoError(t, err) + require.True(t, prevValue.IsTombstone()) + case <-time.After(3 * time.Second): + require.Fail(t, "timed out waiting for foo=final event") + } } // TestUnrecoverableErrors verifies that unrecoverable internal errors are surfaced @@ -951,97 +891,91 @@ func TestUnrecoverableErrors(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - testutils.RunValues(t, "feed_type", procTypes, func(t *testing.T, s feedProcessorType) { - ctx := context.Background() - settings := cluster.MakeTestingClusterSettings() - // We must enable desired scheduler settings before we start cluster, - // otherwise we will trigger processor restarts later and this test can't - // handle duplicated events. - kvserver.RangeFeedUseScheduler.Override(ctx, &settings.SV, s.useScheduler) - srv, sqlDB, kvDB := serverutils.StartServer(t, base.TestServerArgs{ - DefaultTestTenant: base.TestIsForStuffThatShouldWorkWithSecondaryTenantsButDoesntYet(109472), - Knobs: base.TestingKnobs{ - SpanConfig: &spanconfig.TestingKnobs{ - ConfigureScratchRange: true, - }, + ctx := context.Background() + settings := cluster.MakeTestingClusterSettings() + srv, sqlDB, kvDB := serverutils.StartServer(t, base.TestServerArgs{ + DefaultTestTenant: base.TestIsForStuffThatShouldWorkWithSecondaryTenantsButDoesntYet(109472), + Knobs: base.TestingKnobs{ + SpanConfig: &spanconfig.TestingKnobs{ + ConfigureScratchRange: true, }, - Settings: settings, - }) - - defer srv.Stopper().Stop(ctx) - ts := srv.ApplicationLayer() - - scratchKey := append(ts.Codec().TenantPrefix(), keys.ScratchRangeMin...) - _, _, err := srv.SplitRange(scratchKey) - require.NoError(t, err) - scratchKey = scratchKey[:len(scratchKey):len(scratchKey)] + }, + Settings: settings, + }) - sp := roachpb.Span{ - Key: scratchKey, - EndKey: scratchKey.PrefixEnd(), - } - for _, l := range []serverutils.ApplicationLayerInterface{ts, srv.SystemLayer()} { - // Enable rangefeeds, otherwise the thing will retry until they are enabled. - kvserver.RangefeedEnabled.Override(ctx, &l.ClusterSettings().SV, true) - // Lower the closed timestamp target duration to speed up the test. - closedts.TargetDuration.Override(ctx, &l.ClusterSettings().SV, 100*time.Millisecond) - } + defer srv.Stopper().Stop(ctx) + ts := srv.ApplicationLayer() - store, err := srv.GetStores().(*kvserver.Stores).GetStore(srv.GetFirstStoreID()) - require.NoError(t, err) + scratchKey := append(ts.Codec().TenantPrefix(), keys.ScratchRangeMin...) + _, _, err := srv.SplitRange(scratchKey) + require.NoError(t, err) + scratchKey = scratchKey[:len(scratchKey):len(scratchKey)] - { - // Lower the protectedts Cache refresh interval, so that the - // `preGCThresholdTS` defined below is less than the protectedts - // `readAt - GCTTL` window, resulting in a BatchTimestampBelowGCError. - _, err = sqlDB.Exec("SET CLUSTER SETTING kv.protectedts.poll_interval = '10ms'") - require.NoError(t, err) + sp := roachpb.Span{ + Key: scratchKey, + EndKey: scratchKey.PrefixEnd(), + } + for _, l := range []serverutils.ApplicationLayerInterface{ts, srv.SystemLayer()} { + // Enable rangefeeds, otherwise the thing will retry until they are enabled. + kvserver.RangefeedEnabled.Override(ctx, &l.ClusterSettings().SV, true) + // Lower the closed timestamp target duration to speed up the test. + closedts.TargetDuration.Override(ctx, &l.ClusterSettings().SV, 100*time.Millisecond) + } - ptsReader := store.GetStoreConfig().ProtectedTimestampReader - require.NoError(t, - spanconfigptsreader.TestingRefreshPTSState(ctx, t, ptsReader, srv.SystemLayer().Clock().Now())) - } + store, err := srv.GetStores().(*kvserver.Stores).GetStore(srv.GetFirstStoreID()) + require.NoError(t, err) - f, err := rangefeed.NewFactory(ts.AppStopper(), kvDB, ts.ClusterSettings(), nil) + { + // Lower the protectedts Cache refresh interval, so that the + // `preGCThresholdTS` defined below is less than the protectedts + // `readAt - GCTTL` window, resulting in a BatchTimestampBelowGCError. + _, err = sqlDB.Exec("SET CLUSTER SETTING kv.protectedts.poll_interval = '10ms'") require.NoError(t, err) - preGCThresholdTS := hlc.Timestamp{WallTime: 1} - mu := struct { - syncutil.Mutex - internalErr error - }{} + ptsReader := store.GetStoreConfig().ProtectedTimestampReader + require.NoError(t, + spanconfigptsreader.TestingRefreshPTSState(ctx, t, ptsReader, srv.SystemLayer().Clock().Now())) + } - testutils.SucceedsSoon(t, func() error { - repl := store.LookupReplica(roachpb.RKey(scratchKey)) - if conf, err := repl.LoadSpanConfig(ctx); err != nil || conf.GCPolicy.IgnoreStrictEnforcement { - return errors.New("waiting for span config to apply") - } - require.NoError(t, repl.ReadProtectedTimestampsForTesting(ctx)) - return nil - }) + f, err := rangefeed.NewFactory(ts.AppStopper(), kvDB, ts.ClusterSettings(), nil) + require.NoError(t, err) - r, err := f.RangeFeed(ctx, "test", []roachpb.Span{sp}, preGCThresholdTS, - func(context.Context, *kvpb.RangeFeedValue) {}, - rangefeed.WithDiff(true), - rangefeed.WithOnInternalError(func(ctx context.Context, err error) { - mu.Lock() - defer mu.Unlock() + preGCThresholdTS := hlc.Timestamp{WallTime: 1} + mu := struct { + syncutil.Mutex + internalErr error + }{} - mu.internalErr = err - }), - ) - require.NoError(t, err) - defer r.Close() + testutils.SucceedsSoon(t, func() error { + repl := store.LookupReplica(roachpb.RKey(scratchKey)) + if conf, err := repl.LoadSpanConfig(ctx); err != nil || conf.GCPolicy.IgnoreStrictEnforcement { + return errors.New("waiting for span config to apply") + } + require.NoError(t, repl.ReadProtectedTimestampsForTesting(ctx)) + return nil + }) - testutils.SucceedsSoon(t, func() error { + r, err := f.RangeFeed(ctx, "test", []roachpb.Span{sp}, preGCThresholdTS, + func(context.Context, *kvpb.RangeFeedValue) {}, + rangefeed.WithDiff(true), + rangefeed.WithOnInternalError(func(ctx context.Context, err error) { mu.Lock() defer mu.Unlock() - if !errors.HasType(mu.internalErr, &kvpb.BatchTimestampBeforeGCError{}) { - return errors.New("expected internal error") - } - return nil - }) + mu.internalErr = err + }), + ) + require.NoError(t, err) + defer r.Close() + + testutils.SucceedsSoon(t, func() error { + mu.Lock() + defer mu.Unlock() + + if !errors.HasType(mu.internalErr, &kvpb.BatchTimestampBeforeGCError{}) { + return errors.New("expected internal error") + } + return nil }) } @@ -1051,88 +985,82 @@ func TestMVCCHistoryMutationError(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - testutils.RunValues(t, "feed_type", procTypes, func(t *testing.T, s feedProcessorType) { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - settings := cluster.MakeTestingClusterSettings() - // We must enable desired scheduler settings before we start cluster, - // otherwise we will trigger processor restarts later and this test can't - // handle duplicated events. - kvserver.RangeFeedUseScheduler.Override(ctx, &settings.SV, s.useScheduler) - srv, _, db := serverutils.StartServer(t, base.TestServerArgs{ - Settings: settings, - }) - defer srv.Stopper().Stop(ctx) - ts := srv.ApplicationLayer() + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() - scratchKey := append(ts.Codec().TenantPrefix(), keys.ScratchRangeMin...) - _, _, err := srv.SplitRange(scratchKey) - require.NoError(t, err) - scratchKey = scratchKey[:len(scratchKey):len(scratchKey)] - sp := roachpb.Span{ - Key: scratchKey, - EndKey: scratchKey.PrefixEnd(), - } - for _, l := range []serverutils.ApplicationLayerInterface{ts, srv.SystemLayer()} { - // Enable rangefeeds, otherwise the thing will retry until they are enabled. - kvserver.RangefeedEnabled.Override(ctx, &l.ClusterSettings().SV, true) - // Lower the closed timestamp target duration to speed up the test. - closedts.TargetDuration.Override(ctx, &l.ClusterSettings().SV, 100*time.Millisecond) - } + settings := cluster.MakeTestingClusterSettings() + srv, _, db := serverutils.StartServer(t, base.TestServerArgs{ + Settings: settings, + }) + defer srv.Stopper().Stop(ctx) + ts := srv.ApplicationLayer() - // Set up a rangefeed. - f, err := rangefeed.NewFactory(ts.AppStopper(), db, ts.ClusterSettings(), nil) - require.NoError(t, err) + scratchKey := append(ts.Codec().TenantPrefix(), keys.ScratchRangeMin...) + _, _, err := srv.SplitRange(scratchKey) + require.NoError(t, err) + scratchKey = scratchKey[:len(scratchKey):len(scratchKey)] + sp := roachpb.Span{ + Key: scratchKey, + EndKey: scratchKey.PrefixEnd(), + } + for _, l := range []serverutils.ApplicationLayerInterface{ts, srv.SystemLayer()} { + // Enable rangefeeds, otherwise the thing will retry until they are enabled. + kvserver.RangefeedEnabled.Override(ctx, &l.ClusterSettings().SV, true) + // Lower the closed timestamp target duration to speed up the test. + closedts.TargetDuration.Override(ctx, &l.ClusterSettings().SV, 100*time.Millisecond) + } - var once sync.Once - checkpointC := make(chan struct{}) - errC := make(chan error) - r, err := f.RangeFeed(ctx, "test", []roachpb.Span{sp}, ts.Clock().Now(), - func(context.Context, *kvpb.RangeFeedValue) {}, - rangefeed.WithOnCheckpoint(func(ctx context.Context, checkpoint *kvpb.RangeFeedCheckpoint) { - once.Do(func() { - close(checkpointC) - }) - }), - rangefeed.WithOnInternalError(func(ctx context.Context, err error) { - select { - case errC <- err: - case <-ctx.Done(): - } - }), - ) - require.NoError(t, err) - defer r.Close() + // Set up a rangefeed. + f, err := rangefeed.NewFactory(ts.AppStopper(), db, ts.ClusterSettings(), nil) + require.NoError(t, err) - // Wait for initial checkpoint. - select { - case <-checkpointC: - case err := <-errC: - require.NoError(t, err) - case <-time.After(3 * time.Second): - require.Fail(t, "timed out waiting for checkpoint") - } + var once sync.Once + checkpointC := make(chan struct{}) + errC := make(chan error) + r, err := f.RangeFeed(ctx, "test", []roachpb.Span{sp}, ts.Clock().Now(), + func(context.Context, *kvpb.RangeFeedValue) {}, + rangefeed.WithOnCheckpoint(func(ctx context.Context, checkpoint *kvpb.RangeFeedCheckpoint) { + once.Do(func() { + close(checkpointC) + }) + }), + rangefeed.WithOnInternalError(func(ctx context.Context, err error) { + select { + case errC <- err: + case <-ctx.Done(): + } + }), + ) + require.NoError(t, err) + defer r.Close() - // Send a ClearRange request that mutates MVCC history. - _, pErr := kv.SendWrapped(ctx, db.NonTransactionalSender(), &kvpb.ClearRangeRequest{ - RequestHeader: kvpb.RequestHeader{ - Key: sp.Key, - EndKey: sp.EndKey, - }, - }) - require.Nil(t, pErr) + // Wait for initial checkpoint. + select { + case <-checkpointC: + case err := <-errC: + require.NoError(t, err) + case <-time.After(3 * time.Second): + require.Fail(t, "timed out waiting for checkpoint") + } - // Wait for the MVCCHistoryMutationError. - select { - case err := <-errC: - var mvccErr *kvpb.MVCCHistoryMutationError - require.ErrorAs(t, err, &mvccErr) - require.Equal(t, &kvpb.MVCCHistoryMutationError{Span: sp}, err) - case <-time.After(3 * time.Second): - require.Fail(t, "timed out waiting for error") - } + // Send a ClearRange request that mutates MVCC history. + _, pErr := kv.SendWrapped(ctx, db.NonTransactionalSender(), &kvpb.ClearRangeRequest{ + RequestHeader: kvpb.RequestHeader{ + Key: sp.Key, + EndKey: sp.EndKey, + }, }) + require.Nil(t, pErr) + + // Wait for the MVCCHistoryMutationError. + select { + case err := <-errC: + var mvccErr *kvpb.MVCCHistoryMutationError + require.ErrorAs(t, err, &mvccErr) + require.Equal(t, &kvpb.MVCCHistoryMutationError{Span: sp}, err) + case <-time.After(3 * time.Second): + require.Fail(t, "timed out waiting for error") + } } // TestRangefeedWithLabelsOption verifies go routines started by rangefeed are @@ -1141,117 +1069,111 @@ func TestRangefeedWithLabelsOption(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - testutils.RunValues(t, "feed_type", procTypes, func(t *testing.T, s feedProcessorType) { - ctx := context.Background() - settings := cluster.MakeTestingClusterSettings() - // We must enable desired scheduler settings before we start cluster, - // otherwise we will trigger processor restarts later and this test can't - // handle duplicated events. - kvserver.RangeFeedUseScheduler.Override(ctx, &settings.SV, s.useScheduler) - srv, _, db := serverutils.StartServer(t, base.TestServerArgs{ - Settings: settings, - }) - defer srv.Stopper().Stop(ctx) - ts := srv.ApplicationLayer() + ctx := context.Background() + settings := cluster.MakeTestingClusterSettings() + srv, _, db := serverutils.StartServer(t, base.TestServerArgs{ + Settings: settings, + }) + defer srv.Stopper().Stop(ctx) + ts := srv.ApplicationLayer() - scratchKey := append(ts.Codec().TenantPrefix(), keys.ScratchRangeMin...) - _, _, err := srv.SplitRange(scratchKey) + scratchKey := append(ts.Codec().TenantPrefix(), keys.ScratchRangeMin...) + _, _, err := srv.SplitRange(scratchKey) + require.NoError(t, err) + scratchKey = scratchKey[:len(scratchKey):len(scratchKey)] + mkKey := func(k string) roachpb.Key { + return encoding.EncodeStringAscending(scratchKey, k) + } + // Split the range a bunch of times. + const splits = 10 + for i := 0; i < splits; i++ { + _, _, err := srv.SplitRange(mkKey(string([]byte{'a' + byte(i)}))) require.NoError(t, err) - scratchKey = scratchKey[:len(scratchKey):len(scratchKey)] - mkKey := func(k string) roachpb.Key { - return encoding.EncodeStringAscending(scratchKey, k) - } - // Split the range a bunch of times. - const splits = 10 - for i := 0; i < splits; i++ { - _, _, err := srv.SplitRange(mkKey(string([]byte{'a' + byte(i)}))) - require.NoError(t, err) - } + } - require.NoError(t, db.Put(ctx, mkKey("a"), 1)) - require.NoError(t, db.Put(ctx, mkKey("b"), 2)) - afterB := db.Clock().Now() - require.NoError(t, db.Put(ctx, mkKey("c"), 3)) + require.NoError(t, db.Put(ctx, mkKey("a"), 1)) + require.NoError(t, db.Put(ctx, mkKey("b"), 2)) + afterB := db.Clock().Now() + require.NoError(t, db.Put(ctx, mkKey("c"), 3)) - sp := roachpb.Span{ - Key: scratchKey, - EndKey: scratchKey.PrefixEnd(), - } - for _, l := range []serverutils.ApplicationLayerInterface{ts, srv.SystemLayer()} { - // Enable rangefeeds, otherwise the thing will retry until they are enabled. - kvserver.RangefeedEnabled.Override(ctx, &l.ClusterSettings().SV, true) - } + sp := roachpb.Span{ + Key: scratchKey, + EndKey: scratchKey.PrefixEnd(), + } + for _, l := range []serverutils.ApplicationLayerInterface{ts, srv.SystemLayer()} { + // Enable rangefeeds, otherwise the thing will retry until they are enabled. + kvserver.RangefeedEnabled.Override(ctx, &l.ClusterSettings().SV, true) + } - const rangefeedName = "test-feed" - type label struct { - k, v string - } - defaultLabel := label{k: "rangefeed", v: rangefeedName} - label1 := label{k: "caller-label", v: "foo"} - label2 := label{k: "another-label", v: "bar"} - - allLabelsCorrect := struct { - syncutil.Mutex - correct bool - }{correct: true} - - verifyLabels := func(ctx context.Context) { - allLabelsCorrect.Lock() - defer allLabelsCorrect.Unlock() - if !allLabelsCorrect.correct { - return - } + const rangefeedName = "test-feed" + type label struct { + k, v string + } + defaultLabel := label{k: "rangefeed", v: rangefeedName} + label1 := label{k: "caller-label", v: "foo"} + label2 := label{k: "another-label", v: "bar"} - m := make(map[string]string) - pprof.ForLabels(ctx, func(k, v string) bool { - m[k] = v - return true - }) + allLabelsCorrect := struct { + syncutil.Mutex + correct bool + }{correct: true} - allLabelsCorrect.correct = - m[defaultLabel.k] == defaultLabel.v && m[label1.k] == label1.v && m[label2.k] == label2.v + verifyLabels := func(ctx context.Context) { + allLabelsCorrect.Lock() + defer allLabelsCorrect.Unlock() + if !allLabelsCorrect.correct { + return } - f, err := rangefeed.NewFactory(ts.AppStopper(), db, ts.ClusterSettings(), nil) - require.NoError(t, err) - initialScanDone := make(chan struct{}) + m := make(map[string]string) + pprof.ForLabels(ctx, func(k, v string) bool { + m[k] = v + return true + }) - // We'll emit keyD a bit later, once initial scan completes. - keyD := mkKey("d") - var keyDSeen sync.Once - keyDSeenCh := make(chan struct{}) + allLabelsCorrect.correct = + m[defaultLabel.k] == defaultLabel.v && m[label1.k] == label1.v && m[label2.k] == label2.v + } - r, err := f.RangeFeed(ctx, rangefeedName, []roachpb.Span{sp}, afterB, - func(ctx context.Context, value *kvpb.RangeFeedValue) { - verifyLabels(ctx) - if value.Key.Equal(keyD) { - keyDSeen.Do(func() { close(keyDSeenCh) }) - } - }, - rangefeed.WithPProfLabel(label1.k, label1.v), - rangefeed.WithPProfLabel(label2.k, label2.v), - rangefeed.WithInitialScanParallelismFn(func() int { return 3 }), - rangefeed.WithOnScanCompleted(func(ctx context.Context, sp roachpb.Span) error { - verifyLabels(ctx) - return nil - }), - rangefeed.WithInitialScan(func(ctx context.Context) { - verifyLabels(ctx) - close(initialScanDone) - }), - ) - require.NoError(t, err) - defer r.Close() + f, err := rangefeed.NewFactory(ts.AppStopper(), db, ts.ClusterSettings(), nil) + require.NoError(t, err) + initialScanDone := make(chan struct{}) + + // We'll emit keyD a bit later, once initial scan completes. + keyD := mkKey("d") + var keyDSeen sync.Once + keyDSeenCh := make(chan struct{}) + + r, err := f.RangeFeed(ctx, rangefeedName, []roachpb.Span{sp}, afterB, + func(ctx context.Context, value *kvpb.RangeFeedValue) { + verifyLabels(ctx) + if value.Key.Equal(keyD) { + keyDSeen.Do(func() { close(keyDSeenCh) }) + } + }, + rangefeed.WithPProfLabel(label1.k, label1.v), + rangefeed.WithPProfLabel(label2.k, label2.v), + rangefeed.WithInitialScanParallelismFn(func() int { return 3 }), + rangefeed.WithOnScanCompleted(func(ctx context.Context, sp roachpb.Span) error { + verifyLabels(ctx) + return nil + }), + rangefeed.WithInitialScan(func(ctx context.Context) { + verifyLabels(ctx) + close(initialScanDone) + }), + ) + require.NoError(t, err) + defer r.Close() - <-initialScanDone + <-initialScanDone - // Write a new value for "a" and make sure it is seen. - require.NoError(t, db.Put(ctx, keyD, 5)) - <-keyDSeenCh - allLabelsCorrect.Lock() - defer allLabelsCorrect.Unlock() - require.True(t, allLabelsCorrect.correct) - }) + // Write a new value for "a" and make sure it is seen. + require.NoError(t, db.Put(ctx, keyD, 5)) + <-keyDSeenCh + allLabelsCorrect.Lock() + defer allLabelsCorrect.Unlock() + require.True(t, allLabelsCorrect.correct) } // TestRangeFeedStartTimeExclusive tests that the start timestamp of the @@ -1260,66 +1182,60 @@ func TestRangeFeedStartTimeExclusive(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - testutils.RunValues(t, "feed_type", procTypes, func(t *testing.T, s feedProcessorType) { - ctx := context.Background() - settings := cluster.MakeTestingClusterSettings() - // We must enable desired scheduler settings before we start cluster, - // otherwise we will trigger processor restarts later and this test can't - // handle duplicated events. - kvserver.RangeFeedUseScheduler.Override(ctx, &settings.SV, s.useScheduler) - srv, _, db := serverutils.StartServer(t, base.TestServerArgs{ - Settings: settings, - }) - defer srv.Stopper().Stop(ctx) - ts := srv.ApplicationLayer() + ctx := context.Background() + settings := cluster.MakeTestingClusterSettings() + srv, _, db := serverutils.StartServer(t, base.TestServerArgs{ + Settings: settings, + }) + defer srv.Stopper().Stop(ctx) + ts := srv.ApplicationLayer() - scratchKey := append(ts.Codec().TenantPrefix(), keys.ScratchRangeMin...) - _, _, err := srv.SplitRange(scratchKey) - require.NoError(t, err) - scratchKey = scratchKey[:len(scratchKey):len(scratchKey)] - mkKey := func(k string) roachpb.Key { - return encoding.EncodeStringAscending(scratchKey, k) - } - span := roachpb.Span{Key: scratchKey, EndKey: scratchKey.PrefixEnd()} + scratchKey := append(ts.Codec().TenantPrefix(), keys.ScratchRangeMin...) + _, _, err := srv.SplitRange(scratchKey) + require.NoError(t, err) + scratchKey = scratchKey[:len(scratchKey):len(scratchKey)] + mkKey := func(k string) roachpb.Key { + return encoding.EncodeStringAscending(scratchKey, k) + } + span := roachpb.Span{Key: scratchKey, EndKey: scratchKey.PrefixEnd()} - // Write three versions of "foo". Get the timestamp of the second version. - require.NoError(t, db.Put(ctx, mkKey("foo"), 1)) - require.NoError(t, db.Put(ctx, mkKey("foo"), 2)) - kv, err := db.Get(ctx, mkKey("foo")) - require.NoError(t, err) - ts2 := kv.Value.Timestamp - require.NoError(t, db.Put(ctx, mkKey("foo"), 3)) + // Write three versions of "foo". Get the timestamp of the second version. + require.NoError(t, db.Put(ctx, mkKey("foo"), 1)) + require.NoError(t, db.Put(ctx, mkKey("foo"), 2)) + kv, err := db.Get(ctx, mkKey("foo")) + require.NoError(t, err) + ts2 := kv.Value.Timestamp + require.NoError(t, db.Put(ctx, mkKey("foo"), 3)) - for _, l := range []serverutils.ApplicationLayerInterface{ts, srv.SystemLayer()} { - // Enable rangefeeds, otherwise the thing will retry until they are enabled. - kvserver.RangefeedEnabled.Override(ctx, &l.ClusterSettings().SV, true) - } + for _, l := range []serverutils.ApplicationLayerInterface{ts, srv.SystemLayer()} { + // Enable rangefeeds, otherwise the thing will retry until they are enabled. + kvserver.RangefeedEnabled.Override(ctx, &l.ClusterSettings().SV, true) + } - f, err := rangefeed.NewFactory(ts.AppStopper(), db, ts.ClusterSettings(), nil) - require.NoError(t, err) - rows := make(chan *kvpb.RangeFeedValue) - r, err := f.RangeFeed(ctx, "test", []roachpb.Span{span}, ts2, - func(ctx context.Context, value *kvpb.RangeFeedValue) { - select { - case rows <- value: - case <-ctx.Done(): - } - }, - ) - require.NoError(t, err) - defer r.Close() + f, err := rangefeed.NewFactory(ts.AppStopper(), db, ts.ClusterSettings(), nil) + require.NoError(t, err) + rows := make(chan *kvpb.RangeFeedValue) + r, err := f.RangeFeed(ctx, "test", []roachpb.Span{span}, ts2, + func(ctx context.Context, value *kvpb.RangeFeedValue) { + select { + case rows <- value: + case <-ctx.Done(): + } + }, + ) + require.NoError(t, err) + defer r.Close() - // The first emitted version should be 3. - select { - case row := <-rows: - require.Equal(t, mkKey("foo"), row.Key) - v, err := row.Value.GetInt() - require.NoError(t, err) - require.EqualValues(t, 3, v) - case <-time.After(3 * time.Second): - t.Fatal("timed out waiting for event") - } - }) + // The first emitted version should be 3. + select { + case row := <-rows: + require.Equal(t, mkKey("foo"), row.Key) + v, err := row.Value.GetInt() + require.NoError(t, err) + require.EqualValues(t, 3, v) + case <-time.After(3 * time.Second): + t.Fatal("timed out waiting for event") + } } // TestRangeFeedIntentResolutionRace is a regression test for @@ -1666,120 +1582,114 @@ func TestRangeFeedMetadataManualSplit(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - testutils.RunValues(t, "feed_type", procTypes, func(t *testing.T, s feedProcessorType) { - ctx := context.Background() - settings := cluster.MakeTestingClusterSettings() - // We must enable desired scheduler settings before we start cluster, - // otherwise we will trigger processor restarts later and this test can't - // handle duplicated events. - kvserver.RangeFeedUseScheduler.Override(ctx, &settings.SV, s.useScheduler) - kvserver.RangefeedEnabled.Override(ctx, &settings.SV, true) - srv, _, db := serverutils.StartServer(t, base.TestServerArgs{ - Settings: settings, - }) - defer srv.Stopper().Stop(ctx) - ts := srv.ApplicationLayer() + ctx := context.Background() + settings := cluster.MakeTestingClusterSettings() + kvserver.RangefeedEnabled.Override(ctx, &settings.SV, true) + srv, _, db := serverutils.StartServer(t, base.TestServerArgs{ + Settings: settings, + }) + defer srv.Stopper().Stop(ctx) + ts := srv.ApplicationLayer() - scratchKey := append(ts.Codec().TenantPrefix(), keys.ScratchRangeMin...) - _, _, err := srv.SplitRange(scratchKey) - require.NoError(t, err) - scratchKey = scratchKey[:len(scratchKey):len(scratchKey)] - mkKey := func(k string) roachpb.Key { - return encoding.EncodeStringAscending(scratchKey, k) - } + scratchKey := append(ts.Codec().TenantPrefix(), keys.ScratchRangeMin...) + _, _, err := srv.SplitRange(scratchKey) + require.NoError(t, err) + scratchKey = scratchKey[:len(scratchKey):len(scratchKey)] + mkKey := func(k string) roachpb.Key { + return encoding.EncodeStringAscending(scratchKey, k) + } - sp := roachpb.Span{ - Key: scratchKey, - EndKey: scratchKey.PrefixEnd(), - } + sp := roachpb.Span{ + Key: scratchKey, + EndKey: scratchKey.PrefixEnd(), + } - f, err := rangefeed.NewFactory(ts.AppStopper(), db, ts.ClusterSettings(), nil) - require.NoError(t, err) + f, err := rangefeed.NewFactory(ts.AppStopper(), db, ts.ClusterSettings(), nil) + require.NoError(t, err) - metadata := make(chan *kvpb.RangeFeedMetadata) - initialScanDone := make(chan struct{}) - rows := make(chan *kvpb.RangeFeedValue) - r, err := f.RangeFeed(ctx, "test", []roachpb.Span{sp}, ts.DB().Clock().Now(), - func(ctx context.Context, value *kvpb.RangeFeedValue) { - select { - case rows <- value: - case <-ctx.Done(): - } - }, - rangefeed.WithInitialScan(func(ctx context.Context) { - close(initialScanDone) - }), - rangefeed.WithOnMetadata(func(ctx context.Context, value *kvpb.RangeFeedMetadata) { - select { - case metadata <- value: - case <-ctx.Done(): - return - } - }), - ) - require.NoError(t, err) + metadata := make(chan *kvpb.RangeFeedMetadata) + initialScanDone := make(chan struct{}) + rows := make(chan *kvpb.RangeFeedValue) + r, err := f.RangeFeed(ctx, "test", []roachpb.Span{sp}, ts.DB().Clock().Now(), + func(ctx context.Context, value *kvpb.RangeFeedValue) { + select { + case rows <- value: + case <-ctx.Done(): + } + }, + rangefeed.WithInitialScan(func(ctx context.Context) { + close(initialScanDone) + }), + rangefeed.WithOnMetadata(func(ctx context.Context, value *kvpb.RangeFeedMetadata) { + select { + case metadata <- value: + case <-ctx.Done(): + return + } + }), + ) + require.NoError(t, err) - defer func() { - r.Close() - close(metadata) - close(rows) - }() - { - // First meta msg for the new rangefeed. - meta := <-metadata - t.Logf("initial span %s-%s", meta.Span.Key, meta.Span.EndKey) - require.False(t, meta.FromManualSplit) - require.Equal(t, sp.Key, meta.Span.Key) - require.Equal(t, sp.EndKey, meta.Span.EndKey) - require.Empty(t, meta.ParentStartKey) - } - <-initialScanDone + defer func() { + r.Close() + close(metadata) + close(rows) + }() + { + // First meta msg for the new rangefeed. + meta := <-metadata + t.Logf("initial span %s-%s", meta.Span.Key, meta.Span.EndKey) + require.False(t, meta.FromManualSplit) + require.Equal(t, sp.Key, meta.Span.Key) + require.Equal(t, sp.EndKey, meta.Span.EndKey) + require.Empty(t, meta.ParentStartKey) + } + <-initialScanDone - // Confirm the rangefeed is running on the expected range. - require.NoError(t, db.Put(ctx, mkKey("c"), 2)) - val := <-rows - require.Equal(t, mkKey("c"), val.Key) + // Confirm the rangefeed is running on the expected range. + require.NoError(t, db.Put(ctx, mkKey("c"), 2)) + val := <-rows + require.Equal(t, mkKey("c"), val.Key) - go func() { - // The key above may be emitted again due to rangefeed semantics. Receive - // these repeat updates to ensure the rows channel is never blocked. - for range rows { - } - }() + go func() { + // The key above may be emitted again due to rangefeed semantics. Receive + // these repeat updates to ensure the rows channel is never blocked. + for range rows { + } + }() - splitKey := mkKey("b") - _, _, err = srv.SplitRange(splitKey) - require.NoError(t, err) - for { - // Expect a metadata event from the manual split. - meta := <-metadata - t.Logf("manual split new range key span %s-%s; manual split %t", meta.Span.Key, meta.Span.EndKey, meta.FromManualSplit) - require.Equal(t, true, meta.FromManualSplit) - if !meta.Span.EndKey.Equal(sp.EndKey) { - // New Rangefeed for LHS. - require.Equal(t, splitKey, meta.Span.EndKey) - require.Equal(t, sp.Key, meta.Span.Key) - require.Equal(t, sp.Key, meta.ParentStartKey) - break - } - // Due to outdated rangefeed cache, we could spawn a rangefeed with the og - // span induced by manual split. We expect this rangefeed to error before - // starting with a rangekey mismatch error, which should spawn the correct - // rangefeeds with the manual split flag. + splitKey := mkKey("b") + _, _, err = srv.SplitRange(splitKey) + require.NoError(t, err) + for { + // Expect a metadata event from the manual split. + meta := <-metadata + t.Logf("manual split new range key span %s-%s; manual split %t", meta.Span.Key, meta.Span.EndKey, meta.FromManualSplit) + require.Equal(t, true, meta.FromManualSplit) + if !meta.Span.EndKey.Equal(sp.EndKey) { + // New Rangefeed for LHS. + require.Equal(t, splitKey, meta.Span.EndKey) require.Equal(t, sp.Key, meta.Span.Key) - require.Equal(t, sp.EndKey, meta.Span.EndKey) - } - { - // New Rangefeed for the RHS. - meta := <-metadata - t.Logf("another split new range key span %s-%s; manual %t", meta.Span.Key, meta.Span.EndKey, meta.FromManualSplit) - require.True(t, meta.FromManualSplit) - require.Equal(t, splitKey, meta.Span.Key) - require.Equal(t, sp.EndKey, meta.Span.EndKey) require.Equal(t, sp.Key, meta.ParentStartKey) - } + break + } + // Due to outdated rangefeed cache, we could spawn a rangefeed with the og + // span induced by manual split. We expect this rangefeed to error before + // starting with a rangekey mismatch error, which should spawn the correct + // rangefeeds with the manual split flag. + require.Equal(t, sp.Key, meta.Span.Key) + require.Equal(t, sp.EndKey, meta.Span.EndKey) + } + { + // New Rangefeed for the RHS. + meta := <-metadata + t.Logf("another split new range key span %s-%s; manual %t", meta.Span.Key, meta.Span.EndKey, meta.FromManualSplit) + require.True(t, meta.FromManualSplit) + require.Equal(t, splitKey, meta.Span.Key) + require.Equal(t, sp.EndKey, meta.Span.EndKey) + require.Equal(t, sp.Key, meta.ParentStartKey) + } - }) } // TestRangeFeedMetadataAutoSplit tests that a spawned rangefeed emits metadata @@ -1790,94 +1700,88 @@ func TestRangeFeedMetadataAutoSplit(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - testutils.RunValues(t, "feed_type", procTypes, func(t *testing.T, s feedProcessorType) { - ctx := context.Background() - settings := cluster.MakeTestingClusterSettings() - // We must enable desired scheduler settings before we start cluster, - // otherwise we will trigger processor restarts later and this test can't - // handle duplicated events. - kvserver.RangeFeedUseScheduler.Override(ctx, &settings.SV, s.useScheduler) - kvserver.RangefeedEnabled.Override(ctx, &settings.SV, true) - // Lower the closed timestamp target duration to speed up the test. - closedts.TargetDuration.Override(ctx, &settings.SV, 100*time.Millisecond) - srv, conn, _ := serverutils.StartServer(t, base.TestServerArgs{ - Settings: settings, - }) - sql := sqlutils.MakeSQLRunner(conn) + ctx := context.Background() + settings := cluster.MakeTestingClusterSettings() + kvserver.RangefeedEnabled.Override(ctx, &settings.SV, true) + // Lower the closed timestamp target duration to speed up the test. + closedts.TargetDuration.Override(ctx, &settings.SV, 100*time.Millisecond) + srv, conn, _ := serverutils.StartServer(t, base.TestServerArgs{ + Settings: settings, + }) + sql := sqlutils.MakeSQLRunner(conn) - defer srv.Stopper().Stop(ctx) - ts := srv.ApplicationLayer() + defer srv.Stopper().Stop(ctx) + ts := srv.ApplicationLayer() - var maxTableID uint32 - sql.QueryRow(t, "SELECT max(id) FROM system.namespace").Scan(&maxTableID) - tenantPrefixEnd := srv.Codec().TenantPrefix().PrefixEnd() + var maxTableID uint32 + sql.QueryRow(t, "SELECT max(id) FROM system.namespace").Scan(&maxTableID) + tenantPrefixEnd := srv.Codec().TenantPrefix().PrefixEnd() - // Create a rangefeed that listens to key updates at the highest table in - // system key space and above. - sp := roachpb.Span{ - Key: ts.Codec().TablePrefix(maxTableID), - EndKey: tenantPrefixEnd, - } + // Create a rangefeed that listens to key updates at the highest table in + // system key space and above. + sp := roachpb.Span{ + Key: ts.Codec().TablePrefix(maxTableID), + EndKey: tenantPrefixEnd, + } - // Wait for foo to have its own range. - f, err := rangefeed.NewFactory(ts.AppStopper(), ts.DB(), ts.ClusterSettings(), nil) - require.NoError(t, err) + // Wait for foo to have its own range. + f, err := rangefeed.NewFactory(ts.AppStopper(), ts.DB(), ts.ClusterSettings(), nil) + require.NoError(t, err) - metadata := make(chan *kvpb.RangeFeedMetadata) - initialScanDone := make(chan struct{}) - r, err := f.RangeFeed(ctx, "test", []roachpb.Span{sp}, ts.DB().Clock().Now(), - func(ctx context.Context, value *kvpb.RangeFeedValue) { - }, - rangefeed.WithInitialScan(func(ctx context.Context) { - close(initialScanDone) - }), - rangefeed.WithOnMetadata(func(ctx context.Context, value *kvpb.RangeFeedMetadata) { - select { - case metadata <- value: - case <-ctx.Done(): - return - } - }), - ) + metadata := make(chan *kvpb.RangeFeedMetadata) + initialScanDone := make(chan struct{}) + r, err := f.RangeFeed(ctx, "test", []roachpb.Span{sp}, ts.DB().Clock().Now(), + func(ctx context.Context, value *kvpb.RangeFeedValue) { + }, + rangefeed.WithInitialScan(func(ctx context.Context) { + close(initialScanDone) + }), + rangefeed.WithOnMetadata(func(ctx context.Context, value *kvpb.RangeFeedMetadata) { + select { + case metadata <- value: + case <-ctx.Done(): + return + } + }), + ) - require.NoError(t, err) - defer r.Close() - <-initialScanDone - { - // First meta msg for the new rangefeed. - meta := <-metadata - require.False(t, meta.FromManualSplit) - require.Equal(t, sp.Key, meta.Span.Key) - require.Equal(t, sp.EndKey, meta.Span.EndKey) - require.Empty(t, meta.ParentStartKey) - } + require.NoError(t, err) + defer r.Close() + <-initialScanDone + { + // First meta msg for the new rangefeed. + meta := <-metadata + require.False(t, meta.FromManualSplit) + require.Equal(t, sp.Key, meta.Span.Key) + require.Equal(t, sp.EndKey, meta.Span.EndKey) + require.Empty(t, meta.ParentStartKey) + } - sql.Exec(t, "CREATE TABLE foo (key INT PRIMARY KEY)") - for { - meta := <-metadata - t.Logf("new range key span %s-%s; manual split %t", meta.Span.Key, meta.Span.EndKey, meta.FromManualSplit) - require.False(t, meta.FromManualSplit) - if !meta.Span.EndKey.Equal(sp.EndKey) { - // Verify metadata for LHS rangefeed. - require.Equal(t, sp.Key, meta.Span.Key) - require.Equal(t, sp.Key, meta.ParentStartKey) - break - } - // Due to an outdated rangefeed cache, we could spawn a rangefeed with the - // og span induced by the split. We expect this rangefeed to error before - // starting with a rangekey mismatch error, which should spawn the correct - // rangefeeds with the manual split flag. + sql.Exec(t, "CREATE TABLE foo (key INT PRIMARY KEY)") + for { + meta := <-metadata + t.Logf("new range key span %s-%s; manual split %t", meta.Span.Key, meta.Span.EndKey, meta.FromManualSplit) + require.False(t, meta.FromManualSplit) + if !meta.Span.EndKey.Equal(sp.EndKey) { + // Verify metadata for LHS rangefeed. require.Equal(t, sp.Key, meta.Span.Key) - require.Equal(t, sp.EndKey, meta.Span.EndKey) - require.Equal(t, sp.Key, meta.ParentStartKey) - } - { - // Verify the RHS rangefeed metadata. - meta := <-metadata - require.False(t, meta.FromManualSplit) - require.NotEqual(t, sp.Key, meta.Span.Key) - require.Equal(t, sp.EndKey, meta.Span.EndKey) require.Equal(t, sp.Key, meta.ParentStartKey) - } - }) + break + } + // Due to an outdated rangefeed cache, we could spawn a rangefeed with the + // og span induced by the split. We expect this rangefeed to error before + // starting with a rangekey mismatch error, which should spawn the correct + // rangefeeds with the manual split flag. + require.Equal(t, sp.Key, meta.Span.Key) + require.Equal(t, sp.EndKey, meta.Span.EndKey) + require.Equal(t, sp.Key, meta.ParentStartKey) + } + { + // Verify the RHS rangefeed metadata. + meta := <-metadata + require.False(t, meta.FromManualSplit) + require.NotEqual(t, sp.Key, meta.Span.Key) + require.Equal(t, sp.EndKey, meta.Span.EndKey) + require.Equal(t, sp.Key, meta.ParentStartKey) + } } diff --git a/pkg/kv/kvserver/BUILD.bazel b/pkg/kv/kvserver/BUILD.bazel index 0f272da5b5dc..6377a7074a10 100644 --- a/pkg/kv/kvserver/BUILD.bazel +++ b/pkg/kv/kvserver/BUILD.bazel @@ -397,7 +397,6 @@ go_test( "//pkg/kv", "//pkg/kv/kvclient", "//pkg/kv/kvclient/kvcoord", - "//pkg/kv/kvclient/rangefeed", "//pkg/kv/kvclient/rangefeed/rangefeedcache", "//pkg/kv/kvpb", "//pkg/kv/kvserver/abortspan", diff --git a/pkg/kv/kvserver/rangefeed/BUILD.bazel b/pkg/kv/kvserver/rangefeed/BUILD.bazel index 2bbf34686ace..1afb9d85f741 100644 --- a/pkg/kv/kvserver/rangefeed/BUILD.bazel +++ b/pkg/kv/kvserver/rangefeed/BUILD.bazel @@ -21,7 +21,6 @@ go_library( "stream_manager.go", "task.go", "test_helpers.go", - "testutil.go", "unbuffered_registration.go", "unbuffered_sender.go", ], diff --git a/pkg/kv/kvserver/rangefeed/processor.go b/pkg/kv/kvserver/rangefeed/processor.go index c79eafaf8b56..a803f9f5da66 100644 --- a/pkg/kv/kvserver/rangefeed/processor.go +++ b/pkg/kv/kvserver/rangefeed/processor.go @@ -7,7 +7,6 @@ package rangefeed import ( "context" - "fmt" "sync" "time" @@ -20,7 +19,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/stop" - "github.com/cockroachdb/errors" ) var ( @@ -79,13 +77,6 @@ type Config struct { Span roachpb.RSpan TxnPusher TxnPusher - // PushTxnsInterval specifies the interval at which a Processor will push - // all transactions in the unresolvedIntentQueue that are above the age - // specified by PushTxnsAge. - // - // This option only applies to LegacyProcessor since ScheduledProcessor is - // relying on store to push events to scheduler to initiate transaction push. - PushTxnsInterval time.Duration // PushTxnsAge specifies the age at which a Processor will begin to consider // a transaction old enough to push. PushTxnsAge time.Duration @@ -126,16 +117,10 @@ func (sc *Config) SetDefaults() { // Some tests don't set the TxnPusher, so we avoid setting a default push txn // interval in such cases #121429. if sc.TxnPusher == nil { - if sc.PushTxnsInterval != 0 { - panic("nil TxnPusher with non-zero PushTxnsInterval") - } if sc.PushTxnsAge != 0 { panic("nil TxnPusher with non-zero PushTxnsAge") } } else { - if sc.PushTxnsInterval == 0 { - sc.PushTxnsInterval = DefaultPushTxnsInterval - } if sc.PushTxnsAge == 0 { sc.PushTxnsAge = defaultPushTxnsAge } @@ -256,27 +241,7 @@ type Processor interface { func NewProcessor(cfg Config) Processor { cfg.SetDefaults() cfg.AmbientContext.AddLogTag("rangefeed", nil) - if cfg.Scheduler != nil { - return NewScheduledProcessor(cfg) - } - return NewLegacyProcessor(cfg) -} - -type LegacyProcessor struct { - Config - reg registry - rts resolvedTimestamp - - regC chan *bufferedRegistration - unregC chan *bufferedRegistration - lenReqC chan struct{} - lenResC chan int - filterReqC chan struct{} - filterResC chan *Filter - eventC chan *event - spanErrC chan spanErr - stopC chan *kvpb.Error - stoppedC chan struct{} + return NewScheduledProcessor(cfg) } var eventSyncPool = sync.Pool{ @@ -346,643 +311,11 @@ type logicalOpMetadata struct { originID uint32 } -func NewLegacyProcessor(cfg Config) *LegacyProcessor { - p := &LegacyProcessor{ - Config: cfg, - reg: makeRegistry(cfg.Metrics), - rts: makeResolvedTimestamp(cfg.Settings), - - regC: make(chan *bufferedRegistration), - unregC: make(chan *bufferedRegistration), - lenReqC: make(chan struct{}), - lenResC: make(chan int), - filterReqC: make(chan struct{}), - filterResC: make(chan *Filter), - eventC: make(chan *event, cfg.EventChanCap), - spanErrC: make(chan spanErr), - stopC: make(chan *kvpb.Error, 1), - stoppedC: make(chan struct{}), - } - return p -} - // IntentScannerConstructor is used to construct an IntentScanner. It // should be called from underneath a stopper task to ensure that the // engine has not been closed. type IntentScannerConstructor func() IntentScanner -// Start implements Processor interface. -// -// LegacyProcessor launches a goroutine to process rangefeed events and send -// them to registrations. -// -// Note that to fulfill newRtsIter contract, LegacyProcessor will create -// iterator at the start of its work loop prior to firing async task. -func (p *LegacyProcessor) Start(stopper *stop.Stopper, newRtsIter IntentScannerConstructor) error { - ctx := p.AnnotateCtx(context.Background()) - if err := stopper.RunAsyncTask(ctx, "rangefeed.LegacyProcessor", func(ctx context.Context) { - p.Metrics.RangeFeedProcessorsGO.Inc(1) - defer p.Metrics.RangeFeedProcessorsGO.Dec(1) - p.run(ctx, p.RangeID, newRtsIter, stopper) - }); err != nil { - p.reg.DisconnectWithErr(ctx, all, kvpb.NewError(err)) - close(p.stoppedC) - return err - } - return nil -} - -// run is called from Start and runs the rangefeed. -func (p *LegacyProcessor) run( - ctx context.Context, - _forStacks roachpb.RangeID, - rtsIterFunc IntentScannerConstructor, - stopper *stop.Stopper, -) { - // Close the memory budget last, or there will be a period of time during - // which requests are still ongoing but will run into the closed budget, - // causing shutdown noise and busy retries. - // Closing the budget after stoppedC ensures that all other goroutines are - // (very close to being) shut down by the time the budget goes away. - defer p.MemBudget.Close(ctx) - defer close(p.stoppedC) - ctx, cancelOutputLoops := context.WithCancel(ctx) - defer cancelOutputLoops() - - // Launch an async task to scan over the resolved timestamp iterator and - // initialize the unresolvedIntentQueue. Ignore error if quiescing. - if rtsIterFunc != nil { - rtsIter := rtsIterFunc() - initScan := newInitResolvedTSScan(p.Span, p, rtsIter) - err := stopper.RunAsyncTask(ctx, "rangefeed: init resolved ts", initScan.Run) - if err != nil { - initScan.Cancel() - } - } else { - p.initResolvedTS(ctx) - } - - // txnPushTicker periodically pushes the transaction record of all - // unresolved intents that are above a certain age, helping to ensure - // that the resolved timestamp continues to make progress. - var txnPushTicker *time.Ticker - var txnPushTickerC <-chan time.Time - var txnPushAttemptC chan struct{} - if p.PushTxnsInterval > 0 { - txnPushTicker = time.NewTicker(p.PushTxnsInterval) - txnPushTickerC = txnPushTicker.C - defer txnPushTicker.Stop() - } - - for { - select { - - // Handle new registrations. - case r := <-p.regC: - if !p.Span.AsRawSpanWithNoLocals().Contains(r.span) { - log.Fatalf(ctx, "registration %s not in Processor's key range %v", r, p.Span) - } - - // Add the new registration to the registry. - p.reg.Register(ctx, r) - - // Publish an updated filter that includes the new registration. - p.filterResC <- p.reg.NewFilter() - - // Immediately publish a checkpoint event to the registry. This will be the first event - // published to this registration after its initial catch-up scan completes. The resolved - // timestamp might be empty but the checkpoint event is still useful to indicate that the - // catch-up scan has completed. This allows clients to rely on stronger ordering semantics - // once they observe the first checkpoint event. - r.publish(ctx, p.newCheckpointEvent(), nil) - - // Run an output loop for the registry. - runOutputLoop := func(ctx context.Context) { - r.runOutputLoop(ctx, p.RangeID) - select { - case p.unregC <- r: - if r.unreg != nil { - r.unreg() - } - case <-p.stoppedC: - } - } - if err := stopper.RunAsyncTask(ctx, "rangefeed: output loop", runOutputLoop); err != nil { - r.Disconnect(kvpb.NewError(err)) - p.reg.Unregister(ctx, r) - } - - // Respond to unregistration requests; these come from registrations that - // encounter an error during their output loop. - case r := <-p.unregC: - p.reg.Unregister(ctx, r) - - // Send errors to registrations overlapping the span and disconnect them. - // Requested via DisconnectSpanWithErr(). - case e := <-p.spanErrC: - p.reg.DisconnectWithErr(ctx, e.span, e.pErr) - - // Respond to answers about the processor goroutine state. - case <-p.lenReqC: - p.lenResC <- p.reg.Len() - - // Respond to answers about which operations can be filtered before - // reaching the Processor. - case <-p.filterReqC: - p.filterResC <- p.reg.NewFilter() - - // Transform and route events. - case e := <-p.eventC: - p.consumeEvent(ctx, e) - e.alloc.Release(ctx) - putPooledEvent(e) - - // Check whether any unresolved intents need a push. - case <-txnPushTickerC: - // Don't perform transaction push attempts if disabled, until the resolved - // timestamp has been initialized, or if we're not tracking any intents. - if !PushTxnsEnabled.Get(&p.Settings.SV) || !p.rts.IsInit() || p.rts.intentQ.Len() == 0 { - continue - } - - now := p.Clock.Now() - before := now.Add(-p.PushTxnsAge.Nanoseconds(), 0) - oldTxns := p.rts.intentQ.Before(before) - - if len(oldTxns) > 0 { - toPush := make([]enginepb.TxnMeta, len(oldTxns)) - for i, txn := range oldTxns { - toPush[i] = txn.asTxnMeta() - } - - // Set the ticker channel to nil so that it can't trigger a - // second concurrent push. Create a push attempt response - // channel that is closed when the push attempt completes. - txnPushTickerC = nil - txnPushAttemptC = make(chan struct{}) - - // Launch an async transaction push attempt that pushes the - // timestamp of all transactions beneath the push offset. - // Ignore error if quiescing. - pushTxns := newTxnPushAttempt(p.Settings, p.Span, p.TxnPusher, p, toPush, now, func() { - close(txnPushAttemptC) - }) - err := stopper.RunAsyncTask(ctx, "rangefeed: pushing old txns", pushTxns.Run) - if err != nil { - pushTxns.Cancel() - } - } - - // Update the resolved timestamp based on the push attempt. - case <-txnPushAttemptC: - // Reset the ticker channel so that it can trigger push attempts - // again. Set the push attempt channel back to nil. - txnPushTickerC = txnPushTicker.C - txnPushAttemptC = nil - - // Close registrations and exit when signaled. - case pErr := <-p.stopC: - p.reg.DisconnectAllOnShutdown(ctx, pErr) - return - - // Exit on stopper. - case <-stopper.ShouldQuiesce(): - pErr := kvpb.NewError(&kvpb.NodeUnavailableError{}) - p.reg.DisconnectAllOnShutdown(ctx, pErr) - return - } - } -} - -// Stop implements Processor interface. -func (p *LegacyProcessor) Stop() { - p.StopWithErr(nil) -} - -// StopWithErr implements Processor interface. -func (p *LegacyProcessor) StopWithErr(pErr *kvpb.Error) { - // Flush any remaining events before stopping. - p.syncEventC() - // Send the processor a stop signal. - p.sendStop(pErr) -} - -// DisconnectSpanWithErr implements Processor interface. -func (p *LegacyProcessor) DisconnectSpanWithErr(span roachpb.Span, pErr *kvpb.Error) { - select { - case p.spanErrC <- spanErr{span: span, pErr: pErr}: - case <-p.stoppedC: - // Already stopped. Do nothing. - } -} - -func (p *LegacyProcessor) sendStop(pErr *kvpb.Error) { - select { - case p.stopC <- pErr: - // stopC has non-zero capacity so this should not block unless - // multiple callers attempt to stop the Processor concurrently. - case <-p.stoppedC: - // Already stopped. Do nothing. - } -} - -// Register implements Processor interface. -func (p *LegacyProcessor) Register( - streamCtx context.Context, - span roachpb.RSpan, - startTS hlc.Timestamp, - catchUpIter *CatchUpIterator, - withDiff bool, - withFiltering bool, - withOmitRemote bool, - stream Stream, - disconnectFn func(), -) (bool, Disconnector, *Filter) { - // Synchronize the event channel so that this registration doesn't see any - // events that were consumed before this registration was called. Instead, - // it should see these events during its catch up scan. - p.syncEventC() - - blockWhenFull := p.Config.EventChanTimeout == 0 // for testing - r := newBufferedRegistration( - span.AsRawSpanWithNoLocals(), startTS, catchUpIter, withDiff, withFiltering, withOmitRemote, - p.Config.EventChanCap, blockWhenFull, p.Metrics, stream, disconnectFn, func(r registration) { - p.reg.Unregister(context.Background(), r) - }, - ) - select { - case p.regC <- r: - // Wait for response. - f := <-p.filterResC - return true, r, f - case <-p.stoppedC: - return false, nil, nil - } -} - -// Len implements Processor interface. -func (p *LegacyProcessor) Len() int { - // Ask the processor goroutine. - select { - case p.lenReqC <- struct{}{}: - // Wait for response. - return <-p.lenResC - case <-p.stoppedC: - return 0 - } -} - -// Filter implements Processor interface. -func (p *LegacyProcessor) Filter() *Filter { - // Ask the processor goroutine. - select { - case p.filterReqC <- struct{}{}: - // Wait for response. - return <-p.filterResC - case <-p.stoppedC: - return nil - } -} - -// ConsumeLogicalOps implements Processor interface. -func (p *LegacyProcessor) ConsumeLogicalOps( - ctx context.Context, ops ...enginepb.MVCCLogicalOp, -) bool { - if len(ops) == 0 { - return true - } - return p.sendEvent(ctx, event{ops: ops}, p.EventChanTimeout) -} - -// ConsumeSSTable implements Processor interface. -func (p *LegacyProcessor) ConsumeSSTable( - ctx context.Context, sst []byte, sstSpan roachpb.Span, writeTS hlc.Timestamp, -) bool { - return p.sendEvent(ctx, event{sst: &sstEvent{sst, sstSpan, writeTS}}, p.EventChanTimeout) -} - -// ForwardClosedTS implements Processor interface. -func (p *LegacyProcessor) ForwardClosedTS(ctx context.Context, closedTS hlc.Timestamp) bool { - if closedTS.IsEmpty() { - return true - } - return p.sendEvent(ctx, event{ct: ctEvent{closedTS}}, p.EventChanTimeout) -} - -// sendEvent informs the Processor of a new event. If a timeout is specified, -// the method will wait for no longer than that duration before giving up, -// shutting down the Processor, and returning false. 0 for no timeout. -func (p *LegacyProcessor) sendEvent(ctx context.Context, e event, timeout time.Duration) bool { - // The code is a bit unwieldy because we try to avoid any allocations on fast - // path where we have enough budget and outgoing channel is free. If not, we - // try to set up timeout for acquiring budget and then reuse this timeout when - // inserting value into channel. - var alloc *SharedBudgetAllocation - if p.MemBudget != nil { - size := calculateDateEventSize(e) - if size > 0 { - var err error - // First we will try non-blocking fast path to allocate memory budget. - alloc, err = p.MemBudget.TryGet(ctx, size) - // If budget is already closed, then just let it through because processor - // is terminating. - if err != nil && !errors.Is(err, budgetClosedError) { - // Since we don't have enough budget, we should try to wait for - // allocation returns before failing. - if timeout > 0 { - var cancel context.CancelFunc - ctx, cancel = context.WithTimeout(ctx, timeout) // nolint:context - defer cancel() - // We reset timeout here so that subsequent channel write op doesn't - // try to wait beyond what is already set up. - timeout = 0 - } - p.Metrics.RangeFeedBudgetBlocked.Inc(1) - alloc, err = p.MemBudget.WaitAndGet(ctx, size) - } - if err != nil && !errors.Is(err, budgetClosedError) { - p.Metrics.RangeFeedBudgetExhausted.Inc(1) - p.sendStop(newErrBufferCapacityExceeded()) - return false - } - // Always release allocation pointer after sending as it is nil safe. - // In normal case its value is moved into event, in case of allocation - // errors it is nil, in case of send errors it is non-nil and this call - // ensures that unused allocation is released. - defer func() { - alloc.Release(ctx) - }() - } - } - ev := getPooledEvent(e) - ev.alloc = alloc - if timeout == 0 { - // Timeout is zero if no timeout was requested or timeout is already set on - // the context by budget allocation. Just try to write using context as a - // timeout. - select { - case p.eventC <- ev: - // Reset allocation after successful posting to prevent deferred cleanup - // from freeing it (see comment on defer for explanation). - alloc = nil - case <-p.stoppedC: - // Already stopped. Do nothing. - case <-ctx.Done(): - p.sendStop(newErrBufferCapacityExceeded()) - return false - } - } else { - // First try fast path operation without blocking and without creating any - // contexts in case channel has capacity. - select { - case p.eventC <- ev: - // Reset allocation after successful posting to prevent deferred cleanup - // from freeing it (see comment on defer for explanation). - alloc = nil - case <-p.stoppedC: - // Already stopped. Do nothing. - default: - // Fast path failed since we don't have capacity in channel. Wait for - // slots to clear up using context timeout. - var cancel context.CancelFunc - ctx, cancel = context.WithTimeout(ctx, timeout) // nolint:context - defer cancel() - select { - case p.eventC <- ev: - // Reset allocation after successful posting to prevent deferred cleanup - // from freeing it (see comment on defer for explanation). - alloc = nil - case <-p.stoppedC: - // Already stopped. Do nothing. - case <-ctx.Done(): - // Sending on the eventC channel would have blocked. - // Instead, tear down the processor and return immediately. - p.sendStop(newErrBufferCapacityExceeded()) - return false - } - } - } - return true -} - -// setResolvedTSInitialized informs the Processor that its resolved timestamp has -// all the information it needs to be considered initialized. -func (p *LegacyProcessor) setResolvedTSInitialized(ctx context.Context) { - p.sendEvent(ctx, event{initRTS: true}, 0) -} - -// syncEventC synchronizes access to the Processor goroutine, allowing the -// caller to establish causality with actions taken by the Processor goroutine. -// It does so by flushing the event pipeline. -func (p *LegacyProcessor) syncEventC() { - p.syncEventCWithEvent(&syncEvent{c: make(chan struct{})}) -} - -// syncEventCWithEvent allows sync event to be sent and waited on its channel. -// Exposed to allow special test syncEvents that contain span to be sent. -func (p *LegacyProcessor) syncEventCWithEvent(se *syncEvent) { - ev := getPooledEvent(event{sync: se}) - select { - case p.eventC <- ev: - select { - case <-se.c: - // Synchronized. - case <-p.stoppedC: - // Already stopped. Do nothing. - } - case <-p.stoppedC: - // Already stopped. Return event back to the pool. - putPooledEvent(ev) - } -} - -func (p *LegacyProcessor) consumeEvent(ctx context.Context, e *event) { - switch { - case e.ops != nil: - p.consumeLogicalOps(ctx, e.ops, e.alloc) - case !e.ct.IsEmpty(): - p.forwardClosedTS(ctx, e.ct.Timestamp) - case bool(e.initRTS): - p.initResolvedTS(ctx) - case e.sst != nil: - p.consumeSSTable(ctx, e.sst.data, e.sst.span, e.sst.ts, e.alloc) - case e.sync != nil: - if e.sync.testRegCatchupSpan != nil { - if err := p.reg.waitForCaughtUp(ctx, *e.sync.testRegCatchupSpan); err != nil { - log.Errorf( - ctx, - "error waiting for registries to catch up during test, results might be impacted: %s", - err, - ) - } - } - close(e.sync.c) - default: - panic(fmt.Sprintf("missing event variant: %+v", e)) - } -} - -func (p *LegacyProcessor) consumeLogicalOps( - ctx context.Context, ops []enginepb.MVCCLogicalOp, alloc *SharedBudgetAllocation, -) { - for _, op := range ops { - // Publish RangeFeedValue updates, if necessary. - switch t := op.GetValue().(type) { - // OmitInRangefeeds is relevant only for transactional writes, so it's - // propagated only in the case of a MVCCCommitIntentOp and - // MVCCWriteValueOp (could be the result of a 1PC write). - case *enginepb.MVCCWriteValueOp: - // Publish the new value directly. - p.publishValue(ctx, t.Key, t.Timestamp, t.Value, t.PrevValue, logicalOpMetadata{omitInRangefeeds: t.OmitInRangefeeds, originID: t.OriginID}, alloc) - - case *enginepb.MVCCDeleteRangeOp: - // Publish the range deletion directly. - p.publishDeleteRange(ctx, t.StartKey, t.EndKey, t.Timestamp, alloc) - - case *enginepb.MVCCWriteIntentOp: - // No updates to publish. - - case *enginepb.MVCCUpdateIntentOp: - // No updates to publish. - - case *enginepb.MVCCCommitIntentOp: - // Publish the newly committed value. - p.publishValue(ctx, t.Key, t.Timestamp, t.Value, t.PrevValue, logicalOpMetadata{omitInRangefeeds: t.OmitInRangefeeds, originID: t.OriginID}, alloc) - - case *enginepb.MVCCAbortIntentOp: - // No updates to publish. - - case *enginepb.MVCCAbortTxnOp: - // No updates to publish. - - default: - panic(errors.AssertionFailedf("unknown logical op %T", t)) - } - - // Determine whether the operation caused the resolved timestamp to - // move forward. If so, publish a RangeFeedCheckpoint notification. - if p.rts.ConsumeLogicalOp(ctx, op) { - p.publishCheckpoint(ctx) - } - } -} - -func (p *LegacyProcessor) consumeSSTable( - ctx context.Context, - sst []byte, - sstSpan roachpb.Span, - sstWTS hlc.Timestamp, - alloc *SharedBudgetAllocation, -) { - p.publishSSTable(ctx, sst, sstSpan, sstWTS, alloc) -} - -func (p *LegacyProcessor) forwardClosedTS(ctx context.Context, newClosedTS hlc.Timestamp) { - if p.rts.ForwardClosedTS(ctx, newClosedTS) { - p.publishCheckpoint(ctx) - } -} - -func (p *LegacyProcessor) initResolvedTS(ctx context.Context) { - if p.rts.Init(ctx) { - p.publishCheckpoint(ctx) - } -} - -func (p *LegacyProcessor) publishValue( - ctx context.Context, - key roachpb.Key, - timestamp hlc.Timestamp, - value, prevValue []byte, - valueMetadata logicalOpMetadata, - alloc *SharedBudgetAllocation, -) { - if !p.Span.ContainsKey(roachpb.RKey(key)) { - log.Fatalf(ctx, "key %v not in Processor's key range %v", key, p.Span) - } - - var prevVal roachpb.Value - if prevValue != nil { - prevVal.RawBytes = prevValue - } - var event kvpb.RangeFeedEvent - event.MustSetValue(&kvpb.RangeFeedValue{ - Key: key, - Value: roachpb.Value{ - RawBytes: value, - Timestamp: timestamp, - }, - PrevValue: prevVal, - }) - p.reg.PublishToOverlapping(ctx, roachpb.Span{Key: key}, &event, valueMetadata, alloc) -} - -func (p *LegacyProcessor) publishDeleteRange( - ctx context.Context, - startKey, endKey roachpb.Key, - timestamp hlc.Timestamp, - alloc *SharedBudgetAllocation, -) { - span := roachpb.Span{Key: startKey, EndKey: endKey} - if !p.Span.ContainsKeyRange(roachpb.RKey(startKey), roachpb.RKey(endKey)) { - log.Fatalf(ctx, "span %s not in Processor's key range %v", span, p.Span) - } - - var event kvpb.RangeFeedEvent - event.MustSetValue(&kvpb.RangeFeedDeleteRange{ - Span: span, - Timestamp: timestamp, - }) - p.reg.PublishToOverlapping(ctx, span, &event, logicalOpMetadata{}, alloc) -} - -func (p *LegacyProcessor) publishSSTable( - ctx context.Context, - sst []byte, - sstSpan roachpb.Span, - sstWTS hlc.Timestamp, - alloc *SharedBudgetAllocation, -) { - if sstSpan.Equal(roachpb.Span{}) { - panic(errors.AssertionFailedf("received SSTable without span")) - } - if sstWTS.IsEmpty() { - panic(errors.AssertionFailedf("received SSTable without write timestamp")) - } - p.reg.PublishToOverlapping(ctx, sstSpan, &kvpb.RangeFeedEvent{ - SST: &kvpb.RangeFeedSSTable{ - Data: sst, - Span: sstSpan, - WriteTS: sstWTS, - }, - }, logicalOpMetadata{}, alloc) -} - -func (p *LegacyProcessor) publishCheckpoint(ctx context.Context) { - // TODO(nvanbenschoten): persist resolvedTimestamp. Give Processor a client.DB. - // TODO(nvanbenschoten): rate limit these? send them periodically? - - event := p.newCheckpointEvent() - p.reg.PublishToOverlapping(ctx, all, event, logicalOpMetadata{}, nil) -} - -func (p *LegacyProcessor) newCheckpointEvent() *kvpb.RangeFeedEvent { - // Create a RangeFeedCheckpoint over the Processor's entire span. Each - // individual registration will trim this down to just the key span that - // it is listening on in registration.maybeStripEvent before publishing. - var event kvpb.RangeFeedEvent - event.MustSetValue(&kvpb.RangeFeedCheckpoint{ - Span: p.Span.AsRawSpanWithNoLocals(), - ResolvedTS: p.rts.Get(), - }) - return &event -} - -// ID implements Processor interface. -func (p *LegacyProcessor) ID() int64 { - return 0 -} - // calculateDateEventSize returns estimated size of the event that contain actual // data. We only account for logical ops and sst's. Those events come from raft // and are budgeted. Other events come from processor jobs and update timestamps diff --git a/pkg/kv/kvserver/rangefeed/processor_helpers_test.go b/pkg/kv/kvserver/rangefeed/processor_helpers_test.go index b87963eaf421..7ee13f73c0b9 100644 --- a/pkg/kv/kvserver/rangefeed/processor_helpers_test.go +++ b/pkg/kv/kvserver/rangefeed/processor_helpers_test.go @@ -262,7 +262,6 @@ func (h *processorTestHelper) triggerTxnPushUntilPushed(t *testing.T, pushedC <- type procType bool const ( - legacy procType = false scheduler procType = true ) @@ -272,11 +271,6 @@ type rangefeedTestType struct { } var ( - legacyProcessor = rangefeedTestType{ - processorType: legacy, - regType: buffered, - } - scheduledProcessorWithUnbufferedReg = rangefeedTestType{ processorType: scheduler, regType: unbuffered, @@ -288,7 +282,6 @@ var ( } testTypes = []rangefeedTestType{ - legacyProcessor, scheduledProcessorWithUnbufferedReg, scheduledProcessorWithBufferedReg, } @@ -296,8 +289,6 @@ var ( func (t rangefeedTestType) String() string { switch t { - case legacyProcessor: - return "legacy" case scheduledProcessorWithBufferedReg: return "scheduled processor with buffered registration" case scheduledProcessorWithUnbufferedReg: @@ -316,7 +307,6 @@ type option func(*testConfig) func withPusher(txnPusher TxnPusher) option { return func(config *testConfig) { - config.PushTxnsInterval = 10 * time.Millisecond config.PushTxnsAge = 50 * time.Millisecond config.TxnPusher = txnPusher } @@ -380,9 +370,8 @@ func withSettings(st *cluster.Settings) option { } } -func withPushTxnsIntervalAge(interval, age time.Duration) option { +func withPushTxnsAge(age time.Duration) option { return func(config *testConfig) { - config.PushTxnsInterval = interval config.PushTxnsAge = age } } @@ -458,38 +447,25 @@ func newTestProcessor( for _, o := range opts { o(&cfg) } - if cfg.feedType != legacyProcessor { - sch := NewScheduler(SchedulerConfig{ - Workers: 1, - PriorityWorkers: 1, - Metrics: NewSchedulerMetrics(time.Second), - }) - require.NoError(t, sch.Start(context.Background(), stopper)) - cfg.Scheduler = sch - // Also create a dummy priority processor to populate priorityIDs for - // BenchmarkRangefeed. It should never be called. - noop := func(e processorEventType) processorEventType { - if e != Stopped { - t.Errorf("unexpected event %s for noop priority processor", e) - } - return 0 + sch := NewScheduler(SchedulerConfig{ + Workers: 1, + PriorityWorkers: 1, + Metrics: NewSchedulerMetrics(time.Second), + }) + require.NoError(t, sch.Start(context.Background(), stopper)) + cfg.Scheduler = sch + // Also create a dummy priority processor to populate priorityIDs for + // BenchmarkRangefeed. It should never be called. + noop := func(e processorEventType) processorEventType { + if e != Stopped { + t.Errorf("unexpected event %s for noop priority processor", e) } - require.NoError(t, sch.register(9, noop, true /* priority */)) + return 0 } + require.NoError(t, sch.register(9, noop, true /* priority */)) s := NewProcessor(cfg.Config) h := processorTestHelper{} switch p := s.(type) { - case *LegacyProcessor: - h.rts = &p.rts - h.span = p.Span - h.syncEventC = p.syncEventC - h.sendSpanSync = func(span *roachpb.Span) { - p.syncEventCWithEvent(&syncEvent{c: make(chan struct{}), testRegCatchupSpan: span}) - } - h.toBufferedStreamIfNeeded = func(s Stream) Stream { - // Legacy processor does not support buffered streams. - return s - } case *ScheduledProcessor: h.rts = &p.rts h.span = p.Span diff --git a/pkg/kv/kvserver/rangefeed/processor_test.go b/pkg/kv/kvserver/rangefeed/processor_test.go index 72ba5d816203..b4b8e316eba2 100644 --- a/pkg/kv/kvserver/rangefeed/processor_test.go +++ b/pkg/kv/kvserver/rangefeed/processor_test.go @@ -109,38 +109,17 @@ func TestProcessorBasic(t *testing.T) { // Test value with one registration. p.ConsumeLogicalOps(ctx, - writeValueOpWithKV(roachpb.Key("c"), hlc.Timestamp{WallTime: 6}, []byte("val"))) - h.syncEventAndRegistrations() - require.Equal(t, - []*kvpb.RangeFeedEvent{ - rangeFeedValue( - roachpb.Key("c"), - roachpb.Value{ - RawBytes: []byte("val"), - Timestamp: hlc.Timestamp{WallTime: 6}, - }, - ), - }, - r1Stream.Events(), - ) - - // Test value to non-overlapping key with one registration. - p.ConsumeLogicalOps(ctx, - writeValueOpWithKV(roachpb.Key("s"), hlc.Timestamp{WallTime: 6}, []byte("val"))) - h.syncEventAndRegistrations() - require.Equal(t, []*kvpb.RangeFeedEvent(nil), r1Stream.Events()) - - // Test intent that is aborted with one registration. - txn1 := uuid.MakeV4() - // Write intent. - p.ConsumeLogicalOps(ctx, writeIntentOp(txn1, hlc.Timestamp{WallTime: 6})) - h.syncEventAndRegistrations() - require.Equal(t, []*kvpb.RangeFeedEvent(nil), r1Stream.Events()) - // Abort. - p.ConsumeLogicalOps(ctx, abortIntentOp(txn1)) + writeValueOp(hlc.Timestamp{WallTime: 1}), + writeIntentOp(txn1, hlc.Timestamp{WallTime: 2}), + updateIntentOp(txn1, hlc.Timestamp{WallTime: 3}), + commitIntentOp(txn1, hlc.Timestamp{WallTime: 4}), + writeIntentOp(txn2, hlc.Timestamp{WallTime: 5}), + abortIntentOp(txn2)) h.syncEventC() - require.Equal(t, []*kvpb.RangeFeedEvent(nil), r1Stream.Events()) require.Equal(t, 0, h.rts.intentQ.Len()) + }) + require.NotPanics(t, func() { p.ForwardClosedTS(ctx, hlc.Timestamp{}) }) + require.NotPanics(t, func() { p.ForwardClosedTS(ctx, hlc.Timestamp{WallTime: 1}) }) // Test intent that is committed with one registration. txn2 := uuid.MakeV4() @@ -238,76 +217,213 @@ func TestProcessorBasic(t *testing.T) { chEventAM := []*kvpb.RangeFeedEvent{ rangeFeedCheckpoint( roachpb.Span{Key: roachpb.Key("a"), EndKey: roachpb.Key("m")}, - hlc.Timestamp{WallTime: 20}, + hlc.Timestamp{WallTime: 1}, ), - } - require.Equal(t, chEventAM, r1Stream.Events()) - chEventCZ := []*kvpb.RangeFeedEvent{ + }, + r1Stream.Events(), + ) + + // Test the processor's operation filter. + require.True(t, r1Filter.NeedVal(roachpb.Span{Key: roachpb.Key("a")})) + require.True(t, r1Filter.NeedVal(roachpb.Span{Key: roachpb.Key("d"), EndKey: roachpb.Key("r")})) + require.False(t, r1Filter.NeedVal(roachpb.Span{Key: roachpb.Key("z")})) + require.False(t, r1Filter.NeedPrevVal(roachpb.Span{Key: roachpb.Key("a")})) + require.False(t, + r1Filter.NeedPrevVal(roachpb.Span{Key: roachpb.Key("d"), EndKey: roachpb.Key("r")})) + require.False(t, r1Filter.NeedPrevVal(roachpb.Span{Key: roachpb.Key("z")})) + + // Test checkpoint with one registration. + p.ForwardClosedTS(ctx, hlc.Timestamp{WallTime: 5}) + h.syncEventAndRegistrations() + require.Equal(t, + []*kvpb.RangeFeedEvent{ rangeFeedCheckpoint( - roachpb.Span{Key: roachpb.Key("c"), EndKey: roachpb.Key("z")}, - hlc.Timestamp{WallTime: 20}, + roachpb.Span{Key: roachpb.Key("a"), EndKey: roachpb.Key("m")}, + hlc.Timestamp{WallTime: 5}, ), - } - require.Equal(t, chEventCZ, r2Stream.Events()) + }, + r1Stream.Events(), + ) - // Test value with two registration that overlaps both. - p.ConsumeLogicalOps(ctx, - writeValueOpWithKV(roachpb.Key("k"), hlc.Timestamp{WallTime: 22}, []byte("val2"))) - h.syncEventAndRegistrations() - valEvent := []*kvpb.RangeFeedEvent{ + // Test value with one registration. + p.ConsumeLogicalOps(ctx, + writeValueOpWithKV(roachpb.Key("c"), hlc.Timestamp{WallTime: 6}, []byte("val"))) + h.syncEventAndRegistrations() + require.Equal(t, + []*kvpb.RangeFeedEvent{ rangeFeedValue( - roachpb.Key("k"), + roachpb.Key("c"), roachpb.Value{ - RawBytes: []byte("val2"), - Timestamp: hlc.Timestamp{WallTime: 22}, + RawBytes: []byte("val"), + Timestamp: hlc.Timestamp{WallTime: 6}, }, ), - } - require.Equal(t, valEvent, r1Stream.Events()) - require.Equal(t, valEvent, r2Stream.Events()) + }, + r1Stream.Events(), + ) - // Test value that only overlaps the second registration. - p.ConsumeLogicalOps(ctx, - writeValueOpWithKV(roachpb.Key("v"), hlc.Timestamp{WallTime: 23}, []byte("val3"))) - h.syncEventAndRegistrations() - valEvent2 := []*kvpb.RangeFeedEvent{ - rangeFeedValue( - roachpb.Key("v"), - roachpb.Value{ - RawBytes: []byte("val3"), - Timestamp: hlc.Timestamp{WallTime: 23}, - }, - ), - } - require.Equal(t, []*kvpb.RangeFeedEvent(nil), r1Stream.Events()) - require.Equal(t, valEvent2, r2Stream.Events()) + // Test value to non-overlapping key with one registration. + p.ConsumeLogicalOps(ctx, + writeValueOpWithKV(roachpb.Key("s"), hlc.Timestamp{WallTime: 6}, []byte("val"))) + h.syncEventAndRegistrations() + require.Equal(t, []*kvpb.RangeFeedEvent(nil), r1Stream.Events()) - // Test committing intent with OmitInRangefeeds that overlaps two - // registration (one withFiltering = true and one withFiltering = false). - p.ConsumeLogicalOps(ctx, - commitIntentOpWithKV(txn2, roachpb.Key("k"), hlc.Timestamp{WallTime: 22}, - []byte("val3"), true /* omitInRangefeeds */, 0 /* originID */)) - h.syncEventAndRegistrations() - valEvent3 := []*kvpb.RangeFeedEvent{ + // Test intent that is aborted with one registration. + txn1 := uuid.MakeV4() + // Write intent. + p.ConsumeLogicalOps(ctx, writeIntentOp(txn1, hlc.Timestamp{WallTime: 6})) + h.syncEventAndRegistrations() + require.Equal(t, []*kvpb.RangeFeedEvent(nil), r1Stream.Events()) + // Abort. + p.ConsumeLogicalOps(ctx, abortIntentOp(txn1)) + h.syncEventC() + require.Equal(t, []*kvpb.RangeFeedEvent(nil), r1Stream.Events()) + require.Equal(t, 0, h.rts.intentQ.Len()) + + // Test intent that is committed with one registration. + txn2 := uuid.MakeV4() + // Write intent. + p.ConsumeLogicalOps(ctx, writeIntentOp(txn2, hlc.Timestamp{WallTime: 10})) + h.syncEventAndRegistrations() + require.Equal(t, []*kvpb.RangeFeedEvent(nil), r1Stream.Events()) + // Forward closed timestamp. Should now be stuck on intent. + p.ForwardClosedTS(ctx, hlc.Timestamp{WallTime: 15}) + h.syncEventAndRegistrations() + require.Equal(t, + []*kvpb.RangeFeedEvent{ + rangeFeedCheckpoint( + roachpb.Span{Key: roachpb.Key("a"), EndKey: roachpb.Key("m")}, + hlc.Timestamp{WallTime: 9}, + ), + }, + r1Stream.Events(), + ) + // Update the intent. Should forward resolved timestamp. + p.ConsumeLogicalOps(ctx, updateIntentOp(txn2, hlc.Timestamp{WallTime: 12})) + h.syncEventAndRegistrations() + require.Equal(t, + []*kvpb.RangeFeedEvent{ + rangeFeedCheckpoint( + roachpb.Span{Key: roachpb.Key("a"), EndKey: roachpb.Key("m")}, + hlc.Timestamp{WallTime: 11}, + ), + }, + r1Stream.Events(), + ) + // Commit intent. Should forward resolved timestamp to closed timestamp. + p.ConsumeLogicalOps(ctx, + commitIntentOpWithKV(txn2, roachpb.Key("e"), hlc.Timestamp{WallTime: 13}, + []byte("ival"), false /* omitInRangefeeds */, 0 /* originID */)) + h.syncEventAndRegistrations() + require.Equal(t, + []*kvpb.RangeFeedEvent{ rangeFeedValue( - roachpb.Key("k"), + roachpb.Key("e"), roachpb.Value{ - RawBytes: []byte("val3"), - Timestamp: hlc.Timestamp{WallTime: 22}, + RawBytes: []byte("ival"), + Timestamp: hlc.Timestamp{WallTime: 13}, }, ), - } - require.Equal(t, valEvent3, r1Stream.Events()) - // r2Stream should not see the event. + rangeFeedCheckpoint( + roachpb.Span{Key: roachpb.Key("a"), EndKey: roachpb.Key("m")}, + hlc.Timestamp{WallTime: 15}, + ), + }, + r1Stream.Events(), + ) + + // Add another registration with withDiff = true and withFiltering = true. + r2Stream := newTestStream() + r2OK, r1And2Filter := p.Register( + roachpb.RSpan{Key: roachpb.RKey("c"), EndKey: roachpb.RKey("z")}, + hlc.Timestamp{WallTime: 1}, + nil, /* catchUpIter */ + true, /* withDiff */ + true, /* withFiltering */ + false, /* withOmitRemote */ + r2Stream, + func() {}, + ) + require.True(t, r2OK) + h.syncEventAndRegistrations() + require.Equal(t, 2, p.Len()) + require.Equal(t, + []*kvpb.RangeFeedEvent{ + rangeFeedCheckpoint( + roachpb.Span{Key: roachpb.Key("c"), EndKey: roachpb.Key("z")}, + hlc.Timestamp{WallTime: 15}, + ), + }, + r2Stream.Events(), + ) + + // Test the processor's new operation filter. + require.True(t, r1And2Filter.NeedVal(roachpb.Span{Key: roachpb.Key("a")})) + require.True(t, r1And2Filter.NeedVal(roachpb.Span{Key: roachpb.Key("y")})) + require.True(t, + r1And2Filter.NeedVal(roachpb.Span{Key: roachpb.Key("y"), EndKey: roachpb.Key("zzz")})) + require.False(t, r1And2Filter.NeedVal(roachpb.Span{Key: roachpb.Key("zzz")})) + require.False(t, r1And2Filter.NeedPrevVal(roachpb.Span{Key: roachpb.Key("a")})) + require.True(t, r1And2Filter.NeedPrevVal(roachpb.Span{Key: roachpb.Key("y")})) + require.True(t, + r1And2Filter.NeedPrevVal(roachpb.Span{Key: roachpb.Key("y"), EndKey: roachpb.Key("zzz")})) + require.False(t, r1And2Filter.NeedPrevVal(roachpb.Span{Key: roachpb.Key("zzz")})) + + // Both registrations should see checkpoint. + p.ForwardClosedTS(ctx, hlc.Timestamp{WallTime: 20}) + h.syncEventAndRegistrations() + chEventAM := []*kvpb.RangeFeedEvent{ + rangeFeedCheckpoint( + roachpb.Span{Key: roachpb.Key("a"), EndKey: roachpb.Key("m")}, + hlc.Timestamp{WallTime: 20}, + ), + } + require.Equal(t, chEventAM, r1Stream.Events()) + chEventCZ := []*kvpb.RangeFeedEvent{ + rangeFeedCheckpoint( + roachpb.Span{Key: roachpb.Key("c"), EndKey: roachpb.Key("z")}, + hlc.Timestamp{WallTime: 20}, + ), + } + require.Equal(t, chEventCZ, r2Stream.Events()) + + // Test value with two registration that overlaps both. + p.ConsumeLogicalOps(ctx, + writeValueOpWithKV(roachpb.Key("k"), hlc.Timestamp{WallTime: 22}, []byte("val2"))) + h.syncEventAndRegistrations() + valEvent := []*kvpb.RangeFeedEvent{ + rangeFeedValue( + roachpb.Key("k"), + roachpb.Value{ + RawBytes: []byte("val2"), + Timestamp: hlc.Timestamp{WallTime: 22}, + }, + ), + } + require.Equal(t, valEvent, r1Stream.Events()) + require.Equal(t, valEvent, r2Stream.Events()) // Cancel the first registration. r1Stream.SendError(kvpb.NewError(fmt.Errorf("disconnection error"))) require.NotNil(t, r1Stream.WaitForError(t)) - // Stop the processor with an error. - pErr := kvpb.NewErrorf("stop err") - p.StopWithErr(pErr) - require.NotNil(t, r2Stream.WaitForError(t)) + // Test committing intent with OmitInRangefeeds that overlaps two + // registration (one withFiltering = true and one withFiltering = false). + p.ConsumeLogicalOps(ctx, + commitIntentOpWithKV(txn2, roachpb.Key("k"), hlc.Timestamp{WallTime: 22}, + []byte("val3"), true /* omitInRangefeeds */, 0 /* originID */)) + h.syncEventAndRegistrations() + valEvent3 := []*kvpb.RangeFeedEvent{ + rangeFeedValue( + roachpb.Key("k"), + roachpb.Value{ + RawBytes: []byte("val3"), + Timestamp: hlc.Timestamp{WallTime: 22}, + }, + ), + } + require.Equal(t, valEvent3, r1Stream.Events()) + // r2Stream should not see the event. // Adding another registration should fail. r3Stream := newTestStream() @@ -333,7 +449,7 @@ func TestProcessorOmitRemote(t *testing.T) { ctx := context.Background() defer stopper.Stop(ctx) - require.NotPanics(t, func() { p.ForwardClosedTS(ctx, hlc.Timestamp{WallTime: 1}) }) + require.NotPanics(t, func() { p.ForwardClosedTS(ctx, hlc.Timestamp{WallTime: 1}) }) // Add a registration. r1Stream := newTestStream() @@ -401,11 +517,53 @@ func TestProcessorOmitRemote(t *testing.T) { Timestamp: hlc.Timestamp{WallTime: 22}, }, ), - } + }, + r1Stream.Events(), + ) + + // Add another registration with withOmitRemote = true. + r2Stream := newTestStream() + r2OK, _ := p.Register( + roachpb.RSpan{Key: roachpb.RKey("a"), EndKey: roachpb.RKey("m")}, + hlc.Timestamp{WallTime: 1}, + nil, /* catchUpIter */ + false, /* withDiff */ + false, /* withFiltering */ + true, /* withOmitRemote */ + r2Stream, + func() {}, + ) + require.True(t, r2OK) + h.syncEventAndRegistrations() - require.Equal(t, valEvent3, r1Stream.Events()) - require.Equal(t, []*kvpb.RangeFeedEvent(nil), r2Stream.Events()) - }) + require.Equal(t, + []*kvpb.RangeFeedEvent{ + rangeFeedCheckpoint( + roachpb.Span{Key: roachpb.Key("a"), EndKey: roachpb.Key("m")}, + hlc.Timestamp{WallTime: 1}, + ), + }, + r2Stream.Events(), + ) + + txn2 := uuid.MakeV4() + p.ConsumeLogicalOps(ctx, + commitIntentOpWithKV(txn2, roachpb.Key("k"), hlc.Timestamp{WallTime: 22}, + []byte("val3"), false /* omitInRangefeeds */, 1)) + h.syncEventAndRegistrations() + + valEvent3 := []*kvpb.RangeFeedEvent{ + rangeFeedValue( + roachpb.Key("k"), + roachpb.Value{ + RawBytes: []byte("val3"), + Timestamp: hlc.Timestamp{WallTime: 22}, + }, + ), + } + + require.Equal(t, valEvent3, r1Stream.Events()) + require.Equal(t, []*kvpb.RangeFeedEvent(nil), r2Stream.Events()) } // TestProcessorSlowConsumer tests that buffered registration will drop events @@ -413,7 +571,7 @@ func TestProcessorOmitRemote(t *testing.T) { // doesn't apply to unbuffered registrations. func TestProcessorSlowConsumer(t *testing.T) { defer leaktest.AfterTest(t)() - testutils.RunValues(t, "feed type", []rangefeedTestType{legacyProcessor, scheduledProcessorWithBufferedReg}, + testutils.RunValues(t, "feed type", []rangefeedTestType{scheduledProcessorWithBufferedReg}, func(t *testing.T, rt rangefeedTestType) { p, h, stopper := newTestProcessor(t, withRangefeedTestType(rt)) ctx := context.Background() @@ -559,18 +717,28 @@ func TestProcessorMemoryBudgetExceeded(t *testing.T) { break } } - // Ensure that stop event generated by memory budget error is processed. - h.syncEventC() + }() - // Unblock the 'send' channel. The events should quickly be consumed. - unblock() - unblock = nil - h.syncEventAndRegistrations() + // Write entries till budget is exhausted + for i := 0; i < 10; i++ { + if !p.ConsumeLogicalOps(ctx, writeValueOpWithKV( + roachpb.Key("k"), + hlc.Timestamp{WallTime: int64(i + 2)}, + []byte(fmt.Sprintf("this is big value %02d", i)))) { + break + } + } + // Ensure that stop event generated by memory budget error is processed. + h.syncEventC() - require.Equal(t, newErrBufferCapacityExceeded().GoError(), r1Stream.WaitForError(t)) - require.Equal(t, 0, p.Len(), "registration was not removed") - require.Equal(t, int64(1), m.RangeFeedBudgetExhausted.Count()) - }) + // Unblock the 'send' channel. The events should quickly be consumed. + unblock() + unblock = nil + h.syncEventAndRegistrations() + + require.Equal(t, newErrBufferCapacityExceeded().GoError(), r1Stream.WaitForError(t)) + require.Equal(t, 0, p.Len(), "registration was not removed") + require.Equal(t, int64(1), m.RangeFeedBudgetExhausted.Count()) } // TestProcessorMemoryBudgetReleased that memory budget is correctly released. @@ -598,27 +766,26 @@ func TestProcessorMemoryBudgetReleased(t *testing.T) { ) h.syncEventAndRegistrations() - // Write entries and check they are consumed so that we could write more - // data than total budget if inflight messages are within budget. - const eventCount = 10 - for i := 0; i < eventCount; i++ { - p.ConsumeLogicalOps(ctx, writeValueOpWithKV( - roachpb.Key("k"), - hlc.Timestamp{WallTime: int64(i + 2)}, - []byte("value"))) - } - h.syncEventAndRegistrations() + // Write entries and check they are consumed so that we could write more + // data than total budget if inflight messages are within budget. + const eventCount = 10 + for i := 0; i < eventCount; i++ { + p.ConsumeLogicalOps(ctx, writeValueOpWithKV( + roachpb.Key("k"), + hlc.Timestamp{WallTime: int64(i + 2)}, + []byte("value"))) + } + h.syncEventAndRegistrations() - // Count consumed values - consumedOps := 0 - for _, e := range r1Stream.Events() { - if e.Val != nil { - consumedOps++ - } + // Count consumed values + consumedOps := 0 + for _, e := range r1Stream.Events() { + if e.Val != nil { + consumedOps++ } - require.Equal(t, 1, p.Len(), "registration was removed") - require.Equal(t, 10, consumedOps) - }) + } + require.Equal(t, 1, p.Len(), "registration was removed") + require.Equal(t, 10, consumedOps) } // TestProcessorInitializeResolvedTimestamp tests that when a Processor is given @@ -626,6 +793,34 @@ func TestProcessorMemoryBudgetReleased(t *testing.T) { // until it has consumed all intents in the iterator. func TestProcessorInitializeResolvedTimestamp(t *testing.T) { defer leaktest.AfterTest(t)() + txn1 := makeTxn("txn1", uuid.MakeV4(), isolation.Serializable, hlc.Timestamp{}) + txn2 := makeTxn("txn2", uuid.MakeV4(), isolation.Serializable, hlc.Timestamp{}) + txnWithTs := func(txn roachpb.Transaction, ts int64) *roachpb.Transaction { + txnTs := hlc.Timestamp{WallTime: ts} + txn.TxnMeta.MinTimestamp = txnTs + txn.TxnMeta.WriteTimestamp = txnTs + txn.ReadTimestamp = txnTs + return &txn + } + data := []storeOp{ + {kv: makeKV("a", "val1", 10)}, + {kv: makeKV("c", "val4", 9)}, + {kv: makeKV("c", "val3", 11)}, + {kv: makeProvisionalKV("c", "txnKey1", 15), txn: txnWithTs(txn1, 15)}, + {kv: makeKV("d", "val6", 19)}, + {kv: makeKV("d", "val5", 20)}, + {kv: makeProvisionalKV("d", "txnKey2", 21), txn: txnWithTs(txn2, 21)}, + {kv: makeKV("m", "val8", 1)}, + {kv: makeProvisionalKV("n", "txnKey1", 12), txn: txnWithTs(txn1, 12)}, + {kv: makeKV("r", "val9", 4)}, + {kv: makeProvisionalKV("r", "txnKey1", 19), txn: txnWithTs(txn1, 19)}, + {kv: makeProvisionalKV("w", "txnKey1", 3), txn: txnWithTs(txn1, 3)}, + {kv: makeKV("z", "val11", 4)}, + {kv: makeProvisionalKV("z", "txnKey2", 21), txn: txnWithTs(txn2, 21)}, + } + scanner, cleanup, err := makeIntentScanner(data, roachpb.RSpan{Key: roachpb.RKey("a"), EndKey: roachpb.RKey("zz")}) + require.NoError(t, err, "failed to prepare test data") + defer cleanup() testutils.RunValues(t, "feed type", testTypes, func(t *testing.T, rt rangefeedTestType) { txn1 := makeTxn("txn1", uuid.MakeV4(), isolation.Serializable, hlc.Timestamp{}) @@ -661,9 +856,20 @@ func TestProcessorInitializeResolvedTimestamp(t *testing.T) { ctx := context.Background() defer stopper.Stop(ctx) - // The resolved timestamp should not be initialized. - require.False(t, h.rts.IsInit()) - require.Equal(t, hlc.Timestamp{}, h.rts.Get()) + // Add a registration. + r1Stream := newTestStream() + p.Register( + roachpb.RSpan{Key: roachpb.RKey("a"), EndKey: roachpb.RKey("m")}, + hlc.Timestamp{WallTime: 1}, + nil, /* catchUpIter */ + false, /* withDiff */ + false, /* withFiltering */ + false, /* withOmitRemote */ + r1Stream, + func() {}, + ) + h.syncEventAndRegistrations() + require.Equal(t, 1, p.Len()) // Add a registration. r1Stream := newTestStream() @@ -681,48 +887,37 @@ func TestProcessorInitializeResolvedTimestamp(t *testing.T) { h.syncEventAndRegistrations() require.Equal(t, 1, p.Len()) - // The registration should be provided a checkpoint immediately with an - // empty resolved timestamp because it did not perform a catch-up scan. - chEvent := []*kvpb.RangeFeedEvent{ - rangeFeedCheckpoint( - roachpb.Span{Key: roachpb.Key("a"), EndKey: roachpb.Key("m")}, - hlc.Timestamp{}, - ), - } - require.Equal(t, chEvent, r1Stream.Events()) - - // The resolved timestamp should still not be initialized. - require.False(t, h.rts.IsInit()) - require.Equal(t, hlc.Timestamp{}, h.rts.Get()) - - // Forward the closed timestamp. The resolved timestamp should still - // not be initialized. - p.ForwardClosedTS(ctx, hlc.Timestamp{WallTime: 20}) - require.False(t, h.rts.IsInit()) - require.Equal(t, hlc.Timestamp{}, h.rts.Get()) - - // Let the scan proceed. - close(scanner.block) - <-scanner.done - - // Synchronize the event channel then verify that the resolved timestamp is - // initialized and that it's blocked on the oldest unresolved intent's txn - // timestamp. Txn1 has intents at many times but the unresolvedIntentQueue - // tracks its latest, which is 19, so the resolved timestamp is - // 19.FloorPrev() = 18. - h.syncEventAndRegistrations() - require.True(t, h.rts.IsInit()) - require.Equal(t, hlc.Timestamp{WallTime: 18}, h.rts.Get()) - - // The registration should have been informed of the new resolved timestamp. - chEvent = []*kvpb.RangeFeedEvent{ - rangeFeedCheckpoint( - roachpb.Span{Key: roachpb.Key("a"), EndKey: roachpb.Key("m")}, - hlc.Timestamp{WallTime: 18}, - ), - } - require.Equal(t, chEvent, r1Stream.Events()) - }) + // The resolved timestamp should still not be initialized. + require.False(t, h.rts.IsInit()) + require.Equal(t, hlc.Timestamp{}, h.rts.Get()) + + // Forward the closed timestamp. The resolved timestamp should still + // not be initialized. + p.ForwardClosedTS(ctx, hlc.Timestamp{WallTime: 20}) + require.False(t, h.rts.IsInit()) + require.Equal(t, hlc.Timestamp{}, h.rts.Get()) + + // Let the scan proceed. + close(scanner.block) + <-scanner.done + + // Synchronize the event channel then verify that the resolved timestamp is + // initialized and that it's blocked on the oldest unresolved intent's txn + // timestamp. Txn1 has intents at many times but the unresolvedIntentQueue + // tracks its latest, which is 19, so the resolved timestamp is + // 19.FloorPrev() = 18. + h.syncEventAndRegistrations() + require.True(t, h.rts.IsInit()) + require.Equal(t, hlc.Timestamp{WallTime: 18}, h.rts.Get()) + + // The registration should have been informed of the new resolved timestamp. + chEvent = []*kvpb.RangeFeedEvent{ + rangeFeedCheckpoint( + roachpb.Span{Key: roachpb.Key("a"), EndKey: roachpb.Key("m")}, + hlc.Timestamp{WallTime: 18}, + ), + } + require.Equal(t, chEvent, r1Stream.Events()) } func TestProcessorTxnPushAttempt(t *testing.T) { @@ -738,172 +933,164 @@ func TestProcessorTxnPushAttempt(t *testing.T) { ts70 := hlc.Timestamp{WallTime: 70} ts90 := hlc.Timestamp{WallTime: 90} - // Create a set of transactions. - txn1, txn2, txn3 := uuid.MakeV4(), uuid.MakeV4(), uuid.MakeV4() - txn1Meta := enginepb.TxnMeta{ID: txn1, Key: keyA, IsoLevel: isolation.Serializable, WriteTimestamp: ts10, MinTimestamp: ts10} - txn2Meta := enginepb.TxnMeta{ID: txn2, Key: keyB, IsoLevel: isolation.Snapshot, WriteTimestamp: ts20, MinTimestamp: ts20} - txn3Meta := enginepb.TxnMeta{ID: txn3, Key: keyC, IsoLevel: isolation.ReadCommitted, WriteTimestamp: ts30, MinTimestamp: ts30} - txn1Proto := &roachpb.Transaction{TxnMeta: txn1Meta, Status: roachpb.PENDING} - txn2Proto := &roachpb.Transaction{TxnMeta: txn2Meta, Status: roachpb.PENDING} - txn3Proto := &roachpb.Transaction{TxnMeta: txn3Meta, Status: roachpb.PENDING} - - // Modifications for test 2. - txn1MetaT2Pre := enginepb.TxnMeta{ID: txn1, Key: keyA, IsoLevel: isolation.Serializable, WriteTimestamp: ts25, MinTimestamp: ts10} - txn1MetaT2Post := enginepb.TxnMeta{ID: txn1, Key: keyA, IsoLevel: isolation.Serializable, WriteTimestamp: ts50, MinTimestamp: ts10} - txn2MetaT2Post := enginepb.TxnMeta{ID: txn2, Key: keyB, IsoLevel: isolation.Snapshot, WriteTimestamp: ts60, MinTimestamp: ts20} - txn3MetaT2Post := enginepb.TxnMeta{ID: txn3, Key: keyC, IsoLevel: isolation.ReadCommitted, WriteTimestamp: ts70, MinTimestamp: ts30} - txn1ProtoT2 := &roachpb.Transaction{TxnMeta: txn1MetaT2Post, Status: roachpb.COMMITTED} - txn2ProtoT2 := &roachpb.Transaction{TxnMeta: txn2MetaT2Post, Status: roachpb.PENDING} - txn3ProtoT2 := &roachpb.Transaction{TxnMeta: txn3MetaT2Post, Status: roachpb.PENDING} - - // Modifications for test 3. - txn2MetaT3Post := enginepb.TxnMeta{ID: txn2, Key: keyB, IsoLevel: isolation.Snapshot, WriteTimestamp: ts60, MinTimestamp: ts20} - txn3MetaT3Post := enginepb.TxnMeta{ID: txn3, Key: keyC, IsoLevel: isolation.ReadCommitted, WriteTimestamp: ts90, MinTimestamp: ts30} - txn2ProtoT3 := &roachpb.Transaction{TxnMeta: txn2MetaT3Post, Status: roachpb.ABORTED} - txn3ProtoT3 := &roachpb.Transaction{TxnMeta: txn3MetaT3Post, Status: roachpb.PENDING} - - testNum := 0 - pausePushAttemptsC := make(chan struct{}) - resumePushAttemptsC := make(chan struct{}) - defer close(pausePushAttemptsC) - defer close(resumePushAttemptsC) - - // Create a TxnPusher that performs assertions during the first 3 uses. - var tp testTxnPusher - tp.mockPushTxns(func( - ctx context.Context, txns []enginepb.TxnMeta, ts hlc.Timestamp, - ) ([]*roachpb.Transaction, bool, error) { - // The txns are not in a sorted order. Enforce one. - sort.Slice(txns, func(i, j int) bool { - return bytes.Compare(txns[i].Key, txns[j].Key) < 0 - }) - - testNum++ - switch testNum { - case 1: - assert.Equal(t, 3, len(txns)) - assert.Equal(t, txn1Meta, txns[0]) - assert.Equal(t, txn2Meta, txns[1]) - assert.Equal(t, txn3Meta, txns[2]) - if t.Failed() { - return nil, false, errors.New("test failed") - } - - // Push does not succeed. Protos not at larger ts. - return []*roachpb.Transaction{txn1Proto, txn2Proto, txn3Proto}, false, nil - case 2: - assert.Equal(t, 3, len(txns)) - assert.Equal(t, txn1MetaT2Pre, txns[0]) - assert.Equal(t, txn2Meta, txns[1]) - assert.Equal(t, txn3Meta, txns[2]) - if t.Failed() { - return nil, false, errors.New("test failed") - } + // Create a set of transactions. + txn1, txn2, txn3 := uuid.MakeV4(), uuid.MakeV4(), uuid.MakeV4() + txn1Meta := enginepb.TxnMeta{ID: txn1, Key: keyA, IsoLevel: isolation.Serializable, WriteTimestamp: ts10, MinTimestamp: ts10} + txn2Meta := enginepb.TxnMeta{ID: txn2, Key: keyB, IsoLevel: isolation.Snapshot, WriteTimestamp: ts20, MinTimestamp: ts20} + txn3Meta := enginepb.TxnMeta{ID: txn3, Key: keyC, IsoLevel: isolation.ReadCommitted, WriteTimestamp: ts30, MinTimestamp: ts30} + txn1Proto := &roachpb.Transaction{TxnMeta: txn1Meta, Status: roachpb.PENDING} + txn2Proto := &roachpb.Transaction{TxnMeta: txn2Meta, Status: roachpb.PENDING} + txn3Proto := &roachpb.Transaction{TxnMeta: txn3Meta, Status: roachpb.PENDING} + + // Modifications for test 2. + txn1MetaT2Pre := enginepb.TxnMeta{ID: txn1, Key: keyA, IsoLevel: isolation.Serializable, WriteTimestamp: ts25, MinTimestamp: ts10} + txn1MetaT2Post := enginepb.TxnMeta{ID: txn1, Key: keyA, IsoLevel: isolation.Serializable, WriteTimestamp: ts50, MinTimestamp: ts10} + txn2MetaT2Post := enginepb.TxnMeta{ID: txn2, Key: keyB, IsoLevel: isolation.Snapshot, WriteTimestamp: ts60, MinTimestamp: ts20} + txn3MetaT2Post := enginepb.TxnMeta{ID: txn3, Key: keyC, IsoLevel: isolation.ReadCommitted, WriteTimestamp: ts70, MinTimestamp: ts30} + txn1ProtoT2 := &roachpb.Transaction{TxnMeta: txn1MetaT2Post, Status: roachpb.COMMITTED} + txn2ProtoT2 := &roachpb.Transaction{TxnMeta: txn2MetaT2Post, Status: roachpb.PENDING} + txn3ProtoT2 := &roachpb.Transaction{TxnMeta: txn3MetaT2Post, Status: roachpb.PENDING} + + // Modifications for test 3. + txn2MetaT3Post := enginepb.TxnMeta{ID: txn2, Key: keyB, IsoLevel: isolation.Snapshot, WriteTimestamp: ts60, MinTimestamp: ts20} + txn3MetaT3Post := enginepb.TxnMeta{ID: txn3, Key: keyC, IsoLevel: isolation.ReadCommitted, WriteTimestamp: ts90, MinTimestamp: ts30} + txn2ProtoT3 := &roachpb.Transaction{TxnMeta: txn2MetaT3Post, Status: roachpb.ABORTED} + txn3ProtoT3 := &roachpb.Transaction{TxnMeta: txn3MetaT3Post, Status: roachpb.PENDING} + + testNum := 0 + pausePushAttemptsC := make(chan struct{}) + resumePushAttemptsC := make(chan struct{}) + defer close(pausePushAttemptsC) + defer close(resumePushAttemptsC) + + // Create a TxnPusher that performs assertions during the first 3 uses. + var tp testTxnPusher + tp.mockPushTxns(func( + ctx context.Context, txns []enginepb.TxnMeta, ts hlc.Timestamp, + ) ([]*roachpb.Transaction, bool, error) { + // The txns are not in a sorted order. Enforce one. + sort.Slice(txns, func(i, j int) bool { + return bytes.Compare(txns[i].Key, txns[j].Key) < 0 + }) - // Push succeeds. Return new protos. - return []*roachpb.Transaction{txn1ProtoT2, txn2ProtoT2, txn3ProtoT2}, false, nil - case 3: - assert.Equal(t, 2, len(txns)) - assert.Equal(t, txn2MetaT2Post, txns[0]) - assert.Equal(t, txn3MetaT2Post, txns[1]) - if t.Failed() { - return nil, false, errors.New("test failed") - } + p, h, stopper := newTestProcessor(t, withPusher(&tp), withRangefeedTestType(rt)) + ctx := context.Background() + defer stopper.Stop(ctx) - // Push succeeds. Return new protos. - return []*roachpb.Transaction{txn2ProtoT3, txn3ProtoT3}, false, nil - default: - return nil, false, nil + // Push does not succeed. Protos not at larger ts. + return []*roachpb.Transaction{txn1Proto, txn2Proto, txn3Proto}, false, nil + case 2: + assert.Equal(t, 3, len(txns)) + assert.Equal(t, txn1MetaT2Pre, txns[0]) + assert.Equal(t, txn2Meta, txns[1]) + assert.Equal(t, txn3Meta, txns[2]) + if t.Failed() { + return nil, false, errors.New("test failed") } - }) - tp.mockResolveIntentsFn(func(ctx context.Context, intents []roachpb.LockUpdate) error { - // There's nothing to assert here. We expect the intents to correspond to - // transactions that had their LockSpans populated when we pushed them. This - // test doesn't simulate that. - if testNum > 3 { - return nil + // Push succeeds. Return new protos. + return []*roachpb.Transaction{txn1ProtoT2, txn2ProtoT2, txn3ProtoT2}, false, nil + case 3: + assert.Equal(t, 2, len(txns)) + assert.Equal(t, txn2MetaT2Post, txns[0]) + assert.Equal(t, txn3MetaT2Post, txns[1]) + if t.Failed() { + return nil, false, errors.New("test failed") } - pausePushAttemptsC <- struct{}{} - <-resumePushAttemptsC + // Push succeeds. Return new protos. + return []*roachpb.Transaction{txn2ProtoT3, txn3ProtoT3}, false, nil + default: + return nil, false, nil + } + }) + tp.mockResolveIntentsFn(func(ctx context.Context, intents []roachpb.LockUpdate) error { + // There's nothing to assert here. We expect the intents to correspond to + // transactions that had their LockSpans populated when we pushed them. This + // test doesn't simulate that. + + if testNum > 3 { return nil - }) + } - p, h, stopper := newTestProcessor(t, withPusher(&tp), withRangefeedTestType(rt)) - ctx := context.Background() - defer stopper.Stop(ctx) + pausePushAttemptsC <- struct{}{} + <-resumePushAttemptsC + return nil + }) - // Add a few intents and move the closed timestamp forward. - p.ConsumeLogicalOps(ctx, - writeIntentOpFromMeta(txn1Meta), - writeIntentOpFromMeta(txn2Meta), - writeIntentOpFromMeta(txn2Meta), - writeIntentOpFromMeta(txn3Meta), - ) - p.ForwardClosedTS(ctx, hlc.Timestamp{WallTime: 40}) - h.syncEventC() - require.Equal(t, hlc.Timestamp{WallTime: 9}, h.rts.Get()) + p, h, stopper := newTestProcessor(t, withPusher(&tp)) + ctx := context.Background() + defer stopper.Stop(ctx) - // Wait for the first txn push attempt to complete. - h.triggerTxnPushUntilPushed(t, pausePushAttemptsC) + // Add a few intents and move the closed timestamp forward. + p.ConsumeLogicalOps(ctx, + writeIntentOpFromMeta(txn1Meta), + writeIntentOpFromMeta(txn2Meta), + writeIntentOpFromMeta(txn2Meta), + writeIntentOpFromMeta(txn3Meta), + ) + p.ForwardClosedTS(ctx, hlc.Timestamp{WallTime: 40}) + h.syncEventC() + require.Equal(t, hlc.Timestamp{WallTime: 9}, h.rts.Get()) - // The resolved timestamp hasn't moved. - h.syncEventC() - require.Equal(t, hlc.Timestamp{WallTime: 9}, h.rts.Get()) + // Wait for the first txn push attempt to complete. + h.triggerTxnPushUntilPushed(t, pausePushAttemptsC) - // Write another intent for one of the txns. This moves the resolved - // timestamp forward. - p.ConsumeLogicalOps(ctx, writeIntentOpFromMeta(txn1MetaT2Pre)) - h.syncEventC() - require.Equal(t, hlc.Timestamp{WallTime: 19}, h.rts.Get()) + // The resolved timestamp hasn't moved. + h.syncEventC() + require.Equal(t, hlc.Timestamp{WallTime: 9}, h.rts.Get()) - // Unblock the second txn push attempt and wait for it to complete. - resumePushAttemptsC <- struct{}{} - h.triggerTxnPushUntilPushed(t, pausePushAttemptsC) + // Write another intent for one of the txns. This moves the resolved + // timestamp forward. + p.ConsumeLogicalOps(ctx, writeIntentOpFromMeta(txn1MetaT2Pre)) + h.syncEventC() + require.Equal(t, hlc.Timestamp{WallTime: 19}, h.rts.Get()) - // The resolved timestamp should have moved forwards to the closed - // timestamp. - h.syncEventC() - require.Equal(t, hlc.Timestamp{WallTime: 40}, h.rts.Get()) + // Unblock the second txn push attempt and wait for it to complete. + resumePushAttemptsC <- struct{}{} + h.triggerTxnPushUntilPushed(t, pausePushAttemptsC) - // Forward the closed timestamp. - p.ForwardClosedTS(ctx, hlc.Timestamp{WallTime: 80}) - h.syncEventC() - require.Equal(t, hlc.Timestamp{WallTime: 49}, h.rts.Get()) + // The resolved timestamp should have moved forwards to the closed + // timestamp. + h.syncEventC() + require.Equal(t, hlc.Timestamp{WallTime: 40}, h.rts.Get()) - // Txn1's first intent is committed. Resolved timestamp doesn't change. - p.ConsumeLogicalOps(ctx, commitIntentOp(txn1MetaT2Post.ID, txn1MetaT2Post.WriteTimestamp)) - h.syncEventC() - require.Equal(t, hlc.Timestamp{WallTime: 49}, h.rts.Get()) + // Forward the closed timestamp. + p.ForwardClosedTS(ctx, hlc.Timestamp{WallTime: 80}) + h.syncEventC() + require.Equal(t, hlc.Timestamp{WallTime: 49}, h.rts.Get()) - // Txn1's second intent is committed. Resolved timestamp moves forward. - p.ConsumeLogicalOps(ctx, commitIntentOp(txn1MetaT2Post.ID, txn1MetaT2Post.WriteTimestamp)) - h.syncEventC() - require.Equal(t, hlc.Timestamp{WallTime: 59}, h.rts.Get()) + // Txn1's first intent is committed. Resolved timestamp doesn't change. + p.ConsumeLogicalOps(ctx, commitIntentOp(txn1MetaT2Post.ID, txn1MetaT2Post.WriteTimestamp)) + h.syncEventC() + require.Equal(t, hlc.Timestamp{WallTime: 49}, h.rts.Get()) - // Unblock the third txn push attempt and wait for it to complete. - resumePushAttemptsC <- struct{}{} - h.triggerTxnPushUntilPushed(t, pausePushAttemptsC) + // Txn1's second intent is committed. Resolved timestamp moves forward. + p.ConsumeLogicalOps(ctx, commitIntentOp(txn1MetaT2Post.ID, txn1MetaT2Post.WriteTimestamp)) + h.syncEventC() + require.Equal(t, hlc.Timestamp{WallTime: 59}, h.rts.Get()) - // The resolved timestamp should have moved forwards to the closed - // timestamp. - h.syncEventC() - require.Equal(t, hlc.Timestamp{WallTime: 80}, h.rts.Get()) + // Unblock the third txn push attempt and wait for it to complete. + resumePushAttemptsC <- struct{}{} + h.triggerTxnPushUntilPushed(t, pausePushAttemptsC) - // Forward the closed timestamp. - p.ForwardClosedTS(ctx, hlc.Timestamp{WallTime: 100}) - h.syncEventC() - require.Equal(t, hlc.Timestamp{WallTime: 89}, h.rts.Get()) + // The resolved timestamp should have moved forwards to the closed + // timestamp. + h.syncEventC() + require.Equal(t, hlc.Timestamp{WallTime: 80}, h.rts.Get()) - // Commit txn3's only intent. Resolved timestamp moves forward. - p.ConsumeLogicalOps(ctx, commitIntentOp(txn3MetaT3Post.ID, txn3MetaT3Post.WriteTimestamp)) - h.syncEventC() - require.Equal(t, hlc.Timestamp{WallTime: 100}, h.rts.Get()) + // Forward the closed timestamp. + p.ForwardClosedTS(ctx, hlc.Timestamp{WallTime: 100}) + h.syncEventC() + require.Equal(t, hlc.Timestamp{WallTime: 89}, h.rts.Get()) - // Release push attempt to avoid deadlock. - resumePushAttemptsC <- struct{}{} - }) + // Commit txn3's only intent. Resolved timestamp moves forward. + p.ConsumeLogicalOps(ctx, commitIntentOp(txn3MetaT3Post.ID, txn3MetaT3Post.WriteTimestamp)) + h.syncEventC() + require.Equal(t, hlc.Timestamp{WallTime: 100}, h.rts.Get()) + + // Release push attempt to avoid deadlock. + resumePushAttemptsC <- struct{}{} } // TestProcessorTxnPushDisabled tests that processors don't attempt txn pushes @@ -944,7 +1131,7 @@ func TestProcessorTxnPushDisabled(t *testing.T) { }) p, h, stopper := newTestProcessor(t, withSettings(st), withPusher(&tp), - withPushTxnsIntervalAge(pushInterval, time.Millisecond)) + withPushTxnsAge(time.Millisecond)) defer stopper.Stop(ctx) // Move the resolved ts forward to just before the txn timestamp. @@ -1025,23 +1212,15 @@ func TestProcessorRegistrationObservesOnlyNewEvents(t *testing.T) { ctx := context.Background() defer stopper.Stop(ctx) - firstC := make(chan int64) - regDone := make(chan struct{}) - regs := make(map[*testStream]int64) - - var wg sync.WaitGroup - wg.Add(2) - go func() { - defer wg.Done() - for i := int64(1); i < 250; i++ { - // Add a new registration every 10 ops. - if i%10 == 0 { - firstC <- i - <-regDone - } - - // Consume the logical op. Encode the index in the timestamp. - p.ConsumeLogicalOps(ctx, writeValueOp(hlc.Timestamp{WallTime: i})) + var wg sync.WaitGroup + wg.Add(2) + go func() { + defer wg.Done() + for i := int64(1); i < 250; i++ { + // Add a new registration every 10 ops. + if i%10 == 0 { + firstC <- i + <-regDone } h.syncEventC() close(firstC) @@ -1062,18 +1241,38 @@ func TestProcessorRegistrationObservesOnlyNewEvents(t *testing.T) { wg.Wait() h.syncEventAndRegistrations() - // Verify that no registrations were given operations - // from before they registered. - for s, expFirstIdx := range regs { - events := s.Events() - require.IsType(t, &kvpb.RangeFeedCheckpoint{}, events[0].GetValue()) - require.IsType(t, &kvpb.RangeFeedValue{}, events[1].GetValue()) - - firstVal := events[1].GetValue().(*kvpb.RangeFeedValue) - firstIdx := firstVal.Value.Timestamp.WallTime - require.Equal(t, expFirstIdx, firstIdx) + // Consume the logical op. Encode the index in the timestamp. + p.ConsumeLogicalOps(ctx, writeValueOp(hlc.Timestamp{WallTime: i})) } - }) + h.syncEventC() + close(firstC) + }() + go func() { + defer wg.Done() + for firstIdx := range firstC { + // For each index, create a new registration. The first + // operation is should see is firstIdx. + s := newTestStream() + regs[s] = firstIdx + p.Register(h.span, hlc.Timestamp{}, nil, /* catchUpIter */ + false /* withDiff */, false /* withFiltering */, false /* withOmitRemote */, s, func() {}) + regDone <- struct{}{} + } + }() + wg.Wait() + h.syncEventAndRegistrations() + + // Verify that no registrations were given operations + // from before they registered. + for s, expFirstIdx := range regs { + events := s.Events() + require.IsType(t, &kvpb.RangeFeedCheckpoint{}, events[0].GetValue()) + require.IsType(t, &kvpb.RangeFeedValue{}, events[1].GetValue()) + + firstVal := events[1].GetValue().(*kvpb.RangeFeedValue) + firstIdx := firstVal.Value.Timestamp.WallTime + require.Equal(t, expFirstIdx, firstIdx) + } } func TestBudgetReleaseOnProcessorStop(t *testing.T) { @@ -1096,8 +1295,8 @@ func TestBudgetReleaseOnProcessorStop(t *testing.T) { }) m.Start(context.Background(), nil, mon.NewStandaloneBudget(math.MaxInt64)) - b := m.MakeBoundAccount() - fb := NewFeedBudget(&b, 0, &s.SV) + b := m.MakeBoundAccount() + fb := NewFeedBudget(&b, 0, &s.SV) p, h, stopper := newTestProcessor(t, withBudget(fb), withChanCap(channelCapacity), withEventTimeout(100*time.Millisecond), withRangefeedTestType(rt)) @@ -1120,49 +1319,48 @@ func TestBudgetReleaseOnProcessorStop(t *testing.T) { ) h.syncEventAndRegistrations() - for i := 0; i < totalEvents; i++ { - p.ConsumeLogicalOps(ctx, writeValueOpWithKV( - roachpb.Key("k"), - hlc.Timestamp{WallTime: int64(i + 2)}, - []byte(fmt.Sprintf("this is value %04d", i)))) - } + for i := 0; i < totalEvents; i++ { + p.ConsumeLogicalOps(ctx, writeValueOpWithKV( + roachpb.Key("k"), + hlc.Timestamp{WallTime: int64(i + 2)}, + []byte(fmt.Sprintf("this is value %04d", i)))) + } - // Wait for half of the event to be processed by stream then stop processor. - select { - case <-rStream.blocked: - case err := <-rStream.done: - t.Fatal("stream failed with error before all data was consumed", err) - } + // Wait for half of the event to be processed by stream then stop processor. + select { + case <-rStream.blocked: + case err := <-rStream.done: + t.Fatal("stream failed with error before all data was consumed", err) + } - // Since stop is blocking and needs to flush events we need to do that in - // parallel. - stopped := make(chan interface{}) - go func() { - p.Stop() - stopped <- struct{}{} - }() + // Since stop is blocking and needs to flush events we need to do that in + // parallel. + stopped := make(chan interface{}) + go func() { + p.Stop() + stopped <- struct{}{} + }() - // Resume event loop in consumer to unblock any internal loops of processor or - // registrations. - rStream.Resume() + // Resume event loop in consumer to unblock any internal loops of processor or + // registrations. + rStream.Resume() - // Wait for top function to finish processing before verifying that we - // consumed all events. - <-stopped + // Wait for top function to finish processing before verifying that we + // consumed all events. + <-stopped - // We need to wait for budget to drain as Stop would only post stop event - // after flushing the queue, but couldn't determine when main processor loop - // is actually closed. - testutils.SucceedsSoon(t, func() error { - fmt.Printf("Budget now: %d bytes remained, %d events processed\n", + // We need to wait for budget to drain as Stop would only post stop event + // after flushing the queue, but couldn't determine when main processor loop + // is actually closed. + testutils.SucceedsSoon(t, func() error { + fmt.Printf("Budget now: %d bytes remained, %d events processed\n", + m.AllocBytes(), rStream.Consumed()) + if m.AllocBytes() != 0 { + return errors.Errorf( + "Failed to release all budget after stop: %d bytes remained, %d events processed", m.AllocBytes(), rStream.Consumed()) - if m.AllocBytes() != 0 { - return errors.Errorf( - "Failed to release all budget after stop: %d bytes remained, %d events processed", - m.AllocBytes(), rStream.Consumed()) - } - return nil - }) + } + return nil }) } @@ -1200,27 +1398,26 @@ func TestBudgetReleaseOnLastStreamError(t *testing.T) { ) h.syncEventAndRegistrations() - for i := 0; i < totalEvents; i++ { - p.ConsumeLogicalOps(ctx, writeValueOpWithKV( - roachpb.Key("k"), - hlc.Timestamp{WallTime: int64(i + 2)}, - []byte(fmt.Sprintf("this is value %04d", i)))) - } + for i := 0; i < totalEvents; i++ { + p.ConsumeLogicalOps(ctx, writeValueOpWithKV( + roachpb.Key("k"), + hlc.Timestamp{WallTime: int64(i + 2)}, + []byte(fmt.Sprintf("this is value %04d", i)))) + } - // Wait for half of the event to be processed then raise error. - select { - case <-rStream.blocked: - case err := <-rStream.done: - t.Fatal("stream failed with error before stream blocked: ", err) - } + // Wait for half of the event to be processed then raise error. + select { + case <-rStream.blocked: + case err := <-rStream.done: + t.Fatal("stream failed with error before stream blocked: ", err) + } - // Resume event loop in consumer and fail Stream to remove registration. - rStream.ResumeWithFailure(errors.Errorf("Closing down stream")) + // Resume event loop in consumer and fail Stream to remove registration. + rStream.ResumeWithFailure(errors.Errorf("Closing down stream")) - // We need to wait for budget to drain as all pending events are processed - // or dropped. - requireBudgetDrainedSoon(t, fb, rStream) - }) + // We need to wait for budget to drain as all pending events are processed + // or dropped. + requireBudgetDrainedSoon(t, fb, rStream) } func newTestBudget(limit int64) *FeedBudget { @@ -1283,27 +1480,19 @@ func TestBudgetReleaseOnOneStreamError(t *testing.T) { ) h.syncEventAndRegistrations() - for i := 0; i < totalEvents; i++ { - p.ConsumeLogicalOps(ctx, writeValueOpWithKV( - roachpb.Key("k"), - hlc.Timestamp{WallTime: int64(i + 2)}, - []byte(fmt.Sprintf("this is value %04d", i)))) - } - - // Wait for half of the event to be processed then stop processor. - select { - case <-r1Stream.blocked: - case err := <-r1Stream.done: - t.Fatal("stream failed with error before all data was consumed", err) - } + // Wait for half of the event to be processed then stop processor. + select { + case <-r1Stream.blocked: + case err := <-r1Stream.done: + t.Fatal("stream failed with error before all data was consumed", err) + } - // Resume event loop in consumer and fail Stream to remove registration. - r1Stream.ResumeWithFailure(errors.Errorf("Closing down stream")) + // Resume event loop in consumer and fail Stream to remove registration. + r1Stream.ResumeWithFailure(errors.Errorf("Closing down stream")) - // We need to wait for budget to drain as all pending events are processed - // or dropped. - requireBudgetDrainedSoon(t, fb, r1Stream) - }) + // We need to wait for budget to drain as all pending events are processed + // or dropped. + requireBudgetDrainedSoon(t, fb, r1Stream) } // requireBudgetDrainedSoon checks that memory budget drains to zero soon. @@ -1438,7 +1627,7 @@ func TestProcessorBackpressure(t *testing.T) { span := roachpb.RSpan{Key: roachpb.RKey("a"), EndKey: roachpb.RKey("z")} p, h, stopper := newTestProcessor(t, withSpan(span), withBudget(newTestBudget(math.MaxInt64)), - withChanCap(1), withEventTimeout(0), withRangefeedTestType(legacyProcessor)) + withChanCap(1), withEventTimeout(0)) defer stopper.Stop(ctx) defer p.Stop() diff --git a/pkg/kv/kvserver/rangefeed/task_test.go b/pkg/kv/kvserver/rangefeed/task_test.go index 403bd45279bb..80fa3759fe99 100644 --- a/pkg/kv/kvserver/rangefeed/task_test.go +++ b/pkg/kv/kvserver/rangefeed/task_test.go @@ -334,7 +334,9 @@ func TestInitResolvedTSScan(t *testing.T) { defer engine.Close() // Mock processor. We just needs its eventC. - p := LegacyProcessor{ + s := newTestScheduler(1) + p := ScheduledProcessor{ + scheduler: s.NewClientScheduler(), Config: Config{ Span: span, }, @@ -477,7 +479,11 @@ func TestTxnPushAttempt(t *testing.T) { // Mock processor. We configure its key span to exclude one of txn2's lock // spans and a portion of three of txn4's lock spans. - p := LegacyProcessor{eventC: make(chan *event, 100)} + s := newTestScheduler(1) + p := ScheduledProcessor{ + scheduler: s.NewClientScheduler(), + eventC: make(chan *event, 100), + } p.Span = roachpb.RSpan{Key: roachpb.RKey("b"), EndKey: roachpb.RKey("m")} p.TxnPusher = &tp diff --git a/pkg/kv/kvserver/rangefeed/testutil.go b/pkg/kv/kvserver/rangefeed/testutil.go deleted file mode 100644 index eb4a4f3a762f..000000000000 --- a/pkg/kv/kvserver/rangefeed/testutil.go +++ /dev/null @@ -1,15 +0,0 @@ -// Copyright 2023 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -package rangefeed - -func NewTestProcessor(id int64) Processor { - if id > 0 { - return &ScheduledProcessor{ - scheduler: ClientScheduler{id: id}, - } - } - return &LegacyProcessor{} -} diff --git a/pkg/kv/kvserver/replica_rangefeed.go b/pkg/kv/kvserver/replica_rangefeed.go index cd24cd9bf1e2..d1e8b39c6d29 100644 --- a/pkg/kv/kvserver/replica_rangefeed.go +++ b/pkg/kv/kvserver/replica_rangefeed.go @@ -31,7 +31,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/cockroachdb/cockroach/pkg/util/metamorphic" "github.com/cockroachdb/cockroach/pkg/util/syncutil/singleflight" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/uuid" @@ -82,14 +81,10 @@ var RangeFeedUseScheduler = settings.RegisterBoolSetting( "use shared fixed pool of workers for all range feeds instead of a "+ "worker per range (worker pool size is determined by "+ "COCKROACH_RANGEFEED_SCHEDULER_WORKERS env variable)", - metamorphic.ConstantWithTestBool("kv_rangefeed_scheduler_enabled", true), + true, + settings.Retired, ) -// RangefeedSchedulerDisabled is a kill switch for scheduler based rangefeed -// processors. To be removed in 24.1 after new processor becomes default. -var RangefeedSchedulerDisabled = envutil.EnvOrDefaultBool("COCKROACH_RANGEFEED_DISABLE_SCHEDULER", - false) - // RangefeedUseBufferedSender controls whether rangefeed uses a node level // buffered sender to buffer events instead of buffering events separately in a // channel at a per client per registration level. It is currently left @@ -483,29 +478,23 @@ func (r *Replica) registerWithRangefeedRaftMuLocked( // Create a new rangefeed. feedBudget := r.store.GetStoreConfig().RangefeedBudgetFactory.CreateBudget(isSystemSpan) - var sched *rangefeed.Scheduler - if shouldUseRangefeedScheduler(&r.ClusterSettings().SV) { - sched = r.store.getRangefeedScheduler() - } - desc := r.Desc() tp := rangefeedTxnPusher{ir: r.store.intentResolver, r: r, span: desc.RSpan()} cfg := rangefeed.Config{ - AmbientContext: r.AmbientContext, - Clock: r.Clock(), - Stopper: r.store.stopper, - Settings: r.store.ClusterSettings(), - RangeID: r.RangeID, - Span: desc.RSpan(), - TxnPusher: &tp, - PushTxnsInterval: r.store.TestingKnobs().RangeFeedPushTxnsInterval, - PushTxnsAge: r.store.TestingKnobs().RangeFeedPushTxnsAge, - EventChanCap: defaultEventChanCap, - EventChanTimeout: defaultEventChanTimeout, - Metrics: r.store.metrics.RangeFeedMetrics, - MemBudget: feedBudget, - Scheduler: sched, - Priority: isSystemSpan, // only takes effect when Scheduler != nil + AmbientContext: r.AmbientContext, + Clock: r.Clock(), + Stopper: r.store.stopper, + Settings: r.store.ClusterSettings(), + RangeID: r.RangeID, + Span: desc.RSpan(), + TxnPusher: &tp, + PushTxnsAge: r.store.TestingKnobs().RangeFeedPushTxnsAge, + EventChanCap: defaultEventChanCap, + EventChanTimeout: defaultEventChanTimeout, + Metrics: r.store.metrics.RangeFeedMetrics, + MemBudget: feedBudget, + Scheduler: r.store.getRangefeedScheduler(), + Priority: isSystemSpan, // only takes effect when Scheduler != nil UnregisterFromReplica: r.unsetRangefeedProcessor, } p = rangefeed.NewProcessor(cfg) @@ -954,10 +943,6 @@ func (r *Replica) ensureClosedTimestampStarted(ctx context.Context) *kvpb.Error return nil } -func shouldUseRangefeedScheduler(sv *settings.Values) bool { - return RangeFeedUseScheduler.Get(sv) && !RangefeedSchedulerDisabled -} - // TestGetReplicaRangefeedProcessor exposes rangefeed processor for test // introspection. Note that while retrieving processor is threadsafe, invoking // processor methods should be done with caution to not break any invariants. diff --git a/pkg/kv/kvserver/replica_rangefeed_test.go b/pkg/kv/kvserver/replica_rangefeed_test.go index 14ef6aca2013..9f718e2c7e93 100644 --- a/pkg/kv/kvserver/replica_rangefeed_test.go +++ b/pkg/kv/kvserver/replica_rangefeed_test.go @@ -16,13 +16,11 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" - clientrf "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed" "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/rangefeed" "github.com/cockroachdb/cockroach/pkg/raft/raftpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server" @@ -729,61 +727,6 @@ func TestReplicaRangefeedOriginIDFiltering(t *testing.T) { }) } -func TestScheduledProcessorKillSwitch(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - kvserver.RangefeedSchedulerDisabled = true - defer func() { kvserver.RangefeedSchedulerDisabled = false }() - - ctx := context.Background() - ts, err := serverutils.NewServer(base.TestServerArgs{ - DefaultTestTenant: base.TestIsSpecificToStorageLayerAndNeedsASystemTenant, - }) - require.NoError(t, err, "failed to start test server") - require.NoError(t, ts.Start(ctx), "start server") - defer ts.Stopper().Stop(ctx) - - db := ts.SystemLayer().SQLConn(t) - _, err = db.Exec("set cluster setting kv.rangefeed.enabled = t") - require.NoError(t, err, "can't enable rangefeeds") - _, err = db.Exec("set cluster setting kv.rangefeed.scheduler.enabled = t") - require.NoError(t, err, "can't enable rangefeed scheduler") - - sr, err := ts.ScratchRange() - require.NoError(t, err, "can't create scratch range") - f := ts.RangeFeedFactory().(*clientrf.Factory) - rf, err := f.RangeFeed(ctx, "test-feed", []roachpb.Span{{Key: sr, EndKey: sr.PrefixEnd()}}, - hlc.Timestamp{}, - func(ctx context.Context, value *kvpb.RangeFeedValue) {}, - ) - require.NoError(t, err, "failed to start rangefeed") - defer rf.Close() - - rd, err := ts.LookupRange(sr) - require.NoError(t, err, "failed to get descriptor for scratch range") - - stores := ts.GetStores().(*kvserver.Stores) - _ = stores.VisitStores(func(s *kvserver.Store) error { - repl, err := s.GetReplica(rd.RangeID) - require.NoError(t, err, "failed to find scratch range replica in store") - var proc rangefeed.Processor - // Note that we can't rely on checkpoint or event because client rangefeed - // call can return and emit first checkpoint and data before processor is - // actually attached to replica. - testutils.SucceedsSoon(t, func() error { - proc = kvserver.TestGetReplicaRangefeedProcessor(repl) - if proc == nil { - return errors.New("scratch range must have processor") - } - return nil - }) - require.IsType(t, (*rangefeed.LegacyProcessor)(nil), proc, - "kill switch didn't prevent scheduled processor creation") - return nil - }) -} - func TestReplicaRangefeedErrors(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t)