github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/processor/pipeline/mounter.go (about) 1 // Copyright 2021 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 pipeline 15 16 import ( 17 "context" 18 "log" 19 "sync" 20 "time" 21 22 "github.com/edwingeng/deque" 23 "github.com/pingcap/failpoint" 24 "github.com/pingcap/ticdc/cdc/model" 25 "github.com/pingcap/ticdc/pkg/notify" 26 "github.com/pingcap/ticdc/pkg/pipeline" 27 "go.uber.org/zap" 28 "golang.org/x/sync/errgroup" 29 "golang.org/x/time/rate" 30 ) 31 32 const ( 33 waitEventMountedBatchSize = 128 * 1024 // larger means less CPU overhead used by `select` but more memory overhead. 34 maxNotificationsPerSecond = 10.0 // larger means lower latency, but more CPU used by `select`. 35 ) 36 37 // mounterNode is now used to buffer unmounted events. 38 // TODO rename this node, or refactor the mounter to make it synchronous. 39 type mounterNode struct { 40 mu sync.Mutex 41 queue deque.Deque // we use Deque for better memory consumption and support for batching 42 43 wg errgroup.Group 44 cancel context.CancelFunc 45 46 // notifies new events pushed to the queue 47 notifier notify.Notifier 48 // limits the rate at which notifications are sent 49 rl *rate.Limiter 50 } 51 52 func newMounterNode() pipeline.Node { 53 return &mounterNode{ 54 queue: deque.NewDeque(), 55 rl: rate.NewLimiter(maxNotificationsPerSecond, 1 /* burst */), 56 } 57 } 58 59 func (n *mounterNode) Init(ctx pipeline.NodeContext) error { 60 stdCtx, cancel := context.WithCancel(ctx) 61 n.cancel = cancel 62 63 receiver, err := n.notifier.NewReceiver(time.Millisecond * 100) 64 if err != nil { 65 log.Panic("unexpected error", zap.Error(err)) 66 } 67 68 n.wg.Go(func() error { 69 defer receiver.Stop() 70 for { 71 select { 72 case <-stdCtx.Done(): 73 return nil 74 case <-receiver.C: 75 // handles writes to the queue 76 for { 77 n.mu.Lock() 78 msgs := n.queue.PopManyFront(waitEventMountedBatchSize) 79 n.mu.Unlock() 80 if len(msgs) == 0 { 81 break // inner loop 82 } 83 84 for _, msg := range msgs { 85 msg := msg.(pipeline.Message) 86 if msg.Tp != pipeline.MessageTypePolymorphicEvent { 87 // sends the control message directly to the next node 88 ctx.SendToNextNode(msg) 89 continue // to handling the next message 90 } 91 92 // handles PolymorphicEvents 93 event := msg.PolymorphicEvent 94 if event.RawKV.OpType != model.OpTypeResolved { 95 failpoint.Inject("MounterNodeWaitPrepare", func() {}) 96 // only RowChangedEvents need mounting 97 err := event.WaitPrepare(stdCtx) 98 if err != nil { 99 ctx.Throw(err) 100 return nil 101 } 102 } 103 104 ctx.SendToNextNode(msg) 105 } 106 } 107 } 108 } 109 }) 110 111 return nil 112 } 113 114 // Receive receives the message from the previous node 115 func (n *mounterNode) Receive(ctx pipeline.NodeContext) error { 116 msg := ctx.Message() 117 n.mu.Lock() 118 n.queue.PushBack(msg) 119 n.mu.Unlock() 120 121 if n.rl.Allow() { 122 // send notification under the rate limiter 123 n.notifier.Notify() 124 } 125 return nil 126 } 127 128 func (n *mounterNode) Destroy(ctx pipeline.NodeContext) error { 129 defer n.notifier.Close() 130 n.cancel() 131 return n.wg.Wait() 132 }