github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/pkg/sink/codec/bootstraper.go (about) 1 // Copyright 2023 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 "sync" 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/pkg/sink/codec/common" 26 "go.uber.org/zap" 27 ) 28 29 const ( 30 bootstrapWorkerTickerInterval = 5 * time.Second 31 bootstrapWorkerGCInterval = 30 * time.Second 32 33 defaultMaxInactiveDuration = 30 * time.Minute 34 ) 35 36 // bootstrapWorker is used to send bootstrap message to the MQ sink worker. 37 // It will be only used in simple protocol. 38 type bootstrapWorker struct { 39 changefeedID model.ChangeFeedID 40 activeTables sync.Map 41 encoder RowEventEncoder 42 sendBootstrapInterval time.Duration 43 sendBootstrapInMsgCount int32 44 sendBootstrapToAllPartition bool 45 // maxInactiveDuration is the max duration that a table can be inactive 46 maxInactiveDuration time.Duration 47 outCh chan<- *future 48 } 49 50 // newBootstrapWorker creates a new bootstrapWorker instance 51 func newBootstrapWorker( 52 changefeedID model.ChangeFeedID, 53 outCh chan<- *future, 54 encoder RowEventEncoder, 55 sendBootstrapInterval int64, 56 sendBootstrapInMsgCount int32, 57 sendBootstrapToAllPartition bool, 58 maxInactiveDuration time.Duration, 59 ) *bootstrapWorker { 60 log.Info("Sending bootstrap event is enabled for simple protocol. "+ 61 "Both send-bootstrap-interval-in-sec and send-bootstrap-in-msg-count are > 0.", 62 zap.Stringer("changefeed", changefeedID), 63 zap.Int64("sendBootstrapIntervalInSec", sendBootstrapInterval), 64 zap.Int32("sendBootstrapInMsgCount", sendBootstrapInMsgCount)) 65 return &bootstrapWorker{ 66 changefeedID: changefeedID, 67 outCh: outCh, 68 encoder: encoder, 69 activeTables: sync.Map{}, 70 sendBootstrapInterval: time.Duration(sendBootstrapInterval) * time.Second, 71 sendBootstrapInMsgCount: sendBootstrapInMsgCount, 72 sendBootstrapToAllPartition: sendBootstrapToAllPartition, 73 maxInactiveDuration: maxInactiveDuration, 74 } 75 } 76 77 func (b *bootstrapWorker) run(ctx context.Context) error { 78 sendTicker := time.NewTicker(bootstrapWorkerTickerInterval) 79 gcTicker := time.NewTicker(bootstrapWorkerGCInterval) 80 defer func() { 81 gcTicker.Stop() 82 sendTicker.Stop() 83 }() 84 85 var err error 86 for { 87 select { 88 case <-ctx.Done(): 89 return ctx.Err() 90 case <-sendTicker.C: 91 b.activeTables.Range(func(key, value interface{}) bool { 92 table := value.(*tableStatistic) 93 err = b.sendBootstrapMsg(ctx, table) 94 return err == nil 95 }) 96 if err != nil { 97 return errors.Trace(err) 98 } 99 case <-gcTicker.C: 100 b.gcInactiveTables() 101 } 102 } 103 } 104 105 func (b *bootstrapWorker) addEvent( 106 ctx context.Context, 107 key model.TopicPartitionKey, 108 row *model.RowChangedEvent, 109 ) error { 110 table, ok := b.activeTables.Load(row.PhysicalTableID) 111 if !ok { 112 tb := newTableStatistic(key, row) 113 b.activeTables.Store(tb.id, tb) 114 // Send bootstrap message immediately when a new table is added 115 err := b.sendBootstrapMsg(ctx, tb) 116 if err != nil { 117 return errors.Trace(err) 118 } 119 } else { 120 // If the table is already in the activeTables, update its status. 121 table.(*tableStatistic).update(row, key.TotalPartition) 122 } 123 return nil 124 } 125 126 // sendBootstrapMsg sends a bootstrap message if the table meets the condition 127 // 1. The time since last bootstrap message sent is larger than sendBootstrapInterval 128 // 2. The received row event count since last bootstrap message sent is larger than sendBootstrapInMsgCount 129 // Note: It is a blocking method, it will block if the outCh is full. 130 func (b *bootstrapWorker) sendBootstrapMsg(ctx context.Context, table *tableStatistic) error { 131 if !table.shouldSendBootstrapMsg( 132 b.sendBootstrapInterval, 133 b.sendBootstrapInMsgCount) { 134 return nil 135 } 136 table.reset() 137 tableInfo := table.tableInfo.Load().(*model.TableInfo) 138 events, err := b.generateEvents(table.topic, table.totalPartition.Load(), tableInfo) 139 if err != nil { 140 return errors.Trace(err) 141 } 142 for _, e := range events { 143 select { 144 case <-ctx.Done(): 145 return ctx.Err() 146 case b.outCh <- e: 147 } 148 } 149 return nil 150 } 151 152 func (b *bootstrapWorker) generateEvents( 153 topic string, 154 totalPartition int32, 155 tableInfo *model.TableInfo, 156 ) ([]*future, error) { 157 res := make([]*future, 0, totalPartition) 158 msg, err := b.encoder.EncodeDDLEvent(model.NewBootstrapDDLEvent(tableInfo)) 159 if err != nil { 160 return nil, errors.Trace(err) 161 } 162 // If sendBootstrapToAllPartition is true, send bootstrap message to all partition 163 // Otherwise, send bootstrap message to partition 0. 164 if !b.sendBootstrapToAllPartition { 165 totalPartition = 1 166 } 167 for i := int32(0); i < totalPartition; i++ { 168 f := &future{ 169 Key: model.TopicPartitionKey{ 170 Topic: topic, 171 Partition: i, 172 }, 173 done: make(chan struct{}), 174 Messages: []*common.Message{msg}, 175 } 176 close(f.done) 177 res = append(res, f) 178 } 179 return res, nil 180 } 181 182 func (b *bootstrapWorker) gcInactiveTables() { 183 b.activeTables.Range(func(key, value interface{}) bool { 184 table := value.(*tableStatistic) 185 if table.isInactive(b.maxInactiveDuration) { 186 log.Info("A table is removed from the bootstrap worker", 187 zap.Int64("tableID", table.id), 188 zap.String("topic", table.topic), 189 zap.Stringer("changefeed", b.changefeedID)) 190 b.activeTables.Delete(key) 191 } 192 return true 193 }) 194 } 195 196 // tableStatistic is used to record the statistics of a table 197 type tableStatistic struct { 198 // id is the table's ID, it will not change 199 id int64 200 // topic is the table's topic, it will not change 201 topic string 202 // totalPartition is the partition number of the corresponding topic 203 totalPartition atomic.Int32 204 // counter is the number of row event sent since last bootstrap message sent 205 // It is used to check if the bootstrap message should be sent 206 counter atomic.Int32 207 // lastMsgReceivedTime is the last time the row event is received 208 // It is used to check if the table is inactive 209 lastMsgReceivedTime atomic.Value 210 // lastSendTime is the last time the bootstrap message is sent 211 // It is used to check if the bootstrap message should be sent 212 lastSendTime atomic.Value 213 // version is the table version 214 // It is used to check if the table schema is changed since last bootstrap message sent 215 version atomic.Uint64 216 // tableInfo is the tableInfo of the table 217 // It is used to generate bootstrap message 218 tableInfo atomic.Value 219 } 220 221 func newTableStatistic(key model.TopicPartitionKey, row *model.RowChangedEvent) *tableStatistic { 222 res := &tableStatistic{ 223 id: row.PhysicalTableID, 224 topic: key.Topic, 225 } 226 res.totalPartition.Store(key.TotalPartition) 227 res.counter.Add(1) 228 res.lastMsgReceivedTime.Store(time.Now()) 229 res.lastSendTime.Store(time.Unix(0, 0)) 230 res.version.Store(row.TableInfo.UpdateTS) 231 res.tableInfo.Store(row.TableInfo) 232 return res 233 } 234 235 func (t *tableStatistic) shouldSendBootstrapMsg( 236 sendBootstrapInterval time.Duration, 237 sendBootstrapMsgCountInterval int32, 238 ) bool { 239 lastSendTime := t.lastSendTime.Load().(time.Time) 240 return time.Since(lastSendTime) >= sendBootstrapInterval || 241 t.counter.Load() >= sendBootstrapMsgCountInterval 242 } 243 244 func (t *tableStatistic) update(row *model.RowChangedEvent, totalPartition int32) { 245 t.counter.Add(1) 246 t.lastMsgReceivedTime.Store(time.Now()) 247 248 // Note(dongmen): Rename Table DDL is a special case, 249 // the TableInfo.Name is changed but the TableInfo.UpdateTs is not changed. 250 if t.version.Load() != row.TableInfo.UpdateTS || 251 t.tableInfo.Load().(*model.TableInfo).Name != row.TableInfo.Name { 252 t.version.Store(row.TableInfo.UpdateTS) 253 t.tableInfo.Store(row.TableInfo) 254 } 255 if t.totalPartition.Load() != totalPartition { 256 t.totalPartition.Store(totalPartition) 257 } 258 } 259 260 func (t *tableStatistic) isInactive(maxInactiveDuration time.Duration) bool { 261 lastMsgReceivedTime := t.lastMsgReceivedTime.Load().(time.Time) 262 return time.Since(lastMsgReceivedTime) > maxInactiveDuration 263 } 264 265 func (t *tableStatistic) reset() { 266 t.lastSendTime.Store(time.Now()) 267 t.counter.Store(0) 268 }