Skip to content

Commit 72a8b4e

Browse files
author
Yevgeniy Miretskiy
committed
kvcoord: Implement CloseStream for MuxRangeFeed
Extend MuxRangeFeed protocol to support explicit, caller initiated CloseStream operation. The caller may decide to stop receiving events for a particular stream, which is part of MuxRangeFeed. The caller may issue a request to MuxRangeFeed server to close the stream. The server will cancel underlying range feed, and return a `RangeFeedRetryError_REASON_RANGEFEED_CLOSED` error as a response. Note, current mux rangefeed clinet does not use this request. The code to support cancellation is added pre-emptively in case this functionality will be required in the future to support restarts due to stuck rangefeeds. Epic: CRDB-26372 Release note: None
1 parent 29c1a7c commit 72a8b4e

File tree

5 files changed

+236
-19
lines changed

5 files changed

+236
-19
lines changed

pkg/kv/kvclient/kvcoord/dist_sender_mux_rangefeed.go

Lines changed: 27 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -17,10 +17,12 @@ import (
1717
"sync/atomic"
1818
"unsafe"
1919

20+
"github.com/cockroachdb/cockroach/pkg/clusterversion"
2021
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangecache"
2122
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
2223
"github.com/cockroachdb/cockroach/pkg/roachpb"
2324
"github.com/cockroachdb/cockroach/pkg/rpc"
25+
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
2426
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
2527
"github.com/cockroachdb/cockroach/pkg/util/future"
2628
"github.com/cockroachdb/cockroach/pkg/util/grpcutil"
@@ -308,6 +310,17 @@ func (s *activeMuxRangeFeed) start(ctx context.Context, m *rangefeedMuxer) error
308310
}
309311
continue
310312
}
313+
314+
if m.cfg.knobs.captureMuxRangeFeedRequestSender != nil {
315+
m.cfg.knobs.captureMuxRangeFeedRequestSender(
316+
args.Replica.NodeID,
317+
func(req *kvpb.RangeFeedRequest) error {
318+
conn.mu.Lock()
319+
defer conn.mu.Unlock()
320+
return conn.mu.sender.Send(req)
321+
})
322+
}
323+
311324
return nil
312325
}
313326

@@ -441,7 +454,7 @@ func (m *rangefeedMuxer) receiveEventsFromNode(
441454
}
442455

443456
if m.cfg.knobs.onRangefeedEvent != nil {
444-
skip, err := m.cfg.knobs.onRangefeedEvent(ctx, active.Span, &event.RangeFeedEvent)
457+
skip, err := m.cfg.knobs.onRangefeedEvent(ctx, active.Span, event.StreamID, &event.RangeFeedEvent)
445458
if err != nil {
446459
return err
447460
}
@@ -604,3 +617,16 @@ func (c *muxStream) close() []*activeMuxRangeFeed {
604617

605618
return toRestart
606619
}
620+
621+
// NewCloseStreamRequest returns a mux rangefeed request to close specified stream.
622+
func NewCloseStreamRequest(
623+
ctx context.Context, st *cluster.Settings, streamID int64,
624+
) (*kvpb.RangeFeedRequest, error) {
625+
if !st.Version.IsActive(ctx, clusterversion.V23_2) {
626+
return nil, errors.Newf("CloseStream request requires cluster version 23.2 or above, found %s", st.Version)
627+
}
628+
return &kvpb.RangeFeedRequest{
629+
StreamID: streamID,
630+
CloseStream: true,
631+
}, nil
632+
}

pkg/kv/kvclient/kvcoord/dist_sender_rangefeed.go

Lines changed: 17 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -97,9 +97,13 @@ type rangeFeedConfig struct {
9797
// onRangefeedEvent invoked on each rangefeed event.
9898
// Returns boolean indicating if event should be skipped or an error
9999
// indicating if rangefeed should terminate.
100-
onRangefeedEvent func(ctx context.Context, s roachpb.Span, event *kvpb.RangeFeedEvent) (skip bool, _ error)
100+
// streamID set only for mux rangefeed.
101+
onRangefeedEvent func(ctx context.Context, s roachpb.Span, muxStreamID int64, event *kvpb.RangeFeedEvent) (skip bool, _ error)
101102
// metrics overrides rangefeed metrics to use.
102103
metrics *DistSenderRangeFeedMetrics
104+
// captureMuxRangeFeedRequestSender is a callback invoked when mux
105+
// rangefeed establishes connection to the node.
106+
captureMuxRangeFeedRequestSender func(nodeID roachpb.NodeID, sender func(req *kvpb.RangeFeedRequest) error)
103107
}
104108
}
105109

