github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/pkg/sink/codec/encoder_group.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 codec 15 16 import ( 17 "context" 18 "strconv" 19 "sync/atomic" 20 "time" 21 22 "github.com/pingcap/errors" 23 "github.com/pingcap/log" 24 "github.com/pingcap/tiflow/cdc/model" 25 "github.com/pingcap/tiflow/cdc/sink/dmlsink" 26 "github.com/pingcap/tiflow/pkg/config" 27 "github.com/pingcap/tiflow/pkg/sink/codec/common" 28 "github.com/pingcap/tiflow/pkg/util" 29 "go.uber.org/zap" 30 "golang.org/x/sync/errgroup" 31 ) 32 33 const ( 34 defaultInputChanSize = 128 35 defaultMetricInterval = 15 * time.Second 36 ) 37 38 // EncoderGroup manages a group of encoders 39 type EncoderGroup interface { 40 // Run start the group 41 Run(ctx context.Context) error 42 // AddEvents add events into the group and encode them by one of the encoders in the group. 43 // Note: The caller should make sure all events should belong to the same topic and partition. 44 AddEvents(ctx context.Context, key model.TopicPartitionKey, events ...*dmlsink.RowChangeCallbackableEvent) error 45 // Output returns a channel produce futures 46 Output() <-chan *future 47 } 48 49 type encoderGroup struct { 50 changefeedID model.ChangeFeedID 51 52 builder RowEventEncoderBuilder 53 // concurrency is the number of encoder pipelines to run 54 concurrency int 55 // inputCh is the input channel for each encoder pipeline 56 inputCh []chan *future 57 index uint64 58 59 outputCh chan *future 60 61 bootstrapWorker *bootstrapWorker 62 } 63 64 // NewEncoderGroup creates a new EncoderGroup instance 65 func NewEncoderGroup( 66 cfg *config.SinkConfig, 67 builder RowEventEncoderBuilder, 68 changefeedID model.ChangeFeedID, 69 ) *encoderGroup { 70 concurrency := util.GetOrZero(cfg.EncoderConcurrency) 71 if concurrency <= 0 { 72 concurrency = config.DefaultEncoderGroupConcurrency 73 } 74 inputCh := make([]chan *future, concurrency) 75 for i := 0; i < concurrency; i++ { 76 inputCh[i] = make(chan *future, defaultInputChanSize) 77 } 78 outCh := make(chan *future, defaultInputChanSize*concurrency) 79 80 var bootstrapWorker *bootstrapWorker 81 if cfg.ShouldSendBootstrapMsg() { 82 bootstrapWorker = newBootstrapWorker( 83 changefeedID, 84 outCh, 85 builder.Build(), 86 util.GetOrZero(cfg.SendBootstrapIntervalInSec), 87 util.GetOrZero(cfg.SendBootstrapInMsgCount), 88 util.GetOrZero(cfg.SendBootstrapToAllPartition), 89 defaultMaxInactiveDuration, 90 ) 91 } 92 93 return &encoderGroup{ 94 changefeedID: changefeedID, 95 builder: builder, 96 concurrency: concurrency, 97 inputCh: inputCh, 98 index: 0, 99 outputCh: outCh, 100 bootstrapWorker: bootstrapWorker, 101 } 102 } 103 104 func (g *encoderGroup) Run(ctx context.Context) error { 105 defer func() { 106 g.cleanMetrics() 107 log.Info("encoder group exited", 108 zap.String("namespace", g.changefeedID.Namespace), 109 zap.String("changefeed", g.changefeedID.ID)) 110 }() 111 eg, ctx := errgroup.WithContext(ctx) 112 for i := 0; i < g.concurrency; i++ { 113 idx := i 114 eg.Go(func() error { 115 return g.runEncoder(ctx, idx) 116 }) 117 } 118 119 if g.bootstrapWorker != nil { 120 eg.Go(func() error { 121 return g.bootstrapWorker.run(ctx) 122 }) 123 } 124 125 return eg.Wait() 126 } 127 128 func (g *encoderGroup) runEncoder(ctx context.Context, idx int) error { 129 encoder := g.builder.Build() 130 inputCh := g.inputCh[idx] 131 metric := encoderGroupInputChanSizeGauge. 132 WithLabelValues(g.changefeedID.Namespace, g.changefeedID.ID, strconv.Itoa(idx)) 133 ticker := time.NewTicker(defaultMetricInterval) 134 defer ticker.Stop() 135 for { 136 select { 137 case <-ctx.Done(): 138 return nil 139 case <-ticker.C: 140 metric.Set(float64(len(inputCh))) 141 case future := <-inputCh: 142 for _, event := range future.events { 143 err := encoder.AppendRowChangedEvent(ctx, future.Key.Topic, event.Event, event.Callback) 144 if err != nil { 145 return errors.Trace(err) 146 } 147 } 148 future.Messages = encoder.Build() 149 close(future.done) 150 } 151 } 152 } 153 154 func (g *encoderGroup) AddEvents( 155 ctx context.Context, 156 key model.TopicPartitionKey, 157 events ...*dmlsink.RowChangeCallbackableEvent, 158 ) error { 159 // bootstrapWorker only not nil when the protocol is simple 160 if g.bootstrapWorker != nil { 161 err := g.bootstrapWorker.addEvent(ctx, key, events[0].Event) 162 if err != nil { 163 return errors.Trace(err) 164 } 165 } 166 167 future := newFuture(key, events...) 168 index := atomic.AddUint64(&g.index, 1) % uint64(g.concurrency) 169 select { 170 case <-ctx.Done(): 171 return ctx.Err() 172 case g.inputCh[index] <- future: 173 } 174 175 select { 176 case <-ctx.Done(): 177 return ctx.Err() 178 case g.outputCh <- future: 179 } 180 181 return nil 182 } 183 184 func (g *encoderGroup) Output() <-chan *future { 185 return g.outputCh 186 } 187 188 func (g *encoderGroup) cleanMetrics() { 189 encoderGroupInputChanSizeGauge.DeleteLabelValues(g.changefeedID.Namespace, g.changefeedID.ID) 190 g.builder.CleanMetrics() 191 common.CleanMetrics(g.changefeedID) 192 } 193 194 // future is a wrapper of the result of encoding events 195 // It's used to notify the caller that the result is ready. 196 type future struct { 197 Key model.TopicPartitionKey 198 events []*dmlsink.RowChangeCallbackableEvent 199 Messages []*common.Message 200 done chan struct{} 201 } 202 203 func newFuture(key model.TopicPartitionKey, 204 events ...*dmlsink.RowChangeCallbackableEvent, 205 ) *future { 206 return &future{ 207 Key: key, 208 events: events, 209 done: make(chan struct{}), 210 } 211 } 212 213 // Ready waits until the response is ready, should be called before consuming the future. 214 func (p *future) Ready(ctx context.Context) error { 215 select { 216 case <-ctx.Done(): 217 return ctx.Err() 218 case <-p.done: 219 } 220 return nil 221 }