Skip to content

Commit

Permalink
kvserver/rangefeed: add unbuffered registration
Browse files Browse the repository at this point in the history
This patch adds unbufferedRegistration.

UnbufferedRegistration is like BufferedRegistration but uses BufferedSender to
buffer live raft updates instead of a using fixed size channel and having a
dedicated per-registration goroutine to volley events to underlying gRPC
stream. Instead, there is only one BufferedSender for each incoming
node.MuxRangefeed gRPC call. BufferedSender is responsible for buffering and
sending its updates to the underlying gRPC stream in a dedicated goroutine
O(node).

Resolved: cockroachdb#110432

Release note: none

Co-authored-by: Steven Danna <[email protected]>
  • Loading branch information
wenyihu6 and stevendanna committed Dec 7, 2024
1 parent f9a6082 commit cb21a6f
Show file tree
Hide file tree
Showing 12 changed files with 1,566 additions and 470 deletions.
2 changes: 2 additions & 0 deletions pkg/kv/kvserver/rangefeed/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ go_library(
"stream_manager.go",
"task.go",
"test_helpers.go",
"unbuffered_registration.go",
"unbuffered_sender.go",
],
importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/rangefeed",
Expand Down Expand Up @@ -77,6 +78,7 @@ go_test(
"sender_helper_test.go",
"stream_manager_test.go",
"task_test.go",
"unbuffered_registration_test.go",
"unbuffered_sender_test.go",
],
embed = [":rangefeed"],
Expand Down
12 changes: 12 additions & 0 deletions pkg/kv/kvserver/rangefeed/buffered_registration.go
Original file line number Diff line number Diff line change
Expand Up @@ -347,3 +347,15 @@ var overflowLogEvery = log.Every(5 * time.Second)
func (br *bufferedRegistration) shouldLogOverflow(checkpointSent bool) bool {
return (!checkpointSent) || log.V(1) || overflowLogEvery.ShouldLog()
}

// Used for testing only.
func (br *bufferedRegistration) getBuf() chan *sharedEvent {
return br.buf
}

// Used for testing only.
func (br *bufferedRegistration) getOverflowed() bool {
br.mu.Lock()
defer br.mu.Unlock()
return br.mu.overflowed
}
3 changes: 2 additions & 1 deletion pkg/kv/kvserver/rangefeed/buffered_sender_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -71,7 +71,8 @@ func TestBufferedSenderDisconnectStream(t *testing.T) {
require.NoError(t, bs.waitForEmptyBuffer(ctx))
sm.DisconnectStream(int64(streamID), err)
require.NoError(t, bs.waitForEmptyBuffer(ctx))
require.Equal(t, 1, testServerStream.totalEventsSent())
require.Equalf(t, 1, testServerStream.totalEventsSent(),
"expected only 1 error event in %s", testServerStream.String())
testRangefeedCounter.waitForRangefeedCount(t, 0)
})
}
Expand Down
54 changes: 46 additions & 8 deletions pkg/kv/kvserver/rangefeed/processor_helpers_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,24 @@ import (
"github.com/stretchr/testify/require"
)

type testBufferedStream struct {
Stream
}

var _ BufferedStream = (*testBufferedStream)(nil)

func (tb *testBufferedStream) SendBuffered(
e *kvpb.RangeFeedEvent, _ *SharedBudgetAllocation,
) error {
// In production code, buffered stream would be responsible for properly using
// and releasing the alloc. We just ignore memory accounting here.
return tb.SendUnbuffered(e)
}

func (tb *testBufferedStream) Disconnect(err *kvpb.Error) {
tb.Stream.SendError(err)
}

func makeLogicalOp(val interface{}) enginepb.MVCCLogicalOp {
var op enginepb.MVCCLogicalOp
op.MustSetValue(val)
Expand Down Expand Up @@ -189,11 +207,12 @@ const testProcessorEventCCap = 16
const testProcessorEventCTimeout = 10 * time.Millisecond

type processorTestHelper struct {
span roachpb.RSpan
rts *resolvedTimestamp
syncEventC func()
sendSpanSync func(*roachpb.Span)
scheduler *ClientScheduler
span roachpb.RSpan
rts *resolvedTimestamp
syncEventC func()
sendSpanSync func(*roachpb.Span)
scheduler *ClientScheduler
toBufferedStreamIfNeeded func(s Stream) Stream
}

// syncEventAndRegistrations waits for all previously sent events to be
Expand Down Expand Up @@ -237,16 +256,23 @@ type rangefeedTestType bool

var (
scheduledProcessorWithUnbufferedSender rangefeedTestType
scheduledProcessorWithBufferedSender rangefeedTestType
)

var testTypes = []rangefeedTestType{
scheduledProcessorWithUnbufferedSender,
scheduledProcessorWithBufferedSender,
}

// NB: When adding new types, please keep make sure existing
// benchmarks will keep their old name.
func (t rangefeedTestType) String() string {
return "scheduled"
switch t {
case scheduledProcessorWithUnbufferedSender:
return "scheduled"
case scheduledProcessorWithBufferedSender:
return "scheduled/registration=buffered"
default:
panic("unknown rangefeed test type")
}
}

type testConfig struct {
Expand Down Expand Up @@ -426,6 +452,18 @@ func newTestProcessor(
p.syncSendAndWait(&syncEvent{c: make(chan struct{}), testRegCatchupSpan: span})
}
h.scheduler = &p.scheduler
switch cfg.feedType {
case scheduledProcessorWithUnbufferedSender:
h.toBufferedStreamIfNeeded = func(s Stream) Stream {
return s
}
case scheduledProcessorWithBufferedSender:
h.toBufferedStreamIfNeeded = func(s Stream) Stream {
return &testBufferedStream{Stream: s}
}
default:
panic("unknown rangefeed test type")
}
default:
panic("unknown processor type")
}
Expand Down
Loading

0 comments on commit cb21a6f

Please sign in to comment.