@@ -795,7 +799,7 @@ func (ds *DistSender) singleRangeFeed(
795799
}
796800

797801
if cfg.knobs.onRangefeedEvent != nil {
798-
skip, err := cfg.knobs.onRangefeedEvent(ctx, span, event)
802+
skip, err := cfg.knobs.onRangefeedEvent(ctx, span, 0 /*streamID */, event)
799803
if err != nil {
800804
return args.Timestamp, err
801805
}
@@ -871,7 +875,7 @@ var errRestartStuckRange = errors.New("rangefeed restarting due to inactivity")
871875

872876
// TestingWithOnRangefeedEvent returns a test only option to modify rangefeed event.
873877
func TestingWithOnRangefeedEvent(
874-
fn func(ctx context.Context, s roachpb.Span, event *kvpb.RangeFeedEvent) (skip bool, _ error),
878+
fn func(ctx context.Context, s roachpb.Span, streamID int64, event *kvpb.RangeFeedEvent) (skip bool, _ error),
875879
) RangeFeedOption {
876880
return optionFunc(func(c *rangeFeedConfig) {
877881
c.knobs.onRangefeedEvent = fn
@@ -886,5 +890,15 @@ func TestingWithRangeFeedMetrics(m *DistSenderRangeFeedMetrics) RangeFeedOption
886890
})
887891
}
888892

893+
// TestingWithMuxRangeFeedRequestSenderCapture returns a test only option to specify a callback
894+
// that will be invoked when mux establishes connection to a node.
895+
func TestingWithMuxRangeFeedRequestSenderCapture(
896+
fn func(nodeID roachpb.NodeID, capture func(request *kvpb.RangeFeedRequest) error),
897+
) RangeFeedOption {
898+
return optionFunc(func(c *rangeFeedConfig) {
899+
c.knobs.captureMuxRangeFeedRequestSender = fn
900+
})
901+
}
902+
889903
// TestingMakeRangeFeedMetrics exposes makeDistSenderRangeFeedMetrics for test use.
890904
var TestingMakeRangeFeedMetrics = makeDistSenderRangeFeedMetrics

pkg/kv/kvclient/kvcoord/dist_sender_rangefeed_test.go

Lines changed: 136 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -12,6 +12,8 @@ package kvcoord_test
1212

1313
import (
1414
"context"
15+
"math/rand"
16+
"sync"
1517
"sync/atomic"
1618
"testing"
1719
"time"
@@ -629,7 +631,7 @@ func TestMuxRangeCatchupScanQuotaReleased(t *testing.T) {
629631
enoughErrors := make(chan struct{})
630632
closeFeed := rangeFeed(ts.DistSenderI(), fooSpan, startTime, noValuesExpected, true,
631633
kvcoord.TestingWithOnRangefeedEvent(
632-
func(_ context.Context, _ roachpb.Span, event *kvpb.RangeFeedEvent) (skip bool, _ error) {
634+
func(_ context.Context, _ roachpb.Span, _ int64, event *kvpb.RangeFeedEvent) (skip bool, _ error) {
633635
*event = transientErrEvent
634636
if numErrors.Add(1) == numErrsToReturn {
635637
close(enoughErrors)
@@ -727,7 +729,7 @@ func TestRangeFeedMetricsManagement(t *testing.T) {
727729
closeFeed := rangeFeed(ts.DistSenderI(), fooSpan, startTime, ignoreValues, useMux,
728730
kvcoord.TestingWithRangeFeedMetrics(&metrics),
729731
kvcoord.TestingWithOnRangefeedEvent(
730-
func(ctx context.Context, s roachpb.Span, event *kvpb.RangeFeedEvent) (skip bool, _ error) {
732+
func(ctx context.Context, s roachpb.Span, _ int64, event *kvpb.RangeFeedEvent) (skip bool, _ error) {
731733
switch t := event.GetValue().(type) {
732734
case *kvpb.RangeFeedValue:
733735
// If we previously arranged for the range to be skipped (stuck catchup scan),
@@ -802,3 +804,135 @@ func TestRangeFeedMetricsManagement(t *testing.T) {
802804
require.EqualValues(t, numCatchupToBlock, metrics.RangefeedCatchupRanges.Value())
803805
})
804806
}
807+
808+
// TestMuxRangeFeedCanCloseStream verifies stream termination functionality in mux rangefeed.
809+
func TestMuxRangeFeedCanCloseStream(t *testing.T) {
810+
defer leaktest.AfterTest(t)()
811+
defer log.Scope(t).Close(t)
812+
813+
ctx := context.Background()
814+
tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{
815+
ReplicationMode: base.ReplicationManual,
816+
})
817+
defer tc.Stopper().Stop(ctx)
818+
819+
ts := tc.Server(0)
820+
sqlDB := sqlutils.MakeSQLRunner(tc.ServerConn(0))
821+
822+
// Insert 1000 rows, and split them into 10 ranges.
823+
sqlDB.ExecMultiple(t,
824+
`SET CLUSTER SETTING kv.rangefeed.enabled = true`,
825+
`SET CLUSTER SETTING kv.closed_timestamp.target_duration='100ms'`,
826+
`ALTER DATABASE defaultdb CONFIGURE ZONE USING num_replicas = 1`,
827+
`CREATE TABLE foo (key INT PRIMARY KEY)`,
828+
`INSERT INTO foo (key) SELECT * FROM generate_series(1, 1000)`,
829+
`ALTER TABLE foo SPLIT AT (SELECT * FROM generate_series(100, 900, 100))`,
830+
)
831+
832+
fooDesc := desctestutils.TestingGetPublicTableDescriptor(
833+
ts.DB(), keys.SystemSQLCodec, "defaultdb", "foo")
834+
fooSpan := fooDesc.PrimaryIndexSpan(keys.SystemSQLCodec)
835+
836+
frontier, err := span.MakeFrontier(fooSpan)
837+
require.NoError(t, err)
838+
839+
expectFrontierAdvance := func() {
840+
t.Helper()
841+
// Closed timestamp for range advances every100ms. We'll require frontier to
842+
// advance a bit more thn that.
843+
threshold := frontier.Frontier().AddDuration(250 * time.Millisecond)
844+
testutils.SucceedsWithin(t, func() error {
845+
if frontier.Frontier().Less(threshold) {
846+
return errors.Newf("waiting for frontier advance to at least %s", threshold)
847+
}
848+
return nil
849+
}, 10*time.Second)
850+
}
851+
852+
var observedStreams sync.Map
853+
var capturedSender atomic.Value
854+
855+
ignoreValues := func(event kvcoord.RangeFeedMessage) {}
856+
var numRestartStreams atomic.Int32
857+
858+
closeFeed := rangeFeed(ts.DistSenderI(), fooSpan, ts.Clock().Now(), ignoreValues, true,
859+
kvcoord.WithMuxRangeFeed(),
860+
kvcoord.TestingWithMuxRangeFeedRequestSenderCapture(
861+
// We expect a single mux sender since we have 1 node in this test.
862+
func(nodeID roachpb.NodeID, capture func(request *kvpb.RangeFeedRequest) error) {
863+
capturedSender.Store(capture)
864+
},
865+
),
866+
kvcoord.TestingWithOnRangefeedEvent(
867+
func(ctx context.Context, s roachpb.Span, streamID int64, event *kvpb.RangeFeedEvent) (skip bool, _ error) {
868+
switch t := event.GetValue().(type) {
869+
case *kvpb.RangeFeedCheckpoint:
870+
observedStreams.Store(streamID, nil)
871+
_, err := frontier.Forward(t.Span, t.ResolvedTS)
872+
if err != nil {
873+
return true, err
874+
}
875+
case *kvpb.RangeFeedError:
876+
// Keep track of mux errors due to RangeFeedRetryError_REASON_RANGEFEED_CLOSED.
877+
// Those results when we issue CloseStream request.
878+
err := t.Error.GoError()
879+
log.Infof(ctx, "Got err: %v", err)
880+
var retryErr *kvpb.RangeFeedRetryError
881+
if ok := errors.As(err, &retryErr); ok && retryErr.Reason == kvpb.RangeFeedRetryError_REASON_RANGEFEED_CLOSED {
882+
numRestartStreams.Add(1)
883+
}
884+
}
885+
886+
return false, nil
887+
}),
888+
)
889+
defer closeFeed()
890+
891+
// Wait until we capture mux rangefeed request sender. There should only be 1.
892+
var muxRangeFeedRequestSender func(req *kvpb.RangeFeedRequest) error
893+
testutils.SucceedsWithin(t, func() error {
894+
v, ok := capturedSender.Load().(func(request *kvpb.RangeFeedRequest) error)
895+
if ok {
896+
muxRangeFeedRequestSender = v
897+
return nil
898+
}
899+
return errors.New("waiting to capture mux rangefeed request sender.")
900+
}, 10*time.Second)
901+
902+
cancelledStreams := make(map[int64]struct{})
903+
for i := 0; i < 5; i++ {
904+
// Wait for the test frontier to advance. Once it advances,
905+
// we know the rangefeed is started, all ranges are running.
906+
expectFrontierAdvance()
907+
908+
// Pick some number of streams to close. Since sync.Map iteration order is non-deterministic,
909+
// we'll pick few random streams.
910+
initialClosed := numRestartStreams.Load()
911+
numToCancel := 1 + rand.Int31n(3)
912+
var numCancelled int32 = 0
913+
observedStreams.Range(func(key any, _ any) bool {
914+
streamID := key.(int64)
915+
if _, wasCancelled := cancelledStreams[streamID]; wasCancelled {
916+
return true // try another stream.
917+
}
918+
numCancelled++
919+
cancelledStreams[streamID] = struct{}{}
920+
req, err := kvcoord.NewCloseStreamRequest(ctx, ts.ClusterSettings(), streamID)
921+
require.NoError(t, err)
922+
require.NoError(t, muxRangeFeedRequestSender(req))
923+
return numCancelled < numToCancel
924+
})
925+
926+
// Observe numToCancel errors.
927+
testutils.SucceedsWithin(t, func() error {
928+
numRestarted := numRestartStreams.Load()
929+
if numRestarted == initialClosed+numCancelled {
930+
return nil
931+
}
932+
return errors.Newf("waiting for %d streams to be closed (%d so far)", numCancelled, numRestarted-initialClosed)
933+
}, 10*time.Second)
934+
935+
// When we close the stream(s), the rangefeed server responds with a retryable error.
936+
// Mux rangefeed should retry, and thus we expect frontier to keep advancing.
937+
}
938+
}

pkg/kv/kvpb/api.proto

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2986,6 +2986,13 @@ message RangeFeedRequest {
29862986

29872987
// StreamID is set by the client issuing MuxRangeFeed requests.
29882988
int64 stream_id = 5 [(gogoproto.customname) = "StreamID"];
2989+
// CloseStream is set by the mux RangeFeed client to indicate that
2990+
// the server should close the stream with the specified stream_id.
2991+
// When this bit is set, the server should attempt, as best effort, to
2992+
// quickly terminate rangefeed for this stream.
2993+
// When CloseStream is set, only the StreamID must be set, and
2994+
// other fields (such as Span) are ignored.
2995+
bool close_stream = 6;
29892996
}
29902997

29912998
// RangeFeedValue is a variant of RangeFeedEvent that represents an update to

0 commit comments

Comments
 (0)