Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

pkg/p2p(cdc,dataflow): Make it safe to write to the same topic concurrently #4808

Merged
merged 3 commits into from
Mar 8, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
118 changes: 52 additions & 66 deletions pkg/p2p/client.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (
"github.com/pingcap/failpoint"
"github.com/pingcap/log"
cerrors "github.com/pingcap/tiflow/pkg/errors"
"github.com/pingcap/tiflow/pkg/p2p/internal"
"github.com/pingcap/tiflow/pkg/security"
"github.com/pingcap/tiflow/proto/p2p"
"github.com/prometheus/client_golang/prometheus"
Expand Down Expand Up @@ -58,7 +59,7 @@ type MessageClientConfig struct {
// MessageClient is a client used to send peer messages.
// `Run` must be running before sending any message.
type MessageClient struct {
sendCh chan *p2p.MessageEntry
sendCh *internal.SendChan

topicMu sync.RWMutex
topics map[string]*topicEntry
Expand Down Expand Up @@ -91,7 +92,7 @@ type topicEntry struct {
// senderID is an identifier for the local node.
func NewMessageClient(senderID NodeID, config *MessageClientConfig) *MessageClient {
return &MessageClient{
sendCh: make(chan *p2p.MessageEntry, config.SendChannelSize),
sendCh: internal.NewSendChan(int64(config.SendChannelSize)),
topics: make(map[string]*topicEntry),
senderID: senderID,
closeCh: make(chan struct{}),
Expand Down Expand Up @@ -227,48 +228,45 @@ func (c *MessageClient) runTx(ctx context.Context, stream clientStream) error {
batchSender := c.newSenderFn(stream)

for {
select {
case <-ctx.Done():
return errors.Trace(ctx.Err())
case msg, ok := <-c.sendCh:
if !ok {
// sendCh has been closed
return nil
msg, ok, err := c.sendCh.Receive(ctx, ticker.C)
if err != nil {
return errors.Trace(err)
}
if !ok {
// The implementation of batchSender guarantees that
// an empty flush does not send any message.
if err := batchSender.Flush(); err != nil {
return errors.Trace(err)
}
continue
}

c.topicMu.RLock()
tpk, ok := c.topics[msg.Topic]
c.topicMu.RUnlock()
if !ok {
// This line should never be reachable unless there is a bug in this file.
log.Panic("topic not found. Report a bug", zap.String("topic", msg.Topic))
}
c.topicMu.RLock()
tpk, ok := c.topics[msg.Topic]
c.topicMu.RUnlock()
if !ok {
// This line should never be reachable unless there is a bug in this file.
log.Panic("topic not found. Report a bug", zap.String("topic", msg.Topic))
}

// We want to assert that `msg.Sequence` is continuous within a topic.
if old := tpk.lastSent.Swap(msg.Sequence); old != initAck && msg.Sequence != old+1 {
log.Panic("unexpected seq of message",
zap.String("topic", msg.Topic),
zap.Int64("seq", msg.Sequence))
}
// We want to assert that `msg.Sequence` is continuous within a topic.
if old := tpk.lastSent.Swap(msg.Sequence); old != initAck && msg.Sequence != old+1 {
log.Panic("unexpected seq of message",
zap.String("topic", msg.Topic),
zap.Int64("seq", msg.Sequence))
}

tpk.sentMessageMu.Lock()
tpk.sentMessages.PushBack(msg)
tpk.sentMessageMu.Unlock()
tpk.sentMessageMu.Lock()
tpk.sentMessages.PushBack(msg)
tpk.sentMessageMu.Unlock()

metricsClientMessageCount.Inc()
metricsClientMessageCount.Inc()

log.Debug("Sending Message",
zap.String("topic", msg.Topic),
zap.Int64("seq", msg.Sequence))
if err := batchSender.Append(msg); err != nil {
return errors.Trace(err)
}
case <-ticker.C:
// The implementation of batchSender guarantees that
// an empty flush does not send any message.
if err := batchSender.Flush(); err != nil {
return errors.Trace(err)
}
log.Debug("Sending Message",
zap.String("topic", msg.Topic),
zap.Int64("seq", msg.Sequence))
if err := batchSender.Append(msg); err != nil {
return errors.Trace(err)
}
}
}
Expand Down Expand Up @@ -381,6 +379,7 @@ func (c *MessageClient) runRx(ctx context.Context, stream clientStream) error {
// SendMessage sends a message. It will block if the client is not ready to
// accept the message for now. Once the function returns without an error,
// the client will try its best to send the message, until `Run` is canceled.
// NOTE The function involves busy-waiting, so DO NOT use it when you expect contention or congestion!
func (c *MessageClient) SendMessage(ctx context.Context, topic Topic, value interface{}) (seq Seq, ret error) {
return c.sendMessage(ctx, topic, value, false)
}
Expand Down Expand Up @@ -415,47 +414,34 @@ func (c *MessageClient) sendMessage(ctx context.Context, topic Topic, value inte
tpk = &topicEntry{
sentMessages: deque.NewDeque(),
}
// the sequence = 0 is reserved.
tpk.nextSeq.Store(1)
tpk.nextSeq.Store(0)
c.topicMu.Lock()
c.topics[topic] = tpk
if newTpk, ok := c.topics[topic]; !ok {
c.topics[topic] = tpk
} else {
tpk = newTpk
}
c.topicMu.Unlock()
}

nextSeq := tpk.nextSeq.Load()

data, err := marshalMessage(value)
if err != nil {
return 0, cerrors.WrapError(cerrors.ErrPeerMessageEncodeError, err)
}

msg := &p2p.MessageEntry{
Topic: topic,
Content: data,
Sequence: nextSeq,
}

if nonblocking {
select {
case <-ctx.Done():
return 0, errors.Trace(ctx.Err())
case c.sendCh <- msg:
default:
ok, seq := c.sendCh.SendAsync(topic, data, tpk.nextSeq.Inc)
if !ok {
return 0, cerrors.ErrPeerMessageSendTryAgain.GenWithStackByArgs()
}
} else {
// blocking
select {
case <-ctx.Done():
return 0, errors.Trace(ctx.Err())
case <-c.closeCh:
return 0, cerrors.ErrPeerMessageClientClosed.GenWithStackByArgs()
case c.sendCh <- msg:
}
return seq, nil
}

tpk.nextSeq.Add(1)
return nextSeq, nil
// blocking
seq, err = c.sendCh.SendSync(ctx, topic, data, c.closeCh, tpk.nextSeq.Inc)
if err != nil {
return 0, errors.Trace(err)
}
return seq, nil
}

// CurrentAck returns (s, true) if all messages with sequence less than or
Expand Down
5 changes: 4 additions & 1 deletion pkg/p2p/client_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,7 @@ type testMessage struct {
}

var clientConfigForUnitTesting = &MessageClientConfig{
SendChannelSize: 0, // unbuffered channel to make tests more reliable
SendChannelSize: 1,
BatchSendInterval: 128 * time.Hour, // essentially disables flushing
MaxBatchBytes: math.MaxInt64,
MaxBatchCount: math.MaxInt64,
Expand Down Expand Up @@ -331,6 +331,9 @@ func TestClientSendAnomalies(t *testing.T) {

// Test point 1: ErrPeerMessageSendTryAgain
_, err := client.TrySendMessage(ctx, "test-topic", &testMessage{Value: 1})
require.NoError(t, err)

_, err = client.TrySendMessage(ctx, "test-topic", &testMessage{Value: 1})
require.Error(t, err)
require.Regexp(t, ".*ErrPeerMessageSendTryAgain.*", err.Error())

Expand Down
153 changes: 153 additions & 0 deletions pkg/p2p/internal/send_chan.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,153 @@
// Copyright 2021 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.

package internal

import (
"context"
"sync"
"time"

"github.com/pingcap/errors"
"github.com/pingcap/log"
cerrors "github.com/pingcap/tiflow/pkg/errors"
proto "github.com/pingcap/tiflow/proto/p2p"
"go.uber.org/zap"
)

// SendChan is a specialized channel used to implement
// the asynchronous interface of the MessageClient.
// SendChan is a MPSC channel.
type SendChan struct {
mu sync.Mutex
buf []*proto.MessageEntry
sendIdx int64
recvIdx int64

ready chan struct{}

cap int64
}

// NewSendChan returns a new SendChan.
func NewSendChan(cap int64) *SendChan {
return &SendChan{
buf: make([]*proto.MessageEntry, cap),
ready: make(chan struct{}, 1),
cap: cap,
}
}

// SendSync sends a message synchronously.
// NOTE SendSync employs busy waiting and is ONLY suitable
// for writing Unit-Tests.
func (c *SendChan) SendSync(
ctx context.Context,
topic string,
value []byte,
closeCh <-chan struct{},
nextSeq func() int64,
) (int64, error) {
for {
select {
case <-ctx.Done():
return 0, errors.Trace(ctx.Err())
case <-closeCh:
return 0, cerrors.ErrPeerMessageClientClosed.GenWithStackByArgs()
default:
}

if ok, seq := c.SendAsync(topic, value, nextSeq); ok {
return seq, nil
}
// Used to reduce contention when race-detector is enabled
time.Sleep(1 * time.Millisecond)
}
}

// SendAsync tries to send a message. If the message is accepted, nextSeq will be called
// once, and the returned value will be used as the Sequence number of the message.
func (c *SendChan) SendAsync(topic string, value []byte, nextSeq func() int64) (ok bool, seq int64) {
c.mu.Lock()
defer c.mu.Unlock()

if c.sendIdx-c.recvIdx > c.cap {
log.Panic("unreachable",
zap.Int64("sendIdx", c.sendIdx),
zap.Int64("recvIndex", c.recvIdx))
}

if c.sendIdx-c.recvIdx == c.cap {
return false, 0
}

seq = nextSeq()
c.buf[c.sendIdx%c.cap] = &proto.MessageEntry{
Topic: topic,
Content: value,
Sequence: seq,
}
c.sendIdx++

select {
case c.ready <- struct{}{}:
default:
}

return true, seq
}

// Receive receives one message from the channel. If there is a tick, the function will return
// (nil, false, nil).
func (c *SendChan) Receive(ctx context.Context, tick <-chan time.Time) (*proto.MessageEntry, bool, error) {
select {
case <-ctx.Done():
return nil, false, errors.Trace(ctx.Err())
default:
}

for {
entry := c.doReceive()
if entry != nil {
return entry, true, nil
}

select {
case <-ctx.Done():
return nil, false, errors.Trace(ctx.Err())
case <-tick:
return nil, false, nil
case <-c.ready:
}
}
}

func (c *SendChan) doReceive() *proto.MessageEntry {
c.mu.Lock()
defer c.mu.Unlock()

if c.sendIdx < c.recvIdx {
log.Panic("unreachable",
zap.Int64("sendIdx", c.sendIdx),
zap.Int64("recvIndex", c.recvIdx))
}

if c.sendIdx == c.recvIdx {
return nil
}

var ret *proto.MessageEntry
ret, c.buf[c.recvIdx%c.cap] = c.buf[c.recvIdx%c.cap], nil
c.recvIdx++
return ret
}
Loading