github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/pkg/p2p/internal/send_chan.go (about) 1 // Copyright 2022 PingCAP, Inc. 2 // 3 // Licensed under the Apache License, Version 2.0 (the "License"); 4 // you may not use this file except in compliance with the License. 5 // You may obtain a copy of the License at 6 // 7 // http://www.apache.org/licenses/LICENSE-2.0 8 // 9 // Unless required by applicable law or agreed to in writing, software 10 // distributed under the License is distributed on an "AS IS" BASIS, 11 // See the License for the specific language governing permissions and 12 // limitations under the License. 13 14 package internal 15 16 import ( 17 "context" 18 "sync" 19 "time" 20 21 "github.com/pingcap/errors" 22 "github.com/pingcap/log" 23 cerror "github.com/pingcap/tiflow/pkg/errors" 24 proto "github.com/pingcap/tiflow/proto/p2p" 25 "go.uber.org/zap" 26 ) 27 28 // SendChan is a specialized channel used to implement 29 // the asynchronous interface of the MessageClient. 30 // SendChan is a MPSC channel. 31 type SendChan struct { 32 mu sync.Mutex 33 buf []*proto.MessageEntry 34 sendIdx int64 35 recvIdx int64 36 37 // hasNewMsg is a buffered channel that the sending goroutine(s) 38 // use to notify the receiver end. 39 hasNewMsg chan struct{} 40 41 // notifyChan is an unbuffered channel that 42 // SendSync writes to in order to determine whether the 43 // consumer is ready. 44 notifyChan chan struct{} 45 46 cap int64 47 } 48 49 // NewSendChan returns a new SendChan. 50 func NewSendChan(cap int64) *SendChan { 51 return &SendChan{ 52 buf: make([]*proto.MessageEntry, cap), 53 hasNewMsg: make(chan struct{}, 1), 54 notifyChan: make(chan struct{}), 55 cap: cap, 56 } 57 } 58 59 // SendSync sends a message synchronously. 60 func (c *SendChan) SendSync( 61 ctx context.Context, 62 topic string, 63 value []byte, 64 closeCh <-chan struct{}, 65 nextSeq func() int64, 66 ) (int64, error) { 67 ticker := time.NewTicker(10 * time.Millisecond) 68 defer ticker.Stop() 69 70 for { 71 if ok, seq := c.SendAsync(topic, value, nextSeq); ok { 72 return seq, nil 73 } 74 75 select { 76 case <-ctx.Done(): 77 return 0, errors.Trace(ctx.Err()) 78 case <-closeCh: 79 return 0, cerror.ErrPeerMessageClientClosed.GenWithStackByArgs() 80 case c.notifyChan <- struct{}{}: 81 case <-ticker.C: 82 } 83 } 84 } 85 86 // SendAsync tries to send a message. If the message is accepted, nextSeq will be called 87 // once, and the returned value will be used as the Sequence number of the message. 88 func (c *SendChan) SendAsync(topic string, value []byte, nextSeq func() int64) (ok bool, seq int64) { 89 c.mu.Lock() 90 defer c.mu.Unlock() 91 92 if c.sendIdx-c.recvIdx > c.cap { 93 log.Panic("unreachable", 94 zap.Int64("sendIdx", c.sendIdx), 95 zap.Int64("recvIndex", c.recvIdx)) 96 } 97 98 if c.sendIdx-c.recvIdx == c.cap { 99 return false, 0 100 } 101 102 seq = nextSeq() 103 c.buf[c.sendIdx%c.cap] = &proto.MessageEntry{ 104 Topic: topic, 105 Content: value, 106 Sequence: seq, 107 } 108 c.sendIdx++ 109 110 select { 111 case c.hasNewMsg <- struct{}{}: 112 default: 113 } 114 115 return true, seq 116 } 117 118 // Receive receives one message from the channel. 119 // If a message is received from `tick`, the function will return 120 // (nil, false, nil). 121 func (c *SendChan) Receive(ctx context.Context, tick <-chan time.Time) (*proto.MessageEntry, bool, error) { 122 select { 123 case <-ctx.Done(): 124 return nil, false, errors.Trace(ctx.Err()) 125 default: 126 } 127 128 for { 129 entry := c.doReceive() 130 if entry != nil { 131 select { 132 case <-c.notifyChan: 133 default: 134 } 135 return entry, true, nil 136 } 137 138 select { 139 case <-ctx.Done(): 140 return nil, false, errors.Trace(ctx.Err()) 141 case <-tick: 142 return nil, false, nil 143 case <-c.hasNewMsg: 144 } 145 } 146 } 147 148 func (c *SendChan) doReceive() *proto.MessageEntry { 149 c.mu.Lock() 150 defer c.mu.Unlock() 151 152 if c.sendIdx < c.recvIdx { 153 log.Panic("unreachable", 154 zap.Int64("sendIdx", c.sendIdx), 155 zap.Int64("recvIndex", c.recvIdx)) 156 } 157 158 if c.sendIdx == c.recvIdx { 159 return nil 160 } 161 162 var ret *proto.MessageEntry 163 ret, c.buf[c.recvIdx%c.cap] = c.buf[c.recvIdx%c.cap], nil 164 c.recvIdx++ 165 166 return ret 167 }